From 6d4d50da7b46155d4ffdb58d4b61eb1742a98932 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 14 Mar 2022 12:16:41 +0100 Subject: [PATCH 001/258] [FLINK-26500][runtime][test] Increases the deadline to wait for parallelism The deadline of 10s wasn't sufficient enough for AzureCI runs. --- .../scheduler/adaptive/AdaptiveSchedulerClusterITCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java index 272b022383f57..6973722ccc2a3 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java @@ -282,6 +282,6 @@ private void waitUntilParallelismForVertexReached( return executionJobVertex.getParallelism() == targetParallelism; }, - Deadline.fromNow(Duration.ofSeconds(10))); + Deadline.fromNow(Duration.ofMinutes(5))); } } From 5feff4a46f7c4c9d395347e42b9d68b8fe00c8a5 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 1 Mar 2022 14:57:13 +0200 Subject: [PATCH 002/258] [FLINK-26421][table] Remove planner & executor string identifiers Remove planner&executor string identifiers from `EnvironmentSettings` and use the default strings which are only used anyway. --- .../connectors/hive/HiveDialectITCase.java | 8 +++- .../table/catalog/hive/HiveCatalogITCase.java | 13 +++++-- .../gateway/context/ExecutionContext.java | 14 ++----- .../AbstractStreamTableEnvironmentImpl.java | 6 +-- .../internal/StreamTableEnvironmentImpl.java | 10 +---- .../flink/table/api/EnvironmentSettings.java | 37 +------------------ .../api/internal/TableEnvironmentImpl.java | 9 +---- .../table/factories/PlannerFactoryUtil.java | 3 +- .../internal/StreamTableEnvironmentImpl.scala | 6 +-- .../table/planner/utils/TableTestBase.scala | 11 +++--- 10 files changed, 37 insertions(+), 80 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java index 30537c20369f2..24c6904550fab 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.internal.TableEnvironmentInternal; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogPartitionSpec; @@ -75,8 +76,6 @@ import java.util.LinkedHashMap; import java.util.List; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; import static org.hamcrest.CoreMatchers.instanceOf; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; @@ -90,6 +89,11 @@ /** Test Hive syntax when Hive dialect is used. */ public class HiveDialectITCase { + private static final String DEFAULT_BUILTIN_CATALOG = + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(); + private static final String DEFAULT_BUILTIN_DATABASE = + TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(); + private TableEnvironment tableEnv; private HiveCatalog hiveCatalog; private String warehouse; diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java index fe06b2bdd5830..ad84edeef179f 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java @@ -27,6 +27,7 @@ import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.constraints.UniqueConstraint; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogBaseTable; @@ -86,6 +87,11 @@ */ public class HiveCatalogITCase { + private static final String DEFAULT_BUILTIN_CATALOG = + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(); + private static final String DEFAULT_BUILTIN_DATABASE = + TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(); + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); private static HiveCatalog hiveCatalog; @@ -472,15 +478,14 @@ public void testCreateTableLike() throws Exception { tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); tableEnv.executeSql("create table generic_table (x int) with ('connector'='COLLECTION')"); - tableEnv.useCatalog(EnvironmentSettings.DEFAULT_BUILTIN_CATALOG); + tableEnv.useCatalog(DEFAULT_BUILTIN_CATALOG); tableEnv.executeSql( String.format( "create table copy like `%s`.`default`.generic_table", hiveCatalog.getName())); - Catalog builtInCat = tableEnv.getCatalog(EnvironmentSettings.DEFAULT_BUILTIN_CATALOG).get(); + Catalog builtInCat = tableEnv.getCatalog(DEFAULT_BUILTIN_CATALOG).get(); CatalogBaseTable catalogTable = - builtInCat.getTable( - new ObjectPath(EnvironmentSettings.DEFAULT_BUILTIN_DATABASE, "copy")); + builtInCat.getTable(new ObjectPath(DEFAULT_BUILTIN_DATABASE, "copy")); assertThat(catalogTable.getOptions()).hasSize(1); assertThat(catalogTable.getOptions()) .containsEntry(FactoryUtil.CONNECTOR.key(), "COLLECTION"); diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java index 135cf8f26f29b..ae1a829a9b639 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java @@ -106,7 +106,7 @@ private StreamTableEnvironment createTableEnvironment() { StreamExecutionEnvironment streamExecEnv = createStreamExecutionEnvironment(); - final Executor executor = lookupExecutor(settings.getExecutor(), streamExecEnv); + final Executor executor = lookupExecutor(streamExecEnv); return createStreamTableEnvironment( streamExecEnv, settings, @@ -130,12 +130,7 @@ private StreamTableEnvironment createStreamTableEnvironment( final Planner planner = PlannerFactoryUtil.createPlanner( - settings.getPlanner(), - executor, - tableConfig, - moduleManager, - catalogManager, - functionCatalog); + executor, tableConfig, moduleManager, catalogManager, functionCatalog); return new StreamTableEnvironmentImpl( catalogManager, @@ -149,12 +144,11 @@ private StreamTableEnvironment createStreamTableEnvironment( userClassLoader); } - private Executor lookupExecutor( - String executorIdentifier, StreamExecutionEnvironment executionEnvironment) { + private Executor lookupExecutor(StreamExecutionEnvironment executionEnvironment) { try { final ExecutorFactory executorFactory = FactoryUtil.discoverFactory( - classLoader, ExecutorFactory.class, executorIdentifier); + classLoader, ExecutorFactory.class, ExecutorFactory.DEFAULT_IDENTIFIER); final Method createMethod = executorFactory .getClass() diff --git a/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java b/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java index c9fb822c66d4c..c8df9500c9d13 100644 --- a/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java +++ b/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java @@ -98,14 +98,12 @@ public AbstractStreamTableEnvironmentImpl( } public static Executor lookupExecutor( - ClassLoader classLoader, - String executorIdentifier, - StreamExecutionEnvironment executionEnvironment) { + ClassLoader classLoader, StreamExecutionEnvironment executionEnvironment) { final ExecutorFactory executorFactory; try { executorFactory = FactoryUtil.discoverFactory( - classLoader, ExecutorFactory.class, executorIdentifier); + classLoader, ExecutorFactory.class, ExecutorFactory.DEFAULT_IDENTIFIER); } catch (Exception e) { throw new TableException( "Could not instantiate the executor. Make sure a planner module is on the classpath", diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java index c6c7d43e222aa..5d66cfaeb70ca 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java @@ -116,17 +116,11 @@ public static StreamTableEnvironment create( final FunctionCatalog functionCatalog = new FunctionCatalog(tableConfig, catalogManager, moduleManager); - final Executor executor = - lookupExecutor(classLoader, settings.getExecutor(), executionEnvironment); + final Executor executor = lookupExecutor(classLoader, executionEnvironment); final Planner planner = PlannerFactoryUtil.createPlanner( - settings.getPlanner(), - executor, - tableConfig, - moduleManager, - catalogManager, - functionCatalog); + executor, tableConfig, moduleManager, catalogManager, functionCatalog); return new StreamTableEnvironmentImpl( catalogManager, diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java index cc9a32b703be5..d8a246408c159 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java @@ -18,18 +18,11 @@ package org.apache.flink.table.api; -import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.delegation.Executor; -import org.apache.flink.table.delegation.ExecutorFactory; -import org.apache.flink.table.delegation.Planner; -import org.apache.flink.table.delegation.PlannerFactory; import org.apache.flink.table.functions.UserDefinedFunction; -import javax.annotation.Nullable; - import static org.apache.flink.api.common.RuntimeExecutionMode.BATCH; import static org.apache.flink.api.common.RuntimeExecutionMode.STREAMING; import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; @@ -63,12 +56,6 @@ public class EnvironmentSettings { public static final String DEFAULT_BUILTIN_CATALOG = "default_catalog"; public static final String DEFAULT_BUILTIN_DATABASE = "default_database"; - /** Factory identifier of the {@link Planner} to use. */ - private final String planner; - - /** Factory identifier of the {@link Executor} to use. */ - private final String executor; - /** * Specifies the name of the initial catalog to be created when instantiating {@link * TableEnvironment}. @@ -88,13 +75,7 @@ public class EnvironmentSettings { private final boolean isStreamingMode; private EnvironmentSettings( - String planner, - @Nullable String executor, - String builtInCatalogName, - String builtInDatabaseName, - boolean isStreamingMode) { - this.planner = planner; - this.executor = executor; + String builtInCatalogName, String builtInDatabaseName, boolean isStreamingMode) { this.builtInCatalogName = builtInCatalogName; this.builtInDatabaseName = builtInDatabaseName; this.isStreamingMode = isStreamingMode; @@ -180,23 +161,9 @@ public boolean isStreamingMode() { return isStreamingMode; } - /** Returns the identifier of the {@link Planner} to be used. */ - @Internal - public String getPlanner() { - return planner; - } - - /** Returns the {@link Executor} that should submit and execute table programs. */ - @Internal - public String getExecutor() { - return executor; - } - /** A builder for {@link EnvironmentSettings}. */ @PublicEvolving public static class Builder { - private final String planner = PlannerFactory.DEFAULT_IDENTIFIER; - private final String executor = ExecutorFactory.DEFAULT_IDENTIFIER; private String builtInCatalogName = DEFAULT_BUILTIN_CATALOG; private String builtInDatabaseName = DEFAULT_BUILTIN_DATABASE; @@ -255,7 +222,7 @@ public Builder withBuiltInDatabaseName(String builtInDatabaseName) { /** Returns an immutable instance of {@link EnvironmentSettings}. */ public EnvironmentSettings build() { return new EnvironmentSettings( - planner, executor, builtInCatalogName, builtInDatabaseName, isStreamingMode); + builtInCatalogName, builtInDatabaseName, isStreamingMode); } } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java index ea17a9d5e0960..8d6a5a96a4bf5 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java @@ -299,17 +299,12 @@ private static TableEnvironmentImpl create( final ExecutorFactory executorFactory = FactoryUtil.discoverFactory( - classLoader, ExecutorFactory.class, settings.getExecutor()); + classLoader, ExecutorFactory.class, ExecutorFactory.DEFAULT_IDENTIFIER); final Executor executor = executorFactory.create(configuration); final Planner planner = PlannerFactoryUtil.createPlanner( - settings.getPlanner(), - executor, - tableConfig, - moduleManager, - catalogManager, - functionCatalog); + executor, tableConfig, moduleManager, catalogManager, functionCatalog); return new TableEnvironmentImpl( catalogManager, diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/PlannerFactoryUtil.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/PlannerFactoryUtil.java index ef64f32bdb984..674c916fc486a 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/PlannerFactoryUtil.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/factories/PlannerFactoryUtil.java @@ -35,7 +35,6 @@ public class PlannerFactoryUtil { /** Discovers a planner factory and creates a planner instance. */ public static Planner createPlanner( - String plannerIdentifier, Executor executor, TableConfig tableConfig, ModuleManager moduleManager, @@ -45,7 +44,7 @@ public static Planner createPlanner( FactoryUtil.discoverFactory( Thread.currentThread().getContextClassLoader(), PlannerFactory.class, - plannerIdentifier); + PlannerFactory.DEFAULT_IDENTIFIER); final Context context = new DefaultPlannerContext( diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala index f56ba48f10278..a968e49c22ea3 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala @@ -317,10 +317,10 @@ object StreamTableEnvironmentImpl { val functionCatalog = new FunctionCatalog(tableConfig, catalogManager, moduleManager) val executor = AbstractStreamTableEnvironmentImpl.lookupExecutor( - classLoader, settings.getExecutor, executionEnvironment.getWrappedStreamExecutionEnvironment) + classLoader, executionEnvironment.getWrappedStreamExecutionEnvironment) - val planner = PlannerFactoryUtil.createPlanner(settings.getPlanner, executor, tableConfig, - moduleManager, catalogManager, functionCatalog) + val planner = PlannerFactoryUtil.createPlanner( + executor, tableConfig, moduleManager, catalogManager, functionCatalog) new StreamTableEnvironmentImpl( catalogManager, diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index 6cb12cce545f8..0d088d6defbaf 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -26,13 +26,13 @@ import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation} import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo} import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo import org.apache.flink.configuration.ExecutionOptions - import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.{JsonNode, ObjectMapper} import org.apache.flink.streaming.api.datastream.DataStream import org.apache.flink.streaming.api.environment.{LocalStreamEnvironment, StreamExecutionEnvironment} import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => ScalaStreamExecEnv} import org.apache.flink.streaming.api.{TimeCharacteristic, environment} import org.apache.flink.table.api._ +import org.apache.flink.table.api.bridge.internal.AbstractStreamTableEnvironmentImpl import org.apache.flink.table.api.bridge.java.{StreamTableEnvironment => JavaStreamTableEnv} import org.apache.flink.table.api.bridge.scala.{StreamTableEnvironment => ScalaStreamTableEnv} import org.apache.flink.table.api.config.ExecutionConfigOptions @@ -1549,12 +1549,13 @@ object TestingTableEnvironment { val functionCatalog = new FunctionCatalog(tableConfig, catalogMgr, moduleManager) - val executorFactory = - FactoryUtil.discoverFactory(classLoader, classOf[ExecutorFactory], settings.getExecutor) + val executorFactory = FactoryUtil.discoverFactory( + classLoader, classOf[ExecutorFactory], ExecutorFactory.DEFAULT_IDENTIFIER) + val executor = executorFactory.create(tableConfig.getConfiguration) - val planner = PlannerFactoryUtil.createPlanner(settings.getPlanner, executor, tableConfig, - moduleManager, catalogMgr, functionCatalog).asInstanceOf[PlannerBase] + val planner = PlannerFactoryUtil.createPlanner( + executor, tableConfig, moduleManager, catalogMgr, functionCatalog).asInstanceOf[PlannerBase] new TestingTableEnvironment( catalogMgr, From f39d5fc0c02058810d86b7bd10927669505cb601 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Wed, 2 Mar 2022 11:36:20 +0200 Subject: [PATCH 003/258] [hotfix][table-api] Remove deprecated create methods Those 2 methods accept also a TableConfig, which is not necessary, since with the new approach, extra configuration will be passed through `EnvironmentSettings` instead. If those methods are still around will create confusion. --- .../bridge/java/StreamTableEnvironment.java | 34 ------------------- .../bridge/scala/StreamTableEnvironment.scala | 34 ------------------- 2 files changed, 68 deletions(-) diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java index 6481acff4504a..09e651981be54 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java @@ -127,40 +127,6 @@ static StreamTableEnvironment create( return StreamTableEnvironmentImpl.create(executionEnvironment, settings, tableConfig); } - /** - * Creates a table environment that is the entry point and central context for creating Table - * and SQL API programs that integrate with the Java-specific {@link DataStream} API. - * - *

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

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

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

Note: If you don't intend to use the {@link DataStream} API, {@link TableEnvironment} is - * meant for pure table programs. - * - * @param executionEnvironment The Java {@link StreamExecutionEnvironment} of the {@link - * TableEnvironment}. - * @param tableConfig The configuration of the {@link TableEnvironment}. - * @deprecated Use {@link #create(StreamExecutionEnvironment)} and {@link #getConfig()} for - * manipulating {@link TableConfig}. - */ - @Deprecated - static StreamTableEnvironment create( - StreamExecutionEnvironment executionEnvironment, TableConfig tableConfig) { - return StreamTableEnvironmentImpl.create( - executionEnvironment, - EnvironmentSettings.fromConfiguration(tableConfig.getConfiguration()), - tableConfig); - } - /** * Registers a {@link TableFunction} under a unique name in the TableEnvironment's catalog. * Registered functions can be referenced in Table API and SQL queries. diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala index 49d0b7a556f3a..e8463eeb2d806 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala @@ -904,38 +904,4 @@ object StreamTableEnvironment { StreamTableEnvironmentImpl .create(executionEnvironment, settings, config) } - - /** - * Creates a table environment that is the entry point and central context for creating Table and - * SQL API programs that integrate with the Scala-specific [[DataStream]] API. - * - * It is unified for bounded and unbounded data processing. - * - * A stream table environment is responsible for: - * - * - Convert a [[DataStream]] into [[Table]] and vice-versa. - * - Connecting to external systems. - * - Registering and retrieving [[Table]]s and other meta objects from a catalog. - * - Executing SQL statements. - * - Offering further configuration options. - * - * Note: If you don't intend to use the [[DataStream]] API, [[TableEnvironment]] is meant for - * pure table programs. - * - * @param executionEnvironment The Scala [[StreamExecutionEnvironment]] of the - * [[TableEnvironment]]. - * @param tableConfig The configuration of the [[TableEnvironment]]. - * @deprecated Use [[create(StreamExecutionEnvironment)]] and - * [[StreamTableEnvironment#getConfig()]] for manipulating the [[TableConfig]]. - */ - @deprecated - def create(executionEnvironment: StreamExecutionEnvironment, tableConfig: TableConfig) - : StreamTableEnvironment = { - - StreamTableEnvironmentImpl - .create( - executionEnvironment, - EnvironmentSettings.fromConfiguration(tableConfig.getConfiguration), - tableConfig) - } } From 567b28e26b65601759c51908c10fe362e48e1a89 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Wed, 9 Mar 2022 11:10:48 +0200 Subject: [PATCH 004/258] [FLINK-26421] Use only EnvironmentSettings to configure the environment Use `EnvironmentSettings` with the new method `withConfiguration` in its `Builder` to specify configuration options on top of the one inherited by the environment (flink-conf.yml, CLI params). The `TableConfig` contains all of the config options specified by the env (flink-conf.yml, CLI params) + all the extra configuration defined by the user app through the `EnvironmentSettings`. --- .../generated/table_config_configuration.html | 12 ++ .../connectors/hive/HiveDialectITCase.java | 16 +-- .../table/catalog/hive/HiveCatalogITCase.java | 13 +-- flink-python/pyflink/java_gateway.py | 1 + .../pyflink/table/environment_settings.py | 23 ++++ .../table/tests/test_environment_settings.py | 4 +- .../tests/test_table_config_completeness.py | 3 +- .../gateway/context/ExecutionContext.java | 28 ++--- .../gateway/context/SessionContext.java | 3 +- .../bridge/java/StreamTableEnvironment.java | 9 +- .../internal/StreamTableEnvironmentImpl.java | 14 ++- .../flink/table/api/EnvironmentSettings.java | 104 ++++++++---------- .../apache/flink/table/api/TableConfig.java | 50 +++++++-- .../table/api/config/TableConfigOptions.java | 18 +++ .../api/internal/TableEnvironmentImpl.java | 20 ++-- .../table/api/EnvironmentSettingsTest.java | 7 +- .../table/catalog/FunctionCatalogTest.java | 4 +- .../table/utils/CatalogManagerMocks.java | 8 +- .../table/utils/TableEnvironmentMock.java | 14 +-- .../bridge/scala/StreamTableEnvironment.scala | 10 +- .../internal/StreamTableEnvironmentImpl.scala | 16 ++- .../delegation/DefaultPlannerFactory.java | 9 +- .../flink/table/api/EnvironmentTest.java | 28 +++++ .../plan/nodes/exec/TransformationsTest.java | 2 +- .../serde/DynamicTableSinkSpecSerdeTest.java | 20 ++-- .../DynamicTableSourceSpecSerdeTest.java | 16 +-- .../utils/ExpressionTestBase.scala | 16 ++- .../harness/GroupAggregateHarnessTest.scala | 3 +- .../runtime/harness/HarnessTestBase.scala | 21 +--- .../harness/OverAggregateHarnessTest.scala | 18 +-- .../runtime/harness/RankHarnessTest.scala | 3 +- .../harness/TableAggregateHarnessTest.scala | 3 +- .../sql/StreamTableEnvironmentITCase.scala | 25 +++++ .../table/planner/utils/TableTestBase.scala | 20 ++-- 34 files changed, 331 insertions(+), 230 deletions(-) diff --git a/docs/layouts/shortcodes/generated/table_config_configuration.html b/docs/layouts/shortcodes/generated/table_config_configuration.html index 53e5dbcfc2bc9..fe9784a220722 100644 --- a/docs/layouts/shortcodes/generated/table_config_configuration.html +++ b/docs/layouts/shortcodes/generated/table_config_configuration.html @@ -8,6 +8,18 @@ + +

table.builtin-catalog-name

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

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

Batch Streaming false diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java index 24c6904550fab..1aa6f506963cb 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectITCase.java @@ -25,7 +25,6 @@ import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.internal.TableEnvironmentInternal; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogPartitionSpec; @@ -43,6 +42,7 @@ import org.apache.flink.table.operations.command.ResetOperation; import org.apache.flink.table.operations.command.SetOperation; import org.apache.flink.table.planner.delegation.hive.HiveParser; +import org.apache.flink.table.utils.CatalogManagerMocks; import org.apache.flink.types.Row; import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.FileUtils; @@ -89,11 +89,6 @@ /** Test Hive syntax when Hive dialect is used. */ public class HiveDialectITCase { - private static final String DEFAULT_BUILTIN_CATALOG = - TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(); - private static final String DEFAULT_BUILTIN_DATABASE = - TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(); - private TableEnvironment tableEnv; private HiveCatalog hiveCatalog; private String warehouse; @@ -674,17 +669,18 @@ public void testCatalog() { List catalogs = CollectionUtil.iteratorToList(tableEnv.executeSql("show catalogs").collect()); assertEquals(2, catalogs.size()); - tableEnv.executeSql("use catalog " + DEFAULT_BUILTIN_CATALOG); + tableEnv.executeSql("use catalog " + CatalogManagerMocks.DEFAULT_CATALOG); List databases = CollectionUtil.iteratorToList(tableEnv.executeSql("show databases").collect()); assertEquals(1, databases.size()); - assertEquals("+I[" + DEFAULT_BUILTIN_DATABASE + "]", databases.get(0).toString()); + assertEquals( + "+I[" + CatalogManagerMocks.DEFAULT_DATABASE + "]", databases.get(0).toString()); String catalogName = tableEnv.executeSql("show current catalog").collect().next().toString(); - assertEquals("+I[" + DEFAULT_BUILTIN_CATALOG + "]", catalogName); + assertEquals("+I[" + CatalogManagerMocks.DEFAULT_CATALOG + "]", catalogName); String databaseName = tableEnv.executeSql("show current database").collect().next().toString(); - assertEquals("+I[" + DEFAULT_BUILTIN_DATABASE + "]", databaseName); + assertEquals("+I[" + CatalogManagerMocks.DEFAULT_DATABASE + "]", databaseName); } @Test diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java index ad84edeef179f..b18e11de92d1f 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java @@ -27,7 +27,6 @@ import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.constraints.UniqueConstraint; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogBaseTable; @@ -42,6 +41,7 @@ import org.apache.flink.table.factories.TestManagedTableFactory; import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory; import org.apache.flink.table.types.AbstractDataType; +import org.apache.flink.table.utils.CatalogManagerMocks; import org.apache.flink.types.Row; import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.FileUtils; @@ -87,11 +87,6 @@ */ public class HiveCatalogITCase { - private static final String DEFAULT_BUILTIN_CATALOG = - TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(); - private static final String DEFAULT_BUILTIN_DATABASE = - TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(); - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); private static HiveCatalog hiveCatalog; @@ -478,14 +473,14 @@ public void testCreateTableLike() throws Exception { tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog); tableEnv.useCatalog(hiveCatalog.getName()); tableEnv.executeSql("create table generic_table (x int) with ('connector'='COLLECTION')"); - tableEnv.useCatalog(DEFAULT_BUILTIN_CATALOG); + tableEnv.useCatalog(CatalogManagerMocks.DEFAULT_CATALOG); tableEnv.executeSql( String.format( "create table copy like `%s`.`default`.generic_table", hiveCatalog.getName())); - Catalog builtInCat = tableEnv.getCatalog(DEFAULT_BUILTIN_CATALOG).get(); + Catalog builtInCat = tableEnv.getCatalog(CatalogManagerMocks.DEFAULT_CATALOG).get(); CatalogBaseTable catalogTable = - builtInCat.getTable(new ObjectPath(DEFAULT_BUILTIN_DATABASE, "copy")); + builtInCat.getTable(new ObjectPath(CatalogManagerMocks.DEFAULT_DATABASE, "copy")); assertThat(catalogTable.getOptions()).hasSize(1); assertThat(catalogTable.getOptions()) .containsEntry(FactoryUtil.CONNECTOR.key(), "COLLECTION"); diff --git a/flink-python/pyflink/java_gateway.py b/flink-python/pyflink/java_gateway.py index b1272407ac481..fdb47a8c754ec 100644 --- a/flink-python/pyflink/java_gateway.py +++ b/flink-python/pyflink/java_gateway.py @@ -132,6 +132,7 @@ def import_flink_view(gateway): """ # Import the classes used by PyFlink java_import(gateway.jvm, "org.apache.flink.table.api.*") + java_import(gateway.jvm, "org.apache.flink.table.api.config.*") java_import(gateway.jvm, "org.apache.flink.table.api.java.*") java_import(gateway.jvm, "org.apache.flink.table.api.bridge.java.*") java_import(gateway.jvm, "org.apache.flink.table.api.dataview.*") diff --git a/flink-python/pyflink/table/environment_settings.py b/flink-python/pyflink/table/environment_settings.py index 578567d7094cf..6819c4bd68532 100644 --- a/flink-python/pyflink/table/environment_settings.py +++ b/flink-python/pyflink/table/environment_settings.py @@ -50,6 +50,15 @@ def __init__(self): gateway = get_gateway() self._j_builder = gateway.jvm.EnvironmentSettings.Builder() + def with_configuration(self, config: Configuration) -> 'EnvironmentSettings.Builder': + """ + Creates the EnvironmentSetting with specified Configuration. + + :return: EnvironmentSettings. + """ + self._j_builder = self._j_builder.withConfiguration(config._j_configuration) + return self + def in_batch_mode(self) -> 'EnvironmentSettings.Builder': """ Sets that the components should work in a batch mode. Streaming mode by default. @@ -155,9 +164,20 @@ def to_configuration(self) -> Configuration: Convert to `pyflink.common.Configuration`. :return: Configuration with specified value. + + .. note:: Deprecated in 1.15. Please use + :func:`EnvironmentSettings.get_configuration` instead. """ return Configuration(j_configuration=self._j_environment_settings.toConfiguration()) + def get_configuration(self) -> Configuration: + """ + Get the underlying `pyflink.common.Configuration`. + + :return: Configuration with specified value. + """ + return Configuration(j_configuration=self._j_environment_settings.getConfiguration()) + @staticmethod def new_instance() -> 'EnvironmentSettings.Builder': """ @@ -173,6 +193,9 @@ def from_configuration(config: Configuration) -> 'EnvironmentSettings': Creates the EnvironmentSetting with specified Configuration. :return: EnvironmentSettings. + + .. note:: Deprecated in 1.15. Please use + :func:`EnvironmentSettings.Builder.with_configuration` instead. """ return EnvironmentSettings( get_gateway().jvm.EnvironmentSettings.fromConfiguration(config._j_configuration)) diff --git a/flink-python/pyflink/table/tests/test_environment_settings.py b/flink-python/pyflink/table/tests/test_environment_settings.py index 8722904a68429..1a2f1ed612e92 100644 --- a/flink-python/pyflink/table/tests/test_environment_settings.py +++ b/flink-python/pyflink/table/tests/test_environment_settings.py @@ -50,7 +50,7 @@ def test_with_built_in_catalog_name(self): gateway = get_gateway() - DEFAULT_BUILTIN_CATALOG = gateway.jvm.EnvironmentSettings.DEFAULT_BUILTIN_CATALOG + DEFAULT_BUILTIN_CATALOG = gateway.jvm.TableConfigOptions.TABLE_CATALOG_NAME.defaultValue() builder = EnvironmentSettings.new_instance() @@ -67,7 +67,7 @@ def test_with_built_in_database_name(self): gateway = get_gateway() - DEFAULT_BUILTIN_DATABASE = gateway.jvm.EnvironmentSettings.DEFAULT_BUILTIN_DATABASE + DEFAULT_BUILTIN_DATABASE = gateway.jvm.TableConfigOptions.TABLE_DATABASE_NAME.defaultValue() builder = EnvironmentSettings.new_instance() diff --git a/flink-python/pyflink/table/tests/test_table_config_completeness.py b/flink-python/pyflink/table/tests/test_table_config_completeness.py index 396f5d7e5c7d5..b2dc4c6b91f0f 100644 --- a/flink-python/pyflink/table/tests/test_table_config_completeness.py +++ b/flink-python/pyflink/table/tests/test_table_config_completeness.py @@ -37,7 +37,8 @@ def java_class(cls): @classmethod def excluded_methods(cls): # internal interfaces, no need to expose to users. - return {'getPlannerConfig', 'setPlannerConfig', 'addJobParameter'} + return {'getPlannerConfig', 'setPlannerConfig', 'addJobParameter', + 'setRootConfiguration', 'get', 'getOptional'} @classmethod def java_method_name(cls, python_method_name): diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java index ae1a829a9b639..4c0f7f4ee6a9e 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/ExecutionContext.java @@ -98,19 +98,20 @@ public StreamTableEnvironment getTableEnvironment() { // ------------------------------------------------------------------------------------------------------------------ private StreamTableEnvironment createTableEnvironment() { - // checks the value of RUNTIME_MODE - EnvironmentSettings settings = EnvironmentSettings.fromConfiguration(flinkConfig); + EnvironmentSettings settings = + EnvironmentSettings.newInstance().withConfiguration(flinkConfig).build(); - TableConfig tableConfig = new TableConfig(); - tableConfig.addConfiguration(flinkConfig); - - StreamExecutionEnvironment streamExecEnv = createStreamExecutionEnvironment(); + // We need not different StreamExecutionEnvironments to build and submit flink job, + // instead we just use StreamExecutionEnvironment#executeAsync(StreamGraph) method + // to execute existing StreamGraph. + // This requires StreamExecutionEnvironment to have a full flink configuration. + StreamExecutionEnvironment streamExecEnv = + new StreamExecutionEnvironment(new Configuration(flinkConfig), classLoader); final Executor executor = lookupExecutor(streamExecEnv); return createStreamTableEnvironment( streamExecEnv, settings, - tableConfig, executor, sessionState.catalogManager, sessionState.moduleManager, @@ -121,13 +122,16 @@ private StreamTableEnvironment createTableEnvironment() { private StreamTableEnvironment createStreamTableEnvironment( StreamExecutionEnvironment env, EnvironmentSettings settings, - TableConfig tableConfig, Executor executor, CatalogManager catalogManager, ModuleManager moduleManager, FunctionCatalog functionCatalog, ClassLoader userClassLoader) { + TableConfig tableConfig = TableConfig.getDefault(); + tableConfig.setRootConfiguration(executor.getConfiguration()); + tableConfig.addConfiguration(settings.getConfiguration()); + final Planner planner = PlannerFactoryUtil.createPlanner( executor, tableConfig, moduleManager, catalogManager, functionCatalog); @@ -161,12 +165,4 @@ private Executor lookupExecutor(StreamExecutionEnvironment executionEnvironment) e); } } - - private StreamExecutionEnvironment createStreamExecutionEnvironment() { - // We need not different StreamExecutionEnvironments to build and submit flink job, - // instead we just use StreamExecutionEnvironment#executeAsync(StreamGraph) method - // to execute existing StreamGraph. - // This requires StreamExecutionEnvironment to have a full flink configuration. - return new StreamExecutionEnvironment(new Configuration(flinkConfig), classLoader); - } } diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java index c45d5af79b702..d392460b1dde8 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java @@ -221,7 +221,8 @@ public static SessionContext create(DefaultContext defaultContext, String sessio ModuleManager moduleManager = new ModuleManager(); - final EnvironmentSettings settings = EnvironmentSettings.fromConfiguration(configuration); + final EnvironmentSettings settings = + EnvironmentSettings.newInstance().withConfiguration(configuration).build(); CatalogManager catalogManager = CatalogManager.newBuilder() diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java index 09e651981be54..6c968582294f7 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/StreamTableEnvironment.java @@ -31,7 +31,6 @@ import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.StatementSet; import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl; import org.apache.flink.table.connector.ChangelogMode; @@ -92,9 +91,7 @@ public interface StreamTableEnvironment extends TableEnvironment { * TableEnvironment}. */ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnvironment) { - return create( - executionEnvironment, - EnvironmentSettings.fromConfiguration(executionEnvironment.getConfiguration())); + return create(executionEnvironment, EnvironmentSettings.newInstance().build()); } /** @@ -122,9 +119,7 @@ static StreamTableEnvironment create(StreamExecutionEnvironment executionEnviron */ static StreamTableEnvironment create( StreamExecutionEnvironment executionEnvironment, EnvironmentSettings settings) { - TableConfig tableConfig = new TableConfig(); - tableConfig.addConfiguration(settings.toConfiguration()); - return StreamTableEnvironmentImpl.create(executionEnvironment, settings, tableConfig); + return StreamTableEnvironmentImpl.create(executionEnvironment, settings); } /** diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java index 5d66cfaeb70ca..cb8d38b8c6b41 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/table/api/bridge/java/internal/StreamTableEnvironmentImpl.java @@ -92,19 +92,23 @@ public StreamTableEnvironmentImpl( } public static StreamTableEnvironment create( - StreamExecutionEnvironment executionEnvironment, - EnvironmentSettings settings, - TableConfig tableConfig) { + StreamExecutionEnvironment executionEnvironment, EnvironmentSettings settings) { // temporary solution until FLINK-15635 is fixed final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + final Executor executor = lookupExecutor(classLoader, executionEnvironment); + + final TableConfig tableConfig = TableConfig.getDefault(); + tableConfig.setRootConfiguration(executor.getConfiguration()); + tableConfig.addConfiguration(settings.getConfiguration()); + final ModuleManager moduleManager = new ModuleManager(); final CatalogManager catalogManager = CatalogManager.newBuilder() .classLoader(classLoader) - .config(tableConfig.getConfiguration()) + .config(tableConfig) .defaultCatalog( settings.getBuiltInCatalogName(), new GenericInMemoryCatalog( @@ -116,8 +120,6 @@ public static StreamTableEnvironment create( final FunctionCatalog functionCatalog = new FunctionCatalog(tableConfig, catalogManager, moduleManager); - final Executor executor = lookupExecutor(classLoader, executionEnvironment); - final Planner planner = PlannerFactoryUtil.createPlanner( executor, tableConfig, moduleManager, catalogManager, functionCatalog); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java index d8a246408c159..258fcb8acdb64 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/EnvironmentSettings.java @@ -21,11 +21,14 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.functions.UserDefinedFunction; import static org.apache.flink.api.common.RuntimeExecutionMode.BATCH; import static org.apache.flink.api.common.RuntimeExecutionMode.STREAMING; import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; +import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_CATALOG_NAME; +import static org.apache.flink.table.api.config.TableConfigOptions.TABLE_DATABASE_NAME; /** * Defines all parameters that initialize a table environment. Those parameters are used only during @@ -53,32 +56,14 @@ public class EnvironmentSettings { private static final EnvironmentSettings DEFAULT_BATCH_MODE_SETTINGS = EnvironmentSettings.newInstance().inBatchMode().build(); - public static final String DEFAULT_BUILTIN_CATALOG = "default_catalog"; - public static final String DEFAULT_BUILTIN_DATABASE = "default_database"; - - /** - * Specifies the name of the initial catalog to be created when instantiating {@link - * TableEnvironment}. - */ - private final String builtInCatalogName; - - /** - * Specifies the name of the default database in the initial catalog to be created when - * instantiating {@link TableEnvironment}. - */ - private final String builtInDatabaseName; - /** - * Determines if the table environment should work in a batch ({@code false}) or streaming - * ({@code true}) mode. + * Holds all the configuration generated by the builder, together with any required additional + * configuration. */ - private final boolean isStreamingMode; + private final Configuration configuration; - private EnvironmentSettings( - String builtInCatalogName, String builtInDatabaseName, boolean isStreamingMode) { - this.builtInCatalogName = builtInCatalogName; - this.builtInDatabaseName = builtInDatabaseName; - this.isStreamingMode = isStreamingMode; + private EnvironmentSettings(Configuration configuration) { + this.configuration = configuration; } /** @@ -111,32 +96,28 @@ public static Builder newInstance() { return new Builder(); } - /** Creates an instance of {@link EnvironmentSettings} from configuration. */ + /** + * Creates an instance of {@link EnvironmentSettings} from configuration. + * + * @deprecated use {@link Builder#withConfiguration(Configuration)} instead. + */ + @Deprecated public static EnvironmentSettings fromConfiguration(ReadableConfig configuration) { - final Builder builder = new Builder(); - switch (configuration.get(RUNTIME_MODE)) { - case STREAMING: - builder.inStreamingMode(); - break; - case BATCH: - builder.inBatchMode(); - break; - case AUTOMATIC: - default: - throw new TableException( - String.format( - "Unsupported mode '%s' for '%s'. " - + "Only an explicit BATCH or STREAMING mode is supported in Table API.", - configuration.get(RUNTIME_MODE), RUNTIME_MODE.key())); - } - - return builder.build(); + return new EnvironmentSettings((Configuration) configuration); } - /** Convert the environment setting to the {@link Configuration}. */ + /** + * Convert the environment setting to the {@link Configuration}. + * + * @deprecated use {@link #getConfiguration} instead. + */ + @Deprecated public Configuration toConfiguration() { - Configuration configuration = new Configuration(); - configuration.set(RUNTIME_MODE, isStreamingMode() ? STREAMING : BATCH); + return configuration; + } + + /** Get the underlying {@link Configuration}. */ + public Configuration getConfiguration() { return configuration; } @@ -145,7 +126,7 @@ public Configuration toConfiguration() { * TableEnvironment}. */ public String getBuiltInCatalogName() { - return builtInCatalogName; + return configuration.get(TABLE_CATALOG_NAME); } /** @@ -153,31 +134,31 @@ public String getBuiltInCatalogName() { * instantiating a {@link TableEnvironment}. */ public String getBuiltInDatabaseName() { - return builtInDatabaseName; + return configuration.get(TABLE_DATABASE_NAME); } /** Tells if the {@link TableEnvironment} should work in a batch or streaming mode. */ public boolean isStreamingMode() { - return isStreamingMode; + return configuration.get(RUNTIME_MODE) == STREAMING; } /** A builder for {@link EnvironmentSettings}. */ @PublicEvolving public static class Builder { - private String builtInCatalogName = DEFAULT_BUILTIN_CATALOG; - private String builtInDatabaseName = DEFAULT_BUILTIN_DATABASE; - private boolean isStreamingMode = true; + private final Configuration configuration = new Configuration(); + + public Builder() {} /** Sets that the components should work in a batch mode. Streaming mode by default. */ public Builder inBatchMode() { - this.isStreamingMode = false; + configuration.set(RUNTIME_MODE, BATCH); return this; } /** Sets that the components should work in a streaming mode. Enabled by default. */ public Builder inStreamingMode() { - this.isStreamingMode = true; + configuration.set(RUNTIME_MODE, STREAMING); return this; } @@ -193,10 +174,10 @@ public Builder inStreamingMode() { *

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

Default: "default_catalog". + *

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

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

Default: "default_database". + *

Default: {@link TableConfigOptions#TABLE_DATABASE_NAME}{@code .defaultValue()}. */ public Builder withBuiltInDatabaseName(String builtInDatabaseName) { - this.builtInDatabaseName = builtInDatabaseName; + configuration.set(TABLE_DATABASE_NAME, builtInDatabaseName); + return this; + } + + /** Add extra configuration to {@link EnvironmentSettings}. */ + public Builder withConfiguration(Configuration configuration) { + this.configuration.addAll(configuration); return this; } /** Returns an immutable instance of {@link EnvironmentSettings}. */ public EnvironmentSettings build() { - return new EnvironmentSettings( - builtInCatalogName, builtInDatabaseName, isStreamingMode); + return new EnvironmentSettings(configuration); } } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java index 2565bb4b61d49..141ad78f8c779 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java @@ -19,11 +19,13 @@ package org.apache.flink.table.api; import org.apache.flink.annotation.Experimental; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.configuration.WritableConfig; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.config.OptimizerConfigOptions; @@ -35,6 +37,7 @@ import java.time.ZoneId; import java.util.HashMap; import java.util.Map; +import java.util.Optional; import static java.time.ZoneId.SHORT_IDS; @@ -68,7 +71,15 @@ * @see OptimizerConfigOptions */ @PublicEvolving -public class TableConfig implements WritableConfig { +public final class TableConfig implements WritableConfig, ReadableConfig { + // + // TableConfig is also a ReadableConfig which is built once the TableEnvironment is created and + // contains both the configuration defined in the environment (flink-conf.yaml + CLI params), + // stored in rootConfiguration, but also any extra configuration defined by the user in the + // application, which has precedence over the environment configuration. This way, any consumer + // of TableConfig cat get the complete view of the configuration (environment + user defined) by + // calling the get() and getOptional() methods. + /** Defines if all fields need to be checked for NULL first. */ private Boolean nullCheck = true; @@ -84,6 +95,8 @@ public class TableConfig implements WritableConfig { /** A configuration object to hold all key/value configuration. */ private final Configuration configuration = new Configuration(); + private ReadableConfig rootConfiguration = new Configuration(); + /** * Sets a value for the given {@link ConfigOption}. * @@ -121,11 +134,36 @@ public TableConfig set(String key, String value) { return this; } + @Override + public T get(ConfigOption option) { + return configuration.getOptional(option).orElseGet(() -> rootConfiguration.get(option)); + } + + @Override + public Optional getOptional(ConfigOption option) { + final Optional tableValue = configuration.getOptional(option); + if (tableValue.isPresent()) { + return tableValue; + } + return rootConfiguration.getOptional(option); + } + /** Gives direct access to the underlying key-value map for advanced configuration. */ public Configuration getConfiguration() { return configuration; } + /** + * Sets the given key-value configuration as {@link #rootConfiguration}, which contains any + * configuration set in the environment ({@code flink-conf.yaml} + {@code CLI} parameters. + * + * @param rootConfiguration key-value root configuration to be set + */ + @Internal + public void setRootConfiguration(ReadableConfig rootConfiguration) { + this.rootConfiguration = rootConfiguration; + } + /** * Adds the given key-value configuration to the underlying configuration. It overwrites * existing keys. @@ -139,8 +177,7 @@ public void addConfiguration(Configuration configuration) { /** Returns the current SQL dialect. */ public SqlDialect getSqlDialect() { - return SqlDialect.valueOf( - getConfiguration().getString(TableConfigOptions.TABLE_SQL_DIALECT).toUpperCase()); + return SqlDialect.valueOf(get(TableConfigOptions.TABLE_SQL_DIALECT).toUpperCase()); } /** Sets the current SQL dialect to parse a SQL query. Flink's SQL behavior by default. */ @@ -320,9 +357,9 @@ public void setIdleStateRetentionTime(Time minTime, Time maxTime) { && !(maxTime.toMilliseconds() == 0 && minTime.toMilliseconds() == 0)) { throw new IllegalArgumentException( "Difference between minTime: " - + minTime.toString() + + minTime + " and maxTime: " - + maxTime.toString() + + maxTime + " should be at least 5 minutes."); } setIdleStateRetention(Duration.ofMillis(minTime.toMilliseconds())); @@ -398,8 +435,7 @@ public Duration getIdleStateRetention() { @Experimental public void addJobParameter(String key, String value) { Map params = - getConfiguration() - .getOptional(PipelineOptions.GLOBAL_JOB_PARAMETERS) + getOptional(PipelineOptions.GLOBAL_JOB_PARAMETERS) .map(HashMap::new) .orElseGet(HashMap::new); params.put(key, value); diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java index 8601f38bb85c3..d4afa20f7939f 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/TableConfigOptions.java @@ -40,6 +40,24 @@ public class TableConfigOptions { private TableConfigOptions() {} + @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING) + public static final ConfigOption TABLE_CATALOG_NAME = + key("table.builtin-catalog-name") + .stringType() + .defaultValue("default_catalog") + .withDescription( + "The name of the initial catalog to be created when " + + "instantiating a TableEnvironment."); + + @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING) + public static final ConfigOption TABLE_DATABASE_NAME = + key("table.builtin-database-name") + .stringType() + .defaultValue("default_database") + .withDescription( + "The name of the default database in the initial catalog to be created " + + "when instantiating TableEnvironment."); + @Documentation.TableOption(execMode = Documentation.ExecMode.BATCH_STREAMING) public static final ConfigOption TABLE_DML_SYNC = key("table.dml-sync") diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java index 8d6a5a96a4bf5..a692e5510b3e7 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java @@ -265,21 +265,22 @@ protected TableEnvironmentImpl( } public static TableEnvironmentImpl create(Configuration configuration) { - return create(EnvironmentSettings.fromConfiguration(configuration), configuration); + return create(EnvironmentSettings.newInstance().withConfiguration(configuration).build()); } public static TableEnvironmentImpl create(EnvironmentSettings settings) { - return create(settings, settings.toConfiguration()); - } - - private static TableEnvironmentImpl create( - EnvironmentSettings settings, Configuration configuration) { // temporary solution until FLINK-15635 is fixed final ClassLoader classLoader = Thread.currentThread().getContextClassLoader(); + final ExecutorFactory executorFactory = + FactoryUtil.discoverFactory( + classLoader, ExecutorFactory.class, ExecutorFactory.DEFAULT_IDENTIFIER); + final Executor executor = executorFactory.create(settings.getConfiguration()); + // use configuration to init table config final TableConfig tableConfig = new TableConfig(); - tableConfig.addConfiguration(configuration); + tableConfig.setRootConfiguration(executor.getConfiguration()); + tableConfig.addConfiguration(settings.getConfiguration()); final ModuleManager moduleManager = new ModuleManager(); @@ -297,11 +298,6 @@ private static TableEnvironmentImpl create( final FunctionCatalog functionCatalog = new FunctionCatalog(tableConfig, catalogManager, moduleManager); - final ExecutorFactory executorFactory = - FactoryUtil.discoverFactory( - classLoader, ExecutorFactory.class, ExecutorFactory.DEFAULT_IDENTIFIER); - final Executor executor = executorFactory.create(configuration); - final Planner planner = PlannerFactoryUtil.createPlanner( executor, tableConfig, moduleManager, catalogManager, functionCatalog); diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/EnvironmentSettingsTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/EnvironmentSettingsTest.java index bc2428627fa3b..ad4630b6c08e4 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/EnvironmentSettingsTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/api/EnvironmentSettingsTest.java @@ -34,15 +34,16 @@ public class EnvironmentSettingsTest { public void testFromConfiguration() { Configuration configuration = new Configuration(); configuration.setString("execution.runtime-mode", "batch"); - EnvironmentSettings settings = EnvironmentSettings.fromConfiguration(configuration); + EnvironmentSettings settings = + EnvironmentSettings.newInstance().withConfiguration(configuration).build(); assertFalse("Expect batch mode.", settings.isStreamingMode()); } @Test - public void testToConfiguration() { + public void testGetConfiguration() { EnvironmentSettings settings = new EnvironmentSettings.Builder().inBatchMode().build(); - Configuration configuration = settings.toConfiguration(); + Configuration configuration = settings.getConfiguration(); assertEquals(RuntimeExecutionMode.BATCH, configuration.get(RUNTIME_MODE)); } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java index c94dee9d798c3..b30f80665e473 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/FunctionCatalogTest.java @@ -18,7 +18,7 @@ package org.apache.flink.table.catalog; -import org.apache.flink.table.api.TableConfig; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.functions.AggregateFunction; import org.apache.flink.table.functions.FunctionDefinition; @@ -85,7 +85,7 @@ public void init() { functionCatalog = new FunctionCatalog( - TableConfig.getDefault(), + new Configuration(), CatalogManagerMocks.preparedCatalogManager() .defaultCatalog(DEFAULT_CATALOG, catalog) .build(), diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/CatalogManagerMocks.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/CatalogManagerMocks.java index 94bc1823a4c13..c3fb2e45ae456 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/CatalogManagerMocks.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/CatalogManagerMocks.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.GenericInMemoryCatalog; @@ -30,9 +30,11 @@ /** Mock implementations of {@link CatalogManager} for testing purposes. */ public final class CatalogManagerMocks { - public static final String DEFAULT_CATALOG = EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; + public static final String DEFAULT_CATALOG = + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(); - public static final String DEFAULT_DATABASE = EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; + public static final String DEFAULT_DATABASE = + TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(); public static CatalogManager createEmptyCatalogManager() { return createCatalogManager(null); diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java index cb09387a842ff..f917132686d1a 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java @@ -18,6 +18,7 @@ package org.apache.flink.table.utils; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.internal.TableEnvironmentImpl; @@ -69,7 +70,7 @@ public static TableEnvironmentMock getBatchInstance() { } private static TableEnvironmentMock getInstance(boolean isStreamingMode) { - final TableConfig tableConfig = createTableConfig(); + final TableConfig tableConfig = TableConfig.getDefault(); final CatalogManager catalogManager = CatalogManagerMocks.createEmptyCatalogManager(); final ModuleManager moduleManager = new ModuleManager(); return new TableEnvironmentMock( @@ -77,22 +78,19 @@ private static TableEnvironmentMock getInstance(boolean isStreamingMode) { moduleManager, tableConfig, createExecutor(), - createFunctionCatalog(tableConfig, catalogManager, moduleManager), + createFunctionCatalog( + tableConfig.getConfiguration(), catalogManager, moduleManager), createPlanner(), isStreamingMode); } - private static TableConfig createTableConfig() { - return TableConfig.getDefault(); - } - private static ExecutorMock createExecutor() { return new ExecutorMock(); } private static FunctionCatalog createFunctionCatalog( - TableConfig tableConfig, CatalogManager catalogManager, ModuleManager moduleManager) { - return new FunctionCatalog(tableConfig, catalogManager, moduleManager); + ReadableConfig config, CatalogManager catalogManager, ModuleManager moduleManager) { + return new FunctionCatalog(config, catalogManager, moduleManager); } private static PlannerMock createPlanner() { diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala index e8463eeb2d806..345e2f921ca11 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/StreamTableEnvironment.scala @@ -18,7 +18,6 @@ package org.apache.flink.table.api.bridge.scala import org.apache.flink.annotation.PublicEvolving -import org.apache.flink.api.common.JobExecutionResult import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.CompositeType import org.apache.flink.streaming.api.scala.{DataStream, StreamExecutionEnvironment} @@ -869,9 +868,7 @@ object StreamTableEnvironment { * [[TableEnvironment]]. */ def create(executionEnvironment: StreamExecutionEnvironment): StreamTableEnvironment = { - create( - executionEnvironment, - EnvironmentSettings.fromConfiguration(executionEnvironment.getConfiguration)) + create(executionEnvironment, EnvironmentSettings.newInstance().build) } /** @@ -899,9 +896,6 @@ object StreamTableEnvironment { executionEnvironment: StreamExecutionEnvironment, settings: EnvironmentSettings) : StreamTableEnvironment = { - val config = new TableConfig() - config.addConfiguration(settings.toConfiguration) - StreamTableEnvironmentImpl - .create(executionEnvironment, settings, config) + StreamTableEnvironmentImpl.create(executionEnvironment, settings) } } diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala index a968e49c22ea3..cbc4f382809ec 100644 --- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala +++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.table.api.bridge.scala.internal -import org.apache.flink.annotation.Internal +import org.apache.flink.annotation.{Internal, VisibleForTesting} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.scala._ import org.apache.flink.streaming.api.TimeCharacteristic @@ -40,6 +40,7 @@ import org.apache.flink.types.Row import org.apache.flink.util.Preconditions import java.util.Optional + import scala.collection.JavaConverters._ /** @@ -294,13 +295,19 @@ object StreamTableEnvironmentImpl { def create( executionEnvironment: StreamExecutionEnvironment, - settings: EnvironmentSettings, - tableConfig: TableConfig) + settings: EnvironmentSettings) : StreamTableEnvironmentImpl = { // temporary solution until FLINK-15635 is fixed val classLoader = Thread.currentThread.getContextClassLoader + val executor = AbstractStreamTableEnvironmentImpl.lookupExecutor( + classLoader, executionEnvironment.getWrappedStreamExecutionEnvironment) + + val tableConfig = TableConfig.getDefault + tableConfig.setRootConfiguration(executor.getConfiguration) + tableConfig.addConfiguration(settings.getConfiguration) + val moduleManager = new ModuleManager val catalogManager = CatalogManager.newBuilder @@ -316,9 +323,6 @@ object StreamTableEnvironmentImpl { val functionCatalog = new FunctionCatalog(tableConfig, catalogManager, moduleManager) - val executor = AbstractStreamTableEnvironmentImpl.lookupExecutor( - classLoader, executionEnvironment.getWrappedStreamExecutionEnvironment) - val planner = PlannerFactoryUtil.createPlanner( executor, tableConfig, moduleManager, catalogManager, functionCatalog) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/DefaultPlannerFactory.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/DefaultPlannerFactory.java index 4476a7a7517f9..b8659ba3250f8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/DefaultPlannerFactory.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/DefaultPlannerFactory.java @@ -29,6 +29,8 @@ import java.util.Collections; import java.util.Set; +import static org.apache.flink.configuration.ExecutionOptions.RUNTIME_MODE; + /** Factory for the default {@link Planner}. */ @Internal public final class DefaultPlannerFactory implements PlannerFactory { @@ -51,7 +53,7 @@ public Set> optionalOptions() { @Override public Planner create(Context context) { final RuntimeExecutionMode runtimeExecutionMode = - context.getTableConfig().getConfiguration().get(ExecutionOptions.RUNTIME_MODE); + context.getTableConfig().get(ExecutionOptions.RUNTIME_MODE); switch (runtimeExecutionMode) { case STREAMING: return new StreamPlanner( @@ -70,8 +72,9 @@ public Planner create(Context context) { default: throw new TableException( String.format( - "Unknown runtime mode '%s'. This is a bug. Please consider filing an issue.", - runtimeExecutionMode)); + "Unsupported mode '%s' for '%s'. Only an explicit BATCH or " + + "STREAMING mode is supported in Table API.", + runtimeExecutionMode, RUNTIME_MODE.key())); } } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/EnvironmentTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/EnvironmentTest.java index ff349536723bb..eaa25823996df 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/EnvironmentTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/EnvironmentTest.java @@ -24,12 +24,15 @@ import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.types.Row; import org.junit.Test; import java.time.Duration; +import java.util.concurrent.ExecutionException; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; /** Tests for {@link TableEnvironment} that require a planner. */ @@ -61,4 +64,29 @@ public void testPassingExecutionParameters() { assertEquals(800, env.getConfig().getAutoWatermarkInterval()); assertEquals(30000, env.getCheckpointConfig().getCheckpointInterval()); } + + @Test + public void testEnvironmentSettings() throws ExecutionException, InterruptedException { + Configuration conf = new Configuration(); + conf.set(TableConfigOptions.TABLE_CATALOG_NAME, "myCatalog"); + EnvironmentSettings settings = + EnvironmentSettings.newInstance().withConfiguration(conf).build(); + + TableEnvironment tEnv = TableEnvironment.create(settings); + assertThat(tEnv.getConfig().get(TableConfigOptions.TABLE_CATALOG_NAME)) + .isEqualTo("myCatalog"); + assertThat(tEnv.getCurrentCatalog()).isEqualTo("myCatalog"); + + StreamTableEnvironment stEnv = + StreamTableEnvironment.create( + StreamExecutionEnvironment.getExecutionEnvironment(), settings); + assertThat(stEnv.getConfig().get(TableConfigOptions.TABLE_CATALOG_NAME)) + .isEqualTo("myCatalog"); + + stEnv.getConfig() + .set( + TableConfigOptions.TABLE_CATALOG_NAME, + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue()); + assertThat(stEnv.getCurrentCatalog()).isEqualTo("myCatalog"); + } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/TransformationsTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/TransformationsTest.java index 31ac8b52d3771..b0667b14ff937 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/TransformationsTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/TransformationsTest.java @@ -110,7 +110,7 @@ public void testLegacyBatchValues() { @Test public void testLegacyUid() { final TableEnvironment env = - TableEnvironment.create(EnvironmentSettings.inStreamingMode().toConfiguration()); + TableEnvironment.create(EnvironmentSettings.inStreamingMode().getConfiguration()); env.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_LEGACY_TRANSFORMATION_UIDS, true); env.createTemporaryTable( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java index 57a712f2ab367..d6857b515dfb2 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSinkSpecSerdeTest.java @@ -44,6 +44,7 @@ import org.apache.flink.table.types.logical.BigIntType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.CatalogManagerMocks; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.Execution; @@ -57,8 +58,6 @@ import java.util.Map; import java.util.stream.Stream; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; import static org.apache.flink.table.api.config.TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS; import static org.apache.flink.table.api.config.TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; @@ -99,8 +98,8 @@ static Stream testDynamicTableSinkSpecSerde() { new DynamicTableSinkSpec( ContextResolvedTable.temporary( ObjectIdentifier.of( - DEFAULT_BUILTIN_CATALOG, - DEFAULT_BUILTIN_DATABASE, + CatalogManagerMocks.DEFAULT_CATALOG, + CatalogManagerMocks.DEFAULT_DATABASE, "MyTable"), new ResolvedCatalogTable(catalogTable1, resolvedSchema1)), null); @@ -129,8 +128,8 @@ static Stream testDynamicTableSinkSpecSerde() { new DynamicTableSinkSpec( ContextResolvedTable.temporary( ObjectIdentifier.of( - DEFAULT_BUILTIN_CATALOG, - DEFAULT_BUILTIN_DATABASE, + CatalogManagerMocks.DEFAULT_CATALOG, + CatalogManagerMocks.DEFAULT_DATABASE, "MyTable"), new ResolvedCatalogTable(catalogTable2, resolvedSchema2)), Arrays.asList( @@ -165,8 +164,8 @@ static Stream testDynamicTableSinkSpecSerde() { new DynamicTableSinkSpec( ContextResolvedTable.temporary( ObjectIdentifier.of( - DEFAULT_BUILTIN_CATALOG, - DEFAULT_BUILTIN_DATABASE, + CatalogManagerMocks.DEFAULT_CATALOG, + CatalogManagerMocks.DEFAULT_DATABASE, "MyTable"), new ResolvedCatalogTable(catalogTable3, resolvedSchema3)), Collections.singletonList( @@ -214,7 +213,10 @@ void testDynamicTableSinkSpecSerde(DynamicTableSinkSpec spec) throws IOException void testDynamicTableSinkSpecSerdeWithEnrichmentOptions() throws Exception { // Test model ObjectIdentifier identifier = - ObjectIdentifier.of(DEFAULT_BUILTIN_CATALOG, DEFAULT_BUILTIN_DATABASE, "my_table"); + ObjectIdentifier.of( + CatalogManagerMocks.DEFAULT_CATALOG, + CatalogManagerMocks.DEFAULT_DATABASE, + "my_table"); String formatPrefix = FactoryUtil.getFormatPrefix(FORMAT, TestFormatFactory.IDENTIFIER); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java index a15acb7716cd2..3764f6b6b1512 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/DynamicTableSourceSpecSerdeTest.java @@ -23,6 +23,7 @@ import org.apache.flink.formats.testcsv.TestCsvFormatFactory; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.config.TableConfigOptions.CatalogPlanCompilation; import org.apache.flink.table.api.config.TableConfigOptions.CatalogPlanRestore; import org.apache.flink.table.catalog.CatalogManager; @@ -72,8 +73,6 @@ import java.util.Map; import java.util.stream.Stream; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_CATALOG; -import static org.apache.flink.table.api.EnvironmentSettings.DEFAULT_BUILTIN_DATABASE; import static org.apache.flink.table.api.config.TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS; import static org.apache.flink.table.api.config.TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS; import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; @@ -114,8 +113,8 @@ public static Stream testDynamicTableSinkSpecSerde() { new DynamicTableSourceSpec( ContextResolvedTable.temporary( ObjectIdentifier.of( - DEFAULT_BUILTIN_CATALOG, - DEFAULT_BUILTIN_DATABASE, + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(), + TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(), "MyTable"), new ResolvedCatalogTable(catalogTable1, resolvedSchema1)), null); @@ -154,8 +153,8 @@ public static Stream testDynamicTableSinkSpecSerde() { new DynamicTableSourceSpec( ContextResolvedTable.temporary( ObjectIdentifier.of( - DEFAULT_BUILTIN_CATALOG, - DEFAULT_BUILTIN_DATABASE, + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(), + TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(), "MyTable"), new ResolvedCatalogTable(catalogTable2, resolvedSchema2)), Arrays.asList( @@ -274,7 +273,10 @@ void testDynamicTableSourceSpecSerde(DynamicTableSourceSpec spec) throws IOExcep void testDynamicTableSourceSpecSerdeWithEnrichmentOptions() throws Exception { // Test model ObjectIdentifier identifier = - ObjectIdentifier.of(DEFAULT_BUILTIN_CATALOG, DEFAULT_BUILTIN_DATABASE, "my_table"); + ObjectIdentifier.of( + TableConfigOptions.TABLE_CATALOG_NAME.defaultValue(), + TableConfigOptions.TABLE_DATABASE_NAME.defaultValue(), + "my_table"); String formatPrefix = FactoryUtil.getFormatPrefix(FORMAT, TestFormatFactory.IDENTIFIER); diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala index 07dff77f4a4fa..52fc78bf5fcb6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala @@ -22,11 +22,11 @@ import org.apache.flink.api.common.TaskInfo import org.apache.flink.api.common.functions.util.RuntimeUDFContext import org.apache.flink.api.common.functions.{MapFunction, RichFunction, RichMapFunction} import org.apache.flink.api.java.typeutils.RowTypeInfo -import org.apache.flink.configuration.Configuration +import org.apache.flink.configuration.{ConfigOption, Configuration} import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.table.api import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl -import org.apache.flink.table.api.config.ExecutionConfigOptions +import org.apache.flink.table.api.config.{ExecutionConfigOptions, TableConfigOptions} import org.apache.flink.table.api.{EnvironmentSettings, TableConfig, TableException, ValidationException} import org.apache.flink.table.data.RowData import org.apache.flink.table.data.binary.BinaryRowData @@ -38,6 +38,7 @@ import org.apache.flink.table.expressions.Expression import org.apache.flink.table.functions.ScalarFunction import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, ExprCodeGenerator, FunctionCodeGenerator} import org.apache.flink.table.planner.delegation.PlannerBase +import org.apache.flink.table.planner.utils.TestingTableEnvironment import org.apache.flink.table.runtime.generated.GeneratedFunction import org.apache.flink.table.runtime.types.TypeInfoLogicalTypeConverter.fromTypeInfoToLogicalType import org.apache.flink.table.types.AbstractDataType @@ -51,10 +52,12 @@ import org.apache.calcite.rel.logical.LogicalCalc import org.apache.calcite.rel.rules._ import org.apache.calcite.rex.RexNode import org.apache.calcite.sql.`type`.SqlTypeName.VARCHAR + import org.junit.Assert.{assertEquals, assertTrue, fail} import org.junit.rules.ExpectedException import org.junit.{After, Before, Rule} +import java.time.ZoneId import java.util.Collections import scala.collection.JavaConverters._ @@ -62,8 +65,6 @@ import scala.collection.mutable abstract class ExpressionTestBase { - val config = new TableConfig() - // (originalExpr, optimizedExpr, expectedResult) private val validExprs = mutable.ArrayBuffer[(String, RexNode, String)]() // (originalSqlExpr, keywords, exceptionClass) @@ -73,11 +74,14 @@ abstract class ExpressionTestBase { .ArrayBuffer[(Expression, String, Class[_ <: Throwable])]() private val env = StreamExecutionEnvironment.createLocalEnvironment(4) - private val setting = EnvironmentSettings.newInstance().inStreamingMode().build() + private val settings = EnvironmentSettings.newInstance().inStreamingMode().build() // use impl class instead of interface class to avoid // "Static methods in interface require -target:jvm-1.8" - private val tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + private val tEnv = StreamTableEnvironmentImpl.create(env, settings) .asInstanceOf[StreamTableEnvironmentImpl] + + val config = tEnv.getConfig + private val resolvedDataType = if (containsLegacyTypes) { TypeConversions.fromLegacyInfoToDataType(typeInfo) } else { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala index 39b73ef0cbd45..8a81fc662de3d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/GroupAggregateHarnessTest.scala @@ -56,8 +56,7 @@ class GroupAggregateHarnessTest(mode: StateBackendMode, miniBatch: MiniBatchMode override def before(): Unit = { super.before() val setting = EnvironmentSettings.newInstance().inStreamingMode().build() - val config = new TestTableConfig - this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + this.tEnv = StreamTableEnvironmentImpl.create(env, setting) // set mini batch val tableConfig = tEnv.getConfig miniBatch match { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/HarnessTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/HarnessTestBase.scala index 6aff43d554341..f634b1528e09b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/HarnessTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/HarnessTestBase.scala @@ -17,7 +17,6 @@ */ package org.apache.flink.table.planner.runtime.harness -import org.apache.flink.api.common.time.Time import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.dag.Transformation import org.apache.flink.api.java.functions.KeySelector @@ -30,14 +29,13 @@ import org.apache.flink.streaming.api.scala.DataStream import org.apache.flink.streaming.api.transformations.{OneInputTransformation, PartitionTransformation} import org.apache.flink.streaming.api.watermark.Watermark import org.apache.flink.streaming.util.{KeyedOneInputStreamOperatorTestHarness, OneInputStreamOperatorTestHarness} -import org.apache.flink.table.api.TableConfig import org.apache.flink.table.data.RowData import org.apache.flink.table.planner.JLong import org.apache.flink.table.planner.runtime.utils.StreamingTestBase import org.apache.flink.table.planner.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode} + import org.junit.runners.Parameterized -import java.time.Duration import java.util import scala.collection.JavaConversions._ @@ -120,23 +118,6 @@ class HarnessTestBase(mode: StateBackendMode) extends StreamingTestBase { def dropWatermarks(elements: Array[AnyRef]): util.Collection[AnyRef] = { elements.filter(e => !e.isInstanceOf[Watermark]).toList } - - class TestTableConfig extends TableConfig { - - private var minIdleStateRetentionTime = 0L - - private var maxIdleStateRetentionTime = 0L - - override def getMinIdleStateRetentionTime: Long = minIdleStateRetentionTime - - override def getMaxIdleStateRetentionTime: Long = maxIdleStateRetentionTime - - override def setIdleStateRetentionTime(minTime: Time, maxTime: Time): Unit = { - super.setIdleStateRetention(Duration.ofMillis(minTime.toMilliseconds)) - minIdleStateRetentionTime = minTime.toMilliseconds - maxIdleStateRetentionTime = maxTime.toMilliseconds - } - } } object HarnessTestBase { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/OverAggregateHarnessTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/OverAggregateHarnessTest.scala index 7c1f566f02a9f..e2d280434bc1e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/OverAggregateHarnessTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/OverAggregateHarnessTest.scala @@ -38,6 +38,7 @@ import org.junit.runners.Parameterized import org.junit.{Before, Test} import java.lang.{Long => JLong} +import java.time.Duration import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.mutable @@ -49,8 +50,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m override def before(): Unit = { super.before() val setting = EnvironmentSettings.newInstance().inStreamingMode().build() - val config = new TestTableConfig - this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + this.tEnv = StreamTableEnvironmentImpl.create(env, setting) } @Test @@ -127,9 +127,9 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m expectedOutput.add(new StreamRecord( row(2L: JLong, "aaa", 8L: JLong, null, 7L: JLong, 8L: JLong))) expectedOutput.add(new StreamRecord( - row(2L: JLong, "aaa", 9L: JLong, null, 8L: JLong, 9L: JLong))) + row(2L: JLong, "aaa", 10L: JLong, null, 10L: JLong, 10L: JLong))) expectedOutput.add(new StreamRecord( - row(2L: JLong, "aaa", 10L: JLong, null, 9L: JLong, 10L: JLong))) + row(2L: JLong, "aaa", 9L: JLong, null, 8L: JLong, 9L: JLong))) expectedOutput.add(new StreamRecord( row(2L: JLong, "bbb", 40L: JLong, null, 40L: JLong, 40L: JLong))) @@ -155,7 +155,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m """.stripMargin val t1 = tEnv.sqlQuery(sql) - tEnv.getConfig.setIdleStateRetentionTime(Time.seconds(2), Time.seconds(4)) + tEnv.getConfig.setIdleStateRetention(Duration.ofSeconds(2)) val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate") val outputType = Array( DataTypes.BIGINT().getLogicalType, @@ -306,7 +306,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m """.stripMargin val t1 = tEnv.sqlQuery(sql) - tEnv.getConfig.setIdleStateRetentionTime(Time.seconds(2), Time.seconds(4)) + tEnv.getConfig.setIdleStateRetention(Duration.ofSeconds(2)) val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate") val assertor = new RowDataHarnessAssertor( Array( @@ -533,7 +533,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m """.stripMargin val t1 = tEnv.sqlQuery(sql) - tEnv.getConfig.setIdleStateRetentionTime(Time.seconds(1), Time.seconds(2)) + tEnv.getConfig.setIdleStateRetention(Duration.ofSeconds(1)) val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate") val assertor = new RowDataHarnessAssertor( Array( @@ -685,7 +685,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m """.stripMargin val t1 = tEnv.sqlQuery(sql) - tEnv.getConfig.setIdleStateRetentionTime(Time.seconds(1), Time.seconds(2)) + tEnv.getConfig.setIdleStateRetention(Duration.ofSeconds(1)) val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate") val assertor = new RowDataHarnessAssertor( Array( @@ -827,7 +827,7 @@ class OverAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase(m """.stripMargin val t1 = tEnv.sqlQuery(sql) - tEnv.getConfig.setIdleStateRetentionTime(Time.seconds(1), Time.seconds(2)) + tEnv.getConfig.setIdleStateRetention(Duration.ofSeconds(1)) val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate") val assertor = new RowDataHarnessAssertor( Array( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/RankHarnessTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/RankHarnessTest.scala index 3c97370339d4c..14724a6dc7243 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/RankHarnessTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/RankHarnessTest.scala @@ -49,8 +49,7 @@ class RankHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode) { override def before(): Unit = { super.before() val setting = EnvironmentSettings.newInstance().inStreamingMode().build() - val config = new TestTableConfig - this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + this.tEnv = StreamTableEnvironmentImpl.create(env, setting) } @Test diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/TableAggregateHarnessTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/TableAggregateHarnessTest.scala index d9dcf7fbc5745..8a4378d66ed8f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/TableAggregateHarnessTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/harness/TableAggregateHarnessTest.scala @@ -49,8 +49,7 @@ class TableAggregateHarnessTest(mode: StateBackendMode) extends HarnessTestBase( override def before(): Unit = { super.before() val setting = EnvironmentSettings.newInstance().inStreamingMode().build() - val config = new TestTableConfig - this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config) + this.tEnv = StreamTableEnvironmentImpl.create(env, setting) } val data = new mutable.MutableList[(Int, Int)] diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala index 1282aba6cdb13..939d2c357e5c0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/StreamTableEnvironmentITCase.scala @@ -19,11 +19,20 @@ package org.apache.flink.table.planner.runtime.stream.sql import org.apache.flink.api.scala._ +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.datastream.DataStreamSource +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment import org.apache.flink.streaming.api.scala.DataStream import org.apache.flink.table.api._ +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment +import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl +import org.apache.flink.table.api.bridge.scala import org.apache.flink.table.api.bridge.scala._ +import org.apache.flink.table.api.config.TableConfigOptions import org.apache.flink.table.planner.runtime.utils.JavaPojos.{Device, Order, Person, ProductItem} import org.apache.flink.table.planner.runtime.utils.{StreamingTestBase, StringSink} +import org.assertj.core.api.Assertions.assertThat + import org.junit.Assert.assertEquals import org.junit.Test @@ -137,4 +146,20 @@ class StreamTableEnvironmentITCase extends StreamingTestBase { "(true,Order{user=1, product='Product{name='beer', id=10}', amount=3})") assertEquals(expected.sorted, sink.getResults.sorted) } + + @Test + def testTableConfigInheritsEnvironmentSettings(): Unit = { + val config = new Configuration + config.setString(TableConfigOptions.TABLE_CATALOG_NAME, "myCatalog") + val env = StreamExecutionEnvironment.getExecutionEnvironment(config) + val tEnv = StreamTableEnvironment.create(env) + assertThat(tEnv.getConfig.get(TableConfigOptions.TABLE_CATALOG_NAME)).isEqualTo("myCatalog") + + val scalaEnv = org.apache.flink.streaming.api.scala.StreamExecutionEnvironment + .getExecutionEnvironment + val scalaTEnv = scala.StreamTableEnvironment.create( + scalaEnv, EnvironmentSettings.newInstance.withConfiguration(config).build) + assertThat(scalaTEnv.getConfig.get(TableConfigOptions.TABLE_CATALOG_NAME)) + .isEqualTo("myCatalog") + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala index 0d088d6defbaf..cb564470db523 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala @@ -1526,11 +1526,18 @@ object TestingTableEnvironment { catalogManager: Option[CatalogManager] = None, tableConfig: TableConfig): TestingTableEnvironment = { - tableConfig.addConfiguration(settings.toConfiguration) - // temporary solution until FLINK-15635 is fixed val classLoader = Thread.currentThread.getContextClassLoader + val executorFactory = FactoryUtil.discoverFactory( + classLoader, classOf[ExecutorFactory], ExecutorFactory.DEFAULT_IDENTIFIER) + + val executor = executorFactory.create(settings.getConfiguration) + + tableConfig.setRootConfiguration(executor.getConfiguration) + tableConfig.addConfiguration(settings.getConfiguration) + + val moduleManager = new ModuleManager val catalogMgr = catalogManager match { @@ -1538,7 +1545,7 @@ object TestingTableEnvironment { case _ => CatalogManager.newBuilder .classLoader(classLoader) - .config(tableConfig.getConfiguration) + .config(tableConfig) .defaultCatalog( settings.getBuiltInCatalogName, new GenericInMemoryCatalog( @@ -1547,12 +1554,7 @@ object TestingTableEnvironment { .build } - val functionCatalog = new FunctionCatalog(tableConfig, catalogMgr, moduleManager) - - val executorFactory = FactoryUtil.discoverFactory( - classLoader, classOf[ExecutorFactory], ExecutorFactory.DEFAULT_IDENTIFIER) - - val executor = executorFactory.create(tableConfig.getConfiguration) + val functionCatalog = new FunctionCatalog(settings.getConfiguration, catalogMgr, moduleManager) val planner = PlannerFactoryUtil.createPlanner( executor, tableConfig, moduleManager, catalogMgr, functionCatalog).asInstanceOf[PlannerBase] From f14f63623056d8ea2c3aba6d0bc017e0418a5ea2 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 14 Mar 2022 17:08:04 +0100 Subject: [PATCH 005/258] [FLINK-26421][table-api-java] Improve JavaDocs of TableConfig This closes #18980. --- .../apache/flink/table/api/TableConfig.java | 104 ++++++++++++++---- 1 file changed, 81 insertions(+), 23 deletions(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java index 141ad78f8c779..9b6de0c668447 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java @@ -30,6 +30,7 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.config.OptimizerConfigOptions; import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.delegation.Executor; import org.apache.flink.util.Preconditions; import java.math.MathContext; @@ -45,6 +46,29 @@ * Configuration for the current {@link TableEnvironment} session to adjust Table & SQL API * programs. * + *

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

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

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

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

For common or important configuration options, this class provides getters and setters methods * with detailed inline documentation. * @@ -72,13 +96,18 @@ */ @PublicEvolving public final class TableConfig implements WritableConfig, ReadableConfig { + + // Note to implementers: + // TableConfig is a ReadableConfig which is built once the TableEnvironment is created and + // contains both the configuration defined in the execution context (flink-conf.yaml + CLI + // params), stored in rootConfiguration, but also any extra configuration defined by the user in + // the application, which has precedence over the execution configuration. // - // TableConfig is also a ReadableConfig which is built once the TableEnvironment is created and - // contains both the configuration defined in the environment (flink-conf.yaml + CLI params), - // stored in rootConfiguration, but also any extra configuration defined by the user in the - // application, which has precedence over the environment configuration. This way, any consumer - // of TableConfig cat get the complete view of the configuration (environment + user defined) by - // calling the get() and getOptional() methods. + // This way, any consumer of TableConfig can get the complete view of the configuration + // (environment + user-defined/application-specific) by calling the get() and getOptional() + // methods. + // + // The set() methods only impact the application-specific configuration. /** Defines if all fields need to be checked for NULL first. */ private Boolean nullCheck = true; @@ -92,13 +121,17 @@ public final class TableConfig implements WritableConfig, ReadableConfig { */ private MathContext decimalContext = MathContext.DECIMAL128; - /** A configuration object to hold all key/value configuration. */ + /** + * A configuration object to hold all configuration that has been set specifically in the Table + * API. It does not contain configuration from outer layers. + */ private final Configuration configuration = new Configuration(); + /** Configuration adopted from the outer layer (i.e. the {@link Executor}). */ private ReadableConfig rootConfiguration = new Configuration(); /** - * Sets a value for the given {@link ConfigOption}. + * Sets an application-specific value for the given {@link ConfigOption}. * *

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

The value will be parsed by the framework on access. * @@ -134,11 +167,33 @@ public TableConfig set(String key, String value) { return this; } + /** + * {@inheritDoc} + * + *

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

This method gives read-only access to the full configuration. However, + * application-specific configuration has precedence. Configuration of outer layers is used for + * defaults and fallbacks. See the docs of {@link TableConfig} for more information. + * + * @param option metadata of the option to read + * @param type of the value to read + * @return read value or {@link Optional#empty()} if not found + */ @Override public Optional getOptional(ConfigOption option) { final Optional tableValue = configuration.getOptional(option); @@ -148,25 +203,17 @@ public Optional getOptional(ConfigOption option) { return rootConfiguration.getOptional(option); } - /** Gives direct access to the underlying key-value map for advanced configuration. */ - public Configuration getConfiguration() { - return configuration; - } - /** - * Sets the given key-value configuration as {@link #rootConfiguration}, which contains any - * configuration set in the environment ({@code flink-conf.yaml} + {@code CLI} parameters. - * - * @param rootConfiguration key-value root configuration to be set + * Gives direct access to the underlying application-specific key-value map for advanced + * configuration. */ - @Internal - public void setRootConfiguration(ReadableConfig rootConfiguration) { - this.rootConfiguration = rootConfiguration; + public Configuration getConfiguration() { + return configuration; } /** - * Adds the given key-value configuration to the underlying configuration. It overwrites - * existing keys. + * Adds the given key-value configuration to the underlying application-specific configuration. + * It overwrites existing keys. * * @param configuration key-value configuration to be added */ @@ -442,6 +489,17 @@ public void addJobParameter(String key, String value) { set(PipelineOptions.GLOBAL_JOB_PARAMETERS, params); } + /** + * Sets the given configuration as {@link #rootConfiguration}, which contains any configuration + * set in the execution context. See the docs of {@link TableConfig} for more information. + * + * @param rootConfiguration root configuration to be set + */ + @Internal + public void setRootConfiguration(ReadableConfig rootConfiguration) { + this.rootConfiguration = rootConfiguration; + } + public static TableConfig getDefault() { return new TableConfig(); } From 3b7eac6ca52d4dc019ee83a41a42b71109406514 Mon Sep 17 00:00:00 2001 From: Roc Marshal Date: Tue, 15 Mar 2022 13:30:09 +0800 Subject: [PATCH 006/258] [BP-1.15][docs] Update wrong links in the datastream/execution_mode.md page. --- docs/content.zh/docs/dev/datastream/execution_mode.md | 4 ++-- docs/content.zh/docs/dev/datastream/operators/overview.md | 2 ++ docs/content/docs/dev/datastream/execution_mode.md | 5 ++--- 3 files changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/content.zh/docs/dev/datastream/execution_mode.md b/docs/content.zh/docs/dev/datastream/execution_mode.md index e0655b707f9c4..2cf146a0c537c 100644 --- a/docs/content.zh/docs/dev/datastream/execution_mode.md +++ b/docs/content.zh/docs/dev/datastream/execution_mode.md @@ -201,11 +201,11 @@ Checkpointing 用于故障恢复的特点之一是,在发生故障时,Flink `批`模式下的行为变化: -* “滚动"操作,如 [reduce()]({{< ref "docs/dev/datastream/operators/overview" >}}#reduce) 或 [sum()]({{< ref "docs/dev/datastream/operators/overview" >}}#aggregations),会对`流`模式下每一条新记录发出增量更新。在`批`模式下,这些操作不是"滚动”。它们只发出最终结果。 +* “滚动"操作,如 [reduce()]({{< ref "docs/dev/datastream/operators/overview" >}}#reduce) 或 sum(),会对`流`模式下每一条新记录发出增量更新。在`批`模式下,这些操作不是"滚动”。它们只发出最终结果。 `批`模式下不支持的: -* [Checkpointing]({{< ref "docs/concepts/stateful-stream-processing" >}}#stateful-stream-processing) 和任何依赖于 checkpointing 的操作都不支持。 +* [Checkpointing]({{< ref "docs/concepts/stateful-stream-processing" >}}#checkpointing) 和任何依赖于 checkpointing 的操作都不支持。 * [迭代(Iterations)]({{< ref "docs/dev/datastream/operators/overview" >}}#iterate) 自定义算子应谨慎执行,否则可能会有不恰当的行为。更多细节请参见下面的补充说明。 diff --git a/docs/content.zh/docs/dev/datastream/operators/overview.md b/docs/content.zh/docs/dev/datastream/operators/overview.md index 223839bae502e..3c622664fa8be 100644 --- a/docs/content.zh/docs/dev/datastream/operators/overview.md +++ b/docs/content.zh/docs/dev/datastream/operators/overview.md @@ -680,6 +680,8 @@ data_stream.broadcast() {{< /tab >}} {{< /tabs>}} + + ## 算子链和资源组 将两个算子链接在一起能使得它们在同一个线程中执行,从而提升性能。Flink 默认会将能链接的算子尽可能地进行链接(例如, 两个 map 转换操作)。此外, Flink 还提供了对链接更细粒度控制的 API 以满足更多需求: diff --git a/docs/content/docs/dev/datastream/execution_mode.md b/docs/content/docs/dev/datastream/execution_mode.md index 7837558876766..415a82d7f6af2 100644 --- a/docs/content/docs/dev/datastream/execution_mode.md +++ b/docs/content/docs/dev/datastream/execution_mode.md @@ -343,15 +343,14 @@ others are not supported. Behavior Change in BATCH mode: * "Rolling" operations such as [reduce()]({{< ref "docs/dev/datastream/operators/overview" >}}#reduce) - or [sum()]({{< ref "docs/dev/datastream/operators/overview" >}}#aggregations) - emit an incremental update for every new record that arrives in `STREAMING` + or sum() emit an incremental update for every new record that arrives in `STREAMING` mode. In `BATCH` mode, these operations are not "rolling". They emit only the final result. Unsupported in BATCH mode: -* [Checkpointing]({{< ref "docs/concepts/stateful-stream-processing" >}}#stateful-stream-processing) +* [Checkpointing]({{< ref "docs/concepts/stateful-stream-processing" >}}#checkpointing) and any operations that depend on checkpointing do not work. * [Iterations]({{< ref "docs/dev/datastream/operators/overview" >}}#iterate) From 5e160cc7b59f16484557a4747e6a75b373c56d22 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Tue, 15 Mar 2022 11:01:10 +0100 Subject: [PATCH 007/258] [FLINK-25771][cassandra][tests] Remove keyspace drop Since each test uses a separate table we no longer need to re-create the keyspace for each run, reducing the load on the cluster. --- .../cassandra/CassandraConnectorITCase.java | 19 +------------------ 1 file changed, 1 insertion(+), 18 deletions(-) diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java index 924c6846ace2f..de14e861ccb20 100644 --- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java +++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java @@ -62,7 +62,6 @@ import net.bytebuddy.ByteBuddy; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; -import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -165,9 +164,6 @@ protected Cluster buildCluster(Cluster.Builder builder) { "CREATE KEYSPACE " + KEYSPACE + " WITH replication= {'class':'SimpleStrategy', 'replication_factor':1};"; - private static final String DROP_KEYSPACE_QUERY = "DROP KEYSPACE IF EXISTS " + KEYSPACE + " ;"; - private static final String DROP_TABLE_QUERY = - "DROP TABLE IF EXISTS " + KEYSPACE + "." + TABLE_NAME_VARIABLE + " ;"; private static final String CREATE_TABLE_QUERY = "CREATE TABLE " + KEYSPACE @@ -386,10 +382,7 @@ public static void startAndInitializeCassandra() { } } } - session.execute(DROP_KEYSPACE_QUERY); session.execute(CREATE_KEYSPACE_QUERY); - session.execute( - CREATE_TABLE_QUERY.replace(TABLE_NAME_VARIABLE, TABLE_NAME_PREFIX + "initial")); } @Before @@ -398,14 +391,6 @@ public void createTable() { session.execute(injectTableName(CREATE_TABLE_QUERY)); } - @After - public void dropTables() { - // need to drop tables in case of retrials. Need to drop all the tables - // that are created in test because this method is executed with every test - session.execute(DROP_KEYSPACE_QUERY); - session.execute(CREATE_KEYSPACE_QUERY); - } - @AfterClass public static void closeCassandra() { if (session != null) { @@ -722,9 +707,7 @@ builderForWriting, injectTableName(INSERT_DATA_QUERY)) private static int retrialsCount = 0; @Test - public void testRetrialAndDropTables() { - // should not fail with table exists upon retrial - // as @After method that truncate the keyspace is called upon retrials. + public void testRetrial() { annotatePojoWithTable(KEYSPACE, TABLE_NAME_PREFIX + tableID); if (retrialsCount < 2) { retrialsCount++; From 75fe877872b48febc139e972e657f31b7ed3d7dc Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Mon, 7 Mar 2022 15:10:52 +0100 Subject: [PATCH 008/258] [FLINK-26551][table-planner] Change legacy casting config option to DISABLED by default This closes #19020. --- docs/content/docs/dev/table/types.md | 4 +-- .../execution_config_configuration.html | 2 +- .../connectors/hive/HiveRunnerITCase.java | 6 ++--- .../jdbc/catalog/PostgresCatalogITCase.java | 2 +- .../kafka/table/KafkaTableITCase.java | 25 ++++++++----------- .../AdvancedFunctionsExampleITCase.java | 18 ++++++------- .../api/config/ExecutionConfigOptions.java | 2 +- .../jsonplan/CorrelateJsonPlanITCase.java | 4 ++- .../runtime/batch/sql/CalcITCase.scala | 4 +-- 9 files changed, 33 insertions(+), 34 deletions(-) diff --git a/docs/content/docs/dev/table/types.md b/docs/content/docs/dev/table/types.md index 27d4c2860d3f5..9d3fb63c9e65d 100644 --- a/docs/content/docs/dev/table/types.md +++ b/docs/content/docs/dev/table/types.md @@ -1553,7 +1553,7 @@ regardless of whether the function used is `CAST` or `TRY_CAST`. ### Legacy casting Pre Flink 1.15 casting behaviour can be enabled by setting `table.exec.legacy-cast-behaviour` to `enabled`. -In Flink 1.15 this flag is enabled by default. +In Flink 1.15 this flag is disabled by default. In particular, this will: @@ -1562,7 +1562,7 @@ In particular, this will: * Formatting of some casting to `CHAR`/`VARCHAR`/`STRING` produces slightly different results. {{< hint warning >}} -We **discourage** the use of this flag and we **strongly suggest** for new projects to disable this flag and use the new casting behaviour. +We **discourage** the use of this flag and we **strongly suggest** for new projects to keep this flag disabled and use the new casting behaviour. This flag will be removed in the next Flink versions. {{< /hint >}} diff --git a/docs/layouts/shortcodes/generated/execution_config_configuration.html b/docs/layouts/shortcodes/generated/execution_config_configuration.html index 74cf7f2e3daa5..364d9bf9d814c 100644 --- a/docs/layouts/shortcodes/generated/execution_config_configuration.html +++ b/docs/layouts/shortcodes/generated/execution_config_configuration.html @@ -42,7 +42,7 @@

table.exec.legacy-cast-behaviour

Batch Streaming - ENABLED + DISABLED

Enum

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

Possible values:
  • "ENABLED": CAST will operate following the legacy behaviour.
  • "DISABLED": CAST will operate following the new correct behaviour.
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java index fbf49e139cd17..2b2d79327f5e1 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java @@ -256,9 +256,9 @@ public void testDecimal() throws Exception { TableEnvironment tableEnv = getTableEnvWithHiveCatalog(); tableEnv.executeSql("create database db1"); try { - tableEnv.executeSql("create table db1.src1 (x decimal(10,2))"); - tableEnv.executeSql("create table db1.src2 (x decimal(10,2))"); - tableEnv.executeSql("create table db1.dest (x decimal(10,2))"); + tableEnv.executeSql("create table db1.src1 (x decimal(12,2))"); + tableEnv.executeSql("create table db1.src2 (x decimal(12,2))"); + tableEnv.executeSql("create table db1.dest (x decimal(12,2))"); // populate src1 from Hive // TABLE keyword in INSERT INTO is mandatory prior to 1.1.0 hiveShell.execute( diff --git a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java index e6ec9cc2f5e7d..936230b0274e9 100644 --- a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java +++ b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java @@ -114,7 +114,7 @@ public void testGroupByInsert() throws Exception { tEnv.executeSql( String.format( "insert into `%s` " - + "select `int`, cast('A' as bytes), `short`, max(`long`), max(`real`), " + + "select `int`, cast('41' as bytes), `short`, max(`long`), max(`real`), " + "max(`double_precision`), max(`numeric`), max(`decimal`), max(`boolean`), " + "max(`text`), 'B', 'C', max(`character_varying`), max(`timestamp`), " + "max(`date`), max(`time`), max(`default_numeric`) " diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java index a6dab6da98843..c26845315e669 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java @@ -25,6 +25,7 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; import org.apache.flink.table.api.TableResult; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.utils.EncodingUtils; import org.apache.flink.test.util.SuccessException; import org.apache.flink.types.Row; @@ -44,7 +45,6 @@ import java.util.Collection; import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Random; @@ -58,6 +58,8 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.readLines; import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_SOURCE_IDLE_TIMEOUT; import static org.apache.flink.table.utils.TableTestMatchers.deepEqualTo; +import static org.apache.flink.util.CollectionUtil.entry; +import static org.apache.flink.util.CollectionUtil.map; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -299,29 +301,20 @@ public void testKafkaSourceSinkWithMetadata() throws Exception { + " %s\n" + ")", topic, bootstraps, groupId, formatOptions()); - tEnv.executeSql(createTable); String initialValues = "INSERT INTO kafka\n" + "VALUES\n" - + " ('data 1', 1, TIMESTAMP '2020-03-08 13:12:11.123', MAP['k1', X'C0FFEE', 'k2', X'BABE'], TRUE),\n" + + " ('data 1', 1, TIMESTAMP '2020-03-08 13:12:11.123', MAP['k1', X'C0FFEE', 'k2', X'BABE01'], TRUE),\n" + " ('data 2', 2, TIMESTAMP '2020-03-09 13:12:11.123', CAST(NULL AS MAP), FALSE),\n" - + " ('data 3', 3, TIMESTAMP '2020-03-10 13:12:11.123', MAP['k1', X'10', 'k2', X'20'], TRUE)"; + + " ('data 3', 3, TIMESTAMP '2020-03-10 13:12:11.123', MAP['k1', X'102030', 'k2', X'203040'], TRUE)"; tEnv.executeSql(initialValues).await(); // ---------- Consume stream from Kafka ------------------- final List result = collectRows(tEnv.sqlQuery("SELECT * FROM kafka"), 3); - final Map headers1 = new HashMap<>(); - headers1.put("k1", new byte[] {(byte) 0xC0, (byte) 0xFF, (byte) 0xEE}); - headers1.put("k2", new byte[] {(byte) 0xBA, (byte) 0xBE}); - - final Map headers3 = new HashMap<>(); - headers3.put("k1", new byte[] {(byte) 0x10}); - headers3.put("k2", new byte[] {(byte) 0x20}); - final List expected = Arrays.asList( Row.of( @@ -330,7 +323,9 @@ public void testKafkaSourceSinkWithMetadata() throws Exception { "CreateTime", LocalDateTime.parse("2020-03-08T13:12:11.123"), 0, - headers1, + map( + entry("k1", EncodingUtils.decodeHex("C0FFEE")), + entry("k2", EncodingUtils.decodeHex("BABE01"))), 0, topic, true), @@ -350,7 +345,9 @@ public void testKafkaSourceSinkWithMetadata() throws Exception { "CreateTime", LocalDateTime.parse("2020-03-10T13:12:11.123"), 0, - headers3, + map( + entry("k1", EncodingUtils.decodeHex("102030")), + entry("k2", EncodingUtils.decodeHex("203040"))), 0, topic, true)); diff --git a/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java b/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java index 4bc98c2e11b20..3306911da9ead 100644 --- a/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java +++ b/flink-examples/flink-examples-table/src/test/java/org/apache/flink/table/examples/java/functions/AdvancedFunctionsExampleITCase.java @@ -41,41 +41,41 @@ private void testExecuteLastDatedValueFunction(String consoleOutput) { assertThat( consoleOutput, containsString( - "| Guillermo Smith | (5,2020-12-05) |")); + "| Guillermo Smith | (5, 2020-12-05) |")); assertThat( consoleOutput, containsString( - "| John Turner | (12,2020-10-02) |")); + "| John Turner | (12, 2020-10-02) |")); assertThat( consoleOutput, containsString( - "| Brandy Sanders | (1,2020-10-14) |")); + "| Brandy Sanders | (1, 2020-10-14) |")); assertThat( consoleOutput, containsString( - "| Valeria Mendoza | (10,2020-06-02) |")); + "| Valeria Mendoza | (10, 2020-06-02) |")); assertThat( consoleOutput, containsString( - "| Ellen Ortega | (100,2020-06-18) |")); + "| Ellen Ortega | (100, 2020-06-18) |")); assertThat( consoleOutput, containsString( - "| Leann Holloway | (9,2020-05-26) |")); + "| Leann Holloway | (9, 2020-05-26) |")); } private void testExecuteInternalRowMergerFunction(String consoleOutput) { assertThat( consoleOutput, containsString( - "| Guillermo Smith | (1992-12-12,New Jersey,816-... |")); + "| Guillermo Smith | (1992-12-12, New Jersey, 81... |")); assertThat( consoleOutput, containsString( - "| Valeria Mendoza | (1970-03-28,Los Angeles,928... |")); + "| Valeria Mendoza | (1970-03-28, Los Angeles, 9... |")); assertThat( consoleOutput, containsString( - "| Leann Holloway | (1989-05-21,Eugene,614-889-... |")); + "| Leann Holloway | (1989-05-21, Eugene, 614-88... |")); } } diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java index 9a92e48b2054c..9aa8642dee4ab 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/config/ExecutionConfigOptions.java @@ -420,7 +420,7 @@ public class ExecutionConfigOptions { public static final ConfigOption TABLE_EXEC_LEGACY_CAST_BEHAVIOUR = key("table.exec.legacy-cast-behaviour") .enumType(LegacyCastBehaviour.class) - .defaultValue(LegacyCastBehaviour.ENABLED) + .defaultValue(LegacyCastBehaviour.DISABLED) .withDescription( "Determines whether CAST will operate following the legacy behaviour " + "or the new one that introduces various fixes and improvements."); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java index 18ae4e98ebba1..a12a14f402a2b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/jsonplan/CorrelateJsonPlanITCase.java @@ -99,7 +99,9 @@ public void testFilter() throws ExecutionException, InterruptedException, IOExce "STRING_SPLIT", new JavaUserDefinedTableFunctions.StringSplit()); createTestValuesSinkTable("MySink", "a STRING", "b STRING"); String query = - "insert into MySink SELECT a, v FROM MyTable, lateral table(STRING_SPLIT(a, ',')) as T(v) where cast(v as int) > 0"; + "insert into MySink " + + "SELECT a, v FROM MyTable, lateral table(STRING_SPLIT(a, ',')) as T(v) " + + "where try_cast(v as int) > 0"; compileSqlAndExecutePlan(query).await(); List expected = Arrays.asList("+I[1,1,hi, 1]", "+I[1,1,hi, 1]"); assertResult(expected, TestValuesTableFactory.getResults("MySink")); diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala index e591f1076fea0..09699c532b411 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala @@ -1640,10 +1640,10 @@ class CalcITCase extends BatchTestBase { tEnv.executeSql(ddl) checkResult( - "select a from MyTable where cast(b as boolean)", + "select a from MyTable where try_cast(b as boolean)", Seq(row(1))) checkResult( - "select cast(b as boolean) from MyTable", + "select try_cast(b as boolean) from MyTable", Seq(row(true), row(false), row(null), row(null))) } From 0562fc3df7492e21e577c169f4a65c2a1dce693a Mon Sep 17 00:00:00 2001 From: Yun Tang Date: Tue, 15 Mar 2022 17:43:10 +0800 Subject: [PATCH 009/258] [FLINK-26650][checkpoint] Avoid to print stack trace for checkpoint trigger failure if not all tasks are started --- .../checkpoint/CheckpointFailureManager.java | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java index 5b73fe29d8aad..08cf49e41ee34 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointFailureManager.java @@ -104,12 +104,20 @@ public void handleCheckpointException( : pendingCheckpoint.getCheckpointID(); updateStatsAfterCheckpointFailed(pendingCheckpointStats, statsTracker, exception); - LOG.warn( - "Failed to trigger or complete checkpoint {} for job {}. ({} consecutive failed attempts so far)", - checkpointId == UNKNOWN_CHECKPOINT_ID ? "UNKNOWN_CHECKPOINT_ID" : checkpointId, - job, - continuousFailureCounter.get(), - exception); + if (CheckpointFailureReason.NOT_ALL_REQUIRED_TASKS_RUNNING.equals( + exception.getCheckpointFailureReason())) { + LOG.info( + "Failed to trigger checkpoint for job {} since {}.", + job, + exception.getMessage()); + } else { + LOG.warn( + "Failed to trigger or complete checkpoint {} for job {}. ({} consecutive failed attempts so far)", + checkpointId == UNKNOWN_CHECKPOINT_ID ? "UNKNOWN_CHECKPOINT_ID" : checkpointId, + job, + continuousFailureCounter.get(), + exception); + } if (isJobManagerFailure(exception, executionAttemptID)) { handleJobLevelCheckpointException(checkpointProperties, exception, checkpointId); } else { From e31c1b0f88cbcd3f3a720948123b262fc21e4cd5 Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Mon, 14 Mar 2022 15:38:45 +0100 Subject: [PATCH 010/258] [FLINK-26063][runtime] Set current key before polling next PQ element Some InternalPriorityQueue implementations need a correct key/group set before performing poll() or remove(). In particular, ChangelogKeyGroupedPriorityQueue logs key group so that state changes can be re-distributed or shuffled. This change re-orders queue.poll and keyContext.setCurrentKey. --- .../runtime/state/InternalPriorityQueue.java | 6 + .../operators/InternalTimerServiceImpl.java | 4 +- .../test/state/ChangelogRescalingITCase.java | 354 ++++++++++++++++++ 3 files changed, 362 insertions(+), 2 deletions(-) create mode 100644 flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/InternalPriorityQueue.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/InternalPriorityQueue.java index 25816eb49b4d8..17bdd6ad35fe6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/InternalPriorityQueue.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/InternalPriorityQueue.java @@ -39,6 +39,9 @@ public interface InternalPriorityQueue { * Retrieves and removes the first element (w.r.t. the order) of this set, or returns {@code * null} if this set is empty. * + *

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

NOTE: Correct key (i.e. the key of the polled element) must be set on KeyContext before + * calling this method. + * * @param toRemove the element to remove. * @return true if the operation changed the head element or if it is unclear if * the head element changed. Only returns false if the head element was not diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java index a9977d79bd80d..b2cf4c6b94aee 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceImpl.java @@ -279,8 +279,8 @@ private void onProcessingTime(long time) throws Exception { InternalTimer timer; while ((timer = processingTimeTimersQueue.peek()) != null && timer.getTimestamp() <= time) { - processingTimeTimersQueue.poll(); keyContext.setCurrentKey(timer.getKey()); + processingTimeTimersQueue.poll(); triggerTarget.onProcessingTime(timer); } @@ -297,8 +297,8 @@ public void advanceWatermark(long time) throws Exception { InternalTimer timer; while ((timer = eventTimeTimersQueue.peek()) != null && timer.getTimestamp() <= time) { - eventTimeTimersQueue.poll(); keyContext.setCurrentKey(timer.getKey()); + eventTimeTimersQueue.poll(); triggerTarget.onEventTime(timer); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java new file mode 100644 index 0000000000000..85a30c8d76616 --- /dev/null +++ b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java @@ -0,0 +1,354 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.test.state; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.lib.NumberSequenceSource; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; +import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.datastream.DataStreamUtils; +import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.SlidingProcessingTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.Collector; +import org.apache.flink.util.TestLogger; + +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.time.Duration; +import java.util.Iterator; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; + +import static org.apache.flink.changelog.fs.FsStateChangelogOptions.PREEMPTIVE_PERSIST_THRESHOLD; +import static org.apache.flink.configuration.CheckpointingOptions.CHECKPOINTS_DIRECTORY; +import static org.apache.flink.configuration.CheckpointingOptions.CHECKPOINT_STORAGE; +import static org.apache.flink.configuration.CheckpointingOptions.FS_SMALL_FILE_THRESHOLD; +import static org.apache.flink.configuration.CheckpointingOptions.LOCAL_RECOVERY; +import static org.apache.flink.configuration.CoreOptions.DEFAULT_PARALLELISM; +import static org.apache.flink.configuration.PipelineOptions.OBJECT_REUSE; +import static org.apache.flink.configuration.RestartStrategyOptions.RESTART_STRATEGY; +import static org.apache.flink.configuration.StateBackendOptions.STATE_BACKEND; +import static org.apache.flink.configuration.StateChangelogOptions.ENABLE_STATE_CHANGE_LOG; +import static org.apache.flink.configuration.StateChangelogOptions.PERIODIC_MATERIALIZATION_INTERVAL; +import static org.apache.flink.configuration.TaskManagerOptions.BUFFER_DEBLOAT_ENABLED; +import static org.apache.flink.runtime.jobgraph.SavepointRestoreSettings.forPath; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; +import static org.apache.flink.streaming.api.environment.CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION; +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.ALIGNED_CHECKPOINT_TIMEOUT; +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL; +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_MODE; +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.ENABLE_UNALIGNED; +import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.EXTERNALIZED_CHECKPOINT; +import static org.apache.flink.test.util.TestUtils.getMostRecentCompletedCheckpoint; +import static org.apache.flink.test.util.TestUtils.getMostRecentCompletedCheckpointMaybe; +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Tests rescaling with Changelog enabled and with timers in state. It uses an adaptation of a + * ChangelogTestProgram that aims to generate the highest load possible while still allowing + * checkpointing. For that, it uses rate-limited FLIP-27 source and Unaligned checkpoints. + */ +@RunWith(Parameterized.class) +public class ChangelogRescalingITCase extends TestLogger { + /** The rate at which events will be generated by the source. */ + private static final int EVENTS_PER_SECOND_PER_READER = 100; + /** Payload size of each event generated randomly. */ + private static final int PAYLOAD_SIZE = 1000; + /** Size of (ProcessingTime) windows. */ + private static final Time WINDOW_SIZE = Time.milliseconds(100); + /** Slide size. */ + private static final Time WINDOW_SLIDE = Time.milliseconds(10); + /** Time to Accumulate some timer delete operations. */ + private static final int ACCUMULATE_TIME_MILLIS = 5_000; + + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @Parameters(name = "Rescale {0} -> {1}") + public static Object[] parameters() { + return new Object[][] {new Object[] {6, 4}, new Object[] {4, 6}}; + } + + private final int parallelism1; + private final int parallelism2; + + private MiniClusterWithClientResource cluster; + + public ChangelogRescalingITCase(int parallelism1, int parallelism2) { + this.parallelism1 = parallelism1; + this.parallelism2 = parallelism2; + } + + @Before + public void before() throws Exception { + Configuration configuration = new Configuration(); + FsStateChangelogStorageFactory.configure(configuration, temporaryFolder.newFolder()); + cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(configuration) + .setNumberSlotsPerTaskManager(Math.max(parallelism1, parallelism2)) + .build()); + cluster.before(); + } + + @After + public void after() { + if (cluster != null) { + cluster.after(); + cluster = null; + } + } + + @Test + public void test() throws Exception { + // before rescale + File cpDir1 = temporaryFolder.newFolder(); + JobID jobID1 = submit(configureJob(parallelism1, cpDir1), graph -> {}); + + Thread.sleep(ACCUMULATE_TIME_MILLIS); + File cpLocation = checkpointAndCancel(jobID1, cpDir1); + + // rescale and checkpoint to verify + JobID jobID2 = + submit( + configureJob(parallelism2, temporaryFolder.newFolder()), + graph -> + graph.setSavepointRestoreSettings( + forPath(cpLocation.toURI().toString()))); + waitForAllTaskRunning(cluster.getMiniCluster(), jobID2, true); + cluster.getClusterClient().cancel(jobID2).get(); + } + + private JobID submit(Configuration conf, Consumer updateGraph) + throws InterruptedException, ExecutionException { + JobGraph jobGraph = createJobGraph(conf); + updateGraph.accept(jobGraph); + return cluster.getClusterClient().submitJob(jobGraph).get(); + } + + private JobGraph createJobGraph(Configuration conf) { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + SingleOutputStreamOperator map = + env.fromSource( + new ThrottlingNumberSequenceSource( + 0, Long.MAX_VALUE, EVENTS_PER_SECOND_PER_READER), + WatermarkStrategy.noWatermarks(), + "Sequence Source") + .keyBy(ChangelogRescalingITCase::key) + .map( + el -> { + // Thread.sleep(100); // don't block barriers + byte[] bytes = new byte[PAYLOAD_SIZE]; + ThreadLocalRandom.current().nextBytes(bytes); + return new TestEvent(el, bytes); + }); + DataStreamUtils.reinterpretAsKeyedStream(map, e -> key(e.id)) + .window(SlidingProcessingTimeWindows.of(WINDOW_SIZE, WINDOW_SLIDE)) + .process( + new ProcessWindowFunction() { + @Override + public void process( + Long key, + ProcessWindowFunction + .Context + context, + Iterable elements, + Collector out) {} + }) + .addSink(new DiscardingSink<>()); + + return env.getStreamGraph().getJobGraph(); + } + + private static long key(Long num) { + return num % 1000; + } + + private Configuration configureJob(int parallelism, File cpDir) { + Configuration conf = new Configuration(); + + conf.set(EXTERNALIZED_CHECKPOINT, RETAIN_ON_CANCELLATION); + conf.set(DEFAULT_PARALLELISM, parallelism); + conf.set(ENABLE_STATE_CHANGE_LOG, true); + conf.set(CHECKPOINTING_MODE, CheckpointingMode.EXACTLY_ONCE); + conf.set(CHECKPOINTING_INTERVAL, Duration.ofMillis(10)); + conf.set(CHECKPOINT_STORAGE, "filesystem"); + conf.set(CHECKPOINTS_DIRECTORY, cpDir.toURI().toString()); + conf.set(STATE_BACKEND, "hashmap"); + conf.set(LOCAL_RECOVERY, false); // not supported by changelog + // tune changelog + conf.set(PREEMPTIVE_PERSIST_THRESHOLD, MemorySize.ofMebiBytes(10)); + conf.set(PERIODIC_MATERIALIZATION_INTERVAL, Duration.ofMinutes(3)); + // tune flink + conf.set(FS_SMALL_FILE_THRESHOLD, MemorySize.ofMebiBytes(1)); + conf.set(OBJECT_REUSE, true); + + conf.set(ENABLE_UNALIGNED, true); // speedup + conf.set(ALIGNED_CHECKPOINT_TIMEOUT, Duration.ZERO); // prevent randomization + conf.set(BUFFER_DEBLOAT_ENABLED, false); // prevent randomization + conf.set(RESTART_STRATEGY, "none"); // not expecting any failures + + return conf; + } + + private static final class TestEvent implements Serializable { + private final long id; + + @SuppressWarnings({"FieldCanBeLocal", "unused"}) + private final byte[] payload; + + private TestEvent(long id, byte[] payload) { + this.id = id; + this.payload = payload; + } + } + + private static class ThrottlingNumberSequenceSource extends NumberSequenceSource { + private final int numbersPerSecond; + + public ThrottlingNumberSequenceSource(long from, long to, int numbersPerSecondPerReader) { + super(from, to); + this.numbersPerSecond = numbersPerSecondPerReader; + } + + @Override + public SourceReader createReader( + SourceReaderContext readerContext) { + return new ThrottlingIteratorSourceReader<>( + readerContext, new SourceRateLimiter(numbersPerSecond)); + } + } + + private static class ThrottlingIteratorSourceReader< + E, IterT extends Iterator, SplitT extends IteratorSourceSplit> + extends IteratorSourceReader { + private final SourceRateLimiter rateLimiter; + + public ThrottlingIteratorSourceReader( + SourceReaderContext context, SourceRateLimiter rateLimiter) { + super(context); + this.rateLimiter = rateLimiter; + } + + @Override + public InputStatus pollNext(ReaderOutput output) { + if (rateLimiter.request()) { + return super.pollNext(output); + } else { + return InputStatus.NOTHING_AVAILABLE; + } + } + } + + private static final class SourceRateLimiter { + private final AtomicBoolean newTokensAdded = new AtomicBoolean(false); + private final int tokensToAdd; + private int tokensAvailable; + + public SourceRateLimiter(int tokensPerSecond) { + this( + tokensPerSecond < 10 ? 1000 : 100, + tokensPerSecond < 10 ? tokensPerSecond : tokensPerSecond / 10); + } + + public SourceRateLimiter(int intervalMs, int tokensToAdd) { + checkArgument(intervalMs > 0); + checkArgument(tokensToAdd > 0); + this.tokensToAdd = tokensToAdd; + this.tokensAvailable = tokensToAdd; + new Timer("source-limiter", true) + .scheduleAtFixedRate( + new TimerTask() { + @Override + public void run() { + newTokensAdded.set(true); // "catch up" is ok + } + }, + intervalMs, + intervalMs); + } + + public boolean request() { + if (tokensAvailable == 0 && newTokensAdded.compareAndSet(true, false)) { + tokensAvailable = tokensToAdd; + } + if (tokensAvailable > 0) { + tokensAvailable--; + return true; + } else { + return false; + } + } + } + + private File checkpointAndCancel(JobID jobID, File cpDir) + throws IOException, InterruptedException, ExecutionException { + while (!getMostRecentCompletedCheckpointMaybe(cpDir).isPresent()) { + checkStatus(jobID); + Thread.sleep(50); + } + cluster.getClusterClient().cancel(jobID).get(); + checkStatus(jobID); + return getMostRecentCompletedCheckpoint(cpDir); + } + + private void checkStatus(JobID jobID) throws InterruptedException, ExecutionException { + if (cluster.getClusterClient().getJobStatus(jobID).get().isGloballyTerminalState()) { + cluster.getClusterClient() + .requestJobResult(jobID) + .get() + .getSerializedThrowable() + .ifPresent( + serializedThrowable -> { + throw new RuntimeException(serializedThrowable); + }); + } + } +} From 828456bd77a14bed34f54b3e523f2bc56cd4d7a4 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Fri, 4 Mar 2022 16:14:29 +0100 Subject: [PATCH 011/258] [FLINK-26520][table] Implement SEARCH operator in codegen Implements the SEARCH operator in the codegen and removes the scalar implementation of IN and NOT_IN. Now every scalar IN/NOT_IN using a constant set is implemented through SEARCH (following Calcite's development on the topic CALCITE-4173) and plans will only have SEARCH. This closes #19001. --- .../functions/casting/CastRuleProvider.java | 23 ++ .../planner/plan/utils/RexLiteralUtil.java | 195 ++++++++++++++ .../table/planner/codegen/CodeGenUtils.scala | 18 +- .../codegen/CodeGeneratorContext.scala | 33 ++- .../planner/codegen/ExprCodeGenerator.scala | 57 ++--- .../table/planner/codegen/GenerateUtils.scala | 172 ++----------- .../codegen/LookupJoinCodeGenerator.scala | 7 +- .../planner/codegen/calls/LikeCallGen.scala | 8 +- .../codegen/calls/ScalarOperatorGens.scala | 76 ------ .../codegen/calls/SearchOperatorGen.scala | 241 ++++++++++++++++++ .../logical/ConvertToNotInOrInRule.scala | 6 +- .../planner/plan/utils/FlinkRexUtil.scala | 14 +- .../table/planner/plan/batch/sql/CalcTest.xml | 18 ++ .../logical/ConvertToNotInOrInRuleTest.xml | 20 +- .../planner/plan/batch/sql/CalcTest.scala | 6 + .../runtime/batch/sql/CalcITCase.scala | 11 + 16 files changed, 599 insertions(+), 306 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/RexLiteralUtil.java create mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/SearchOperatorGen.scala diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java index 9ef2abd127b69..c2681f8dab6cb 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleProvider.java @@ -165,6 +165,29 @@ public static boolean canFail(LogicalType inputType, LogicalType targetType) { context, inputTerm, inputIsNullTerm, inputLogicalType, targetLogicalType); } + /** + * Create a {@link CastExecutor} and execute the cast on the provided {@code value}. Fails with + * {@link IllegalArgumentException} if the rule cannot be resolved, or with an exception from + * the {@link CastExecutor} itself if the rule can fail. + */ + @SuppressWarnings("unchecked") + public static @Nullable Object cast( + CastRule.Context context, + LogicalType inputLogicalType, + LogicalType targetLogicalType, + Object value) { + CastExecutor castExecutor = + (CastExecutor) + CastRuleProvider.create(context, inputLogicalType, targetLogicalType); + + if (castExecutor == null) { + throw new NullPointerException( + "Unsupported casting from " + inputLogicalType + " to " + targetLogicalType); + } + + return castExecutor.cast(value); + } + /** * This method wraps {@link #generateCodeBlock(CodeGeneratorCastRule.Context, String, String, * LogicalType, LogicalType)}, but adding the assumption that the inputTerm is always non-null. diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/RexLiteralUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/RexLiteralUtil.java new file mode 100644 index 0000000000000..b981c4f2e5e08 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/RexLiteralUtil.java @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.utils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.binary.BinaryStringData; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.codegen.CodeGenException; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; + +import org.apache.calcite.avatica.util.ByteString; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.util.DateString; +import org.apache.calcite.util.NlsString; +import org.apache.calcite.util.TimeString; +import org.apache.calcite.util.TimestampString; + +import java.math.BigDecimal; +import java.time.ZoneOffset; + +import static org.apache.flink.table.planner.utils.TimestampStringUtils.toLocalDateTime; + +/** Utilities to work with {@link RexLiteral}. */ +@Internal +public class RexLiteralUtil { + + private RexLiteralUtil() {} + + /** See {@link #toFlinkInternalValue(Comparable, LogicalType)}. */ + public static Tuple2 toFlinkInternalValue(RexLiteral literal) { + LogicalType targetType = FlinkTypeFactory.toLogicalType(literal.getType()); + return Tuple2.of( + toFlinkInternalValue(literal.getValueAs(Comparable.class), targetType), targetType); + } + + /** + * Convert a value from Calcite's {@link Comparable} data structures to Flink internal data + * structures and also tries to be a bit flexible by accepting usual Java types such as String + * and boxed numerics. + * + *

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

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

Check {@link RexLiteral#valueMatchesType(Comparable, SqlTypeName, boolean)} for details on + * the {@link Comparable} data structures and {@link org.apache.flink.table.data.RowData} for + * details on Flink's internal data structures. + * + * @param value the value in Calcite's {@link Comparable} data structures + * @param valueType the type of the value + * @return the value in Flink's internal data structures + * @throws IllegalArgumentException in case the class of value does not match the expectations + * of valueType + */ + public static Object toFlinkInternalValue(Comparable value, LogicalType valueType) { + if (value == null) { + return null; + } + switch (valueType.getTypeRoot()) { + case CHAR: + case VARCHAR: + if (value instanceof NlsString) { + return BinaryStringData.fromString(((NlsString) value).getValue()); + } + if (value instanceof String) { + return BinaryStringData.fromString((String) value); + } + break; + case BOOLEAN: + if (value instanceof Boolean) { + return value; + } + break; + case BINARY: + case VARBINARY: + if (value instanceof ByteString) { + return ((ByteString) value).getBytes(); + } + break; + case DECIMAL: + if (value instanceof BigDecimal) { + return DecimalData.fromBigDecimal( + (BigDecimal) value, + LogicalTypeChecks.getPrecision(valueType), + LogicalTypeChecks.getScale(valueType)); + } + break; + case TINYINT: + if (value instanceof Number) { + return ((Number) value).byteValue(); + } + break; + case SMALLINT: + if (value instanceof Number) { + return ((Number) value).shortValue(); + } + break; + case INTEGER: + case INTERVAL_YEAR_MONTH: + if (value instanceof Number) { + return ((Number) value).intValue(); + } + break; + case BIGINT: + case INTERVAL_DAY_TIME: + if (value instanceof Number) { + return ((Number) value).longValue(); + } + break; + case FLOAT: + if (value instanceof Number) { + return ((Number) value).floatValue(); + } + break; + case DOUBLE: + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } + break; + case DATE: + if (value instanceof DateString) { + return ((DateString) value).getDaysSinceEpoch(); + } + if (value instanceof Number) { + return ((Number) value).intValue(); + } + break; + case TIME_WITHOUT_TIME_ZONE: + if (value instanceof TimeString) { + return ((TimeString) value).getMillisOfDay(); + } + if (value instanceof Number) { + return ((Number) value).intValue(); + } + break; + case TIMESTAMP_WITHOUT_TIME_ZONE: + if (value instanceof TimestampString) { + return TimestampData.fromLocalDateTime( + toLocalDateTime((TimestampString) value)); + } + break; + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + if (value instanceof TimestampString) { + return TimestampData.fromInstant( + toLocalDateTime((TimestampString) value) + .atOffset(ZoneOffset.UTC) + .toInstant()); + } + break; + case DISTINCT_TYPE: + return toFlinkInternalValue(value, ((DistinctType) valueType).getSourceType()); + case SYMBOL: + if (value instanceof Enum) { + return value; + } + break; + case TIMESTAMP_WITH_TIME_ZONE: + case ARRAY: + case MULTISET: + case MAP: + case ROW: + case STRUCTURED_TYPE: + case NULL: + case UNRESOLVED: + throw new CodeGenException("Type not supported: " + valueType); + } + throw new IllegalStateException( + "Unexpected class " + value.getClass() + " for value of type " + valueType); + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala index a015377cdd154..683b23183f82a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGenUtils.scala @@ -209,13 +209,27 @@ object CodeGenUtils { case _: JShort => s"((short)$value)" case _: JInt => value.toString case _: JLong => value.toString + "L" - case _: JFloat => value.toString + "f" - case _: JDouble => value.toString + "d" + case _: JFloat => value match { + case JFloat.NEGATIVE_INFINITY => "java.lang.Float.NEGATIVE_INFINITY" + case JFloat.POSITIVE_INFINITY => "java.lang.Float.POSITIVE_INFINITY" + case _ => value.toString + "f" + } + case _: JDouble => value match { + case JDouble.NEGATIVE_INFINITY => "java.lang.Double.NEGATIVE_INFINITY" + case JDouble.POSITIVE_INFINITY => "java.lang.Double.POSITIVE_INFINITY" + case _ => value.toString + "d" + } case sd: StringData => qualifyMethod(BINARY_STRING_DATA_FROM_STRING) + "(\"" + EncodingUtils.escapeJava(sd.toString) + "\")" case td: TimestampData => s"$TIMESTAMP_DATA.fromEpochMillis(${td.getMillisecond}L, ${td.getNanoOfMillisecond})" + case decimalData: DecimalData => + s"""$DECIMAL_UTIL.castFrom( + |"${decimalData.toString}", + |${decimalData.precision()}, + |${decimalData.scale()})""" + .stripMargin case _ => throw new IllegalArgumentException("Illegal literal type: " + value.getClass) } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala index ab8ba8f3a7904..f55295808c44b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/CodeGeneratorContext.scala @@ -420,14 +420,26 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { } val addElementsCode = elements.map { element => - s""" - |${element.code} - |if (${element.nullTerm}) { - | $fieldTerm.addNull(); - |} else { - | $fieldTerm.add(${element.resultTerm}); - |} - |""".stripMargin + if (element.literalValue.isDefined) { + // Don't generate the null check in case the element is a literal expression + if (element.literalValue.get != null) { + s""" + |${element.code} + |$fieldTerm.add(${element.resultTerm}); + |""".stripMargin + } else if (element.literalValue.get == null) { + s"$fieldTerm.addNull();" + } + } else { + s""" + |${element.code} + |if (${element.nullTerm}) { + | $fieldTerm.addNull(); + |} else { + | $fieldTerm.add(${element.resultTerm}); + |} + |""".stripMargin + } }.mkString("\n") val setBuildingFunctionName = newName("buildSet") val setBuildingFunctionCode = @@ -886,8 +898,11 @@ class CodeGeneratorContext(val tableConfig: TableConfig) { /** * Adds a reusable string constant to the member area of the generated class. + * + * The string must be already escaped with + * [[org.apache.flink.table.utils.EncodingUtils.escapeJava()]]. */ - def addReusableStringConstants(value: String): String = { + def addReusableEscapedStringConstant(value: String): String = { reusableStringConstants.get(value) match { case Some(field) => field case None => diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala index aebaf6b181d6f..3c3f8581bf625 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExprCodeGenerator.scala @@ -18,28 +18,25 @@ package org.apache.flink.table.planner.codegen -import org.apache.calcite.rex._ -import org.apache.calcite.sql.`type`.{ReturnTypes, SqlTypeName} -import org.apache.calcite.sql.{SqlKind, SqlOperator} -import org.apache.calcite.util.{Sarg, TimestampString} import org.apache.flink.streaming.api.functions.ProcessFunction import org.apache.flink.table.api.TableException import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.data.RowData import org.apache.flink.table.data.binary.BinaryRowData -import org.apache.flink.table.data.util.DataFormatConverters.{DataFormatConverter, getConverterForDataType} +import org.apache.flink.table.data.util.DataFormatConverters.{getConverterForDataType, DataFormatConverter} import org.apache.flink.table.functions.BuiltInFunctionDefinitions -import org.apache.flink.table.planner.calcite.{FlinkRexBuilder, FlinkTypeFactory, RexDistinctKeyVariable, RexFieldVariable} +import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, RexDistinctKeyVariable, RexFieldVariable} import org.apache.flink.table.planner.codegen.CodeGenUtils._ import org.apache.flink.table.planner.codegen.GenerateUtils._ import org.apache.flink.table.planner.codegen.GeneratedExpression.{NEVER_NULL, NO_CODE} import org.apache.flink.table.planner.codegen.calls.ScalarOperatorGens._ +import org.apache.flink.table.planner.codegen.calls.SearchOperatorGen.generateSearch import org.apache.flink.table.planner.codegen.calls._ import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable._ import org.apache.flink.table.planner.functions.sql.SqlThrowExceptionFunction import org.apache.flink.table.planner.functions.utils.{ScalarSqlFunction, TableSqlFunction} -import org.apache.flink.table.planner.plan.utils.FlinkRexUtil +import org.apache.flink.table.planner.plan.utils.RexLiteralUtil import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType import org.apache.flink.table.runtime.types.PlannerTypeUtils.isInteroperable import org.apache.flink.table.runtime.typeutils.TypeCheckUtils @@ -48,6 +45,10 @@ import org.apache.flink.table.types.logical._ import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.{getFieldCount, isCompositeType} import org.apache.flink.table.typeutils.TimeIndicatorTypeInfo +import org.apache.calcite.rex._ +import org.apache.calcite.sql.`type`.{ReturnTypes, SqlTypeName} +import org.apache.calcite.sql.{SqlKind, SqlOperator} + import scala.collection.JavaConversions._ /** @@ -411,15 +412,8 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) } override def visitLiteral(literal: RexLiteral): GeneratedExpression = { - val resultType = FlinkTypeFactory.toLogicalType(literal.getType) - val value = resultType.getTypeRoot match { - case LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE | - LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE => - literal.getValueAs(classOf[TimestampString]) - case _ => - literal.getValue3 - } - generateLiteral(ctx, resultType, value) + val res = RexLiteralUtil.toFlinkInternalValue(literal) + generateLiteral(ctx, res.f0, res.f1) } override def visitCorrelVariable(correlVariable: RexCorrelVariable): GeneratedExpression = { @@ -471,19 +465,10 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) override def visitCall(call: RexCall): GeneratedExpression = { val resultType = FlinkTypeFactory.toLogicalType(call.getType) if (call.getKind == SqlKind.SEARCH) { - val sarg = call.getOperands.get(1).asInstanceOf[RexLiteral] - .getValueAs(classOf[Sarg[_]]) - val rexBuilder = new FlinkRexBuilder(FlinkTypeFactory.INSTANCE) - if (sarg.isPoints) { - val operands = FlinkRexUtil.expandSearchOperands(rexBuilder, call) - .map(operand => operand.accept(this)) - return generateCallExpression(ctx, call, operands, resultType) - } else { - return RexUtil.expandSearch( - rexBuilder, - null, - call).accept(this) - } + return generateSearch( + ctx, + generateExpression(call.getOperands.get(0)), + call.getOperands.get(1).asInstanceOf[RexLiteral]) } // convert operands and help giving untyped NULL literals a type @@ -691,16 +676,6 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) requireBoolean(operand) generateIsNotFalse(operand) - case SEARCH | IN => - val left = operands.head - val right = operands.tail - generateIn(ctx, left, right) - - case NOT_IN => - val left = operands.head - val right = operands.tail - generateNot(ctx, generateIn(ctx, left, right)) - // casting case CAST => generateCast(ctx, operands.head, resultType, nullOnFailure = ctx.tableConfig @@ -876,8 +851,8 @@ class ExprCodeGenerator(ctx: CodeGeneratorContext, nullableInput: Boolean) case None => null case Some(literal) => getConverterForDataType(fromLogicalTypeToDataType(expr.resultType)) - .asInstanceOf[DataFormatConverter[AnyRef, AnyRef] - ].toExternal(literal.asInstanceOf[AnyRef]) + .asInstanceOf[DataFormatConverter[AnyRef, AnyRef]] + .toExternal(literal.asInstanceOf[AnyRef]) } }.toArray } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala index be0f4532aff38..5c6018d8d8f79 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/GenerateUtils.scala @@ -21,7 +21,7 @@ package org.apache.flink.table.planner.codegen import org.apache.flink.api.common.ExecutionConfig import org.apache.flink.api.common.typeinfo.{AtomicType => AtomicTypeInfo} import org.apache.flink.table.data._ -import org.apache.flink.table.data.binary.BinaryRowData +import org.apache.flink.table.data.binary.{BinaryRowData, BinaryStringData} import org.apache.flink.table.data.utils.JoinedRowData import org.apache.flink.table.data.writer.BinaryRowWriter import org.apache.flink.table.planner.codegen.CodeGenUtils._ @@ -30,18 +30,11 @@ import org.apache.flink.table.planner.codegen.calls.CurrentTimePointCallGen import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec import org.apache.flink.table.planner.plan.utils.SortUtil import org.apache.flink.table.planner.typeutils.SymbolUtil.calciteToCommon -import org.apache.flink.table.planner.utils.TimestampStringUtils.toLocalDateTime import org.apache.flink.table.runtime.typeutils.TypeCheckUtils.{isCharacterString, isReference, isTemporal} -import org.apache.flink.table.types.logical.LogicalTypeRoot._ import org.apache.flink.table.types.logical._ +import org.apache.flink.table.types.logical.LogicalTypeRoot._ import org.apache.flink.table.types.logical.utils.LogicalTypeChecks.{getFieldCount, getFieldTypes} - -import org.apache.calcite.avatica.util.ByteString -import org.apache.calcite.util.TimestampString -import org.apache.commons.lang3.StringEscapeUtils - -import java.math.{BigDecimal => JBigDecimal} -import java.time.ZoneOffset +import org.apache.flink.table.utils.EncodingUtils import scala.annotation.tailrec import scala.collection.mutable @@ -313,164 +306,53 @@ object GenerateUtils { literalValue = Some(literalValue)) } + /** + * This function accepts the Flink's internal data structures. + * + * Check [[org.apache.flink.table.planner.plan.utils.RexLiteralUtil.toFlinkInternalValue]] to + * convert RexLiteral value to Flink's internal data structures. + */ @tailrec def generateLiteral( ctx: CodeGeneratorContext, - literalType: LogicalType, - literalValue: Any): GeneratedExpression = { + literalValue: Any, + literalType: LogicalType): GeneratedExpression = { if (literalValue == null) { return generateNullLiteral(literalType, ctx.nullCheck) } - // non-null values literalType.getTypeRoot match { - // ordered by type root definition + // For strings, binary and decimal, we add the literal as reusable field, + // as they're not cheap to construct. For the other types, the return term is directly + // the literal value case CHAR | VARCHAR => - val escapedValue = StringEscapeUtils.ESCAPE_JAVA.translate(literalValue.toString) - val field = ctx.addReusableStringConstants(escapedValue) - generateNonNullLiteral(literalType, field, StringData.fromString(escapedValue)) - - case BOOLEAN => - generateNonNullLiteral(literalType, literalValue.toString, literalValue) + val str = literalValue.asInstanceOf[BinaryStringData] + val field = ctx.addReusableEscapedStringConstant(EncodingUtils.escapeJava(str.toString)) + generateNonNullLiteral(literalType, field, str) case BINARY | VARBINARY => - val bytesVal = literalValue.asInstanceOf[ByteString].getBytes + val bytesVal = literalValue.asInstanceOf[Array[Byte]] val fieldTerm = ctx.addReusableObject( bytesVal, "binary", bytesVal.getClass.getCanonicalName) generateNonNullLiteral(literalType, fieldTerm, bytesVal) case DECIMAL => - val dt = literalType.asInstanceOf[DecimalType] - val precision = dt.getPrecision - val scale = dt.getScale val fieldTerm = newName("decimal") - val decimalClass = className[DecimalData] - val fieldDecimal = - s""" - |$decimalClass $fieldTerm = - | $DECIMAL_UTIL.castFrom("${literalValue.toString}", $precision, $scale); - |""".stripMargin - ctx.addReusableMember(fieldDecimal) - val value = DecimalData.fromBigDecimal( - literalValue.asInstanceOf[JBigDecimal], precision, scale) - if (value == null) { - generateNullLiteral(literalType, ctx.nullCheck) - } else { - generateNonNullLiteral(literalType, fieldTerm, value) - } - - case TINYINT => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - generateNonNullLiteral(literalType, decimal.byteValue().toString, decimal.byteValue()) - - case SMALLINT => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - generateNonNullLiteral(literalType, decimal.shortValue().toString, decimal.shortValue()) - - case INTEGER => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - generateNonNullLiteral(literalType, decimal.intValue().toString, decimal.intValue()) - - case BIGINT => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - generateNonNullLiteral( - literalType, decimal.longValue().toString + "L", decimal.longValue()) - - case FLOAT => - val floatValue = literalValue.asInstanceOf[JBigDecimal].floatValue() - floatValue match { - case Float.NegativeInfinity => - generateNonNullLiteral( - literalType, - "java.lang.Float.NEGATIVE_INFINITY", - Float.NegativeInfinity) - case Float.PositiveInfinity => generateNonNullLiteral( - literalType, - "java.lang.Float.POSITIVE_INFINITY", - Float.PositiveInfinity) - case _ => generateNonNullLiteral( - literalType, floatValue.toString + "f", floatValue) - } - - case DOUBLE => - val doubleValue = literalValue.asInstanceOf[JBigDecimal].doubleValue() - doubleValue match { - case Double.NegativeInfinity => - generateNonNullLiteral( - literalType, - "java.lang.Double.NEGATIVE_INFINITY", - Double.NegativeInfinity) - case Double.PositiveInfinity => - generateNonNullLiteral( - literalType, - "java.lang.Double.POSITIVE_INFINITY", - Double.PositiveInfinity) - case _ => generateNonNullLiteral( - literalType, doubleValue.toString + "d", doubleValue) - } - - case DATE => - generateNonNullLiteral(literalType, literalValue.toString, literalValue) - - case TIME_WITHOUT_TIME_ZONE => - generateNonNullLiteral(literalType, literalValue.toString, literalValue) - - case TIMESTAMP_WITHOUT_TIME_ZONE => - val fieldTerm = newName("timestamp") - val ldt = toLocalDateTime(literalValue.asInstanceOf[TimestampString]) - val ts = TimestampData.fromLocalDateTime(ldt) - val fieldTimestamp = - s""" - |$TIMESTAMP_DATA $fieldTerm = - | $TIMESTAMP_DATA.fromEpochMillis(${ts.getMillisecond}L, ${ts.getNanoOfMillisecond}); - """.stripMargin - ctx.addReusableMember(fieldTimestamp) - generateNonNullLiteral(literalType, fieldTerm, ts) - - case TIMESTAMP_WITH_TIME_ZONE => - throw new UnsupportedOperationException("Unsupported type: " + literalType) - - case TIMESTAMP_WITH_LOCAL_TIME_ZONE => - val fieldTerm = newName("timestampWithLocalZone") - val ins = - toLocalDateTime(literalValue.asInstanceOf[TimestampString]) - .atOffset(ZoneOffset.UTC) - .toInstant - val ts = TimestampData.fromInstant(ins) - val fieldTimestampWithLocalZone = - s""" - |$TIMESTAMP_DATA $fieldTerm = - | $TIMESTAMP_DATA.fromEpochMillis(${ts.getMillisecond}L, ${ts.getNanoOfMillisecond}); - """.stripMargin - ctx.addReusableMember(fieldTimestampWithLocalZone) + ctx.addReusableMember(s""" + |${className[DecimalData]} $fieldTerm = ${primitiveLiteralForType(literalValue)}; + |""".stripMargin) generateNonNullLiteral(literalType, fieldTerm, literalValue) - case INTERVAL_YEAR_MONTH => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - if (decimal.isValidInt) { - generateNonNullLiteral(literalType, decimal.intValue().toString, decimal.intValue()) - } else { - throw new CodeGenException( - s"Decimal '$decimal' can not be converted to interval of months.") - } - - case INTERVAL_DAY_TIME => - val decimal = BigDecimal(literalValue.asInstanceOf[JBigDecimal]) - if (decimal.isValidLong) { - generateNonNullLiteral( - literalType, - decimal.longValue().toString + "L", - decimal.longValue()) - } else { - throw new CodeGenException( - s"Decimal '$decimal' can not be converted to interval of milliseconds.") - } - case DISTINCT_TYPE => - generateLiteral(ctx, literalType.asInstanceOf[DistinctType].getSourceType, literalValue) + generateLiteral(ctx, literalValue, literalType.asInstanceOf[DistinctType].getSourceType) case SYMBOL => generateSymbol(literalValue.asInstanceOf[Enum[_]]) + case BOOLEAN | TINYINT | SMALLINT | INTEGER | BIGINT | FLOAT | DOUBLE | DATE | + TIME_WITHOUT_TIME_ZONE | TIMESTAMP_WITHOUT_TIME_ZONE | TIMESTAMP_WITH_LOCAL_TIME_ZONE | + INTERVAL_YEAR_MONTH | INTERVAL_DAY_TIME => + generateNonNullLiteral(literalType, primitiveLiteralForType(literalValue), literalValue) + case ARRAY | MULTISET | MAP | ROW | STRUCTURED_TYPE | NULL | UNRESOLVED => throw new CodeGenException(s"Type not supported: $literalType") } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala index ef7b5b097edcb..5268a4b65855f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala @@ -35,6 +35,7 @@ import org.apache.flink.table.planner.codegen.calls.BridgingFunctionGenUtil.veri import org.apache.flink.table.planner.delegation.PlannerBase import org.apache.flink.table.planner.functions.inference.LookupCallContext import org.apache.flink.table.planner.plan.utils.LookupJoinUtil.{ConstantLookupKey, FieldRefLookupKey, LookupKey} +import org.apache.flink.table.planner.plan.utils.RexLiteralUtil import org.apache.flink.table.planner.utils.JavaScalaConversionUtil.toScala import org.apache.flink.table.runtime.collector.{TableFunctionCollector, TableFunctionResultFuture} import org.apache.flink.table.runtime.generated.{GeneratedCollector, GeneratedFunction, GeneratedResultFuture} @@ -213,10 +214,8 @@ object LookupJoinCodeGenerator { .map(lookupKeys.get) .map { case constantKey: ConstantLookupKey => - generateLiteral( - ctx, - constantKey.sourceType, - constantKey.literal.getValue3) + val res = RexLiteralUtil.toFlinkInternalValue(constantKey.literal) + generateLiteral(ctx, res.f0, res.f1) case fieldKey: FieldRefLookupKey => generateInputAccess( ctx, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala index cd8035d22f57b..15bdacb8e02a9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/LikeCallGen.scala @@ -109,16 +109,16 @@ class LikeCallGen extends CallGenerator { val middleMatcher = MIDDLE_PATTERN.matcher(newPattern) if (noneMatcher.matches()) { - val reusePattern = ctx.addReusableStringConstants(newPattern) + val reusePattern = ctx.addReusableEscapedStringConstant(newPattern) s"${terms.head}.equals($reusePattern)" } else if (beginMatcher.matches()) { - val field = ctx.addReusableStringConstants(beginMatcher.group(1)) + val field = ctx.addReusableEscapedStringConstant(beginMatcher.group(1)) s"${terms.head}.startsWith($field)" } else if (endMatcher.matches()) { - val field = ctx.addReusableStringConstants(endMatcher.group(1)) + val field = ctx.addReusableEscapedStringConstant(endMatcher.group(1)) s"${terms.head}.endsWith($field)" } else if (middleMatcher.matches()) { - val field = ctx.addReusableStringConstants(middleMatcher.group(1)) + val field = ctx.addReusableEscapedStringConstant(middleMatcher.group(1)) s"${terms.head}.contains($field)" } else { val field = className[SqlLikeChainChecker] diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 7dcb13c7813b9..6288b966ad379 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -334,82 +334,6 @@ object ScalarOperatorGens { // scalar expression generate utils // ---------------------------------------------------------------------------------------- - /** - * Generates IN expression using a HashSet - */ - def generateIn( - ctx: CodeGeneratorContext, - needle: GeneratedExpression, - haystack: Seq[GeneratedExpression]) - : GeneratedExpression = { - - // add elements to hash set if they are constant - if (haystack.forall(_.literal)) { - - // determine common numeric type - val widerType = toScala(findCommonType(asList(needle.resultType, haystack.head.resultType))) - .orElse(throw new CodeGenException(s"Unable to find common type of $needle and $haystack.")) - - // we need to normalize the values for the hash set - val castNumeric = widerType match { - case Some(t) => (value: GeneratedExpression) => - numericCasting(ctx, value.resultType, t)(value.resultTerm) - case None => (value: GeneratedExpression) => value.resultTerm - } - - val resultType = widerType match { - case Some(t) => t - case None => needle.resultType - } - - val elements = haystack.map { element => - element.copy( - castNumeric(element), // cast element to wider type - element.nullTerm, - element.code, - resultType) - } - val setTerm = ctx.addReusableHashSet(elements, resultType) - - val castedNeedle = needle.copy( - castNumeric(needle), // cast needle to wider type - needle.nullTerm, - needle.code, - resultType) - - val Seq(resultTerm, nullTerm) = newNames("result", "isNull") - val resultTypeTerm = primitiveTypeTermForType(new BooleanType()) - val defaultValue = primitiveDefaultValue(new BooleanType()) - - val operatorCode = if (ctx.nullCheck) { - s""" - |${castedNeedle.code} - |$resultTypeTerm $resultTerm = $defaultValue; - |boolean $nullTerm = true; - |if (!${castedNeedle.nullTerm}) { - | $resultTerm = $setTerm.contains(${castedNeedle.resultTerm}); - | $nullTerm = !$resultTerm && $setTerm.containsNull(); - |} - |""".stripMargin.trim - } - else { - s""" - |${castedNeedle.code} - |$resultTypeTerm $resultTerm = $setTerm.contains(${castedNeedle.resultTerm}); - |""".stripMargin.trim - } - - GeneratedExpression(resultTerm, nullTerm, operatorCode, new BooleanType()) - } else { - // we use a chain of ORs for a set that contains non-constant elements - haystack - .map(generateEquals(ctx, needle, _)) - .reduce((left, right) => - generateOr(ctx, left, right) - ) - } - } - /** * check the validity of implicit type conversion * See: https://cwiki.apache.org/confluence/display/FLINK/FLIP-154%3A+SQL+Implicit+Type+Coercion diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/SearchOperatorGen.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/SearchOperatorGen.scala new file mode 100644 index 0000000000000..42c68d267e260 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/SearchOperatorGen.scala @@ -0,0 +1,241 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.codegen.calls + +import org.apache.flink.table.planner.calcite.FlinkTypeFactory +import org.apache.flink.table.planner.codegen.{CodeGeneratorContext, CodeGenException, GeneratedExpression} +import org.apache.flink.table.planner.codegen.CodeGenUtils.newNames +import org.apache.flink.table.planner.codegen.GenerateUtils.{generateLiteral, generateNullLiteral} +import org.apache.flink.table.planner.codegen.calls.ScalarOperatorGens._ +import org.apache.flink.table.planner.functions.casting.CastRuleProvider +import org.apache.flink.table.planner.plan.utils.RexLiteralUtil.toFlinkInternalValue +import org.apache.flink.table.types.logical.{BooleanType, LogicalType} +import org.apache.flink.table.types.logical.utils.LogicalTypeMerging.findCommonType + +import org.apache.calcite.rex.RexLiteral +import org.apache.calcite.util.{RangeSets, Sarg} + +import java.util.Arrays.asList + +import scala.collection.JavaConverters._ + +/** + * Class containing utilities to implement the SEARCH operator. + * + * This does not implement [[CallGenerator]] as the interface does not fit, because the [[Sarg]] + * argument cannot be converted directly to [[GeneratedExpression]]. + */ +object SearchOperatorGen { + + /** + * Generates SEARCH expression using either an HashSet or a concatenation of OR, + * depending on whether the elements of the haystack are all literals or not. + * + * Note that both IN/NOT IN are converted to SEARCH when the set has only constant values, + * otherwise the IN/NOT IN are converted to a set of disjunctions. See + * [[org.apache.calcite.rex.RexBuilder#makeIn(org.apache.calcite.rex.RexNode, java.util.List)]]. + */ + def generateSearch( + ctx: CodeGeneratorContext, + target: GeneratedExpression, + sargLiteral: RexLiteral): GeneratedExpression = { + val sarg: Sarg[Nothing] = sargLiteral.getValueAs(classOf[Sarg[Nothing]]) + val targetType = target.resultType + val sargType = FlinkTypeFactory.toLogicalType(sargLiteral.getType) + + val commonType: LogicalType = findCommonType(asList(targetType, sargType)) + .orElseThrow(() => + new CodeGenException(s"Unable to find common type of $target and $sargLiteral.")) + + val needle = generateCast( + ctx, + target, + commonType, + nullOnFailure = false + ) + + // In case the search is among points we use the hashset implementation + if (sarg.isPoints || sarg.isComplementedPoints) { + val rangeSet = if (sarg.isPoints) sarg.rangeSet else sarg.rangeSet.complement() + val haystack = rangeSet + .asRanges() + .asScala + // We need to go through the generateLiteral to normalize the value from calcite + .map(r => toFlinkInternalValue(r.lowerEndpoint, sargType)) + // The elements are constant, we perform the cast immediately + .map(CastRuleProvider.cast(toCastContext(ctx), sargType, commonType, _)) + .map(generateLiteral(ctx, _, commonType)) + if (sarg.containsNull) { + haystack += generateNullLiteral(commonType, ctx.nullCheck) + } + val setTerm = ctx.addReusableHashSet(haystack.toSeq, commonType) + val negation = if (sarg.isComplementedPoints) "!" else "" + + val Seq(resultTerm, nullTerm) = newNames("result", "isNull") + + val operatorCode = if (ctx.nullCheck) { + s""" + |${needle.code} + |// --- Begin SEARCH ${target.resultTerm} + |boolean $resultTerm = false; + |boolean $nullTerm = true; + |if (!${needle.nullTerm}) { + | $resultTerm = $negation$setTerm.contains(${needle.resultTerm}); + | $nullTerm = !$resultTerm && $setTerm.containsNull(); + |} + |// --- End SEARCH ${target.resultTerm} + |""".stripMargin.trim + } + else { + s""" + |${needle.code} + |// --- Begin SEARCH ${target.resultTerm} + |boolean $resultTerm = $negation$setTerm.contains(${needle.resultTerm}); + |// --- End SEARCH ${target.resultTerm} + |""".stripMargin.trim + } + + GeneratedExpression(resultTerm, nullTerm, operatorCode, new BooleanType()) + } else { + // We copy the target to don't re-evaluate on each range check + val dummyTarget = target.copy(code = "") + + val rangeToExpression = new RangeToExpression(ctx, sargType, dummyTarget) + + // We use a chain of ORs and range comparisons + var rangeChecks: Seq[GeneratedExpression] = sarg + .rangeSet + .asRanges + .asScala + .toSeq + .map(RangeSets.map(_, rangeToExpression)) + + if (sarg.containsNull) { + rangeChecks = Seq(generateIsNull(ctx, target)) ++ rangeChecks + } + + val generatedRangeChecks = rangeChecks + .reduce((left, right) => generateOr(ctx, left, right)) + + // Add the target expression code + val finalCode = + s""" + |${target.code} + |// --- Begin SEARCH ${target.resultTerm} + |${generatedRangeChecks.code} + |// --- End SEARCH ${target.resultTerm} + |""".stripMargin.trim + generatedRangeChecks.copy(code = finalCode) + } + } + + private class RangeToExpression[C <: Comparable[C]]( + ctx: CodeGeneratorContext, + boundType: LogicalType, + target: GeneratedExpression) extends RangeSets.Handler[C, GeneratedExpression] { + + override def all(): GeneratedExpression = { + generateLiteral(ctx, true, new BooleanType()) + } + + /** + * lower <= target + */ + override def atLeast(lower: C): GeneratedExpression = { + generateComparison(ctx, "<=", lit(lower), target) + } + + /** + * target <= upper + */ + override def atMost(upper: C): GeneratedExpression = { + generateComparison(ctx, "<=", target, lit(upper)) + } + + /** + * lower < target + */ + override def greaterThan(lower: C): GeneratedExpression = { + generateComparison(ctx, "<", lit(lower), target) + } + + /** + * target < upper + */ + override def lessThan(upper: C): GeneratedExpression = { + generateComparison(ctx, "<", target, lit(upper)) + } + + /** + * value == target + */ + override def singleton(value: C): GeneratedExpression = { + generateComparison(ctx, "==", lit(value), target) + } + + /** + * lower <= target && target <= upper + */ + override def closed(lower: C, upper: C): GeneratedExpression = { + generateAnd( + ctx, + generateComparison(ctx, "<=", lit(lower), target), + generateComparison(ctx, "<=", target, lit(upper)) + ) + } + + /** + * lower <= target && target < upper + */ + override def closedOpen(lower: C, upper: C): GeneratedExpression = { + generateAnd( + ctx, + generateComparison(ctx, "<=", lit(lower), target), + generateComparison(ctx, "<", target, lit(upper)) + ) + } + + /** + * lower < target && target <= upper + */ + override def openClosed(lower: C, upper: C): GeneratedExpression = { + generateAnd( + ctx, + generateComparison(ctx, "<", lit(lower), target), + generateComparison(ctx, "<=", target, lit(upper)) + ) + } + + /** + * lower < target && target < upper + */ + override def open(lower: C, upper: C): GeneratedExpression = { + generateAnd( + ctx, + generateComparison(ctx, "<", lit(lower), target), + generateComparison(ctx, "<", target, lit(upper)) + ) + } + + private def lit(value: C): GeneratedExpression = { + generateLiteral(ctx, toFlinkInternalValue(value, boundType), boundType) + } + } + +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRule.scala index c47443430d452..96980f6142c35 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRule.scala @@ -26,7 +26,7 @@ import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelOptUtil} import org.apache.calcite.rel.core.Filter import org.apache.calcite.rex.{RexCall, RexLiteral, RexNode} import org.apache.calcite.sql.SqlBinaryOperator -import org.apache.calcite.sql.fun.SqlStdOperatorTable.{AND, EQUALS, IN, NOT_EQUALS, NOT_IN, OR} +import org.apache.calcite.sql.fun.SqlStdOperatorTable.{AND, EQUALS, IN, NOT, NOT_EQUALS, NOT_IN, OR} import org.apache.calcite.tools.RelBuilder import scala.collection.JavaConversions._ @@ -154,7 +154,9 @@ class ConvertToNotInOrInRule val values = list.map(_.getOperands.last) val call = toOperator match { case IN => builder.getRexBuilder.makeIn(inputRef, values) - case _ => builder.getRexBuilder.makeCall(toOperator, List(inputRef) ++ values) + case NOT_IN => builder + .getRexBuilder + .makeCall(NOT, builder.getRexBuilder.makeIn(inputRef, values)) } rexBuffer += call beenConverted = true diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala index 6f78ac30d4fc8..6f40ad25b7e6e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRexUtil.scala @@ -20,7 +20,6 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.annotation.Experimental import org.apache.flink.configuration.ConfigOption import org.apache.flink.configuration.ConfigOptions.key -import org.apache.flink.table.planner.JList import org.apache.flink.table.planner.functions.sql.SqlTryCastFunction import org.apache.flink.table.planner.plan.utils.ExpressionDetail.ExpressionDetail import org.apache.flink.table.planner.plan.utils.ExpressionFormat.ExpressionFormat @@ -35,7 +34,7 @@ import org.apache.calcite.sql.`type`.SqlTypeName import org.apache.calcite.sql.fun.SqlStdOperatorTable._ import org.apache.calcite.sql.fun.{SqlCastFunction, SqlStdOperatorTable} import org.apache.calcite.sql.{SqlAsOperator, SqlKind, SqlOperator} -import org.apache.calcite.util.{ControlFlowException, DateString, ImmutableBitSet, NlsString, Sarg, TimeString, TimestampString, Util} +import org.apache.calcite.util._ import java.lang.{Iterable => JIterable} import java.math.BigDecimal @@ -397,17 +396,6 @@ object FlinkRexUtil { rex.accept(shuttle) } - /** Expands the Sarg operands to literals. */ - def expandSearchOperands(rexBuilder: RexBuilder, call: RexCall): JList[RexNode] = { - require(call.getKind == SqlKind.SEARCH) - val sargLiteral = call.getOperands.get(1).asInstanceOf[RexLiteral] - val sarg = sargLiteral.getValueAs(classOf[Sarg[_]]) - require(sarg.isPoints) - val sargOperands = sarg.rangeSet.asRanges().map(range => - rexBuilder.makeLiteral(range.lowerEndpoint(), sargLiteral.getType, false)) - List(call.getOperands.head) ++ sargOperands - } - /** * Adjust the expression's field indices according to fieldsOldToNewIndexMapping. * diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml index 5cbb2330a72e7..77ae802044512 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/CalcTest.xml @@ -156,6 +156,24 @@ LogicalProject(a=[$0], b=[$1], c=[$2]) + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml index f8d4745254ba0..599e31846232c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/ConvertToNotInOrInRuleTest.xml @@ -231,7 +231,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -250,7 +250,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -269,7 +269,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -288,7 +288,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -307,7 +307,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -326,7 +326,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -383,7 +383,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -402,7 +402,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -421,7 +421,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) @@ -478,7 +478,7 @@ LogicalProject(a=[$0], b=[$1], c=[$2], d=[$3], e=[$4]) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala index cdd34e22df15b..c72f013922369 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/CalcTest.scala @@ -80,6 +80,12 @@ class CalcTest extends TableTestBase { util.verifyExecPlan(sql) } + @Test + def testInNonConstantValues(): Unit = { + val sql = s"SELECT * FROM MyTable WHERE b IN (1, 3, CAST(a AS INT), 5, 6) AND c = 'xx'" + util.verifyExecPlan(sql) + } + @Test def testNotIn(): Unit = { val sql = s"SELECT * FROM MyTable WHERE b NOT IN (1, 3, 4, 5, 6) OR c = 'xx'" diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala index 09699c532b411..db904a19f425f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala @@ -616,12 +616,23 @@ class CalcITCase extends BatchTestBase { // )) // } + @Test + def testInNonConstantValue(): Unit = { + checkResult( + "SELECT a FROM Table3 WHERE a IN (CAST(b AS INT), 21)", + Seq(row(1), row(2), row(21))) + } + @Test def testInSmallValues(): Unit = { checkResult( "SELECT a FROM Table3 WHERE a in (1, 2)", Seq(row(1), row(2))) + checkResult( + "SELECT a FROM Table3 WHERE a in (1, 2, NULL)", + Seq(row(1), row(2))) + checkResult( "SELECT a FROM Table3 WHERE a in (1, 2) and b = 2", Seq(row(2))) From 537b871962b806fc1e40a5c987e046e482a509c5 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Mon, 14 Mar 2022 18:23:25 +0100 Subject: [PATCH 012/258] [FLINK-26106][runtime] Used 'filesystem' for state change log storage in BoundedSourceITCase --- .../lifecycle/BoundedSourceITCase.java | 37 +++++++++++++++++-- 1 file changed, 34 insertions(+), 3 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/BoundedSourceITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/BoundedSourceITCase.java index d03e9cbaa621e..6933fe63b9e47 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/BoundedSourceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/BoundedSourceITCase.java @@ -17,19 +17,27 @@ package org.apache.flink.runtime.operators.lifecycle; +import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory; +import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.operators.lifecycle.event.CheckpointCompletedEvent; import org.apache.flink.runtime.operators.lifecycle.graph.TestJobBuilders.TestingGraphBuilder; import org.apache.flink.runtime.operators.lifecycle.validation.DrainingValidator; import org.apache.flink.runtime.operators.lifecycle.validation.FinishingValidator; -import org.apache.flink.test.util.AbstractTestBase; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.testutils.junit.SharedObjects; +import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import org.junit.runners.Parameterized.Parameter; +import java.io.IOException; + import static org.apache.flink.runtime.operators.lifecycle.command.TestCommand.FINISH_SOURCES; import static org.apache.flink.runtime.operators.lifecycle.command.TestCommandDispatcher.TestCommandScope.ALL_SUBTASKS; import static org.apache.flink.runtime.operators.lifecycle.graph.TestJobBuilders.COMPLEX_GRAPH_BUILDER; @@ -46,10 +54,33 @@ * same. */ @RunWith(Parameterized.class) -public class BoundedSourceITCase extends AbstractTestBase { +public class BoundedSourceITCase extends TestBaseUtils { + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setConfiguration(configuration()) + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(4) + .build()); @Rule public final SharedObjects sharedObjects = SharedObjects.create(); + private static Configuration configuration() { + Configuration conf = new Configuration(); + + try { + FsStateChangelogStorageFactory.configure(conf, TEMPORARY_FOLDER.newFolder()); + } catch (IOException e) { + throw new RuntimeException(e); + } + + return conf; + } + @Parameter public TestingGraphBuilder graphBuilder; @Parameterized.Parameters(name = "{0}") @@ -68,7 +99,7 @@ public void test() throws Exception { .setCheckpointStorage( TEMPORARY_FOLDER.newFolder().toURI())); - TestJobExecutor.execute(testJob, MINI_CLUSTER_RESOURCE) + TestJobExecutor.execute(testJob, miniClusterResource) .waitForEvent(CheckpointCompletedEvent.class) .sendBroadcastCommand(FINISH_SOURCES, ALL_SUBTASKS) .waitForTermination() From b6d57585bd4cab4183dd4197aef75fea17537f5a Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Wed, 9 Mar 2022 19:45:18 +0200 Subject: [PATCH 013/258] [FLINK-26418][runtime][test] Use java.io.tmpdir for tmpWorkingDir Previously, the tmpWorkingDirectory was created in the current working directory, and as a result there were directories created in the root directories of the modules, i.e. `flink-table/flink-table-planner` which were not cleaned up with `mvn clean`. --- .../flink/runtime/util/TestingTaskManagerRuntimeInfo.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java index ac4023094d271..739716f67a85d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/util/TestingTaskManagerRuntimeInfo.java @@ -38,7 +38,8 @@ public class TestingTaskManagerRuntimeInfo implements TaskManagerRuntimeInfo { public TestingTaskManagerRuntimeInfo() { this( new Configuration(), - System.getProperty("java.io.tmpdir").split(",|" + File.pathSeparator)); + EnvironmentInformation.getTemporaryFileDirectory() + .split(",|" + File.pathSeparator)); } public TestingTaskManagerRuntimeInfo(Configuration configuration) { @@ -62,7 +63,9 @@ public TestingTaskManagerRuntimeInfo(Configuration configuration, String[] tmpDi configuration, tmpDirectories, InetAddress.getLoopbackAddress().getHostAddress(), - new File("tmp_" + UUID.randomUUID())); + new File( + EnvironmentInformation.getTemporaryFileDirectory(), + "tmp_" + UUID.randomUUID())); } public TestingTaskManagerRuntimeInfo( From cc7a6407cd006ea20843ae6771125e4d54e0e72f Mon Sep 17 00:00:00 2001 From: Hangxiang Yu Date: Mon, 14 Mar 2022 20:49:00 +0800 Subject: [PATCH 014/258] [FLINK-26573][test] Do not resolve the metadata file which is in progress --- .../java/org/apache/flink/test/util/TestUtils.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java index 66ce4a4bd3ca7..8cbf3e32275a8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java @@ -35,7 +35,9 @@ import java.io.DataInputStream; import java.io.File; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.file.Files; +import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.nio.file.attribute.BasicFileAttributes; import java.util.Comparator; @@ -138,8 +140,15 @@ private static boolean hasMetadata(Path file) { path.getFileName().toString().equals(METADATA_FILE_NAME)) .findAny() .isPresent(); - } catch (IOException e) { - ExceptionUtils.rethrow(e); + } catch (UncheckedIOException uncheckedIOException) { + // return false when the metadata file is in progress due to subsumed checkpoint + if (ExceptionUtils.findThrowable(uncheckedIOException, NoSuchFileException.class) + .isPresent()) { + return false; + } + throw uncheckedIOException; + } catch (IOException ioException) { + ExceptionUtils.rethrow(ioException); return false; // should never happen } } From 0d8412fdac72a2a03fed482f05be21cf885580cc Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 15 Mar 2022 15:59:08 +0100 Subject: [PATCH 015/258] [FLINK-26652][runtime] Makes the cleanup not fail fatally The user explicitly marked the cleanup retry logic to terminate after a certain amount of attempts. This should be considered as desired behavior and shouldn't make the cluster fail fatally. --- .../content.zh/docs/deployment/ha/overview.md | 20 +-- docs/content.zh/docs/deployment/overview.md | 5 + docs/content/docs/deployment/ha/overview.md | 20 +-- docs/content/docs/deployment/overview.md | 7 +- .../generated/cleanup_configuration.html | 2 +- ..._delay_cleanup_strategy_configuration.html | 2 +- ..._delay_cleanup_strategy_configuration.html | 2 +- .../flink/configuration/CleanupOptions.java | 24 +++- .../flink/runtime/dispatcher/Dispatcher.java | 19 ++- .../dispatcher/DispatcherCleanupITCase.java | 115 ++++++++---------- 10 files changed, 126 insertions(+), 90 deletions(-) diff --git a/docs/content.zh/docs/deployment/ha/overview.md b/docs/content.zh/docs/deployment/ha/overview.md index c7e516b55b2c0..0cfcd3087f770 100644 --- a/docs/content.zh/docs/deployment/ha/overview.md +++ b/docs/content.zh/docs/deployment/ha/overview.md @@ -76,14 +76,16 @@ Flink 提供了两种高可用服务实现: ## JobResultStore -In order to preserve a job's scheduling status across failover events and prevent erroneous -re-execution of globally terminated (i.e. finished, cancelled or failed) jobs, Flink persists -status of terminated jobs to a filesystem using the JobResultStore. -The JobResultStore allows job results to outlive a finished job, and can be used by -Flink components involved in the recovery of a highly-available cluster in order to -determine whether a job should be subject to recovery. - -The JobResultStore has sensible defaults for its behaviour, such as result storage -location, but these can be [configured]({{< ref "docs/deployment/config#high-availability" >}}). +The JobResultStore is used to archive the final result of a job that reached a globally-terminal +state (i.e. finished, cancelled or failed). The data is stored on a file system (see +[job-result-store.storage-path]({{< ref "docs/deployment/config#job-result-store-storage-path" >}})). +Entries in this store are marked as dirty as long as the corresponding job wasn't cleaned up properly +(artifacts are found in the job's subfolder in [high-availability.storageDir]({{< ref "docs/deployment/config#high-availability-storagedir" >}})). + +Dirty entries are subject to cleanup, i.e. the corresponding job is either cleaned up by Flink at +the moment or will be picked up for cleanup as part of a recovery. The entries will be deleted as +soon as the cleanup succeeds. Check the JobResultStore configuration parameters under +[HA configuration options]({{< ref "docs/deployment/config#high-availability" >}}) for further +details on how to adapt the behavior. {{< top >}} diff --git a/docs/content.zh/docs/deployment/overview.md b/docs/content.zh/docs/deployment/overview.md index df6abeaf97826..f10dab34e10c5 100644 --- a/docs/content.zh/docs/deployment/overview.md +++ b/docs/content.zh/docs/deployment/overview.md @@ -158,6 +158,11 @@ Once a job has reached a globally terminal state of either finished, failed or c external component resources associated with the job are then cleaned up. In the event of a failure when cleaning up a resource, Flink will attempt to retry the cleanup. You can [configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used. +Reaching the maximum number of retries without succeeding will leave the job in a dirty state. +Its artifacts would need to be cleaned up manually (see the +[High Availability Services / JobResultStore]({{< ref "docs/deployment/ha/overview#jobresultstore" >}}) +section for further details). Restarting the very same job (i.e. using the same +job ID) will result in the cleanup being restarted without running the job again. There is currently an issue with the cleanup of CompletedCheckpoints that failed to be deleted while subsuming them as part of the usual CompletedCheckpoint management. These artifacts are diff --git a/docs/content/docs/deployment/ha/overview.md b/docs/content/docs/deployment/ha/overview.md index 36cdcb460820c..1939474680a26 100644 --- a/docs/content/docs/deployment/ha/overview.md +++ b/docs/content/docs/deployment/ha/overview.md @@ -82,14 +82,16 @@ Once this happens, all the HA data, including the metadata stored in the HA serv ## JobResultStore -In order to preserve a job's scheduling status across failover events and prevent erroneous -re-execution of globally terminated (i.e. finished, cancelled or failed) jobs, Flink persists -status of terminated jobs to a filesystem using the JobResultStore. -The JobResultStore allows job results to outlive a finished job, and can be used by -Flink components involved in the recovery of a highly-available cluster in order to -determine whether a job should be subject to recovery. - -The JobResultStore has sensible defaults for its behaviour, such as result storage -location, but these can be [configured]({{< ref "docs/deployment/config#high-availability" >}}). +The JobResultStore is used to archive the final result of a job that reached a globally-terminal +state (i.e. finished, cancelled or failed). The data is stored on a file system (see +[job-result-store.storage-path]({{< ref "docs/deployment/config#job-result-store-storage-path" >}})). +Entries in this store are marked as dirty as long as the corresponding job wasn't cleaned up properly +(artifacts are found in the job's subfolder in [high-availability.storageDir]({{< ref "docs/deployment/config#high-availability-storagedir" >}})). + +Dirty entries are subject to cleanup, i.e. the corresponding job is either cleaned up by Flink at +the moment or will be picked up for cleanup as part of a recovery. The entries will be deleted as +soon as the cleanup succeeds. Check the JobResultStore configuration parameters under +[HA configuration options]({{< ref "docs/deployment/config#high-availability" >}}) for further +details on how to adapt the behavior. {{< top >}} diff --git a/docs/content/docs/deployment/overview.md b/docs/content/docs/deployment/overview.md index 64bac268ad8d1..247605ee811ac 100644 --- a/docs/content/docs/deployment/overview.md +++ b/docs/content/docs/deployment/overview.md @@ -158,7 +158,12 @@ When deploying Flink, there are often multiple options available for each buildi Once a job has reached a globally terminal state of either finished, failed or cancelled, the external component resources associated with the job are then cleaned up. In the event of a failure when cleaning up a resource, Flink will attempt to retry the cleanup. You can -[configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used. +[configure]({{< ref "docs/deployment/config#retryable-cleanup" >}}) the retry strategy used. +Reaching the maximum number of retries without succeeding will leave the job in a dirty state. +Its artifacts would need to be cleaned up manually (see the +[High Availability Services / JobResultStore]({{< ref "docs/deployment/ha/overview#jobresultstore" >}}) +section for further details). Restarting the very same job (i.e. using the same +job ID) will result in the cleanup being restarted without running the job again. There is currently an issue with the cleanup of CompletedCheckpoints that failed to be deleted while subsuming them as part of the usual CompletedCheckpoint management. These artifacts are diff --git a/docs/layouts/shortcodes/generated/cleanup_configuration.html b/docs/layouts/shortcodes/generated/cleanup_configuration.html index 02e861816713d..e0663c3648c1f 100644 --- a/docs/layouts/shortcodes/generated/cleanup_configuration.html +++ b/docs/layouts/shortcodes/generated/cleanup_configuration.html @@ -12,7 +12,7 @@

cleanup-strategy
"exponential-delay" String - Defines the cleanup strategy to use in case of cleanup failures.
Accepted values are:
  • none, disable, off: Cleanup is only performed once. No retry will be initiated in case of failure.
  • fixed-delay, fixeddelay: Cleanup attempts will be separated by a fixed interval up to the point where the cleanup is considered successful or a set amount of retries is reached.
  • exponential-delay, exponentialdelay: Exponential delay restart strategy triggers the cleanup with an exponentially increasing delay up to the point where the cleanup succeeded or a set amount of retries is reached.
The default configuration relies on an exponentially delayed retry strategy with the given default values. + Defines the cleanup strategy to use in case of cleanup failures.
Accepted values are:
  • none, disable, off: Cleanup is only performed once. No retry will be initiated in case of failure. The job artifacts (and the job's JobResultStore entry) have to be cleaned up manually in case of a failure.
  • fixed-delay, fixeddelay: Cleanup attempts will be separated by a fixed interval up to the point where the cleanup is considered successful or a set amount of retries is reached. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually.
  • exponential-delay, exponentialdelay: Exponential delay restart strategy triggers the cleanup with an exponentially increasing delay up to the point where the cleanup succeeded or a set amount of retries is reached. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually.
The default configuration relies on an exponentially delayed retry strategy with the given default values. diff --git a/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html b/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html index f301253418b1e..ea2933b39c6eb 100644 --- a/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html +++ b/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html @@ -12,7 +12,7 @@
cleanup-strategy.exponential-delay.attempts
(none) Integer - The number of times a failed cleanup is retried if cleanup-strategy has been set to exponential-delay. (no value means: infinitely). + The number of times a failed cleanup is retried if cleanup-strategy has been set to exponential-delay. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually. Setting no value corresponds to unlimited retries.
cleanup-strategy.exponential-delay.initial-backoff
diff --git a/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html b/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html index 6890b1b8292b5..6312784571155 100644 --- a/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html +++ b/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html @@ -12,7 +12,7 @@
cleanup-strategy.fixed-delay.attempts
1 Integer - The number of times that Flink retries the cleanup before giving up if cleanup-strategy has been set to fixed-delay. + The number of times that Flink retries the cleanup before giving up if cleanup-strategy has been set to fixed-delay. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually.
cleanup-strategy.fixed-delay.delay
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java index 49d5dacfb28df..31268eff51da5 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java @@ -93,7 +93,9 @@ public static String extractAlphaNumericCharacters(String paramName) { Collectors.joining( ", ")) + ": Cleanup is only performed once. No retry " - + "will be initiated in case of failure.", + + "will be initiated in case of failure. The job " + + "artifacts (and the job's JobResultStore entry) have " + + "to be cleaned up manually in case of a failure.", NONE_PARAM_VALUES.stream() .map(TextElement::code) .collect(Collectors.toList()) @@ -105,7 +107,9 @@ public static String extractAlphaNumericCharacters(String paramName) { "%s, %s: Cleanup attempts will be separated by a fixed " + "interval up to the point where the cleanup is " + "considered successful or a set amount of retries " - + "is reached.", + + "is reached. Reaching the configured limit means that " + + "the job artifacts (and the job's JobResultStore entry) " + + "might need to be cleaned up manually.", code(FIXED_DELAY_LABEL), code( extractAlphaNumericCharacters( @@ -115,7 +119,9 @@ public static String extractAlphaNumericCharacters(String paramName) { + "triggers the cleanup with an exponentially " + "increasing delay up to the point where the " + "cleanup succeeded or a set amount of retries " - + "is reached.", + + "is reached. Reaching the configured limit means that " + + "the job artifacts (and the job's JobResultStore entry) " + + "might need to be cleaned up manually.", code(EXPONENTIAL_DELAY_LABEL), code( extractAlphaNumericCharacters( @@ -133,7 +139,10 @@ public static String extractAlphaNumericCharacters(String paramName) { Description.builder() .text( "The number of times that Flink retries the cleanup " - + "before giving up if %s has been set to %s.", + + "before giving up if %s has been set to %s. " + + "Reaching the configured limit means that " + + "the job artifacts (and the job's JobResultStore entry) " + + "might need to be cleaned up manually.", code(CLEANUP_STRATEGY_PARAM), code(FIXED_DELAY_LABEL)) .build()); @@ -188,8 +197,11 @@ public static String extractAlphaNumericCharacters(String paramName) { Description.builder() .text( "The number of times a failed cleanup is retried " - + "if %s has been set to %s. (no value means: " - + "infinitely).", + + "if %s has been set to %s. Reaching the " + + "configured limit means that the job artifacts " + + "(and the job's JobResultStore entry) " + + "might need to be cleaned up manually. " + + "Setting no value corresponds to unlimited retries.", code(CLEANUP_STRATEGY_PARAM), code(EXPONENTIAL_DELAY_LABEL)) .build()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 393a4291544c5..68f387dad1a60 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -26,6 +26,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.blob.BlobServer; @@ -44,6 +45,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServices; import org.apache.flink.runtime.highavailability.JobResultEntry; import org.apache.flink.runtime.highavailability.JobResultStore; +import org.apache.flink.runtime.highavailability.JobResultStoreOptions; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -615,7 +617,22 @@ private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionTy final CompletableFuture jobTerminationFuture = cleanupJobStateFuture.thenCompose( - cleanupJobState -> removeJob(jobId, cleanupJobState)); + cleanupJobState -> + removeJob(jobId, cleanupJobState) + .exceptionally( + throwable -> { + log.warn( + "The cleanup of job {} failed. The job's artifacts in '{}' and its JobResultStore entry in '{}' needs to be cleaned manually.", + jobId, + configuration.get( + HighAvailabilityOptions + .HA_STORAGE_PATH), + configuration.get( + JobResultStoreOptions + .STORAGE_PATH), + throwable); + return null; + })); FutureUtils.handleUncaughtException( jobTerminationFuture, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java index 8cfb1ad017532..60312f16194a0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.configuration.CleanupOptions; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.runtime.checkpoint.EmbeddedCompletedCheckpointStore; @@ -49,7 +50,6 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.runtime.testutils.TestingJobResultStore; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TimeUtils; import org.apache.flink.util.concurrent.FutureUtils; @@ -63,11 +63,9 @@ import java.time.Duration; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.LinkedBlockingQueue; @@ -133,12 +131,23 @@ public void testCleanupThroughRetries() throws Exception { final int numberOfErrors = 5; final RuntimeException temporaryError = new RuntimeException("Expected RuntimeException: Unable to remove job graph."); + final AtomicInteger failureCount = new AtomicInteger(numberOfErrors); final JobGraphStore jobGraphStore = - createAndStartJobGraphStoreWithCleanupFailures( - numberOfErrors, - temporaryError, - actualGlobalCleanupCallCount, - successfulCleanupLatch); + TestingJobGraphStore.newBuilder() + .setGlobalCleanupFunction( + (ignoredJobId, ignoredExecutor) -> { + actualGlobalCleanupCallCount.incrementAndGet(); + + if (failureCount.getAndDecrement() > 0) { + return FutureUtils.completedExceptionally(temporaryError); + } + + successfulCleanupLatch.trigger(); + return FutureUtils.completedVoidFuture(); + }) + .build(); + + jobGraphStore.start(NoOpJobGraphListener.INSTANCE); haServices.setJobGraphStore(jobGraphStore); // Construct leader election service. @@ -249,11 +258,28 @@ public void testCleanupAfterLeadershipChange() throws Exception { // Construct job graph store. final AtomicInteger actualGlobalCleanupCallCount = new AtomicInteger(); - final OneShotLatch successfulCleanupLatch = new OneShotLatch(); - final RuntimeException temporaryError = new RuntimeException("Unable to remove job graph."); + final OneShotLatch firstCleanupTriggered = new OneShotLatch(); + final CompletableFuture successfulJobGraphCleanup = new CompletableFuture<>(); final JobGraphStore jobGraphStore = - createAndStartJobGraphStoreWithCleanupFailures( - 1, temporaryError, actualGlobalCleanupCallCount, successfulCleanupLatch); + TestingJobGraphStore.newBuilder() + .setGlobalCleanupFunction( + (actualJobId, ignoredExecutor) -> { + final int callCount = + actualGlobalCleanupCallCount.getAndIncrement(); + firstCleanupTriggered.trigger(); + + if (callCount < 1) { + return FutureUtils.completedExceptionally( + new RuntimeException( + "Expected RuntimeException: Unable to remove job graph.")); + } + + successfulJobGraphCleanup.complete(actualJobId); + return FutureUtils.completedVoidFuture(); + }) + .build(); + + jobGraphStore.start(NoOpJobGraphListener.INSTANCE); haServices.setJobGraphStore(jobGraphStore); // Construct leader election service. @@ -262,23 +288,10 @@ public void testCleanupAfterLeadershipChange() throws Exception { haServices.setJobMasterLeaderElectionService(jobId, leaderElectionService); // start the dispatcher with no retries on cleanup - final CountDownLatch jobGraphRemovalErrorReceived = new CountDownLatch(1); - final Dispatcher dispatcher = - createTestingDispatcherBuilder() - .setFatalErrorHandler( - throwable -> { - final Optional maybeError = - ExceptionUtils.findThrowable( - throwable, temporaryError::equals); - if (maybeError.isPresent()) { - jobGraphRemovalErrorReceived.countDown(); - } else { - testingFatalErrorHandlerResource - .getFatalErrorHandler() - .onFatalError(throwable); - } - }) - .build(); + configuration.set( + CleanupOptions.CLEANUP_STRATEGY, + CleanupOptions.NONE_PARAM_VALUES.iterator().next()); + final Dispatcher dispatcher = createTestingDispatcherBuilder().build(); dispatcher.start(); toTerminate.add(dispatcher); @@ -288,7 +301,7 @@ public void testCleanupAfterLeadershipChange() throws Exception { dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); waitForJobToFinish(leaderElectionService, dispatcherGateway, jobId); - jobGraphRemovalErrorReceived.await(); + firstCleanupTriggered.await(); // Remove job master leadership. leaderElectionService.notLeader(); @@ -296,18 +309,25 @@ public void testCleanupAfterLeadershipChange() throws Exception { // This will clear internal state of election service, so a new contender can register. leaderElectionService.stop(); - assertThat(successfulCleanupLatch.isTriggered(), CoreMatchers.is(false)); + assertThat( + "The cleanup should have been triggered only once.", + actualGlobalCleanupCallCount.get(), + equalTo(1)); + assertThat( + "The cleanup should not have reached the successful cleanup code path.", + successfulJobGraphCleanup.isDone(), + equalTo(false)); assertThat( "The JobGraph is still stored in the JobGraphStore.", haServices.getJobGraphStore().getJobIds(), - CoreMatchers.is(Collections.singleton(jobId))); + equalTo(Collections.singleton(jobId))); assertThat( "The JobResultStore has this job marked as dirty.", haServices.getJobResultStore().getDirtyResults().stream() .map(JobResult::getJobId) .collect(Collectors.toSet()), - CoreMatchers.is(Collections.singleton(jobId))); + equalTo(Collections.singleton(jobId))); // Run a second dispatcher, that restores our finished job. final Dispatcher secondDispatcher = @@ -331,38 +351,11 @@ public void testCleanupAfterLeadershipChange() throws Exception { "The JobResultStore has the job listed as clean.", haServices.getJobResultStore().hasJobResultEntry(jobId)); - // wait for the successful cleanup to be triggered - successfulCleanupLatch.await(); + assertThat(successfulJobGraphCleanup.get(), equalTo(jobId)); assertThat(actualGlobalCleanupCallCount.get(), equalTo(2)); } - private JobGraphStore createAndStartJobGraphStoreWithCleanupFailures( - int numberOfCleanupFailures, - Throwable throwable, - AtomicInteger actualCleanupCallCount, - OneShotLatch successfulCleanupLatch) - throws Exception { - final AtomicInteger failureCount = new AtomicInteger(numberOfCleanupFailures); - final JobGraphStore jobGraphStore = - TestingJobGraphStore.newBuilder() - .setGlobalCleanupFunction( - (ignoredJobId, ignoredExecutor) -> { - actualCleanupCallCount.incrementAndGet(); - - if (failureCount.getAndDecrement() > 0) { - return FutureUtils.completedExceptionally(throwable); - } - - successfulCleanupLatch.trigger(); - return FutureUtils.completedVoidFuture(); - }) - .build(); - - jobGraphStore.start(null); - return jobGraphStore; - } - private void waitForJobToFinish( TestingLeaderElectionService leaderElectionService, DispatcherGateway dispatcherGateway, From 51068fe0d894da5b43281f326e8fcbe39080d534 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 15 Mar 2022 16:14:59 +0100 Subject: [PATCH 016/258] [hotfix][docs] Updates the default value from the fixed delay strategy We want to try infinitely if nothing is specified. --- .../fixed_delay_cleanup_strategy_configuration.html | 4 ++-- .../org/apache/flink/configuration/CleanupOptions.java | 5 +++-- .../cleanup/CleanupRetryStrategyFactoryTest.java | 8 ++++++-- 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html b/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html index 6312784571155..231fc9d07b028 100644 --- a/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html +++ b/docs/layouts/shortcodes/generated/fixed_delay_cleanup_strategy_configuration.html @@ -10,13 +10,13 @@
cleanup-strategy.fixed-delay.attempts
- 1 + infinite Integer The number of times that Flink retries the cleanup before giving up if cleanup-strategy has been set to fixed-delay. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually.
cleanup-strategy.fixed-delay.delay
- 1 s + 1 min Duration Amount of time that Flink waits before re-triggering the cleanup after a failed attempt if the cleanup-strategy is set to fixed-delay. It can be specified using the following notation: "1 min", "20 s" diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java index 31268eff51da5..7c6ff646b1caa 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java @@ -131,10 +131,11 @@ public static String extractAlphaNumericCharacters(String paramName) { + "retry strategy with the given default values.") .build()); + @Documentation.OverrideDefault("infinite") public static final ConfigOption CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS = ConfigOptions.key(createFixedDelayParameterPrefix("attempts")) .intType() - .defaultValue(1) + .defaultValue(Integer.MAX_VALUE) .withDescription( Description.builder() .text( @@ -149,7 +150,7 @@ public static String extractAlphaNumericCharacters(String paramName) { public static final ConfigOption CLEANUP_STRATEGY_FIXED_DELAY_DELAY = ConfigOptions.key(createFixedDelayParameterPrefix("delay")) .durationType() - .defaultValue(Duration.ofSeconds(1)) + .defaultValue(Duration.ofMinutes(1)) .withDescription( Description.builder() .text( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java index ed6ef75a5e6bf..9cf1db6494d46 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.CleanupOptions; import org.apache.flink.configuration.Configuration; +import org.apache.flink.util.Preconditions; import org.apache.flink.util.concurrent.ExponentialBackoffRetryStrategy; import org.apache.flink.util.concurrent.FixedRetryStrategy; import org.apache.flink.util.concurrent.RetryStrategy; @@ -115,8 +116,11 @@ public void testFixedDelayStrategyWithCustomDelay() { public void testFixedDelayStrategyWithCustomMaxAttempts() { final Configuration config = createConfigurationWithRetryStrategy(CleanupOptions.FIXED_DELAY_LABEL); - final int customMaxAttempts = - CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS.defaultValue() + 2; + final int customMaxAttempts = 1; + Preconditions.checkArgument( + customMaxAttempts + != CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS.defaultValue(), + "The custom value should be different from the default value to make it possible that the overwritten value is selected."); config.set(CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_ATTEMPTS, customMaxAttempts); testFixedDelayStrategyCreation( From 9162c7e35ae6458e778832678ffe685f27f9fea7 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 16 Mar 2022 13:19:51 +0100 Subject: [PATCH 017/258] [hotfix][docs] Uses @OverrideDefault instead of noDefaultValue for exponential-delay.attempts --- ...ential_delay_cleanup_strategy_configuration.html | 4 ++-- .../apache/flink/configuration/CleanupOptions.java | 7 ++++--- .../cleanup/CleanupRetryStrategyFactory.java | 4 +--- .../cleanup/CleanupRetryStrategyFactoryTest.java | 13 +++++++++---- 4 files changed, 16 insertions(+), 12 deletions(-) diff --git a/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html b/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html index ea2933b39c6eb..23f13e0a5145c 100644 --- a/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html +++ b/docs/layouts/shortcodes/generated/exponential_delay_cleanup_strategy_configuration.html @@ -10,9 +10,9 @@
cleanup-strategy.exponential-delay.attempts
- (none) + infinite Integer - The number of times a failed cleanup is retried if cleanup-strategy has been set to exponential-delay. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually. Setting no value corresponds to unlimited retries. + The number of times a failed cleanup is retried if cleanup-strategy has been set to exponential-delay. Reaching the configured limit means that the job artifacts (and the job's JobResultStore entry) might need to be cleaned up manually.
cleanup-strategy.exponential-delay.initial-backoff
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java index 7c6ff646b1caa..1efc4c59d1724 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/CleanupOptions.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.annotation.docs.ConfigGroup; import org.apache.flink.annotation.docs.ConfigGroups; +import org.apache.flink.annotation.docs.Documentation; import org.apache.flink.configuration.description.Description; import org.apache.flink.configuration.description.TextElement; @@ -190,10 +191,11 @@ public static String extractAlphaNumericCharacters(String paramName) { code(EXPONENTIAL_DELAY_LABEL)) .build()); + @Documentation.OverrideDefault("infinite") public static final ConfigOption CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS = ConfigOptions.key(createExponentialBackoffParameterPrefix("attempts")) .intType() - .noDefaultValue() + .defaultValue(Integer.MAX_VALUE) .withDescription( Description.builder() .text( @@ -201,8 +203,7 @@ public static String extractAlphaNumericCharacters(String paramName) { + "if %s has been set to %s. Reaching the " + "configured limit means that the job artifacts " + "(and the job's JobResultStore entry) " - + "might need to be cleaned up manually. " - + "Setting no value corresponds to unlimited retries.", + + "might need to be cleaned up manually.", code(CLEANUP_STRATEGY_PARAM), code(EXPONENTIAL_DELAY_LABEL)) .build()); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactory.java index 79557b0a6adca..1ec88a100a5f2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactory.java @@ -78,9 +78,7 @@ private static ExponentialBackoffRetryStrategy createExponentialBackoffRetryStra final Duration maxDelay = configuration.get(CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF); final int maxAttempts = - configuration.getInteger( - CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS, - Integer.MAX_VALUE); + configuration.get(CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS); return new ExponentialBackoffRetryStrategy(maxAttempts, minDelay, maxDelay); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java index 9cf1db6494d46..c1dc76b2c4d8d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java @@ -65,7 +65,7 @@ public void testDefaultStrategyCreation() { new Configuration(), CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_INITIAL_BACKOFF.defaultValue(), CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF.defaultValue(), - Integer.MAX_VALUE); + CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS.defaultValue()); } private static Configuration createConfigurationWithRetryStrategy(String configValue) { @@ -162,7 +162,7 @@ private static void testExponentialBackoffDelayRetryStrategyWithDefaultValues(St config, CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_INITIAL_BACKOFF.defaultValue(), CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF.defaultValue(), - Integer.MAX_VALUE); + CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS.defaultValue()); } @Test @@ -180,7 +180,7 @@ public void testExponentialBackoffDelayRetryStrategyWithCustomMinimumDelay() { config, customMinDelay, CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF.defaultValue(), - Integer.MAX_VALUE); + CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS.defaultValue()); } @Test @@ -197,7 +197,7 @@ public void testExponentialBackoffDelayRetryStrategyWithCustomMaximumDelay() { config, CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_INITIAL_BACKOFF.defaultValue(), customMaxDelay, - Integer.MAX_VALUE); + CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS.defaultValue()); } @Test @@ -207,6 +207,11 @@ public void testExponentialBackoffDelayRetryStrategyWithCustomMaxAttempts() { // 13 is the minimum we can use for this test; otherwise, assertMaxDelay would fail due to a // Precondition in ExponentialBackoffRetryStrategy final int customMaxAttempts = 13; + Preconditions.checkArgument( + customMaxAttempts + != CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS + .defaultValue(), + "The custom value should be different from the default value to make it possible that the overwritten value is selected."); config.set( CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_ATTEMPTS, customMaxAttempts); From 7e768d5a734fcdabd0eeba0d56991f0aa9bf3ef1 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 15 Mar 2022 20:12:00 +0100 Subject: [PATCH 018/258] [hotfix][runtime][test] Improves assert message --- .../flink/runtime/dispatcher/DispatcherCleanupITCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java index 60312f16194a0..4b4ce27433325 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java @@ -323,7 +323,7 @@ public void testCleanupAfterLeadershipChange() throws Exception { haServices.getJobGraphStore().getJobIds(), equalTo(Collections.singleton(jobId))); assertThat( - "The JobResultStore has this job marked as dirty.", + "The JobResultStore should have this job marked as dirty.", haServices.getJobResultStore().getDirtyResults().stream() .map(JobResult::getJobId) .collect(Collectors.toSet()), From 4e4d77bfba0a873878b96b4a0cac4786c64831f3 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 15 Mar 2022 21:06:32 +0100 Subject: [PATCH 019/258] [hotfix][docs] Adds missing JRS configuration parameter in Chinese documentation --- docs/content.zh/docs/deployment/config.md | 4 ++++ docs/content/docs/deployment/config.md | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/content.zh/docs/deployment/config.md b/docs/content.zh/docs/deployment/config.md index 9348ecc2518f4..912ac8bb1d859 100644 --- a/docs/content.zh/docs/deployment/config.md +++ b/docs/content.zh/docs/deployment/config.md @@ -166,6 +166,10 @@ The JobManager ensures consistency during recovery across TaskManagers. For the {{< generated/common_high_availability_section >}} +**Options for the JobResultStore in high-availability setups** + +{{< generated/common_high_availability_jrs_section >}} + **Options for high-availability setups with ZooKeeper** {{< generated/common_high_availability_zk_section >}} diff --git a/docs/content/docs/deployment/config.md b/docs/content/docs/deployment/config.md index 9a00287658e3e..a38e72d0e1828 100644 --- a/docs/content/docs/deployment/config.md +++ b/docs/content/docs/deployment/config.md @@ -167,7 +167,7 @@ The JobManager ensures consistency during recovery across TaskManagers. For the {{< generated/common_high_availability_section >}} -**Options for the Job Result Store in high-availability setups** +**Options for the JobResultStore in high-availability setups** {{< generated/common_high_availability_jrs_section >}} From 282079b86f80e60ad00afa01fe944d0b34b5e13a Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 15 Mar 2022 21:22:28 +0100 Subject: [PATCH 020/258] [hotfix][tests] Fixed wrong default value in test --- .../dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java index c1dc76b2c4d8d..cd3d0a573b746 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CleanupRetryStrategyFactoryTest.java @@ -217,7 +217,7 @@ public void testExponentialBackoffDelayRetryStrategyWithCustomMaxAttempts() { testExponentialBackoffDelayRetryStrategyCreation( config, - CleanupOptions.CLEANUP_STRATEGY_FIXED_DELAY_DELAY.defaultValue(), + CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_INITIAL_BACKOFF.defaultValue(), CleanupOptions.CLEANUP_STRATEGY_EXPONENTIAL_DELAY_MAX_BACKOFF.defaultValue(), customMaxAttempts); } From 991c74a12332ac4473f4fe13ec380b809fbbfed2 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 14 Mar 2022 11:38:14 +0100 Subject: [PATCH 021/258] [FLINK-26658][docs] Migrate documentation build to Github Actions --- .github/workflows/docs.sh | 71 +++++++++++++++++++++++++++++++++++++++ tools/ci/build_docs.sh | 38 --------------------- 2 files changed, 71 insertions(+), 38 deletions(-) create mode 100755 .github/workflows/docs.sh delete mode 100755 tools/ci/build_docs.sh diff --git a/.github/workflows/docs.sh b/.github/workflows/docs.sh new file mode 100755 index 0000000000000..fa0fd21bd6d07 --- /dev/null +++ b/.github/workflows/docs.sh @@ -0,0 +1,71 @@ +#!/usr/bin/env bash +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +set -e + +mvn --version +java -version +javadoc -J-version + +# setup hugo +HUGO_REPO=https://github.com/gohugoio/hugo/releases/download/v0.80.0/hugo_extended_0.80.0_Linux-64bit.tar.gz +HUGO_ARTIFACT=hugo_extended_0.80.0_Linux-64bit.tar.gz +if ! curl --fail -OL $HUGO_REPO ; then + echo "Failed to download Hugo binary" + exit 1 +fi +tar -zxvf $HUGO_ARTIFACT +git submodule update --init --recursive +# generate docs into docs/target +./hugo -v --source docs --destination target +if [ $? -ne 0 ]; then + echo "Error building the docs" + exit 1 +fi + +# build Flink; required for Javadoc step +mvn clean install -B -DskipTests -Dfast -Pskip-webui-build + +# build java/scala docs +mkdir -p docs/target/api +mvn javadoc:aggregate -B \ + -Paggregate-scaladoc \ + -DadditionalJOption="-Xdoclint:none --allow-script-in-comments" \ + -Dmaven.javadoc.failOnError=false \ + -Dcheckstyle.skip=true \ + -Dspotless.check.skip=true \ + -Denforcer.skip=true \ + -Dheader="

Back to Flink Website

" +mv target/site/apidocs docs/target/api/java +pushd flink-scala +mvn scala:doc -B +mv target/site/scaladocs ../docs/target/api/scala +popd + +# build python docs +if [ -f ./flink-python/dev/lint-python.sh ]; then + # Just completely ignore sudo in conda. + unset SUDO_UID SUDO_GID SUDO_USER + + # build python docs + # disable the gateway, because otherwise it tries to find FLINK_HOME to access Java classes + PYFLINK_GATEWAY_DISABLED=1 ./flink-python/dev/lint-python.sh -i "sphinx" + + # move python docs + mv flink-python/docs/_build/html docs/target/api/python +fi diff --git a/tools/ci/build_docs.sh b/tools/ci/build_docs.sh deleted file mode 100755 index 165d99f0e9bd1..0000000000000 --- a/tools/ci/build_docs.sh +++ /dev/null @@ -1,38 +0,0 @@ -#!/usr/bin/env bash -################################################################################ -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -################################################################################ - -HUGO_REPO=https://github.com/gohugoio/hugo/releases/download/v0.80.0/hugo_extended_0.80.0_Linux-64bit.tar.gz -HUGO_ARTIFACT=hugo_extended_0.80.0_Linux-64bit.tar.gz - -if ! curl --fail -OL $HUGO_REPO ; then - echo "Failed to download Hugo binary" - exit 1 -fi - -tar -zxvf $HUGO_ARTIFACT - -git submodule update --init --recursive -# generate docs into docs/target -./hugo -v --source docs --destination target - -if [ $? -ne 0 ]; then - echo "Error building the docs" - exit 1 -fi - From 891ec9dfb05fa05c91cc4e9fd0108b56d8d7a3db Mon Sep 17 00:00:00 2001 From: zhangjingcun Date: Tue, 15 Mar 2022 11:23:13 +0800 Subject: [PATCH 022/258] [FLINK-26607][python] Correct the MAX_LONG_VALUE/MIN_LONG_VALUE in several places This closes #19089. --- flink-python/pyflink/common/constants.py | 26 +++++++++++++++++++ flink-python/pyflink/datastream/window.py | 4 +-- .../datastream/window/window_operator.py | 2 +- .../table/window_aggregate_fast.pyx | 4 +-- .../table/window_aggregate_slow.py | 3 +-- .../fn_execution/table/window_context.py | 4 +-- .../table/window_process_function.py | 4 +-- flink-python/pyflink/table/functions.py | 5 +--- 8 files changed, 33 insertions(+), 19 deletions(-) create mode 100644 flink-python/pyflink/common/constants.py diff --git a/flink-python/pyflink/common/constants.py b/flink-python/pyflink/common/constants.py new file mode 100644 index 0000000000000..3fe6e0e1a0d84 --- /dev/null +++ b/flink-python/pyflink/common/constants.py @@ -0,0 +1,26 @@ +################################################################################ +# 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. +################################################################################ +""" +A constant holding the maximum value a long can have, 2^63 – 1. +""" +MAX_LONG_VALUE = 0x7fffffffffffffff + +""" +A constant holding the minimum value a long can have, -2^63 +""" +MIN_LONG_VALUE = - MAX_LONG_VALUE - 1 diff --git a/flink-python/pyflink/datastream/window.py b/flink-python/pyflink/datastream/window.py index 53266e0051f8e..f310030ae14ba 100644 --- a/flink-python/pyflink/datastream/window.py +++ b/flink-python/pyflink/datastream/window.py @@ -15,12 +15,12 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from abc import ABC, abstractmethod from enum import Enum from io import BytesIO from typing import TypeVar, Generic, Iterable, Collection +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.common.serializer import TypeSerializer from pyflink.datastream.functions import RuntimeContext, InternalWindowFunction from pyflink.datastream.state import StateDescriptor, State @@ -36,8 +36,6 @@ 'TimeWindowSerializer', 'CountWindowSerializer'] -MAX_LONG_VALUE = sys.maxsize - def long_to_int_with_bit_mixing(x: int) -> int: x = (x ^ (x >> 30)) * 0xbf58476d1ce4e5b9 diff --git a/flink-python/pyflink/fn_execution/datastream/window/window_operator.py b/flink-python/pyflink/fn_execution/datastream/window/window_operator.py index be172f20435d8..0e4575f91b41b 100644 --- a/flink-python/pyflink/fn_execution/datastream/window/window_operator.py +++ b/flink-python/pyflink/fn_execution/datastream/window/window_operator.py @@ -18,13 +18,13 @@ import typing from typing import TypeVar, Iterable, Collection +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.datastream import WindowAssigner, Trigger, MergingWindowAssigner, TriggerResult from pyflink.datastream.functions import KeyedStateStore, RuntimeContext, InternalWindowFunction from pyflink.datastream.state import StateDescriptor, ListStateDescriptor, \ ReducingStateDescriptor, AggregatingStateDescriptor, ValueStateDescriptor, MapStateDescriptor, \ State, AggregatingState, ReducingState, MapState, ListState, ValueState, AppendingState from pyflink.fn_execution.datastream.timerservice import InternalTimerService -from pyflink.datastream.window import MAX_LONG_VALUE from pyflink.fn_execution.datastream.window.merging_window_set import MergingWindowSet from pyflink.fn_execution.internal_state import InternalMergingState, InternalKvState, \ InternalAppendingState diff --git a/flink-python/pyflink/fn_execution/table/window_aggregate_fast.pyx b/flink-python/pyflink/fn_execution/table/window_aggregate_fast.pyx index 7a21249a8542f..bcbee00f0a716 100644 --- a/flink-python/pyflink/fn_execution/table/window_aggregate_fast.pyx +++ b/flink-python/pyflink/fn_execution/table/window_aggregate_fast.pyx @@ -25,11 +25,11 @@ from pyflink.fn_execution.table.aggregate_fast cimport DistinctViewDescriptor, R from pyflink.fn_execution.coder_impl_fast cimport InternalRowKind import datetime -import sys from typing import List, Dict import pytz +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.fn_execution.datastream.timerservice_impl import LegacyInternalTimerServiceImpl from pyflink.fn_execution.coders import PickleCoder from pyflink.fn_execution.table.state_data_view import DataViewSpec, ListViewSpec, MapViewSpec, \ @@ -42,8 +42,6 @@ from pyflink.fn_execution.table.window_process_function import GeneralWindowProc from pyflink.fn_execution.table.window_trigger import Trigger from pyflink.table.udf import ImperativeAggregateFunction -MAX_LONG_VALUE = sys.maxsize - cdef InternalRow join_row(list left, list right, InternalRowKind row_kind): return InternalRow(left.__add__(right), row_kind) diff --git a/flink-python/pyflink/fn_execution/table/window_aggregate_slow.py b/flink-python/pyflink/fn_execution/table/window_aggregate_slow.py index a7e557b385a18..9fc1f2e6f3814 100644 --- a/flink-python/pyflink/fn_execution/table/window_aggregate_slow.py +++ b/flink-python/pyflink/fn_execution/table/window_aggregate_slow.py @@ -16,13 +16,13 @@ # limitations under the License. ################################################################################ import datetime -import sys from abc import ABC, abstractmethod from typing import TypeVar, Generic, List, Dict import pytz from pyflink.common import Row, RowKind +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.fn_execution.datastream.timerservice import InternalTimer from pyflink.fn_execution.datastream.timerservice_impl import LegacyInternalTimerServiceImpl from pyflink.fn_execution.coders import PickleCoder @@ -37,7 +37,6 @@ from pyflink.fn_execution.table.window_trigger import Trigger from pyflink.table.udf import ImperativeAggregateFunction, FunctionContext -MAX_LONG_VALUE = sys.maxsize N = TypeVar('N') diff --git a/flink-python/pyflink/fn_execution/table/window_context.py b/flink-python/pyflink/fn_execution/table/window_context.py index 2e1946e0d6913..5e2bebc86e98a 100644 --- a/flink-python/pyflink/fn_execution/table/window_context.py +++ b/flink-python/pyflink/fn_execution/table/window_context.py @@ -15,10 +15,10 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from abc import ABC, abstractmethod from typing import Generic, TypeVar, List, Iterable +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.datastream.state import StateDescriptor, State, ValueStateDescriptor, \ ListStateDescriptor, MapStateDescriptor from pyflink.datastream.window import TimeWindow, CountWindow @@ -26,8 +26,6 @@ from pyflink.fn_execution.coders import from_type_info, MapCoder, GenericArrayCoder from pyflink.fn_execution.internal_state import InternalMergingState -MAX_LONG_VALUE = sys.maxsize - K = TypeVar('K') W = TypeVar('W', TimeWindow, CountWindow) diff --git a/flink-python/pyflink/fn_execution/table/window_process_function.py b/flink-python/pyflink/fn_execution/table/window_process_function.py index 733d4c1fdac2e..333d903f90182 100644 --- a/flink-python/pyflink/fn_execution/table/window_process_function.py +++ b/flink-python/pyflink/fn_execution/table/window_process_function.py @@ -15,18 +15,16 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys from abc import abstractmethod, ABC from typing import Generic, List, Iterable, Dict, Set from pyflink.common import Row +from pyflink.common.constants import MAX_LONG_VALUE from pyflink.datastream.state import MapState from pyflink.fn_execution.table.window_assigner import WindowAssigner, PanedWindowAssigner, \ MergingWindowAssigner from pyflink.fn_execution.table.window_context import Context, K, W -MAX_LONG_VALUE = sys.maxsize - def join_row(left: List, right: List): return Row(*(left + right)) diff --git a/flink-python/pyflink/table/functions.py b/flink-python/pyflink/table/functions.py index d5b78cee0c318..9fe27dfa1d864 100644 --- a/flink-python/pyflink/table/functions.py +++ b/flink-python/pyflink/table/functions.py @@ -15,16 +15,13 @@ # See the License for the specific language governing permissions and # limitations under the License. ################################################################################ -import sys import time from abc import abstractmethod from decimal import Decimal +from pyflink.common.constants import MAX_LONG_VALUE, MIN_LONG_VALUE from pyflink.table import AggregateFunction, MapView, ListView -MAX_LONG_VALUE = sys.maxsize -MIN_LONG_VALUE = -MAX_LONG_VALUE - 1 - class AvgAggFunction(AggregateFunction): From 311122c4f7fd4a2cb5b68cb547c1c57c3fe5b976 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Thu, 17 Mar 2022 14:26:41 +0800 Subject: [PATCH 023/258] [hotfix][ci] Try to fix the e2e ci pipeline upgrading the libssl version --- tools/azure-pipelines/e2e-template.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tools/azure-pipelines/e2e-template.yml b/tools/azure-pipelines/e2e-template.yml index f590c15758c1c..637ea476d30fd 100644 --- a/tools/azure-pipelines/e2e-template.yml +++ b/tools/azure-pipelines/e2e-template.yml @@ -102,8 +102,8 @@ jobs: echo "Installing required software" sudo apt-get install -y bc libapr1 # install libssl1.0.0 for netty tcnative - wget http://security.ubuntu.com/ubuntu/pool/main/o/openssl1.0/libssl1.0.0_1.0.2n-1ubuntu5.7_amd64.deb - sudo apt install ./libssl1.0.0_1.0.2n-1ubuntu5.7_amd64.deb + wget http://security.ubuntu.com/ubuntu/pool/main/o/openssl1.0/libssl1.0.0_1.0.2n-1ubuntu5.8_amd64.deb + sudo apt install ./libssl1.0.0_1.0.2n-1ubuntu5.8_amd64.deb displayName: Prepare E2E run condition: not(eq(variables['SKIP'], '1')) - script: ${{parameters.environment}} PROFILE="$PROFILE -Dfast -Pskip-webui-build" ./tools/ci/compile.sh @@ -128,4 +128,4 @@ jobs: - script: ./tools/azure-pipelines/cache_docker_images.sh save displayName: Save docker images condition: and(not(eq(variables['SKIP'], '1')), ne(variables.DOCKER_IMAGES_CACHE_HIT, 'true')) - continueOnError: true \ No newline at end of file + continueOnError: true From 31cd7376e64cce16808f01eedc70b1e678170781 Mon Sep 17 00:00:00 2001 From: Fabian Paul Date: Mon, 14 Mar 2022 10:41:00 +0100 Subject: [PATCH 024/258] [FLINK-26613][streaming] Allow setting operator uid hashes for predefined sink operators Since the topology has changes between Flink 1.14 and 1.15 it might happen that stateful upgrades are not possible if no pior operator uids were set. With this commit, users can set operator uid hashes for the respective operators. --- .../CustomSinkOperatorUidHashes.java | 130 ++++++++++++++++++ .../streaming/api/datastream/DataStream.java | 38 ++++- .../api/datastream/DataStreamSink.java | 14 +- .../transformations/SinkTransformation.java | 10 +- .../SinkTransformationTranslator.java | 26 ++++ .../SinkTransformationTranslatorTest.java | 30 ++++ .../nodes/exec/common/CommonExecSink.java | 9 +- 7 files changed, 248 insertions(+), 9 deletions(-) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CustomSinkOperatorUidHashes.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CustomSinkOperatorUidHashes.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CustomSinkOperatorUidHashes.java new file mode 100644 index 0000000000000..ae4a60eb6e435 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CustomSinkOperatorUidHashes.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.streaming.api.datastream; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; + +import javax.annotation.Nullable; + +/** + * This class is responsible to hold operator Uid hashes from the common operators of the sink. With + * this, users can recover a sink snapshot that did not bind uids to the operator before changing + * the topology. + */ +@PublicEvolving +public class CustomSinkOperatorUidHashes { + + /** Default instance providing no custom sink operator hashes. */ + public static final CustomSinkOperatorUidHashes DEFAULT = + CustomSinkOperatorUidHashes.builder().build(); + + @Nullable private final String writerUidHash; + @Nullable private final String committerUidHash; + @Nullable private final String globalCommitterUidHash; + + private CustomSinkOperatorUidHashes( + @Nullable String writerUidHash, + @Nullable String committerUidHash, + @Nullable String globalCommitterUidHash) { + this.writerUidHash = writerUidHash; + this.committerUidHash = committerUidHash; + this.globalCommitterUidHash = globalCommitterUidHash; + } + + /** + * Creates a builder to construct {@link CustomSinkOperatorUidHashes}. + * + * @return {@link SinkOperatorUidHashesBuilder} + */ + public static SinkOperatorUidHashesBuilder builder() { + return new SinkOperatorUidHashesBuilder(); + } + + @Internal + @Nullable + public String getWriterUidHash() { + return writerUidHash; + } + + @Internal + @Nullable + public String getCommitterUidHash() { + return committerUidHash; + } + + @Internal + @Nullable + public String getGlobalCommitterUidHash() { + return globalCommitterUidHash; + } + + /** Builder to construct {@link CustomSinkOperatorUidHashes}. */ + @PublicEvolving + public static class SinkOperatorUidHashesBuilder { + + @Nullable String writerUidHash = null; + @Nullable String committerUidHash = null; + @Nullable String globalCommitterUidHash = null; + + /** + * Sets the uid hash of the writer operator used to recover state. + * + * @param writerUidHash uid hash denoting writer operator + * @return {@link SinkOperatorUidHashesBuilder} + */ + public SinkOperatorUidHashesBuilder setWriterUidHash(String writerUidHash) { + this.writerUidHash = writerUidHash; + return this; + } + + /** + * Sets the uid hash of the committer operator used to recover state. + * + * @param committerUidHash uid hash denoting the committer operator + * @return {@link SinkOperatorUidHashesBuilder} + */ + public SinkOperatorUidHashesBuilder setCommitterUidHash(String committerUidHash) { + this.committerUidHash = committerUidHash; + return this; + } + + /** + * Sets the uid hash of the global committer operator used to recover state. + * + * @param globalCommitterUidHash uid hash denoting the global committer operator + * @return {@link SinkOperatorUidHashesBuilder} + */ + public SinkOperatorUidHashesBuilder setGlobalCommitterUidHash( + String globalCommitterUidHash) { + this.globalCommitterUidHash = globalCommitterUidHash; + return this; + } + + /** + * Constructs the {@link CustomSinkOperatorUidHashes} with the given uid hashes. + * + * @return {@link CustomSinkOperatorUidHashes} + */ + public CustomSinkOperatorUidHashes build() { + return new CustomSinkOperatorUidHashes( + writerUidHash, committerUidHash, globalCommitterUidHash); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java index 3382b01804d71..b0c3d64966579 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java @@ -1250,10 +1250,27 @@ public DataStreamSink addSink(SinkFunction sinkFunction) { */ @PublicEvolving public DataStreamSink sinkTo(org.apache.flink.api.connector.sink.Sink sink) { + return this.sinkTo(sink, CustomSinkOperatorUidHashes.DEFAULT); + } + + /** + * Adds the given {@link Sink} to this DataStream. Only streams with sinks added will be + * executed once the {@link StreamExecutionEnvironment#execute()} method is called. + * + *

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

This method is intended to be used only to recover a snapshot where no uids have been set + * before taking the snapshot. + * + * @param customSinkOperatorUidHashes operator hashes to support state binding + * @param sink The user defined sink. + * @return The closed DataStream. + */ + @PublicEvolving + public DataStreamSink sinkTo( + Sink sink, CustomSinkOperatorUidHashes customSinkOperatorUidHashes) { // read the output type of the input Transform to coax out errors about MissingTypeInfo transformation.getOutputType(); - return DataStreamSink.forSink(this, sink); + return DataStreamSink.forSink(this, sink, customSinkOperatorUidHashes); } /** diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java index f93c2c09ebf11..f9d60e722538f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java @@ -65,7 +65,10 @@ static DataStreamSink forSinkFunction( } @Internal - public static DataStreamSink forSink(DataStream inputStream, Sink sink) { + public static DataStreamSink forSink( + DataStream inputStream, + Sink sink, + CustomSinkOperatorUidHashes customSinkOperatorUidHashes) { final StreamExecutionEnvironment executionEnvironment = inputStream.getExecutionEnvironment(); SinkTransformation transformation = @@ -74,15 +77,18 @@ public static DataStreamSink forSink(DataStream inputStream, Sink s sink, inputStream.getType(), "Sink", - executionEnvironment.getParallelism()); + executionEnvironment.getParallelism(), + customSinkOperatorUidHashes); executionEnvironment.addOperator(transformation); return new DataStreamSink<>(transformation); } @Internal public static DataStreamSink forSinkV1( - DataStream inputStream, org.apache.flink.api.connector.sink.Sink sink) { - return forSink(inputStream, SinkV1Adapter.wrap(sink)); + DataStream inputStream, + org.apache.flink.api.connector.sink.Sink sink, + CustomSinkOperatorUidHashes customSinkOperatorUidHashes) { + return forSink(inputStream, SinkV1Adapter.wrap(sink), customSinkOperatorUidHashes); } /** Returns the transformation that contains the actual sink operator of this sink. */ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java index 967f3642c57f8..bb11c260e95fb 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java @@ -23,6 +23,7 @@ Licensed to the Apache Software Foundation (ASF) under one import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.operators.ChainingStrategy; @@ -47,6 +48,7 @@ public class SinkTransformation extends PhysicalTransformation< private final DataStream inputStream; private final Sink sink; private final Transformation input; + private final CustomSinkOperatorUidHashes customSinkOperatorUidHashes; private ChainingStrategy chainingStrategy; @@ -55,11 +57,13 @@ public SinkTransformation( Sink sink, TypeInformation outputType, String name, - int parallelism) { + int parallelism, + CustomSinkOperatorUidHashes customSinkOperatorUidHashes) { super(name, outputType, parallelism); this.inputStream = checkNotNull(inputStream); this.sink = checkNotNull(sink); this.input = inputStream.getTransformation(); + this.customSinkOperatorUidHashes = checkNotNull(customSinkOperatorUidHashes); } @Override @@ -92,4 +96,8 @@ public DataStream getInputStream() { public Sink getSink() { return sink; } + + public CustomSinkOperatorUidHashes getSinkOperatorsUidHashes() { + return customSinkOperatorUidHashes; + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java index 1bdcc505a83d1..782cccaf64795 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java @@ -31,6 +31,7 @@ import org.apache.flink.streaming.api.connector.sink2.WithPostCommitTopology; import org.apache.flink.streaming.api.connector.sink2.WithPreCommitTopology; import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology; +import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.TransformationTranslator; @@ -257,7 +258,10 @@ private R adjustTransformations( List> expandedTransformations = transformations.subList(numTransformsBefore, transformations.size()); + final CustomSinkOperatorUidHashes operatorsUidHashes = + transformation.getSinkOperatorsUidHashes(); for (Transformation subTransformation : expandedTransformations) { + String subUid = subTransformation.getUid(); if (isExpandedTopology && subUid != null && !subUid.isEmpty()) { checkState( @@ -268,6 +272,18 @@ private R adjustTransformations( + " has set uid for some operators."); } + // Set the operator uid hashes to support stateful upgrades without prior uids + setOperatorUidHashIfPossible( + subTransformation, WRITER_NAME, operatorsUidHashes.getWriterUidHash()); + setOperatorUidHashIfPossible( + subTransformation, + COMMITTER_NAME, + operatorsUidHashes.getCommitterUidHash()); + setOperatorUidHashIfPossible( + subTransformation, + StandardSinkTopologies.GLOBAL_COMMITTER_TRANSFORMATION_NAME, + operatorsUidHashes.getGlobalCommitterUidHash()); + concatUid( subTransformation, Transformation::getUid, @@ -323,6 +339,16 @@ private R adjustTransformations( return result; } + private void setOperatorUidHashIfPossible( + Transformation transformation, + String writerName, + @Nullable String operatorUidHash) { + if (operatorUidHash == null || !transformation.getName().equals(writerName)) { + return; + } + transformation.setUidHash(operatorUidHash); + } + private void concatUid( Transformation subTransformation, Function, String> getter, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java index 44189e061980e..164a8c46bf967 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.core.io.SimpleVersionedSerializerTypeSerializerProxy; +import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -47,6 +48,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; /** Tests for {@link org.apache.flink.streaming.api.transformations.SinkTransformation}. */ @RunWith(Parameterized.class) @@ -263,6 +265,34 @@ public void disableOperatorChain() { is(ChainingStrategy.NEVER)); } + @Test + public void testSettingOperatorUidHash() { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + final DataStreamSource src = env.fromElements(1, 2); + final String writerHash = "f6b178ce445dc3ffaa06bad27a51fead"; + final String committerHash = "68ac8ae79eae4e3135a54f9689c4aa10"; + final String globalCommitterHash = "77e6aa6eeb1643b3765e1e4a7a672f37"; + final CustomSinkOperatorUidHashes operatorsUidHashes = + CustomSinkOperatorUidHashes.builder() + .setWriterUidHash(writerHash) + .setCommitterUidHash(committerHash) + .setGlobalCommitterUidHash(globalCommitterHash) + .build(); + src.sinkTo( + TestSink.newBuilder() + .setDefaultCommitter() + .setDefaultGlobalCommitter() + .build(), + operatorsUidHashes) + .name(NAME); + + final StreamGraph streamGraph = env.getStreamGraph(); + + assertEquals(findWriter(streamGraph).getUserHash(), writerHash); + assertEquals(findCommitter(streamGraph).getUserHash(), committerHash); + assertEquals(findGlobalCommitter(streamGraph).getUserHash(), globalCommitterHash); + } + private void validateTopology( StreamNode src, Class srcOutTypeInfo, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java index ce657c9fec14a..a5cf73879b445 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java @@ -23,6 +23,7 @@ import org.apache.flink.api.java.typeutils.InputTypeConfigurable; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.streaming.api.datastream.CustomSinkOperatorUidHashes; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -491,7 +492,9 @@ public Optional generateUid(String name) { final DataStream dataStream = new DataStream<>(env, sinkTransformation); final Transformation transformation = DataStreamSink.forSinkV1( - dataStream, ((SinkProvider) runtimeProvider).createSink()) + dataStream, + ((SinkProvider) runtimeProvider).createSink(), + CustomSinkOperatorUidHashes.DEFAULT) .getTransformation(); transformation.setParallelism(sinkParallelism); sinkMeta.fill(transformation); @@ -503,7 +506,9 @@ public Optional generateUid(String name) { final DataStream dataStream = new DataStream<>(env, sinkTransformation); final Transformation transformation = DataStreamSink.forSink( - dataStream, ((SinkV2Provider) runtimeProvider).createSink()) + dataStream, + ((SinkV2Provider) runtimeProvider).createSink(), + CustomSinkOperatorUidHashes.DEFAULT) .getTransformation(); transformation.setParallelism(sinkParallelism); sinkMeta.fill(transformation); From 3d0646836c149222cf9c2184dd70b83bacc42c16 Mon Sep 17 00:00:00 2001 From: Fabian Paul Date: Tue, 15 Mar 2022 13:13:02 +0100 Subject: [PATCH 025/258] [FLINK-26613][streaming] Use Flink 1.13 sink committer operator uid pattern Since there is no dedicated committer operator in Flink 1.14 it is safe to use the uid pattern of 1.13 to ease upgrades from Flink 1.13 to 1.15. --- .../SinkTransformationTranslator.java | 7 +++--- .../SinkTransformationTranslatorTest.java | 23 +++++++++++++++++++ 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java index 782cccaf64795..0dd608755cfc9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java @@ -355,9 +355,10 @@ private void concatUid( BiConsumer, String> setter, @Nullable String transformationName) { if (transformationName != null && getter.apply(transformation) != null) { - // Use the same uid pattern than for Sink V1 + // Use the same uid pattern than for Sink V1. We deliberately decided to use the uid + // pattern of Flink 1.13 because 1.14 did not have a dedicated committer operator. if (transformationName.equals(COMMITTER_NAME)) { - final String committerFormat = "Sink %s Committer"; + final String committerFormat = "Sink Committer: %s"; setter.accept( subTransformation, String.format(committerFormat, getter.apply(transformation))); @@ -369,7 +370,7 @@ private void concatUid( return; } - // Use the same uid pattern than for Sink V1 + // Use the same uid pattern than for Sink V1 in Flink 1.14. if (transformationName.equals( StandardSinkTopologies.GLOBAL_COMMITTER_TRANSFORMATION_NAME)) { final String committerFormat = "Sink %s Global Committer"; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java index 164a8c46bf967..e2086ec63ac38 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java @@ -293,6 +293,29 @@ public void testSettingOperatorUidHash() { assertEquals(findGlobalCommitter(streamGraph).getUserHash(), globalCommitterHash); } + /** + * When ever you need to change something in this test case please think about possible state + * upgrade problems introduced by your changes. + */ + @Test + public void testSettingOperatorUids() { + final String sinkUid = "f6b178ce445dc3ffaa06bad27a51fead"; + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + final DataStreamSource src = env.fromElements(1, 2); + src.sinkTo(TestSink.newBuilder().setDefaultCommitter().setDefaultGlobalCommitter().build()) + .name(NAME) + .uid(sinkUid); + + final StreamGraph streamGraph = env.getStreamGraph(); + assertEquals(findWriter(streamGraph).getTransformationUID(), sinkUid); + assertEquals( + findCommitter(streamGraph).getTransformationUID(), + String.format("Sink Committer: %s", sinkUid)); + assertEquals( + findGlobalCommitter(streamGraph).getTransformationUID(), + String.format("Sink %s Global Committer", sinkUid)); + } + private void validateTopology( StreamNode src, Class srcOutTypeInfo, From 63817b5ffdf7ba24a168aeec95464d13e4d78e13 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 17 Mar 2022 11:22:26 +0100 Subject: [PATCH 026/258] [FLINK-26680][coordination] Properly handle deleted jobs during recovery --- .../SessionDispatcherLeaderProcess.java | 14 ++-- .../SessionDispatcherLeaderProcessTest.java | 66 +++++++++++++++++++ 2 files changed, 76 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java index 6f9d0bf2fd377..63187002acce4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcess.java @@ -147,7 +147,7 @@ private Collection recoverJobs(Set recoveredDirtyJobResults) { for (JobID jobId : jobIds) { if (!recoveredDirtyJobResults.contains(jobId)) { - recoveredJobGraphs.add(recoverJob(jobId)); + tryRecoverJob(jobId).ifPresent(recoveredJobGraphs::add); } } @@ -164,10 +164,16 @@ private Collection getJobIds() { } } - private JobGraph recoverJob(JobID jobId) { + private Optional tryRecoverJob(JobID jobId) { log.info("Trying to recover job with job id {}.", jobId); try { - return jobGraphStore.recoverJobGraph(jobId); + final JobGraph jobGraph = jobGraphStore.recoverJobGraph(jobId); + if (jobGraph == null) { + log.info( + "Skipping recovery of job with job id {}, because it already finished in a previous execution", + jobId); + } + return Optional.ofNullable(jobGraph); } catch (Exception e) { throw new FlinkRuntimeException( String.format("Could not recover job with job id %s.", jobId), e); @@ -264,7 +270,7 @@ private DispatcherGateway getDispatcherGatewayInternal() { } private Optional recoverJobIfRunning(JobID jobId) { - return supplyUnsynchronizedIfRunning(() -> recoverJob(jobId)); + return supplyUnsynchronizedIfRunning(() -> tryRecoverJob(jobId)).flatMap(x -> x); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java index f48ca0697be9d..a90a245a7c2a6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/SessionDispatcherLeaderProcessTest.java @@ -46,6 +46,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import java.time.Duration; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -237,6 +238,71 @@ private void testJobRecovery( } } + @Test + public void testRecoveryWhileJobGraphRecoveryIsScheduledConcurrently() throws Exception { + final JobResult dirtyJobResult = + TestingJobResultStore.createSuccessfulJobResult(new JobID()); + + OneShotLatch recoveryInitiatedLatch = new OneShotLatch(); + OneShotLatch jobGraphAddedLatch = new OneShotLatch(); + + jobGraphStore = + TestingJobGraphStore.newBuilder() + // mimic behavior when recovering a JobGraph that is marked for deletion + .setRecoverJobGraphFunction((jobId, jobs) -> null) + .build(); + + jobResultStore = + TestingJobResultStore.builder() + .withGetDirtyResultsSupplier( + () -> { + recoveryInitiatedLatch.trigger(); + try { + jobGraphAddedLatch.await(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + return Collections.singleton(dirtyJobResult); + }) + .build(); + + final CompletableFuture> recoveredJobGraphsFuture = + new CompletableFuture<>(); + final CompletableFuture> recoveredDirtyJobResultsFuture = + new CompletableFuture<>(); + dispatcherServiceFactory = + (ignoredDispatcherId, + recoveredJobs, + recoveredDirtyJobResults, + ignoredJobGraphWriter, + ignoredJobResultStore) -> { + recoveredJobGraphsFuture.complete(recoveredJobs); + recoveredDirtyJobResultsFuture.complete(recoveredDirtyJobResults); + return TestingDispatcherGatewayService.newBuilder().build(); + }; + + try (final SessionDispatcherLeaderProcess dispatcherLeaderProcess = + createDispatcherLeaderProcess()) { + dispatcherLeaderProcess.start(); + + // start returns without the initial recovery being completed + // mimic ZK message about an added jobgraph while the recovery is ongoing + recoveryInitiatedLatch.await(); + dispatcherLeaderProcess.onAddedJobGraph(dirtyJobResult.getJobId()); + jobGraphAddedLatch.trigger(); + + assertThat(recoveredJobGraphsFuture) + .succeedsWithin(Duration.ofHours(1)) + .satisfies(recovedJobGraphs -> assertThat(recovedJobGraphs).isEmpty()); + assertThat(recoveredDirtyJobResultsFuture) + .succeedsWithin(Duration.ofHours(1)) + .satisfies( + recoveredDirtyJobResults -> + assertThat(recoveredDirtyJobResults) + .containsExactly(dirtyJobResult)); + } + } + @Test public void closeAsync_stopsJobGraphStoreAndDispatcher() throws Exception { final CompletableFuture jobGraphStopFuture = new CompletableFuture<>(); From ed0be9d621f388273875b0205852abb671706fed Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 17 Mar 2022 20:54:23 +0800 Subject: [PATCH 027/258] [hotfix][python][docs] Improve the documentation about how to specify jar files on Windows --- docs/content.zh/docs/dev/python/dependency_management.md | 6 ++++++ docs/content/docs/dev/python/dependency_management.md | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/docs/content.zh/docs/dev/python/dependency_management.md b/docs/content.zh/docs/dev/python/dependency_management.md index c8186865136b5..8665fee61bc47 100644 --- a/docs/content.zh/docs/dev/python/dependency_management.md +++ b/docs/content.zh/docs/dev/python/dependency_management.md @@ -53,6 +53,9 @@ If third-party JARs are used, you can specify the JARs in the Python Table API a # NOTE: Only local file URLs (start with "file://") are supported. table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") +# It looks like the following on Windows: +table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///E:/my/jar/path/connector.jar;file:///E:/my/jar/path/udf.jar") + # Specify a list of URLs via "pipeline.classpaths". The URLs are separated by ";" # and will be added to the classpath during job execution. # NOTE: The paths must specify a protocol (e.g. file://) and users should ensure that the URLs are accessible on both the client and the cluster. @@ -66,6 +69,9 @@ or in the Python DataStream API as following: # NOTE: Only local file URLs (start with "file://") are supported. stream_execution_environment.add_jars("file:///my/jar/path/connector1.jar", "file:///my/jar/path/connector2.jar") +# It looks like the following on Windows: +stream_execution_environment.add_jars("file:///E:/my/jar/path/connector1.jar", "file:///E:/my/jar/path/connector2.jar") + # Use the add_classpaths() to add the dependent jars URLs into the classpath. # The URLs will also be added to the classpath of both the client and the cluster. # NOTE: The paths must specify a protocol (e.g. file://) and users should ensure that the diff --git a/docs/content/docs/dev/python/dependency_management.md b/docs/content/docs/dev/python/dependency_management.md index e27ec80b00820..0fb08fad370e7 100644 --- a/docs/content/docs/dev/python/dependency_management.md +++ b/docs/content/docs/dev/python/dependency_management.md @@ -53,6 +53,9 @@ If third-party JARs are used, you can specify the JARs in the Python Table API a # NOTE: Only local file URLs (start with "file://") are supported. table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar;file:///my/jar/path/udf.jar") +# It looks like the following on Windows: +table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///E:/my/jar/path/connector.jar;file:///E:/my/jar/path/udf.jar") + # Specify a list of URLs via "pipeline.classpaths". The URLs are separated by ";" # and will be added to the classpath during job execution. # NOTE: The paths must specify a protocol (e.g. file://) and users should ensure that the URLs are accessible on both the client and the cluster. @@ -66,6 +69,9 @@ or in the Python DataStream API as following: # NOTE: Only local file URLs (start with "file://") are supported. stream_execution_environment.add_jars("file:///my/jar/path/connector1.jar", "file:///my/jar/path/connector2.jar") +# It looks like the following on Windows: +stream_execution_environment.add_jars("file:///E:/my/jar/path/connector1.jar", "file:///E:/my/jar/path/connector2.jar") + # Use the add_classpaths() to add the dependent jars URLs into the classpath. # The URLs will also be added to the classpath of both the client and the cluster. # NOTE: The paths must specify a protocol (e.g. file://) and users should ensure that the From 1e3344854b3bfd7b6f5498a0297ba49d55127003 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Thu, 10 Mar 2022 11:20:52 +0200 Subject: [PATCH 028/258] [FLINK-26194][table-api-java] Deprecate unused options in TableConfig Deprecate `nullCheck` and `decimalContext`. --- .../apache/flink/table/api/TableConfig.java | 31 ++++++++++++++++--- 1 file changed, 26 insertions(+), 5 deletions(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java index 9b6de0c668447..7b6f859576645 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java @@ -109,8 +109,12 @@ public final class TableConfig implements WritableConfig, ReadableConfig { // // The set() methods only impact the application-specific configuration. - /** Defines if all fields need to be checked for NULL first. */ - private Boolean nullCheck = true; + /** + * Defines if all fields need to be checked for NULL first. + * + * @deprecated This option is not used anymore and will be removed in next releases. + */ + @Deprecated private Boolean nullCheck = true; /** Defines the configuration of Planner for Table API and SQL queries. */ private PlannerConfig plannerConfig = PlannerConfig.EMPTY_CONFIG; @@ -118,8 +122,10 @@ public final class TableConfig implements WritableConfig, ReadableConfig { /** * Defines the default context for decimal division calculation. We use Scala's default * MathContext.DECIMAL128. + * + * @deprecated This option is not used anymore and will be removed in next releases. */ - private MathContext decimalContext = MathContext.DECIMAL128; + @Deprecated private MathContext decimalContext = MathContext.DECIMAL128; /** * A configuration object to hold all configuration that has been set specifically in the Table @@ -312,12 +318,21 @@ private void validateTimeZone(String zone) { } } - /** Returns the NULL check. If enabled, all fields need to be checked for NULL first. */ + /** + * Returns the NULL check. If enabled, all fields need to be checked for NULL first. + * + * @deprecated This option is not used anymore and will be removed in next releases. + */ + @Deprecated public Boolean getNullCheck() { return nullCheck; } - /** Sets the NULL check. If enabled, all fields need to be checked for NULL first. */ + /** + * Sets the NULL check. If enabled, all fields need to be checked for NULL first. + * + * @deprecated This option is not used anymore and will be removed in next releases. + */ public void setNullCheck(Boolean nullCheck) { this.nullCheck = Preconditions.checkNotNull(nullCheck); } @@ -338,7 +353,10 @@ public void setPlannerConfig(PlannerConfig plannerConfig) { /** * Returns the default context for decimal division calculation. {@link * java.math.MathContext#DECIMAL128} by default. + * + * @deprecated This option is not used anymore and will be removed in next releases. */ + @Deprecated public MathContext getDecimalContext() { return decimalContext; } @@ -346,7 +364,10 @@ public MathContext getDecimalContext() { /** * Sets the default context for decimal division calculation. {@link * java.math.MathContext#DECIMAL128} by default. + * + * @deprecated This option is not used anymore and will be removed in next releases. */ + @Deprecated public void setDecimalContext(MathContext decimalContext) { this.decimalContext = Preconditions.checkNotNull(decimalContext); } From 265788bf7a6f12f5c6b417326cb60437a5f313d6 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 17 Mar 2022 09:37:31 +0100 Subject: [PATCH 029/258] [FLINK-26698][runtime] Uses the actual basePath instance instead of only the path of Path instance The issue before the fix was, that using getPath would strip off the scheme information which causes problems in situations where the FileSystem is not the default FileSystem --- .../FileSystemJobResultStore.java | 9 +++++-- .../FileSystemJobResultStoreTestInternal.java | 25 +++++++++++++++++-- 2 files changed, 30 insertions(+), 4 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java index a7ed3c5d4fc53..5c19c2e980906 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java @@ -114,7 +114,7 @@ public static String createDefaultJobResultStorePath(String baseDir, String clus * @return A path for a dirty entry for the given the Job ID. */ private Path constructDirtyPath(JobID jobId) { - return new Path(this.basePath.getPath(), jobId.toString() + DIRTY_FILE_EXTENSION); + return constructEntryPath(jobId.toString() + DIRTY_FILE_EXTENSION); } /** @@ -125,7 +125,12 @@ private Path constructDirtyPath(JobID jobId) { * @return A path for a clean entry for the given the Job ID. */ private Path constructCleanPath(JobID jobId) { - return new Path(this.basePath.getPath(), jobId.toString() + ".json"); + return constructEntryPath(jobId.toString() + ".json"); + } + + @VisibleForTesting + Path constructEntryPath(String fileName) { + return new Path(this.basePath, fileName); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreTestInternal.java b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreTestInternal.java index 3a2de32559d7c..20d6f8a4c2994 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreTestInternal.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStoreTestInternal.java @@ -51,10 +51,31 @@ public class FileSystemJobResultStoreTestInternal { @TempDir File temporaryFolder; + private Path basePath; + @BeforeEach public void setupTest() throws IOException { - Path path = new Path(temporaryFolder.toURI()); - fileSystemJobResultStore = new FileSystemJobResultStore(path.getFileSystem(), path, false); + basePath = new Path(temporaryFolder.toURI()); + fileSystemJobResultStore = + new FileSystemJobResultStore(basePath.getFileSystem(), basePath, false); + } + + @Test + public void testValidEntryPathCreation() { + final Path entryParent = + fileSystemJobResultStore.constructEntryPath("random-name").getParent(); + assertThat(entryParent) + .extracting(FileSystemJobResultStoreTestInternal::stripSucceedingSlash) + .isEqualTo(stripSucceedingSlash(basePath)); + } + + private static String stripSucceedingSlash(Path path) { + final String uriStr = path.toUri().toString(); + if (uriStr.charAt(uriStr.length() - 1) == '/') { + return uriStr.substring(0, uriStr.length() - 1); + } + + return uriStr; } @Test From 604668e0ca3fbac43eb921f2bc80c07e28e7859f Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 17 Mar 2022 09:49:31 +0100 Subject: [PATCH 030/258] [hotfix][runtime] Makes use of static variable --- .../runtime/highavailability/FileSystemJobResultStore.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java index 5c19c2e980906..5f05180c05c75 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java @@ -125,7 +125,7 @@ private Path constructDirtyPath(JobID jobId) { * @return A path for a clean entry for the given the Job ID. */ private Path constructCleanPath(JobID jobId) { - return constructEntryPath(jobId.toString() + ".json"); + return constructEntryPath(jobId.toString() + FILE_EXTENSION); } @VisibleForTesting From b07209413f307eabe000dc01155cfe33a6cce680 Mon Sep 17 00:00:00 2001 From: Jing Ge Date: Mon, 14 Mar 2022 14:58:37 +0100 Subject: [PATCH 031/258] [FLINK-26604][doc] add more information for Avro records support and clean up redundant content of bounded and unbounded data. - mvn dependency - using namespace in schema for reflect records [FLINK-26604][doc] bug fix (cherry picked from commit 579e554393e858bd7c2faf55aa4531f853c4ebe5). This closes #19134 --- .../connectors/datastream/formats/parquet.md | 240 ++++++++++------- .../connectors/datastream/formats/parquet.md | 242 +++++++++++------- 2 files changed, 287 insertions(+), 195 deletions(-) diff --git a/docs/content.zh/docs/connectors/datastream/formats/parquet.md b/docs/content.zh/docs/connectors/datastream/formats/parquet.md index 46828e50bd905..87c1a1b3dd794 100644 --- a/docs/content.zh/docs/connectors/datastream/formats/parquet.md +++ b/docs/content.zh/docs/connectors/datastream/formats/parquet.md @@ -30,7 +30,7 @@ under the License. Flink supports reading [Parquet](https://parquet.apache.org/) files, producing {{< javadoc file="org/apache/flink/table/data/RowData.html" name="Flink RowData">}} and producing [Avro](https://avro.apache.org/) records. -To use the format you need to add the Flink Parquet dependency to your project: +To use the format you need to add the `flink-parquet` dependency to your project: ```xml @@ -40,45 +40,77 @@ To use the format you need to add the Flink Parquet dependency to your project: ``` -This format is compatible with the new Source that can be used in both batch and streaming modes. +To read Avro records, you will need to add the `parquet-avro` dependency: + +```xml + + org.apache.parquet + parquet-avro + 1.12.2 + true + + + org.apache.hadoop + hadoop-client + + + it.unimi.dsi + fastutil + + + +``` + +This format is compatible with the new Source that can be used in both batch and streaming execution modes. Thus, you can use this format for two kinds of data: -- Bounded data -- Unbounded data: monitors a directory for new files that appear -## Flink RowData +- Bounded data: lists all files and reads them all. +- Unbounded data: monitors a directory for new files that appear. -#### Bounded data example +{{< hint info >}} +When you start a File Source it is configured for bounded data by default. +To configure the File Source for unbounded data, you must additionally call +`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`. +{{< /hint >}} -In this example, you will create a DataStream containing Parquet records as Flink RowDatas. The schema is projected to read only the specified fields ("f7", "f4" and "f99"). -Flink will read records in batches of 500 records. The first boolean parameter specifies that timestamp columns will be interpreted as UTC. -The second boolean instructs the application that the projected Parquet fields names are case-sensitive. -There is no watermark strategy defined as records do not contain event timestamps. +**Vectorized reader** ```java -final LogicalType[] fieldTypes = - new LogicalType[] { - new DoubleType(), new IntType(), new VarCharType() - }; -final ParquetColumnarRowInputFormat format = - new ParquetColumnarRowInputFormat<>( - new Configuration(), - RowType.of(fieldTypes, new String[] {"f7", "f4", "f99"}), - 500, - false, - true); -final FileSource source = - FileSource.forBulkFileFormat(format, /* Flink Path */) - .build(); -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); +// Parquet rows are decoded in batches +FileSource.forBulkFileFormat(BulkFormat,Path...) + +// Monitor the Paths to read data as unbounded data +FileSource.forBulkFileFormat(BulkFormat,Path...) + .monitorContinuously(Duration.ofMillis(5L)) + .build(); + +``` + +**Avro Parquet reader** + +```java + +// Parquet rows are decoded in batches +FileSource.forRecordStreamFormat(StreamFormat,Path...) + +// Monitor the Paths to read data as unbounded data +FileSource.forRecordStreamFormat(StreamFormat,Path...) + .monitorContinuously(Duration.ofMillis(5L)) + .build(); + + ``` -#### Unbounded data example +{{< hint info >}} +Following examples are all configured for bounded data. +To configure the File Source for unbounded data, you must additionally call +`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`. +{{< /hint >}} -In this example, you will create a DataStream containing Parquet records as Flink RowDatas that will -infinitely grow as new files are added to the directory. It will monitor for new files each second. -The schema is projected to read only the specified fields ("f7", "f4" and "f99"). +## Flink RowData + +In this example, you will create a DataStream containing Parquet records as Flink RowDatas. The schema is projected to read only the specified fields ("f7", "f4" and "f99"). Flink will read records in batches of 500 records. The first boolean parameter specifies that timestamp columns will be interpreted as UTC. The second boolean instructs the application that the projected Parquet fields names are case-sensitive. There is no watermark strategy defined as records do not contain event timestamps. @@ -98,7 +130,6 @@ final ParquetColumnarRowInputFormat format = true); final FileSource source = FileSource.forBulkFileFormat(format, /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) .build(); final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); @@ -131,9 +162,7 @@ This example uses an Avro schema example similar to the one described in the [of This schema defines a record representing a user with three fields: name, favoriteNumber, and favoriteColor. You can find more details at [record specification](https://avro.apache.org/docs/1.10.0/spec.html#schema_record) for how to define an Avro schema. -#### Bounded data example - -In this example, you will create a DataStream containing Parquet records as Avro Generic records. +In the following example, you will create a DataStream containing Parquet records as Avro Generic records. It will parse the Avro schema based on the JSON string. There are many other ways to parse a schema, e.g. from java.io.File or java.io.InputStream. Please refer to [Avro Schema](https://avro.apache.org/docs/1.10.0/api/java/org/apache/avro/Schema.html) for details. After that, you will create an `AvroParquetRecordFormat` via `AvroParquetReaders` for Avro Generic records. @@ -163,37 +192,6 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example - -This example is similar to the bounded batch example. The application monitors for new files every second -and reads Avro Generic records from Parquet files infinitely as new files are added to the directory. -```java -// parsing avro schema -final Schema schema = - new Schema.Parser() - .parse( - "{\"type\": \"record\", " - + "\"name\": \"User\", " - + "\"fields\": [\n" - + " {\"name\": \"name\", \"type\": \"string\" },\n" - + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" - + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" - + " ]\n" - + " }"); - -final FileSource source = - FileSource.forRecordStreamFormat( - AvroParquetReaders.forGenericRecord(schema), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) - .build(); - -final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.enableCheckpointing(10L); - -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); -``` - ### Specific record Based on the previously defined schema, you can generate classes by leveraging Avro code generation. @@ -202,9 +200,7 @@ You can either use `avro-tools.jar` to generate code manually or you could use t code generation on any .avsc files present in the configured source directory. Please refer to [Avro Getting Started](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html) for more information. -#### Bounded data example - -This example uses the example schema [testdata.avsc](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc): +The following example uses the example schema [testdata.avsc](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc): ```json lines [ @@ -247,26 +243,6 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example - -This example, similar to the bounded batch example, uses the same generated Address Java class -and monitors for the new files every second to read Avro Specific records from Parquet files -infinitely as new files are added to the directory. - -```java -final FileSource source = - FileSource.forRecordStreamFormat( - AvroParquetReaders.forSpecificRecord(Address.class), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) - .build(); - -final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.enableCheckpointing(10L); - -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); -``` - ### Reflect record Beyond Avro Generic and Specific record that requires a predefined Avro schema, @@ -274,8 +250,6 @@ Flink also supports creating a DataStream from Parquet files based on existing J In this case, Avro will use Java reflection to generate schemas and protocols for these POJO classes. Java types are mapped to Avro schemas, please refer to the [Avro reflect](https://avro.apache.org/docs/1.10.0/api/java/index.html) documentation for more details. -#### Bounded data example - This example uses a simple Java POJO class [Datum](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/Datum.java): ```java @@ -329,22 +303,94 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example +#### Prerequisite for Parquet files + +In order to support reading Avro reflect records, the Parquet file must contain specific meta information. +The Avro schema used for creating the Parquet data must contain a `namespace`, +which will be used by the program to identify the concrete Java class for the reflection process. + +The following example shows the `User` schema used previously. But this time it contains a namespace +pointing to the location(in this case the package), where the `User` class for the reflection could be found. + +```java +// avro schema with namespace +final String schema = + "{\"type\": \"record\", " + + "\"name\": \"User\", " + + "\"namespace\": \"org.apache.flink.formats.parquet.avro\", " + + "\"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\" },\n" + + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" + + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" + + " ]\n" + + " }"; + +``` + +Parquet files created with this schema will contain meta information like: -This example, similar to the bounded batch example, uses the same POJO Java class `Datum` -and monitors for the new files every second to read Avro Reflect records from Parquet files -infinitely as new files are added to the directory. +```text +creator: parquet-mr version 1.12.2 (build 77e30c8093386ec52c3cfa6c34b7ef3321322c94) +extra: parquet.avro.schema = +{"type":"record","name":"User","namespace":"org.apache.flink.formats.parquet.avro","fields":[{"name":"name","type":"string"},{"name":"favoriteNumber","type":["int","null"]},{"name":"favoriteColor","type":["string","null"]}]} +extra: writer.model.name = avro + +file schema: org.apache.flink.formats.parquet.avro.User +-------------------------------------------------------------------------------- +name: REQUIRED BINARY L:STRING R:0 D:0 +favoriteNumber: OPTIONAL INT32 R:0 D:1 +favoriteColor: OPTIONAL BINARY L:STRING R:0 D:1 + +row group 1: RC:3 TS:143 OFFSET:4 +-------------------------------------------------------------------------------- +name: BINARY UNCOMPRESSED DO:0 FPO:4 SZ:47/47/1.00 VC:3 ENC:PLAIN,BIT_PACKED ST:[min: Jack, max: Tom, num_nulls: 0] +favoriteNumber: INT32 UNCOMPRESSED DO:0 FPO:51 SZ:41/41/1.00 VC:3 ENC:RLE,PLAIN,BIT_PACKED ST:[min: 1, max: 3, num_nulls: 0] +favoriteColor: BINARY UNCOMPRESSED DO:0 FPO:92 SZ:55/55/1.00 VC:3 ENC:RLE,PLAIN,BIT_PACKED ST:[min: green, max: yellow, num_nulls: 0] + +``` + +With the `User` class defined in the package org.apache.flink.formats.parquet.avro: + +```java +public class User { + private String name; + private Integer favoriteNumber; + private String favoriteColor; + + public User() {} + + public User(String name, Integer favoriteNumber, String favoriteColor) { + this.name = name; + this.favoriteNumber = favoriteNumber; + this.favoriteColor = favoriteColor; + } + + public String getName() { + return name; + } + + public Integer getFavoriteNumber() { + return favoriteNumber; + } + + public String getFavoriteColor() { + return favoriteColor; + } + } + +``` + +you can write the following program to read Avro Reflect records of User type from parquet files: ```java final FileSource source = FileSource.forRecordStreamFormat( - AvroParquetReaders.forReflectRecord(Datum.class), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) + AvroParquetReaders.forReflectRecord(User.class), /* Flink Path */) .build(); final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(10L); - + final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` diff --git a/docs/content/docs/connectors/datastream/formats/parquet.md b/docs/content/docs/connectors/datastream/formats/parquet.md index 71ed416f4da68..8a6f9f9971d0e 100644 --- a/docs/content/docs/connectors/datastream/formats/parquet.md +++ b/docs/content/docs/connectors/datastream/formats/parquet.md @@ -30,7 +30,7 @@ under the License. Flink supports reading [Parquet](https://parquet.apache.org/) files, producing {{< javadoc file="org/apache/flink/table/data/RowData.html" name="Flink RowData">}} and producing [Avro](https://avro.apache.org/) records. -To use the format you need to add the Flink Parquet dependency to your project: +To use the format you need to add the `flink-parquet` dependency to your project: ```xml @@ -39,46 +39,78 @@ To use the format you need to add the Flink Parquet dependency to your project: {{< version >}} ``` - -This format is compatible with the new Source that can be used in both batch and streaming modes. + +To read Avro records, you will need to add the `parquet-avro` dependency: + +```xml + + org.apache.parquet + parquet-avro + 1.12.2 + true + + + org.apache.hadoop + hadoop-client + + + it.unimi.dsi + fastutil + + + +``` + +This format is compatible with the new Source that can be used in both batch and streaming execution modes. Thus, you can use this format for two kinds of data: -- Bounded data -- Unbounded data: monitors a directory for new files that appear -## Flink RowData +- Bounded data: lists all files and reads them all. +- Unbounded data: monitors a directory for new files that appear. -#### Bounded data example +{{< hint info >}} +When you start a File Source it is configured for bounded data by default. +To configure the File Source for unbounded data, you must additionally call +`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`. +{{< /hint >}} -In this example, you will create a DataStream containing Parquet records as Flink RowDatas. The schema is projected to read only the specified fields ("f7", "f4" and "f99"). -Flink will read records in batches of 500 records. The first boolean parameter specifies that timestamp columns will be interpreted as UTC. -The second boolean instructs the application that the projected Parquet fields names are case-sensitive. -There is no watermark strategy defined as records do not contain event timestamps. +**Vectorized reader** ```java -final LogicalType[] fieldTypes = - new LogicalType[] { - new DoubleType(), new IntType(), new VarCharType() - }; -final ParquetColumnarRowInputFormat format = - new ParquetColumnarRowInputFormat<>( - new Configuration(), - RowType.of(fieldTypes, new String[] {"f7", "f4", "f99"}), - 500, - false, - true); -final FileSource source = - FileSource.forBulkFileFormat(format, /* Flink Path */) - .build(); -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); +// Parquet rows are decoded in batches +FileSource.forBulkFileFormat(BulkFormat,Path...) + +// Monitor the Paths to read data as unbounded data +FileSource.forBulkFileFormat(BulkFormat,Path...) + .monitorContinuously(Duration.ofMillis(5L)) + .build(); + ``` -#### Unbounded data example +**Avro Parquet reader** -In this example, you will create a DataStream containing Parquet records as Flink RowDatas that will -infinitely grow as new files are added to the directory. It will monitor for new files each second. -The schema is projected to read only the specified fields ("f7", "f4" and "f99"). +```java + +// Parquet rows are decoded in batches +FileSource.forRecordStreamFormat(StreamFormat,Path...) + +// Monitor the Paths to read data as unbounded data +FileSource.forRecordStreamFormat(StreamFormat,Path...) + .monitorContinuously(Duration.ofMillis(5L)) + .build(); + + +``` + +{{< hint info >}} +Following examples are all configured for bounded data. +To configure the File Source for unbounded data, you must additionally call +`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`. +{{< /hint >}} + +## Flink RowData + +In this example, you will create a DataStream containing Parquet records as Flink RowDatas. The schema is projected to read only the specified fields ("f7", "f4" and "f99"). Flink will read records in batches of 500 records. The first boolean parameter specifies that timestamp columns will be interpreted as UTC. The second boolean instructs the application that the projected Parquet fields names are case-sensitive. There is no watermark strategy defined as records do not contain event timestamps. @@ -98,7 +130,6 @@ final ParquetColumnarRowInputFormat format = true); final FileSource source = FileSource.forBulkFileFormat(format, /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) .build(); final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); @@ -131,9 +162,7 @@ This example uses an Avro schema example similar to the one described in the [of This schema defines a record representing a user with three fields: name, favoriteNumber, and favoriteColor. You can find more details at [record specification](https://avro.apache.org/docs/1.10.0/spec.html#schema_record) for how to define an Avro schema. -#### Bounded data example - -In this example, you will create a DataStream containing Parquet records as Avro Generic records. +In the following example, you will create a DataStream containing Parquet records as Avro Generic records. It will parse the Avro schema based on the JSON string. There are many other ways to parse a schema, e.g. from java.io.File or java.io.InputStream. Please refer to [Avro Schema](https://avro.apache.org/docs/1.10.0/api/java/org/apache/avro/Schema.html) for details. After that, you will create an `AvroParquetRecordFormat` via `AvroParquetReaders` for Avro Generic records. @@ -163,37 +192,6 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example - -This example is similar to the bounded batch example. The application monitors for new files every second -and reads Avro Generic records from Parquet files infinitely as new files are added to the directory. -```java -// parsing avro schema -final Schema schema = - new Schema.Parser() - .parse( - "{\"type\": \"record\", " - + "\"name\": \"User\", " - + "\"fields\": [\n" - + " {\"name\": \"name\", \"type\": \"string\" },\n" - + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" - + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" - + " ]\n" - + " }"); - -final FileSource source = - FileSource.forRecordStreamFormat( - AvroParquetReaders.forGenericRecord(schema), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) - .build(); - -final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.enableCheckpointing(10L); - -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); -``` - ### Specific record Based on the previously defined schema, you can generate classes by leveraging Avro code generation. @@ -202,9 +200,7 @@ You can either use `avro-tools.jar` to generate code manually or you could use t code generation on any .avsc files present in the configured source directory. Please refer to [Avro Getting Started](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html) for more information. -#### Bounded data example - -This example uses the example schema [testdata.avsc](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc): +The following example uses the example schema [testdata.avsc](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc): ```json lines [ @@ -247,26 +243,6 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example - -This example, similar to the bounded batch example, uses the same generated Address Java class -and monitors for the new files every second to read Avro Specific records from Parquet files -infinitely as new files are added to the directory. - -```java -final FileSource source = - FileSource.forRecordStreamFormat( - AvroParquetReaders.forSpecificRecord(Address.class), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) - .build(); - -final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.enableCheckpointing(10L); - -final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); -``` - ### Reflect record Beyond Avro Generic and Specific record that requires a predefined Avro schema, @@ -274,8 +250,6 @@ Flink also supports creating a DataStream from Parquet files based on existing J In this case, Avro will use Java reflection to generate schemas and protocols for these POJO classes. Java types are mapped to Avro schemas, please refer to the [Avro reflect](https://avro.apache.org/docs/1.10.0/api/java/index.html) documentation for more details. -#### Bounded data example - This example uses a simple Java POJO class [Datum](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/Datum.java): ```java @@ -329,22 +303,94 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Unbounded data example +#### Prerequisite for Parquet files + +In order to support reading Avro reflect records, the Parquet file must contain specific meta information. +The Avro schema used for creating the Parquet data must contain a `namespace`, +which will be used by the program to identify the concrete Java class for the reflection process. + +The following example shows the `User` schema used previously. But this time it contains a namespace +pointing to the location(in this case the package), where the `User` class for the reflection could be found. + +```java +// avro schema with namespace +final String schema = + "{\"type\": \"record\", " + + "\"name\": \"User\", " + + "\"namespace\": \"org.apache.flink.formats.parquet.avro\", " + + "\"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\" },\n" + + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" + + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" + + " ]\n" + + " }"; + +``` + +Parquet files created with this schema will contain meta information like: -This example, similar to the bounded batch example, uses the same POJO Java class `Datum` -and monitors for the new files every second to read Avro Reflect records from Parquet files -infinitely as new files are added to the directory. +```text +creator: parquet-mr version 1.12.2 (build 77e30c8093386ec52c3cfa6c34b7ef3321322c94) +extra: parquet.avro.schema = +{"type":"record","name":"User","namespace":"org.apache.flink.formats.parquet.avro","fields":[{"name":"name","type":"string"},{"name":"favoriteNumber","type":["int","null"]},{"name":"favoriteColor","type":["string","null"]}]} +extra: writer.model.name = avro + +file schema: org.apache.flink.formats.parquet.avro.User +-------------------------------------------------------------------------------- +name: REQUIRED BINARY L:STRING R:0 D:0 +favoriteNumber: OPTIONAL INT32 R:0 D:1 +favoriteColor: OPTIONAL BINARY L:STRING R:0 D:1 + +row group 1: RC:3 TS:143 OFFSET:4 +-------------------------------------------------------------------------------- +name: BINARY UNCOMPRESSED DO:0 FPO:4 SZ:47/47/1.00 VC:3 ENC:PLAIN,BIT_PACKED ST:[min: Jack, max: Tom, num_nulls: 0] +favoriteNumber: INT32 UNCOMPRESSED DO:0 FPO:51 SZ:41/41/1.00 VC:3 ENC:RLE,PLAIN,BIT_PACKED ST:[min: 1, max: 3, num_nulls: 0] +favoriteColor: BINARY UNCOMPRESSED DO:0 FPO:92 SZ:55/55/1.00 VC:3 ENC:RLE,PLAIN,BIT_PACKED ST:[min: green, max: yellow, num_nulls: 0] + +``` + +With the `User` class defined in the package org.apache.flink.formats.parquet.avro: + +```java +public class User { + private String name; + private Integer favoriteNumber; + private String favoriteColor; + + public User() {} + + public User(String name, Integer favoriteNumber, String favoriteColor) { + this.name = name; + this.favoriteNumber = favoriteNumber; + this.favoriteColor = favoriteColor; + } + + public String getName() { + return name; + } + + public Integer getFavoriteNumber() { + return favoriteNumber; + } + + public String getFavoriteColor() { + return favoriteColor; + } + } + +``` + +you can write the following program to read Avro Reflect records of User type from parquet files: ```java final FileSource source = FileSource.forRecordStreamFormat( - AvroParquetReaders.forReflectRecord(Datum.class), /* Flink Path */) - .monitorContinuously(Duration.ofSeconds(1L)) + AvroParquetReaders.forReflectRecord(User.class), /* Flink Path */) .build(); final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(10L); - + final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` From e0dfcda7112492433da016734f376e4e11d23af5 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Fri, 18 Mar 2022 16:49:46 +0800 Subject: [PATCH 032/258] Update for 1.15 --- docs/config.toml | 23 ++++++++++--------- .../test-scripts/common_docker.sh | 2 +- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/docs/config.toml b/docs/config.toml index 8dfb6e3afbff6..1025c3f0c3b83 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -14,7 +14,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -baseURL = '//nightlies.apache.org/flink/flink-docs-master' +baseURL = '//nightlies.apache.org/flink/flink-docs-release-1.15' languageCode = "en-us" title = "Apache Flink" enableGitInfo = false @@ -24,7 +24,7 @@ pygmentsUseClasses = true [params] # Flag whether this is a stable version or not. # Used for the quickstart page. - IsStable = false + IsStable = true # Flag to indicate whether an outdated warning should be shown. ShowOutDatedWarning = false @@ -34,14 +34,14 @@ pygmentsUseClasses = true # we change the version for the complete docs when forking of a release branch # etc. # The full version string as referenced in Maven (e.g. 1.2.1) - Version = "1.15-SNAPSHOT" + Version = "1.15.0" # For stable releases, leave the bugfix version out (e.g. 1.2). For snapshot # release this should be the same as the regular version - VersionTitle = "1.15-SNAPSHOT" + VersionTitle = "1.15" # The branch for this version of Apache Flink - Branch = "master" + Branch = "release-1.15" # The github repository for Apache Flink Repo = "//github.com/apache/flink" @@ -60,22 +60,23 @@ pygmentsUseClasses = true ZhDownloadPage = "//flink.apache.org/zh/downloads.html" - JavaDocs = "//nightlies.apache.org/flink/flink-docs-master/api/java/" + JavaDocs = "//nightlies.apache.org/flink/flink-docs-release-1.15/api/java/" - ScalaDocs = "//nightlies.apache.org/flink/flink-docs-master/api/scala/index.html#org.apache.flink.api.scala.package" + ScalaDocs = "//nightlies.apache.org/flink/flink-docs-release-1.15/api/scala/index.html#org.apache.flink.api.scala.package" - PyDocs = "//nightlies.apache.org/flink/flink-docs-master/api/python/" + PyDocs = "//nightlies.apache.org/flink/flink-docs-release-1.15/api/python/" # External links at the bottom # of the menu MenuLinks = [ ["Project Homepage", "//flink.apache.org"], - ["JavaDocs", "//nightlies.apache.org/flink/flink-docs-master/api/java/"], - ["ScalaDocs", "//nightlies.apache.org/flink/flink-docs-master/api/scala/index.html#org.apache.flink.api.scala.package"], - ["PyDocs", "//nightlies.apache.org/flink/flink-docs-master/api/python/"] + ["JavaDocs", "//nightlies.apache.org/flink/flink-docs-release-1.15/api/java/"], + ["ScalaDocs", "//nightlies.apache.org/flink/flink-docs-release-1.15/api/scala/index.html#org.apache.flink.api.scala.package/"] + ["PyDocs", "//nightlies.apache.org/flink/flink-docs-release-1.15/api/python/"] ] PreviousDocs = [ + ["1.15", "http://nightlies.apache.org/flink/flink-docs-release-1.15"], ["1.14", "http://nightlies.apache.org/flink/flink-docs-release-1.14"], ["1.13", "http://nightlies.apache.org/flink/flink-docs-release-1.13"], ["1.12", "http://nightlies.apache.org/flink/flink-docs-release-1.12"], diff --git a/flink-end-to-end-tests/test-scripts/common_docker.sh b/flink-end-to-end-tests/test-scripts/common_docker.sh index 44040eee52a48..cd97d81419688 100644 --- a/flink-end-to-end-tests/test-scripts/common_docker.sh +++ b/flink-end-to-end-tests/test-scripts/common_docker.sh @@ -48,7 +48,7 @@ function build_image() { local server_pid=$! echo "Preparing Dockeriles" - retry_times_with_exponential_backoff 5 git clone https://github.com/apache/flink-docker.git --branch dev-master --single-branch + retry_times_with_exponential_backoff 5 git clone https://github.com/apache/flink-docker.git --branch dev-1.15 --single-branch local java_version=8 if [[ ${PROFILE} == *"jdk11"* ]]; then From 70c4ce9a195c967822065d4fc0b54d48e9ae6a4d Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Fri, 18 Mar 2022 21:52:12 +0800 Subject: [PATCH 033/258] [hotfix][release] Fix the broken doc config and the script to generate the change --- docs/config.toml | 2 +- tools/releasing/create_snapshot_branch.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/config.toml b/docs/config.toml index 1025c3f0c3b83..ec0aec5e6de6a 100644 --- a/docs/config.toml +++ b/docs/config.toml @@ -71,7 +71,7 @@ pygmentsUseClasses = true MenuLinks = [ ["Project Homepage", "//flink.apache.org"], ["JavaDocs", "//nightlies.apache.org/flink/flink-docs-release-1.15/api/java/"], - ["ScalaDocs", "//nightlies.apache.org/flink/flink-docs-release-1.15/api/scala/index.html#org.apache.flink.api.scala.package/"] + ["ScalaDocs", "//nightlies.apache.org/flink/flink-docs-release-1.15/api/scala/index.html#org.apache.flink.api.scala.package/"], ["PyDocs", "//nightlies.apache.org/flink/flink-docs-release-1.15/api/python/"] ] diff --git a/tools/releasing/create_snapshot_branch.sh b/tools/releasing/create_snapshot_branch.sh index c3d86747299a0..a010001bcc560 100755 --- a/tools/releasing/create_snapshot_branch.sh +++ b/tools/releasing/create_snapshot_branch.sh @@ -55,7 +55,7 @@ perl -pi -e "s#^baseURL = .*#baseURL = \'${url_base}${SHORT_RELEASE_VERSION}\'#" perl -pi -e "s#^ JavaDocs = .*# JavaDocs = \"${url_base}${SHORT_RELEASE_VERSION}/api/java/\"#" ${config_file} perl -pi -e "s#^ \[\"JavaDocs\", .*# \[\"JavaDocs\", \"${url_base}${SHORT_RELEASE_VERSION}/api/java/\"\],#" ${config_file} perl -pi -e "s#^ ScalaDocs = .*# ScalaDocs = \"${url_base}${SHORT_RELEASE_VERSION}/api/scala/index.html\#org.apache.flink.api.scala.package\"#" ${config_file} -perl -pi -e "s#^ \[\"ScalaDocs\", .*# \[\"ScalaDocs\", \"${url_base}${SHORT_RELEASE_VERSION}/api/scala/index.html\#org.apache.flink.api.scala.package/\"\]#" ${config_file} +perl -pi -e "s#^ \[\"ScalaDocs\", .*# \[\"ScalaDocs\", \"${url_base}${SHORT_RELEASE_VERSION}/api/scala/index.html\#org.apache.flink.api.scala.package/\"\],#" ${config_file} perl -pi -e "s#^ PyDocs = .*# PyDocs = \"${url_base}${SHORT_RELEASE_VERSION}/api/python/\"#" ${config_file} perl -pi -e "s#^ \[\"PyDocs\", .*# \[\"PyDocs\", \"${url_base}${SHORT_RELEASE_VERSION}/api/python/\"\]#" ${config_file} From 3966744a18d9f0e408474969c1f61ea43c8b41d5 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 16 Mar 2022 18:09:01 +0100 Subject: [PATCH 034/258] [FLINK-26690][runtime] Makes globalCleanupAsync call the removal even if the JobGraph is not put into the JobGraphStore, yet This can happen if cleanup is triggered after a failover of a dirty JobResultStore entry (i.e. of a globally-terminated job). In that case, no recovery of the JobGraph happens and, therefore, no JobGraph is added to the internal addedJobGraphs collection. This required KubernetesStateHandleStore.releaseAndTryRemove to work for non-existing state as well. The ZooKeeperStateHandleStore implementation is already idempotent in this matter. ZooKeeperStateHandleStore.releaseAndTryRemove already works like that. --- .../KubernetesStateHandleStore.java | 9 +++++-- .../KubernetesStateHandleStoreTest.java | 27 +++++++++++++++++++ .../jobmanager/DefaultJobGraphStore.java | 8 +++--- .../runtime/persistence/StateHandleStore.java | 3 ++- .../jobmanager/DefaultJobGraphStoreTest.java | 27 ++++++++++++------- .../ZooKeeperStateHandleStoreTest.java | 12 +++++++++ 6 files changed, 69 insertions(+), 17 deletions(-) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java index cc7153d16b79a..0716b58ec3489 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStore.java @@ -51,6 +51,7 @@ import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Predicate; @@ -464,13 +465,14 @@ public Collection getAllHandles() throws Exception { * It returns the {@link RetrievableStateHandle} stored under the given state node if any. * * @param key Key to be removed from ConfigMap - * @return True if the state handle is removed successfully + * @return True if the state handle isn't listed anymore. * @throws Exception if removing the key or discarding the state failed */ @Override public boolean releaseAndTryRemove(String key) throws Exception { checkNotNull(key, "Key in ConfigMap."); final AtomicReference> stateHandleRefer = new AtomicReference<>(); + final AtomicBoolean stateHandleDoesNotExist = new AtomicBoolean(false); return updateConfigMap( configMap -> { final String content = configMap.getData().get(key); @@ -496,6 +498,8 @@ public boolean releaseAndTryRemove(String key) throws Exception { Objects.requireNonNull(configMap.getData().remove(key)); } return Optional.of(configMap); + } else { + stateHandleDoesNotExist.set(true); } return Optional.empty(); }) @@ -516,7 +520,8 @@ public boolean releaseAndTryRemove(String key) throws Exception { throw new CompletionException(e); } } - return CompletableFuture.completedFuture(updated); + return CompletableFuture.completedFuture( + stateHandleDoesNotExist.get() || updated); }) .get(); } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java index 9896d49512d2d..51294f40b810b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java @@ -803,6 +803,33 @@ public void testRemove() throws Exception { }; } + @Test + public void testRemoveOfNonExistingState() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final KubernetesStateHandleStore< + TestingLongStateHandleHelper.LongStateHandle> + store = + new KubernetesStateHandleStore<>( + flinkKubeClient, + LEADER_CONFIGMAP_NAME, + longStateStorage, + filter, + LOCK_IDENTITY); + assertThat(store.getAllAndLock().size(), is(0)); + assertThat(store.releaseAndTryRemove(key), is(true)); + assertThat(store.getAllAndLock().size(), is(0)); + + assertThat(TestingLongStateHandleHelper.getGlobalDiscardCount(), is(0)); + }); + } + }; + } + @Test public void testRemoveFailedShouldNotDiscardState() throws Exception { new Context() { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java index fda4964c31f88..e66a1729f82f0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStore.java @@ -249,12 +249,10 @@ public CompletableFuture globalCleanupAsync(JobID jobId, Executor executor () -> { LOG.debug("Removing job graph {} from {}.", jobId, jobGraphStateHandleStore); - if (addedJobGraphs.contains(jobId)) { - final String name = jobGraphStoreUtil.jobIDToName(jobId); - releaseAndRemoveOrThrowCompletionException(jobId, name); + final String name = jobGraphStoreUtil.jobIDToName(jobId); + releaseAndRemoveOrThrowCompletionException(jobId, name); - addedJobGraphs.remove(jobId); - } + addedJobGraphs.remove(jobId); LOG.info("Removed job graph {} from {}.", jobId, jobGraphStateHandleStore); }, diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java index 5cbfeac741ec6..f8e6d69cbbd31 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java @@ -121,7 +121,8 @@ void replace(String name, R resourceVersion, T state) * node if any. Also the state on the external storage will be discarded. * * @param name Key name in ConfigMap or child path name in ZooKeeper - * @return True if the state handle could be removed. + * @return {@code true} if the state handle is removed (also if it didn't exist in the first + * place); otherwise {@code false}. * @throws Exception if releasing, removing the handles or discarding the state failed */ boolean releaseAndTryRemove(String name) throws Exception; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java index d45a39aeb25a4..1343e7f648375 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/DefaultJobGraphStoreTest.java @@ -41,8 +41,8 @@ import java.util.Collection; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -51,6 +51,7 @@ import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; /** @@ -220,14 +221,22 @@ public void testGlobalCleanupWithNonExistName() throws Exception { .globalCleanupAsync(testingJobGraph.getJobID(), Executors.directExecutor()) .join(); - try { - removeFuture.get(timeout, TimeUnit.MILLISECONDS); - fail( - "We should get an expected timeout because we are removing a non-existed job graph."); - } catch (TimeoutException ex) { - // expected - } - assertThat(removeFuture.isDone(), is(false)); + assertThat(removeFuture.isDone(), is(true)); + } + + @Test + public void testGlobalCleanupFailsIfRemovalReturnsFalse() throws Exception { + final TestingStateHandleStore stateHandleStore = + builder.setRemoveFunction(name -> false).build(); + + final JobGraphStore jobGraphStore = createAndStartJobGraphStore(stateHandleStore); + assertThrows( + ExecutionException.class, + () -> + jobGraphStore + .globalCleanupAsync( + testingJobGraph.getJobID(), Executors.directExecutor()) + .get()); } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java index dc81131fdf6e4..ddf6fc7b2071a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java @@ -223,6 +223,18 @@ public void testRepeatableCleanup() throws Exception { assertTrue(stateHandle.isDiscarded()); } + @Test + public void testCleanupOfNonExistingState() throws Exception { + final ZooKeeperStateHandleStore testInstance = + new ZooKeeperStateHandleStore<>( + ZOOKEEPER.getClient(), new TestingLongStateHandleHelper()); + + final String pathInZooKeeper = "/testCleanupOfNonExistingState"; + + assertTrue(testInstance.releaseAndTryRemove(pathInZooKeeper)); + assertFalse(testInstance.exists(pathInZooKeeper).isExisting()); + } + @Test public void testRepeatableCleanupWithLockOnNode() throws Exception { final CuratorFramework client = From 9ed108b55d7092ffac4f3fc66b72f744facfce14 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Wed, 16 Mar 2022 19:39:28 +0100 Subject: [PATCH 035/258] [hotfix][docs] Aligns JavaDoc with method signature --- .../org/apache/flink/runtime/persistence/StateHandleStore.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java index f8e6d69cbbd31..6c9f521bd50f7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/persistence/StateHandleStore.java @@ -117,8 +117,7 @@ void replace(String name, R resourceVersion, T state) /** * Releases the lock for the given state handle and tries to remove the state handle if it is no - * longer locked. It returns the {@link RetrievableStateHandle} stored under the given state - * node if any. Also the state on the external storage will be discarded. + * longer locked. Also the state on the external storage will be discarded. * * @param name Key name in ConfigMap or child path name in ZooKeeper * @return {@code true} if the state handle is removed (also if it didn't exist in the first From e3992ab17a4515076c287b8d12003e1449718df1 Mon Sep 17 00:00:00 2001 From: zoucao Date: Sat, 19 Mar 2022 00:45:19 +0800 Subject: [PATCH 036/258] [FLINK-26723][runtime]fix the error message thrown by SourceCoordinatorContext --- .../coordinator/SourceCoordinatorContext.java | 19 +++++++++++-------- .../SourceCoordinatorContextTest.java | 2 +- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java index 5e560fad7f2cf..4434a18b5dc56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContext.java @@ -195,14 +195,17 @@ public void assignSplits(SplitsAssignment assignment) { callInCoordinatorThread( () -> { // Ensure all the subtasks in the assignment have registered. - for (Integer subtaskId : assignment.assignment().keySet()) { - if (!registeredReaders.containsKey(subtaskId)) { - throw new IllegalArgumentException( - String.format( - "Cannot assign splits %s to subtask %d because the subtask is not registered.", - registeredReaders.get(subtaskId), subtaskId)); - } - } + assignment + .assignment() + .forEach( + (id, splits) -> { + if (!registeredReaders.containsKey(id)) { + throw new IllegalArgumentException( + String.format( + "Cannot assign splits %s to subtask %d because the subtask is not registered.", + splits, id)); + } + }); assignmentTracker.recordSplitAssignment(assignment); assignment diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java index d15f28e7705bd..380e65c1c7e32 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/source/coordinator/SourceCoordinatorContextTest.java @@ -153,7 +153,7 @@ private void testAssignSplitToUnregisterdReader(boolean fromCoordinatorExecutor) } }, "assignSplits() should fail to assign the splits to a reader that is not registered.", - "Cannot assign splits"); + "Cannot assign splits " + splitsAssignment.assignment().get(0)); } @Test From 8dbef3f91fa763fc12310875b350317e1763e642 Mon Sep 17 00:00:00 2001 From: Paul Lin Date: Sun, 20 Mar 2022 12:47:58 +0800 Subject: [PATCH 037/258] [FLINK-26618][sql-client] Fix 'remove jar' statement is not aligned with pipeline jars This closes #19133. --- .../gateway/context/SessionContext.java | 34 ++++++++++++------- .../src/test/resources/sql/set.q | 8 +++++ 2 files changed, 29 insertions(+), 13 deletions(-) diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java index d392460b1dde8..c31f6f5f5191b 100644 --- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java +++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/context/SessionContext.java @@ -263,6 +263,9 @@ public void addJar(String jarPath) { } Set newDependencies = new HashSet<>(dependencies); + // merge the jars in config with the jars maintained in session + Set jarsInConfig = getJarsInConfig(); + newDependencies.addAll(jarsInConfig); newDependencies.add(jarURL); updateClassLoaderAndDependencies(newDependencies); @@ -281,6 +284,9 @@ public void removeJar(String jarPath) { } Set newDependencies = new HashSet<>(dependencies); + // merge the jars in config with the jars maintained in session + Set jarsInConfig = getJarsInConfig(); + newDependencies.addAll(jarsInConfig); newDependencies.remove(jarURL); updateClassLoaderAndDependencies(newDependencies); @@ -325,22 +331,10 @@ private void resetSessionConfigurationToDefault(Configuration defaultConf) { } private void updateClassLoaderAndDependencies(Collection newDependencies) { - // merge the jar in config with the jar maintained in session - Set jarsInConfig; - try { - jarsInConfig = - new HashSet<>( - ConfigUtils.decodeListFromConfig( - sessionConfiguration, PipelineOptions.JARS, URL::new)); - } catch (MalformedURLException e) { - throw new SqlExecutionException( - "Failed to parse the option `pipeline.jars` in configuration.", e); - } - jarsInConfig.addAll(newDependencies); ConfigUtils.encodeCollectionToConfig( sessionConfiguration, PipelineOptions.JARS, - new ArrayList<>(jarsInConfig), + new ArrayList<>(newDependencies), URL::toString); // TODO: update the classloader in CatalogManager. @@ -375,4 +369,18 @@ private URL getURLFromPath(String jarPath, String message) { e); } } + + private Set getJarsInConfig() { + Set jarsInConfig; + try { + jarsInConfig = + new HashSet<>( + ConfigUtils.decodeListFromConfig( + sessionConfiguration, PipelineOptions.JARS, URL::new)); + } catch (MalformedURLException e) { + throw new SqlExecutionException( + "Failed to parse the option `pipeline.jars` in configuration.", e); + } + return jarsInConfig; + } } diff --git a/flink-table/flink-sql-client/src/test/resources/sql/set.q b/flink-table/flink-sql-client/src/test/resources/sql/set.q index 5da0b8841c166..5d214ef175921 100644 --- a/flink-table/flink-sql-client/src/test/resources/sql/set.q +++ b/flink-table/flink-sql-client/src/test/resources/sql/set.q @@ -170,3 +170,11 @@ SELECT id, func1(str) FROM (VALUES (1, 'Hello World')) AS T(id, str) ; +----+-------------+--------------------------------+ Received a total of 1 row !ok + +REMOVE JAR '$VAR_UDF_JAR_PATH'; +[INFO] The specified jar is removed from session classloader. +!info + +SHOW JARS; +Empty set +!ok From 41099c475db6077ca5c7b75e6d68266fecdd44f6 Mon Sep 17 00:00:00 2001 From: Lijie Wang Date: Mon, 14 Feb 2022 16:59:25 +0800 Subject: [PATCH 038/258] [FLINK-25226][doc] Add documentation about the AdaptiveBatchScheduler This closes #18757. --- .../docs/deployment/elastic_scaling.md | 42 ++++++++++++++++++ .../docs/deployment/elastic_scaling.md | 43 +++++++++++++++++++ 2 files changed, 85 insertions(+) diff --git a/docs/content.zh/docs/deployment/elastic_scaling.md b/docs/content.zh/docs/deployment/elastic_scaling.md index c1effbdf9153f..f58b36c634d43 100644 --- a/docs/content.zh/docs/deployment/elastic_scaling.md +++ b/docs/content.zh/docs/deployment/elastic_scaling.md @@ -149,5 +149,47 @@ Adaptive 调度器可以通过[所有在名字包含 `adaptive-scheduler` 的配 - **空闲 Slot**: 如果 Slot 共享组的最大并行度不相等,提供给 Adaptive 调度器所使用的的 Slot 可能不会被使用。 - 扩缩容事件会触发 Job 和 Task 重启,Task 重试的次数也会增加。 +## Adaptive Batch Scheduler + +Adaptive Batch Scheduler 是一种可以自动推导每个算子并行度的批作业调度器。如果算子未设置并行度,调度器将根据其消费的数据量的大小来推导其并行度。这可以带来诸多好处: +- 批作业用户可以从并行度调优中解脱出来 +- 根据数据量自动推导并行度可以更好地适应每天变化的数据量 +- SQL作业中的算子也可以分配不同的并行度 + +### 用法 + +使用 Adaptive Batch Scheduler 自动推导算子的并行度,需要: +- 启用 Adaptive Batch Scheduler +- 配置算子的并行度为 `-1` + +#### 启用 Adaptive Batch Scheduler +为了启用 Adaptive Batch Scheduler, 你需要: +- 配置 `jobmanager.scheduler: AdpaptiveBatch` +- 由于 ["只支持所有数据交换都为 BLOCKING 模式的作业"](#局限性-2), 需要将 [`execution.batch-shuffle-mode`]({{< ref "docs/deployment/config" >}}#execution-batch-shuffle-mode) 配置为 `ALL-EXCHANGES-BLOCKING`(默认值) 。 + +除此之外,使用 Adaptive Batch Scheduler 时,以下相关配置也可以调整: +- [`jobmanager.adaptive-batch-scheduler.min-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-min-parallelism): 允许自动设置的并行度最小值。需要配置为 2 的幂,否则也会被自动调整为最接近且大于其的 2 的幂。 +- [`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism): 允许自动设置的并行度最大值。需要配置为 2 的幂,否则也会被自动调整为最接近且小于其的 2 的幂。 +- [`jobmanager.adaptive-batch-scheduler.avg-data-volume-per-task`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-avg-data-volume-per-task): 期望每个任务平均处理的数据量大小。由于顶点的并行度会被调整为 2^N,因此实际每个任务平均处理的数据量大小将是该值的 0.75~1.5 倍。 另外需要注意的是,当出现数据倾斜,或者确定的并行度达到最大并行度(由于数据过多)时,一些任务实际处理的数据可能会远远超过这个值。 +- [`jobmanager.adaptive-batch-scheduler.default-source-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-default-source-parallelism): source 算子的默认并行度 + +#### 配置算子的并行度为 `-1` +Adaptive Batch Scheduler 只会为用户未指定并行度的算子(并行度为 `-1`)推导并行度。 所以如果你想自动推导算子的并行度,需要进行以下配置: +- 配置 `parallelism.default: -1` +- 对于 SQL 作业,需要配置 `table.exec.resource.default-parallelism: -1` +- 对于 DataStream/DataSet 作业,不要在作业中通过算子的 `setParallelism()` 方法来指定并行度 +- 对于 DataStream/DataSet 作业,不要在作业中通过 `StreamExecutionEnvironment/ExecutionEnvironment` 的 `setParallelism()` 方法来指定并行度 + +### 性能调优 + +1. 建议使用 [Sort Shuffle](https://flink.apache.org/2021/10/26/sort-shuffle-part1.html) 并且设置 [`taskmanager.network.memory.buffers-per-channel`]({{< ref "docs/deployment/config" >}}#taskmanager-network-memory-buffers-per-channel) 为 `0`。 这会解耦并行度与需要的网络内存,对于大规模作业,这样可以降低遇到 "Insufficient number of network buffers" 错误的可能性。 +2. 建议将 [`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism) 设置为最坏情况下预期需要的并行度。不建议配置太大的值,否则可能会影响性能。这个配置项会影响上游任务产出的 subpartition 的数量,过多的 subpartition 可能会影响 hash shuffle 的性能,或者由于小包影响网络传输的性能。 + +### 局限性 +- **只支持批作业**: Adaptive Batch Scheduler 只支持批作业。当提交的是一个流作业时,会抛出异常。 +- **只支持所有数据交换都为 BLOCKING 模式的作业**: 目前 Adaptive Batch Scheduler 只支持 [shuffle mode]({{< ref "docs/deployment/config" >}}#execution-batch-shuffle-mode) 为 ALL-EXCHANGES-BLOCKING 的作业。 +- **推导出的并行度是 2 的幂**: 为了使子分区可以均匀分配给下游任务,[`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism) 应该被配置为 2^N, 推导出的并行度会是 2^M, 且满足 M <= N。 +- **不支持 FileInputFormat 类型的 source**: 不支持 FileInputFormat 类型的 source, 包括 `StreamExecutionEnvironment#readFile(...)` `StreamExecutionEnvironment#readTextFile(...)` 和 `StreamExecutionEnvironment#createInput(FileInputFormat, ...)`。 当使用 Adaptive Batch Scheduler 时,用户应该使用新版的 Source API ([FileSystem DataStream Connector]({{< ref "docs/connectors/datastream/filesystem.md" >}}) 或 [FileSystem SQL Connector]({{< ref "docs/connectors/table/filesystem.md" >}})) 来读取文件. +- **Web UI 上展示的上游输出的数据量和下游收到的数据量可能不一致**: 在使用 Adaptive Batch Scheduler 时,对于 broadcast 边,上游算子发送的数据量和下游算子接收的数据量可能会不相等,这在 Web UI 的显示上可能会困扰用户。细节详见 [FLIP-187](https://cwiki.apache.org/confluence/display/FLINK/FLIP-187%3A+Adaptive+Batch+Job+Scheduler)。 {{< top >}} diff --git a/docs/content/docs/deployment/elastic_scaling.md b/docs/content/docs/deployment/elastic_scaling.md index 47e220d6cb44c..386f083334015 100644 --- a/docs/content/docs/deployment/elastic_scaling.md +++ b/docs/content/docs/deployment/elastic_scaling.md @@ -151,5 +151,48 @@ The behavior of Adaptive Scheduler is configured by [all configuration options c - **Unused slots**: If the max parallelism for slot sharing groups is not equal, slots offered to Adaptive Scheduler might be unused. - Scaling events trigger job and task restarts, which will increase the number of Task attempts. +## Adaptive Batch Scheduler + +The Adaptive Batch Scheduler can automatically decide parallelisms of operators for batch jobs. If an operator is not set with a parallelism, the scheduler will decide parallelism for it according to the size of its consumed datasets. This can bring many benefits: +- Batch job users can be relieved from parallelism tuning +- Automatically tuned parallelisms can better fit consumed datasets which have a varying volume size every day +- Operators from SQL batch jobs can be assigned with different parallelisms which are automatically tuned + +### Usage + +To automatically decide parallelisms for operators with Adaptive Batch Scheduler, you need to: +- Configure to use Adaptive Batch Scheduler. +- Set the parallelism of operators to `-1`. + +#### Configure to use Adaptive Batch Scheduler +To use Adaptive Batch Scheduler, you need to: +- Set `jobmanager.scheduler: AdaptiveBatch`. +- Leave the [`execution.batch-shuffle-mode`]({{< ref "docs/deployment/config" >}}#execution-batch-shuffle-mode) unset or explicitly set it to `ALL-EXCHANGES-BLOCKING` (default value) due to ["ALL-EXCHANGES-BLOCKING jobs only"](#limitations-2). + +In addition, there are several related configuration options that may need adjustment when using Adaptive Batch Scheduler: +- [`jobmanager.adaptive-batch-scheduler.min-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-min-parallelism): The lower bound of allowed parallelism to set adaptively. Currently, this option should be configured as a power of 2, otherwise it will be rounded up to a power of 2 automatically. +- [`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism): The upper bound of allowed parallelism to set adaptively. Currently, this option should be configured as a power of 2, otherwise it will be rounded down to a power of 2 automatically. +- [`jobmanager.adaptive-batch-scheduler.avg-data-volume-per-task`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-avg-data-volume-per-task): The average size of data volume to expect each task instance to process. Note that since the parallelism of the vertices is adjusted to a power of 2, the actual average size will be 0.75~1.5 times this value. It is also important to note that when data skew occurs, or the decided parallelism reaches the max parallelism (due to too much data), the data actually processed by some tasks may far exceed this value. +- [`jobmanager.adaptive-batch-scheduler.default-source-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-default-source-parallelism): The default parallelism of data source. + +#### Set the parallelism of operators to `-1` +Adaptive Batch Scheduler will only decide parallelism for operators whose parallelism is not specified by users (parallelism is `-1`). So if you want the parallelism of operators to be decided automatically, you should configure as follows: +- Set `parallelism.default: -1` +- Set `table.exec.resource.default-parallelism: -1` in SQL jobs. +- Don't call `setParallelism()` for operators in DataStream/DataSet jobs. +- Don't call `setParallelism()` on `StreamExecutionEnvironment/ExecutionEnvironment` in DataStream/DataSet jobs. + +### Performance tuning + +1. It's recommended to use [Sort Shuffle](https://flink.apache.org/2021/10/26/sort-shuffle-part1.html) and set [`taskmanager.network.memory.buffers-per-channel`]({{< ref "docs/deployment/config" >}}#taskmanager-network-memory-buffers-per-channel) to `0`. This can decouple the required network memory from parallelism, so that for large scale jobs, the "Insufficient number of network buffers" errors are less likely to happen. +2. It's recommended to set [`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism) to the parallelism you expect to need in the worst case. Values larger than that are not recommended, because excessive value may affect the performance. This option can affect the number of subpartitions produced by upstream tasks, large number of subpartitions may degrade the performance of hash shuffle and the performance of network transmission due to small packets. + +### Limitations + +- **Batch jobs only**: Adaptive Batch Scheduler only supports batch jobs. Exception will be thrown if a streaming job is submitted. +- **ALL-EXCHANGES-BLOCKING jobs only**: At the moment, Adaptive Batch Scheduler only supports jobs whose [shuffle mode]({{< ref "docs/deployment/config" >}}#execution-batch-shuffle-mode) is `ALL-EXCHANGES-BLOCKING`. +- **The decided parallelism will be a power of 2**: In order to ensure downstream tasks to consume the same count of subpartitions, the configuration option [`jobmanager.adaptive-batch-scheduler.max-parallelism`]({{< ref "docs/deployment/config" >}}#jobmanager-adaptive-batch-scheduler-max-parallelism) should be set to be a power of 2 (2^N), and the decided parallelism will also be a power of 2 (2^M and M <= N). +- **FileInputFormat sources are not supported**: FileInputFormat sources are not supported, including `StreamExecutionEnvironment#readFile(...)` `StreamExecutionEnvironment#readTextFile(...)` and `StreamExecutionEnvironment#createInput(FileInputFormat, ...)`. Users should use the new sources([FileSystem DataStream Connector]({{< ref "docs/connectors/datastream/filesystem.md" >}}) or [FileSystem SQL Connector]({{< ref "docs/connectors/table/filesystem.md" >}})) to read files when using the Adaptive Batch Scheduler. +- **Inconsistent broadcast results metrics on WebUI**: In Adaptive Batch Scheduler, for broadcast results, the number of bytes/records sent by the upstream task counted by metric is not equal to the number of bytes/records received by the downstream task, which may confuse users when displayed on the Web UI. See [FLIP-187](https://cwiki.apache.org/confluence/display/FLINK/FLIP-187%3A+Adaptive+Batch+Job+Scheduler) for details. {{< top >}} From 34034a6c54f3b23825bc318806c683b3495f72d1 Mon Sep 17 00:00:00 2001 From: lovewin99 Date: Tue, 1 Mar 2022 10:18:05 +0800 Subject: [PATCH 039/258] [FLINK-26151]Avoid inprogressfileRecoverable not be clean up after restoring the bucket This closes #18776. --- .../api/functions/sink/filesystem/Bucket.java | 1 + .../functions/sink/filesystem/BucketTest.java | 27 +++++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java index f1f34baf1e25a..8b5d2ab73fe5f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/filesystem/Bucket.java @@ -142,6 +142,7 @@ private void restoreInProgressFile(final BucketState state) throws IOE bucketId, inProgressFileRecoverable, state.getInProgressFileCreationTime()); + inProgressFileRecoverablesPerCheckpoint.put(Long.MIN_VALUE, inProgressFileRecoverable); } else { // if the writer does not support resume, then we close the // in-progress part and commit it, as done in the case of pending files. diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java index 9fc09cdb110f0..245e895878b41 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/filesystem/BucketTest.java @@ -117,6 +117,33 @@ public void shouldNotCallCleanupWithoutInProgressPartFiles() throws Exception { assertThat(recoverableWriter, hasCalledDiscard(0)); // we have no in-progress file. } + @Test + public void shouldCleanupOutdatedResumablesAfterResumed() throws Exception { + final File outDir = TEMP_FOLDER.newFolder(); + final Path path = new Path(outDir.toURI()); + + final TestRecoverableWriter recoverableWriter = getRecoverableWriter(path); + final Bucket bucketUnderTest = + createBucket(recoverableWriter, path, 0, 0, OutputFileConfig.builder().build()); + + bucketUnderTest.write("test-element", 0L); + final BucketState state0 = bucketUnderTest.onReceptionOfCheckpoint(0L); + assertThat(state0, hasActiveInProgressFile()); + bucketUnderTest.onSuccessfulCompletionOfCheckpoint(0L); + assertThat(recoverableWriter, hasCalledDiscard(0)); + + final File newOutDir = TEMP_FOLDER.newFolder(); + final Path newPath = new Path(newOutDir.toURI()); + final TestRecoverableWriter newRecoverableWriter = getRecoverableWriter(newPath); + final Bucket bucketAfterResume = + restoreBucket( + newRecoverableWriter, 0, 0, state0, OutputFileConfig.builder().build()); + final BucketState state1 = bucketAfterResume.onReceptionOfCheckpoint(1L); + assertThat(state1, hasActiveInProgressFile()); + bucketAfterResume.onSuccessfulCompletionOfCheckpoint(1L); + assertThat(newRecoverableWriter, hasCalledDiscard(1)); + } + // --------------------------- Checking Restore --------------------------- @Test From 3cdcbaf1fb874d751f3ae625cfebf67384a8abcb Mon Sep 17 00:00:00 2001 From: Hangxiang Yu Date: Fri, 18 Mar 2022 12:59:37 +0800 Subject: [PATCH 040/258] [FLINK-5151][docs] Add descriptions about object mutations of state backends This closes #19143. --- docs/content.zh/docs/ops/state/state_backends.md | 3 +++ docs/content/docs/ops/state/state_backends.md | 4 +++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/docs/content.zh/docs/ops/state/state_backends.md b/docs/content.zh/docs/ops/state/state_backends.md index e6d6764a0f3d2..f5776dbdaea3c 100644 --- a/docs/content.zh/docs/ops/state/state_backends.md +++ b/docs/content.zh/docs/ops/state/state_backends.md @@ -59,6 +59,8 @@ HashMapStateBackend 的适用场景: 建议同时将 [managed memory]({{< ref "docs/deployment/memory/mem_setup_tm" >}}#managed-memory) 设为0,以保证将最大限度的内存分配给 JVM 上的用户代码。 +与 EmbeddedRocksDBStateBackend 不同的是,由于 HashMapStateBackend 将数据以对象形式存储在堆中,因此重用这些对象数据是不安全的。 + ### EmbeddedRocksDBStateBackend EmbeddedRocksDBStateBackend 将正在运行中的状态数据保存在 [RocksDB](http://rocksdb.org) 数据库中,RocksDB 数据库默认将数据存储在 TaskManager 的数据目录。 @@ -79,6 +81,7 @@ EmbeddedRocksDBStateBackend 的适用场景: 注意,你可以保留的状态大小仅受磁盘空间的限制。与状态存储在内存中的 HashMapStateBackend 相比,EmbeddedRocksDBStateBackend 允许存储非常大的状态。 然而,这也意味着使用 EmbeddedRocksDBStateBackend 将会使应用程序的最大吞吐量降低。 所有的读写都必须序列化、反序列化操作,这个比基于堆内存的 state backend 的效率要低很多。 +同时因为存在这些序列化、反序列化操作,重用放入 EmbeddedRocksDBStateBackend 的对象是安全的。 请同时参考 [Task Executor 内存配置]({{< ref "docs/deployment/memory/mem_tuning" >}}#rocksdb-state-backend) 中关于 EmbeddedRocksDBStateBackend 的建议。 diff --git a/docs/content/docs/ops/state/state_backends.md b/docs/content/docs/ops/state/state_backends.md index aa06c909fb336..065c7a444ac71 100644 --- a/docs/content/docs/ops/state/state_backends.md +++ b/docs/content/docs/ops/state/state_backends.md @@ -61,6 +61,8 @@ The HashMapStateBackend is encouraged for: It is also recommended to set [managed memory]({{< ref "docs/deployment/memory/mem_setup_tm" >}}#managed-memory) to zero. This will ensure that the maximum amount of memory is allocated for user code on the JVM. +Unlike EmbeddedRocksDBStateBackend, the HashMapStateBackend stores data as objects on the heap so that it is unsafe to reuse objects. + ### The EmbeddedRocksDBStateBackend The EmbeddedRocksDBStateBackend holds in-flight data in a [RocksDB](http://rocksdb.org) database @@ -83,7 +85,7 @@ Note that the amount of state that you can keep is only limited by the amount of This allows keeping very large state, compared to the HashMapStateBackend that keeps state in memory. This also means, however, that the maximum throughput that can be achieved will be lower with this state backend. All reads/writes from/to this backend have to go through de-/serialization to retrieve/store the state objects, which is also more expensive than always working with the -on-heap representation as the heap-based backends are doing. +on-heap representation as the heap-based backends are doing. It's safe for EmbeddedRocksDBStateBackend to reuse objects due to the de-/serialization. Check also recommendations about the [task executor memory configuration]({{< ref "docs/deployment/memory/mem_tuning" >}}#rocksdb-state-backend) for the EmbeddedRocksDBStateBackend. From e4b8d9285fde28f63fec26be97c4e27742d06c23 Mon Sep 17 00:00:00 2001 From: Yubin Li Date: Sun, 13 Mar 2022 22:35:23 +0800 Subject: [PATCH 041/258] [FLINK-26578][docs-zh] Translate new Project Configuration section to Chinese. This closes #19162 (cherry picked from commit 2cb0c7d01f505e7a4d3fa75f2b9da2671f81d65c) --- .../docs/connectors/datastream/cassandra.md | 2 +- .../connectors/datastream/elasticsearch.md | 4 +- .../docs/connectors/datastream/jdbc.md | 2 +- .../docs/connectors/datastream/kafka.md | 2 +- .../docs/connectors/datastream/nifi.md | 2 +- .../docs/connectors/datastream/pubsub.md | 2 +- .../docs/connectors/datastream/pulsar.md | 2 +- .../docs/connectors/datastream/rabbitmq.md | 2 +- .../docs/connectors/table/elasticsearch.md | 2 + .../content.zh/docs/connectors/table/hbase.md | 1 + docs/content.zh/docs/connectors/table/jdbc.md | 4 +- .../content.zh/docs/connectors/table/kafka.md | 2 +- .../docs/connectors/table/kinesis.md | 2 + .../docs/connectors/table/overview.md | 2 + .../docs/connectors/table/upsert-kafka.md | 2 + .../docs/dev/configuration/_index.md | 23 + .../docs/dev/configuration/advanced.md | 106 ++++ .../docs/dev/configuration/connector.md | 56 ++ .../docs/dev/configuration/gradle.md | 92 +++ .../docs/dev/configuration/maven.md | 142 +++++ .../docs/dev/configuration/overview.md | 206 +++++++ .../docs/dev/configuration/testing.md | 49 ++ docs/content.zh/docs/dev/datastream/_index.md | 4 +- .../fault-tolerance/queryable_state.md | 2 +- .../dev/datastream/project-configuration.md | 570 ------------------ .../content.zh/docs/dev/datastream/testing.md | 6 +- .../docs/dev/table/data_stream_api.md | 2 + docs/content.zh/docs/dev/table/overview.md | 69 +-- .../content.zh/docs/dev/table/sourcesSinks.md | 27 + .../dev/table/sql/queries/match_recognize.md | 2 +- docs/content.zh/docs/dev/table/sqlClient.md | 11 +- docs/content.zh/docs/flinkDev/ide_setup.md | 2 +- docs/content.zh/docs/libs/cep.md | 6 +- docs/content.zh/docs/libs/gelly/overview.md | 2 +- 34 files changed, 744 insertions(+), 666 deletions(-) create mode 100644 docs/content.zh/docs/dev/configuration/_index.md create mode 100644 docs/content.zh/docs/dev/configuration/advanced.md create mode 100644 docs/content.zh/docs/dev/configuration/connector.md create mode 100644 docs/content.zh/docs/dev/configuration/gradle.md create mode 100644 docs/content.zh/docs/dev/configuration/maven.md create mode 100644 docs/content.zh/docs/dev/configuration/overview.md create mode 100644 docs/content.zh/docs/dev/configuration/testing.md delete mode 100644 docs/content.zh/docs/dev/datastream/project-configuration.md diff --git a/docs/content.zh/docs/connectors/datastream/cassandra.md b/docs/content.zh/docs/connectors/datastream/cassandra.md index bddb9f2f1ba27..b13bfadf3bd7f 100644 --- a/docs/content.zh/docs/connectors/datastream/cassandra.md +++ b/docs/content.zh/docs/connectors/datastream/cassandra.md @@ -37,7 +37,7 @@ To use this connector, add the following dependency to your project: {{< artifact flink-connector-cassandra withScalaVersion >}} -Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/datastream/project-configuration" >}}). +Note that the streaming connectors are currently __NOT__ part of the binary distribution. See how to link with them for cluster execution [here]({{< ref "docs/dev/configuration/overview" >}}). ## Installing Apache Cassandra There are multiple ways to bring up a Cassandra instance on local machine: diff --git a/docs/content.zh/docs/connectors/datastream/elasticsearch.md b/docs/content.zh/docs/connectors/datastream/elasticsearch.md index c97e85221afa8..28d2f8fcdb7da 100644 --- a/docs/content.zh/docs/connectors/datastream/elasticsearch.md +++ b/docs/content.zh/docs/connectors/datastream/elasticsearch.md @@ -52,7 +52,7 @@ under the License. 请注意,流连接器目前不是二进制发行版的一部分。 -有关如何将程序和用于集群执行的库一起打包,参考[此文档]({{< ref "docs/dev/datastream/project-configuration" >}}) +有关如何将程序和用于集群执行的库一起打包,参考[此文档]({{< ref "docs/dev/configuration/overview" >}})。 ## 安装 Elasticsearch @@ -373,7 +373,7 @@ checkpoint 会进行等待,直到 Elasticsearch 节点队列有足够的容量 ## 将 Elasticsearch 连接器打包到 Uber-Jar 中 建议构建一个包含所有依赖的 uber-jar (可执行的 jar),以便更好地执行你的 Flink 程序。 -(更多信息参见[此文档]({{< ref "docs/dev/datastream/project-configuration" >}}))。 +(更多信息参见[此文档]({{< ref "docs/dev/configuration/overview" >}}))。 或者,你可以将连接器的 jar 文件放入 Flink 的 `lib/` 目录下,使其在全局范围内可用,即可用于所有的作业。 diff --git a/docs/content.zh/docs/connectors/datastream/jdbc.md b/docs/content.zh/docs/connectors/datastream/jdbc.md index 50fde2ea1d339..31e4f182fcdf4 100644 --- a/docs/content.zh/docs/connectors/datastream/jdbc.md +++ b/docs/content.zh/docs/connectors/datastream/jdbc.md @@ -32,7 +32,7 @@ under the License. {{< artifact flink-connector-jdbc >}} -注意该连接器目前还 __不是__ 二进制发行版的一部分,如何在集群中运行请参考 [这里]({{< ref "docs/dev/datastream/project-configuration" >}})。 +注意该连接器目前还 __不是__ 二进制发行版的一部分,如何在集群中运行请参考 [这里]({{< ref "docs/dev/configuration/overview" >}})。 已创建的 JDBC Sink 能够保证至少一次的语义。 更有效的精确执行一次可以通过 upsert 语句或幂等更新实现。 diff --git a/docs/content.zh/docs/connectors/datastream/kafka.md b/docs/content.zh/docs/connectors/datastream/kafka.md index 3908f19d747a9..d064d601be864 100644 --- a/docs/content.zh/docs/connectors/datastream/kafka.md +++ b/docs/content.zh/docs/connectors/datastream/kafka.md @@ -43,7 +43,7 @@ Apache Flink 集成了通用的 Kafka 连接器,它会尽力与 Kafka client {{< artifact flink-connector-base >}} Flink 目前的流连接器还不是二进制发行版的一部分。 -[在此处]({{< ref "docs/dev/datastream/project-configuration" >}})可以了解到如何链接它们,从而在集群中运行。 +[在此处]({{< ref "docs/dev/configuration/overview" >}})可以了解到如何链接它们,从而在集群中运行。 ## Kafka Source {{< hint info >}} diff --git a/docs/content.zh/docs/connectors/datastream/nifi.md b/docs/content.zh/docs/connectors/datastream/nifi.md index d8e53e6ec59a4..a67b9cbb666e9 100644 --- a/docs/content.zh/docs/connectors/datastream/nifi.md +++ b/docs/content.zh/docs/connectors/datastream/nifi.md @@ -35,7 +35,7 @@ The NiFi connector is deprecated and will be removed with Flink 1.16. {{< artifact flink-connector-nifi >}} -注意这些连接器目前还没有包含在二进制发行版中。添加依赖、打包配置以及集群运行的相关信息请参考 [这里]({{< ref "docs/dev/datastream/project-configuration" >}})。 +注意这些连接器目前还没有包含在二进制发行版中。添加依赖、打包配置以及集群运行的相关信息请参考 [这里]({{< ref "docs/dev/configuration/overview" >}})。 #### 安装 Apache NiFi diff --git a/docs/content.zh/docs/connectors/datastream/pubsub.md b/docs/content.zh/docs/connectors/datastream/pubsub.md index 18225ca674b55..04e2a01ea6b34 100644 --- a/docs/content.zh/docs/connectors/datastream/pubsub.md +++ b/docs/content.zh/docs/connectors/datastream/pubsub.md @@ -34,7 +34,7 @@ under the License. 注意:此连接器最近才加到 Flink 里,还未接受广泛测试。

-注意连接器目前还不是二进制发行版的一部分,添加依赖、打包配置以及集群运行信息请参考[这里]({{< ref "docs/dev/datastream/project-configuration" >}}) +注意连接器目前还不是二进制发行版的一部分,添加依赖、打包配置以及集群运行信息请参考[这里]({{< ref "docs/dev/configuration/overview" >}}) ## Consuming or Producing PubSubMessages diff --git a/docs/content.zh/docs/connectors/datastream/pulsar.md b/docs/content.zh/docs/connectors/datastream/pulsar.md index 1779c729fd7eb..301f24222e775 100644 --- a/docs/content.zh/docs/connectors/datastream/pulsar.md +++ b/docs/content.zh/docs/connectors/datastream/pulsar.md @@ -35,7 +35,7 @@ Flink 当前只提供 [Apache Pulsar](https://pulsar.apache.org) 数据源,用 {{< artifact flink-connector-pulsar >}} -Flink 的流连接器并不会放到发行文件里面一同发布,阅读[此文档]({{< ref "docs/dev/datastream/project-configuration" >}}),了解如何将连接器添加到集群实例内。 +Flink 的流连接器并不会放到发行文件里面一同发布,阅读[此文档]({{< ref "docs/dev/configuration/overview" >}}),了解如何将连接器添加到集群实例内。 ## Pulsar 数据源 diff --git a/docs/content.zh/docs/connectors/datastream/rabbitmq.md b/docs/content.zh/docs/connectors/datastream/rabbitmq.md index 7c3a2a10c452f..668987fe6e9f7 100644 --- a/docs/content.zh/docs/connectors/datastream/rabbitmq.md +++ b/docs/content.zh/docs/connectors/datastream/rabbitmq.md @@ -40,7 +40,7 @@ Flink 自身既没有复用 "RabbitMQ AMQP Java Client" 的代码,也没有将 {{< artifact flink-connector-rabbitmq >}} -注意连接器现在没有包含在二进制发行版中。集群执行的相关信息请参考 [这里]({{< ref "docs/dev/datastream/project-configuration" >}}). +注意连接器现在没有包含在二进制发行版中。集群执行的相关信息请参考 [这里]({{< ref "docs/dev/configuration/overview" >}}). ### 安装 RabbitMQ 安装 RabbitMQ 请参考 [RabbitMQ 下载页面](http://www.rabbitmq.com/download.html)。安装完成之后,服务会自动拉起,应用程序就可以尝试连接到 RabbitMQ 了。 diff --git a/docs/content.zh/docs/connectors/table/elasticsearch.md b/docs/content.zh/docs/connectors/table/elasticsearch.md index 339d9d56f82fa..bb2ca45572db9 100644 --- a/docs/content.zh/docs/connectors/table/elasticsearch.md +++ b/docs/content.zh/docs/connectors/table/elasticsearch.md @@ -40,6 +40,8 @@ Elasticsearch 连接器允许将数据写入到 Elasticsearch 引擎的索引中 {{< sql_download_table "elastic" >}} +Elasticsearch 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Elasticsearch 连接器。 + 如何创建 Elasticsearch 表 ---------------- diff --git a/docs/content.zh/docs/connectors/table/hbase.md b/docs/content.zh/docs/connectors/table/hbase.md index 85b376f722673..2ff3fa49ea176 100644 --- a/docs/content.zh/docs/connectors/table/hbase.md +++ b/docs/content.zh/docs/connectors/table/hbase.md @@ -40,6 +40,7 @@ HBase 连接器在 upsert 模式下运行,可以使用 DDL 中定义的主键 {{< sql_download_table "hbase" >}} +HBase 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 HBase 连接器。 如何使用 HBase 表 ---------------- diff --git a/docs/content.zh/docs/connectors/table/jdbc.md b/docs/content.zh/docs/connectors/table/jdbc.md index ec8adaa522113..9a812fc891815 100644 --- a/docs/content.zh/docs/connectors/table/jdbc.md +++ b/docs/content.zh/docs/connectors/table/jdbc.md @@ -44,6 +44,8 @@ JDBC 连接器允许使用 JDBC 驱动向任意类型的关系型数据库读取 {{< sql_download_table "jdbc" >}} +JDBC 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 JDBC 连接器。 + 在连接到具体数据库时,也需要对应的驱动依赖,目前支持的驱动如下: | Driver | Group Id | Artifact Id | JAR | @@ -53,7 +55,7 @@ JDBC 连接器允许使用 JDBC 驱动向任意类型的关系型数据库读取 | PostgreSQL | `org.postgresql` | `postgresql` | [下载](https://jdbc.postgresql.org/download.html) | | Derby | `org.apache.derby` | `derby` | [下载](http://db.apache.org/derby/derby_downloads.html) | | -当前,JDBC 连接器和驱动不在 Flink 二进制发布包中,请参阅[这里]({{< ref "docs/dev/datastream/project-configuration" >}})了解在集群上执行时何连接它们。 +当前,JDBC 连接器和驱动不在 Flink 二进制发布包中,请参阅[这里]({{< ref "docs/dev/configuration" >}})了解在集群上执行时何连接它们。 diff --git a/docs/content.zh/docs/connectors/table/kafka.md b/docs/content.zh/docs/connectors/table/kafka.md index 6e3d3f5873848..d02d1cfba2b39 100644 --- a/docs/content.zh/docs/connectors/table/kafka.md +++ b/docs/content.zh/docs/connectors/table/kafka.md @@ -36,7 +36,7 @@ Kafka 连接器提供从 Kafka topic 中消费和写入数据的能力。 {{< sql_download_table "kafka" >}} -Kafka 连接器目前并不包含在 Flink 的二进制发行版中,请查阅 [这里]({{< ref "docs/dev/datastream/project-configuration" >}}) 了解如何在集群运行中引用 Kafka 连接器。 +Kafka 连接器目前并不包含在 Flink 的二进制发行版中,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Kafka 连接器。 如何创建 Kafka 表 ---------------- diff --git a/docs/content.zh/docs/connectors/table/kinesis.md b/docs/content.zh/docs/connectors/table/kinesis.md index 93d765719ca17..e706c32d2d5ef 100644 --- a/docs/content.zh/docs/connectors/table/kinesis.md +++ b/docs/content.zh/docs/connectors/table/kinesis.md @@ -36,6 +36,8 @@ Dependencies {{< sql_download_table "kinesis" >}} +Kinesis 连接器目前并不包含在 Flink 的二进制发行版中,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Kinesis 连接器。 + How to create a Kinesis data stream table ----------------------------------------- diff --git a/docs/content.zh/docs/connectors/table/overview.md b/docs/content.zh/docs/connectors/table/overview.md index 01fa0f849fac9..03fb840f5e6e4 100644 --- a/docs/content.zh/docs/connectors/table/overview.md +++ b/docs/content.zh/docs/connectors/table/overview.md @@ -95,6 +95,8 @@ Flink natively support various connectors. The following tables list all availab {{< top >}} +请查阅[配置]({{< ref "docs/dev/configuration/connector" >}})小节了解如何添加连接器依赖。 + How to use connectors -------- diff --git a/docs/content.zh/docs/connectors/table/upsert-kafka.md b/docs/content.zh/docs/connectors/table/upsert-kafka.md index 298a5f97aaecc..40df1fa203f27 100644 --- a/docs/content.zh/docs/connectors/table/upsert-kafka.md +++ b/docs/content.zh/docs/connectors/table/upsert-kafka.md @@ -40,6 +40,8 @@ Upsert Kafka 连接器支持以 upsert 方式从 Kafka topic 中读取数据并 {{< sql_download_table "upsert-kafka" >}} +Upsert Kafka 连接器不是二进制发行版的一部分,请查阅[这里]({{< ref "docs/dev/configuration/overview" >}})了解如何在集群运行中引用 Upsert Kafka 连接器。 + 完整示例 ---------------- diff --git a/docs/content.zh/docs/dev/configuration/_index.md b/docs/content.zh/docs/dev/configuration/_index.md new file mode 100644 index 0000000000000..0ad3d6b8a22b5 --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/_index.md @@ -0,0 +1,23 @@ +--- +title: "项目配置" +bookCollapseSection: true +weight: 1 +--- + diff --git a/docs/content.zh/docs/dev/configuration/advanced.md b/docs/content.zh/docs/dev/configuration/advanced.md new file mode 100644 index 0000000000000..b94094732fac6 --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/advanced.md @@ -0,0 +1,106 @@ +--- +title: "高级配置" +weight: 10 +type: docs +--- + + +# 高级配置主题 + +## Flink 依赖剖析 + +Flink 自身由一组类和依赖项组成,这些共同构成了 Flink 运行时的核心,在 Flink 应用程序启动时必须存在,会提供诸如通信协调、网络管理、检查点、容错、API、算子(如窗口)、资源管理等领域的服务。 + +这些核心类和依赖项都打包在 `flink-dist.jar`,可以在下载的发行版 `/lib` 文件夹中找到,也是 Flink 容器镜像的基础部分。您可以将其近似地看作是包含 `String` 和 `List` 等公用类的 Java 核心库。 + +为了保持核心依赖项尽可能小并避免依赖冲突,Flink Core Dependencies 不包含任何连接器或库(如 CEP、SQL、ML),以避免在类路径中有过多的类和依赖项。 + +Flink 发行版的 `/lib` 目录里还有包括常用模块在内的各种 JAR 文件,例如 [执行 Table 作业的必需模块](#Table-依赖剖析) 、一组连接器和 format。默认情况下会自动加载,若要禁止加载只需将它们从 classpath 中的 `/lib` 目录中删除即可。 + +Flink 还在 `/opt` 文件夹下提供了额外的可选依赖项,可以通过移动这些 JAR 文件到 `/lib` 目录来启用这些依赖项。 + +有关类加载的更多细节,请查阅 [Flink 类加载]({{< ref "docs/ops/debugging/debugging_classloading.zh.md" >}})。 + +## Scala 版本 + +不同的 Scala 版本二进制不兼容,所有(传递地)依赖于 Scala 的 Flink 依赖项都以它们构建的 Scala 版本为后缀(如 `flink-streaming-scala_2.12`)。 + +如果您只使用 Flink 的 Java API,您可以使用任何 Scala 版本。如果您使用 Flink 的 Scala API,则需要选择与应用程序的 Scala 匹配的 Scala 版本。 + +有关如何为特定 Scala 版本构建 Flink 的细节,请查阅[构建指南]({{< ref "docs/flinkDev/building" >}}#scala-versions)。 + +2.12.8 之后的 Scala 版本与之前的 2.12.x 版本二进制不兼容,使 Flink 项目无法将其 2.12.x 版本直接升级到 2.12.8 以上。您可以按照[构建指南]({{< ref "docs/flinkDev/building" >}}#scala-versions)在本地为更高版本的 Scala 构建 Flink 。为此,您需要在构建时添加 `-Djapicmp.skip` 以跳过二进制兼容性检查。 + +有关更多细节,请查阅 [Scala 2.12.8 版本说明](https://github.com/scala/scala/releases/tag/v2.12.8)。相关部分指出: + +第二项修改是二进制不兼容的:2.12.8 编译器忽略了由更早版本的 2.12 编译器生成的某些方法。然而我们相信这些方法永远不会被使用,现有的编译代码仍可工作。有关更多详细信息,请查阅[pull request 描述](https://github.com/scala/scala/pull/7469)。 + +## Table 依赖剖析 + +Flink 发行版默认包含执行 Flink SQL 任务的必要 JAR 文件(位于 `/lib` 目录),主要有: + +- `flink-table-api-java-uber-{{< version >}}.jar` → 包含所有的 Java API; +- `flink-table-runtime-{{< version >}}.jar` → 包含 Table 运行时; +- `flink-table-planner-loader-{{< version >}}.jar` → 包含查询计划器。 + +{{< hint warning >}} +以前,这些 JAR 都打包进了 `flink-table.jar`,自从 Flink 1.15 开始,已将其划分成三个 JAR,以允许用户使用 `flink-table-planner-loader-{{< version >}}.jar` 充当 `flink-table-planner{{< scala_version >}}-{{< version >}}.jar`。 +{{< /hint >}} + +虽然 Table Java API 内置于发行版中,但默认情况下不包含 Table Scala API。在 Flink Scala API 中使用格式和连接器时,您需要手动下载这些 JAR 包并将其放到发行版的 `/lib` 文件夹中(推荐),或者将它们打包为 Flink SQL 作业的 uber/fat JAR 包中的依赖项。 + +有关更多细节,请查阅如何[连接外部系统]({{< ref "docs/connectors/table/overview" >}})。 + +### Table Planner 和 Table Planner 加载器 + +从 Flink 1.15 开始,发行版包含两个 planner: + +- `flink-table-planner{{< scala_version >}}-{{< version >}}.jar`, 位于 `/opt` 目录, 包含查询计划器; +- `flink-table-planner-loader-{{< version >}}.jar`, 位于 `/lib` 目录默认被加载, 包含隐藏在单独的 classpath 里的查询计划器 (您无法直接使用 `io.apache.flink.table.planner` 包)。 + +这两个 planner JAR 文件的代码功能相同,但打包方式不同。若使用第一个文件,您必须使用与其相同版本的 Scala;若使用第二个,由于 Scala 已经被打包进该文件里,您不需要考虑 Scala 版本问题。 + +默认情况下,发行版使用 `flink-table-planner-loader`。如果想使用内部查询计划器,您可以换掉 JAR 包(拷贝 `flink-table-planner{{< scala_version >}}.jar` 并复制到发行版的 `/lib` 目录)。请注意,此时会被限制用于 Flink 发行版的 Scala 版本。 + +{{< hint danger >}} +这两个 planner 无法同时存在于 classpath,如果您在 `/lib` 目录同时加载他们,Table 任务将会失败。 +{{< /hint >}} + +{{< hint warning >}} +在即将发布的 Flink 版本中,我们将停止在 Flink 发行版中发布 `flink-table-planner{{< scala_version >}}` 组件。我们强烈建议迁移您的作业/自定义连接器/格式以使用前述 API 模块,而不依赖此内部 planner。如果您需要 planner 中尚未被 API 模块暴露的一些功能,请与社区讨论。 +{{< /hint >}} + +## Hadoop 依赖 + +**一般规则:** 没有必要直接添加 Hadoop 依赖到您的应用程序里,唯一的例外是您通过 [Hadoop 兼容](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/dataset/hadoop_compatibility/) 使用已有的 Hadoop 读写 format。 + +如果您想将 Flink 与 Hadoop 一起使用,您需要有一个包含 Hadoop 依赖项的 Flink 系统,而不是添加 Hadoop 作为应用程序依赖项。换句话说,Hadoop 必须是 Flink 系统本身的依赖,而不是用户代码的依赖。Flink 将使用 `HADOOP_CLASSPATH` 环境变量指定 Hadoop 依赖项,可以这样设置: + +```bash +export HADOOP_CLASSPATH=`hadoop classpath` +``` + +这样设计有两个主要原因: + +- 一些 Hadoop 交互可能在用户应用程序启动之前就发生在 Flink 内核。其中包括为检查点配置 HDFS、通过 Hadoop 的 Kerberos 令牌进行身份验证或在 YARN 上部署; + +- Flink 的反向类加载方式在核心依赖项中隐藏了许多传递依赖项。这不仅适用于 Flink 自己的核心依赖项,也适用于已有的 Hadoop 依赖项。这样,应用程序可以使用相同依赖项的不同版本,而不会遇到依赖项冲突。当依赖树变得非常大时,这非常有用。 + +如果您在 IDE 内开发或测试期间需要 Hadoop 依赖项(比如用于 HDFS 访问),应该限定这些依赖项的使用范围(如 *test* 或 *provided*)。 diff --git a/docs/content.zh/docs/dev/configuration/connector.md b/docs/content.zh/docs/dev/configuration/connector.md new file mode 100644 index 0000000000000..95f131a946609 --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/connector.md @@ -0,0 +1,56 @@ +--- +title: "连接器和格式" +weight: 5 +type: docs +--- + + +# 连接器和格式 + +Flink 应用程序可以通过连接器读取和写入各种外部系统。它支持多种格式,以便对数据进行编码和解码以匹配 Flink 的数据结构。 + +[DataStream]({{< ref "docs/connectors/datastream/overview.zh.md" >}}) 和 [Table API/SQL]({{< ref "docs/connectors/table/overview.zh.md" >}}) 都提供了连接器和格式的概述。 + +## 可用的组件 + +为了使用连接器和格式,您需要确保 Flink 可以访问实现了这些功能的组件。对于 Flink 社区支持的每个连接器,我们在 [Maven Central](https://search.maven.org) 发布了两类组件: + +* `flink-connector-` 这是一个精简 JAR,仅包括连接器代码,但不包括最终的第三方依赖项; +* `flink-sql-connector-` 这是一个包含连接器第三方依赖项的 uber JAR; + +这同样适用于格式。请注意,某些连接器可能没有相应的 `flink-sql-connector-` 组件,因为它们不需要第三方依赖项。 + +{{< hint info >}} +uber/fat JAR 主要与[SQL 客户端]({{< ref "docs/dev/table/sqlClient" >}})一起使用,但您也可以在任何 DataStream/Table 应用程序中使用它们。 +{{< /hint >}} + +## 使用组件 + +为了使用连接器/格式模块,您可以: + +* 把精简 JAR 及其传递依赖项打包进您的作业 JAR; +* 把 uber JAR 打包进您的作业 JAR; +* 把 uber JAR 直接复制到 Flink 发行版的 `/lib` 文件夹内; + +关于打包依赖项,请查看 [Maven]({{< ref "docs/dev/configuration/maven" >}}) 和 [Gradle]({{< ref "docs/dev/configuration/gradle" >}}) 指南。有关 Flink 发行版的参考,请查看[Flink 依赖剖析]({{< ref "docs/dev/configuration/overview" >}}#Flink-依赖剖析)。 + +{{< hint info >}} +决定是打成 uber JAR、精简 JAR 还是仅在发行版包含依赖项取决于您和您的使用场景。如果您使用 uber JAR,您将对作业里的依赖项版本有更多的控制权;如果您使用精简 JAR,由于您可以在不更改连接器版本的情况下更改版本(允许二进制兼容),您将对传递依赖项有更多的控制权;如果您直接在 Flink 发行版的 `/lib` 目录里内嵌连接器 uber JAR,您将能够在一处控制所有作业的连接器版本。 +{{< /hint >}} diff --git a/docs/content.zh/docs/dev/configuration/gradle.md b/docs/content.zh/docs/dev/configuration/gradle.md new file mode 100644 index 0000000000000..a7455060b51ef --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/gradle.md @@ -0,0 +1,92 @@ +--- +title: "使用 Gradle" +weight: 3 +type: docs +--- + + +# 如何使用 Gradle 配置您的项目 + +您可能需要一个构建工具来配置您的 Flink 项目,本指南将向您展示如何使用 [Gradle](https://gradle.org) 执行此操作。Gradle 是一个开源的通用构建工具,可用于在开发过程中自动化执行任务。 + +## 要求 + +- Gradle 7.x +- Java 11 + +## 将项目导入 IDE + +创建[项目目录和文件]({{< ref "docs/dev/configuration/overview#getting-started" >}})后,我们建议您将此项目导入到 IDE 进行开发和测试。 + +IntelliJ IDEA 通过 `Gradle` 插件支持 Gradle 项目。 + +Eclipse 通过 [Eclipse Buildship](https://projects.eclipse.org/projects/tools.buildship) 插件执行此操作(确保在导入向导的最后一步中指定 Gradle 版本 >= 3.0,`shadow` 插件会用到它)。您还可以使用 [Gradle 的 IDE 集成](https://docs.gradle.org/current/userguide/userguide.html#ide-integration) 来使用 Gradle 创建项目文件。 + +**注意:** Java 的默认 JVM 堆大小对于 Flink 来说可能太小,您应该手动增加它。在 Eclipse 中,选中 `Run Configurations -> Arguments` 并在 `VM Arguments` 框里填上:`-Xmx800m`。在 IntelliJ IDEA 中,推荐选中 `Help | Edit Custom VM Options` 菜单修改 JVM 属性。详情请查阅[本文](https://intellij-support.jetbrains.com/hc/en-us/articles/206544869-Configuring-JVM-options-and-platform-properties)。 + +**关于 IntelliJ 的注意事项:** 要使应用程序在 IntelliJ IDEA 中运行,需要在运行配置中的 `Include dependencies with "Provided" scope` 打勾。如果此选项不可用(可能是由于使用了较旧的 IntelliJ IDEA 版本),可创建一个调用应用程序 `main()` 方法的测试用例。 + +## 构建项目 + +如果您想 __构建/打包__ 您的项目,请转到您的项目目录并运行 '`gradle clean shadowJar`' 命令。您将 __找到一个 JAR 文件__,其中包含您的应用程序,还有已作为依赖项添加到应用程序的连接器和库:`build/libs/--all.jar`。 + +__注意:__ 如果您使用不同于 *StreamingJob* 的类作为应用程序的主类/入口点,我们建议您对 `build.gradle` 文件里的 `mainClassName` 配置进行相应的修改。这样,Flink 可以通过 JAR 文件运行应用程序,而无需额外指定主类。 + +## 向项目添加依赖项 + +在 `build.gradle` 文件的 dependencies 块中配置依赖项 + +例如,如果您使用我们的 Gradle 构建脚本或快速启动脚本创建了项目,如下所示,可以将 Kafka 连接器添加为依赖项: + +**build.gradle** + +```gradle +... +dependencies { + ... + flinkShadowJar "org.apache.flink:flink-connector-kafka:${flinkVersion}" + ... +} +... +``` + +**重要提示:** 请注意,应将所有这些(核心)依赖项的生效范围置为 [*provided*](https://maven.apache.org/guides/introduction/introduction-to-dependency-mechanism.html#dependency-scope)。这意味着需要对它们进行编译,但不应将它们打包进项目生成的应用程序 JAR 文件中。如果不设置为 *provided*,最好的情况是生成的 JAR 变得过大,因为它还包含所有 Flink 核心依赖项。最坏的情况是添加到应用程序 JAR 文件中的 Flink 核心依赖项与您自己的一些依赖项的版本冲突(通常通过反向类加载来避免)。 + +要将依赖项正确地打包进应用程序 JAR 中,必须把应用程序依赖项的生效范围设置为 *compile* 。 + +## 打包应用程序 + +在部署应用到 Flink 环境之前,您需要根据使用场景用不同的方式打包 Flink 应用程序。 + +如果您想为 Flink 作业创建 JAR 并且只使用 Flink 依赖而不使用任何第三方依赖(比如使用 JSON 格式的文件系统连接器),您不需要创建一个 uber/fat JAR 或将任何依赖打进包。 + +您可以使用 `gradle clean installDist` 命令,如果您使用的是 [Gradle Wrapper](https://docs.gradle.org/current/userguide/gradle_wrapper.html) ,则用 `./gradlew clean installDist`。 + +如果您想为 Flink 作业创建 JAR 并使用未内置在 Flink 发行版中的外部依赖项,您可以将它们添加到发行版的类路径中,或者将它们打包进您的 uber/fat 应用程序 JAR 中。 + +您可以使用该命令 `gradle clean installShadowDist`,该命令将在 `/build/install/yourProject/lib` 目录生成一个 fat JAR。如果您使用的是 [Gradle Wrapper](https://docs.gradle.org/current/userguide/gradle_wrapper.html) ,则用 `./gradlew clean installShadowDist`。 + +您可以将生成的 uber/fat JAR 提交到本地或远程集群: + +```sh +bin/flink run -c org.example.MyJob myFatJar.jar +``` + +要了解有关如何部署 Flink 作业的更多信息,请查看[部署指南]({{< ref "docs/deployment/cli" >}})。 diff --git a/docs/content.zh/docs/dev/configuration/maven.md b/docs/content.zh/docs/dev/configuration/maven.md new file mode 100644 index 0000000000000..bc1a71578257b --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/maven.md @@ -0,0 +1,142 @@ +--- +title: "使用 Maven" +weight: 2 +type: docs +--- + + +# 如何使用 Maven 配置您的项目 + +本指南将向您展示如何使用 [Maven](https://maven.apache.org) 配置 Flink 作业项目,Maven是 由 Apache Software Foundation 开源的自动化构建工具,使您能够构建、发布和部署项目。您可以使用它来管理软件项目的整个生命周期。 + +## 要求 + +- Maven 3.0.4 (or higher) +- Java 11 + +## 将项目导入 IDE + +创建[项目目录和文件]({{< ref "docs/dev/configuration/overview#getting-started" >}})后,我们建议您将此项目导入到 IDE 进行开发和测试。 + +IntelliJ IDEA 支持开箱即用的 Maven 项目。Eclipse 提供了 [m2e 插件](http://www.eclipse.org/m2e/) 来[导入 Maven 项目](http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import)。 + +**注意:** Java 的默认 JVM 堆大小对于 Flink 来说可能太小,您应该手动增加它。在 Eclipse 中,选中 `Run Configurations -> Arguments` 并在 `VM Arguments` 框里填上:`-Xmx800m`。在 IntelliJ IDEA 中,推荐选中 `Help | Edit Custom VM Options` 菜单修改 JVM 属性。详情请查阅[本文](https://intellij-support.jetbrains.com/hc/en-us/articles/206544869-Configuring-JVM-options-and-platform-properties)。 + +**关于 IntelliJ 的注意事项:** 要使应用程序在 IntelliJ IDEA 中运行,需要在运行配置中的 Include dependencies with "Provided" scope` 打勾。如果此选项不可用(可能是由于使用了较旧的 IntelliJ IDEA 版本),可创建一个调用应用程序 `main()` 方法的测试用例。 + +## 构建项目 + +如果您想 __构建/打包__ 您的项目,请转到您的项目目录并运行 '`mvn clean package`' 命令。您将 __找到一个 JAR 文件__,其中包含您的应用程序(还有已作为依赖项添加到应用程序的连接器和库):`target/-.jar`。 + +__注意:__ 如果您使用不同于 `DataStreamJob` 的类作为应用程序的主类/入口点,我们建议您对 `pom.xml` 文件里的 `mainClassName` 配置进行相应的修改。这样,Flink 可以通过 JAR 文件运行应用程序,而无需额外指定主类。 + +## 向项目添加依赖项 + +打开您项目目录的 `pom.xml`,在 `dependencies` 标签内添加依赖项。 + +例如,您可以用如下方式添加 Kafka 连接器依赖: + +```xml + + + + org.apache.flink + flink-connector-kafka + {{< version >}} + + + +``` + +然后在命令行执行 `mvn install`。 + +当您在由 `Java Project Template`、`Scala Project Template` 或 Gradle 创建出来的项目里,运行 `mvn clean package` 会自动将应用程序依赖打包进应用程序 JAR。对于不是通过这些模板创建的项目,我们建议使用 Maven Shade 插件以将所有必需的依赖项打包进应用程序 jar。 + +**重要提示:** 请注意,应将所有这些(核心)依赖项的生效范围置为 [*provided*](https://maven.apache.org/guides/introduction/introduction-to-dependency-mechanism.html#dependency-scope)。这意味着需要对它们进行编译,但不应将它们打包进项目生成的应用程序 JAR 文件中。如果不设置为 *provided*,最好的情况是生成的 JAR 变得过大,因为它还包含所有 Flink 核心依赖项。最坏的情况是添加到应用程序 JAR 文件中的 Flink 核心依赖项与您自己的一些依赖项的版本冲突(通常通过反向类加载来避免)。 + +要将依赖项正确地打包进应用程序 JAR 中,必须把应用程序依赖项的生效范围设置为 *compile* 。 + +## 打包应用程序 + +在部署应用到 Flink 环境之前,您需要根据使用场景用不同的方式打包 Flink 应用程序。 + +如果您想为 Flink 作业创建 JAR 并且只使用 Flink 依赖而不使用任何第三方依赖(比如使用 JSON 格式的文件系统连接器),您不需要创建一个 uber/fat JAR 或将任何依赖打进包。 + +如果您想为 Flink 作业创建 JAR 并使用未内置在 Flink 发行版中的外部依赖项,您可以将它们添加到发行版的类路径中,或者将它们打包进您的 uber/fat 应用程序 JAR 中。 + +您可以将生成的 uber/fat JAR 提交到本地或远程集群: + +```sh +bin/flink run -c org.example.MyJob myFatJar.jar +``` + +要了解有关如何部署 Flink 作业的更多信息,请查看[部署指南]({{< ref "docs/deployment/cli" >}})。 + +## 创建包含依赖项的 uber/fat JAR 的模板 + +为构建一个包含所有必需的连接器、 类库依赖项的应用程序 JAR,您可以使用如下 shade 插件定义: + +```xml + + + + org.apache.maven.plugins + maven-shade-plugin + 3.1.1 + + + package + + shade + + + + + com.google.code.findbugs:jsr305 + + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + my.programs.main.clazz + + + + + + + + + +``` + +[Maven shade 插件](https://maven.apache.org/plugins/maven-shade-plugin/index.html) 默认会包含所有的生效范围是 "runtime" 或 "compile" 的依赖项。 diff --git a/docs/content.zh/docs/dev/configuration/overview.md b/docs/content.zh/docs/dev/configuration/overview.md new file mode 100644 index 0000000000000..84f20abd67858 --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/overview.md @@ -0,0 +1,206 @@ +--- +title: "概览" +weight: 1 +type: docs +aliases: +- /dev/project-configuration.html +- /start/dependencies.html +- /getting-started/project-setup/dependencies.html +- /quickstart/java_api_quickstart.html +- /dev/projectsetup/java_api_quickstart.html +- /dev/linking_with_flink.html +- /dev/linking.html +- /dev/projectsetup/dependencies.html +- /dev/projectsetup/java_api_quickstart.html +- /getting-started/project-setup/java_api_quickstart.html +- /dev/getting-started/project-setup/scala_api_quickstart.html +- /getting-started/project-setup/scala_api_quickstart.html +- /quickstart/scala_api_quickstart.html +--- + + +# 项目配置 + +本节将向您展示如何通过流行的构建工具 ([Maven]({{< ref "docs/dev/configuration/maven" >}})、[Gradle]({{< ref "docs/dev/configuration/gradle" >}})) 配置您的项目,必要的依赖项(比如[连接器和格式]({{< ref "docs/dev/configuration/connector" >}})),以及覆盖一些[高级]({{< ref "docs/dev/configuration/advanced" >}})配置主题。 + +每个 Flink 应用程序都依赖于一组 Flink 库。应用程序至少依赖于 Flink API,此外还依赖于某些连接器库(比如 Kafka、Cassandra),以及用户开发的自定义的数据处理逻辑所需要的第三方依赖项。 + +## 开始 + +要开始使用 Flink 应用程序,请使用以下命令、脚本和模板来创建 Flink 项目。 + +{{< tabs "creating project" >}} +{{< tab "Maven" >}} + +您可以使用如下的 Maven 命令或快速启动脚本,基于[原型](https://maven.apache.org/guides/introduction/introduction-to-archetypes.html)创建一个项目。 + +### Maven 命令 +```bash +$ mvn archetype:generate \ + -DarchetypeGroupId=org.apache.flink \ + -DarchetypeArtifactId=flink-quickstart-java \ + -DarchetypeVersion={{< version >}} +``` +这允许您命名新建的项目,而且会交互式地询问 groupId、artifactId、package 的名字。 + +### 快速启动脚本 +```bash +$ curl https://flink.apache.org/q/quickstart.sh | bash -s {{< version >}} +``` + +{{< /tab >}} +{{< tab "Gradle" >}} +您可以使用如下的 Gradle 构建脚本或快速启动脚本创建一个项目。 + +### Gradle 构建脚本 + +请在脚本的所在目录执行 `gradle` 命令来执行这些构建配置脚本。 + +**build.gradle** + +```gradle +plugins { + id 'java' + id 'application' + // shadow plugin to produce fat JARs + id 'com.github.johnrengelman.shadow' version '7.1.2' +} +// artifact properties +group = 'org.quickstart' +version = '0.1-SNAPSHOT' +mainClassName = 'org.quickstart.StreamingJob' +mainClassName = 'org.quickstart.StreamingJob' +description = """Flink Quickstart Job""" +ext { + javaVersion = '1.8' + flinkVersion = '{{< version >}}' + slf4jVersion = '1.7.32' + log4jVersion = '2.17.1' +} +sourceCompatibility = javaVersion +targetCompatibility = javaVersion +tasks.withType(JavaCompile) { + options.encoding = 'UTF-8' +} +applicationDefaultJvmArgs = ["-Dlog4j.configurationFile=log4j2.properties"] + +// declare where to find the dependencies of your project +repositories { + mavenCentral() +} +// NOTE: We cannot use "compileOnly" or "shadow" configurations since then we could not run code +// in the IDE or with "gradle run". We also cannot exclude transitive dependencies from the +// shadowJar yet (see https://github.com/johnrengelman/shadow/issues/159). +// -> Explicitly define the // libraries we want to be included in the "flinkShadowJar" configuration! +configurations { + flinkShadowJar // dependencies which go into the shadowJar + // always exclude these (also from transitive dependencies) since they are provided by Flink + flinkShadowJar.exclude group: 'org.apache.flink', module: 'force-shading' + flinkShadowJar.exclude group: 'com.google.code.findbugs', module: 'jsr305' + flinkShadowJar.exclude group: 'org.slf4j' + flinkShadowJar.exclude group: 'org.apache.logging.log4j' +} +// declare the dependencies for your production and test code +dependencies { + // -------------------------------------------------------------- + // Compile-time dependencies that should NOT be part of the + // shadow (uber) jar and are provided in the lib folder of Flink + // -------------------------------------------------------------- + implementation "org.apache.flink:flink-streaming-java:${flinkVersion}" + implementation "org.apache.flink:flink-clients:${flinkVersion}" + // -------------------------------------------------------------- + // Dependencies that should be part of the shadow jar, e.g. + // connectors. These must be in the flinkShadowJar configuration! + // -------------------------------------------------------------- + //flinkShadowJar "org.apache.flink:flink-connector-kafka:${flinkVersion}" + runtimeOnly "org.apache.logging.log4j:log4j-api:${log4jVersion}" + runtimeOnly "org.apache.logging.log4j:log4j-core:${log4jVersion}" + runtimeOnly "org.apache.logging.log4j:log4j-slf4j-impl:${log4jVersion}" + runtimeOnly "org.slf4j:slf4j-log4j12:${slf4jVersion}" + // Add test dependencies here. + // testCompile "junit:junit:4.12" +} +// make compileOnly dependencies available for tests: +sourceSets { + main.compileClasspath += configurations.flinkShadowJar + main.runtimeClasspath += configurations.flinkShadowJar + test.compileClasspath += configurations.flinkShadowJar + test.runtimeClasspath += configurations.flinkShadowJar + javadoc.classpath += configurations.flinkShadowJar +} +run.classpath = sourceSets.main.runtimeClasspath + +shadowJar { + configurations = [project.configurations.flinkShadowJar] +} +``` + +**settings.gradle** + +```gradle +rootProject.name = 'quickstart' +``` + +### 快速启动脚本 + +```bash +bash -c "$(curl https://flink.apache.org/q/gradle-quickstart.sh)" -- {{< version >}} {{< scala_version >}} +``` +{{< /tab >}} +{{< /tabs >}} + +## 需要哪些依赖项? + +要开始一个 Flink 作业,您通常需要如下依赖项: + +* Flink API, 用来开发您的作业 +* [连接器和格式]({{< ref "docs/dev/configuration/connector" >}}), 以将您的作业与外部系统集成 +* [测试实用程序]({{< ref "docs/dev/configuration/testing" >}}), 以测试您的作业 + +除此之外,若要开发自定义功能,您还要添加必要的第三方依赖项。 + +### Flink API + +Flink提供了两大 API:[Datastream API]({{< ref "docs/dev/datastream/overview" >}}) 和 [Table API & SQL]({{< ref "docs/dev/table/overview" >}}),它们可以单独使用,也可以混合使用,具体取决于您的使用场景: + +| 您要使用的 API | 您需要添加的依赖项 | +|-----------------------------------------------------------------------------------|-----------------------------------------------------| +| [DataStream]({{< ref "docs/dev/datastream/overview" >}}) | `flink-streaming-java` | +| [DataStream Scala 版]({{< ref "docs/dev/datastream/scala_api_extensions" >}}) | `flink-streaming-scala{{< scala_version >}}` | +| [Table API]({{< ref "docs/dev/table/common" >}}) | `flink-table-api-java` | +| [Table API Scala 版]({{< ref "docs/dev/table/common" >}}) | `flink-table-api-scala{{< scala_version >}}` | +| [Table API + DataStream]({{< ref "docs/dev/table/data_stream_api" >}}) | `flink-table-api-java-bridge` | +| [Table API + DataStream Scala 版]({{< ref "docs/dev/table/data_stream_api" >}}) | `flink-table-api-scala-bridge{{< scala_version >}}` | + +您只需将它们包含在您的构建工具脚本/描述符中,就可以开发您的作业了! + +## 运行和打包 + +如果您想通过简单地执行主类来运行你的作业,您需要 classpath 里有 `flink-runtime`。对于 Table API 程序,您还需要 `flink-table-runtime` 和 `flink-table-planner-loader`。 + +根据经验,我们**建议**将应用程序代码及其所有必需的依赖项打包进一个 fat/uber JAR 中。这包括打包您作业用到的连接器、格式和第三方依赖项。此规则**不适用于** Java API、DataStream Scala API 以及前面提到的运行时模块,它们已经由 Flink 本身提供,**不应**包含在作业的 uber JAR 中。您可以把该作业 JAR 提交到已经运行的 Flink 集群,也可以轻松将其添加到 Flink 应用程序容器镜像中,而无需修改发行版。 + +## 下一步是什么? + +* 要开发您的作业,请查阅 [DataStream API]({{< ref "docs/dev/datastream/overview" >}}) 和 [Table API & SQL]({{< ref "docs/dev/table/overview" >}}); +* 关于如何使用特定的构建工具打包您的作业的更多细节,请查阅如下指南: + * [Maven]({{< ref "docs/dev/configuration/maven" >}}) + * [Gradle]({{< ref "docs/dev/configuration/gradle" >}}) +* 关于项目配置的高级内容,请查阅[高级主题]({{< ref "docs/dev/configuration/advanced" >}})部分。 diff --git a/docs/content.zh/docs/dev/configuration/testing.md b/docs/content.zh/docs/dev/configuration/testing.md new file mode 100644 index 0000000000000..33e0e9e8f5a8e --- /dev/null +++ b/docs/content.zh/docs/dev/configuration/testing.md @@ -0,0 +1,49 @@ +--- +title: "测试的依赖项" +weight: 6 +type: docs +--- + + +# 用于测试的依赖项 + +Flink 提供了用于测试作业的实用程序,您可以将其添加为依赖项。 + +## DataStream API 测试 + +如果要为使用 DataStream API 构建的作业开发测试用例,则需要添加以下依赖项: + +{{< artifact_tabs flink-test-utils withTestScope >}} + +在各种测试实用程序中,该模块提供了 `MiniCluster` (一个可配置的轻量级 Flink 集群,能在 JUnit 测试中运行),可以直接执行作业。 + +有关如何使用这些实用程序的更多细节,请查看 [DataStream API 测试]({{< ref "docs/dev/datastream/testing" >}})。 + +## Table API 测试 + +如果您想在您的 IDE 中本地测试 Table API 和 SQL 程序,除了前述提到的 `flink-test-utils` 之外,您还要添加以下依赖项: + +{{< artifact_tabs flink-table-test-utils withTestScope >}} + +这将自动引入查询计划器和运行时,分别用于计划和执行查询。 + +{{< hint info >}} +`flink-table-test-utils` 模块已在 Flink 1.15 中引入,目前被认为是实验性的。 +{{< /hint >}} diff --git a/docs/content.zh/docs/dev/datastream/_index.md b/docs/content.zh/docs/dev/datastream/_index.md index 1a3281350eb61..5e035cc14d0e3 100644 --- a/docs/content.zh/docs/dev/datastream/_index.md +++ b/docs/content.zh/docs/dev/datastream/_index.md @@ -1,7 +1,7 @@ --- title: DataStream API bookCollapseSection: true -weight: 1 +weight: 2 --- \ No newline at end of file +--> diff --git a/docs/content.zh/docs/dev/datastream/fault-tolerance/queryable_state.md b/docs/content.zh/docs/dev/datastream/fault-tolerance/queryable_state.md index 982854e3f8757..3c2c7e5c2d5cc 100644 --- a/docs/content.zh/docs/dev/datastream/fault-tolerance/queryable_state.md +++ b/docs/content.zh/docs/dev/datastream/fault-tolerance/queryable_state.md @@ -143,7 +143,7 @@ descriptor.setQueryable("query-name"); // queryable state name ``` -关于依赖的更多信息, 可以参考如何 [配置 Flink 项目]({{< ref "docs/dev/datastream/project-configuration" >}}). +关于依赖的更多信息, 可以参考如何[配置 Flink 项目]({{< ref "docs/dev/configuration/overview" >}})。 `QueryableStateClient` 将提交你的请求到内部代理,代理会处理请求并返回结果。客户端的初始化只需要提供一个有效的 `TaskManager` 主机名 (每个 task manager 上都运行着一个 queryable state 代理),以及代理监听的端口号。关于如何配置代理以及端口号可以参考 [Configuration Section](#configuration). diff --git a/docs/content.zh/docs/dev/datastream/project-configuration.md b/docs/content.zh/docs/dev/datastream/project-configuration.md deleted file mode 100644 index 24c8bf88939d5..0000000000000 --- a/docs/content.zh/docs/dev/datastream/project-configuration.md +++ /dev/null @@ -1,570 +0,0 @@ ---- -title: "Project Configuration" -weight: 302 -type: docs -aliases: - - /zh/dev/project-configuration.html - - /zh/start/dependencies.html - - /zh/getting-started/project-setup/dependencies.html - - /zh/quickstart/java_api_quickstart.html - - /zh/dev/projectsetup/java_api_quickstart.html - - /zh/dev/linking_with_flink.html - - /zh/dev/linking.html - - /zh/dev/projectsetup/dependencies.html - - /zh/dev/projectsetup/java_api_quickstart.html - - /zh/getting-started/project-setup/java_api_quickstart.html - - /zh/dev/projectsetup/scala_api_quickstart.html - - /zh/getting-started/project-setup/scala_api_quickstart.html - - /zh/quickstart/scala_api_quickstart.html ---- - - -# Project Configuration - -Every Flink application depends on a set of Flink libraries. At the bare minimum, the application depends -on the Flink APIs. Many applications depend in addition on certain connector libraries (like Kafka, Cassandra, etc.). -When running Flink applications (either in a distributed deployment, or in the IDE for testing), the Flink -runtime library must be available as well. - -## Flink Core and Application Dependencies - -As with most systems that run user-defined applications, there are two broad categories of dependencies and libraries in Flink: - - - **Flink Core Dependencies**: Flink itself consists of a set of classes and dependencies that are needed to run the system, for example - coordination, networking, checkpoints, failover, APIs, operations (such as windowing), resource management, etc. - The set of all these classes and dependencies forms the core of Flink's runtime and must be present when a Flink - application is started. - - These core classes and dependencies are packaged in the `flink-dist` jar. They are part of Flink's `lib` folder and - part of the basic Flink container images. Think of these dependencies as similar to Java's core library (`rt.jar`, `charsets.jar`, etc.), - which contains the classes like `String` and `List`. - - The Flink Core Dependencies do not contain any connectors or libraries (CEP, SQL, ML, etc.) in order to avoid having an excessive - number of dependencies and classes in the classpath by default. In fact, we try to keep the core dependencies as slim as possible - to keep the default classpath small and avoid dependency clashes. - - - The **User Application Dependencies** are all connectors, formats, or libraries that a specific user application needs. - - The user application is typically packaged into an *application jar*, which contains the application code and the required - connector and library dependencies. - - The user application dependencies explicitly do not include the Flink DataStream APIs and runtime dependencies, - because those are already part of Flink's Core Dependencies. - - -## Setting up a Project: Basic Dependencies - -Every Flink application needs as the bare minimum the API dependencies, to develop against. - -When setting up a project manually, you need to add the following dependencies for the Java/Scala API -(here presented in Maven syntax, but the same dependencies apply to other build tools (Gradle, SBT, etc.) as well. - -{{< tabs "a49d57a4-27ee-4dd3-a2b8-a673b99b011e" >}} -{{< tab "Java" >}} -```xml - - org.apache.flink - flink-streaming-java - {{< version >}} - provided - -``` -{{< /tab >}} -{{< tab "Scala" >}} -```xml - - org.apache.flink - flink-streaming-scala{{< scala_version >}} - {{< version >}} - provided - -``` -{{< /tab >}} -{{< /tabs >}} - -**Important:** Please note that all these dependencies have their scope set to *provided*. -That means that they are needed to compile against, but that they should not be packaged into the -project's resulting application jar file - these dependencies are Flink Core Dependencies, -which are already available in any setup. - -It is highly recommended keeping the dependencies in scope *provided*. If they are not set to *provided*, -the best case is that the resulting JAR becomes excessively large, because it also contains all Flink core -dependencies. The worst case is that the Flink core dependencies that are added to the application's jar file -clash with some of your own dependency versions (which is normally avoided through inverted classloading). - -**Note on IntelliJ:** To make the applications run within IntelliJ IDEA it is necessary to tick the -`Include dependencies with "Provided" scope` box in the run configuration. -If this option is not available (possibly due to using an older IntelliJ IDEA version), then a simple workaround -is to create a test that calls the applications `main()` method. - - -## Adding Connector and Library Dependencies - -Most applications need specific connectors or libraries to run, for example a connector to Kafka, Cassandra, etc. -These connectors are not part of Flink's core dependencies and must be added as dependencies to the application. - -Below is an example adding the connector for Kafka as a dependency (Maven syntax): -```xml - - org.apache.flink - flink-connector-kafka - {{< version >}} - -``` - -We recommend packaging the application code and all its required dependencies into one *jar-with-dependencies* which -we refer to as the *application jar*. The application jar can be submitted to an already running Flink cluster, -or added to a Flink application container image. - -Projects created from the [Java Project Template]({{< ref "docs/dev/datastream/project-configuration" >}}) or -[Scala Project Template]({{< ref "docs/dev/datastream/project-configuration" >}}) are configured to automatically include -the application dependencies into the application jar when running `mvn clean package`. For projects that are -not set up from those templates, we recommend adding the Maven Shade Plugin (as listed in the Appendix below) -to build the application jar with all required dependencies. - -**Important:** For Maven (and other build tools) to correctly package the dependencies into the application jar, -these application dependencies must be specified in scope *compile* (unlike the core dependencies, which -must be specified in scope *provided*). - - -## Scala Versions - -Scala versions (2.11, 2.12, etc.) are not binary compatible with one another. -For that reason, Flink for Scala 2.11 cannot be used with an application that uses -Scala 2.12. - -All Flink dependencies that (transitively) depend on Scala are suffixed with the -Scala version that they are built for, for example `flink-streaming-scala_2.12`. - -Developers that only use Java can pick any Scala version, Scala developers need to -pick the Scala version that matches their application's Scala version. - -Please refer to the [build guide]({{< ref "docs/flinkDev/building" >}}#scala-versions) -for details on how to build Flink for a specific Scala version. - -Scala versions after 2.12.8 are not binary compatible with previous 2.12.x -versions, preventing the Flink project from upgrading its 2.12.x builds beyond -2.12.8. Users can build Flink locally for latter Scala versions by following -the above mentioned [build guide]({{< ref "docs/flinkDev/building" >}}#scala-versions). -For this to work, users need to add `-Djapicmp.skip` to -skip binary compatibility checks when building. - -See the [Scala 2.12.8 release notes](https://github.com/scala/scala/releases/tag/v2.12.8) for more details, -the relevant quote is this: - -> The second fix is not binary compatible: the 2.12.8 compiler omits certain -> methods that are generated by earlier 2.12 compilers. However, we believe -> that these methods are never used and existing compiled code will continue to -> work. See the [pull request -> description](https://github.com/scala/scala/pull/7469) for more details. - -## Hadoop Dependencies - -**General rule: It should never be necessary to add Hadoop dependencies directly to your application.** -*(The only exception being when using existing Hadoop input-/output formats with Flink's Hadoop compatibility wrappers)* - -If you want to use Flink with Hadoop, you need to have a Flink setup that includes the Hadoop dependencies, rather than -adding Hadoop as an application dependency. Flink will use the Hadoop dependencies specified by the `HADOOP_CLASSPATH` -environment variable, which can be set in the following way: - -```bash -export HADOOP_CLASSPATH=`hadoop classpath` -``` - -There are two main reasons for that design: - - - Some Hadoop interaction happens in Flink's core, possibly before the user application is started, for example - setting up HDFS for checkpoints, authenticating via Hadoop's Kerberos tokens, or deployment on YARN. - - - Flink's inverted classloading approach hides many transitive dependencies from the core dependencies. That applies not only - to Flink's own core dependencies, but also to Hadoop's dependencies when present in the setup. - That way, applications can use different versions of the same dependencies without running into dependency conflicts (and - trust us, that's a big deal, because Hadoops dependency tree is huge.) - -If you need Hadoop dependencies during testing or development inside the IDE (for example for HDFS access), please configure -these dependencies similar to the scope of the dependencies to *test* or to *provided*. - -## Maven Quickstart - -#### Requirements - -The only requirements are working __Maven 3.0.4__ (or higher) and __Java 11__ installations. - -#### Create Project - -Use one of the following commands to __create a project__: - -{{< tabs "maven" >}} -{{< tab "Maven Archetypes" >}} -```bash -$ mvn archetype:generate \ - -DarchetypeGroupId=org.apache.flink \ - -DarchetypeArtifactId=flink-quickstart-java \ - -DarchetypeVersion={{< version >}} -``` -This allows you to **name your newly created project**. -It will interactively ask you for the groupId, artifactId, and package name. -{{< /tab >}} -{{< tab "Quickstart Script" >}} -{{< stable >}} -```bash -$ curl https://flink.apache.org/q/quickstart.sh | bash -s {{< version >}} -``` -{{< /stable >}} -{{< unstable >}} -```bash -$ curl https://flink.apache.org/q/quickstart-SNAPSHOT.sh | bash -s {{< version >}} - -``` -{{< /unstable >}} -{{< /tab >}} -{{< /tabs >}} - -{{< unstable >}} -{{< hint info >}} -For Maven 3.0 or higher, it is no longer possible to specify the repository (-DarchetypeCatalog) via the command line. For details about this change, please refer to Maven official document -If you wish to use the snapshot repository, you need to add a repository entry to your settings.xml. For example: - -```xml - - - apache - - - - apache - - - apache-snapshots - https://repository.apache.org/content/repositories/snapshots/ - - - - - -``` - -{{< /hint >}} -{{< /unstable >}} - -We recommend you __import this project into your IDE__ to develop and -test it. IntelliJ IDEA supports Maven projects out of the box. -If you use Eclipse, the [m2e plugin](http://www.eclipse.org/m2e/) -allows to [import Maven projects](http://books.sonatype.com/m2eclipse-book/reference/creating-sect-importing-projects.html#fig-creating-import). -Some Eclipse bundles include that plugin by default, others require you -to install it manually. - -*Please note*: The default JVM heapsize for Java may be too -small for Flink. You have to manually increase it. -In Eclipse, choose `Run Configurations -> Arguments` and write into the `VM Arguments` box: `-Xmx800m`. -In IntelliJ IDEA recommended way to change JVM options is from the `Help | Edit Custom VM Options` menu. See [this article](https://intellij-support.jetbrains.com/hc/en-us/articles/206544869-Configuring-JVM-options-and-platform-properties) for details. - - -#### Build Project - -If you want to __build/package your project__, go to your project directory and -run the '`mvn clean package`' command. -You will __find a JAR file__ that contains your application, plus connectors and libraries -that you may have added as dependencies to the application: `target/-.jar`. - -__Note:__ If you use a different class than *StreamingJob* as the application's main class / entry point, -we recommend you change the `mainClass` setting in the `pom.xml` file accordingly. That way, Flink -can run the application from the JAR file without additionally specifying the main class. - -## Gradle - -#### Requirements - -The only requirements are working __Gradle 3.x__ (or higher) and __Java 11__ installations. - -#### Create Project - -Use one of the following commands to __create a project__: - -{{< tabs gradle >}} -{{< tab "Gradle Example" >}} -**build.gradle** - -```gradle -buildscript { - repositories { - jcenter() // this applies only to the Gradle 'Shadow' plugin - } - dependencies { - classpath 'com.github.jengelman.gradle.plugins:shadow:2.0.4' - } -} - -plugins { - id 'java' - id 'application' - // shadow plugin to produce fat JARs - id 'com.github.johnrengelman.shadow' version '2.0.4' -} - - -// artifact properties -group = 'org.myorg.quickstart' -version = '0.1-SNAPSHOT' -mainClassName = 'org.myorg.quickstart.StreamingJob' -description = """Flink Quickstart Job""" - -ext { - javaVersion = '1.8' - flinkVersion = '1.13-SNAPSHOT' - scalaBinaryVersion = '2.11' - slf4jVersion = '1.7.32' - log4jVersion = '2.17.1' -} - - -sourceCompatibility = javaVersion -targetCompatibility = javaVersion -tasks.withType(JavaCompile) { - options.encoding = 'UTF-8' -} - -applicationDefaultJvmArgs = ["-Dlog4j.configurationFile=log4j2.properties"] - -task wrapper(type: Wrapper) { - gradleVersion = '3.1' -} - -// declare where to find the dependencies of your project -repositories { - mavenCentral() - maven { url "https://repository.apache.org/content/repositories/snapshots/" } -} - -// NOTE: We cannot use "compileOnly" or "shadow" configurations since then we could not run code -// in the IDE or with "gradle run". We also cannot exclude transitive dependencies from the -// shadowJar yet (see https://github.com/johnrengelman/shadow/issues/159). -// -> Explicitly define the // libraries we want to be included in the "flinkShadowJar" configuration! -configurations { - flinkShadowJar // dependencies which go into the shadowJar - - // always exclude these (also from transitive dependencies) since they are provided by Flink - flinkShadowJar.exclude group: 'org.apache.flink', module: 'force-shading' - flinkShadowJar.exclude group: 'com.google.code.findbugs', module: 'jsr305' - flinkShadowJar.exclude group: 'org.slf4j' - flinkShadowJar.exclude group: 'org.apache.logging.log4j' -} - -// declare the dependencies for your production and test code -dependencies { - // -------------------------------------------------------------- - // Compile-time dependencies that should NOT be part of the - // shadow jar and are provided in the lib folder of Flink - // -------------------------------------------------------------- - compile "org.apache.flink:flink-streaming-java:${flinkVersion}" - - // -------------------------------------------------------------- - // Dependencies that should be part of the shadow jar, e.g. - // connectors. These must be in the flinkShadowJar configuration! - // -------------------------------------------------------------- - //flinkShadowJar "org.apache.flink:flink-connector-kafka:${flinkVersion}" - - compile "org.apache.logging.log4j:log4j-api:${log4jVersion}" - compile "org.apache.logging.log4j:log4j-core:${log4jVersion}" - compile "org.apache.logging.log4j:log4j-slf4j-impl:${log4jVersion}" - compile "org.slf4j:slf4j-log4j12:${slf4jVersion}" - - // Add test dependencies here. - // testCompile "junit:junit:4.12" -} - -// make compileOnly dependencies available for tests: -sourceSets { - main.compileClasspath += configurations.flinkShadowJar - main.runtimeClasspath += configurations.flinkShadowJar - - test.compileClasspath += configurations.flinkShadowJar - test.runtimeClasspath += configurations.flinkShadowJar - - javadoc.classpath += configurations.flinkShadowJar -} - -run.classpath = sourceSets.main.runtimeClasspath - -jar { - manifest { - attributes 'Built-By': System.getProperty('user.name'), - 'Build-Jdk': System.getProperty('java.version') - } -} - -shadowJar { - configurations = [project.configurations.flinkShadowJar] -} -``` - -**settings.gradle** -```gradle -rootProject.name = 'quickstart' -``` -{{< /tab >}} -{{< tab "Quickstart Script">}} -```bash -bash -c "$(curl https://flink.apache.org/q/gradle-quickstart.sh)" -- {{< version >}} {{< scala_version >}} -``` -{{< /tab >}} -{{< /tabs >}} - -We recommend you __import this project into your IDE__ to develop and -test it. IntelliJ IDEA supports Gradle projects after installing the `Gradle` plugin. -Eclipse does so via the [Eclipse Buildship](https://projects.eclipse.org/projects/tools.buildship) plugin -(make sure to specify a Gradle version >= 3.0 in the last step of the import wizard; the `shadow` plugin requires it). -You may also use [Gradle's IDE integration](https://docs.gradle.org/current/userguide/userguide.html#ide-integration) -to create project files from Gradle. - - -*Please note*: The default JVM heapsize for Java may be too -small for Flink. You have to manually increase it. -In Eclipse, choose `Run Configurations -> Arguments` and write into the `VM Arguments` box: `-Xmx800m`. -In IntelliJ IDEA recommended way to change JVM options is from the `Help | Edit Custom VM Options` menu. See [this article](https://intellij-support.jetbrains.com/hc/en-us/articles/206544869-Configuring-JVM-options-and-platform-properties) for details. - -#### Build Project - -If you want to __build/package your project__, go to your project directory and -run the '`gradle clean shadowJar`' command. -You will __find a JAR file__ that contains your application, plus connectors and libraries -that you may have added as dependencies to the application: `build/libs/--all.jar`. - -__Note:__ If you use a different class than *DataStreamJob* as the application's main class / entry point, -we recommend you change the `mainClassName` setting in the `build.gradle` file accordingly. That way, Flink -can run the application from the JAR file without additionally specifying the main class. - -## SBT - -#### Create Project - -You can scaffold a new project via either of the following two methods: - -{{< tabs sbt >}} -{{< tab "SBT Template" >}} -```bash -$ sbt new tillrohrmann/flink-project.g8 -``` -{{< /tab >}} -{{< tab "Quickstart Script" >}} -```bash -$ bash <(curl https://flink.apache.org/q/sbt-quickstart.sh) -``` -{{< /tab >}} -{{< /tabs >}} - -#### Build Project - -In order to build your project you simply have to issue the `sbt clean assembly` command. -This will create the fat-jar __your-project-name-assembly-0.1-SNAPSHOT.jar__ in the directory __target/scala_your-major-scala-version/__. - -#### Run Project - -In order to run your project you have to issue the `sbt run` command. - -Per default, this will run your job in the same JVM as `sbt` is running. -In order to run your job in a distinct JVM, add the following line to `build.sbt` - -```scala -fork in run := true -``` - -#### IntelliJ - -We recommend using [IntelliJ](https://www.jetbrains.com/idea/) for your Flink job development. -In order to get started, you have to import your newly created project into IntelliJ. -You can do this via `File -> New -> Project from Existing Sources...` and then choosing your project's directory. -IntelliJ will then automatically detect the `build.sbt` file and set everything up. - -In order to run your Flink job, it is recommended to choose the `mainRunner` module as the classpath of your __Run/Debug Configuration__. -This will ensure, that all dependencies which are set to _provided_ will be available upon execution. -You can configure the __Run/Debug Configurations__ via `Run -> Edit Configurations...` and then choose `mainRunner` from the _Use classpath of module_ dropbox. - -#### Eclipse - -In order to import the newly created project into [Eclipse](https://eclipse.org/), you first have to create Eclipse project files for it. -These project files can be created via the [sbteclipse](https://github.com/typesafehub/sbteclipse) plugin. -Add the following line to your `PROJECT_DIR/project/plugins.sbt` file: - -```bash -addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "4.0.0") -``` - -In `sbt` use the following command to create the Eclipse project files - -```bash -> eclipse -``` - -Now you can import the project into Eclipse via `File -> Import... -> Existing Projects into Workspace` and then select the project directory. - - -## Appendix: Template for building a Jar with Dependencies - -To build an application JAR that contains all dependencies required for declared connectors and libraries, -you can use the following shade plugin definition: - -```xml - - - - org.apache.maven.plugins - maven-shade-plugin - 3.1.1 - - - package - - shade - - - - - com.google.code.findbugs:jsr305 - org.slf4j:* - log4j:* - - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - my.programs.main.clazz - - - - - - - - -``` - -{{< top >}} diff --git a/docs/content.zh/docs/dev/datastream/testing.md b/docs/content.zh/docs/dev/datastream/testing.md index c8e709366665e..5fdd52c746c3d 100644 --- a/docs/content.zh/docs/dev/datastream/testing.md +++ b/docs/content.zh/docs/dev/datastream/testing.md @@ -151,11 +151,7 @@ class IncrementFlatMapFunctionTest extends FlatSpec with MockFactory { * `TwoInputStreamOperatorTestHarness` (f适用于两个 `DataStream` 的 `ConnectedStreams` 算子) * `KeyedTwoInputStreamOperatorTestHarness` (适用于两个 `KeyedStream` 上的 `ConnectedStreams` 算子) -要使用测试工具,还需要一组其他的依赖项(测试范围)。 - -{{< artifact flink-test-utils withTestScope >}} -{{< artifact flink-runtime withTestScope >}} -{{< artifact flink-streaming-java withTestScope withTestClassifier >}} +要使用测试工具,还需要一组其他的依赖项,请查阅[配置]({{< ref "docs/dev/configuration/testing" >}})小节了解更多细节。 现在,可以使用测试工具将记录和 watermark 推送到用户自定义函数或自定义算子中,控制处理时间,最后对算子的输出(包括旁路输出)进行校验。 diff --git a/docs/content.zh/docs/dev/table/data_stream_api.md b/docs/content.zh/docs/dev/table/data_stream_api.md index ff12448050eec..3c91213bf25ba 100644 --- a/docs/content.zh/docs/dev/table/data_stream_api.md +++ b/docs/content.zh/docs/dev/table/data_stream_api.md @@ -467,6 +467,8 @@ import org.apache.flink.table.api.bridge.scala._ {{< /tab >}} {{< /tabs >}} +请查阅[配置]({{< ref "docs/dev/configuration/overview" >}})小节了解更多细节。 + ### Configuration The `TableEnvironment` will adopt all configuration options from the passed `StreamExecutionEnvironment`. diff --git a/docs/content.zh/docs/dev/table/overview.md b/docs/content.zh/docs/dev/table/overview.md index b3038cb0814dd..3365f0d0dafa6 100644 --- a/docs/content.zh/docs/dev/table/overview.md +++ b/docs/content.zh/docs/dev/table/overview.md @@ -33,74 +33,11 @@ Table API 和 SQL 两种 API 是紧密集成的,以及 DataStream API。你可 ## Table 程序依赖 -取决于你使用的编程语言,选择 Java 或者 Scala API 来构建你的 Table API 和 SQL 程序: +您需要将 Table API 作为依赖项添加到项目中,以便用 Table API 和 SQL 定义数据管道。 -{{< tabs "94f8aceb-507f-4c8f-977e-df00fe903203" >}} -{{< tab "Java" >}} -```xml - - org.apache.flink - flink-table-api-java-bridge{{< scala_version >}} - {{< version >}} - provided - -``` -{{< /tab >}} -{{< tab "Scala" >}} -```xml - - org.apache.flink - flink-table-api-scala-bridge{{< scala_version >}} - {{< version >}} - provided - -``` -{{< /tab >}} -{{< tab "Python" >}} -{{< stable >}} -```bash -$ python -m pip install apache-flink {{< version >}} -``` -{{< /stable >}} -{{< unstable >}} -```bash -$ python -m pip install apache-flink -``` -{{< /unstable >}} -{{< /tab >}} -{{< /tabs >}} +有关如何为 Java 和 Scala 配置这些依赖项的更多细节,请查阅[项目配置]({{< ref "docs/dev/configuration/overview" >}})小节。 -除此之外,如果你想在 IDE 本地运行你的程序,你需要添加下面的模块,具体用哪个取决于你使用哪个 Planner: - -```xml - - org.apache.flink - flink-table-planner{{< scala_version >}} - {{< version >}} - provided - - - org.apache.flink - flink-streaming-scala{{< scala_version >}} - {{< version >}} - provided - -``` - -### 扩展依赖 - -如果你想实现[自定义格式或连接器]({{< ref "docs/dev/table/sourcesSinks" >}}) 用于(反)序列化行或一组[用户定义的函数]({{< ref "docs/dev/table/functions/udfs" >}}),下面的依赖就足够了,编译出来的 jar 文件可以直接给 SQL Client 使用: - -```xml - - org.apache.flink - flink-table-common - {{< version >}} - provided - -``` - -{{< top >}} +如果您使用 Python,请查阅 [Python API]({{< ref "docs/dev/python/overview" >}}) 文档。 接下来? ----------------- diff --git a/docs/content.zh/docs/dev/table/sourcesSinks.md b/docs/content.zh/docs/dev/table/sourcesSinks.md index 4f1c5080d68ce..d1940bab57929 100644 --- a/docs/content.zh/docs/dev/table/sourcesSinks.md +++ b/docs/content.zh/docs/dev/table/sourcesSinks.md @@ -106,6 +106,33 @@ that the planner can handle. {{< top >}} + +Project Configuration +--------------------- + +If you want to implement a custom connector or a custom format, the following dependency is usually +sufficient: + +{{< artifact_tabs flink-table-common withProvidedScope >}} + +If you want to develop a connector that needs to bridge with DataStream APIs (i.e. if you want to adapt +a DataStream connector to the Table API), you need to add this dependency: + +{{< artifact_tabs flink-table-api-java-bridge withProvidedScope >}} + +When developing the connector/format, we suggest shipping both a thin JAR and an uber JAR, so users +can easily load the uber JAR in the SQL client or in the Flink distribution and start using it. +The uber JAR should include all the third-party dependencies of the connector, +excluding the table dependencies listed above. + +{{< hint warning >}} +You should not depend on `flink-table-planner{{< scala_version >}}` in production code. +With the new module `flink-table-planner-loader` introduced in Flink 1.15, the +application's classpath will not have direct access to `org.apache.flink.table.planner` classes anymore. +If you need a feature available only internally within the `org.apache.flink.table.planner` package and subpackages, please open an issue. +To learn more, check out [Anatomy of Table Dependencies]({{< ref "docs/dev/configuration/advanced" >}}#anatomy-of-table-dependencies). +{{< /hint >}} + Extension Points ---------------- diff --git a/docs/content.zh/docs/dev/table/sql/queries/match_recognize.md b/docs/content.zh/docs/dev/table/sql/queries/match_recognize.md index e6b11fc99af4c..9f3e15b0edd0d 100644 --- a/docs/content.zh/docs/dev/table/sql/queries/match_recognize.md +++ b/docs/content.zh/docs/dev/table/sql/queries/match_recognize.md @@ -84,7 +84,7 @@ Flink 的 `MATCH_RECOGNIZE` 子句实现是一个完整标准子集。仅支持 ``` -或者,也可以将依赖项添加到集群的 classpath(查看 [dependency section]({{< ref "docs/dev/datastream/project-configuration" >}}) 获取更多相关依赖信息)。 +或者,也可以将依赖项添加到集群的 classpath(查看 [dependency section]({{< ref "docs/dev/configuration/overview" >}}) 获取更多相关依赖信息)。 如果你想在 [SQL Client]({{< ref "docs/dev/table/sqlClient" >}}) 中使用 `MATCH_RECOGNIZE` 子句,你无需执行任何操作,因为默认情况下包含所有依赖项。 diff --git a/docs/content.zh/docs/dev/table/sqlClient.md b/docs/content.zh/docs/dev/table/sqlClient.md index 2239121c50c8f..6f6067010b36a 100644 --- a/docs/content.zh/docs/dev/table/sqlClient.md +++ b/docs/content.zh/docs/dev/table/sqlClient.md @@ -368,16 +368,17 @@ When execute queries or insert statements, please enter the interactive mode or ### Dependencies -The SQL Client does not require to setup a Java project using Maven or SBT. Instead, you can pass the -dependencies as regular JAR files that get submitted to the cluster. You can either specify each JAR -file separately (using `--jar`) or define entire library directories (using `--library`). For +The SQL Client does not require setting up a Java project using Maven, Gradle, or sbt. Instead, you +can pass the dependencies as regular JAR files that get submitted to the cluster. You can either specify +each JAR file separately (using `--jar`) or define entire library directories (using `--library`). For connectors to external systems (such as Apache Kafka) and corresponding data formats (such as JSON), Flink provides **ready-to-use JAR bundles**. These JAR files can be downloaded for each release from the Maven central repository. -The full list of offered SQL JARs and documentation about how to use them can be found on the [connection to external systems page]({{< ref "docs/connectors/table/overview" >}}). +The full list of offered SQL JARs can be found on the [connection to external systems page]({{< ref "docs/connectors/table/overview" >}}). -{{< top >}} +You can refer to the [configuration]({{< ref "docs/dev/configuration/connector" >}}) section for +information on how to configure connector and format dependencies. Use SQL Client to submit job ---------------------------- diff --git a/docs/content.zh/docs/flinkDev/ide_setup.md b/docs/content.zh/docs/flinkDev/ide_setup.md index f17b32d36519a..a4c985917b328 100644 --- a/docs/content.zh/docs/flinkDev/ide_setup.md +++ b/docs/content.zh/docs/flinkDev/ide_setup.md @@ -28,7 +28,7 @@ under the License. # 导入 Flink 到 IDE 中 -以下章节描述了如何将 Flink 项目导入到 IDE 中以进行 Flink 本身的源码开发。有关 Flink 程序编写的信息,请参阅 [Java API]({{< ref "docs/dev/datastream/project-configuration" >}}) 和 [Scala API]({{< ref "docs/dev/datastream/project-configuration" >}}) 快速入门指南。 +以下章节描述了如何将 Flink 项目导入到 IDE 中以进行 Flink 本身的源码开发。有关 Flink 程序编写的信息,请参阅 [Java API]({{< ref "docs/dev/configuration/overview" >}}) 和 [Scala API]({{< ref "docs/dev/configuration/overview" >}}) 快速入门指南。 {{< hint info >}} 每当你的 IDE 无法正常工作时,请优先尝试使用 Maven 命令行(`mvn clean package -DskipTests`),因为它可能是由于你的 IDE 中存在错误或未正确设置。 diff --git a/docs/content.zh/docs/libs/cep.md b/docs/content.zh/docs/libs/cep.md index a8f53ed572c29..f2b67750dcb4c 100644 --- a/docs/content.zh/docs/libs/cep.md +++ b/docs/content.zh/docs/libs/cep.md @@ -38,8 +38,8 @@ FlinkCEP是在Flink上层实现的复杂事件处理库。 ## 开始 -如果你想现在开始尝试,[创建一个Flink程序]({{< ref "docs/dev/datastream/project-configuration" >}}), -添加FlinkCEP的依赖到项目的`pom.xml`文件中。 +如果你想现在开始尝试,[创建一个 Flink 程序]({{< ref "docs/dev/configuration/overview" >}}), +添加 FlinkCEP 的依赖到项目的`pom.xml`文件中。 {{< tabs "722d55a5-7f12-4bcc-b080-b28d5e8860ac" >}} {{< tab "Java" >}} @@ -51,7 +51,7 @@ FlinkCEP是在Flink上层实现的复杂事件处理库。 {{< /tabs >}} {{< hint info >}} -FlinkCEP不是二进制发布包的一部分。在集群上执行如何链接它可以看[这里]({{< ref "docs/dev/datastream/project-configuration" >}})。 +FlinkCEP 不是二进制发布包的一部分。在集群上执行如何链接它可以看[这里]({{< ref "docs/dev/configuration/overview" >}})。 {{< /hint >}} 现在可以开始使用Pattern API写你的第一个CEP程序了。 diff --git a/docs/content.zh/docs/libs/gelly/overview.md b/docs/content.zh/docs/libs/gelly/overview.md index 1c8248cf4ed09..3abe4e54156f7 100644 --- a/docs/content.zh/docs/libs/gelly/overview.md +++ b/docs/content.zh/docs/libs/gelly/overview.md @@ -52,7 +52,7 @@ Add the following dependency to your `pom.xml` to use Gelly. {{< /tab >}} {{< /tabs >}} -Note that Gelly is not part of the binary distribution. See [linking]({{< ref "docs/dev/datastream/project-configuration" >}}) for +Note that Gelly is not part of the binary distribution. See [linking]({{< ref "docs/dev/configuration/overview" >}}) for instructions on packaging Gelly libraries into Flink user programs. The remaining sections provide a description of available methods and present several examples of how to use Gelly and how to mix it with the Flink DataSet API. From 94d03f1e3a95f21585353dc74407c6f060ffa879 Mon Sep 17 00:00:00 2001 From: Paul Zhang Date: Mon, 7 Mar 2022 10:13:17 +0800 Subject: [PATCH 042/258] [FLINK-26381][docs] Wrong document order of Chinese version --- docs/content.zh/docs/learn-flink/datastream_api.md | 2 +- docs/content.zh/docs/learn-flink/event_driven.md | 2 +- docs/content.zh/docs/learn-flink/fault_tolerance.md | 2 +- docs/content.zh/docs/learn-flink/streaming_analytics.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/content.zh/docs/learn-flink/datastream_api.md b/docs/content.zh/docs/learn-flink/datastream_api.md index fdad73b9588fe..a032c96d65b7f 100644 --- a/docs/content.zh/docs/learn-flink/datastream_api.md +++ b/docs/content.zh/docs/learn-flink/datastream_api.md @@ -1,6 +1,6 @@ --- title: DataStream API 简介 -weight: 2 +weight: 3 type: docs --- + + + + + + + + + + + + + + + + + + + + + + + + From d480fec487ce2b3d86338b9a09e88bfcfccd49e2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Tue, 22 Mar 2022 14:48:49 +0100 Subject: [PATCH 078/258] [FLINK-26281][connectors/elasticsearch] Remove unused 'connection.max-retry-timeout' option This closes #19203. --- docs/content/docs/connectors/table/elasticsearch.md | 8 -------- .../table/ElasticsearchConnectorOptions.java | 6 ------ .../table/Elasticsearch6DynamicSinkFactory.java | 2 -- .../table/Elasticsearch7DynamicSinkFactory.java | 2 -- 4 files changed, 18 deletions(-) diff --git a/docs/content/docs/connectors/table/elasticsearch.md b/docs/content/docs/connectors/table/elasticsearch.md index a28d67f22739d..c205313331fc7 100644 --- a/docs/content/docs/connectors/table/elasticsearch.md +++ b/docs/content/docs/connectors/table/elasticsearch.md @@ -225,14 +225,6 @@ Connector Options Duration Delay between each backoff attempt. For CONSTANT backoff, this is simply the delay between each retry. For EXPONENTIAL backoff, this is the initial base delay. - -
connection.max-retry-timeout
- optional - no - (none) - Duration - Maximum timeout between retries. -
connection.path-prefix
optional diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java index 1b0fc5e4aec8b..38d65a7fba62b 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/ElasticsearchConnectorOptions.java @@ -133,12 +133,6 @@ public class ElasticsearchConnectorOptions { .noDefaultValue() .withDescription("Delay between each backoff attempt."); - public static final ConfigOption CONNECTION_MAX_RETRY_TIMEOUT_OPTION = - ConfigOptions.key("connection.max-retry-timeout") - .durationType() - .noDefaultValue() - .withDescription("Maximum timeout between retries."); - public static final ConfigOption CONNECTION_PATH_PREFIX = ConfigOptions.key("connection.path-prefix") .stringType() diff --git a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java index ff600a7a8624a..73f5cf889de43 100644 --- a/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch6/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch6DynamicSinkFactory.java @@ -47,7 +47,6 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.DOCUMENT_TYPE_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; @@ -75,7 +74,6 @@ public class Elasticsearch6DynamicSinkFactory implements DynamicTableSinkFactory BULK_FLUSH_BACKOFF_TYPE_OPTION, BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, BULK_FLUSH_BACKOFF_DELAY_OPTION, - CONNECTION_MAX_RETRY_TIMEOUT_OPTION, CONNECTION_PATH_PREFIX, FORMAT_OPTION, PASSWORD_OPTION, diff --git a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java index 50c37f0708bb5..70b0134d226d5 100644 --- a/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java +++ b/flink-connectors/flink-connector-elasticsearch7/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/Elasticsearch7DynamicSinkFactory.java @@ -47,7 +47,6 @@ import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_BACKOFF_TYPE_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_INTERVAL_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.BULK_FLUSH_MAX_ACTIONS_OPTION; -import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_MAX_RETRY_TIMEOUT_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.CONNECTION_PATH_PREFIX; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FAILURE_HANDLER_OPTION; import static org.apache.flink.streaming.connectors.elasticsearch.table.ElasticsearchConnectorOptions.FLUSH_ON_CHECKPOINT_OPTION; @@ -74,7 +73,6 @@ public class Elasticsearch7DynamicSinkFactory implements DynamicTableSinkFactory BULK_FLUSH_BACKOFF_TYPE_OPTION, BULK_FLUSH_BACKOFF_MAX_RETRIES_OPTION, BULK_FLUSH_BACKOFF_DELAY_OPTION, - CONNECTION_MAX_RETRY_TIMEOUT_OPTION, CONNECTION_PATH_PREFIX, FORMAT_OPTION, PASSWORD_OPTION, From 778a15c8e1cda342ea39ad373334bbe9ffca0a5e Mon Sep 17 00:00:00 2001 From: wangfeifan Date: Mon, 21 Mar 2022 17:42:54 +0800 Subject: [PATCH 079/258] [FLINK-26766][Runtime/StateBackends] Fix ChangelogStateHandleStreamImpl#getIntersection --- .../runtime/state/changelog/ChangelogStateHandleStreamImpl.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateHandleStreamImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateHandleStreamImpl.java index f9fd5042152c6..5a996b239bfe0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateHandleStreamImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateHandleStreamImpl.java @@ -100,7 +100,7 @@ public KeyGroupRange getKeyGroupRange() { @Nullable @Override public KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange) { - KeyGroupRange offsets = keyGroupRange.getIntersection(keyGroupRange); + KeyGroupRange offsets = this.keyGroupRange.getIntersection(keyGroupRange); if (offsets.getNumberOfKeyGroups() == 0) { return null; } From 1b04ba84222dd44cce9b3241e7a939275557478a Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Thu, 17 Mar 2022 11:20:16 +0100 Subject: [PATCH 080/258] [FLINK-26701][connectors] Revert FLINK-25927 apart from adding connector-base to flink-dist --- .../pom.xml | 34 ++--------- .../pom.xml | 34 ++--------- .../flink-connector-elasticsearch6/pom.xml | 27 -------- .../flink-connector-elasticsearch7/pom.xml | 27 -------- .../flink-connector-files/pom.xml | 6 -- flink-connectors/flink-connector-hive/pom.xml | 7 --- .../flink-connector-kafka/pom.xml | 27 -------- .../flink-connector-pulsar/pom.xml | 26 -------- .../pom.xml | 1 + .../pom.xml | 1 + .../flink-sql-connector-hbase-1.4/pom.xml | 4 -- .../flink-sql-connector-hbase-2.2/pom.xml | 4 -- .../flink-sql-connector-kafka/pom.xml | 1 + .../flink-sql-connector-kinesis/pom.xml | 1 + .../pom.xml | 8 --- .../pom.xml | 7 --- .../flink-streaming-kinesis-test/pom.xml | 7 --- flink-formats/flink-orc/pom.xml | 33 ---------- flink-formats/flink-parquet/pom.xml | 33 ---------- tools/ci/compile.sh | 4 -- tools/ci/shade.sh | 61 +------------------ 21 files changed, 17 insertions(+), 336 deletions(-) diff --git a/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml b/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml index 624b66e62838c..74639990bf8d1 100644 --- a/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml +++ b/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml @@ -46,6 +46,12 @@ under the License. provided + + org.apache.flink + flink-connector-base + ${project.version} + + org.apache.flink flink-connector-aws-base @@ -158,34 +164,6 @@ under the License. - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - true - - - org.apache.flink:flink-connector-base - org.apache.flink:flink-connector-aws-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.firehose.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml b/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml index 5620a1b38debc..7c864efd5e5ab 100644 --- a/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml +++ b/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml @@ -46,6 +46,12 @@ under the License. provided + + org.apache.flink + flink-connector-base + ${project.version} + + org.apache.flink flink-connector-aws-base @@ -139,34 +145,6 @@ under the License. - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - true - - - org.apache.flink:flink-connector-base - org.apache.flink:flink-connector-aws-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.kinesis.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 92bc8f125e824..6a5b5e8657871 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -199,33 +199,6 @@ under the License. 1 - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - org.apache.flink:flink-connector-elasticsearch-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.elasticsearch6.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 4f01962fad790..1c1a1951b3249 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -196,33 +196,6 @@ under the License. 1 - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - org.apache.flink:flink-connector-elasticsearch-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.elasticsearch7.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-files/pom.xml b/flink-connectors/flink-connector-files/pom.xml index e49cc84e2ecda..2a0ed9d4ffcd9 100644 --- a/flink-connectors/flink-connector-files/pom.xml +++ b/flink-connectors/flink-connector-files/pom.xml @@ -152,12 +152,6 @@ under the License. org.apache.flink:flink-connector-base - - - org.apache.flink.connector.base - org.apache.flink.connector.files.shaded.org.apache.flink.connector.base - - diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml index 94728ee9f344e..28db2b0cc31cc 100644 --- a/flink-connectors/flink-connector-hive/pom.xml +++ b/flink-connectors/flink-connector-hive/pom.xml @@ -549,13 +549,6 @@ under the License. test - - org.apache.flink - flink-connector-base - ${project.version} - test - - org.apache.flink diff --git a/flink-connectors/flink-connector-kafka/pom.xml b/flink-connectors/flink-connector-kafka/pom.xml index 96484a993d145..3cacec0b2ea27 100644 --- a/flink-connectors/flink-connector-kafka/pom.xml +++ b/flink-connectors/flink-connector-kafka/pom.xml @@ -288,33 +288,6 @@ under the License. -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} -XX:-UseGCOverheadLimit -Duser.country=US -Duser.language=en - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - - - true - - - org.apache.flink.connector.base - org.apache.flink.connector.kafka.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index 53e5ca400a0a0..655c57fe18a81 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -340,32 +340,6 @@ under the License. - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - - - - - org.apache.flink.connector.base - org.apache.flink.connector.pulsar.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml b/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml index 8bcdd1948852a..8b4a2df0d4db2 100644 --- a/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml +++ b/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml @@ -54,6 +54,7 @@ + org.apache.flink:flink-connector-base org.apache.flink:flink-connector-aws-base org.apache.flink:flink-connector-aws-kinesis-firehose software.amazon.awssdk:* diff --git a/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml b/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml index 65fc447aafbc0..0804a3a48f0be 100644 --- a/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml +++ b/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml @@ -55,6 +55,7 @@ + org.apache.flink:flink-connector-base org.apache.flink:flink-connector-aws-base org.apache.flink:flink-connector-aws-kinesis-streams software.amazon.awssdk:* diff --git a/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml b/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml index 7cf5b8dc882c6..78c1e5f32bbaa 100644 --- a/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml +++ b/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml @@ -113,10 +113,6 @@ under the License. - - org.apache.flink.connector.base - org.apache.flink.connector.sql.hbase14.shaded.org.apache.flink.connector.base - org.apache.zookeeper diff --git a/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml b/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml index 9cfae8bd10f6e..2b2873d4b0936 100644 --- a/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml +++ b/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml @@ -117,10 +117,6 @@ under the License. - - org.apache.flink.connector.base - org.apache.flink.connector.sql.hbase22.shaded.org.apache.flink.connector.base - org.apache.zookeeper diff --git a/flink-connectors/flink-sql-connector-kafka/pom.xml b/flink-connectors/flink-sql-connector-kafka/pom.xml index 256b9a087ffc1..ad63e6c1f6680 100644 --- a/flink-connectors/flink-sql-connector-kafka/pom.xml +++ b/flink-connectors/flink-sql-connector-kafka/pom.xml @@ -58,6 +58,7 @@ under the License. + org.apache.flink:flink-connector-base org.apache.flink:flink-connector-kafka org.apache.kafka:* diff --git a/flink-connectors/flink-sql-connector-kinesis/pom.xml b/flink-connectors/flink-sql-connector-kinesis/pom.xml index d75a85fe86b30..1eb1694577f3b 100644 --- a/flink-connectors/flink-sql-connector-kinesis/pom.xml +++ b/flink-connectors/flink-sql-connector-kinesis/pom.xml @@ -58,6 +58,7 @@ under the License. + org.apache.flink:flink-connector-base org.apache.flink:flink-connector-kinesis com.fasterxml.jackson.core:jackson-core com.fasterxml.jackson.core:jackson-databind diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml index f32d6f1e097e6..ed4a6c80c8f0a 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml @@ -51,14 +51,6 @@ test test-jar - - - org.apache.flink - flink-connector-aws-base - ${project.version} - test - - org.apache.flink flink-table-planner_${scala.binary.version} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml index d2aa1323fd142..423afd529b88f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml @@ -65,13 +65,6 @@ jackson-databind test - - - org.apache.flink - flink-connector-aws-base - ${project.version} - test - diff --git a/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml index c6cefd8f7ae05..fafa40790462e 100644 --- a/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml @@ -88,13 +88,6 @@ under the License. test-jar - - org.apache.flink - flink-connector-aws-base - ${project.version} - test - - org.junit.jupiter junit-jupiter diff --git a/flink-formats/flink-orc/pom.xml b/flink-formats/flink-orc/pom.xml index bf5a380a828b1..fad8c272dba3f 100644 --- a/flink-formats/flink-orc/pom.xml +++ b/flink-formats/flink-orc/pom.xml @@ -45,12 +45,6 @@ under the License. provided - - org.apache.flink - flink-connector-base - ${project.version} - - @@ -210,33 +204,6 @@ under the License. - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - - - - - org.apache.flink.connector.base - org.apache.flink.format.orc.shaded.org.apache.flink.connector.base - - - - - - diff --git a/flink-formats/flink-parquet/pom.xml b/flink-formats/flink-parquet/pom.xml index 8efa5a2c6824b..f5e340a9ac008 100644 --- a/flink-formats/flink-parquet/pom.xml +++ b/flink-formats/flink-parquet/pom.xml @@ -45,12 +45,6 @@ under the License. provided - - org.apache.flink - flink-connector-base - ${project.version} - - @@ -314,33 +308,6 @@ under the License. - - - org.apache.maven.plugins - maven-shade-plugin - - - shade-flink - package - - shade - - - - - org.apache.flink:flink-connector-base - - - - - org.apache.flink.connector.base - org.apache.flink.format.parquet.shaded.org.apache.flink.connector.base - - - - - - diff --git a/tools/ci/compile.sh b/tools/ci/compile.sh index c98fbae3f4af6..e6013f9650f3e 100755 --- a/tools/ci/compile.sh +++ b/tools/ci/compile.sh @@ -108,10 +108,6 @@ check_shaded_artifacts_connector_elasticsearch 6 EXIT_CODE=$(($EXIT_CODE+$?)) check_shaded_artifacts_connector_elasticsearch 7 EXIT_CODE=$(($EXIT_CODE+$?)) -check_one_per_package_file_connector_base -EXIT_CODE=$(($EXIT_CODE+$?)) -check_relocated_file_connector_base -EXIT_CODE=$(($EXIT_CODE+$?)) echo "============ Run license check ============" diff --git a/tools/ci/shade.sh b/tools/ci/shade.sh index 5931faffea719..922bba42f04f5 100755 --- a/tools/ci/shade.sh +++ b/tools/ci/shade.sh @@ -153,7 +153,7 @@ check_shaded_artifacts_connector_elasticsearch() { VARIANT=$1 find flink-connectors/flink-connector-elasticsearch${VARIANT}/target/flink-connector-elasticsearch${VARIANT}*.jar ! -name "*-tests.jar" -exec jar tf {} \; > allClasses - UNSHADED_CLASSES=`cat allClasses | grep -v -e '^META-INF' -e '^assets' -e "^org/apache/flink/connector/elasticsearch/" -e "^org/apache/flink/streaming/connectors/elasticsearch/" -e "^org/apache/flink/streaming/connectors/elasticsearch${VARIANT}/" -e "^org/apache/flink/connector/elasticsearch${VARIANT}/shaded/" -e "^org/apache/flink/table/descriptors/" -e "^org/elasticsearch/" | grep '\.class$'` + UNSHADED_CLASSES=`cat allClasses | grep -v -e '^META-INF' -e '^assets' -e "^org/apache/flink/connector/elasticsearch/" -e "^org/apache/flink/streaming/connectors/elasticsearch/" -e "^org/apache/flink/streaming/connectors/elasticsearch${VARIANT}/" -e "^org/apache/flink/table/descriptors/" -e "^org/elasticsearch/" | grep '\.class$'` if [ "$?" = "0" ]; then echo "==============================================================================" echo "Detected unshaded dependencies in flink-connector-elasticsearch${VARIANT}'s fat jar:" @@ -173,62 +173,3 @@ check_shaded_artifacts_connector_elasticsearch() { return 0 } - -check_one_per_package() { - read foo - if [ $foo -gt 1 ] - then - echo "ERROR - CHECK FAILED: $1 is shaded multiple times!" - exit 1 - else - echo "OK" - fi -} - -check_relocated() { - read foo - if [ $foo -ne 0 ] - then - echo "ERROR - CHECK FAILED: found $1 classes that where not relocated!" - exit 1 - else - echo "OK" - fi -} - -check_one_per_package_file_connector_base() { - echo "Checking that flink-connector-base is included only once:" - echo "__________________________________________________________________________" - - CONNECTOR_JARS=$(find flink-connectors -type f -name '*.jar' | grep -vE "original|connector-hive" | grep -v '\-test'); - EXIT_CODE=0 - - for i in $CONNECTOR_JARS; - do - echo -n "- $i: "; - jar tf $i | grep 'org/apache/flink/connector/base/source/reader/RecordEmitter' | wc -l | check_one_per_package "flink-connector-base"; - EXIT_CODE=$((EXIT_CODE+$?)) - done; - return $EXIT_CODE; -} - -check_relocated_file_connector_base() { - echo -e "\n\n" - echo "Checking that flink-connector-base is relocated:" - echo "__________________________________________________________________________" - - CONNECTOR_JARS=$(find flink-connectors -type f -name '*.jar' | \ - grep -v original | grep -v '\-test' | grep -v 'flink-connectors/flink-connector-base'); - - EXIT_CODE=0 - for i in $CONNECTOR_JARS; - do - echo -n "- $i: "; - jar tf $i | grep '^org/apache/flink/connector/base/source/reader/RecordEmitter' | wc -l | check_relocated "flink-connector-base"; - EXIT_CODE=$((EXIT_CODE+$?)) - done; - return $EXIT_CODE; -} - - - From d1445f115473efd389a2eadd8ec756cbcdfbde47 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Mon, 21 Mar 2022 10:47:57 +0100 Subject: [PATCH 081/258] [FLINK-26770][table-planner] Fix ArrayToArrayCastRule result data structure This closes #19188. --- .../casting/ArrayToArrayCastRule.java | 28 ++----------- .../functions/casting/CastRulesTest.java | 42 +++++++++++++++++-- 2 files changed, 42 insertions(+), 28 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java index b67b76a566311..f3478af123108 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/ArrayToArrayCastRule.java @@ -20,8 +20,8 @@ import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.planner.codegen.CodeGenUtils; import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.DistinctType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeRoot; @@ -109,30 +109,8 @@ protected String generateCodeBlockInternal( private static String arrayElementType(LogicalType t) { if (t.isNullable()) { - return "Object"; + return CodeGenUtils.boxedTypeTermForType(t); } - switch (t.getTypeRoot()) { - case BOOLEAN: - return "boolean"; - case TINYINT: - return "byte"; - case SMALLINT: - return "short"; - case INTEGER: - case DATE: - case TIME_WITHOUT_TIME_ZONE: - case INTERVAL_YEAR_MONTH: - return "int"; - case BIGINT: - case INTERVAL_DAY_TIME: - return "long"; - case FLOAT: - return "float"; - case DOUBLE: - return "double"; - case DISTINCT_TYPE: - return arrayElementType(((DistinctType) t).getSourceType()); - } - return "Object"; + return CodeGenUtils.primitiveTypeTermForType(t); } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java index 126127975ce94..6fdc98455818d 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java @@ -23,6 +23,7 @@ import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericMapData; import org.apache.flink.table.data.GenericRowData; @@ -1258,8 +1259,8 @@ Stream testCases() { CastTestSpecBuilder.testCastTo(ARRAY(BIGINT().nullable())) .fromCase( ARRAY(INT().nullable()), - new GenericArrayData(new Object[] {1, null, 2}), - new GenericArrayData(new Object[] {1L, null, 2L})), + new GenericArrayData(new Integer[] {1, null, 2}), + new GenericArrayData(new Long[] {1L, null, 2L})), CastTestSpecBuilder.testCastTo(ARRAY(BIGINT().notNull())) .fromCase( ARRAY(INT().notNull()), @@ -1273,7 +1274,42 @@ Stream testCases() { new GenericArrayData(new Integer[] {1, 2, null}), new GenericArrayData(new Integer[] {3}) }), - NullPointerException.class), + NullPointerException.class) + .fromCase( + ARRAY(ARRAY(INT().nullable())), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData(new Integer[] {1, 2}), + new GenericArrayData(new Integer[] {3}) + }), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData(new Long[] {1L, 2L}), + new GenericArrayData(new Long[] {3L}) + })), + CastTestSpecBuilder.testCastTo(ARRAY(ARRAY(DECIMAL(10, 2).notNull()))) + .fromCase( + ARRAY(ARRAY(INT().notNull())), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData(new Integer[] {1, 2}), + new GenericArrayData(new Integer[] {3}) + }), + new GenericArrayData( + new GenericArrayData[] { + new GenericArrayData( + new Object[] { + DecimalData.fromBigDecimal( + BigDecimal.ONE, 10, 2), + DecimalData.fromBigDecimal( + new BigDecimal(2), 10, 2) + }), + new GenericArrayData( + new Object[] { + DecimalData.fromBigDecimal( + new BigDecimal(3), 10, 2) + }) + })), CastTestSpecBuilder.testCastTo(MAP(DOUBLE().notNull(), DOUBLE().notNull())) .fromCase( MAP(INT().nullable(), INT().nullable()), From ecd3bd748c75014a2bd7b7e338af09defd9e0aab Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 23 Mar 2022 13:42:45 +0100 Subject: [PATCH 082/258] [hotfix][table-common] Improve JavaDocs of GenericArrayData --- .../flink/table/data/GenericArrayData.java | 21 ++++++++++++++++--- 1 file changed, 18 insertions(+), 3 deletions(-) diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java index 841e0f7fb42f8..6c694799f38fb 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/data/GenericArrayData.java @@ -29,13 +29,28 @@ /** * An internal data structure representing data of {@link ArrayType}. * + *

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

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

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

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

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

{@code
+ * // ARRAY < ARRAY < INT NOT NULL > >
+ * new GenericArrayData(
+ *   new GenericArrayData[]{
+ *     new GenericArrayData(new int[3]),
+ *     new GenericArrayData(new int[5])
+ *   }
+ * )
+ * }
*/ @PublicEvolving public final class GenericArrayData implements ArrayData { From 1e1b182e702ec25a38c3c02d585734415eae110a Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 9 Mar 2022 12:14:46 +0100 Subject: [PATCH 083/258] [FLINK-26518][table-planner] Port FlinkRelBuilder to Java --- .../planner/calcite/FlinkRelBuilder.java | 235 ++++++++++++++++++ .../QueryOperationCatalogViewTable.java | 8 +- .../planner/delegation/PlannerContext.java | 2 +- .../rules/logical/SubQueryDecorrelator.java | 3 +- ...atchPhysicalPythonWindowAggregateRule.java | 3 +- ...hysicalPythonGroupWindowAggregateRule.java | 2 +- .../planner/calcite/FlinkRelBuilder.scala | 233 ----------------- .../planner/expressions/fieldExpression.scala | 7 - .../expressions/windowProperties.scala | 5 - .../calcite/LogicalWindowAggregate.scala | 6 +- .../calcite/LogicalWindowTableAggregate.scala | 4 +- .../plan/nodes/calcite/WindowAggregate.scala | 11 +- .../nodes/calcite/WindowTableAggregate.scala | 10 +- .../logical/FlinkLogicalWindowAggregate.scala | 2 +- .../FlinkLogicalWindowTableAggregate.scala | 2 +- .../logical/CorrelateSortToRankRule.scala | 2 +- 16 files changed, 267 insertions(+), 268 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java delete mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java new file mode 100644 index 0000000000000..35ab473426e65 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java @@ -0,0 +1,235 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.calcite; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.operations.QueryOperation; +import org.apache.flink.table.planner.calcite.FlinkRelFactories.ExpandFactory; +import org.apache.flink.table.planner.calcite.FlinkRelFactories.RankFactory; +import org.apache.flink.table.planner.hint.FlinkHints; +import org.apache.flink.table.planner.plan.QueryOperationConverter; +import org.apache.flink.table.planner.plan.logical.LogicalWindow; +import org.apache.flink.table.planner.plan.nodes.calcite.LogicalTableAggregate; +import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWatermarkAssigner; +import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWindowAggregate; +import org.apache.flink.table.planner.plan.nodes.calcite.LogicalWindowTableAggregate; +import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty; +import org.apache.flink.table.runtime.operators.rank.RankRange; +import org.apache.flink.table.runtime.operators.rank.RankType; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; + +import org.apache.calcite.plan.Context; +import org.apache.calcite.plan.Contexts; +import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptSchema; +import org.apache.calcite.plan.RelOptTable.ToRelContext; +import org.apache.calcite.plan.ViewExpanders; +import org.apache.calcite.rel.RelCollation; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.hint.RelHint; +import org.apache.calcite.rel.logical.LogicalAggregate; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.tools.RelBuilderFactory; +import org.apache.calcite.util.ImmutableBitSet; +import org.apache.calcite.util.Util; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.function.UnaryOperator; + +import static org.apache.flink.table.planner.plan.utils.AggregateUtil.isTableAggregate; + +/** Flink-specific {@link RelBuilder}. */ +@Internal +public final class FlinkRelBuilder extends RelBuilder { + + private final QueryOperationConverter toRelNodeConverter; + + private final ExpandFactory expandFactory; + + private final RankFactory rankFactory; + + private FlinkRelBuilder(Context context, RelOptCluster cluster, RelOptSchema relOptSchema) { + super(context, cluster, relOptSchema); + + this.toRelNodeConverter = + new QueryOperationConverter(this, context.unwrap(FlinkContext.class).isBatchMode()); + this.expandFactory = + Util.first( + context.unwrap(ExpandFactory.class), + FlinkRelFactories.DEFAULT_EXPAND_FACTORY()); + this.rankFactory = + Util.first( + context.unwrap(RankFactory.class), + FlinkRelFactories.DEFAULT_RANK_FACTORY()); + } + + public static FlinkRelBuilder of( + Context context, RelOptCluster cluster, RelOptSchema relOptSchema) { + return new FlinkRelBuilder(Preconditions.checkNotNull(context), cluster, relOptSchema); + } + + public static FlinkRelBuilder of(RelOptCluster cluster, RelOptSchema relOptSchema) { + return FlinkRelBuilder.of(cluster.getPlanner().getContext(), cluster, relOptSchema); + } + + public static RelBuilderFactory proto(Context context) { + return (cluster, schema) -> { + final Context clusterContext = cluster.getPlanner().getContext(); + final Context chain = Contexts.chain(context, clusterContext); + return FlinkRelBuilder.of(chain, cluster, schema); + }; + } + + public RelBuilder expand(List> projects, int expandIdIndex) { + final RelNode input = build(); + final RelNode expand = expandFactory.createExpand(input, projects, expandIdIndex); + return push(expand); + } + + public RelBuilder rank( + ImmutableBitSet partitionKey, + RelCollation orderKey, + RankType rankType, + RankRange rankRange, + RelDataTypeField rankNumberType, + boolean outputRankNumber) { + final RelNode input = build(); + final RelNode rank = + rankFactory.createRank( + input, + partitionKey, + orderKey, + rankType, + rankRange, + rankNumberType, + outputRankNumber); + return push(rank); + } + + /** Build non-window aggregate for either aggregate or table aggregate. */ + @Override + public RelBuilder aggregate( + RelBuilder.GroupKey groupKey, Iterable aggCalls) { + // build a relNode, the build() may also return a project + RelNode relNode = super.aggregate(groupKey, aggCalls).build(); + + if (relNode instanceof LogicalAggregate) { + final LogicalAggregate logicalAggregate = (LogicalAggregate) relNode; + if (isTableAggregate(logicalAggregate.getAggCallList())) { + relNode = LogicalTableAggregate.create(logicalAggregate); + } else if (isCountStarAgg(logicalAggregate)) { + final RelNode newAggInput = + push(logicalAggregate.getInput(0)).project(literal(0)).build(); + relNode = + logicalAggregate.copy( + logicalAggregate.getTraitSet(), ImmutableList.of(newAggInput)); + } + } + + return push(relNode); + } + + /** Build window aggregate for either aggregate or table aggregate. */ + public RelBuilder windowAggregate( + LogicalWindow window, + GroupKey groupKey, + List namedProperties, + Iterable aggCalls) { + // build logical aggregate + + // Because of: + // [CALCITE-3763] RelBuilder.aggregate should prune unused fields from the input, + // if the input is a Project. + // + // the field can not be pruned if it is referenced by other expressions + // of the window aggregation(i.e. the TUMBLE_START/END). + // To solve this, we config the RelBuilder to forbidden this feature. + final LogicalAggregate aggregate = + (LogicalAggregate) + super.transform(t -> t.withPruneInputOfAggregate(false)) + .push(build()) + .aggregate(groupKey, aggCalls) + .build(); + + // build logical window aggregate from it + final RelNode windowAggregate; + if (isTableAggregate(aggregate.getAggCallList())) { + windowAggregate = + LogicalWindowTableAggregate.create(window, namedProperties, aggregate); + } else { + windowAggregate = LogicalWindowAggregate.create(window, namedProperties, aggregate); + } + return push(windowAggregate); + } + + /** Build watermark assigner relational node. */ + public RelBuilder watermark(int rowtimeFieldIndex, RexNode watermarkExpr) { + final RelNode input = build(); + final RelNode relNode = + LogicalWatermarkAssigner.create(cluster, input, rowtimeFieldIndex, watermarkExpr); + return push(relNode); + } + + public RelBuilder queryOperation(QueryOperation queryOperation) { + final RelNode relNode = queryOperation.accept(toRelNodeConverter); + return push(relNode); + } + + public RelBuilder scan(ObjectIdentifier identifier, Map dynamicOptions) { + final List hints = new ArrayList<>(); + hints.add( + RelHint.builder(FlinkHints.HINT_NAME_OPTIONS).hintOptions(dynamicOptions).build()); + final ToRelContext toRelContext = ViewExpanders.simpleContext(cluster, hints); + final RelNode relNode = + relOptSchema.getTableForMember(identifier.toList()).toRel(toRelContext); + return push(relNode); + } + + @Override + public FlinkTypeFactory getTypeFactory() { + return (FlinkTypeFactory) super.getTypeFactory(); + } + + @Override + public RelBuilder transform(UnaryOperator transform) { + // Override in order to return a FlinkRelBuilder. + final Context mergedContext = + Contexts.of(transform.apply(Config.DEFAULT), cluster.getPlanner().getContext()); + return FlinkRelBuilder.of(mergedContext, cluster, relOptSchema); + } + + private static boolean isCountStarAgg(LogicalAggregate agg) { + if (agg.getGroupCount() != 0 || agg.getAggCallList().size() != 1) { + return false; + } + final AggregateCall call = agg.getAggCallList().get(0); + return call.getAggregation().getKind() == SqlKind.COUNT + && call.filterArg == -1 + && call.getArgList().isEmpty(); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java index 9ba8e6e7e2e3f..32353c182de87 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/catalog/QueryOperationCatalogViewTable.java @@ -25,6 +25,9 @@ import org.apache.flink.table.planner.plan.schema.ExpandingPreparingTable; import org.apache.flink.table.planner.plan.stats.FlinkStatistic; +import org.apache.calcite.plan.Context; +import org.apache.calcite.plan.Contexts; +import org.apache.calcite.plan.RelOptCluster; import org.apache.calcite.plan.RelOptSchema; import org.apache.calcite.plan.RelOptTable; import org.apache.calcite.rel.RelNode; @@ -70,8 +73,9 @@ public List getQualifiedName() { @Override public RelNode convertToRel(RelOptTable.ToRelContext context) { - FlinkRelBuilder relBuilder = - FlinkRelBuilder.of(context, context.getCluster(), this.getRelOptSchema()); + final RelOptCluster cluster = context.getCluster(); + final Context chain = Contexts.of(context, cluster.getPlanner().getContext()); + final FlinkRelBuilder relBuilder = FlinkRelBuilder.of(chain, cluster, getRelOptSchema()); return relBuilder.queryOperation(catalogView.getQueryOperation()).build(); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java index 19a4b6ec34f40..e98e582baa171 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerContext.java @@ -184,7 +184,7 @@ public FlinkRelBuilder createRelBuilder(String currentCatalog, String currentDat context, // Sets up the ViewExpander explicitly for FlinkRelBuilder. createFlinkPlanner(currentCatalog, currentDatabase).createToRelContext()); - return new FlinkRelBuilder(chain, cluster, relOptSchema); + return FlinkRelBuilder.of(chain, cluster, relOptSchema); } /** diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java index b80b884da8b62..828286b5fc665 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/SubQueryDecorrelator.java @@ -134,8 +134,7 @@ public static Result decorrelateQuery(RelNode rootRel) { } RelOptCluster cluster = rootRel.getCluster(); - RelBuilder relBuilder = - new FlinkRelBuilder(cluster.getPlanner().getContext(), cluster, null); + RelBuilder relBuilder = FlinkRelBuilder.of(cluster, null); RexBuilder rexBuilder = cluster.getRexBuilder(); final SubQueryDecorrelator decorrelator = diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalPythonWindowAggregateRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalPythonWindowAggregateRule.java index 9eadb76c50ce6..847b803c1b9d8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalPythonWindowAggregateRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalPythonWindowAggregateRule.java @@ -34,6 +34,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateUtil; import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil; import org.apache.flink.table.planner.plan.utils.PythonUtil; +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; import org.apache.flink.table.types.DataType; import org.apache.calcite.plan.RelOptRule; @@ -160,7 +161,7 @@ public void onMatch(RelOptRuleCall call) { window, inputTimeFieldIndex, inputTimeIsDate, - agg.getNamedProperties()); + JavaScalaConversionUtil.toScala(agg.getNamedProperties())); call.transformTo(windowAgg); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalPythonGroupWindowAggregateRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalPythonGroupWindowAggregateRule.java index 002f1ae59481d..5f080d76da7aa 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalPythonGroupWindowAggregateRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/stream/StreamPhysicalPythonGroupWindowAggregateRule.java @@ -143,7 +143,7 @@ public RelNode convert(RelNode rel) { agg.getGroupSet().toArray(), JavaScalaConversionUtil.toScala(aggCalls), agg.getWindow(), - agg.getNamedProperties(), + JavaScalaConversionUtil.toScala(agg.getNamedProperties()), emitStrategy); } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala deleted file mode 100644 index b96e51023bc12..0000000000000 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/FlinkRelBuilder.scala +++ /dev/null @@ -1,233 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.table.planner.calcite - -import org.apache.flink.table.operations.QueryOperation -import org.apache.flink.table.planner.calcite.FlinkRelFactories.{ExpandFactory, RankFactory} -import org.apache.flink.table.planner.expressions.WindowProperty -import org.apache.flink.table.planner.plan.QueryOperationConverter -import org.apache.flink.table.planner.plan.logical.LogicalWindow -import org.apache.flink.table.planner.plan.nodes.calcite.{LogicalTableAggregate, LogicalWatermarkAssigner, LogicalWindowAggregate, LogicalWindowTableAggregate} -import org.apache.flink.table.planner.plan.utils.AggregateUtil -import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty -import org.apache.flink.table.runtime.operators.rank.{RankRange, RankType} - -import com.google.common.collect.ImmutableList -import org.apache.calcite.plan._ -import org.apache.calcite.rel.RelCollation -import org.apache.calcite.rel.`type`.RelDataTypeField -import org.apache.calcite.rel.hint.RelHint -import org.apache.calcite.rel.logical.LogicalAggregate -import org.apache.calcite.rex.RexNode -import org.apache.calcite.sql.SqlKind -import org.apache.calcite.tools.RelBuilder.{AggCall, Config, GroupKey} -import org.apache.calcite.tools.{RelBuilder, RelBuilderFactory} -import org.apache.calcite.util.{ImmutableBitSet, Util} -import org.apache.flink.table.catalog.ObjectIdentifier -import org.apache.flink.table.planner.hint.FlinkHints - -import java.lang.Iterable -import java.util -import java.util.List -import java.util.function.UnaryOperator - -import scala.collection.JavaConversions._ - -/** - * Flink specific [[RelBuilder]] that changes the default type factory to a [[FlinkTypeFactory]]. - */ -class FlinkRelBuilder( - context: Context, - relOptCluster: RelOptCluster, - relOptSchema: RelOptSchema) - extends RelBuilder( - context, - relOptCluster, - relOptSchema) { - - require(context != null) - - private val toRelNodeConverter = { - new QueryOperationConverter(this, context.unwrap(classOf[FlinkContext]).isBatchMode) - } - - private val expandFactory: ExpandFactory = { - Util.first(context.unwrap(classOf[ExpandFactory]), FlinkRelFactories.DEFAULT_EXPAND_FACTORY) - } - - private val rankFactory: RankFactory = { - Util.first(context.unwrap(classOf[RankFactory]), FlinkRelFactories.DEFAULT_RANK_FACTORY) - } - - override def getRelOptSchema: RelOptSchema = relOptSchema - - override def getCluster: RelOptCluster = relOptCluster - - override def getTypeFactory: FlinkTypeFactory = - super.getTypeFactory.asInstanceOf[FlinkTypeFactory] - - override def transform(transform: UnaryOperator[RelBuilder.Config]): FlinkRelBuilder = { - // Override in order to return a FlinkRelBuilder. - FlinkRelBuilder.of(transform.apply(Config.DEFAULT), cluster, relOptSchema) - } - - def expand( - projects: util.List[util.List[RexNode]], - expandIdIndex: Int): RelBuilder = { - val input = build() - val expand = expandFactory.createExpand(input, projects, expandIdIndex) - push(expand) - } - - def rank( - partitionKey: ImmutableBitSet, - orderKey: RelCollation, - rankType: RankType, - rankRange: RankRange, - rankNumberType: RelDataTypeField, - outputRankNumber: Boolean): RelBuilder = { - val input = build() - val rank = rankFactory.createRank(input, partitionKey, orderKey, rankType, rankRange, - rankNumberType, outputRankNumber) - push(rank) - } - - /** - * Build non-window aggregate for either aggregate or table aggregate. - */ - override def aggregate(groupKey: GroupKey, aggCalls: Iterable[AggCall]): RelBuilder = { - // build a relNode, the build() may also return a project - val relNode = super.aggregate(groupKey, aggCalls).build() - - def isCountStartAgg(agg: LogicalAggregate): Boolean = { - if (agg.getGroupCount != 0 || agg.getAggCallList.size() != 1) { - return false - } - val call = agg.getAggCallList.head - call.getAggregation.getKind == SqlKind.COUNT && - call.filterArg == -1 && call.getArgList.isEmpty - } - - relNode match { - case logicalAggregate: LogicalAggregate - if AggregateUtil.isTableAggregate(logicalAggregate.getAggCallList) => - push(LogicalTableAggregate.create(logicalAggregate)) - case logicalAggregate2: LogicalAggregate - if isCountStartAgg(logicalAggregate2) => - val newAggInput = push(logicalAggregate2.getInput(0)) - .project(literal(0)).build() - push(logicalAggregate2.copy(logicalAggregate2.getTraitSet, ImmutableList.of(newAggInput))) - case _ => push(relNode) - } - } - - /** - * Build window aggregate for either aggregate or table aggregate. - */ - def windowAggregate( - window: LogicalWindow, - groupKey: GroupKey, - namedProperties: List[NamedWindowProperty], - aggCalls: Iterable[AggCall]): RelBuilder = { - // build logical aggregate - - // Because of: - // [CALCITE-3763] RelBuilder.aggregate should prune unused fields from the input, - // if the input is a Project. - // - // the field can not be pruned if it is referenced by other expressions - // of the window aggregation(i.e. the TUMBLE_START/END). - // To solve this, we config the RelBuilder to forbidden this feature. - val aggregate = super.transform( - new UnaryOperator[RelBuilder.Config] { - override def apply(t: RelBuilder.Config) - : RelBuilder.Config = t.withPruneInputOfAggregate(false) - }) - .push(build()) - .aggregate(groupKey, aggCalls) - .build() - .asInstanceOf[LogicalAggregate] - - // build logical window aggregate from it - aggregate match { - case logicalAggregate: LogicalAggregate - if AggregateUtil.isTableAggregate(logicalAggregate.getAggCallList) => - push(LogicalWindowTableAggregate.create(window, namedProperties, aggregate)) - case _ => push(LogicalWindowAggregate.create(window, namedProperties, aggregate)) - } - } - - /** - * Build watermark assigner relation node. - */ - def watermark(rowtimeFieldIndex: Int, watermarkExpr: RexNode): RelBuilder = { - val input = build() - val watermarkAssigner = LogicalWatermarkAssigner - .create(cluster, input, rowtimeFieldIndex, watermarkExpr) - push(watermarkAssigner) - this - } - - def queryOperation(queryOperation: QueryOperation): RelBuilder = { - val relNode = queryOperation.accept(toRelNodeConverter) - push(relNode) - this - } - - def scan( - identifier: ObjectIdentifier, - dynamicOptions: util.Map[String, String]): RelBuilder = { - val hints = new util.ArrayList[RelHint] - hints.add(RelHint.builder(FlinkHints.HINT_NAME_OPTIONS).hintOptions(dynamicOptions).build) - val toRelContext = ViewExpanders.simpleContext(cluster, hints) - push(relOptSchema.getTableForMember(identifier.toList).toRel(toRelContext)) - this - } -} - -object FlinkRelBuilder { - - case class NamedWindowProperty(name: String, property: WindowProperty) - - def proto(context: Context): RelBuilderFactory = new RelBuilderFactory() { - def create(cluster: RelOptCluster, schema: RelOptSchema): RelBuilder = { - val clusterContext = cluster.getPlanner.getContext.unwrap(classOf[FlinkContext]) - val mergedContext = Contexts.chain(context, clusterContext) - - new FlinkRelBuilder(mergedContext, cluster, schema) - } - } - - def of(cluster: RelOptCluster, relOptSchema: RelOptSchema): FlinkRelBuilder = { - val clusterContext = cluster.getPlanner.getContext - new FlinkRelBuilder( - clusterContext, - cluster, - relOptSchema) - } - - def of(contextVar: Object, cluster: RelOptCluster, relOptSchema: RelOptSchema) - : FlinkRelBuilder = { - val mergedContext = Contexts.of(contextVar, cluster.getPlanner.getContext) - new FlinkRelBuilder( - mergedContext, - cluster, - relOptSchema) - } -} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/fieldExpression.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/fieldExpression.scala index fd86f6749124f..013e687734ce8 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/fieldExpression.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/fieldExpression.scala @@ -20,7 +20,6 @@ package org.apache.flink.table.planner.expressions import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.table.api._ import org.apache.flink.table.operations.QueryOperation -import org.apache.flink.table.planner.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.calcite.FlinkTypeFactory._ import org.apache.flink.table.planner.validate.{ValidationFailure, ValidationResult, ValidationSuccess} @@ -150,9 +149,6 @@ case class RowtimeAttribute(expr: PlannerExpression) extends TimeAttribute(expr) } } - override def toNamedWindowProperty(name: String): NamedWindowProperty = - NamedWindowProperty(name, this) - override def toString: String = s"rowtime($child)" } @@ -174,9 +170,6 @@ case class ProctimeAttribute(expr: PlannerExpression) extends TimeAttribute(expr override def resultType: TypeInformation[_] = TimeIndicatorTypeInfo.PROCTIME_INDICATOR - override def toNamedWindowProperty(name: String): NamedWindowProperty = - NamedWindowProperty(name, this) - override def toString: String = s"proctime($child)" } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/windowProperties.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/windowProperties.scala index 0e68163edd3d5..ce6940a58171f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/windowProperties.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/expressions/windowProperties.scala @@ -19,13 +19,10 @@ package org.apache.flink.table.planner.expressions import org.apache.flink.api.common.typeinfo.{SqlTimeTypeInfo, TypeInformation} -import org.apache.flink.table.planner.calcite.FlinkRelBuilder.NamedWindowProperty import org.apache.flink.table.planner.validate.{ValidationFailure, ValidationSuccess} trait WindowProperty { - def toNamedWindowProperty(name: String): NamedWindowProperty - def resultType: TypeInformation[_] } @@ -42,8 +39,6 @@ abstract class AbstractWindowProperty(child: PlannerExpression) } else { ValidationFailure("Child must be a window reference.") } - - def toNamedWindowProperty(name: String): NamedWindowProperty = NamedWindowProperty(name, this) } case class WindowStart(child: PlannerExpression) extends AbstractWindowProperty(child) { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowAggregate.scala index 0d70fd7403b35..f3e35e14a4e10 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowAggregate.scala @@ -36,7 +36,7 @@ final class LogicalWindowAggregate( groupSet: ImmutableBitSet, aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends WindowAggregate(cluster, traitSet, child, groupSet, aggCalls, window, namedProperties) { override def copy( @@ -55,7 +55,7 @@ final class LogicalWindowAggregate( namedProperties) } - def copy(namedProperties: Seq[NamedWindowProperty]): LogicalWindowAggregate = { + def copy(namedProperties: util.List[NamedWindowProperty]): LogicalWindowAggregate = { new LogicalWindowAggregate( cluster, traitSet, @@ -71,7 +71,7 @@ object LogicalWindowAggregate { def create( window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty], + namedProperties: util.List[NamedWindowProperty], agg: Aggregate): LogicalWindowAggregate = { require(agg.getGroupType == Group.SIMPLE) val cluster: RelOptCluster = agg.getCluster diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowTableAggregate.scala index 6ae042f6185f8..f9a2ed286f406 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/LogicalWindowTableAggregate.scala @@ -41,7 +41,7 @@ class LogicalWindowTableAggregate( groupSets: util.List[ImmutableBitSet], aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends WindowTableAggregate( cluster, traitSet, @@ -69,7 +69,7 @@ object LogicalWindowTableAggregate { def create( window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty], + namedProperties: util.List[NamedWindowProperty], aggregate: Aggregate): LogicalWindowTableAggregate = { val cluster: RelOptCluster = aggregate.getCluster diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowAggregate.scala index 884be0ab90c5c..c28dd1a732305 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowAggregate.scala @@ -31,6 +31,9 @@ import org.apache.calcite.util.ImmutableBitSet import java.util +import scala.collection.JavaConverters._ + + /** * Relational operator that eliminates duplicates and computes totals with time window group. * @@ -43,7 +46,7 @@ abstract class WindowAggregate( groupSet: ImmutableBitSet, aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends Aggregate( cluster, traitSet, @@ -54,7 +57,7 @@ abstract class WindowAggregate( def getWindow: LogicalWindow = window - def getNamedProperties: Seq[NamedWindowProperty] = namedProperties + def getNamedProperties: util.List[NamedWindowProperty] = namedProperties override def accept(shuttle: RelShuttle): RelNode = shuttle.visit(this) @@ -63,7 +66,7 @@ abstract class WindowAggregate( val typeFactory = getCluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] val builder = typeFactory.builder builder.addAll(aggregateRowType.getFieldList) - namedProperties.foreach { namedProp => + namedProperties.asScala.foreach { namedProp => builder.add( namedProp.getName, typeFactory.createFieldTypeFromLogicalType(namedProp.getProperty.getResultType) @@ -82,6 +85,6 @@ abstract class WindowAggregate( override def explainTerms(pw: RelWriter): RelWriter = { super.explainTerms(pw) .item("window", window) - .item("properties", namedProperties.map(_.getName).mkString(", ")) + .item("properties", namedProperties.asScala.map(_.getName).mkString(", ")) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowTableAggregate.scala index 98bcaea381f04..d388b5b313d3e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/calcite/WindowTableAggregate.scala @@ -30,6 +30,8 @@ import org.apache.calcite.util.ImmutableBitSet import java.util +import scala.collection.JavaConverters._ + /** * Relational operator that represents a window table aggregate. A TableAggregate is similar to the * [[org.apache.calcite.rel.core.Aggregate]] but may output 0 or more records for a group. @@ -42,19 +44,19 @@ abstract class WindowTableAggregate( groupSets: util.List[ImmutableBitSet], aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends TableAggregate(cluster, traitSet, input, groupSet, groupSets, aggCalls) { def getWindow: LogicalWindow = window - def getNamedProperties: Seq[NamedWindowProperty] = namedProperties + def getNamedProperties: util.List[NamedWindowProperty] = namedProperties override def deriveRowType(): RelDataType = { val aggregateRowType = super.deriveRowType() val typeFactory = getCluster.getTypeFactory.asInstanceOf[FlinkTypeFactory] val builder = typeFactory.builder builder.addAll(aggregateRowType.getFieldList) - namedProperties.foreach { namedProp => + namedProperties.asScala.foreach { namedProp => builder.add( namedProp.getName, typeFactory.createFieldTypeFromLogicalType(namedProp.getProperty.getResultType) @@ -66,6 +68,6 @@ abstract class WindowTableAggregate( override def explainTerms(pw: RelWriter): RelWriter = { super.explainTerms(pw) .item("window", window) - .item("properties", namedProperties.map(_.getName).mkString(", ")) + .item("properties", namedProperties.asScala.map(_.getName).mkString(", ")) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowAggregate.scala index 8cef1170aee3c..7884f59d7fd51 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowAggregate.scala @@ -43,7 +43,7 @@ class FlinkLogicalWindowAggregate( groupSet: ImmutableBitSet, aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends WindowAggregate(cluster, traitSet, child, groupSet, aggCalls, window, namedProperties) with FlinkLogicalRel { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowTableAggregate.scala index add5d4149e04d..7dcc81bc12a9e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalWindowTableAggregate.scala @@ -44,7 +44,7 @@ class FlinkLogicalWindowTableAggregate( groupSets: util.List[ImmutableBitSet], aggCalls: util.List[AggregateCall], window: LogicalWindow, - namedProperties: Seq[NamedWindowProperty]) + namedProperties: util.List[NamedWindowProperty]) extends WindowTableAggregate( cluster, traitSet, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRule.scala index b2d0797c7af27..e8013a8495965 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRule.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/CorrelateSortToRankRule.scala @@ -175,7 +175,7 @@ class CorrelateSortToRankRule extends RelOptRule( 1, sort.fetch.asInstanceOf[RexLiteral].getValueAs(classOf[java.lang.Long])), null, - outputRankNumber = false) + false) .project(projects) .build() From aea176e1fc15a9cf0388e8b32ce3bb3688e876bf Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Wed, 9 Mar 2022 15:38:52 +0100 Subject: [PATCH 084/258] [FLINK-26518][table] Support BridgingSqlFunction with SqlTableFunction for Scala implicits This closes #19137. --- .../utils/CalculatedTableFactory.java | 6 +- .../api/ImplicitExpressionConversions.scala | 8 +-- .../sql/validate/ProcedureNamespace.java | 35 +++++------ .../planner/calcite/FlinkRelBuilder.java | 59 +++++++++++++++++++ .../bridging/BridgingSqlFunction.java | 39 +++++++++++- .../planner/plan/QueryOperationConverter.java | 12 ++-- .../planner/plan/utils/SetOpRewriteUtil.scala | 12 ++-- .../runtime/stream/sql/FunctionITCase.java | 33 ++++------- .../runtime/stream/table/FunctionITCase.java | 39 ++++++------ .../utils/JavaUserDefinedTableFunctions.java | 6 +- .../plan/batch/sql/SetOperatorsTest.xml | 8 +-- .../plan/batch/table/CorrelateTest.xml | 28 ++++----- .../plan/batch/table/SetOperatorsTest.xml | 8 +-- .../CorrelateStringExpressionTest.xml | 34 +++++------ .../planner/plan/common/PartialInsertTest.xml | 16 ++--- .../logical/RewriteIntersectAllRuleTest.xml | 12 ++-- .../rules/logical/RewriteMinusAllRuleTest.xml | 12 ++-- .../plan/stream/sql/SetOperatorsTest.xml | 8 +-- .../plan/stream/table/ColumnFunctionsTest.xml | 4 +- .../plan/stream/table/CorrelateTest.xml | 51 ++++++++-------- .../table/TemporalTableFunctionJoinTest.xml | 10 ++-- .../plan/batch/sql/TableSourceTest.scala | 1 - .../plan/batch/table/CorrelateTest.scala | 12 ---- .../CorrelateStringExpressionTest.scala | 4 -- .../validation/CorrelateValidationTest.scala | 1 - .../plan/stream/table/CorrelateTest.scala | 17 ------ .../validation/CorrelateValidationTest.scala | 3 +- .../runtime/batch/table/CorrelateITCase.scala | 3 +- .../runtime/utils/StreamingTestBase.scala | 3 + .../utils/UserDefinedTableFunctions.scala | 24 +++----- 30 files changed, 274 insertions(+), 234 deletions(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/CalculatedTableFactory.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/CalculatedTableFactory.java index 09b9eca32aa44..6d20abddefe3a 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/CalculatedTableFactory.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/operations/utils/CalculatedTableFactory.java @@ -28,6 +28,7 @@ import org.apache.flink.table.expressions.ResolvedExpression; import org.apache.flink.table.expressions.utils.ResolvedExpressionDefaultVisitor; import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionKind; import org.apache.flink.table.operations.CalculatedQueryOperation; import org.apache.flink.table.operations.QueryOperation; import org.apache.flink.table.types.DataType; @@ -38,6 +39,7 @@ import java.util.List; import static java.util.stream.Collectors.toList; +import static org.apache.flink.table.expressions.ApiExpressionUtils.isFunctionOfKind; import static org.apache.flink.table.functions.BuiltInFunctionDefinitions.AS; /** Utility class for creating a valid {@link CalculatedQueryOperation} operation. */ @@ -89,7 +91,7 @@ private CalculatedQueryOperation unwrapFromAlias(CallExpression call) { + alias))) .collect(toList()); - if (!(children.get(0) instanceof CallExpression)) { + if (!isFunctionOfKind(children.get(0), FunctionKind.TABLE)) { throw fail(); } @@ -156,7 +158,7 @@ protected CalculatedQueryOperation defaultMethod(ResolvedExpression expression) private ValidationException fail() { return new ValidationException( - "A lateral join only accepts a string expression which defines a table function " + "A lateral join only accepts an expression which defines a table function " + "call that might be followed by some alias."); } } diff --git a/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/ImplicitExpressionConversions.scala b/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/ImplicitExpressionConversions.scala index b2c0ca2bfd29c..c36c3d10deeaa 100644 --- a/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/ImplicitExpressionConversions.scala +++ b/flink-table/flink-table-api-scala/src/main/scala/org/apache/flink/table/api/ImplicitExpressionConversions.scala @@ -158,17 +158,13 @@ trait ImplicitExpressionConversions { } } - implicit class TableFunctionCall[T: TypeInformation](val t: TableFunction[T]) { + implicit class TableFunctionCall(val t: TableFunction[_]) { /** * Calls a table function for the given parameters. */ def apply(params: Expression*): Expression = { - val resultTypeInfo: TypeInformation[T] = UserDefinedFunctionHelper - .getReturnTypeOfTableFunction(t, implicitly[TypeInformation[T]]) - unresolvedCall( - new TableFunctionDefinition(t.getClass.getName, t, resultTypeInfo), - params.map(ApiExpressionUtils.objectToExpression): _*) + unresolvedCall(t, params.map(ApiExpressionUtils.objectToExpression): _*) } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java index cf9beec2f45cb..22e93800aeb9a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/calcite/sql/validate/ProcedureNamespace.java @@ -21,13 +21,12 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlCallBinding; -import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlNode; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlTableFunction; -import org.apache.calcite.sql.SqlUtil; import org.apache.calcite.sql.type.SqlReturnTypeInference; -import org.apache.calcite.sql.type.SqlTypeName; + +import static java.util.Objects.requireNonNull; /** * Namespace whose contents are defined by the result of a call to a user-defined procedure. @@ -56,25 +55,23 @@ public final class ProcedureNamespace extends AbstractNamespace { public RelDataType validateImpl(RelDataType targetRowType) { validator.inferUnknownTypes(validator.unknownType, scope, call); - final RelDataType type = validator.deriveTypeImpl(scope, call); + // The result is ignored but the type is derived to trigger the validation + validator.deriveTypeImpl(scope, call); final SqlOperator operator = call.getOperator(); final SqlCallBinding callBinding = new SqlCallBinding(validator, scope, call); - if (operator instanceof SqlTableFunction) { - final SqlTableFunction tableFunction = (SqlTableFunction) operator; - if (type.getSqlTypeName() != SqlTypeName.CURSOR) { - throw new IllegalArgumentException( - "Table function should have CURSOR " + "type, not " + type); - } - final SqlReturnTypeInference rowTypeInference = tableFunction.getRowTypeInference(); - RelDataType retType = rowTypeInference.inferReturnType(callBinding); - return validator.getTypeFactory().createTypeWithNullability(retType, false); - } - - // special handling of collection tables TABLE(function(...)) - if (SqlUtil.stripAs(enclosingNode).getKind() == SqlKind.COLLECTION_TABLE) { - return toStruct(type, getNode()); + if (!(operator instanceof SqlTableFunction)) { + throw new IllegalArgumentException( + "Argument must be a table function: " + operator.getNameAsId()); } - return type; + final SqlTableFunction tableFunction = (SqlTableFunction) operator; + final SqlReturnTypeInference rowTypeInference = tableFunction.getRowTypeInference(); + final RelDataType rowRelDataType = + requireNonNull( + rowTypeInference.inferReturnType(callBinding), + () -> "got null from inferReturnType for call " + callBinding.getCall()); + // For BridgingSqlFunction the type can still be atomic + // and will be wrapped with a proper field alias + return toStruct(rowRelDataType, getNode()); } /** Converts a type to a struct if it is not already. */ diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java index 35ab473426e65..c5b774de45f41 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/calcite/FlinkRelBuilder.java @@ -23,6 +23,7 @@ import org.apache.flink.table.operations.QueryOperation; import org.apache.flink.table.planner.calcite.FlinkRelFactories.ExpandFactory; import org.apache.flink.table.planner.calcite.FlinkRelFactories.RankFactory; +import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction; import org.apache.flink.table.planner.hint.FlinkHints; import org.apache.flink.table.planner.plan.QueryOperationConverter; import org.apache.flink.table.planner.plan.logical.LogicalWindow; @@ -33,6 +34,7 @@ import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty; import org.apache.flink.table.runtime.operators.rank.RankRange; import org.apache.flink.table.runtime.operators.rank.RankType; +import org.apache.flink.util.CollectionUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; @@ -40,6 +42,7 @@ import org.apache.calcite.plan.Context; import org.apache.calcite.plan.Contexts; import org.apache.calcite.plan.RelOptCluster; +import org.apache.calcite.plan.RelOptRule; import org.apache.calcite.plan.RelOptSchema; import org.apache.calcite.plan.RelOptTable.ToRelContext; import org.apache.calcite.plan.ViewExpanders; @@ -48,18 +51,26 @@ import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rel.hint.RelHint; import org.apache.calcite.rel.logical.LogicalAggregate; +import org.apache.calcite.rel.logical.LogicalTableFunctionScan; +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.tools.RelBuilder; import org.apache.calcite.tools.RelBuilderFactory; import org.apache.calcite.util.ImmutableBitSet; import org.apache.calcite.util.Util; import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.function.UnaryOperator; +import java.util.stream.Collectors; import static org.apache.flink.table.planner.plan.utils.AggregateUtil.isTableAggregate; @@ -105,6 +116,54 @@ public static RelBuilderFactory proto(Context context) { }; } + /** + * {@link RelBuilder#functionScan(SqlOperator, int, Iterable)} cannot work smoothly with aliases + * which is why we implement a custom one. The method is static because some {@link RelOptRule}s + * don't use {@link FlinkRelBuilder}. + */ + public static RelBuilder pushFunctionScan( + RelBuilder relBuilder, + SqlOperator operator, + int inputCount, + Iterable operands, + List aliases) { + Preconditions.checkArgument( + operator instanceof BridgingSqlFunction.WithTableFunction, + "Table function expected."); + final RexBuilder rexBuilder = relBuilder.getRexBuilder(); + final RelDataTypeFactory typeFactory = relBuilder.getTypeFactory(); + + final List inputs = new LinkedList<>(); + for (int i = 0; i < inputCount; i++) { + inputs.add(0, relBuilder.build()); + } + + final List operandList = CollectionUtil.iterableToList(operands); + + final RelDataType functionRelDataType = rexBuilder.deriveReturnType(operator, operandList); + final List fieldRelDataTypes; + if (functionRelDataType.isStruct()) { + fieldRelDataTypes = + functionRelDataType.getFieldList().stream() + .map(RelDataTypeField::getType) + .collect(Collectors.toList()); + } else { + fieldRelDataTypes = Collections.singletonList(functionRelDataType); + } + final RelDataType rowRelDataType = typeFactory.createStructType(fieldRelDataTypes, aliases); + + final RexNode call = rexBuilder.makeCall(rowRelDataType, operator, operandList); + final RelNode functionScan = + LogicalTableFunctionScan.create( + relBuilder.getCluster(), + inputs, + call, + null, + rowRelDataType, + Collections.emptySet()); + return relBuilder.push(functionScan); + } + public RelBuilder expand(List> projects, int expandIdIndex) { final RelNode input = build(); final RelNode expand = expandFactory.createExpand(input, projects, expandIdIndex); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java index 5dbe4d8e7ce37..d66178760d8ce 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/bridging/BridgingSqlFunction.java @@ -26,6 +26,7 @@ import org.apache.flink.table.functions.FunctionIdentifier; import org.apache.flink.table.functions.FunctionKind; import org.apache.flink.table.planner.calcite.FlinkContext; +import org.apache.flink.table.planner.calcite.FlinkRelBuilder; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.utils.ShortcutUtils; import org.apache.flink.table.types.DataType; @@ -35,6 +36,10 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlTableFunction; +import org.apache.calcite.sql.type.SqlReturnTypeInference; +import org.apache.calcite.tools.RelBuilder; import java.util.List; @@ -52,7 +57,7 @@ * (either a system or user-defined function). */ @Internal -public final class BridgingSqlFunction extends SqlFunction { +public class BridgingSqlFunction extends SqlFunction { private final DataTypeFactory dataTypeFactory; @@ -108,6 +113,10 @@ public static BridgingSqlFunction of( functionKind == FunctionKind.SCALAR || functionKind == FunctionKind.TABLE, "Scalar or table function kind expected."); + if (functionKind == FunctionKind.TABLE) { + return new BridgingSqlFunction.WithTableFunction( + dataTypeFactory, typeFactory, kind, resolvedFunction, typeInference); + } return new BridgingSqlFunction( dataTypeFactory, typeFactory, kind, resolvedFunction, typeInference); } @@ -177,4 +186,32 @@ public List getParamNames() { public boolean isDeterministic() { return resolvedFunction.getDefinition().isDeterministic(); } + + // -------------------------------------------------------------------------------------------- + // Table function extension + // -------------------------------------------------------------------------------------------- + + /** Special flavor of {@link BridgingSqlFunction} to indicate a table function to Calcite. */ + public static class WithTableFunction extends BridgingSqlFunction implements SqlTableFunction { + + private WithTableFunction( + DataTypeFactory dataTypeFactory, + FlinkTypeFactory typeFactory, + SqlKind kind, + ContextResolvedFunction resolvedFunction, + TypeInference typeInference) { + super(dataTypeFactory, typeFactory, kind, resolvedFunction, typeInference); + } + + /** + * The conversion to a row type is handled on the caller side. This allows us to perform it + * SQL/Table API-specific. This is in particular important to set the aliases of fields + * correctly (see {@link FlinkRelBuilder#pushFunctionScan(RelBuilder, SqlOperator, int, + * Iterable, List)}). + */ + @Override + public SqlReturnTypeInference getRowTypeInference() { + return getReturnTypeInference(); + } + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java index 837b3b25451ec..3fcf62fafbe3e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java @@ -305,10 +305,14 @@ public RelNode visit(CalculatedQueryOperation calculatedTable) { final BridgingSqlFunction sqlFunction = BridgingSqlFunction.of(relBuilder.getCluster(), resolvedFunction); - return relBuilder - .functionScan(sqlFunction, 0, parameters) - .rename(calculatedTable.getResolvedSchema().getColumnNames()) - .build(); + FlinkRelBuilder.pushFunctionScan( + relBuilder, + sqlFunction, + 0, + parameters, + calculatedTable.getResolvedSchema().getColumnNames()); + + return relBuilder.build(); } private RelNode convertLegacyTableFunction( diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala index eb6d422711403..32e400f1a1b14 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/SetOpRewriteUtil.scala @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.utils import org.apache.flink.table.functions.BuiltInFunctionDefinitions +import org.apache.flink.table.planner.calcite.FlinkRelBuilder import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction import org.apache.calcite.plan.RelOptUtil @@ -73,12 +74,15 @@ object SetOpRewriteUtil { val cluster = relBuilder.getCluster val sqlFunction = BridgingSqlFunction.of( - relBuilder.getCluster, + cluster, BuiltInFunctionDefinitions.INTERNAL_REPLICATE_ROWS) - relBuilder - .functionScan(sqlFunction, 0, relBuilder.fields(Util.range(fields.size() + 1))) - .rename(outputRelDataType.getFieldNames) + FlinkRelBuilder.pushFunctionScan( + relBuilder, + sqlFunction, + 0, + relBuilder.fields(Util.range(fields.size() + 1)), + outputRelDataType.getFieldNames) // correlated join val corSet = Collections.singleton(cluster.createCorrel()) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java index d39d180f1b5c5..ed19aba020a7b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java @@ -64,6 +64,7 @@ import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -925,16 +926,12 @@ public void testInvalidUseOfScalarFunction() { public void testInvalidUseOfSystemScalarFunction() { tEnv().executeSql("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')"); - try { - tEnv().explainSql("INSERT INTO SinkTable " + "SELECT * FROM TABLE(MD5('3'))"); - fail(); - } catch (ValidationException e) { - assertThat( - e, - hasMessage( - containsString( - "Currently, only table functions can be used in a correlate operation."))); - } + assertThatThrownBy( + () -> + tEnv().explainSql( + "INSERT INTO SinkTable " + + "SELECT * FROM TABLE(MD5('3'))")) + .hasMessageContaining("Argument must be a table function: MD5"); } @Test @@ -946,16 +943,12 @@ public void testInvalidUseOfTableFunction() { tEnv().createTemporarySystemFunction("RowTableFunction", RowTableFunction.class); - try { - tEnv().explainSql("INSERT INTO SinkTable " + "SELECT RowTableFunction('test')"); - fail(); - } catch (ValidationException e) { - assertThat( - e, - hasMessage( - containsString( - "Currently, only scalar functions can be used in a projection or filter operation."))); - } + assertThatThrownBy( + () -> + tEnv().explainSql( + "INSERT INTO SinkTable " + + "SELECT RowTableFunction('test')")) + .hasMessageContaining("Cannot call table function here: 'RowTableFunction'"); } @Test diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java index 6891d62c68b74..06af8ef372c01 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java @@ -28,27 +28,23 @@ import org.apache.flink.table.planner.runtime.utils.StreamingTestBase; import org.apache.flink.types.Row; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.List; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.api.Expressions.$; import static org.apache.flink.table.api.Expressions.call; -import static org.hamcrest.CoreMatchers.containsString; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.hamcrest.CoreMatchers.equalTo; import static org.junit.Assert.assertThat; -import static org.junit.internal.matchers.ThrowableMessageMatcher.hasMessage; /** Tests for user defined functions in the Table API. */ public class FunctionITCase extends StreamingTestBase { - @Rule public ExpectedException thrown = ExpectedException.none(); - @Test - public void testScalarFunction() throws Exception { + void testScalarFunction() throws Exception { final List sourceData = Arrays.asList(Row.of(1, 1L, 1L), Row.of(2, 2L, 1L), Row.of(3, 3L, 1L)); @@ -61,7 +57,7 @@ public void testScalarFunction() throws Exception { tEnv().executeSql( "CREATE TABLE TestTable(a INT, b BIGINT, c BIGINT) WITH ('connector' = 'COLLECTION')"); - Table table = + final Table table = tEnv().from("TestTable") .select( $("a"), @@ -75,7 +71,7 @@ public void testScalarFunction() throws Exception { } @Test - public void testJoinWithTableFunction() throws Exception { + void testJoinWithTableFunction() throws Exception { final List sourceData = Arrays.asList( Row.of("1,2,3"), Row.of("2,3,4"), Row.of("3,4,5"), Row.of((String) null)); @@ -103,23 +99,22 @@ public void testJoinWithTableFunction() throws Exception { } @Test - public void testLateralJoinWithScalarFunction() throws Exception { - thrown.expect(ValidationException.class); - thrown.expect( - hasMessage( - containsString( - "Currently, only table functions can be used in a correlate operation."))); - + void testLateralJoinWithScalarFunction() throws Exception { TestCollectionTableFactory.reset(); tEnv().executeSql("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')"); tEnv().executeSql( "CREATE TABLE SinkTable(s STRING, sa ARRAY) WITH ('connector' = 'COLLECTION')"); - tEnv().from("SourceTable") - .joinLateral(call(new RowScalarFunction(), $("s")).as("a", "b")) - .select($("a"), $("b")) - .executeInsert("SinkTable") - .await(); + assertThatThrownBy( + () -> { + tEnv().from("SourceTable") + .joinLateral( + call(new RowScalarFunction(), $("s")).as("a", "b")); + }) + .satisfies( + anyCauseMatches( + ValidationException.class, + "A lateral join only accepts an expression which defines a table function")); } // -------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java index 95f6e51e14ef5..9d7d559713b86 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/utils/JavaUserDefinedTableFunctions.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple12; +import org.apache.flink.table.annotation.DataTypeHint; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.functions.TableFunction; @@ -34,7 +35,10 @@ public class JavaUserDefinedTableFunctions { /** Emit inputs as long. */ public static class JavaTableFunc0 extends TableFunction { - public void eval(Integer a, Long b, TimestampData c) { + public void eval( + @DataTypeHint("DATE") Integer a, + Long b, + @DataTypeHint("TIMESTAMP(0)") TimestampData c) { collect(a.longValue()); collect(b); collect(c.getMillisecond()); diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml index d34b6e84fc807..7b89148eab2f5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.xml @@ -57,8 +57,8 @@ LogicalIntersect(all=[true]) vcol_right_cnt), vcol_right_cnt, vcol_left_cnt) AS $f0, c], where=[((vcol_left_cnt >= 1) AND (vcol_right_cnt >= 1))]) +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_COUNT(count$0) AS vcol_left_cnt, Final_COUNT(count$1) AS vcol_right_cnt]) +- Exchange(distribution=[hash[c]]) @@ -180,8 +180,8 @@ LogicalMinus(all=[true]) 0)]) +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS sum_vcol_marker]) +- Exchange(distribution=[hash[c]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml index d6347f7086dce..a2d90a00ed11a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.xml @@ -26,13 +26,13 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> ($1, 20)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc0*(c))], select=[a,b,c,d,e], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) d, INTEGER e)], joinType=[INNER], condition=[>($1, 20)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -42,12 +42,12 @@ Calc(select=[c, d]) @@ -62,13 +62,13 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> ($1, 20)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc0*(c))], select=[a,b,c,d,e], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) d, INTEGER e)], joinType=[INNER], condition=[>($1, 20)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -79,13 +79,13 @@ Calc(select=[c, d]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -96,13 +96,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -114,13 +114,13 @@ LogicalFilter(condition=[>($1, _UTF-16LE'')]) +- LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> '')]) -+- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], correlate=[table(TableFunc1(c))], select=[a,b,c,s], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)], joinType=[LEFT]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc1*(c))], select=[a,b,c,s], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) s)], joinType=[LEFT]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -131,13 +131,13 @@ Calc(select=[c, s], where=[(s > '')]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml index 598ee0e9af542..d31bda912a895 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/SetOperatorsTest.xml @@ -33,8 +33,8 @@ Calc(select=[EXPR$0 AS a, b, EXPR$1 AS c]) +- HashAggregate(isMerge=[true], groupBy=[b], select=[b, Final_SUM(sum$0) AS EXPR$0, Final_COUNT(count$1) AS EXPR$1]) +- Exchange(distribution=[hash[b]]) +- LocalHashAggregate(groupBy=[b], select=[b, Partial_SUM(a) AS sum$0, Partial_COUNT(c) AS count$1]) - +- Calc(select=[f0 AS a, f1 AS b, f2 AS c]) - +- Correlate(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], correlate=[table($REPLICATE_ROWS$1(sum_vcol_marker,a,b,c))], select=[sum_vcol_marker,a,b,c,f0,f1,f2], rowType=[RecordType(BIGINT sum_vcol_marker, INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER f0, BIGINT f1, VARCHAR(2147483647) f2)], joinType=[INNER]) + +- Calc(select=[a0 AS a, b0 AS b, c0 AS c]) + +- Correlate(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], correlate=[table($REPLICATE_ROWS$1(sum_vcol_marker,a,b,c))], select=[sum_vcol_marker,a,b,c,a0,b0,c0], rowType=[RecordType(BIGINT sum_vcol_marker, INTEGER a, BIGINT b, VARCHAR(2147483647) c, INTEGER a0, BIGINT b0, VARCHAR(2147483647) c0)], joinType=[INNER]) +- Calc(select=[sum_vcol_marker, a, b, c], where=[(sum_vcol_marker > 0)]) +- HashAggregate(isMerge=[true], groupBy=[a, b, c], select=[a, b, c, Final_SUM(sum$0) AS sum_vcol_marker]) +- Exchange(distribution=[hash[a, b, c]]) @@ -123,8 +123,8 @@ LogicalProject(b=[$1], c=[$2]) 0)]) +- HashAggregate(isMerge=[true], groupBy=[b, c], select=[b, c, Final_SUM(sum$0) AS sum_vcol_marker]) +- Exchange(distribution=[hash[b, c]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.xml index 09262da4ea9fa..e69868781a231 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.xml @@ -22,13 +22,13 @@ limitations under the License. LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -39,13 +39,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -56,13 +56,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -73,13 +73,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], name=[$3], len=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)]) ]]> @@ -90,13 +90,13 @@ Calc(select=[c, name, len]) LogicalProject(c=[$2], name=[$3], len=[$5], adult=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.HierarchyTableFunction*($2)], rowType=[RecordType(VARCHAR(2147483647) name, BOOLEAN adult, INTEGER len)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.HierarchyTableFunction*($2)], rowType=[RecordType(VARCHAR(2147483647) name, BOOLEAN adult, INTEGER len)]) ]]> @@ -104,16 +104,16 @@ Calc(select=[c, name, len, adult]) @@ -125,13 +125,13 @@ LogicalFilter(condition=[>($2, 2)]) +- LogicalProject(c=[$2], name=[$3], len=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)]) ]]> ($1, 2)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc2*(c))], select=[a,b,c,name,len], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) name, INTEGER len)], joinType=[INNER], condition=[>($1, 2)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -142,13 +142,13 @@ Calc(select=[c, name, len]) LogicalProject(a=[$0], c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Table1, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*(SUBSTRING($2, 2))], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*(SUBSTRING($2, 2))], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml index 067ed8e287aa1..1ede5abe8397a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/common/PartialInsertTest.xml @@ -127,8 +127,8 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], fields=[a (sum_vcol_marker, 0)]) +- GroupAggregate(groupBy=[a, c, d, e, f, g], select=[a, c, d, e, f, g, SUM_RETRACT(vcol_marker) AS sum_vcol_marker]) +- Exchange(distribution=[hash[a, c, d, e, f, g]]) @@ -165,8 +165,8 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], fields=[a (sum_vcol_marker, 0)]) +- HashAggregate(isMerge=[true], groupBy=[a, c, d, e, f, g], select=[a, c, d, e, f, g, Final_SUM(sum$0) AS sum_vcol_marker]) +- Exchange(distribution=[hash[a, c, d, e, f, g]]) @@ -205,8 +205,8 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], fields=[a (vcol_left_cnt, vcol_right_cnt), vcol_right_cnt, vcol_left_cnt) AS $f0, a, c, d, e, f, g], where=[AND(>=(vcol_left_cnt, 1), >=(vcol_right_cnt, 1))]) +- GroupAggregate(groupBy=[a, c, d, e, f, g], select=[a, c, d, e, f, g, COUNT_RETRACT(vcol_left_marker) AS vcol_left_cnt, COUNT_RETRACT(vcol_right_marker) AS vcol_right_cnt]) +- Exchange(distribution=[hash[a, c, d, e, f, g]]) @@ -243,8 +243,8 @@ LogicalSink(table=[default_catalog.default_database.partitioned_sink], fields=[a (vcol_left_cnt, vcol_right_cnt), vcol_right_cnt, vcol_left_cnt) AS $f0, a, c, d, e, f, g], where=[AND(>=(vcol_left_cnt, 1), >=(vcol_right_cnt, 1))]) +- HashAggregate(isMerge=[true], groupBy=[a, c, d, e, f, g], select=[a, c, d, e, f, g, Final_COUNT(count$0) AS vcol_left_cnt, Final_COUNT(count$1) AS vcol_right_cnt]) +- Exchange(distribution=[hash[a, c, d, e, f, g]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml index c5a3d2ff1c104..ad832c4bab505 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteIntersectAllRuleTest.xml @@ -43,8 +43,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$0], vcol_left_marker=[null:BOOLEAN], vcol_right_marker=[true]) : +- LogicalProject(f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -77,8 +76,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$0], vcol_left_marker=[null:BOOLEAN], vcol_right_marker=[true]) : +- LogicalProject(f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -113,8 +111,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(d=[$0], e=[$1], f=[$2], vcol_left_marker=[null:BOOLEAN], vcol_right_marker=[true]) : +- LogicalProject(d=[$0], e=[$1], f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], rowType=[RecordType:peek_no_expand(INTEGER f0, BIGINT f1, VARCHAR(2147483647) f2)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)]) ]]> @@ -147,8 +144,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml index ae6d9dc74ba1c..e8956490d1cb6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/RewriteMinusAllRuleTest.xml @@ -43,8 +43,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$0], vcol_marker=[-1:BIGINT]) : +- LogicalProject(f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -77,8 +76,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$0], vcol_marker=[-1:BIGINT]) : +- LogicalProject(f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -111,8 +109,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(f=[$2]) : +- LogicalFilter(condition=[=(1, 0)]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(c=[$0]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1)], rowType=[RecordType(VARCHAR(2147483647) c)]) ]]> @@ -147,8 +144,7 @@ LogicalProject(c=[$2]) : +- LogicalProject(d=[$0], e=[$1], f=[$2], vcol_marker=[-1:BIGINT]) : +- LogicalProject(d=[$0], e=[$1], f=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, T2, source: [TestTableSource(d, e, f)]]]) - +- LogicalProject(a=[$0], b=[$1], c=[$2]) - +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], rowType=[RecordType:peek_no_expand(INTEGER f0, BIGINT f1, VARCHAR(2147483647) f2)]) + +- LogicalTableFunctionScan(invocation=[$REPLICATE_ROWS$1($0, $1, $2, $3)], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml index 59d848b8a5d98..61552b112b272 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SetOperatorsTest.xml @@ -58,8 +58,8 @@ LogicalIntersect(all=[true]) vcol_right_cnt), vcol_right_cnt, vcol_left_cnt) AS $f0, c], where=[((vcol_left_cnt >= 1) AND (vcol_right_cnt >= 1))]) +- GroupAggregate(groupBy=[c], select=[c, COUNT(vcol_left_marker) AS vcol_left_cnt, COUNT(vcol_right_marker) AS vcol_right_cnt]) +- Exchange(distribution=[hash[c]]) @@ -182,8 +182,8 @@ LogicalMinus(all=[true]) 0)]) +- GroupAggregate(groupBy=[c], select=[c, SUM(vcol_marker) AS sum_vcol_marker]) +- Exchange(distribution=[hash[c]]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml index 1c8dab3ceb709..063fcca673239 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/ColumnFunctionsTest.xml @@ -153,12 +153,12 @@ Join(joinType=[InnerJoin], where=[(int1 = int2)], select=[int1, long1, string1, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml index cbee6a2a6ae08..2542e8ddb26fa 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.xml @@ -21,12 +21,12 @@ limitations under the License. @@ -63,13 +63,13 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> ($1, 20)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc0*(c))], select=[a,b,c,d,e], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) d, INTEGER e)], joinType=[INNER], condition=[>($1, 20)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -84,13 +84,13 @@ LogicalProject(c=[$0], d=[$1]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalProject(a=[$0], b=[$1], c=[$2]) : +- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], rowType=[RecordType(VARCHAR(2147483647) d, INTEGER e)]) ]]> ($1, 20)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc0*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc0*(c))], select=[a,b,c,d,e], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) d, INTEGER e)], joinType=[INNER], condition=[>($1, 20)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -101,13 +101,13 @@ Calc(select=[c, d]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -118,13 +118,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2, _UTF-16LE'$')], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -135,13 +135,13 @@ Calc(select=[c, s]) LogicalProject(c=[$2], name=[$3], len=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*(*org.apache.flink.table.planner.expressions.utils.Func13$aceadf1af6c698a4705a8fbd3984d0a3*($2))], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*(*org.apache.flink.table.planner.expressions.utils.Func13$aceadf1af6c698a4705a8fbd3984d0a3*($2))], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)]) ]]> @@ -153,13 +153,13 @@ LogicalFilter(condition=[>($2, 2)]) +- LogicalProject(c=[$2], name=[$3], len=[$4]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) name, INTEGER len)]) ]]> ($1, 2)]) ++- Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], correlate=[table(*org.apache.flink.table.planner.utils.TableFunc2*(c))], select=[a,b,c,name,len], rowType=[RecordType(INTEGER a, BIGINT b, VARCHAR(2147483647) c, VARCHAR(2147483647) name, INTEGER len)], joinType=[INNER], condition=[>($1, 2)]) +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]], fields=[a, b, c]) ]]> @@ -170,14 +170,13 @@ Calc(select=[c, name, len]) LogicalProject(f0=[AS($3, _UTF-16LE'f0')], f1=[AS($4, _UTF-16LE'f1')]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(f1, f2, f3)]]]) - +- LogicalProject(f0=[$0], f1_0=[$1]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0, INTEGER f1)]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc2*($2)], rowType=[RecordType(VARCHAR(2147483647) f0, INTEGER f1_0)]) ]]> @@ -209,12 +208,12 @@ Correlate(invocation=[str_split(_UTF-16LE'Jack,John')], correlate=[table(str_spl @@ -291,13 +290,13 @@ Correlate(invocation=[str_split(_UTF-16LE'Jack,John', _UTF-16LE',')], correlate= LogicalProject(c=[$2], s=[$3]) +- LogicalCorrelate(correlation=[$cor0], joinType=[left], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, MyTable, source: [TestTableSource(a, b, c)]]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*($2)], rowType=[RecordType(VARCHAR(2147483647) s)]) ]]> @@ -307,12 +306,12 @@ Calc(select=[c, s]) @@ -322,12 +321,12 @@ Correlate(invocation=[*org.apache.flink.table.planner.utils.TableFunc1*(SUBSTRIN diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml index 0525902625f69..47f118e34d5cf 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/table/TemporalTableFunctionJoinTest.xml @@ -25,7 +25,7 @@ LogicalJoin(condition=[=($3, $1)], joinType=[inner]) : +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) : :- LogicalProject(o_rowtime=[AS($0, _UTF-16LE'o_rowtime')], o_comment=[AS($1, _UTF-16LE'o_comment')], o_amount=[AS($2, _UTF-16LE'o_amount')], o_currency=[AS($3, _UTF-16LE'o_currency')], o_secondary_key=[AS($4, _UTF-16LE'o_secondary_key')]) : : +- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) -: +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$bbf912e58a3fb2d083552961c0f87dbe*($0)], rowType=[RecordType(TIMESTAMP(3) *ROWTIME* rowtime, VARCHAR(2147483647) comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)], elementType=[class [Ljava.lang.Object;]) +: +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$bbf912e58a3fb2d083552961c0f87dbe*($0)], rowType=[RecordType(TIMESTAMP(3) *ROWTIME* rowtime, VARCHAR(2147483647) comment, VARCHAR(2147483647) currency, INTEGER rate, INTEGER secondary_key)]) +- LogicalTableScan(table=[[default_catalog, default_database, ThirdTable]]) ]]> @@ -53,7 +53,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$225833b9896cb7740aed9150b8cc7fd9*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$225833b9896cb7740aed9150b8cc7fd9*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) ]]> @@ -74,7 +74,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[Rates($2)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) + +- LogicalTableFunctionScan(invocation=[Rates($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) ]]> @@ -95,7 +95,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, ProctimeOrders]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$170cc46c47df69784f267e43f61e8e9d*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP_LTZ(3) *PROCTIME* proctime)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$170cc46c47df69784f267e43f61e8e9d*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP_LTZ(3) *PROCTIME* proctime)]) ]]> @@ -117,7 +117,7 @@ LogicalProject(rate=[AS(*($0, $4), _UTF-16LE'rate')]) +- LogicalFilter(condition=[=($3, $1)]) +- LogicalCorrelate(correlation=[$cor0], joinType=[inner], requiredColumns=[{}]) :- LogicalTableScan(table=[[default_catalog, default_database, Orders]]) - +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$225833b9896cb7740aed9150b8cc7fd9*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)], elementType=[class [Ljava.lang.Object;]) + +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.TemporalTableFunctionImpl$225833b9896cb7740aed9150b8cc7fd9*($2)], rowType=[RecordType(VARCHAR(2147483647) currency, INTEGER rate, TIMESTAMP(3) *ROWTIME* rowtime)]) ]]> diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala index ca28de3f23494..cddc89da55866 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala @@ -18,7 +18,6 @@ package org.apache.flink.table.planner.plan.batch.sql -import org.apache.flink.table.api.config.TableConfigOptions import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder import org.apache.flink.table.planner.utils._ diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala index 14348aeed7a43..8ebaee682c54d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/CorrelateTest.scala @@ -34,8 +34,6 @@ class CorrelateTest extends TableTestBase { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - util.addFunction("func1", func) - val result1 = table.joinLateral(func('c) as 's).select('c, 's) util.verifyExecPlan(result1) @@ -46,8 +44,6 @@ class CorrelateTest extends TableTestBase { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - util.addFunction("func1", func) - val result2 = table.joinLateral(func('c, "$") as 's).select('c, 's) util.verifyExecPlan(result2) } @@ -57,8 +53,6 @@ class CorrelateTest extends TableTestBase { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - util.addFunction("func1", func) - val result = table.leftOuterJoinLateral(func('c) as 's).select('c, 's).where('s > "") util.verifyExecPlan(result) } @@ -68,8 +62,6 @@ class CorrelateTest extends TableTestBase { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - util.addFunction("func1", func) - val result = table.leftOuterJoinLateral(func('c) as 's, true).select('c, 's) util.verifyExecPlan(result) } @@ -79,8 +71,6 @@ class CorrelateTest extends TableTestBase { val util = batchTestUtil() val sourceTable = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc0 - util.addFunction("func1", func) - val result = sourceTable.select('a, 'b, 'c) .joinLateral(func('c) as('d, 'e)) .select('c, 'd, 'e) @@ -106,8 +96,6 @@ class CorrelateTest extends TableTestBase { val sourceTable = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc0 - util.addFunction("func1", func) - val result = sourceTable.select('a, 'b, 'c) .joinLateral(func('c) as('d, 'e)) .select('c, 'd, 'e) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.scala index 95b340564fdf6..a4aa185d9aa27 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/stringexpr/CorrelateStringExpressionTest.scala @@ -29,9 +29,7 @@ class CorrelateStringExpressionTest extends TableTestBase { private val util = batchTestUtil() private val tab = util.addTableSource[(Int, Long, String)]("Table1", 'a, 'b, 'c) private val func1 = new TableFunc1 - util.addFunction("func1", func1) private val func2 = new TableFunc2 - util.addFunction("func2", func2) @Test def testCorrelateJoins1(): Unit = { @@ -61,7 +59,6 @@ class CorrelateStringExpressionTest extends TableTestBase { def testCorrelateJoins5(): Unit = { // test hierarchy generic type val hierarchy = new HierarchyTableFunction - util.addFunction("hierarchy", hierarchy) val scalaTable = tab.joinLateral( hierarchy('c) as('name, 'adult, 'len)).select('c, 'name, 'len, 'adult) util.verifyExecPlan(scalaTable) @@ -71,7 +68,6 @@ class CorrelateStringExpressionTest extends TableTestBase { def testCorrelateJoins6(): Unit = { // test pojo type val pojo = new PojoTableFunc - util.addFunction("pojo", pojo) val scalaTable = tab.joinLateral(pojo('c)).select('c, 'name, 'age) util.verifyExecPlan(scalaTable) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala index d69383edfcdc0..611928ef5c663 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/table/validation/CorrelateValidationTest.scala @@ -36,7 +36,6 @@ class CorrelateValidationTest extends TableTestBase { val util = batchTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val func = new TableFunc1 - util.addFunction("func1", func) val result = table .leftOuterJoinLateral(func('c) as 's, 'c === 's) .select('c, 's) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala index 339e7cad4d4fd..ff1be68df1ced 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/CorrelateTest.scala @@ -36,8 +36,6 @@ class CorrelateTest extends TableTestBase { val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc1 - util.addFunction("func1", function) - val result1 = table.joinLateral(function('c) as 's).select('c, 's) util.verifyExecPlan(result1) } @@ -48,7 +46,6 @@ class CorrelateTest extends TableTestBase { val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc1 - util.addFunction("func1", function) // test overloading val result2 = table.joinLateral(function('c, "$") as 's).select('c, 's) util.verifyExecPlan(result2) @@ -59,8 +56,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc1 - util.addFunction("func1", function) - val result = table.leftOuterJoinLateral(function('c) as 's, true).select('c, 's) util.verifyExecPlan(result) } @@ -70,7 +65,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc2 - util.addFunction("func2", function) val scalarFunc = new Func13("pre") val result = table.joinLateral( @@ -84,8 +78,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new HierarchyTableFunction - util.addFunction("hierarchy", function) - val result = table.joinLateral(function('c) as ('name, 'adult, 'len)) util.verifyExecPlan(result) } @@ -95,8 +87,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new PojoTableFunc - util.addFunction("pojo", function) - val result = table.joinLateral(function('c)) util.verifyExecPlan(result) } @@ -106,8 +96,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc2 - util.addFunction("func2", function) - val result = table .joinLateral(function('c) as ('name, 'len)) .select('c, 'name, 'len) @@ -120,8 +108,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val table = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc1 - util.addFunction("func1", function) - val result = table.joinLateral(function('c.substring(2)) as 's) util.verifyExecPlan(result) } @@ -131,8 +117,6 @@ class CorrelateTest extends TableTestBase { val util = streamTestUtil() val sourceTable = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc0 - util.addFunction("func1", function) - val result = sourceTable.select('a, 'b, 'c) .joinLateral(function('c) as('d, 'e)) .select('c, 'd, 'e) @@ -158,7 +142,6 @@ class CorrelateTest extends TableTestBase { val sourceTable = util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c) val function = new TableFunc0 - util.addFunction("func1", function) val result = sourceTable.select('a, 'b, 'c) .joinLateral(function('c) as('d, 'e)) .select('c, 'd, 'e) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala index ec2401fd6b241..a991cc531283b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/CorrelateValidationTest.scala @@ -68,11 +68,10 @@ class CorrelateValidationTest extends TableTestBase { util.addFunction("func0", Func0) // SQL API call - // NOTE: it doesn't throw an exception but an AssertionError, maybe a Calcite bug expectExceptionThrown( util.tableEnv.sqlQuery("SELECT * FROM MyTable, LATERAL TABLE(func0(a))"), null, - classOf[AssertionError]) + classOf[ValidationException]) //========== throw exception when the parameters is not correct =============== // Java Table API call diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CorrelateITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CorrelateITCase.scala index 17a35c2b5778b..a125f93c717dd 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CorrelateITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/CorrelateITCase.scala @@ -222,7 +222,8 @@ class CorrelateITCase extends BatchTestBase { 'a.cast(DataTypes.TINYINT) as 'a, 'a.cast(DataTypes.SMALLINT) as 'b, 'b.cast(DataTypes.FLOAT) as 'c) - .joinLateral(tFunc('a, 'b, 'c) as ('a2, 'b2, 'c2)) + .joinLateral( + tFunc('a.ifNull(0.toByte), 'b.ifNull(0.toShort), 'c.ifNull(0.toFloat)) as ('a2, 'b2, 'c2)) val results = executeQuery(result) val expected = Seq( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala index 9f106c4f25634..683543b09b2be 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.factories.TestValuesTableFactory import org.apache.flink.test.util.AbstractTestBase import org.apache.flink.types.Row +import org.junit.jupiter.api.{AfterEach, BeforeEach} import org.junit.rules.{ExpectedException, TemporaryFolder} import org.junit.{After, Before, Rule} @@ -44,6 +45,7 @@ class StreamingTestBase extends AbstractTestBase { def tempFolder: TemporaryFolder = _tempFolder @Before + @BeforeEach def before(): Unit = { this.env = StreamExecutionEnvironment.getExecutionEnvironment env.setParallelism(4) @@ -55,6 +57,7 @@ class StreamingTestBase extends AbstractTestBase { } @After + @AfterEach def after(): Unit = { StreamTestSink.clear() TestValuesTableFactory.clearAllData() diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala index f087d5a91dd7b..6fcad3d50c817 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/UserDefinedTableFunctions.scala @@ -29,8 +29,6 @@ import org.apache.flink.types.Row import org.junit.Assert -import java.lang.Boolean - import scala.annotation.varargs @@ -117,9 +115,10 @@ class TableFunc3(data: String, conf: Map[String, String]) extends TableFunction[ } @SerialVersionUID(1L) +@DataTypeHint("ROW") class MockPythonTableFunction extends TableFunction[Row] with PythonFunction { - def eval(x: Int, y: Int) = ??? + def eval(x: java.lang.Integer, y: java.lang.Integer) = ??? override def getResultType: TypeInformation[Row] = new RowTypeInfo(Types.INT, Types.INT) @@ -368,28 +367,19 @@ class MockPythonTableFunction extends TableFunction[Row] with PythonFunction { //} @SerialVersionUID(1L) +@DataTypeHint("ROW") class TableFunc4 extends TableFunction[Row] { def eval(b: Byte, s: Short, f: Float): Unit = { collect(Row.of("Byte=" + b, "Short=" + s, "Float=" + f)) } - - override def getResultType: TypeInformation[Row] = { - new RowTypeInfo(Types.STRING, Types.STRING, Types.STRING) - } } @SerialVersionUID(1L) +@DataTypeHint("ROW") class TableFunc6 extends TableFunction[Row] { - def eval(row: Row): Unit = { + def eval(@DataTypeHint("ROW") row: Row): Unit = { collect(row) } - - override def getParameterTypes(signature: Array[Class[_]]): Array[TypeInformation[_]] = - Array(new RowTypeInfo(Types.INT, Types.INT, Types.INT)) - - override def getResultType: TypeInformation[Row] = { - new RowTypeInfo(Types.INT, Types.INT, Types.INT) - } } @SerialVersionUID(1L) @@ -421,12 +411,12 @@ class VarArgsFunc0 extends TableFunction[String] { } @SerialVersionUID(1L) -class HierarchyTableFunction extends SplittableTableFunction[Boolean, Integer] { +class HierarchyTableFunction extends SplittableTableFunction[java.lang.Boolean, Integer] { def eval(user: String) { if (user.contains("#")) { val splits = user.split("#") val age = splits(1).toInt - collect(new Tuple3[String, Boolean, Integer](splits(0), age >= 20, age)) + collect(new Tuple3[String, java.lang.Boolean, Integer](splits(0), age >= 20, age)) } } } From 20c71102531e2a49b153a30ea430c316d1632a81 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 24 Mar 2022 17:02:38 +0800 Subject: [PATCH 085/258] [hotfix][python][docs] Fix the example in intro_to_datastream_api --- .../docs/dev/python/datastream/intro_to_datastream_api.md | 2 +- .../docs/dev/python/datastream/intro_to_datastream_api.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/dev/python/datastream/intro_to_datastream_api.md b/docs/content.zh/docs/dev/python/datastream/intro_to_datastream_api.md index 4605c8b750b58..5674e6e0fb6a0 100644 --- a/docs/content.zh/docs/dev/python/datastream/intro_to_datastream_api.md +++ b/docs/content.zh/docs/dev/python/datastream/intro_to_datastream_api.md @@ -78,7 +78,7 @@ def state_access_demo(): # 3. define the execution logic ds = ds.map(lambda a: Row(a % 4, 1), output_type=Types.ROW([Types.LONG(), Types.LONG()])) \ .key_by(lambda a: a[0]) \ - .map(MyMapFunction(), output_type=Types.ROW([Types.LONG(), Types.LONG()])) + .map(MyMapFunction(), output_type=Types.TUPLE([Types.LONG(), Types.LONG()])) # 4. create sink and emit result to sink output_path = '/opt/output/' diff --git a/docs/content/docs/dev/python/datastream/intro_to_datastream_api.md b/docs/content/docs/dev/python/datastream/intro_to_datastream_api.md index ffa71f6d17299..22bec5cd6b0b2 100644 --- a/docs/content/docs/dev/python/datastream/intro_to_datastream_api.md +++ b/docs/content/docs/dev/python/datastream/intro_to_datastream_api.md @@ -78,7 +78,7 @@ def state_access_demo(): # 3. define the execution logic ds = ds.map(lambda a: Row(a % 4, 1), output_type=Types.ROW([Types.LONG(), Types.LONG()])) \ .key_by(lambda a: a[0]) \ - .map(MyMapFunction(), output_type=Types.ROW([Types.LONG(), Types.LONG()])) + .map(MyMapFunction(), output_type=Types.TUPLE([Types.LONG(), Types.LONG()])) # 4. create sink and emit result to sink output_path = '/opt/output/' From 34133b9d040d22f76710b8da987ad848f63063d3 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 21 Mar 2022 14:15:27 +0100 Subject: [PATCH 086/258] [FLINK-26741][runtime] Changes method signature of CheckpointIDCounter.shutdown The new method returns a CompletableFuture that can be processed. This also includes adding idempotency tests for the ZK and k8s implementations. --- .../KubernetesCheckpointIDCounter.java | 31 ++-- .../KubernetesCheckpointIDCounterTest.java | 138 ++++++++++++++++++ .../KubernetesHighAvailabilityTestBase.java | 4 + .../checkpoint/CheckpointIDCounter.java | 5 +- .../DeactivatedCheckpointIDCounter.java | 7 +- .../StandaloneCheckpointIDCounter.java | 6 +- .../ZooKeeperCheckpointIDCounter.java | 58 +++++++- .../CheckpointResourcesCleanupRunner.java | 2 +- .../runtime/scheduler/SchedulerBase.java | 2 +- .../scheduler/adaptive/AdaptiveScheduler.java | 2 +- .../CheckpointCoordinatorTriggeringTest.java | 5 +- .../CheckpointIDCounterTestBase.java | 8 +- .../TestingCheckpointIDCounter.java | 27 ++-- .../ZooKeeperCheckpointIDCounterITCase.java | 97 ++++++++++-- .../scheduler/DefaultSchedulerTest.java | 21 ++- .../adaptive/AdaptiveSchedulerTest.java | 3 +- 16 files changed, 362 insertions(+), 54 deletions(-) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounter.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounter.java index a08146a956e0a..8eac11ec9cc1e 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounter.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounter.java @@ -25,6 +25,7 @@ import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.concurrent.FutureUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,6 +33,7 @@ import javax.annotation.Nullable; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import static org.apache.flink.kubernetes.utils.Constants.CHECKPOINT_COUNTER_KEY; @@ -71,25 +73,32 @@ public void start() { } @Override - public void shutdown(JobStatus jobStatus) { + public CompletableFuture shutdown(JobStatus jobStatus) { if (!running) { - return; + return FutureUtils.completedVoidFuture(); } running = false; LOG.info("Shutting down."); if (jobStatus.isGloballyTerminalState()) { LOG.info("Removing counter from ConfigMap {}", configMapName); - kubeClient.checkAndUpdateConfigMap( - configMapName, - configMap -> { - if (isValidOperation(configMap)) { - configMap.getData().remove(CHECKPOINT_COUNTER_KEY); - return Optional.of(configMap); - } - return Optional.empty(); - }); + return kubeClient + .checkAndUpdateConfigMap( + configMapName, + configMap -> { + if (isValidOperation(configMap)) { + configMap.getData().remove(CHECKPOINT_COUNTER_KEY); + return Optional.of(configMap); + } + return Optional.empty(); + }) + // checkAndUpdateConfigMap only returns false if the callback returned an empty + // ConfigMap. We don't want to continue the cleanup in that case, i.e. we can + // ignore the return value + .thenApply(valueChanged -> null); } + + return FutureUtils.completedVoidFuture(); } private boolean isValidOperation(KubernetesConfigMap configMap) { diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java index d6d4313c62c78..30b8875bd1ca8 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java @@ -18,13 +18,19 @@ package org.apache.flink.kubernetes.highavailability; +import org.apache.flink.api.common.JobStatus; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.kubernetes.kubeclient.resources.KubernetesLeaderElector; +import org.apache.flink.kubernetes.utils.Constants; +import org.apache.flink.kubernetes.utils.KubernetesUtils; import org.junit.Test; +import java.util.concurrent.CompletionException; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; /** Tests for {@link KubernetesCheckpointIDCounter} operations. */ @@ -50,6 +56,138 @@ public void testGetAndIncrement() throws Exception { }; } + @Test + public void testShutdown() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final KubernetesCheckpointIDCounter checkpointIDCounter = + new KubernetesCheckpointIDCounter( + flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); + checkpointIDCounter.start(); + + checkpointIDCounter.setCount(100L); + + assertThat( + getLeaderConfigMap() + .getData() + .get(Constants.CHECKPOINT_COUNTER_KEY), + is("100")); + + checkpointIDCounter.shutdown(JobStatus.FINISHED).join(); + + assertThat( + getLeaderConfigMap() + .getData() + .containsKey(Constants.CHECKPOINT_COUNTER_KEY), + is(false)); + }); + } + }; + } + + @Test + public void testShutdownForLocallyTerminatedJobStatus() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final KubernetesCheckpointIDCounter checkpointIDCounter = + new KubernetesCheckpointIDCounter( + flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); + checkpointIDCounter.start(); + + checkpointIDCounter.setCount(100L); + + assertThat( + getLeaderConfigMap() + .getData() + .get(Constants.CHECKPOINT_COUNTER_KEY), + is("100")); + + checkpointIDCounter.shutdown(JobStatus.SUSPENDED).join(); + + assertThat( + getLeaderConfigMap() + .getData() + .containsKey(Constants.CHECKPOINT_COUNTER_KEY), + is(true)); + }); + } + }; + } + + @Test + public void testIdempotentShutdown() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final KubernetesCheckpointIDCounter checkpointIDCounter = + new KubernetesCheckpointIDCounter( + flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); + checkpointIDCounter.start(); + + assertThat( + getLeaderConfigMap() + .getData() + .containsKey(Constants.CHECKPOINT_COUNTER_KEY), + is(false)); + + checkpointIDCounter.shutdown(JobStatus.FINISHED).join(); + + assertThat( + getLeaderConfigMap() + .getData() + .containsKey(Constants.CHECKPOINT_COUNTER_KEY), + is(false)); + + // a second shutdown should work without causing any errors + checkpointIDCounter.shutdown(JobStatus.FINISHED).join(); + }); + } + }; + } + + @Test + public void testShutdownFailureDueToMissingConfigMap() throws Exception { + new Context() { + { + runTest( + () -> { + leaderCallbackGrantLeadership(); + + final KubernetesCheckpointIDCounter checkpointIDCounter = + new KubernetesCheckpointIDCounter( + flinkKubeClient, LEADER_CONFIGMAP_NAME, LOCK_IDENTITY); + checkpointIDCounter.start(); + + // deleting the ConfigMap from outside of the CheckpointIDCounter while + // still using the counter (which is stored as an entry in the + // ConfigMap) causes an unexpected failure which we want to simulate + // here + flinkKubeClient.deleteConfigMap(LEADER_CONFIGMAP_NAME); + + assertThrows( + CompletionException.class, + () -> checkpointIDCounter.shutdown(JobStatus.FINISHED).get()); + + // fixing the internal issue should make the shutdown succeed again + KubernetesUtils.createConfigMapIfItDoesNotExist( + flinkKubeClient, LEADER_CONFIGMAP_NAME, getClusterId()); + checkpointIDCounter.shutdown(JobStatus.FINISHED).get(); + }); + } + }; + } + @Test public void testGetAndIncrementWithNoLeadership() throws Exception { new Context() { diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java index 5310a8b907efd..543a589a6eb3c 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java @@ -118,6 +118,10 @@ TestingFlinkKubeClient.Builder createFlinkKubeClientBuilder() { return kubernetesTestFixture.createFlinkKubeClientBuilder(); } + String getClusterId() { + return CLUSTER_ID; + } + KubernetesConfigMap getLeaderConfigMap() { return kubernetesTestFixture.getLeaderConfigMap(); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java index 81e0471c4061b..bb4a39522b62c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounter.java @@ -20,6 +20,8 @@ import org.apache.flink.api.common.JobStatus; +import java.util.concurrent.CompletableFuture; + /** A checkpoint ID counter. */ public interface CheckpointIDCounter { int INITIAL_CHECKPOINT_ID = 1; @@ -34,8 +36,9 @@ public interface CheckpointIDCounter { * or kept. * * @param jobStatus Job state on shut down + * @return The {@code CompletableFuture} holding the result of the shutdown operation. */ - void shutdown(JobStatus jobStatus) throws Exception; + CompletableFuture shutdown(JobStatus jobStatus); /** * Atomically increments the current checkpoint ID. diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DeactivatedCheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DeactivatedCheckpointIDCounter.java index ca86159b6f581..7d670102063f2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DeactivatedCheckpointIDCounter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DeactivatedCheckpointIDCounter.java @@ -19,6 +19,9 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.util.concurrent.FutureUtils; + +import java.util.concurrent.CompletableFuture; /** * This class represents a {@link CheckpointIDCounter} if checkpointing is deactivated. @@ -32,7 +35,9 @@ public enum DeactivatedCheckpointIDCounter implements CheckpointIDCounter { public void start() throws Exception {} @Override - public void shutdown(JobStatus jobStatus) throws Exception {} + public CompletableFuture shutdown(JobStatus jobStatus) { + return FutureUtils.completedVoidFuture(); + } @Override public long getAndIncrement() throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java index 30b653949892b..82400350422eb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointIDCounter.java @@ -20,7 +20,9 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.util.concurrent.FutureUtils; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; /** @@ -37,7 +39,9 @@ public class StandaloneCheckpointIDCounter implements CheckpointIDCounter { public void start() throws Exception {} @Override - public void shutdown(JobStatus jobStatus) throws Exception {} + public CompletableFuture shutdown(JobStatus jobStatus) { + return FutureUtils.completedVoidFuture(); + } @Override public long getAndIncrement() throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java index 9c649e6ac7627..6576dde11c21f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounter.java @@ -22,18 +22,27 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.util.ZooKeeperUtils; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.concurrent.FutureUtils; +import org.apache.flink.shaded.curator5.com.google.common.collect.Sets; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework; +import org.apache.flink.shaded.curator5.org.apache.curator.framework.api.CuratorEvent; +import org.apache.flink.shaded.curator5.org.apache.curator.framework.api.CuratorEventType; import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.shared.SharedCount; import org.apache.flink.shaded.curator5.org.apache.curator.framework.recipes.shared.VersionedValue; import org.apache.flink.shaded.curator5.org.apache.curator.framework.state.ConnectionState; +import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.concurrent.GuardedBy; +import java.io.IOException; import java.util.Optional; +import java.util.concurrent.CompletableFuture; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -101,22 +110,65 @@ public void start() throws Exception { } @Override - public void shutdown(JobStatus jobStatus) throws Exception { + public CompletableFuture shutdown(JobStatus jobStatus) { synchronized (startStopLock) { if (isStarted) { LOG.info("Shutting down."); - sharedCount.close(); + try { + sharedCount.close(); + } catch (IOException e) { + return FutureUtils.completedExceptionally(e); + } client.getConnectionStateListenable().removeListener(connectionStateListener); if (jobStatus.isGloballyTerminalState()) { LOG.info("Removing {} from ZooKeeper", counterPath); - client.delete().deletingChildrenIfNeeded().inBackground().forPath(counterPath); + try { + final CompletableFuture deletionFuture = new CompletableFuture<>(); + client.delete() + .inBackground( + (curatorFramework, curatorEvent) -> + handleDeletionOfCounterPath( + curatorEvent, deletionFuture)) + .forPath(counterPath); + return deletionFuture; + } catch (Exception e) { + return FutureUtils.completedExceptionally(e); + } } isStarted = false; } } + + return FutureUtils.completedVoidFuture(); + } + + private void handleDeletionOfCounterPath( + CuratorEvent curatorEvent, CompletableFuture deletionFuture) { + Preconditions.checkArgument( + curatorEvent.getType() == CuratorEventType.DELETE, + "An unexpected CuratorEvent was monitored: " + curatorEvent.getType()); + Preconditions.checkArgument( + counterPath.endsWith(curatorEvent.getPath()), + "An unexpected path was selected for deletion: " + curatorEvent.getPath()); + + final KeeperException.Code eventCode = + KeeperException.Code.get(curatorEvent.getResultCode()); + if (Sets.immutableEnumSet(KeeperException.Code.OK, KeeperException.Code.NONODE) + .contains(eventCode)) { + deletionFuture.complete(null); + } else { + final String namespacedCounterPath = + ZooKeeperUtils.generateZookeeperPath(client.getNamespace(), counterPath); + deletionFuture.completeExceptionally( + new FlinkException( + String.format( + "An error occurred while shutting down the CheckpointIDCounter in path '%s'.", + namespacedCounterPath), + KeeperException.create(eventCode, namespacedCounterPath))); + } } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java index 4ae76780a4f7d..c97f8644a1f6f 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java @@ -129,7 +129,7 @@ private void cleanupCheckpoints() throws Exception { } try { - checkpointIDCounter.shutdown(getJobStatus()); + checkpointIDCounter.shutdown(getJobStatus()).get(); } catch (Exception e) { exception = ExceptionUtils.firstOrSuppressed(e, exception); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java index c997cc2f1098f..0ca22335c078c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java @@ -245,7 +245,7 @@ private void shutDownCheckpointServices(JobStatus jobStatus) { } try { - checkpointIdCounter.shutdown(jobStatus); + checkpointIdCounter.shutdown(jobStatus).get(); } catch (Exception e) { exception = ExceptionUtils.firstOrSuppressed(e, exception); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java index 999b2afb5532d..562c17f05c32a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java @@ -463,7 +463,7 @@ private void stopCheckpointServicesSafely(JobStatus terminalState) { } try { - checkpointIdCounter.shutdown(terminalState); + checkpointIdCounter.shutdown(terminalState).get(); } catch (Exception e) { exception = ExceptionUtils.firstOrSuppressed(e, exception); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java index ddf446bf66501..643c09d95c684 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTriggeringTest.java @@ -35,6 +35,7 @@ import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.concurrent.FutureUtils; import org.apache.flink.util.concurrent.ManuallyTriggeredScheduledExecutor; import org.apache.flink.util.concurrent.ScheduledExecutorServiceAdapter; @@ -835,7 +836,9 @@ public UnstableCheckpointIDCounter(Predicate checkpointFailurePredicate) { public void start() {} @Override - public void shutdown(JobStatus jobStatus) throws Exception {} + public CompletableFuture shutdown(JobStatus jobStatus) { + return FutureUtils.completedVoidFuture(); + } @Override public long getAndIncrement() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java index 1dcfd3344bcd4..59933e856b200 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointIDCounterTestBase.java @@ -58,7 +58,7 @@ public void testCounterIsNeverNegative() throws Exception { counter.start(); assertThat(counter.get()).isGreaterThanOrEqualTo(0L); } finally { - counter.shutdown(JobStatus.FINISHED); + counter.shutdown(JobStatus.FINISHED).join(); } } @@ -78,7 +78,7 @@ public void testSerialIncrementAndGet() throws Exception { assertThat(counter.get()).isEqualTo(4); assertThat(counter.getAndIncrement()).isEqualTo(4); } finally { - counter.shutdown(JobStatus.FINISHED); + counter.shutdown(JobStatus.FINISHED).join(); } } @@ -136,7 +136,7 @@ public void testConcurrentGetAndIncrement() throws Exception { executor.shutdown(); } - counter.shutdown(JobStatus.FINISHED); + counter.shutdown(JobStatus.FINISHED).join(); } } @@ -161,7 +161,7 @@ public void testSetCount() throws Exception { assertThat(counter.get()).isEqualTo(1338); assertThat(counter.getAndIncrement()).isEqualTo(1338); - counter.shutdown(JobStatus.FINISHED); + counter.shutdown(JobStatus.FINISHED).join(); } /** Task repeatedly incrementing the {@link CheckpointIDCounter}. */ diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java index 2365ed59789cf..466a69a25d94f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointIDCounter.java @@ -18,16 +18,18 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.util.concurrent.FutureUtils; import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; /** Test {@link CheckpointIDCounter} implementation for testing the shutdown behavior. */ public final class TestingCheckpointIDCounter implements CheckpointIDCounter { private final Runnable startRunnable; - private final Consumer shutdownConsumer; + private final Function> shutdownFunction; private final Supplier getAndIncrementSupplier; private final Supplier getSupplier; private final Consumer setCountConsumer; @@ -36,18 +38,22 @@ public static TestingCheckpointIDCounter createStoreWithShutdownCheckAndNoStartA CompletableFuture shutdownFuture) { return TestingCheckpointIDCounter.builder() .withStartRunnable(() -> {}) - .withShutdownConsumer(shutdownFuture::complete) + .withShutdownConsumer( + jobStatus -> { + shutdownFuture.complete(jobStatus); + return FutureUtils.completedVoidFuture(); + }) .build(); } private TestingCheckpointIDCounter( Runnable startRunnable, - Consumer shutdownConsumer, + Function> shutdownFunction, Supplier getAndIncrementSupplier, Supplier getSupplier, Consumer setCountConsumer) { this.startRunnable = startRunnable; - this.shutdownConsumer = shutdownConsumer; + this.shutdownFunction = shutdownFunction; this.getAndIncrementSupplier = getAndIncrementSupplier; this.getSupplier = getSupplier; this.setCountConsumer = setCountConsumer; @@ -59,8 +65,8 @@ public void start() { } @Override - public void shutdown(JobStatus jobStatus) { - shutdownConsumer.accept(jobStatus); + public CompletableFuture shutdown(JobStatus jobStatus) { + return shutdownFunction.apply(jobStatus); } @Override @@ -86,7 +92,7 @@ public static Builder builder() { public static class Builder { private Runnable startRunnable; - private Consumer shutdownConsumer; + private Function> shutdownFunction; private Supplier getAndIncrementSupplier; private Supplier getSupplier; private Consumer setCountConsumer; @@ -96,8 +102,9 @@ public Builder withStartRunnable(Runnable startRunnable) { return this; } - public Builder withShutdownConsumer(Consumer shutdownConsumer) { - this.shutdownConsumer = shutdownConsumer; + public Builder withShutdownConsumer( + Function> shutdownFunction) { + this.shutdownFunction = shutdownFunction; return this; } @@ -119,7 +126,7 @@ public Builder withSetCountConsumer(Consumer setCountConsumer) { public TestingCheckpointIDCounter build() { return new TestingCheckpointIDCounter( startRunnable, - shutdownConsumer, + shutdownFunction, getAndIncrementSupplier, getSupplier, setCountConsumer); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java index 798ea795589fe..4e9c0d4e55470 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointIDCounterITCase.java @@ -19,16 +19,21 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.core.testutils.FlinkAssertions; +import org.apache.flink.runtime.util.ZooKeeperUtils; import org.apache.flink.runtime.zookeeper.ZooKeeperTestEnvironment; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework; +import org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.KeeperException; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.util.concurrent.ExecutionException; + import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** * Unit tests for the {@link ZooKeeperCheckpointIDCounter}. The tests are inherited from the test @@ -38,19 +43,21 @@ class ZooKeeperCheckpointIDCounterITCase extends CheckpointIDCounterTestBase { private static ZooKeeperTestEnvironment zookeeper; - @BeforeAll - public static void setUp() throws Exception { + @BeforeEach + void setup() { zookeeper = new ZooKeeperTestEnvironment(1); } - @AfterAll - private static void tearDown() throws Exception { - zookeeper.shutdown(); + @AfterEach + void tearDown() throws Exception { + cleanAndStopZooKeeperIfRunning(); } - @BeforeEach - private void cleanUp() throws Exception { - zookeeper.deleteAll(); + private void cleanAndStopZooKeeperIfRunning() throws Exception { + if (zookeeper.getClient().isStarted()) { + zookeeper.deleteAll(); + zookeeper.shutdown(); + } } /** Tests that counter node is removed from ZooKeeper after shutdown. */ @@ -62,10 +69,73 @@ public void testShutdownRemovesState() throws Exception { CuratorFramework client = zookeeper.getClient(); assertThat(client.checkExists().forPath(counter.getPath())).isNotNull(); - counter.shutdown(JobStatus.FINISHED); + counter.shutdown(JobStatus.FINISHED).join(); + assertThat(client.checkExists().forPath(counter.getPath())).isNull(); + } + + @Test + public void testIdempotentShutdown() throws Exception { + ZooKeeperCheckpointIDCounter counter = createCheckpointIdCounter(); + counter.start(); + + CuratorFramework client = zookeeper.getClient(); + counter.shutdown(JobStatus.FINISHED).join(); + + // shutdown shouldn't fail due to missing path + counter.shutdown(JobStatus.FINISHED).join(); assertThat(client.checkExists().forPath(counter.getPath())).isNull(); } + @Test + public void testShutdownWithFailureDueToMissingConnection() throws Exception { + ZooKeeperCheckpointIDCounter counter = createCheckpointIdCounter(); + counter.start(); + + cleanAndStopZooKeeperIfRunning(); + + assertThatThrownBy(() -> counter.shutdown(JobStatus.FINISHED).get()) + .as("The shutdown should fail because of the client connection being dropped.") + .isInstanceOf(ExecutionException.class) + .hasCauseExactlyInstanceOf(IllegalStateException.class); + } + + @Test + public void testShutdownWithFailureDueToExistingChildNodes() throws Exception { + final ZooKeeperCheckpointIDCounter counter = createCheckpointIdCounter(); + counter.start(); + + final CuratorFramework client = + ZooKeeperUtils.useNamespaceAndEnsurePath(zookeeper.getClient(), "/"); + final String counterNodePath = ZooKeeperUtils.generateZookeeperPath(counter.getPath()); + final String childNodePath = + ZooKeeperUtils.generateZookeeperPath( + counterNodePath, "unexpected-child-node-causing-a-failure"); + client.create().forPath(childNodePath); + + final String namespacedCounterNodePath = + ZooKeeperUtils.generateZookeeperPath(client.getNamespace(), counterNodePath); + final Throwable expectedRootCause = + KeeperException.create(KeeperException.Code.NOTEMPTY, namespacedCounterNodePath); + assertThatThrownBy(() -> counter.shutdown(JobStatus.FINISHED).get()) + .as( + "The shutdown should fail because of a child node being present and the shutdown not performing an explicit recursive deletion.") + .isInstanceOf(ExecutionException.class) + .extracting(FlinkAssertions::chainOfCauses, FlinkAssertions.STREAM_THROWABLE) + .anySatisfy( + cause -> + assertThat(cause) + .isInstanceOf(expectedRootCause.getClass()) + .hasMessage(expectedRootCause.getMessage())); + + client.delete().forPath(childNodePath); + counter.shutdown(JobStatus.FINISHED).join(); + + assertThat(client.checkExists().forPath(counterNodePath)) + .as( + "A retry of the shutdown should have worked now after the root cause was resolved.") + .isNull(); + } + /** Tests that counter node is NOT removed from ZooKeeper after suspend. */ @Test public void testSuspendKeepsState() throws Exception { @@ -75,13 +145,14 @@ public void testSuspendKeepsState() throws Exception { CuratorFramework client = zookeeper.getClient(); assertThat(client.checkExists().forPath(counter.getPath())).isNotNull(); - counter.shutdown(JobStatus.SUSPENDED); + counter.shutdown(JobStatus.SUSPENDED).join(); assertThat(client.checkExists().forPath(counter.getPath())).isNotNull(); } @Override protected ZooKeeperCheckpointIDCounter createCheckpointIdCounter() throws Exception { return new ZooKeeperCheckpointIDCounter( - zookeeper.getClient(), new DefaultLastStateConnectionStateListener()); + ZooKeeperUtils.useNamespaceAndEnsurePath(zookeeper.getClient(), "/"), + new DefaultLastStateConnectionStateListener()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java index a63a7527ae896..387d6679df685 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultSchedulerTest.java @@ -107,6 +107,7 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; import java.time.Duration; import java.util.ArrayList; @@ -1608,7 +1609,8 @@ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws Except throw new RuntimeException(e); } }, - executorService); + executorService, + log); } finally { executorService.shutdownNow(); } @@ -1620,7 +1622,8 @@ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws Except */ public static void doTestCheckpointCleanerIsClosedAfterCheckpointServices( BiFunction schedulerFactory, - ScheduledExecutorService executorService) + ScheduledExecutorService executorService, + Logger logger) throws Exception { final CountDownLatch checkpointServicesShutdownBlocked = new CountDownLatch(1); final CountDownLatch cleanerClosed = new CountDownLatch(1); @@ -1638,9 +1641,17 @@ public void shutdown(JobStatus jobStatus, CheckpointsCleaner checkpointsCleaner) new StandaloneCheckpointIDCounter() { @Override - public void shutdown(JobStatus jobStatus) throws Exception { - checkpointServicesShutdownBlocked.await(); - super.shutdown(jobStatus); + public CompletableFuture shutdown(JobStatus jobStatus) { + try { + checkpointServicesShutdownBlocked.await(); + } catch (InterruptedException e) { + logger.error( + "An error occurred while executing waiting for the CheckpointServices shutdown.", + e); + Thread.currentThread().interrupt(); + } + + return super.shutdown(jobStatus); } }; final CheckpointsCleaner checkpointsCleaner = diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java index e673a288b2c48..6e6f6d3d60036 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerTest.java @@ -1462,7 +1462,8 @@ public void testCheckpointCleanerIsClosedAfterCheckpointServices() throws Except throw new RuntimeException(e); } }, - executorService); + executorService, + log); } finally { executorService.shutdownNow(); } From 588ac8bce2aedf20f388fbce1a196c3ec2bd4b5e Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 21 Mar 2022 15:18:17 +0100 Subject: [PATCH 087/258] [hotfix][runtime] Adds missing configuration parameter to cleanup error log message --- .../flink/runtime/dispatcher/Dispatcher.java | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 68f387dad1a60..5150244134964 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -24,6 +24,8 @@ import org.apache.flink.api.common.operators.ResourceSpec; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.BlobServerOptions; +import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; @@ -620,19 +622,8 @@ private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionTy cleanupJobState -> removeJob(jobId, cleanupJobState) .exceptionally( - throwable -> { - log.warn( - "The cleanup of job {} failed. The job's artifacts in '{}' and its JobResultStore entry in '{}' needs to be cleaned manually.", - jobId, - configuration.get( - HighAvailabilityOptions - .HA_STORAGE_PATH), - configuration.get( - JobResultStoreOptions - .STORAGE_PATH), - throwable); - return null; - })); + throwable -> + logCleanupErrorWarning(jobId, throwable))); FutureUtils.handleUncaughtException( jobTerminationFuture, @@ -640,6 +631,19 @@ private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionTy registerJobManagerRunnerTerminationFuture(jobId, jobTerminationFuture); } + @Nullable + private Void logCleanupErrorWarning(JobID jobId, Throwable cleanupError) { + log.warn( + "The cleanup of job {} failed. The job's artifacts in the different directories ('{}', '{}', '{}') and its JobResultStore entry in '{}' (in HA mode) should be checked for manual cleanup.", + jobId, + configuration.get(HighAvailabilityOptions.HA_STORAGE_PATH), + configuration.get(BlobServerOptions.STORAGE_DIRECTORY), + configuration.get(CheckpointingOptions.CHECKPOINTS_DIRECTORY), + configuration.get(JobResultStoreOptions.STORAGE_PATH), + cleanupError); + return null; + } + private CleanupJobState handleJobManagerRunnerResult( JobManagerRunnerResult jobManagerRunnerResult, ExecutionType executionType) { if (jobManagerRunnerResult.isInitializationFailure() From 6e5240bd7811cac48e0ce66b792f76c97de1b57c Mon Sep 17 00:00:00 2001 From: wangfeifan Date: Wed, 23 Mar 2022 00:57:47 +0800 Subject: [PATCH 088/258] [FLINK-26700][docs] Document restore mode in chinese Co-authored-by: Yun Tang --- docs/content.zh/docs/ops/state/savepoints.md | 47 +++++++++++++++++++- 1 file changed, 46 insertions(+), 1 deletion(-) diff --git a/docs/content.zh/docs/ops/state/savepoints.md b/docs/content.zh/docs/ops/state/savepoints.md index 51785b67e563d..d4450602f4d4f 100644 --- a/docs/content.zh/docs/ops/state/savepoints.md +++ b/docs/content.zh/docs/ops/state/savepoints.md @@ -157,10 +157,55 @@ $ bin/flink run -s :savepointPath [:runArgs] 默认情况下,resume 操作将尝试将 Savepoint 的所有状态映射回你要还原的程序。 如果删除了运算符,则可以通过 `--allowNonRestoredState`(short:`-n`)选项跳过无法映射到新程序的状态: +#### Restore 模式 + +`Restore 模式` 决定了在 restore 之后谁拥有Savepoint 或者 [externalized checkpoint]({{< ref "docs/ops/state/checkpoints" >}}/#resuming-from-a-retained-checkpoint)的文件的所有权。在这种语境下 Savepoint 和 externalized checkpoint 的行为相似。 +这里我们将它们都称为“快照”,除非另有明确说明。 + +如前所述,restore 模式决定了谁来接管我们从中恢复的快照文件的所有权。快照可被用户或者 Flink 自身拥有。如果快照归用户所有,Flink 不会删除其中的文件,而且 Flink 不能依赖该快照中文件的存在,因为它可能在 Flink 的控制之外被删除。 + +每种 restore 模式都有特定的用途。尽管如此,我们仍然认为默认的 *NO_CLAIM* 模式在大多数情况下是一个很好的折中方案,因为它在提供明确的所有权归属的同时只给恢复后第一个 checkpoint 带来较小的代价。 + +你可以通过如下方式指定 restore 模式: ```shell -$ bin/flink run -s :savepointPath -n [:runArgs] +$ bin/flink run -s :savepointPath -restoreMode :mode -n [:runArgs] ``` +**NO_CLAIM (默认的)** + +在 *NO_CLAIM* 模式下,Flink 不会接管快照的所有权。它会将快照的文件置于用户的控制之中,并且永远不会删除其中的任何文件。该模式下可以从同一个快照上启动多个作业。 + +为保证 Flink 不会依赖于该快照的任何文件,它会强制第一个(成功的) checkpoint 为全量 checkpoint 而不是增量的。这仅对`state.backend: rocksdb` 有影响,因为其他 backend 总是创建全量 checkpoint。 + +一旦第一个全量的 checkpoint 完成后,所有后续的 checkpoint 会照常创建。所以,一旦一个 checkpoint 成功制作,就可以删除原快照。在此之前不能删除原快照,因为没有任何完成的 checkpoint,Flink 会在故障时尝试从初始的快照恢复。 + +
+ {{< img src="/fig/restore-mode-no_claim.svg" alt="NO_CLAIM restore mode" width="70%" >}} +
+ +**CLAIM** + +另一个可选的模式是 *CLAIM* 模式。该模式下 Flink 将声称拥有快照的所有权,并且本质上将其作为 checkpoint 对待:控制其生命周期并且可能会在其永远不会被用于恢复的时候删除它。因此,手动删除快照和从同一个快照上启动两个作业都是不安全的。Flink 会保持[配置数量]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}/#state-checkpoints-num-retained)的 checkpoint。 + +
+ {{< img src="/fig/restore-mode-claim.svg" alt="CLAIM restore mode" width="70%" >}} +
+ +{{< hint info >}} +**注意:** +1. Retained checkpoints 被存储在 `//chk_` 这样的目录中。Flink 不会接管 `/` 目录的所有权,而只会接管 `chk_` 的所有权。Flink 不会删除旧作业的目录。 + +2. [Native](#savepoint-format) 格式支持增量的 RocksDB savepoints。对于这些 savepoints,Flink 将所有 SST 存储在 savepoints 目录中。这意味着这些 savepoints 是自包含和目录可移动的。然而,在 CLAIM 模式下恢复时,后续的 checkpoints 可能会复用一些 SST 文件,这反过来会阻止在 savepoints 被清理时删除 savepoints 目录。 Flink 之后运行期间可能会删除复用的SST 文件,但不会删除 savepoints 目录。因此,如果在 CLAIM 模式下恢复,Flink 可能会留下一个空的 savepoints 目录。 +{{< /hint >}} + +**LEGACY** + +Legacy 模式是 Flink 在 1.15 之前的工作方式。该模式下 Flink 永远不会删除初始恢复的 checkpoint。同时,用户也不清楚是否可以删除它。导致该的问题原因是, Flink 会在用来恢复的 checkpoint 之上创建增量的 checkpoint,因此后续的 checkpoint 都有可能会依赖于用于恢复的那个 checkpoint。总而言之,恢复的 checkpoint 的所有权没有明确的界定。 + +
+ {{< img src="/fig/restore-mode-legacy.svg" alt="LEGACY restore mode" width="70%" >}} +
+ ### 删除 Savepoint ```shell From 6ea644d463531e5309122dbcbc1dd681e24fd4ce Mon Sep 17 00:00:00 2001 From: wangfeifan Date: Wed, 23 Mar 2022 23:43:16 +0800 Subject: [PATCH 089/258] [hotfix][docs] correct checkpoint directory in restore mode document --- docs/content.zh/docs/ops/state/savepoints.md | 2 +- docs/content/docs/ops/state/savepoints.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/content.zh/docs/ops/state/savepoints.md b/docs/content.zh/docs/ops/state/savepoints.md index d4450602f4d4f..87a2890ad03dc 100644 --- a/docs/content.zh/docs/ops/state/savepoints.md +++ b/docs/content.zh/docs/ops/state/savepoints.md @@ -193,7 +193,7 @@ $ bin/flink run -s :savepointPath -restoreMode :mode -n [:runArgs] {{< hint info >}} **注意:** -1. Retained checkpoints 被存储在 `//chk_` 这样的目录中。Flink 不会接管 `/` 目录的所有权,而只会接管 `chk_` 的所有权。Flink 不会删除旧作业的目录。 +1. Retained checkpoints 被存储在 `//chk-` 这样的目录中。Flink 不会接管 `/` 目录的所有权,而只会接管 `chk-` 的所有权。Flink 不会删除旧作业的目录。 2. [Native](#savepoint-format) 格式支持增量的 RocksDB savepoints。对于这些 savepoints,Flink 将所有 SST 存储在 savepoints 目录中。这意味着这些 savepoints 是自包含和目录可移动的。然而,在 CLAIM 模式下恢复时,后续的 checkpoints 可能会复用一些 SST 文件,这反过来会阻止在 savepoints 被清理时删除 savepoints 目录。 Flink 之后运行期间可能会删除复用的SST 文件,但不会删除 savepoints 目录。因此,如果在 CLAIM 模式下恢复,Flink 可能会留下一个空的 savepoints 目录。 {{< /hint >}} diff --git a/docs/content/docs/ops/state/savepoints.md b/docs/content/docs/ops/state/savepoints.md index 3058b83da5b53..a24c4db4d941a 100644 --- a/docs/content/docs/ops/state/savepoints.md +++ b/docs/content/docs/ops/state/savepoints.md @@ -256,8 +256,8 @@ of checkpoints. {{< hint info >}} **Attention:** -1. Retained checkpoints are stored in a path like `//chk_`. Flink does not -take ownership of the `/` directory, but only the `chk_`. The directory +1. Retained checkpoints are stored in a path like `//chk-`. Flink does not +take ownership of the `/` directory, but only the `chk-`. The directory of the old job will not be deleted by Flink 2. [Native](#savepoint-format) format supports incremental RocksDB savepoints. For those savepoints Flink puts all From 757869bf306336e6fe34140913a62cbe676b3753 Mon Sep 17 00:00:00 2001 From: Fabian Paul Date: Thu, 17 Mar 2022 11:53:21 +0100 Subject: [PATCH 090/258] [hotfix][elasticsearch] Make TestEmitter#createIndexRequest private --- .../apache/flink/connector/elasticsearch/sink/TestEmitter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java index 49943b23bedd0..2cfc5eb4bcc1d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/TestEmitter.java @@ -58,7 +58,7 @@ public void emit( indexer.add(createIndexRequest(element)); } - public IndexRequest createIndexRequest(Tuple2 element) { + private IndexRequest createIndexRequest(Tuple2 element) { Map document = new HashMap<>(); document.put(dataFieldName, element.f1); try { From 328c0f360834901191a03c59076cd0c41b1745eb Mon Sep 17 00:00:00 2001 From: Fabian Paul Date: Mon, 14 Mar 2022 15:24:35 +0100 Subject: [PATCH 091/258] [FLINK-26633][elasticsearch] Add recordSend metric to elasticsearch sink --- .../sink/ElasticsearchWriter.java | 11 ++- .../sink/ElasticsearchWriterITCase.java | 80 ++++++++++++++++++- 2 files changed, 88 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java index 7ea3dd87bd5a4..fa8ed67cb0785 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriter.java @@ -103,7 +103,7 @@ class ElasticsearchWriter implements SinkWriter { RestClient.builder(hosts.toArray(new HttpHost[0])), networkClientConfig)); this.bulkProcessor = createBulkProcessor(bulkProcessorBuilderFactory, bulkProcessorConfig); - this.requestIndexer = new DefaultRequestIndexer(); + this.requestIndexer = new DefaultRequestIndexer(metricGroup.getNumRecordsSendCounter()); checkNotNull(metricGroup); metricGroup.setCurrentSendTimeGauge(() -> ackTime - lastSendTime); this.numBytesOutCounter = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); @@ -294,9 +294,16 @@ private boolean isClosed() { private class DefaultRequestIndexer implements RequestIndexer { + private final Counter numRecordsSendCounter; + + public DefaultRequestIndexer(Counter numRecordsSendCounter) { + this.numRecordsSendCounter = checkNotNull(numRecordsSendCounter); + } + @Override public void add(DeleteRequest... deleteRequests) { for (final DeleteRequest deleteRequest : deleteRequests) { + numRecordsSendCounter.inc(); pendingActions++; bulkProcessor.add(deleteRequest); } @@ -305,6 +312,7 @@ public void add(DeleteRequest... deleteRequests) { @Override public void add(IndexRequest... indexRequests) { for (final IndexRequest indexRequest : indexRequests) { + numRecordsSendCounter.inc(); pendingActions++; bulkProcessor.add(indexRequest); } @@ -313,6 +321,7 @@ public void add(IndexRequest... indexRequests) { @Override public void add(UpdateRequest... updateRequests) { for (final UpdateRequest updateRequest : updateRequests) { + numRecordsSendCounter.inc(); pendingActions++; bulkProcessor.add(updateRequest); } diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java index d0132a0bb2817..3db12a400d09d 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/sink/ElasticsearchWriterITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.elasticsearch.sink; import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.connector.sink2.SinkWriter; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.elasticsearch.ElasticsearchUtil; import org.apache.flink.metrics.Counter; @@ -25,6 +26,7 @@ import org.apache.flink.metrics.groups.OperatorIOMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.metrics.testutils.MetricListener; +import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.DockerImageVersions; @@ -38,8 +40,11 @@ import org.elasticsearch.action.bulk.BulkProcessor; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestHighLevelClient; @@ -58,8 +63,11 @@ import java.io.IOException; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import java.util.Optional; +import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.DOCUMENT_TYPE; import static org.apache.flink.connector.elasticsearch.sink.TestClientBase.buildMessage; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThan; @@ -204,6 +212,30 @@ void testIncrementByteOutMetric() throws Exception { } } + @Test + void testIncrementRecordsSendMetric() throws Exception { + final String index = "test-inc-records-send"; + final int flushAfterNActions = 2; + final BulkProcessorConfig bulkProcessorConfig = + new BulkProcessorConfig(flushAfterNActions, -1, -1, FlushBackoffType.NONE, 0, 0); + + try (final ElasticsearchWriter> writer = + createWriter(index, false, bulkProcessorConfig)) { + final Optional recordsSend = + metricListener.getCounter(MetricNames.NUM_RECORDS_SEND); + writer.write(Tuple2.of(1, buildMessage(1)), null); + // Update existing index + writer.write(Tuple2.of(1, "u" + buildMessage(2)), null); + // Delete index + writer.write(Tuple2.of(1, "d" + buildMessage(3)), null); + + writer.blockingFlushAllActions(); + + assertTrue(recordsSend.isPresent()); + assertEquals(recordsSend.get().getCount(), 3L); + } + } + @Test void testCurrentSendTime() throws Exception { final String index = "test-current-send-time"; @@ -239,9 +271,9 @@ private ElasticsearchWriter> createWriter( boolean flushOnCheckpoint, BulkProcessorConfig bulkProcessorConfig, SinkWriterMetricGroup metricGroup) { - return new ElasticsearchWriter>( + return new ElasticsearchWriter<>( Collections.singletonList(HttpHost.create(ES_CONTAINER.getHttpHostAddress())), - TestEmitter.jsonEmitter(index, context.getDataFieldName()), + new UpdatingEmitter(index, context.getDataFieldName()), flushOnCheckpoint, bulkProcessorConfig, new TestBulkProcessorBuilderFactory(), @@ -310,6 +342,50 @@ public void accept( } } + private static class UpdatingEmitter implements ElasticsearchEmitter> { + + private final String dataFieldName; + private final String index; + + UpdatingEmitter(String index, String dataFieldName) { + this.index = index; + this.dataFieldName = dataFieldName; + } + + @Override + public void emit( + Tuple2 element, + SinkWriter.Context context, + RequestIndexer indexer) { + + Map document = new HashMap<>(); + document.put(dataFieldName, element.f1); + + final char action = element.f1.charAt(0); + final String id = element.f0.toString(); + switch (action) { + case 'd': + { + indexer.add(new DeleteRequest(index).id(id)); + break; + } + case 'u': + { + indexer.add(new UpdateRequest().index(index).id(id).doc(document)); + break; + } + default: + { + indexer.add( + new IndexRequest(index) + .id(id) + .type(DOCUMENT_TYPE) + .source(document)); + } + } + } + } + private static class TestClient extends TestClientBase { TestClient(RestHighLevelClient client) { From 94c679ef99357a66fc7609ff6f2f0a91caa4476a Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 24 Mar 2022 20:34:09 +0800 Subject: [PATCH 092/258] [FLINK-26846][python] Fix the gauge metric This closes #19226. --- .../apache/flink/python/metric/FlinkMetricContainer.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java b/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java index 96e127faf2782..a184c39c6211e 100644 --- a/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java +++ b/flink-python/src/main/java/org/apache/flink/python/metric/FlinkMetricContainer.java @@ -33,7 +33,6 @@ import org.apache.beam.model.pipeline.v1.MetricsApi.MonitoringInfo; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; -import org.apache.beam.runners.core.metrics.MonitoringInfoConstants; import org.apache.beam.runners.core.metrics.MonitoringInfoMetricName; import org.apache.beam.sdk.metrics.DistributionResult; import org.apache.beam.sdk.metrics.GaugeResult; @@ -106,10 +105,7 @@ private boolean isUserMetric(MetricResult metricResult) { MetricName metricName = metricResult.getKey().metricName(); if (metricName instanceof MonitoringInfoMetricName) { String urn = ((MonitoringInfoMetricName) metricName).getUrn(); - return urn.contains(MonitoringInfoConstants.Urns.USER_SUM_INT64) - || urn.contains(MonitoringInfoConstants.Urns.USER_SUM_DOUBLE) - || urn.contains(MonitoringInfoConstants.Urns.USER_DISTRIBUTION_DOUBLE) - || urn.contains(MonitoringInfoConstants.Urns.USER_DISTRIBUTION_INT64); + return urn.startsWith("beam:metric:user"); } return false; } From 9b9a825d1fb5093184c8b8714a11ede5092ed901 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Wed, 23 Mar 2022 14:50:38 +0800 Subject: [PATCH 093/258] [FLINK-26638][connectors/elasticsearch] update the Chinese document accordingly This closes #19211. --- .../docs/connectors/table/elasticsearch.md | 35 ++++++++++++------- .../docs/connectors/table/elasticsearch.md | 1 + 2 files changed, 23 insertions(+), 13 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/elasticsearch.md b/docs/content.zh/docs/connectors/table/elasticsearch.md index bb2ca45572db9..3395eda13703f 100644 --- a/docs/content.zh/docs/connectors/table/elasticsearch.md +++ b/docs/content.zh/docs/connectors/table/elasticsearch.md @@ -130,11 +130,27 @@ CREATE TABLE myUserTable ( 用于连接 Elasticsearch 实例的密码。如果配置了username,则此选项也必须配置为非空字符串。 -
sink.delivery-guarantee
- 可选 - NONE +
failure-handler
+ optional + fail String - 提交时可选的传输保障策略。有效值为 NONE 或者 AT_LEAST_ONCE。 + 对 Elasticsearch 请求失败情况下的失败处理策略。有效策略为: +
    +
  • fail:如果请求失败并因此导致作业失败,则抛出异常。
  • +
  • ignore:忽略失败并放弃请求。
  • +
  • retry-rejected:重新添加由于队列容量饱和而失败的请求。
  • +
  • 自定义类名称:使用 ActionRequestFailureHandler 的子类进行失败处理。
  • +
+ + + +
sink.flush-on-checkpoint
+ optional + true + Boolean + 在进行 checkpoint 时是否保证刷出缓冲区中的数据。如果关闭这一选项,在进行checkpoint时 sink 将不再为所有进行 + 中的请求等待 Elasticsearch 的执行完成确认。因此,在这种情况下 sink 将不对至少一次的请求的一致性提供任何保证。 +
sink.bulk-flush.max-actions
@@ -166,11 +182,11 @@ CREATE TABLE myUserTable (
sink.bulk-flush.backoff.strategy
可选 - NONE + DISABLED String 指定在由于临时请求错误导致任何 flush 操作失败时如何执行重试。有效策略为:
    -
  • NONE:不执行重试,即第一次请求错误后失败。
  • +
  • DISABLED:不执行重试,即第一次请求错误后失败。
  • CONSTANT:等待重试之间的回退延迟。
  • EXPONENTIAL:先等待回退延迟,然后在重试之间指数递增。
@@ -190,13 +206,6 @@ CREATE TABLE myUserTable ( Duration 每次退避尝试之间的延迟。对于 CONSTANT 退避策略,该值是每次重试之间的延迟。对于 EXPONENTIAL 退避策略,该值是初始的延迟。 - -
sink.parallelism
- 可选 - (none) - Integer - 定义 Elasticsearch sink 算子的并行度。默认情况下,并行度由框架定义为与上游串联的算子相同。 -
connection.path-prefix
可选 diff --git a/docs/content/docs/connectors/table/elasticsearch.md b/docs/content/docs/connectors/table/elasticsearch.md index c205313331fc7..93a50ddbcec65 100644 --- a/docs/content/docs/connectors/table/elasticsearch.md +++ b/docs/content/docs/connectors/table/elasticsearch.md @@ -157,6 +157,7 @@ Connector Options
sink.flush-on-checkpoint
optional + true Boolean Flush on checkpoint or not. When disabled, a sink will not wait for all pending action requests From cbf6eecb0244eb87c769646bd73c76811574a030 Mon Sep 17 00:00:00 2001 From: Juntao Hu Date: Thu, 24 Mar 2022 21:10:48 -0700 Subject: [PATCH 094/258] [FLINK-26855][python] Fix Sphinx check failure caused by Jinja2 dependency (#19238) --- flink-python/dev/lint-python.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-python/dev/lint-python.sh b/flink-python/dev/lint-python.sh index c5dc80dbd4f19..789720208ec72 100755 --- a/flink-python/dev/lint-python.sh +++ b/flink-python/dev/lint-python.sh @@ -322,7 +322,7 @@ function install_sphinx() { fi fi - $CURRENT_DIR/install_command.sh -q Sphinx==2.4.4 Docutils==0.17.1 2>&1 >/dev/null + $CURRENT_DIR/install_command.sh -q Sphinx==2.4.4 Docutils==0.17.1 "Jinja2<3.1.0" 2>&1 >/dev/null if [ $? -ne 0 ]; then echo "pip install sphinx failed \ please try to exec the script again.\ From 3c88f887de26f58cff27c87931d7a2f6928c81da Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Thu, 24 Mar 2022 10:47:26 +0100 Subject: [PATCH 095/258] [FLINK-26809][tests] Assert histogram state after stopping the uploader There is a race condition in ChangelogStorageMetricsTest.testAttemptsPerUpload: - the assertion is made as soon as upload (future) is completed - the histogram is updated after completing the upload (on success) Moving assertion out of try/close block solves the problem. --- .../flink/changelog/fs/ChangelogStorageMetricsTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/ChangelogStorageMetricsTest.java b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/ChangelogStorageMetricsTest.java index 3444139a212c0..9c031175f74ee 100644 --- a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/ChangelogStorageMetricsTest.java +++ b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/ChangelogStorageMetricsTest.java @@ -201,12 +201,12 @@ public void testAttemptsPerUpload() throws Exception { writer.append(0, new byte[] {0, 1, 2, 3}); writer.persist(from).get(); } - HistogramStatistics histogram = metrics.getAttemptsPerUpload().getStatistics(); - assertEquals(maxAttempts, histogram.getMin()); - assertEquals(maxAttempts, histogram.getMax()); } finally { storage.close(); } + HistogramStatistics histogram = metrics.getAttemptsPerUpload().getStatistics(); + assertEquals(maxAttempts, histogram.getMin()); + assertEquals(maxAttempts, histogram.getMax()); } @Test From 608f5e1761b4fc011aec6cc958ef375d6992814c Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 24 Mar 2022 21:04:05 +0800 Subject: [PATCH 096/258] [FLINK-26847][python] Ensure command line option '-py' works in YARN application mode This closes #19227. --- .../org/apache/flink/client/python/PythonDriver.java | 8 ++++++-- .../python/PythonDriverOptionsParserFactory.java | 6 ++++++ .../apache/flink/client/python/PythonDriverTest.java | 11 ++++++----- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java index 57b1eb2f32d51..a25ee91d27069 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriver.java @@ -154,10 +154,14 @@ public static void main(String[] args) throws Throwable { */ static List constructPythonCommands(final PythonDriverOptions pythonDriverOptions) { final List commands = new ArrayList<>(); + commands.add("-m"); if (pythonDriverOptions.getEntryPointScript().isPresent()) { - commands.add(pythonDriverOptions.getEntryPointScript().get()); + String pythonFileName = pythonDriverOptions.getEntryPointScript().get(); + commands.add( + pythonFileName.substring( + pythonFileName.lastIndexOf(File.separator) + 1, + pythonFileName.lastIndexOf(".py"))); } else { - commands.add("-m"); commands.add(pythonDriverOptions.getEntryPointModule()); } commands.addAll(pythonDriverOptions.getProgramArgs()); diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptionsParserFactory.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptionsParserFactory.java index 6e5663e129eb6..ded5577642aa3 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptionsParserFactory.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptionsParserFactory.java @@ -54,6 +54,12 @@ public PythonDriverOptions createResult(@Nonnull CommandLine commandLine) throw new FlinkParseException("Cannot use options -py and -pym simultaneously."); } else if (commandLine.hasOption(PY_OPTION.getOpt())) { entryPointScript = commandLine.getOptionValue(PY_OPTION.getOpt()); + if (!entryPointScript.endsWith(".py")) { + throw new FlinkParseException( + String.format( + "It only accepts Python file which ends with '.py' for option '-py', got '%s'.", + entryPointScript)); + } } else if (commandLine.hasOption(PYMODULE_OPTION.getOpt())) { entryPointModule = commandLine.getOptionValue(PYMODULE_OPTION.getOpt()); } else { diff --git a/flink-python/src/test/java/org/apache/flink/client/python/PythonDriverTest.java b/flink-python/src/test/java/org/apache/flink/client/python/PythonDriverTest.java index 773353d71afe0..fe46868618224 100644 --- a/flink-python/src/test/java/org/apache/flink/client/python/PythonDriverTest.java +++ b/flink-python/src/test/java/org/apache/flink/client/python/PythonDriverTest.java @@ -65,11 +65,12 @@ public void testConstructCommandsWithEntryPointScript() { args.add("--input"); args.add("in.txt"); - PythonDriverOptions pythonDriverOptions = new PythonDriverOptions(null, "xxx", args); + PythonDriverOptions pythonDriverOptions = new PythonDriverOptions(null, "xxx.py", args); List commands = PythonDriver.constructPythonCommands(pythonDriverOptions); - Assert.assertEquals(3, commands.size()); - Assert.assertEquals(commands.get(0), "xxx"); - Assert.assertEquals(commands.get(1), "--input"); - Assert.assertEquals(commands.get(2), "in.txt"); + Assert.assertEquals(4, commands.size()); + Assert.assertEquals(commands.get(0), "-m"); + Assert.assertEquals(commands.get(1), "xxx"); + Assert.assertEquals(commands.get(2), "--input"); + Assert.assertEquals(commands.get(3), "in.txt"); } } From 1d8e2d5cce033b0b6a3c92a175c1381f3d4d4d77 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Thu, 24 Feb 2022 11:05:19 +0200 Subject: [PATCH 097/258] [hotfix][core] Add method to get all ConfigOptions from an "Options" class This is needed for generating a `ReadableConfig` with all consumed options by a give `ExecNode`s to be persisted in JSON plan. (cherry picked from commit b0b92cbbc07ebc944fa121c8faa49dcb6b42673d) --- .../flink/configuration/ConfigUtils.java | 23 +++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java index 1aa36d135a27e..1c275bc02e5ae 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigUtils.java @@ -23,11 +23,14 @@ import javax.annotation.Nullable; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -132,5 +135,25 @@ public static List decodeListFromConfig( return result; } + public static Set> getAllConfigOptions(Class configOptionsClass) + throws IllegalStateException { + final Set> options = new HashSet<>(); + final Field[] fields = configOptionsClass.getDeclaredFields(); + for (Field field : fields) { + if (field.getType() == ConfigOption.class) { + try { + options.add((ConfigOption) field.get(configOptionsClass)); + } catch (IllegalAccessException e) { + throw new IllegalStateException( + "The config option definition for field " + + field.getName() + + " is not accessible.", + e); + } + } + } + return options; + } + private ConfigUtils() {} } From de771725772f3834a8ec24de50a7abd516b03dfc Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Thu, 24 Feb 2022 12:10:32 +0200 Subject: [PATCH 098/258] [hotfix][table-planner] Remove consumed options not affecting the topology. Remove `table.exec.source.idle-timeout` and `table.exec.state.ttl`. (cherry picked from commit b8cdb8d6b6b879faa7f0454ba9145af8190e3c80) --- .../nodes/exec/stream/StreamExecChangelogNormalize.java | 1 - .../plan/nodes/exec/stream/StreamExecDeduplicate.java | 1 - .../nodes/exec/stream/StreamExecGlobalGroupAggregate.java | 6 +----- .../plan/nodes/exec/stream/StreamExecGroupAggregate.java | 6 +----- .../nodes/exec/stream/StreamExecGroupWindowAggregate.java | 1 - .../exec/stream/StreamExecIncrementalGroupAggregate.java | 6 +----- .../planner/plan/nodes/exec/stream/StreamExecLimit.java | 2 +- .../plan/nodes/exec/stream/StreamExecOverAggregate.java | 1 - .../planner/plan/nodes/exec/stream/StreamExecRank.java | 2 +- .../planner/plan/nodes/exec/stream/StreamExecSink.java | 1 - .../planner/plan/nodes/exec/stream/StreamExecSortLimit.java | 2 +- .../plan/nodes/exec/stream/StreamExecTemporalJoin.java | 1 - .../plan/nodes/exec/stream/StreamExecWatermarkAssigner.java | 1 - 13 files changed, 6 insertions(+), 25 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java index c3d0677ed4f12..5c82e812698cd 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java @@ -64,7 +64,6 @@ name = "stream-exec-changelog-normalize", version = 1, consumedOptions = { - "table.exec.state.ttl", "table.exec.mini-batch.enabled", "table.exec.mini-batch.size", }, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java index c4888835661ec..c5bf069867c55 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java @@ -74,7 +74,6 @@ name = "stream-exec-deduplicate", version = 1, consumedOptions = { - "table.exec.state.ttl", "table.exec.mini-batch.enabled", "table.exec.mini-batch.size", "table.exec.deduplicate.insert-update-after-sensitive-enabled", diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java index 9e68aa53864dd..5d1b1db59620c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java @@ -74,11 +74,7 @@ @ExecNodeMetadata( name = "stream-exec-global-group-aggregate", version = 1, - consumedOptions = { - "table.exec.state.ttl", - "table.exec.mini-batch.enabled", - "table.exec.mini-batch.size" - }, + consumedOptions = {"table.exec.mini-batch.enabled", "table.exec.mini-batch.size"}, producedTransformations = StreamExecGlobalGroupAggregate.GLOBAL_GROUP_AGGREGATE_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java index c6d11e6090dcd..c698bd93056f0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java @@ -73,11 +73,7 @@ @ExecNodeMetadata( name = "stream-exec-group-aggregate", version = 1, - consumedOptions = { - "table.exec.state.ttl", - "table.exec.mini-batch.enabled", - "table.exec.mini-batch.size" - }, + consumedOptions = {"table.exec.mini-batch.enabled", "table.exec.mini-batch.size"}, producedTransformations = StreamExecGroupAggregate.GROUP_AGGREGATE_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java index 3944e4bff0db8..3f96de3a5f65c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java @@ -105,7 +105,6 @@ version = 1, consumedOptions = { "table.local-time-zone", - "table.exec.state.ttl", "table.exec.mini-batch.enabled", "table.exec.mini-batch.size" }, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java index 93228992b24a4..5f86176719e95 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java @@ -62,11 +62,7 @@ @ExecNodeMetadata( name = "stream-exec-incremental-group-aggregate", version = 1, - consumedOptions = { - "table.exec.state.ttl", - "table.exec.mini-batch.enabled", - "table.exec.mini-batch.size" - }, + consumedOptions = {"table.exec.mini-batch.enabled", "table.exec.mini-batch.size"}, producedTransformations = StreamExecIncrementalGroupAggregate.INCREMENTAL_GROUP_AGGREGATE_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java index 8f255f9836b9f..3e4bb09ba6565 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java @@ -45,7 +45,7 @@ @ExecNodeMetadata( name = "stream-exec-limit", version = 1, - consumedOptions = {"table.exec.rank.topn-cache-size", "table.exec.state.ttl"}, + consumedOptions = {"table.exec.rank.topn-cache-size"}, producedTransformations = StreamExecRank.RANK_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java index 1eb874c078e90..a183b21bcab69 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java @@ -84,7 +84,6 @@ @ExecNodeMetadata( name = "stream-exec-over-aggregate", version = 1, - consumedOptions = {"table.exec.state.ttl"}, producedTransformations = StreamExecOverAggregate.OVER_AGGREGATE_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java index bcb3f9c8f3430..247e735436bf0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java @@ -75,7 +75,7 @@ @ExecNodeMetadata( name = "stream-exec-rank", version = 1, - consumedOptions = {"table.exec.state.ttl", "table.exec.rank.topn-cache-size"}, + consumedOptions = {"table.exec.rank.topn-cache-size"}, producedTransformations = StreamExecRank.RANK_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java index 23341599f3db9..1d19b456eb94f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java @@ -55,7 +55,6 @@ name = "stream-exec-sink", version = 1, consumedOptions = { - "table.exec.state.ttl", "table.exec.sink.not-null-enforcer", "table.exec.sink.type-length-enforcer", "table.exec.sink.upsert-materialize", diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java index 3c2cf3acf89fe..ab3f5b7bcdba4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java @@ -44,7 +44,7 @@ @ExecNodeMetadata( name = "stream-exec-sort-limit", version = 1, - consumedOptions = {"table.exec.state.ttl", "table.exec.rank.topn-cache-size"}, + consumedOptions = {"table.exec.rank.topn-cache-size"}, producedTransformations = StreamExecRank.RANK_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java index 8b32e1a9b8f26..566e4dcd4b401 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java @@ -70,7 +70,6 @@ @ExecNodeMetadata( name = "stream-exec-temporal-join", version = 1, - consumedOptions = "table.exec.state.ttl", producedTransformations = StreamExecTemporalJoin.TEMPORAL_JOIN_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java index a27c3ff6149ee..177ea4823cc01 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java @@ -55,7 +55,6 @@ @ExecNodeMetadata( name = "stream-exec-watermark-assigner", version = 1, - consumedOptions = "table.exec.source.idle-timeout", producedTransformations = StreamExecWatermarkAssigner.WATERMARK_ASSIGNER_TRANSFORMATION, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) From 00c1439bc3cc277fc37e39d084c4f1bc14ecf5b7 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Fri, 25 Feb 2022 11:21:05 +0200 Subject: [PATCH 099/258] [FLINK-26075][table-planner] Persist node configuration to JSON plan - Add a new JSON field to the plan called configuration which includes all the options used by the ExecNode along with their values. - Use the previously introduced ExecNodeConfig class to merge the configuration deserialized from the JSON plan with the Planner configuration. (cherry picked from commit bc11450bbf2519924e354af7900174e7bd6c33e3) --- .../planner/plan/nodes/exec/ExecNode.java | 1 + .../planner/plan/nodes/exec/ExecNodeBase.java | 21 +- .../plan/nodes/exec/ExecNodeConfig.java | 8 +- .../plan/nodes/exec/ExecNodeContext.java | 30 +- .../batch/BatchExecBoundedStreamScan.java | 3 + .../plan/nodes/exec/batch/BatchExecCalc.java | 3 + .../nodes/exec/batch/BatchExecCorrelate.java | 3 + .../nodes/exec/batch/BatchExecExchange.java | 8 +- .../nodes/exec/batch/BatchExecExpand.java | 3 + .../exec/batch/BatchExecHashAggregate.java | 3 + .../nodes/exec/batch/BatchExecHashJoin.java | 3 + .../batch/BatchExecHashWindowAggregate.java | 3 + .../nodes/exec/batch/BatchExecLegacySink.java | 3 + .../batch/BatchExecLegacyTableSourceScan.java | 4 + .../plan/nodes/exec/batch/BatchExecLimit.java | 3 + .../nodes/exec/batch/BatchExecLookupJoin.java | 3 + .../exec/batch/BatchExecMultipleInput.java | 3 + .../exec/batch/BatchExecNestedLoopJoin.java | 3 + .../exec/batch/BatchExecOverAggregate.java | 3 + .../batch/BatchExecOverAggregateBase.java | 10 +- .../nodes/exec/batch/BatchExecPythonCalc.java | 3 + .../exec/batch/BatchExecPythonCorrelate.java | 3 + .../batch/BatchExecPythonGroupAggregate.java | 4 + .../BatchExecPythonGroupWindowAggregate.java | 4 + .../batch/BatchExecPythonOverAggregate.java | 3 + .../plan/nodes/exec/batch/BatchExecRank.java | 3 + .../plan/nodes/exec/batch/BatchExecSink.java | 3 + .../plan/nodes/exec/batch/BatchExecSort.java | 3 + .../exec/batch/BatchExecSortAggregate.java | 3 + .../nodes/exec/batch/BatchExecSortLimit.java | 3 + .../exec/batch/BatchExecSortMergeJoin.java | 3 + .../batch/BatchExecSortWindowAggregate.java | 3 + .../exec/batch/BatchExecTableSourceScan.java | 7 +- .../plan/nodes/exec/batch/BatchExecUnion.java | 7 +- .../nodes/exec/batch/BatchExecValues.java | 8 +- .../batch/BatchExecWindowTableFunction.java | 3 + .../nodes/exec/common/CommonExecCalc.java | 4 +- .../exec/common/CommonExecCorrelate.java | 4 +- .../nodes/exec/common/CommonExecExchange.java | 4 +- .../nodes/exec/common/CommonExecExpand.java | 4 +- .../exec/common/CommonExecLegacySink.java | 10 +- .../CommonExecLegacyTableSourceScan.java | 4 +- .../exec/common/CommonExecLookupJoin.java | 4 +- .../exec/common/CommonExecPythonCalc.java | 4 +- .../common/CommonExecPythonCorrelate.java | 4 +- .../nodes/exec/common/CommonExecSink.java | 3 +- .../common/CommonExecTableSourceScan.java | 3 +- .../nodes/exec/common/CommonExecUnion.java | 4 +- .../nodes/exec/common/CommonExecValues.java | 4 +- .../common/CommonExecWindowTableFunction.java | 4 +- .../ForwardHashExchangeProcessor.java | 32 ++- .../MultipleInputNodeCreationProcessor.java | 38 ++- .../utils/InputPriorityConflictResolver.java | 19 +- .../serde/ConfigurationJsonDeserializer.java | 52 ++++ .../serde/ConfigurationJsonSerializer.java | 52 ++++ .../ConfigurationJsonSerializerFilter.java | 43 +++ .../plan/nodes/exec/serde/JsonSerdeUtil.java | 3 + .../exec/stream/StreamExecAggregateBase.java | 4 +- .../nodes/exec/stream/StreamExecCalc.java | 5 + .../stream/StreamExecChangelogNormalize.java | 6 +- .../exec/stream/StreamExecCorrelate.java | 5 + .../exec/stream/StreamExecDataStreamScan.java | 3 + .../exec/stream/StreamExecDeduplicate.java | 5 +- .../stream/StreamExecDropUpdateBefore.java | 10 +- .../nodes/exec/stream/StreamExecExchange.java | 11 +- .../nodes/exec/stream/StreamExecExpand.java | 5 + .../StreamExecGlobalGroupAggregate.java | 7 +- .../StreamExecGlobalWindowAggregate.java | 7 +- .../exec/stream/StreamExecGroupAggregate.java | 6 +- .../stream/StreamExecGroupTableAggregate.java | 4 + .../StreamExecGroupWindowAggregate.java | 6 +- .../StreamExecIncrementalGroupAggregate.java | 7 +- .../exec/stream/StreamExecIntervalJoin.java | 5 +- .../nodes/exec/stream/StreamExecJoin.java | 6 +- .../exec/stream/StreamExecLegacySink.java | 3 + .../StreamExecLegacyTableSourceScan.java | 4 + .../nodes/exec/stream/StreamExecLimit.java | 5 + .../stream/StreamExecLocalGroupAggregate.java | 7 +- .../StreamExecLocalWindowAggregate.java | 7 +- .../exec/stream/StreamExecLookupJoin.java | 5 + .../nodes/exec/stream/StreamExecMatch.java | 5 +- .../stream/StreamExecMiniBatchAssigner.java | 6 +- .../exec/stream/StreamExecMultipleInput.java | 7 +- .../exec/stream/StreamExecOverAggregate.java | 6 +- .../exec/stream/StreamExecPythonCalc.java | 6 +- .../stream/StreamExecPythonCorrelate.java | 5 + .../StreamExecPythonGroupAggregate.java | 7 +- .../StreamExecPythonGroupTableAggregate.java | 4 + .../StreamExecPythonGroupWindowAggregate.java | 10 +- .../stream/StreamExecPythonOverAggregate.java | 10 +- .../nodes/exec/stream/StreamExecRank.java | 6 +- .../nodes/exec/stream/StreamExecSink.java | 5 + .../nodes/exec/stream/StreamExecSort.java | 3 + .../exec/stream/StreamExecSortLimit.java | 5 + .../stream/StreamExecTableSourceScan.java | 10 +- .../exec/stream/StreamExecTemporalJoin.java | 6 +- .../exec/stream/StreamExecTemporalSort.java | 6 +- .../nodes/exec/stream/StreamExecUnion.java | 10 +- .../nodes/exec/stream/StreamExecValues.java | 11 +- .../stream/StreamExecWatermarkAssigner.java | 6 +- .../stream/StreamExecWindowAggregate.java | 6 +- .../stream/StreamExecWindowAggregateBase.java | 4 +- .../stream/StreamExecWindowDeduplicate.java | 6 +- .../exec/stream/StreamExecWindowJoin.java | 6 +- .../exec/stream/StreamExecWindowRank.java | 6 +- .../stream/StreamExecWindowTableFunction.java | 14 +- .../plan/utils/ExecNodeMetadataUtil.java | 79 ++++++ .../BatchPhysicalBoundedStreamScan.scala | 2 + .../physical/batch/BatchPhysicalCalc.scala | 4 +- .../batch/BatchPhysicalCorrelate.scala | 4 +- .../batch/BatchPhysicalExchange.scala | 1 + .../physical/batch/BatchPhysicalExpand.scala | 7 +- .../batch/BatchPhysicalHashAggregate.scala | 7 +- .../batch/BatchPhysicalHashJoin.scala | 25 +- .../BatchPhysicalHashWindowAggregate.scala | 5 +- .../batch/BatchPhysicalLegacySink.scala | 5 +- .../BatchPhysicalLegacyTableSourceScan.scala | 2 + .../physical/batch/BatchPhysicalLimit.scala | 16 +- .../BatchPhysicalLocalHashAggregate.scala | 7 +- ...atchPhysicalLocalHashWindowAggregate.scala | 5 +- .../BatchPhysicalLocalSortAggregate.scala | 7 +- ...atchPhysicalLocalSortWindowAggregate.scala | 5 +- .../batch/BatchPhysicalLookupJoin.scala | 2 + .../batch/BatchPhysicalNestedLoopJoin.scala | 5 +- .../batch/BatchPhysicalOverAggregate.scala | 5 +- .../batch/BatchPhysicalPythonCalc.scala | 2 + .../batch/BatchPhysicalPythonCorrelate.scala | 7 +- .../BatchPhysicalPythonGroupAggregate.scala | 7 +- ...chPhysicalPythonGroupWindowAggregate.scala | 5 +- .../BatchPhysicalPythonOverAggregate.scala | 5 +- .../physical/batch/BatchPhysicalRank.scala | 5 +- .../physical/batch/BatchPhysicalSink.scala | 8 +- .../physical/batch/BatchPhysicalSort.scala | 7 +- .../batch/BatchPhysicalSortAggregate.scala | 5 +- .../batch/BatchPhysicalSortLimit.scala | 7 +- .../batch/BatchPhysicalSortMergeJoin.scala | 5 +- .../BatchPhysicalSortWindowAggregate.scala | 5 +- .../batch/BatchPhysicalTableSourceScan.scala | 2 + .../physical/batch/BatchPhysicalUnion.scala | 7 +- .../physical/batch/BatchPhysicalValues.scala | 5 +- .../BatchPhysicalWindowTableFunction.scala | 5 +- .../physical/stream/StreamPhysicalCalc.scala | 2 + .../StreamPhysicalChangelogNormalize.scala | 2 + .../stream/StreamPhysicalCorrelate.scala | 4 +- .../stream/StreamPhysicalDataStreamScan.scala | 2 + .../stream/StreamPhysicalDeduplicate.scala | 7 +- .../StreamPhysicalDropUpdateBefore.scala | 7 +- .../stream/StreamPhysicalExchange.scala | 4 +- .../stream/StreamPhysicalExpand.scala | 7 +- .../StreamPhysicalGlobalGroupAggregate.scala | 7 +- .../StreamPhysicalGlobalWindowAggregate.scala | 5 +- .../stream/StreamPhysicalGroupAggregate.scala | 7 +- .../StreamPhysicalGroupTableAggregate.scala | 7 +- .../StreamPhysicalGroupWindowAggregate.scala | 5 +- ...eamPhysicalGroupWindowTableAggregate.scala | 5 +- ...eamPhysicalIncrementalGroupAggregate.scala | 7 +- .../stream/StreamPhysicalIntervalJoin.scala | 12 +- .../physical/stream/StreamPhysicalJoin.scala | 18 +- .../stream/StreamPhysicalLegacySink.scala | 5 +- .../StreamPhysicalLegacyTableSourceScan.scala | 2 + .../physical/stream/StreamPhysicalLimit.scala | 19 +- .../StreamPhysicalLocalGroupAggregate.scala | 7 +- .../StreamPhysicalLocalWindowAggregate.scala | 5 +- .../stream/StreamPhysicalLookupJoin.scala | 2 + .../physical/stream/StreamPhysicalMatch.scala | 7 +- .../StreamPhysicalMiniBatchAssigner.scala | 7 +- .../stream/StreamPhysicalOverAggregate.scala | 7 +- .../stream/StreamPhysicalPythonCalc.scala | 4 +- .../StreamPhysicalPythonCorrelate.scala | 7 +- .../StreamPhysicalPythonGroupAggregate.scala | 7 +- ...eamPhysicalPythonGroupTableAggregate.scala | 7 +- ...amPhysicalPythonGroupWindowAggregate.scala | 5 +- .../StreamPhysicalPythonOverAggregate.scala | 7 +- .../physical/stream/StreamPhysicalRank.scala | 5 +- .../physical/stream/StreamPhysicalSink.scala | 5 +- .../physical/stream/StreamPhysicalSort.scala | 5 +- .../stream/StreamPhysicalSortLimit.scala | 7 +- .../StreamPhysicalTableSourceScan.scala | 2 + .../stream/StreamPhysicalTemporalJoin.scala | 2 + .../stream/StreamPhysicalTemporalSort.scala | 5 +- .../physical/stream/StreamPhysicalUnion.scala | 7 +- .../stream/StreamPhysicalValues.scala | 5 +- .../StreamPhysicalWatermarkAssigner.scala | 2 + .../StreamPhysicalWindowAggregate.scala | 5 +- .../StreamPhysicalWindowDeduplicate.scala | 5 +- .../stream/StreamPhysicalWindowJoin.scala | 2 + .../stream/StreamPhysicalWindowRank.scala | 5 +- .../StreamPhysicalWindowTableFunction.scala | 5 +- .../flink/table/api/CompiledPlanITCase.java | 51 +++- .../InputPriorityConflictResolverTest.java | 1 + .../plan/utils/ExecNodeMetadataUtilTest.java | 256 +++++++++++++++--- 191 files changed, 1324 insertions(+), 329 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonDeserializer.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializer.java create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializerFilter.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java index 81d07b8d51001..d2b06fd534248 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java @@ -50,6 +50,7 @@ public interface ExecNode extends ExecNodeTranslator { String FIELD_NAME_ID = "id"; String FIELD_NAME_TYPE = "type"; + String FIELD_NAME_CONFIGURATION = "configuration"; String FIELD_NAME_DESCRIPTION = "description"; String FIELD_NAME_INPUT_PROPERTIES = "inputProperties"; String FIELD_NAME_OUTPUT_TYPE = "outputType"; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java index dbb8810fcc7a7..06b88507b3fca 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java @@ -24,6 +24,7 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.table.planner.plan.nodes.exec.serde.ConfigurationJsonSerializerFilter; import org.apache.flink.table.planner.plan.nodes.exec.utils.TransformationMetadata; import org.apache.flink.table.planner.plan.nodes.exec.visitor.ExecNodeVisitor; import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil; @@ -31,6 +32,7 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.util.ArrayList; @@ -70,13 +72,30 @@ protected final ExecNodeContext getContextFromAnnotation() { return ExecNodeContext.newContext(this.getClass()).withId(getId()); } + @JsonProperty(value = FIELD_NAME_CONFIGURATION, access = JsonProperty.Access.WRITE_ONLY) + private final ReadableConfig persistedConfig; + + @JsonProperty( + value = FIELD_NAME_CONFIGURATION, + access = JsonProperty.Access.READ_ONLY, + index = 2) + // Custom filter to exclude node configuration if no consumed options are used + @JsonInclude( + value = JsonInclude.Include.CUSTOM, + valueFilter = ConfigurationJsonSerializerFilter.class) + public ReadableConfig getPersistedConfig() { + return persistedConfig; + } + protected ExecNodeBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, LogicalType outputType, String description) { this.context = checkNotNull(context).withId(id); + this.persistedConfig = persistedConfig == null ? new Configuration() : persistedConfig; this.inputProperties = checkNotNull(inputProperties); this.outputType = checkNotNull(outputType); this.description = checkNotNull(description); @@ -129,7 +148,7 @@ public final Transformation translateToPlan(Planner planner) { translateToPlanInternal( (PlannerBase) planner, new ExecNodeConfig( - ((PlannerBase) planner).getTableConfig(), new Configuration())); + ((PlannerBase) planner).getTableConfig(), persistedConfig)); if (this instanceof SingleTransformationTranslator) { if (inputsContainSingleton()) { transformation.setParallelism(1); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java index 2a3e0a092dc9b..22b0ef24cd428 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java @@ -31,16 +31,16 @@ /** * Configuration view which is used combine the {@link PlannerBase#getTableConfig()} with the {@link - * ExecNodeBase#getNodeConfig()} configuration. The persisted configuration of the {@link ExecNode} - * which is deserialized from the JSON plan has precedence over the {@link + * ExecNodeBase#getPersistedConfig()} configuration. The persisted configuration of the {@link + * ExecNode} which is deserialized from the JSON plan has precedence over the {@link * PlannerBase#getTableConfig()}. */ @Internal public final class ExecNodeConfig implements ReadableConfig { // See https://issues.apache.org/jira/browse/FLINK-26190 - // Used only for the deprecated getMaxIdleStateRetentionTime to also satisfy tests which - // manipulate maxIdleStateRetentionTime, like OverAggregateHarnessTest. + // Used only by CommonPythonUtil#getMergedConfig(StreamExecutionEnvironment, TableConfig)} + // otherwise it can be changed to ReadableConfig. private final TableConfig tableConfig; private final ReadableConfig nodeConfig; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java index d00d681ac4c15..0abb680b13655 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java @@ -20,6 +20,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil; import org.apache.flink.table.types.logical.LogicalType; @@ -33,6 +35,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; +import java.util.stream.Stream; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -81,10 +84,11 @@ private ExecNodeContext(String name, Integer version) { * @param id The unique id of the {@link ExecNode}. See {@link ExecNode#getId()}. It can be null * initially and then later set by using {@link #withId(int)} which creates a new instance * of {@link ExecNodeContext} since it's immutable. This way we can satisfy both the {@link - * ExecNodeBase#ExecNodeBase(int, ExecNodeContext, List, LogicalType, String)} ctor, which - * is used for the {@link JsonCreator} ctors, where the {@code id} and the {@code context} - * are read separately, and the {@link ExecNodeBase#getContextFromAnnotation()} which - * creates a new context with a new id provided by: {@link #newNodeId()}. + * ExecNodeBase#ExecNodeBase(int, ExecNodeContext, ReadableConfig, List, LogicalType, + * String)} ctor, which is used for the {@link JsonCreator} ctors, where the {@code id} and + * the {@code context} are read separately, and the {@link + * ExecNodeBase#getContextFromAnnotation()} which creates a new context with a new id + * provided by: {@link #newNodeId()}. * @param name The name of the {@link ExecNode}. See {@link ExecNodeMetadata#name()}. * @param version The version of the {@link ExecNode}. See {@link ExecNodeMetadata#version()}. */ @@ -176,4 +180,22 @@ public static > ExecNodeContext newContext(Class execNo } return new ExecNodeContext(metadata.name(), metadata.version()); } + + /** + * Create a configuration for the {@link ExecNode}, ready to be persisted to a JSON plan. + * + * @param execNodeClass The {@link ExecNode} class. + * @param tableConfig The planner configuration (include the {@link TableConfig}). + * @return The {@link ExecNode} configuration, which contains the consumed options for the node, + * defined by {@link ExecNodeMetadata#consumedOptions()}, along with their values. + */ + public static > ReadableConfig newPersistedConfig( + Class execNodeClass, ReadableConfig tableConfig) { + return ExecNodeMetadataUtil.newPersistedConfig( + execNodeClass, + tableConfig, + Stream.concat( + ExecNodeMetadataUtil.TABLE_CONFIG_OPTIONS.stream(), + ExecNodeMetadataUtil.EXECUTION_CONFIG_OPTIONS.stream())); + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java index 167870e5f2c58..fb8a21752ceab 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -48,6 +49,7 @@ public class BatchExecBoundedStreamScan extends ExecNodeBase private final List qualifiedName; public BatchExecBoundedStreamScan( + ReadableConfig tableConfig, DataStream dataStream, DataType sourceType, int[] fieldIndexes, @@ -57,6 +59,7 @@ public BatchExecBoundedStreamScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecBoundedStreamScan.class), + ExecNodeContext.newPersistedConfig(BatchExecBoundedStreamScan.class, tableConfig), Collections.emptyList(), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java index a7063afcd9769..f2f3f5b6b3676 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -37,6 +38,7 @@ public class BatchExecCalc extends CommonExecCalc implements BatchExecNode { public BatchExecCalc( + ReadableConfig tableConfig, List projection, @Nullable RexNode condition, InputProperty inputProperty, @@ -45,6 +47,7 @@ public BatchExecCalc( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecCalc.class), + ExecNodeContext.newPersistedConfig(BatchExecCalc.class, tableConfig), projection, condition, TableStreamOperator.class, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java index c8e538b8ddd40..3b5450c094103 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; @@ -37,6 +38,7 @@ public class BatchExecCorrelate extends CommonExecCorrelate implements BatchExecNode { public BatchExecCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, @Nullable RexNode condition, @@ -46,6 +48,7 @@ public BatchExecCorrelate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecCorrelate.class), + ExecNodeContext.newPersistedConfig(BatchExecCorrelate.class, tableConfig), joinType, invocation, condition, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java index c07bd85d5479e..339dc8f40ffad 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.api.transformations.StreamExchangeMode; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; @@ -66,10 +67,15 @@ public class BatchExecExchange extends CommonExecExchange implements BatchExecNo // if it's None, use value from configuration @Nullable private StreamExchangeMode requiredExchangeMode; - public BatchExecExchange(InputProperty inputProperty, RowType outputType, String description) { + public BatchExecExchange( + ReadableConfig tableConfig, + InputProperty inputProperty, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecExchange.class), + ExecNodeContext.newPersistedConfig(BatchExecExchange.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java index 653f1d7e19fcd..c91e21c1da042 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -34,6 +35,7 @@ public class BatchExecExpand extends CommonExecExpand implements BatchExecNode { public BatchExecExpand( + ReadableConfig tableConfig, List> projects, InputProperty inputProperty, RowType outputType, @@ -41,6 +43,7 @@ public BatchExecExpand( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecExpand.class), + ExecNodeContext.newPersistedConfig(BatchExecExpand.class, tableConfig), projects, false, // retainHeader Collections.singletonList(inputProperty), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java index 98511a22300fd..59a6fc65958f9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -59,6 +60,7 @@ public class BatchExecHashAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecHashAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -71,6 +73,7 @@ public BatchExecHashAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecHashAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecHashAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java index e003eb22bd8e7..b6ee3a1a14021 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; @@ -61,6 +62,7 @@ public class BatchExecHashJoin extends ExecNodeBase private final boolean tryDistinctBuildRow; public BatchExecHashJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, int estimatedLeftAvgRowSize, int estimatedRightAvgRowSize, @@ -75,6 +77,7 @@ public BatchExecHashJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecHashJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecHashJoin.class, tableConfig), Arrays.asList(leftInputProperty, rightInputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java index c16f386b4173e..d43c0cddd16e4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -67,6 +68,7 @@ public class BatchExecHashWindowAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecHashWindowAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -84,6 +86,7 @@ public BatchExecHashWindowAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecHashWindowAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecHashWindowAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java index 4440d8725c279..c478366028051 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -41,6 +42,7 @@ public class BatchExecLegacySink extends CommonExecLegacySink implements BatchExecNode { public BatchExecLegacySink( + ReadableConfig tableConfig, TableSink tableSink, @Nullable String[] upsertKeys, InputProperty inputProperty, @@ -49,6 +51,7 @@ public BatchExecLegacySink( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLegacySink.class), + ExecNodeContext.newPersistedConfig(BatchExecLegacySink.class, tableConfig), tableSink, upsertKeys, false, // needRetraction diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java index 69a728b52fe57..5696fd48001ea 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.InputSplit; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; @@ -55,6 +56,7 @@ public class BatchExecLegacyTableSourceScan extends CommonExecLegacyTableSourceS implements BatchExecNode { public BatchExecLegacyTableSourceScan( + ReadableConfig tableConfig, TableSource tableSource, List qualifiedName, RowType outputType, @@ -62,6 +64,8 @@ public BatchExecLegacyTableSourceScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLegacyTableSourceScan.class), + ExecNodeContext.newPersistedConfig( + BatchExecLegacyTableSourceScan.class, tableConfig), tableSource, qualifiedName, outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java index 001001c150ae4..17844d6617b29 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -41,6 +42,7 @@ public class BatchExecLimit extends ExecNodeBase implements BatchExecNo private final boolean isGlobal; public BatchExecLimit( + ReadableConfig tableConfig, long limitStart, long limitEnd, boolean isGlobal, @@ -50,6 +52,7 @@ public BatchExecLimit( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLimit.class), + ExecNodeContext.newPersistedConfig(BatchExecLimit.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java index f756c2ea269ad..c9f9180941fc6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; @@ -38,6 +39,7 @@ /** {@link BatchExecNode} for temporal table join that implemented by lookup. */ public class BatchExecLookupJoin extends CommonExecLookupJoin implements BatchExecNode { public BatchExecLookupJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, @Nullable RexNode joinCondition, TemporalTableSourceSpec temporalTableSourceSpec, @@ -50,6 +52,7 @@ public BatchExecLookupJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLookupJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecLookupJoin.class, tableConfig), joinType, joinCondition, temporalTableSourceSpec, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java index 15fd0498d7686..1420610190f15 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.transformations.MultipleInputTransformation; import org.apache.flink.table.data.RowData; @@ -78,6 +79,7 @@ public class BatchExecMultipleInput extends ExecNodeBase private final List originalEdges; public BatchExecMultipleInput( + ReadableConfig tableConfig, List inputProperties, ExecNode rootNode, List originalEdges, @@ -85,6 +87,7 @@ public BatchExecMultipleInput( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecMultipleInput.class), + ExecNodeContext.newPersistedConfig(BatchExecMultipleInput.class, tableConfig), inputProperties, rootNode.getOutputType(), description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java index 0d868ff353684..9b5401fdc4a66 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -51,6 +52,7 @@ public class BatchExecNestedLoopJoin extends ExecNodeBase private final boolean singleRowJoin; public BatchExecNestedLoopJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, RexNode condition, boolean leftIsBuild, @@ -62,6 +64,7 @@ public BatchExecNestedLoopJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecNestedLoopJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecNestedLoopJoin.class, tableConfig), Arrays.asList(leftInputProperty, rightInputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java index 6e4805ce5a158..357e28dda85f9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.config.ExecutionConfigOptions; @@ -75,6 +76,7 @@ public class BatchExecOverAggregate extends BatchExecOverAggregateBase { public BatchExecOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -82,6 +84,7 @@ public BatchExecOverAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecOverAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecOverAggregate.class, tableConfig), overSpec, inputProperty, outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java index 3121d9d2c2132..860d40b34e14a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -47,11 +48,18 @@ public abstract class BatchExecOverAggregateBase extends ExecNodeBase public BatchExecOverAggregateBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, String description) { - super(id, context, Collections.singletonList(inputProperty), outputType, description); + super( + id, + context, + persistedConfig, + Collections.singletonList(inputProperty), + outputType, + description); this.overSpec = overSpec; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java index 9c6c59876ae00..7ecd93ac0df21 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -34,6 +35,7 @@ public class BatchExecPythonCalc extends CommonExecPythonCalc implements BatchExecNode { public BatchExecPythonCalc( + ReadableConfig tableConfig, List projection, InputProperty inputProperty, RowType outputType, @@ -41,6 +43,7 @@ public BatchExecPythonCalc( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonCalc.class), + ExecNodeContext.newPersistedConfig(BatchExecPythonCalc.class, tableConfig), projection, Collections.singletonList(inputProperty), outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java index 8307f6f27444e..0d58251a80092 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; @@ -34,6 +35,7 @@ public class BatchExecPythonCorrelate extends CommonExecPythonCorrelate implements BatchExecNode { public BatchExecPythonCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, InputProperty inputProperty, @@ -42,6 +44,7 @@ public BatchExecPythonCorrelate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonCorrelate.class), + ExecNodeContext.newPersistedConfig(BatchExecPythonCorrelate.class, tableConfig), joinType, invocation, Collections.singletonList(inputProperty), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java index 46ada1ba62867..7e637014f9600 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -63,6 +64,7 @@ public class BatchExecPythonGroupAggregate extends ExecNodeBase private final AggregateCall[] aggCalls; public BatchExecPythonGroupAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -72,6 +74,8 @@ public BatchExecPythonGroupAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + BatchExecPythonGroupAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java index 1833970894f7b..8dc3f6ca36c7a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -75,6 +76,7 @@ public class BatchExecPythonGroupWindowAggregate extends ExecNodeBase private final NamedWindowProperty[] namedWindowProperties; public BatchExecPythonGroupWindowAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -87,6 +89,8 @@ public BatchExecPythonGroupWindowAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonGroupWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + BatchExecPythonGroupWindowAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java index c9529ad3d1231..ae887103b8341 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -69,6 +70,7 @@ public class BatchExecPythonOverAggregate extends BatchExecOverAggregateBase { private final List aggWindowIndex; public BatchExecPythonOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -76,6 +78,7 @@ public BatchExecPythonOverAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonOverAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecPythonOverAggregate.class, tableConfig), overSpec, inputProperty, outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java index 0a86211687e00..daade9c6dab1d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator; @@ -51,6 +52,7 @@ public class BatchExecRank extends ExecNodeBase implements InputSortedE private final boolean outputRankNumber; public BatchExecRank( + ReadableConfig tableConfig, int[] partitionFields, int[] sortFields, long rankStart, @@ -62,6 +64,7 @@ public BatchExecRank( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecRank.class), + ExecNodeContext.newPersistedConfig(BatchExecRank.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java index d9c5b35d04fa4..da1637f962096 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; @@ -39,6 +40,7 @@ */ public class BatchExecSink extends CommonExecSink implements BatchExecNode { public BatchExecSink( + ReadableConfig tableConfig, DynamicTableSinkSpec tableSinkSpec, InputProperty inputProperty, LogicalType outputType, @@ -46,6 +48,7 @@ public BatchExecSink( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSink.class), + ExecNodeContext.newPersistedConfig(BatchExecSink.class, tableConfig), tableSinkSpec, ChangelogMode.insertOnly(), true, // isBounded diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java index 8f8e42082d9a5..44f1e19db4926 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -47,6 +48,7 @@ public class BatchExecSort extends ExecNodeBase implements BatchExecNod private final SortSpec sortSpec; public BatchExecSort( + ReadableConfig tableConfig, SortSpec sortSpec, InputProperty inputProperty, RowType outputType, @@ -54,6 +56,7 @@ public BatchExecSort( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSort.class), + ExecNodeContext.newPersistedConfig(BatchExecSort.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java index e550fcec7f5e3..4daf095667be1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -58,6 +59,7 @@ public class BatchExecSortAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecSortAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -70,6 +72,7 @@ public BatchExecSortAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecSortAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java index 3cbaefbc61e96..a2fdf205adb69 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -53,6 +54,7 @@ public class BatchExecSortLimit extends ExecNodeBase private final boolean isGlobal; public BatchExecSortLimit( + ReadableConfig tableConfig, SortSpec sortSpec, long limitStart, long limitEnd, @@ -63,6 +65,7 @@ public BatchExecSortLimit( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortLimit.class), + ExecNodeContext.newPersistedConfig(BatchExecSortLimit.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java index 9d8e7d7327520..b38d3264be736 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -65,6 +66,7 @@ public class BatchExecSortMergeJoin extends ExecNodeBase private final boolean leftIsSmaller; public BatchExecSortMergeJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, int[] leftKeys, int[] rightKeys, @@ -78,6 +80,7 @@ public BatchExecSortMergeJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortMergeJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecSortMergeJoin.class, tableConfig), Arrays.asList(leftInputProperty, rightInputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java index d91a4515e75f8..ce95fb9e0259f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -67,6 +68,7 @@ public class BatchExecSortWindowAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecSortWindowAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -84,6 +86,7 @@ public BatchExecSortWindowAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortWindowAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecSortWindowAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java index bfb58159d6337..95e98273fb9ce 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; import org.apache.flink.table.connector.source.ScanTableSource; @@ -42,10 +43,14 @@ public class BatchExecTableSourceScan extends CommonExecTableSourceScan implements BatchExecNode { public BatchExecTableSourceScan( - DynamicTableSourceSpec tableSourceSpec, RowType outputType, String description) { + ReadableConfig tableConfig, + DynamicTableSourceSpec tableSourceSpec, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecTableSourceScan.class), + ExecNodeContext.newPersistedConfig(BatchExecTableSourceScan.class, tableConfig), tableSourceSpec, outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java index 506243b060496..d365206b356f0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -34,10 +35,14 @@ public class BatchExecUnion extends CommonExecUnion implements BatchExecNode { public BatchExecUnion( - List inputProperties, RowType outputType, String description) { + ReadableConfig tableConfig, + List inputProperties, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecUnion.class), + ExecNodeContext.newPersistedConfig(BatchExecUnion.class, tableConfig), inputProperties, outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java index ab8700f44534b..b1be0f8395488 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -35,10 +36,15 @@ /** Batch {@link ExecNode} that read records from given values. */ public class BatchExecValues extends CommonExecValues implements BatchExecNode { - public BatchExecValues(List> tuples, RowType outputType, String description) { + public BatchExecValues( + ReadableConfig tableConfig, + List> tuples, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecValues.class), + ExecNodeContext.newPersistedConfig(BatchExecValues.class, tableConfig), tuples, outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java index 1e36ed1778f94..b00ce302c1940 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -37,6 +38,7 @@ public class BatchExecWindowTableFunction extends CommonExecWindowTableFunction implements BatchExecNode { public BatchExecWindowTableFunction( + ReadableConfig tableConfig, TimeAttributeWindowingStrategy windowingStrategy, InputProperty inputProperty, RowType outputType, @@ -44,6 +46,7 @@ public BatchExecWindowTableFunction( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecWindowTableFunction.class), + ExecNodeContext.newPersistedConfig(BatchExecWindowTableFunction.class, tableConfig), windowingStrategy, Collections.singletonList(inputProperty), outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java index 24024c6e10011..b954afae062c2 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CalcCodeGenerator; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -68,6 +69,7 @@ public abstract class CommonExecCalc extends ExecNodeBase protected CommonExecCalc( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List projection, @Nullable RexNode condition, Class operatorBaseClass, @@ -75,7 +77,7 @@ protected CommonExecCalc( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.projection = checkNotNull(projection); this.condition = condition; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java index bab655c69ea6e..e75bd2b3d33a0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.codegen.CorrelateCodeGenerator; @@ -72,6 +73,7 @@ public abstract class CommonExecCorrelate extends ExecNodeBase public CommonExecCorrelate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, RexCall invocation, @Nullable RexNode condition, @@ -80,7 +82,7 @@ public CommonExecCorrelate( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.joinType = checkNotNull(joinType); this.invocation = checkNotNull(invocation); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java index f19498acecd74..809bcf94f27ae 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -38,9 +39,10 @@ public abstract class CommonExecExchange extends ExecNodeBase public CommonExecExchange( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java index ddb6040ee33f1..a939a06934be5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.codegen.ExpandCodeGenerator; @@ -60,12 +61,13 @@ public abstract class CommonExecExpand extends ExecNodeBase public CommonExecExpand( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List> projects, boolean retainHeader, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.projects = checkNotNull(projects); checkArgument( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java index ea72482dc86d9..8f731f6ba5412 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.table.api.TableException; @@ -70,6 +71,7 @@ public abstract class CommonExecLegacySink extends ExecNodeBase public CommonExecLegacySink( int id, ExecNodeContext context, + ReadableConfig persistedConfig, TableSink tableSink, @Nullable String[] upsertKeys, boolean needRetraction, @@ -77,7 +79,13 @@ public CommonExecLegacySink( InputProperty inputProperty, LogicalType outputType, String description) { - super(id, context, Collections.singletonList(inputProperty), outputType, description); + super( + id, + context, + persistedConfig, + Collections.singletonList(inputProperty), + outputType, + description); this.tableSink = tableSink; this.upsertKeys = upsertKeys; this.needRetraction = needRetraction; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java index 0c57259acce21..5769d8ad86a56 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.InputSplit; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableException; @@ -71,11 +72,12 @@ public abstract class CommonExecLegacyTableSourceScan extends ExecNodeBase tableSource, List qualifiedName, RowType outputType, String description) { - super(id, context, Collections.emptyList(), outputType, description); + super(id, context, persistedConfig, Collections.emptyList(), outputType, description); this.tableSource = tableSource; this.qualifiedName = qualifiedName; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java index c50deb92545a6..1986507cb21b1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.streaming.api.functions.async.AsyncFunction; @@ -176,6 +177,7 @@ public abstract class CommonExecLookupJoin extends ExecNodeBase protected CommonExecLookupJoin( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, @Nullable RexNode joinCondition, // TODO: refactor this into TableSourceTable, once legacy TableSource is removed @@ -186,7 +188,7 @@ protected CommonExecLookupJoin( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.joinType = checkNotNull(joinType); this.joinCondition = joinCondition; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java index 6d63f3b852fb4..c31e3cc0f0101 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -84,11 +85,12 @@ public abstract class CommonExecPythonCalc extends ExecNodeBase public CommonExecPythonCalc( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List projection, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.projection = checkNotNull(projection); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java index 9af214c5a20b6..e60f04aff80ba 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -69,12 +70,13 @@ public abstract class CommonExecPythonCorrelate extends ExecNodeBase public CommonExecPythonCorrelate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, RexCall invocation, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.joinType = joinType; this.invocation = invocation; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java index a5cf73879b445..00cf4daea7368 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java @@ -110,13 +110,14 @@ public abstract class CommonExecSink extends ExecNodeBase protected CommonExecSink( int id, ExecNodeContext context, + ReadableConfig persistedConfig, DynamicTableSinkSpec tableSinkSpec, ChangelogMode inputChangelogMode, boolean isBounded, List inputProperties, LogicalType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.tableSinkSpec = tableSinkSpec; this.inputChangelogMode = inputChangelogMode; this.isBounded = isBounded; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java index 8d335407c3a7e..07e193b81e5bb 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java @@ -74,10 +74,11 @@ public abstract class CommonExecTableSourceScan extends ExecNodeBase protected CommonExecTableSourceScan( int id, ExecNodeContext context, + ReadableConfig persistedConfig, DynamicTableSourceSpec tableSourceSpec, LogicalType outputType, String description) { - super(id, context, Collections.emptyList(), outputType, description); + super(id, context, persistedConfig, Collections.emptyList(), outputType, description); this.tableSourceSpec = tableSourceSpec; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java index 329475906b435..dddf6c6da8944 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.UnionTransformation; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -44,10 +45,11 @@ public abstract class CommonExecUnion extends ExecNodeBase public CommonExecUnion( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); } @SuppressWarnings({"unchecked", "rawtypes"}) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java index 21be27f880fff..7e4639bb6e3cd 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.ValuesCodeGenerator; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -50,10 +51,11 @@ public abstract class CommonExecValues extends ExecNodeBase public CommonExecValues( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List> tuples, RowType outputType, String description) { - super(id, context, Collections.emptyList(), outputType, description); + super(id, context, persistedConfig, Collections.emptyList(), outputType, description); this.tuples = tuples; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java index 998ab8e7db16a..c39189e339bca 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy; @@ -62,11 +63,12 @@ public abstract class CommonExecWindowTableFunction extends ExecNodeBase inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.windowingStrategy = checkNotNull(windowingStrategy); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java index eae6279b901e3..d8238605ec084 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java @@ -17,6 +17,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.processor; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -69,6 +70,7 @@ public ExecNodeGraph process(ExecNodeGraph execGraph, ProcessorContext context) if (!context.getPlanner().getExecEnv().getConfig().isDynamicGraph()) { return execGraph; } + ReadableConfig tableConfig = context.getPlanner().getTableConfig(); ExecNodeVisitor visitor = new AbstractExecNodeExactlyOnceVisitor() { @Override @@ -91,7 +93,8 @@ protected void visitNode(ExecNode node) { // sort node and its output can also be connected by // ForwardPartitioner ExecEdge newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, true); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, true); newEdges.set(i, newEdge); changed = true; } @@ -111,6 +114,7 @@ protected void visitNode(ExecNode node) { ExecNode sort = edge.getSource(); ExecEdge newEdgeOfSort = addExchangeAndReconnectEdge( + tableConfig, sort.getInputEdges().get(0), inputProperty, false); @@ -122,12 +126,14 @@ protected void visitNode(ExecNode node) { // sort node and its output can also be connected by // ForwardPartitioner newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, true); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, true); } else { // add Exchange with keep_input_as_is distribution as the input // of the node newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, false); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, false); updateOriginalEdgeInMultipleInput( node, i, (BatchExecExchange) newEdge.getSource()); } @@ -138,7 +144,8 @@ protected void visitNode(ExecNode node) { // if operation chaining is disabled, this could mark sure the sort // node and its output can also be connected by ForwardPartitioner ExecEdge newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, true); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, true); newEdges.set(i, newEdge); changed = true; } @@ -154,7 +161,10 @@ protected void visitNode(ExecNode node) { // TODO This implementation should be updated once FLINK-21224 is finished. private ExecEdge addExchangeAndReconnectEdge( - ExecEdge edge, InputProperty inputProperty, boolean strict) { + ReadableConfig tableConfig, + ExecEdge edge, + InputProperty inputProperty, + boolean strict) { ExecNode target = edge.getTarget(); ExecNode source = edge.getSource(); if (source instanceof CommonExecExchange) { @@ -168,13 +178,13 @@ private ExecEdge addExchangeAndReconnectEdge( || source instanceof BatchExecPythonCorrelate) { ExecEdge newEdge = addExchangeAndReconnectEdge( - source.getInputEdges().get(0), inputProperty, strict); + tableConfig, source.getInputEdges().get(0), inputProperty, strict); source.setInputEdges(Collections.singletonList(newEdge)); } BatchExecExchange exchange = createExchangeWithKeepInputAsIsDistribution( - inputProperty, strict, (RowType) edge.getOutputType()); + tableConfig, inputProperty, strict, (RowType) edge.getOutputType()); ExecEdge newEdge = new ExecEdge(source, exchange, edge.getShuffle(), edge.getExchangeMode()); exchange.setInputEdges(Collections.singletonList(newEdge)); @@ -182,7 +192,10 @@ private ExecEdge addExchangeAndReconnectEdge( } private BatchExecExchange createExchangeWithKeepInputAsIsDistribution( - InputProperty inputProperty, boolean strict, RowType outputRowType) { + ReadableConfig tableConfig, + InputProperty inputProperty, + boolean strict, + RowType outputRowType) { InputProperty newInputProperty = InputProperty.builder() .requiredDistribution( @@ -191,7 +204,8 @@ private BatchExecExchange createExchangeWithKeepInputAsIsDistribution( .damBehavior(inputProperty.getDamBehavior()) .priority(inputProperty.getPriority()) .build(); - return new BatchExecExchange(newInputProperty, outputRowType, newInputProperty.toString()); + return new BatchExecExchange( + tableConfig, newInputProperty, outputRowType, newInputProperty.toString()); } private boolean hasExchangeInput(ExecEdge edge) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java index fc245d4f4690d..6495b2a2c906c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.SourceTransformation; import org.apache.flink.streaming.api.transformations.StreamExchangeMode; import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; @@ -92,7 +93,8 @@ public ExecNodeGraph process(ExecNodeGraph execGraph, ProcessorContext context) optimizeMultipleInputGroups(orderedWrappers, context); // create the real multiple input nodes - List> newRootNodes = createMultipleInputNodes(rootWrappers); + List> newRootNodes = + createMultipleInputNodes(context.getPlanner().getTableConfig(), rootWrappers); return new ExecNodeGraph(newRootNodes); } @@ -451,23 +453,27 @@ static boolean isChainableSource(ExecNode node, ProcessorContext context) { // Multiple Input Nodes Creating // -------------------------------------------------------------------------------- - private List> createMultipleInputNodes(List rootWrappers) { + private List> createMultipleInputNodes( + ReadableConfig tableConfig, List rootWrappers) { List> result = new ArrayList<>(); Map> visitedMap = new HashMap<>(); for (ExecNodeWrapper rootWrapper : rootWrappers) { - result.add(getMultipleInputNode(rootWrapper, visitedMap)); + result.add(getMultipleInputNode(tableConfig, rootWrapper, visitedMap)); } return result; } private ExecNode getMultipleInputNode( - ExecNodeWrapper wrapper, Map> visitedMap) { + ReadableConfig tableConfig, + ExecNodeWrapper wrapper, + Map> visitedMap) { if (visitedMap.containsKey(wrapper)) { return visitedMap.get(wrapper); } for (int i = 0; i < wrapper.inputs.size(); i++) { - ExecNode multipleInputNode = getMultipleInputNode(wrapper.inputs.get(i), visitedMap); + ExecNode multipleInputNode = + getMultipleInputNode(tableConfig, wrapper.inputs.get(i), visitedMap); ExecEdge execEdge = ExecEdge.builder().source(multipleInputNode).target(wrapper.execNode).build(); wrapper.execNode.replaceInputEdge(i, execEdge); @@ -475,7 +481,7 @@ private ExecNode getMultipleInputNode( ExecNode ret; if (wrapper.group != null && wrapper == wrapper.group.root) { - ret = createMultipleInputNode(wrapper.group, visitedMap); + ret = createMultipleInputNode(tableConfig, wrapper.group, visitedMap); } else { ret = wrapper.execNode; } @@ -484,7 +490,9 @@ private ExecNode getMultipleInputNode( } private ExecNode createMultipleInputNode( - MultipleInputGroup group, Map> visitedMap) { + ReadableConfig tableConfig, + MultipleInputGroup group, + Map> visitedMap) { // calculate the inputs of the multiple input node List, InputProperty, ExecEdge>> inputs = new ArrayList<>(); for (ExecNodeWrapper member : group.members) { @@ -505,14 +513,16 @@ private ExecNode createMultipleInputNode( } if (isStreaming) { - return createStreamMultipleInputNode(group, inputs); + return createStreamMultipleInputNode(tableConfig, group, inputs); } else { - return createBatchMultipleInputNode(group, inputs); + return createBatchMultipleInputNode(tableConfig, group, inputs); } } private StreamExecMultipleInput createStreamMultipleInputNode( - MultipleInputGroup group, List, InputProperty, ExecEdge>> inputs) { + ReadableConfig tableConfig, + MultipleInputGroup group, + List, InputProperty, ExecEdge>> inputs) { ExecNode rootNode = group.root.execNode; List> inputNodes = new ArrayList<>(); for (Tuple3, InputProperty, ExecEdge> tuple3 : inputs) { @@ -523,6 +533,7 @@ private StreamExecMultipleInput createStreamMultipleInputNode( ExecNodeUtil.getMultipleInputDescription(rootNode, inputNodes, new ArrayList<>()); StreamExecMultipleInput multipleInput = new StreamExecMultipleInput( + tableConfig, inputNodes.stream() .map(i -> InputProperty.DEFAULT) .collect(Collectors.toList()), @@ -538,7 +549,9 @@ private StreamExecMultipleInput createStreamMultipleInputNode( } private BatchExecMultipleInput createBatchMultipleInputNode( - MultipleInputGroup group, List, InputProperty, ExecEdge>> inputs) { + ReadableConfig tableConfig, + MultipleInputGroup group, + List, InputProperty, ExecEdge>> inputs) { // first calculate the input orders using InputPriorityConflictResolver Set> inputSet = new HashSet<>(); for (Tuple3, InputProperty, ExecEdge> tuple3 : inputs) { @@ -571,7 +584,8 @@ private BatchExecMultipleInput createBatchMultipleInputNode( String description = ExecNodeUtil.getMultipleInputDescription(rootNode, inputNodes, inputProperties); BatchExecMultipleInput multipleInput = - new BatchExecMultipleInput(inputProperties, rootNode, originalEdges, description); + new BatchExecMultipleInput( + tableConfig, inputProperties, rootNode, originalEdges, description); List inputEdges = new ArrayList<>(inputNodes.size()); for (ExecNode inputNode : inputNodes) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java index 655ca5bdc7a92..a6588df84d9d8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java @@ -43,7 +43,7 @@ public class InputPriorityConflictResolver extends InputPriorityGraphGenerator { private final StreamExchangeMode exchangeMode; - private final ReadableConfig configuration; + private final ReadableConfig tableConfig; /** * Create a {@link InputPriorityConflictResolver} for the given {@link ExecNode} graph. @@ -58,10 +58,10 @@ public InputPriorityConflictResolver( List> roots, InputProperty.DamBehavior safeDamBehavior, StreamExchangeMode exchangeMode, - ReadableConfig configuration) { + ReadableConfig tableConfig) { super(roots, Collections.emptySet(), safeDamBehavior); this.exchangeMode = exchangeMode; - this.configuration = configuration; + this.tableConfig = tableConfig; } public void detectAndResolve() { @@ -87,7 +87,10 @@ protected void resolveInputPriorityConflict(ExecNode node, int higherInput, i // we should split it into two nodes BatchExecExchange newExchange = new BatchExecExchange( - inputProperty, (RowType) exchange.getOutputType(), "Exchange"); + tableConfig, + inputProperty, + (RowType) exchange.getOutputType(), + "Exchange"); newExchange.setRequiredExchangeMode(exchangeMode); newExchange.setInputEdges(exchange.getInputEdges()); newNode = newExchange; @@ -95,6 +98,7 @@ protected void resolveInputPriorityConflict(ExecNode node, int higherInput, i // create new BatchExecExchange with new inputProperty BatchExecExchange newExchange = new BatchExecExchange( + tableConfig, inputProperty, (RowType) exchange.getOutputType(), exchange.getDescription()); @@ -138,7 +142,10 @@ private BatchExecExchange createExchange(ExecNode node, int idx) { .build(); BatchExecExchange exchange = new BatchExecExchange( - newInputProperty, (RowType) inputNode.getOutputType(), "Exchange"); + tableConfig, + newInputProperty, + (RowType) inputNode.getOutputType(), + "Exchange"); exchange.setRequiredExchangeMode(exchangeMode); ExecEdge execEdge = ExecEdge.builder().source(inputNode).target(exchange).build(); exchange.setInputEdges(Collections.singletonList(execEdge)); @@ -170,7 +177,7 @@ protected void visitNode(ExecNode node) { } private InputProperty.DamBehavior getDamBehavior() { - if (getBatchStreamExchangeMode(configuration, exchangeMode) == StreamExchangeMode.BATCH) { + if (getBatchStreamExchangeMode(tableConfig, exchangeMode) == StreamExchangeMode.BATCH) { return InputProperty.DamBehavior.BLOCKING; } else { return InputProperty.DamBehavior.PIPELINED; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonDeserializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonDeserializer.java new file mode 100644 index 0000000000000..d5cf4321e68e7 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonDeserializer.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.serde; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer; + +import java.io.IOException; +import java.util.Map; + +/** + * Custom deserializer for {@link Configuration} used for {@link ExecNodeBase#getPersistedConfig}. + */ +@Internal +class ConfigurationJsonDeserializer extends StdDeserializer { + + private static final TypeReference MAP_STRING_STRING_TYPE_REF = + new TypeReference>() {}; + + ConfigurationJsonDeserializer() { + super(Configuration.class); + } + + @Override + public Configuration deserialize( + JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException { + return Configuration.fromMap(jsonParser.readValueAs(MAP_STRING_STRING_TYPE_REF)); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializer.java new file mode 100644 index 0000000000000..0a1c0afba3168 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializer.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.serde; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; + +import java.io.IOException; +import java.util.TreeMap; + +/** + * Custom serializer for {@link Configuration} used for {@link ExecNodeBase#getPersistedConfig()}. + */ +@Internal +class ConfigurationJsonSerializer extends StdSerializer { + + ConfigurationJsonSerializer() { + super(Configuration.class); + } + + @Override + public void serialize( + Configuration configuration, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) + throws IOException { + // Use TreeMap to have a nice -sorted by key- output of the config options. + serializerProvider.defaultSerializeValue( + new TreeMap<>(configuration.toMap()), jsonGenerator); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializerFilter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializerFilter.java new file mode 100644 index 0000000000000..1d1a4a5e82d0c --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializerFilter.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.serde; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; + +/** + * Custom filtering for {@link Configuration} used by {@link ExecNodeBase#getPersistedConfig()} to + * avoid serializing null or empty configurations. + */ +@Internal +public class ConfigurationJsonSerializerFilter { + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return true; + } + if (obj instanceof Configuration) { + Configuration other = (Configuration) obj; + return other.toMap().isEmpty(); + } + return true; + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java index f044d0d33cd0a..8a601601d3f99 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ContextResolvedTable; @@ -139,6 +140,7 @@ private static Module createFlinkTableJacksonModule() { private static void registerSerializers(SimpleModule module) { module.addSerializer(new ExecNodeGraphJsonSerializer()); module.addSerializer(new FlinkVersionJsonSerializer()); + module.addSerializer(new ConfigurationJsonSerializer()); module.addSerializer(new ObjectIdentifierJsonSerializer()); module.addSerializer(new LogicalTypeJsonSerializer()); module.addSerializer(new DataTypeJsonSerializer()); @@ -161,6 +163,7 @@ private static void registerSerializers(SimpleModule module) { private static void registerDeserializers(SimpleModule module) { module.addDeserializer(ExecNodeGraph.class, new ExecNodeGraphJsonDeserializer()); module.addDeserializer(FlinkVersion.class, new FlinkVersionJsonDeserializer()); + module.addDeserializer(ReadableConfig.class, new ConfigurationJsonDeserializer()); module.addDeserializer(ObjectIdentifier.class, new ObjectIdentifierJsonDeserializer()); module.addDeserializer(LogicalType.class, new LogicalTypeJsonDeserializer()); module.addDeserializer(RowType.class, (StdDeserializer) new LogicalTypeJsonDeserializer()); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java index fde9fda4bc4c6..ee33e0a580c34 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; @@ -43,10 +44,11 @@ public abstract class StreamExecAggregateBase extends ExecNodeBase protected StreamExecAggregateBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, LogicalType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java index 90112df84c2fe..72b4be1a87160 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -48,6 +49,7 @@ public class StreamExecCalc extends CommonExecCalc implements StreamExecNode { public StreamExecCalc( + ReadableConfig tableConfig, List projection, @Nullable RexNode condition, InputProperty inputProperty, @@ -56,6 +58,7 @@ public StreamExecCalc( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecCalc.class), + ExecNodeContext.newPersistedConfig(StreamExecCalc.class, tableConfig), projection, condition, Collections.singletonList(inputProperty), @@ -67,6 +70,7 @@ public StreamExecCalc( public StreamExecCalc( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PROJECTION) List projection, @JsonProperty(FIELD_NAME_CONDITION) @Nullable RexNode condition, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @@ -75,6 +79,7 @@ public StreamExecCalc( super( id, context, + persistedConfig, projection, condition, TableStreamOperator.class, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java index 5c82e812698cd..077c950847687 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -85,6 +86,7 @@ public class StreamExecChangelogNormalize extends ExecNodeBase private final boolean generateUpdateBefore; public StreamExecChangelogNormalize( + ReadableConfig tableConfig, int[] uniqueKeys, boolean generateUpdateBefore, InputProperty inputProperty, @@ -93,6 +95,7 @@ public StreamExecChangelogNormalize( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecChangelogNormalize.class), + ExecNodeContext.newPersistedConfig(StreamExecChangelogNormalize.class, tableConfig), uniqueKeys, generateUpdateBefore, Collections.singletonList(inputProperty), @@ -104,12 +107,13 @@ public StreamExecChangelogNormalize( public StreamExecChangelogNormalize( @JsonProperty(FIELD_NAME_ID) Integer id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_UNIQUE_KEYS) int[] uniqueKeys, @JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.uniqueKeys = uniqueKeys; this.generateUpdateBefore = generateUpdateBefore; } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java index c843ecfbe8afc..eb493898fab4b 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -52,6 +53,7 @@ public class StreamExecCorrelate extends CommonExecCorrelate implements StreamExecNode { public StreamExecCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, @Nullable RexNode condition, @@ -61,6 +63,7 @@ public StreamExecCorrelate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecCorrelate.class), + ExecNodeContext.newPersistedConfig(StreamExecCorrelate.class, tableConfig), joinType, invocation, condition, @@ -73,6 +76,7 @@ public StreamExecCorrelate( public StreamExecCorrelate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType joinType, @JsonProperty(FIELD_NAME_FUNCTION_CALL) RexNode invocation, @JsonProperty(FIELD_NAME_CONDITION) @Nullable RexNode condition, @@ -82,6 +86,7 @@ public StreamExecCorrelate( super( id, context, + persistedConfig, joinType, (RexCall) invocation, condition, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java index 7ea8db3fafbf1..4b599284cf2e8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.calcite.FlinkRelBuilder; @@ -63,6 +64,7 @@ public class StreamExecDataStreamScan extends ExecNodeBase private final List qualifiedName; public StreamExecDataStreamScan( + ReadableConfig tableConfig, DataStream dataStream, DataType sourceType, int[] fieldIndexes, @@ -73,6 +75,7 @@ public StreamExecDataStreamScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecDataStreamScan.class), + ExecNodeContext.newPersistedConfig(StreamExecDataStreamScan.class, tableConfig), Collections.emptyList(), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java index c5bf069867c55..93425a9e35e0e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java @@ -105,6 +105,7 @@ public class StreamExecDeduplicate extends ExecNodeBase private final boolean generateUpdateBefore; public StreamExecDeduplicate( + ReadableConfig tableConfig, int[] uniqueKeys, boolean isRowtime, boolean keepLastRow, @@ -115,6 +116,7 @@ public StreamExecDeduplicate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecDeduplicate.class), + ExecNodeContext.newPersistedConfig(StreamExecDeduplicate.class, tableConfig), uniqueKeys, isRowtime, keepLastRow, @@ -128,6 +130,7 @@ public StreamExecDeduplicate( public StreamExecDeduplicate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_UNIQUE_KEYS) int[] uniqueKeys, @JsonProperty(FIELD_NAME_IS_ROWTIME) boolean isRowtime, @JsonProperty(FIELD_NAME_KEEP_LAST_ROW) boolean keepLastRow, @@ -135,7 +138,7 @@ public StreamExecDeduplicate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.uniqueKeys = checkNotNull(uniqueKeys); this.isRowtime = isRowtime; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java index eb8cdcb54a199..cc2ef75f809fe 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.StreamFilter; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -57,10 +58,14 @@ public class StreamExecDropUpdateBefore extends ExecNodeBase public static final String DROP_UPDATE_BEFORE_TRANSFORMATION = "drop-update-before"; public StreamExecDropUpdateBefore( - InputProperty inputProperty, RowType outputType, String description) { + ReadableConfig tableConfig, + InputProperty inputProperty, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecDropUpdateBefore.class), + ExecNodeContext.newPersistedConfig(StreamExecDropUpdateBefore.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); @@ -70,10 +75,11 @@ public StreamExecDropUpdateBefore( public StreamExecDropUpdateBefore( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); } @SuppressWarnings("unchecked") diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java index f7b5d524aad98..c8d37f23c7162 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner; import org.apache.flink.streaming.runtime.partitioner.KeyGroupStreamPartitioner; @@ -64,10 +65,15 @@ public class StreamExecExchange extends CommonExecExchange implements StreamExec public static final String EXCHANGE_TRANSFORMATION = "exchange"; - public StreamExecExchange(InputProperty inputProperty, RowType outputType, String description) { + public StreamExecExchange( + ReadableConfig tableConfig, + InputProperty inputProperty, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecExchange.class), + ExecNodeContext.newPersistedConfig(StreamExecExchange.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); @@ -77,10 +83,11 @@ public StreamExecExchange(InputProperty inputProperty, RowType outputType, Strin public StreamExecExchange( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java index d46c30f87f2ce..e1ec85678fc0d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -45,6 +46,7 @@ public class StreamExecExpand extends CommonExecExpand implements StreamExecNode { public StreamExecExpand( + ReadableConfig tableConfig, List> projects, InputProperty inputProperty, RowType outputType, @@ -52,6 +54,7 @@ public StreamExecExpand( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecExpand.class), + ExecNodeContext.newPersistedConfig(StreamExecExpand.class, tableConfig), projects, Collections.singletonList(inputProperty), outputType, @@ -62,6 +65,7 @@ public StreamExecExpand( public StreamExecExpand( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PROJECTS) List> projects, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @@ -69,6 +73,7 @@ public StreamExecExpand( super( id, context, + persistedConfig, projects, true, // retainHeader inputProperties, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java index 5d1b1db59620c..08dd8c146eeb1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.api.TableException; @@ -116,6 +117,7 @@ public class StreamExecGlobalGroupAggregate extends StreamExecAggregateBase { protected final Integer indexOfCountStar; public StreamExecGlobalGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -129,6 +131,8 @@ public StreamExecGlobalGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGlobalGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGlobalGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -145,6 +149,7 @@ public StreamExecGlobalGroupAggregate( public StreamExecGlobalGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions, @@ -155,7 +160,7 @@ public StreamExecGlobalGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java index e4a3ec764f95c..41c6a9c1b9dac 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -99,6 +100,7 @@ public class StreamExecGlobalWindowAggregate extends StreamExecWindowAggregateBa private final RowType localAggInputRowType; public StreamExecGlobalWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, WindowingStrategy windowing, @@ -110,6 +112,8 @@ public StreamExecGlobalWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGlobalWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGlobalWindowAggregate.class, tableConfig), grouping, aggCalls, windowing, @@ -124,6 +128,7 @@ public StreamExecGlobalWindowAggregate( public StreamExecGlobalWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing, @@ -133,7 +138,7 @@ public StreamExecGlobalWindowAggregate( @JsonProperty(FIELD_NAME_LOCAL_AGG_INPUT_ROW_TYPE) RowType localAggInputRowType, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.windowing = checkNotNull(windowing); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java index c698bd93056f0..2af340bda8df7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -102,6 +103,7 @@ public class StreamExecGroupAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -113,6 +115,7 @@ public StreamExecGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGroupAggregate.class), + ExecNodeContext.newPersistedConfig(StreamExecGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -127,6 +130,7 @@ public StreamExecGroupAggregate( public StreamExecGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions, @@ -135,7 +139,7 @@ public StreamExecGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java index cd759445a8a0b..d242deec9c4d5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -72,6 +73,7 @@ public class StreamExecGroupTableAggregate extends ExecNodeBase private final boolean needRetraction; public StreamExecGroupTableAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -83,6 +85,8 @@ public StreamExecGroupTableAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGroupTableAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGroupTableAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java index 3f96de3a5f65c..5c96b53896d25 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java @@ -138,6 +138,7 @@ public class StreamExecGroupWindowAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecGroupWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, LogicalWindow window, @@ -149,6 +150,8 @@ public StreamExecGroupWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGroupWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGroupWindowAggregate.class, tableConfig), grouping, aggCalls, window, @@ -163,6 +166,7 @@ public StreamExecGroupWindowAggregate( public StreamExecGroupWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOW) LogicalWindow window, @@ -172,7 +176,7 @@ public StreamExecGroupWindowAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java index 5f86176719e95..3a419e09638df 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.data.RowData; @@ -106,6 +107,7 @@ public class StreamExecIncrementalGroupAggregate extends StreamExecAggregateBase private final boolean partialAggNeedRetraction; public StreamExecIncrementalGroupAggregate( + ReadableConfig tableConfig, int[] partialAggGrouping, int[] finalAggGrouping, AggregateCall[] partialOriginalAggCalls, @@ -118,6 +120,8 @@ public StreamExecIncrementalGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecIncrementalGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecIncrementalGroupAggregate.class, tableConfig), partialAggGrouping, finalAggGrouping, partialOriginalAggCalls, @@ -133,6 +137,7 @@ public StreamExecIncrementalGroupAggregate( public StreamExecIncrementalGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PARTIAL_AGG_GROUPING) int[] partialAggGrouping, @JsonProperty(FIELD_NAME_FINAL_AGG_GROUPING) int[] finalAggGrouping, @JsonProperty(FIELD_NAME_PARTIAL_ORIGINAL_AGG_CALLS) @@ -144,7 +149,7 @@ public StreamExecIncrementalGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.partialAggGrouping = checkNotNull(partialAggGrouping); this.finalAggGrouping = checkNotNull(finalAggGrouping); this.partialOriginalAggCalls = checkNotNull(partialOriginalAggCalls); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java index f9794b18382c4..4c6cf12213831 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java @@ -96,6 +96,7 @@ public class StreamExecIntervalJoin extends ExecNodeBase private final IntervalJoinSpec intervalJoinSpec; public StreamExecIntervalJoin( + ReadableConfig tableConfig, IntervalJoinSpec intervalJoinSpec, InputProperty leftInputProperty, InputProperty rightInputProperty, @@ -104,6 +105,7 @@ public StreamExecIntervalJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecIntervalJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecIntervalJoin.class, tableConfig), intervalJoinSpec, Lists.newArrayList(leftInputProperty, rightInputProperty), outputType, @@ -114,11 +116,12 @@ public StreamExecIntervalJoin( public StreamExecIntervalJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INTERVAL_JOIN_SPEC) IntervalJoinSpec intervalJoinSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); Preconditions.checkArgument(inputProperties.size() == 2); this.intervalJoinSpec = Preconditions.checkNotNull(intervalJoinSpec); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java index 5fb1624498728..e14ef9a11b576 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -85,6 +86,7 @@ public class StreamExecJoin extends ExecNodeBase private final List rightUniqueKeys; public StreamExecJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, List leftUniqueKeys, List rightUniqueKeys, @@ -95,6 +97,7 @@ public StreamExecJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecJoin.class, tableConfig), joinSpec, leftUniqueKeys, rightUniqueKeys, @@ -107,13 +110,14 @@ public StreamExecJoin( public StreamExecJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec, @JsonProperty(FIELD_NAME_LEFT_UNIQUE_KEYS) List leftUniqueKeys, @JsonProperty(FIELD_NAME_RIGHT_UNIQUE_KEYS) List rightUniqueKeys, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 2); this.joinSpec = checkNotNull(joinSpec); this.leftUniqueKeys = leftUniqueKeys; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java index eeb68403c59b2..04fdde6c11b83 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -44,6 +45,7 @@ public class StreamExecLegacySink extends CommonExecLegacySink implements StreamExecNode { public StreamExecLegacySink( + ReadableConfig tableConfig, TableSink tableSink, @Nullable String[] upsertKeys, boolean needRetraction, @@ -53,6 +55,7 @@ public StreamExecLegacySink( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLegacySink.class), + ExecNodeContext.newPersistedConfig(StreamExecLegacySink.class, tableConfig), tableSink, upsertKeys, needRetraction, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java index c5b2128e32a0e..1bf47df754a6e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.InputSplit; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -63,6 +64,7 @@ public class StreamExecLegacyTableSourceScan extends CommonExecLegacyTableSource implements StreamExecNode { public StreamExecLegacyTableSourceScan( + ReadableConfig tableConfig, TableSource tableSource, List qualifiedName, RowType outputType, @@ -70,6 +72,8 @@ public StreamExecLegacyTableSourceScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLegacyTableSourceScan.class), + ExecNodeContext.newPersistedConfig( + StreamExecLegacyTableSourceScan.class, tableConfig), tableSource, qualifiedName, outputType, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java index 3e4bb09ba6565..2f673a04aedd7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -54,6 +55,7 @@ public class StreamExecLimit extends StreamExecRank { private final long limitEnd; public StreamExecLimit( + ReadableConfig tableConfig, long limitStart, long limitEnd, boolean generateUpdateBefore, @@ -64,6 +66,7 @@ public StreamExecLimit( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLimit.class), + ExecNodeContext.newPersistedConfig(StreamExecLimit.class, tableConfig), new ConstantRankRange(limitStart + 1, limitEnd), getRankStrategy(needRetraction), generateUpdateBefore, @@ -76,6 +79,7 @@ public StreamExecLimit( public StreamExecLimit( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_RANK_RANG) ConstantRankRange rankRange, @JsonProperty(FIELD_NAME_RANK_STRATEGY) RankProcessStrategy rankStrategy, @JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore, @@ -85,6 +89,7 @@ public StreamExecLimit( super( id, context, + persistedConfig, RankType.ROW_NUMBER, PartitionSpec.ALL_IN_ONE, SortSpec.ANY, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java index 80a257a9c24e8..aefdcb974ff5e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator; @@ -85,6 +86,7 @@ public class StreamExecLocalGroupAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecLocalGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -95,6 +97,8 @@ public StreamExecLocalGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLocalGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecLocalGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -108,6 +112,7 @@ public StreamExecLocalGroupAggregate( public StreamExecLocalGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions, @@ -115,7 +120,7 @@ public StreamExecLocalGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java index 0c17ac770889f..ae138297449ac 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.data.RowData; @@ -94,6 +95,7 @@ public class StreamExecLocalWindowAggregate extends StreamExecWindowAggregateBas private final WindowingStrategy windowing; public StreamExecLocalWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, WindowingStrategy windowing, @@ -103,6 +105,8 @@ public StreamExecLocalWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLocalWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecLocalWindowAggregate.class, tableConfig), grouping, aggCalls, windowing, @@ -115,13 +119,14 @@ public StreamExecLocalWindowAggregate( public StreamExecLocalWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.windowing = checkNotNull(windowing); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java index fdc80776201ca..8954958882726 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata; @@ -49,6 +50,7 @@ minStateVersion = FlinkVersion.v1_15) public class StreamExecLookupJoin extends CommonExecLookupJoin implements StreamExecNode { public StreamExecLookupJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, @Nullable RexNode joinCondition, TemporalTableSourceSpec temporalTableSourceSpec, @@ -61,6 +63,7 @@ public StreamExecLookupJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLookupJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecLookupJoin.class, tableConfig), joinType, joinCondition, temporalTableSourceSpec, @@ -76,6 +79,7 @@ public StreamExecLookupJoin( public StreamExecLookupJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType joinType, @JsonProperty(FIELD_NAME_JOIN_CONDITION) @Nullable RexNode joinCondition, @JsonProperty(FIELD_NAME_TEMPORAL_TABLE) @@ -91,6 +95,7 @@ public StreamExecLookupJoin( super( id, context, + persistedConfig, joinType, joinCondition, temporalTableSourceSpec, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java index 58f67116ecaa1..74b12c5050b20 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java @@ -111,6 +111,7 @@ public class StreamExecMatch extends ExecNodeBase private final MatchSpec matchSpec; public StreamExecMatch( + ReadableConfig tableConfig, MatchSpec matchSpec, InputProperty inputProperty, RowType outputType, @@ -118,6 +119,7 @@ public StreamExecMatch( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecMatch.class), + ExecNodeContext.newPersistedConfig(StreamExecMatch.class, tableConfig), matchSpec, Collections.singletonList(inputProperty), outputType, @@ -128,11 +130,12 @@ public StreamExecMatch( public StreamExecMatch( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_MATCH_SPEC) MatchSpec matchSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.matchSpec = checkNotNull(matchSpec); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java index 2bbc423214d59..e2a0da4e9c197 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -73,6 +74,7 @@ public class StreamExecMiniBatchAssigner extends ExecNodeBase private final MiniBatchInterval miniBatchInterval; public StreamExecMiniBatchAssigner( + ReadableConfig tableConfig, MiniBatchInterval miniBatchInterval, InputProperty inputProperty, RowType outputType, @@ -80,6 +82,7 @@ public StreamExecMiniBatchAssigner( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecMiniBatchAssigner.class), + ExecNodeContext.newPersistedConfig(StreamExecMiniBatchAssigner.class, tableConfig), miniBatchInterval, Collections.singletonList(inputProperty), outputType, @@ -90,11 +93,12 @@ public StreamExecMiniBatchAssigner( public StreamExecMiniBatchAssigner( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_MINI_BATCH_INTERVAL) MiniBatchInterval miniBatchInterval, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.miniBatchInterval = checkNotNull(miniBatchInterval); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java index b230f3cde2111..278feee81bd44 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -62,10 +63,14 @@ public class StreamExecMultipleInput extends ExecNodeBase private final ExecNode rootNode; public StreamExecMultipleInput( - List inputProperties, ExecNode rootNode, String description) { + ReadableConfig tableConfig, + List inputProperties, + ExecNode rootNode, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecMultipleInput.class), + ExecNodeContext.newPersistedConfig(StreamExecMultipleInput.class, tableConfig), inputProperties, rootNode.getOutputType(), description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java index a183b21bcab69..5eaf5b933739d 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -100,6 +101,7 @@ public class StreamExecOverAggregate extends ExecNodeBase private final OverSpec overSpec; public StreamExecOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -107,6 +109,7 @@ public StreamExecOverAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecOverAggregate.class), + ExecNodeContext.newPersistedConfig(StreamExecOverAggregate.class, tableConfig), overSpec, Collections.singletonList(inputProperty), outputType, @@ -117,11 +120,12 @@ public StreamExecOverAggregate( public StreamExecOverAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_OVER_SPEC) OverSpec overSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.overSpec = checkNotNull(overSpec); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java index 211221dd356b6..496ca96c1eb9a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -34,6 +35,7 @@ public class StreamExecPythonCalc extends CommonExecPythonCalc implements StreamExecNode { public StreamExecPythonCalc( + ReadableConfig tableConfig, List projection, InputProperty inputProperty, RowType outputType, @@ -41,6 +43,7 @@ public StreamExecPythonCalc( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonCalc.class), + ExecNodeContext.newPersistedConfig(StreamExecPythonCalc.class, tableConfig), projection, Collections.singletonList(inputProperty), outputType, @@ -50,10 +53,11 @@ public StreamExecPythonCalc( public StreamExecPythonCalc( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List projection, List inputProperties, RowType outputType, String description) { - super(id, context, projection, inputProperties, outputType, description); + super(id, context, persistedConfig, projection, inputProperties, outputType, description); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java index 992445cf9de05..0ce687a804c47 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; @@ -36,6 +37,7 @@ public class StreamExecPythonCorrelate extends CommonExecPythonCorrelate implements StreamExecNode { public StreamExecPythonCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, InputProperty inputProperty, @@ -44,6 +46,7 @@ public StreamExecPythonCorrelate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonCorrelate.class), + ExecNodeContext.newPersistedConfig(StreamExecPythonCorrelate.class, tableConfig), joinType, invocation, Collections.singletonList(inputProperty), @@ -54,6 +57,7 @@ public StreamExecPythonCorrelate( public StreamExecPythonCorrelate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, RexNode invocation, List inputProperties, @@ -62,6 +66,7 @@ public StreamExecPythonCorrelate( super( id, context, + persistedConfig, joinType, (RexCall) invocation, inputProperties, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java index 0a977a60aa556..d55ac1b7a23ca 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -78,6 +79,7 @@ public class StreamExecPythonGroupAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecPythonGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -89,6 +91,8 @@ public StreamExecPythonGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -103,6 +107,7 @@ public StreamExecPythonGroupAggregate( public StreamExecPythonGroupAggregate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -111,7 +116,7 @@ public StreamExecPythonGroupAggregate( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java index 583fafca69258..2303f03345c6a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -74,6 +75,7 @@ public class StreamExecPythonGroupTableAggregate extends ExecNodeBase private final boolean needRetraction; public StreamExecPythonGroupTableAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -85,6 +87,8 @@ public StreamExecPythonGroupTableAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonGroupTableAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonGroupTableAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java index 2ebc5e2122162..90905af0e82df 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -72,8 +73,6 @@ import org.apache.flink.table.runtime.util.TimeWindowUtil; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; - import org.apache.calcite.rel.core.AggregateCall; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -132,6 +131,7 @@ public class StreamExecPythonGroupWindowAggregate extends StreamExecAggregateBas private final boolean generateUpdateBefore; public StreamExecPythonGroupWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, LogicalWindow window, @@ -144,6 +144,8 @@ public StreamExecPythonGroupWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonGroupWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonGroupWindowAggregate.class, tableConfig), grouping, aggCalls, window, @@ -155,10 +157,10 @@ public StreamExecPythonGroupWindowAggregate( description); } - @JsonCreator public StreamExecPythonGroupWindowAggregate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, int[] grouping, AggregateCall[] aggCalls, LogicalWindow window, @@ -168,7 +170,7 @@ public StreamExecPythonGroupWindowAggregate( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java index 536a26abc7079..f24ba0a414f29 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -50,8 +51,6 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; - import org.apache.calcite.rel.core.AggregateCall; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,6 +92,7 @@ public class StreamExecPythonOverAggregate extends ExecNodeBase private final OverSpec overSpec; public StreamExecPythonOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -100,21 +100,23 @@ public StreamExecPythonOverAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonOverAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonOverAggregate.class, tableConfig), overSpec, Collections.singletonList(inputProperty), outputType, description); } - @JsonCreator public StreamExecPythonOverAggregate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, OverSpec overSpec, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.overSpec = checkNotNull(overSpec); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java index 247e735436bf0..1cb802ded8f0e 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.api.TableException; @@ -114,6 +115,7 @@ public class StreamExecRank extends ExecNodeBase private final boolean generateUpdateBefore; public StreamExecRank( + ReadableConfig tableConfig, RankType rankType, PartitionSpec partitionSpec, SortSpec sortSpec, @@ -127,6 +129,7 @@ public StreamExecRank( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecRank.class), + ExecNodeContext.newPersistedConfig(StreamExecRank.class, tableConfig), rankType, partitionSpec, sortSpec, @@ -143,6 +146,7 @@ public StreamExecRank( public StreamExecRank( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_RANK_TYPE) RankType rankType, @JsonProperty(FIELD_NAME_PARTITION_SPEC) PartitionSpec partitionSpec, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @@ -153,7 +157,7 @@ public StreamExecRank( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.rankType = checkNotNull(rankType); this.rankRange = checkNotNull(rankRange); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java index 1d19b456eb94f..40acb2f964222 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -82,6 +83,7 @@ public class StreamExecSink extends CommonExecSink implements StreamExecNode inputProperties, @@ -112,6 +116,7 @@ public StreamExecSink( super( id, context, + persistedConfig, tableSinkSpec, inputChangelogMode, false, // isBounded diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java index 6d5424c0e2757..cf8e112b43e67 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator; @@ -51,6 +52,7 @@ public class StreamExecSort extends ExecNodeBase implements StreamExecN private final SortSpec sortSpec; public StreamExecSort( + ReadableConfig tableConfig, SortSpec sortSpec, InputProperty inputProperty, RowType outputType, @@ -58,6 +60,7 @@ public StreamExecSort( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecSort.class), + ExecNodeContext.newPersistedConfig(StreamExecSort.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java index ab3f5b7bcdba4..fd36729592264 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -53,6 +54,7 @@ public class StreamExecSortLimit extends StreamExecRank { private final long limitEnd; public StreamExecSortLimit( + ReadableConfig tableConfig, SortSpec sortSpec, long limitStart, long limitEnd, @@ -64,6 +66,7 @@ public StreamExecSortLimit( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecSortLimit.class), + ExecNodeContext.newPersistedConfig(StreamExecSortLimit.class, tableConfig), sortSpec, new ConstantRankRange(limitStart + 1, limitEnd), rankStrategy, @@ -77,6 +80,7 @@ public StreamExecSortLimit( public StreamExecSortLimit( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @JsonProperty(FIELD_NAME_RANK_RANG) ConstantRankRange rankRange, @JsonProperty(FIELD_NAME_RANK_STRATEGY) RankProcessStrategy rankStrategy, @@ -88,6 +92,7 @@ public StreamExecSortLimit( super( id, context, + persistedConfig, RankType.ROW_NUMBER, PartitionSpec.ALL_IN_ONE, sortSpec, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java index d7f642baf65c7..9fb292bd96030 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.connector.source.ScanTableSource; import org.apache.flink.table.data.RowData; @@ -49,10 +50,14 @@ public class StreamExecTableSourceScan extends CommonExecTableSourceScan implements StreamExecNode { public StreamExecTableSourceScan( - DynamicTableSourceSpec tableSourceSpec, RowType outputType, String description) { + ReadableConfig tableConfig, + DynamicTableSourceSpec tableSourceSpec, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecTableSourceScan.class), + ExecNodeContext.newPersistedConfig(StreamExecTableSourceScan.class, tableConfig), tableSourceSpec, outputType, description); @@ -62,10 +67,11 @@ public StreamExecTableSourceScan( public StreamExecTableSourceScan( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_SCAN_TABLE_SOURCE) DynamicTableSourceSpec tableSourceSpec, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, tableSourceSpec, outputType, description); + super(id, context, persistedConfig, tableSourceSpec, outputType, description); } @Override diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java index 566e4dcd4b401..6797ec8406fb6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.table.api.TableException; @@ -97,6 +98,7 @@ public class StreamExecTemporalJoin extends ExecNodeBase private final int rightTimeAttributeIndex; public StreamExecTemporalJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, boolean isTemporalTableFunctionJoin, int leftTimeAttributeIndex, @@ -108,6 +110,7 @@ public StreamExecTemporalJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecTemporalJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecTemporalJoin.class, tableConfig), joinSpec, isTemporalTableFunctionJoin, leftTimeAttributeIndex, @@ -121,6 +124,7 @@ public StreamExecTemporalJoin( public StreamExecTemporalJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec, @JsonProperty(FIELD_NAME_IS_TEMPORAL_FUNCTION_JOIN) boolean isTemporalTableFunctionJoin, @JsonProperty(FIELD_NAME_LEFT_TIME_ATTRIBUTE_INDEX) int leftTimeAttributeIndex, @@ -128,7 +132,7 @@ public StreamExecTemporalJoin( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); Preconditions.checkArgument(inputProperties.size() == 2); Preconditions.checkArgument( rightTimeAttributeIndex == FIELD_INDEX_FOR_PROC_TIME_ATTRIBUTE diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java index ed5d49c4827f2..c0c88a511b06c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -71,6 +72,7 @@ public class StreamExecTemporalSort extends ExecNodeBase private final SortSpec sortSpec; public StreamExecTemporalSort( + ReadableConfig tableConfig, SortSpec sortSpec, InputProperty inputProperty, RowType outputType, @@ -78,6 +80,7 @@ public StreamExecTemporalSort( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecTemporalSort.class), + ExecNodeContext.newPersistedConfig(StreamExecTemporalSort.class, tableConfig), sortSpec, Collections.singletonList(inputProperty), outputType, @@ -88,11 +91,12 @@ public StreamExecTemporalSort( public StreamExecTemporalSort( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.sortSpec = checkNotNull(sortSpec); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java index 68c9f9b4aa20b..0cfbde1fb938f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -44,10 +45,14 @@ public class StreamExecUnion extends CommonExecUnion implements StreamExecNode { public StreamExecUnion( - List inputProperties, RowType outputType, String description) { + ReadableConfig tableConfig, + List inputProperties, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecUnion.class), + ExecNodeContext.newPersistedConfig(StreamExecUnion.class, tableConfig), inputProperties, outputType, description); @@ -57,9 +62,10 @@ public StreamExecUnion( public StreamExecUnion( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java index 3b6c984d51614..9ace41eaf896c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext; @@ -42,10 +43,15 @@ minStateVersion = FlinkVersion.v1_15) public class StreamExecValues extends CommonExecValues implements StreamExecNode { - public StreamExecValues(List> tuples, RowType outputType, String description) { + public StreamExecValues( + ReadableConfig tableConfig, + List> tuples, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecValues.class), + ExecNodeContext.newPersistedConfig(StreamExecValues.class, tableConfig), tuples, outputType, description); @@ -55,9 +61,10 @@ public StreamExecValues(List> tuples, RowType outputType, Strin public StreamExecValues( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_TUPLES) List> tuples, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, tuples, outputType, description); + super(id, context, persistedConfig, tuples, outputType, description); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java index 177ea4823cc01..f79488a05d5df 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.WatermarkGeneratorCodeGenerator; @@ -73,6 +74,7 @@ public class StreamExecWatermarkAssigner extends ExecNodeBase private final int rowtimeFieldIndex; public StreamExecWatermarkAssigner( + ReadableConfig tableConfig, RexNode watermarkExpr, int rowtimeFieldIndex, InputProperty inputProperty, @@ -81,6 +83,7 @@ public StreamExecWatermarkAssigner( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWatermarkAssigner.class), + ExecNodeContext.newPersistedConfig(StreamExecWatermarkAssigner.class, tableConfig), watermarkExpr, rowtimeFieldIndex, Collections.singletonList(inputProperty), @@ -92,12 +95,13 @@ public StreamExecWatermarkAssigner( public StreamExecWatermarkAssigner( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_WATERMARK_EXPR) RexNode watermarkExpr, @JsonProperty(FIELD_NAME_ROWTIME_FIELD_INDEX) int rowtimeFieldIndex, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.watermarkExpr = checkNotNull(watermarkExpr); this.rowtimeFieldIndex = rowtimeFieldIndex; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java index b5a50bf3baaab..24fc63821be71 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -104,6 +105,7 @@ public class StreamExecWindowAggregate extends StreamExecWindowAggregateBase { private final NamedWindowProperty[] namedWindowProperties; public StreamExecWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, WindowingStrategy windowing, @@ -114,6 +116,7 @@ public StreamExecWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowAggregate.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowAggregate.class, tableConfig), grouping, aggCalls, windowing, @@ -127,6 +130,7 @@ public StreamExecWindowAggregate( public StreamExecWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing, @@ -135,7 +139,7 @@ public StreamExecWindowAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); this.grouping = checkNotNull(grouping); this.aggCalls = checkNotNull(aggCalls); this.windowing = checkNotNull(windowing); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java index 59adf5531b484..b9a01a041f269 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.logical.CumulativeWindowSpec; import org.apache.flink.table.planner.plan.logical.HoppingWindowSpec; @@ -53,10 +54,11 @@ public abstract class StreamExecWindowAggregateBase extends StreamExecAggregateB protected StreamExecWindowAggregateBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, LogicalType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java index 35091f74901b4..4f8dd6087e0ca 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -90,6 +91,7 @@ public class StreamExecWindowDeduplicate extends ExecNodeBase private final WindowingStrategy windowing; public StreamExecWindowDeduplicate( + ReadableConfig tableConfig, int[] partitionKeys, int orderKey, boolean keepLastRow, @@ -100,6 +102,7 @@ public StreamExecWindowDeduplicate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowDeduplicate.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowDeduplicate.class, tableConfig), partitionKeys, orderKey, keepLastRow, @@ -113,6 +116,7 @@ public StreamExecWindowDeduplicate( public StreamExecWindowDeduplicate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PARTITION_KEYS) int[] partitionKeys, @JsonProperty(FIELD_NAME_ORDER_KEY) int orderKey, @JsonProperty(FIELD_NAME_KEEP_LAST_ROW) boolean keepLastRow, @@ -120,7 +124,7 @@ public StreamExecWindowDeduplicate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.partitionKeys = checkNotNull(partitionKeys); this.orderKey = orderKey; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java index 1b726d21c9099..750d822cce902 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -84,6 +85,7 @@ public class StreamExecWindowJoin extends ExecNodeBase private final WindowingStrategy rightWindowing; public StreamExecWindowJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, WindowingStrategy leftWindowing, WindowingStrategy rightWindowing, @@ -94,6 +96,7 @@ public StreamExecWindowJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowJoin.class, tableConfig), joinSpec, leftWindowing, rightWindowing, @@ -106,13 +109,14 @@ public StreamExecWindowJoin( public StreamExecWindowJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec, @JsonProperty(FIELD_NAME_LEFT_WINDOWING) WindowingStrategy leftWindowing, @JsonProperty(FIELD_NAME_RIGHT_WINDOWING) WindowingStrategy rightWindowing, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 2); this.joinSpec = checkNotNull(joinSpec); validate(leftWindowing); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java index 2cfb5d8031313..b6911385e4ca5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -106,6 +107,7 @@ public class StreamExecWindowRank extends ExecNodeBase private final WindowingStrategy windowing; public StreamExecWindowRank( + ReadableConfig tableConfig, RankType rankType, PartitionSpec partitionSpec, SortSpec sortSpec, @@ -118,6 +120,7 @@ public StreamExecWindowRank( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowRank.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowRank.class, tableConfig), rankType, partitionSpec, sortSpec, @@ -133,6 +136,7 @@ public StreamExecWindowRank( public StreamExecWindowRank( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_RANK_TYPE) RankType rankType, @JsonProperty(FIELD_NAME_PARTITION_SPEC) PartitionSpec partitionSpec, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @@ -142,7 +146,7 @@ public StreamExecWindowRank( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.rankType = checkNotNull(rankType); this.partitionSpec = checkNotNull(partitionSpec); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java index 38ea6ee89a2a0..7728712a6af29 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -51,6 +52,7 @@ public class StreamExecWindowTableFunction extends CommonExecWindowTableFunction implements StreamExecNode { public StreamExecWindowTableFunction( + ReadableConfig tableConfig, TimeAttributeWindowingStrategy windowingStrategy, InputProperty inputProperty, RowType outputType, @@ -58,6 +60,8 @@ public StreamExecWindowTableFunction( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowTableFunction.class), + ExecNodeContext.newPersistedConfig( + StreamExecWindowTableFunction.class, tableConfig), windowingStrategy, Collections.singletonList(inputProperty), outputType, @@ -68,10 +72,18 @@ public StreamExecWindowTableFunction( public StreamExecWindowTableFunction( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_WINDOWING) TimeAttributeWindowingStrategy windowingStrategy, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, windowingStrategy, inputProperties, outputType, description); + super( + id, + context, + persistedConfig, + windowingStrategy, + inputProperties, + outputType, + description); } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java index c295e31482683..2da4fffb1c46f 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java @@ -20,6 +20,12 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata; import org.apache.flink.table.planner.plan.nodes.exec.MultipleExecNodeMetadata; @@ -73,7 +79,10 @@ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowRank; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowTableFunction; +import javax.annotation.Nullable; + import java.util.ArrayList; +import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -81,6 +90,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.stream.Stream; /** Utility class for {@link ExecNodeMetadata} related functionality. */ @Internal @@ -159,6 +169,18 @@ private ExecNodeMetadataUtil() { } }; + public static final Set> TABLE_CONFIG_OPTIONS; + + static { + TABLE_CONFIG_OPTIONS = ConfigUtils.getAllConfigOptions(TableConfigOptions.class); + } + + public static final Set> EXECUTION_CONFIG_OPTIONS; + + static { + EXECUTION_CONFIG_OPTIONS = ConfigUtils.getAllConfigOptions(ExecutionConfigOptions.class); + } + public static Set>> execNodes() { return EXEC_NODES; } @@ -254,6 +276,63 @@ public static > ExecNodeMetadata latestAnnotation( return sortedAnnotations.get(sortedAnnotations.size() - 1); } + @Nullable + public static > String[] consumedOptions(Class execNodeClass) { + ExecNodeMetadata metadata = latestAnnotation(execNodeClass); + if (metadata == null) { + return null; + } + return metadata.consumedOptions(); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + public static > ReadableConfig newPersistedConfig( + Class execNodeClass, + ReadableConfig tableConfig, + Stream> configOptions) { + final Map> availableConfigOptions = new HashMap<>(); + configOptions.forEach( + co -> { + availableConfigOptions.put(co.key(), co); + co.fallbackKeys().forEach(k -> availableConfigOptions.put(k.getKey(), co)); + }); + + final Configuration persistedConfig = new Configuration(); + final String[] consumedOptions = ExecNodeMetadataUtil.consumedOptions(execNodeClass); + if (consumedOptions == null) { + return persistedConfig; + } + + final Map nodeConfigOptions = new HashMap<>(); + for (final String consumedOption : consumedOptions) { + ConfigOption configOption = availableConfigOptions.get(consumedOption); + if (configOption == null) { + throw new IllegalStateException( + String.format( + "ExecNode: %s, consumedOption: %s not listed in [%s].", + execNodeClass.getCanonicalName(), + consumedOption, + String.join( + ", ", + Arrays.asList( + TableConfigOptions.class.getSimpleName(), + ExecutionConfigOptions.class.getSimpleName())))); + } + if (nodeConfigOptions.containsKey(configOption)) { + throw new IllegalStateException( + String.format( + "ExecNode: %s, consumedOption: %s is listed multiple times in " + + "consumedOptions, potentially also with " + + "fallback/deprecated key.", + execNodeClass.getCanonicalName(), consumedOption)); + } else { + nodeConfigOptions.put(configOption, tableConfig.get(configOption)); + } + } + nodeConfigOptions.forEach(persistedConfig::set); + return persistedConfig; + } + /** Helper Pojo used as a tuple for the {@link #LOOKUP_MAP}. */ private static final class ExecNodeNameVersion { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala index 12282936c2f0b..8122033837cac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecBoundedStreamScan import org.apache.flink.table.planner.plan.schema.DataStreamTable import org.apache.flink.table.planner.plan.utils.RelExplainUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.`type`.RelDataType @@ -71,6 +72,7 @@ class BatchPhysicalBoundedStreamScan( override def translateToExecNode(): ExecNode[_] = { new BatchExecBoundedStreamScan( + unwrapTableConfig(this), boundedStreamTable.dataStream, boundedStreamTable.dataType, boundedStreamTable.fieldIndexes, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala index 6fefa39f436e8..2c62493ce4dd7 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecCalc -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel._ @@ -54,6 +55,7 @@ class BatchPhysicalCalc( } new BatchExecCalc( + unwrapTableConfig(this), projection, condition, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala index f6dc6105e16f6..6ba5e703e1b61 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala @@ -19,9 +19,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -65,6 +66,7 @@ class BatchPhysicalCorrelate( override def translateToExecNode(): ExecNode[_] = { new BatchExecCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], condition.orNull, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala index e58063f96127c..cd28172d21621 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala @@ -49,6 +49,7 @@ class BatchPhysicalExchange( override def translateToExecNode(): ExecNode[_] = { new BatchExecExchange( + unwrapTableConfig(this), getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala index 237929433b7cb..99905c0b8f99f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.calcite.Expand import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecExpand -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -53,10 +54,10 @@ class BatchPhysicalExpand( override def translateToExecNode(): ExecNode[_] = { new BatchExecExpand( + unwrapTableConfig(this), projects, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala index ef2fae7438ac8..d0cc5bfacd49a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala @@ -22,9 +22,10 @@ import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, SINGLETON} @@ -148,6 +149,7 @@ class BatchPhysicalHashAggregate( InputProperty.hashDistribution(grouping) } new BatchExecHashAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -159,7 +161,6 @@ class BatchPhysicalHashAggregate( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala index 1606874fe7dde..708208128448c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashJoin import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, JoinUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.join.HashJoinType import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer @@ -166,18 +167,18 @@ class BatchPhysicalHashJoin( val rightRowCount = Util.first(mq.getRowCount(right), 200000).toLong val (leftEdge, rightEdge) = getInputProperties new BatchExecHashJoin( - joinSpec, - leftRowSize, - rightRowSize, - leftRowCount, - rightRowCount, - leftIsBuild, - tryDistinctBuildRow, - leftEdge, - rightEdge, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + unwrapTableConfig(this), + joinSpec, + leftRowSize, + rightRowSize, + leftRowCount, + rightRowCount, + leftIsBuild, + tryDistinctBuildRow, + leftEdge, + rightEdge, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } private def getInputProperties: (InputProperty, InputProperty) = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala index 5838941603f35..b47f36b2d2216 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -89,6 +90,7 @@ class BatchPhysicalHashWindowAggregate( InputProperty.hashDistribution(grouping) } new BatchExecHashWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -105,7 +107,6 @@ class BatchPhysicalHashWindowAggregate( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala index fcdb53a9591d3..28d1445493e5b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.LegacySink import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecLegacySink import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.UpdatingPlanChecker +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.sinks.{TableSink, UpsertStreamTableSink} @@ -57,13 +58,13 @@ class BatchPhysicalLegacySink[T]( case _ => Option.empty[Array[String]] } new BatchExecLegacySink[T]( + unwrapTableConfig(this), sink, upsertKeys.orNull, // the input records will not trigger any output of a sink because it has no output, // so it's dam behavior is BLOCKING InputProperty.builder().damBehavior(InputProperty.DamBehavior.BLOCKING).build(), fromDataTypeToLogicalType(sink.getConsumedDataType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala index 8778505f8a1ab..be480d501148d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecLegacyTableSourceScan import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLegacyTableSourceScan import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.sources.StreamTableSource import org.apache.calcite.plan._ @@ -61,6 +62,7 @@ class BatchPhysicalLegacyTableSourceScan( override def translateToExecNode(): ExecNode[_] = { new BatchExecLegacyTableSourceScan( + unwrapTableConfig(this), tableSource, getTable.getQualifiedName, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala index d3ee2fa4026d3..eee744c6d5ad1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala @@ -22,9 +22,10 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.cost.FlinkCost._ import org.apache.flink.table.planner.plan.cost.FlinkCostFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.RelExplainUtil.fetchToString import org.apache.flink.table.planner.plan.utils.SortUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel._ @@ -81,11 +82,12 @@ class BatchPhysicalLimit( override def translateToExecNode(): ExecNode[_] = { new BatchExecLimit( - limitStart, - limitEnd, - isGlobal, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription) + unwrapTableConfig(this), + limitStart, + limitEnd, + isGlobal, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala index 98e3da6fed37f..0359472912356 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala @@ -22,8 +22,9 @@ import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.RelExplainUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type @@ -121,6 +122,7 @@ class BatchPhysicalLocalHashAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecHashAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -129,8 +131,7 @@ class BatchPhysicalLocalHashAggregate( false, // isFinal is always false getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def getInputProperty: InputProperty = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala index 7d81fe048f2a9..cf3a735ac17fd 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -82,6 +83,7 @@ class BatchPhysicalLocalHashWindowAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecHashWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -95,7 +97,6 @@ class BatchPhysicalLocalHashWindowAggregate( false, // isFinal is always false InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala index 59faaf0483ca7..93cf6b8ada28f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala @@ -22,8 +22,9 @@ import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type @@ -129,6 +130,7 @@ class BatchPhysicalLocalSortAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -137,8 +139,7 @@ class BatchPhysicalLocalSortAggregate( false, // isFinal is always false getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def getInputProperty: InputProperty = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala index 9a5fdca397f09..05a1902a1ccd7 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -80,6 +81,7 @@ class BatchPhysicalLocalSortWindowAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -93,7 +95,6 @@ class BatchPhysicalLocalSortWindowAggregate( false, // isFinal is always false InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala index e66a32c45368d..900e19fd0e35a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLookupJoin import org.apache.flink.table.planner.plan.utils.{FlinkRexUtil, JoinTypeUtil} import org.apache.flink.table.planner.utils.JavaScalaConversionUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptTable, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -77,6 +78,7 @@ class BatchPhysicalLookupJoin( } new BatchExecLookupJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), remainingCondition.orNull, new TemporalTableSourceSpec(temporalTable), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala index 6faa2b92434a3..93119021dc464 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecNestedLoopJoin import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer import org.apache.calcite.plan._ @@ -114,6 +115,7 @@ class BatchPhysicalNestedLoopJoin( override def translateToExecNode(): ExecNode[_] = { val (leftInputProperty, rightInputProperty) = getInputProperties new BatchExecNestedLoopJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), condition, leftIsBuild, @@ -121,8 +123,7 @@ class BatchPhysicalNestedLoopJoin( leftInputProperty, rightInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } def getInputProperties: (InputProperty, InputProperty) = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala index 5260dfe6e0d93..a45481462627f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecOverAggrega import org.apache.flink.table.planner.plan.nodes.exec.spec.{OverSpec, PartitionSpec} import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel._ @@ -71,6 +72,7 @@ class BatchPhysicalOverAggregate( InputProperty.hashDistribution(partitionKeyIndices) } new BatchExecOverAggregate( + unwrapTableConfig(this), new OverSpec( new PartitionSpec(partitionKeyIndices), offsetAndInsensitiveSensitiveGroups.map(OverAggregateUtil.createGroupSpec(_, logicWindow)), @@ -78,7 +80,6 @@ class BatchPhysicalOverAggregate( OverAggregateUtil.calcOriginalInputFields(logicWindow)), InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala index a32457ba7da71..0fa260bf346ac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonCalc import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -58,6 +59,7 @@ class BatchPhysicalPythonCalc( } new BatchExecPythonCalc( + unwrapTableConfig(this), projection, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala index 298476c0ce1df..3d59cea816f5d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala @@ -20,9 +20,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -74,11 +75,11 @@ class BatchPhysicalPythonCorrelate( } new BatchExecPythonCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala index b5a2b071851ed..136c7260cb2ea 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonGroupAggregate import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, SINGLETON} @@ -92,7 +93,7 @@ class BatchPhysicalPythonGroupAggregate( true } else { // If partialKey is enabled, try to use partial key to satisfy the required distribution - val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this) + val tableConfig = unwrapTableConfig(this) val partialKeyEnabled = tableConfig.get( BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED) partialKeyEnabled && groupKeysList.containsAll(shuffleKeys) @@ -156,6 +157,7 @@ class BatchPhysicalPythonGroupAggregate( InputProperty.hashDistribution(grouping) } new BatchExecPythonGroupAggregate( + unwrapTableConfig(this), grouping, grouping ++ auxGrouping, aggCalls.toArray, @@ -164,8 +166,7 @@ class BatchPhysicalPythonGroupAggregate( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala index ae2c2f57b9373..09e9fcdf9b746 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} @@ -104,6 +105,7 @@ class BatchPhysicalPythonGroupWindowAggregate( InputProperty.hashDistribution(grouping) } new BatchExecPythonGroupWindowAggregate( + unwrapTableConfig(this), grouping, grouping ++ auxGrouping, aggCalls.toArray, @@ -112,7 +114,6 @@ class BatchPhysicalPythonGroupWindowAggregate( namedWindowProperties.toArray, InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala index 9f42ab41ee2ed..63535c045f0a9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonOverA import org.apache.flink.table.planner.plan.nodes.exec.spec.{OverSpec, PartitionSpec} import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel._ @@ -70,6 +71,7 @@ class BatchPhysicalPythonOverAggregate( InputProperty.hashDistribution(partitionKeyIndices) } new BatchExecPythonOverAggregate( + unwrapTableConfig(this), new OverSpec( new PartitionSpec(partitionKeyIndices), offsetAndInsensitiveSensitiveGroups.map(OverAggregateUtil.createGroupSpec(_, logicWindow)), @@ -77,7 +79,6 @@ class BatchPhysicalPythonOverAggregate( OverAggregateUtil.calcOriginalInputFields(logicWindow)), InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala index 900f155b1ab8c..1d3c5720148b9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecRank import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.rank.{ConstantRankRange, RankRange, RankType} import org.apache.calcite.plan._ @@ -234,6 +235,7 @@ class BatchPhysicalRank( InputProperty.hashDistribution(partitionKey.toArray) } new BatchExecRank( + unwrapTableConfig(this), partitionKey.toArray, orderKey.getFieldCollations.map(_.getFieldIndex).toArray, rankStart, @@ -241,7 +243,6 @@ class BatchPhysicalRank( outputRankNumber, InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala index 1400b0990aad1..566fae152bc0f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch -import org.apache.flink.table.catalog.{CatalogTable, ContextResolvedTable, ObjectIdentifier, ResolvedCatalogTable} +import org.apache.flink.table.catalog.ContextResolvedTable import org.apache.flink.table.connector.sink.DynamicTableSink import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.abilities.sink.SinkAbilitySpec @@ -26,7 +26,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.Sink import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSink import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSinkSpec import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} -import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -67,12 +67,12 @@ class BatchPhysicalSink( tableSinkSpec.setTableSink(tableSink) new BatchExecSink( + unwrapTableConfig(this), tableSinkSpec, // the input records will not trigger any output of a sink because it has no output, // so it's dam behavior is BLOCKING InputProperty.builder().damBehavior(InputProperty.DamBehavior.BLOCKING).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala index c1ac0b732eaa2..73236ee2d551d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala @@ -21,8 +21,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSort -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel.core.Sort @@ -79,10 +80,10 @@ class BatchPhysicalSort( @Override override def translateToExecNode(): ExecNode[_] = { new BatchExecSort( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), InputProperty.builder().damBehavior(InputProperty.DamBehavior.END_INPUT).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala index cedc67b616377..cd9ebd078775d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortAggrega import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, SINGLETON} @@ -155,6 +156,7 @@ class BatchPhysicalSortAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -163,8 +165,7 @@ class BatchPhysicalSortAggregate( true, // isFinal is always true getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def getInputProperty: InputProperty = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala index 16228f1a72987..861d7d9811bf0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala @@ -21,8 +21,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel.core.Sort @@ -100,13 +101,13 @@ class BatchPhysicalSortLimit( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortLimit( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), limitStart, limitEnd, isGlobal, InputProperty.builder().damBehavior(InputProperty.DamBehavior.END_INPUT).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala index c2bfa50550f47..909489de3459b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortMergeJoin import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, FlinkRelOptUtil, JoinTypeUtil, JoinUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.join.FlinkJoinType import org.apache.calcite.plan._ @@ -173,6 +174,7 @@ class BatchPhysicalSortMergeJoin( FlinkTypeFactory.toLogicalRowType(right.getRowType)) new BatchExecSortMergeJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), joinSpec.getLeftKeys, joinSpec.getRightKeys, @@ -188,8 +190,7 @@ class BatchPhysicalSortMergeJoin( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def estimateOutputSize(relNode: RelNode): Double = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala index a013e283e6d4d..f48e51e20f354 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -85,6 +86,7 @@ class BatchPhysicalSortWindowAggregate( InputProperty.hashDistribution(grouping) } new BatchExecSortWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -98,7 +100,6 @@ class BatchPhysicalSortWindowAggregate( true, // isFinal is always true InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala index e5f0ad2c7f602..7ab2bf7f7c945 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecTableSource import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpec import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalTableSourceScan import org.apache.flink.table.planner.plan.schema.TableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode @@ -72,6 +73,7 @@ class BatchPhysicalTableSourceScan( tableSourceSpec.setTableSource(tableSourceTable.tableSource) new BatchExecTableSourceScan( + unwrapTableConfig(this), tableSourceSpec, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala index 9d1d9986f0524..a440b01c71c66 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecUnion -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type._ @@ -95,9 +96,9 @@ class BatchPhysicalUnion( override def translateToExecNode(): ExecNode[_] = { new BatchExecUnion( + unwrapTableConfig(this), getInputs.map(_ => InputProperty.DEFAULT), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala index 23ff9dc89e12a..fbbcef7edc041 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecValues +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import com.google.common.collect.ImmutableList import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -57,9 +58,9 @@ class BatchPhysicalValues( override def translateToExecNode(): ExecNode[_] = { new BatchExecValues( + unwrapTableConfig(this), tuples.asList().map(_.asList()), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala index 0c888332bd951..1b75542827333 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrateg import org.apache.flink.table.planner.plan.nodes.common.CommonPhysicalWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -58,11 +59,11 @@ class BatchPhysicalWindowTableFunction( override def translateToExecNode(): ExecNode[_] = { new BatchExecWindowTableFunction( + unwrapTableConfig(this), windowing, // TODO set semantic window (such as session window) require other Dam Behavior InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala index 6603e1b515b3b..396765d8adc50 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -54,6 +55,7 @@ class StreamPhysicalCalc( } new StreamExecCalc( + unwrapTableConfig(this), projection, condition, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala index ce06e55379f65..7b5ff0e45981f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecChangelogNormalize import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -67,6 +68,7 @@ class StreamPhysicalChangelogNormalize( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecChangelogNormalize( + unwrapTableConfig(this), uniqueKeys, generateUpdateBefore, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala index 710c4df7959fa..5fbdfc33b0307 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala @@ -19,9 +19,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -65,6 +66,7 @@ class StreamPhysicalCorrelate( override def translateToExecNode(): ExecNode[_] = { new StreamExecCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], condition.orNull, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala index c80cc0d40fbbb..d3b95bf5575b9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDataStreamScan import org.apache.flink.table.planner.plan.schema.DataStreamTable import org.apache.flink.table.planner.plan.utils.RelExplainUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.`type`.RelDataType @@ -73,6 +74,7 @@ class StreamPhysicalDataStreamScan( override def translateToExecNode(): ExecNode[_] = { new StreamExecDataStreamScan( + unwrapTableConfig(this), dataStreamTable.dataStream, dataStreamTable.dataType, dataStreamTable.fieldIndexes, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala index ddc772ca68848..ae1cabcfc97a0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDeduplicate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -73,13 +74,13 @@ class StreamPhysicalDeduplicate( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecDeduplicate( + unwrapTableConfig(this), uniqueKeys, isRowtime, keepLastRow, generateUpdateBefore, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala index ba24f6e64cf83..3a9e75209e37e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDropUpdateBefore -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -60,9 +61,9 @@ class StreamPhysicalDropUpdateBefore( } new StreamExecDropUpdateBefore( + unwrapTableConfig(this), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala index e07caee902c9e..91c272cc8753b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalExchange +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{RelDistribution, RelNode} @@ -48,6 +49,7 @@ class StreamPhysicalExchange( override def translateToExecNode(): ExecNode[_] = { new StreamExecExchange( + unwrapTableConfig(this), InputProperty.builder.requiredDistribution(getRequiredDistribution).build, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala index 9e0c414c439a8..6cbfc613b1149 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.calcite.Expand import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExpand -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -49,10 +50,10 @@ class StreamPhysicalExpand( override def translateToExecNode(): ExecNode[_] = { new StreamExecExpand( + unwrapTableConfig(this), projects, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala index 2e4dbb05f70fd..8f557037ddfc1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.PartialFinalType import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -104,6 +105,7 @@ class StreamPhysicalGlobalGroupAggregate( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecGlobalGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -113,7 +115,6 @@ class StreamPhysicalGlobalGroupAggregate( indexOfCountStar.map(Integer.valueOf).orNull, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala index e6cffd41dbb08..f55c08fc27f3d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.rules.physical.stream.TwoStageOptimizedWindowAggregateRule import org.apache.flink.table.planner.plan.utils.WindowUtil.checkEmitConfiguration import org.apache.flink.table.planner.plan.utils.{AggregateUtil, FlinkRelOptUtil, RelExplainUtil, WindowUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -128,6 +129,7 @@ class StreamPhysicalGlobalWindowAggregate( override def translateToExecNode(): ExecNode[_] = { checkEmitConfiguration(FlinkRelOptUtil.getTableConfigFromContext(this)) new StreamExecGlobalWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, windowing, @@ -135,7 +137,6 @@ class StreamPhysicalGlobalWindowAggregate( InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(inputRowTypeOfLocalAgg), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala index 07bca07984748..c80ea2efa7e22 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.PartialFinalType import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -84,6 +85,7 @@ class StreamPhysicalGroupAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -91,7 +93,6 @@ class StreamPhysicalGroupAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala index 5097a404a42ca..a94868b54c97b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupTableAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -63,6 +64,7 @@ class StreamPhysicalGroupTableAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupTableAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -70,7 +72,6 @@ class StreamPhysicalGroupTableAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala index 679fc26fa273b..a24b8817ad757 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical._ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, WindowEmitStrategy} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -75,6 +76,7 @@ class StreamPhysicalGroupWindowAggregate( override def translateToExecNode(): ExecNode[_] = { val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, window, @@ -82,7 +84,6 @@ class StreamPhysicalGroupWindowAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala index bdee5f655d06b..cc073cb2ebb5b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.logical._ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, WindowEmitStrategy} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -75,6 +76,7 @@ class StreamPhysicalGroupWindowTableAggregate( override def translateToExecNode(): ExecNode[_] = { val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, window, @@ -82,7 +84,6 @@ class StreamPhysicalGroupWindowTableAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala index 48a2c999208a5..be37d1fef3931 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIncrementalGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -118,6 +119,7 @@ class StreamPhysicalIncrementalGroupAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecIncrementalGroupAggregate( + unwrapTableConfig(this), partialAggGrouping, finalAggGrouping, partialOriginalAggCalls, @@ -126,7 +128,6 @@ class StreamPhysicalIncrementalGroupAggregate( partialAggNeedRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala index 9b26cd447e65c..dd4640d597361 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.core.{Join, JoinRelType} @@ -92,10 +93,11 @@ class StreamPhysicalIntervalJoin( override def translateToExecNode(): ExecNode[_] = { new StreamExecIntervalJoin( - new IntervalJoinSpec(joinSpec, windowBounds), - InputProperty.DEFAULT, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription) + unwrapTableConfig(this), + new IntervalJoinSpec(joinSpec, windowBounds), + InputProperty.DEFAULT, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala index f4b68a9fde8cf..0d33d411774c6 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala @@ -20,10 +20,11 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery -import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.JoinUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.typeutils.InternalTypeInfo import org.apache.calcite.plan._ @@ -126,12 +127,13 @@ class StreamPhysicalJoin( override def translateToExecNode(): ExecNode[_] = { new StreamExecJoin( - joinSpec, - getUniqueKeys(left, joinSpec.getLeftKeys), - getUniqueKeys(right, joinSpec.getRightKeys), - InputProperty.DEFAULT, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription) + unwrapTableConfig(this), + joinSpec, + getUniqueKeys(left, joinSpec.getLeftKeys), + getUniqueKeys(right, joinSpec.getRightKeys), + InputProperty.DEFAULT, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala index 4cb7bb7935d51..bdc2ff017b298 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.LegacySink import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLegacySink import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, UpdatingPlanChecker} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.sinks._ @@ -61,12 +62,12 @@ class StreamPhysicalLegacySink[T]( val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecLegacySink( + unwrapTableConfig(this), sink, upsertKeys.orNull, needRetraction, InputProperty.DEFAULT, fromDataTypeToLogicalType(sink.getConsumedDataType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala index 39ae30021508e..ae71c1f470c61 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLegacyTableSourceScan import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLegacyTableSourceScan import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.sources.StreamTableSource import org.apache.calcite.plan._ @@ -57,6 +58,7 @@ class StreamPhysicalLegacyTableSourceScan( override def translateToExecNode(): ExecNode[_] = { new StreamExecLegacyTableSourceScan( + unwrapTableConfig(this), tableSource, getTable.getQualifiedName, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala index 0e2bd82015bc0..264aaaff5dd6a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -71,13 +72,13 @@ class StreamPhysicalLimit( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecLimit( - limitStart, - limitEnd, - generateUpdateBefore, - needRetraction, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + unwrapTableConfig(this), + limitStart, + limitEnd, + generateUpdateBefore, + needRetraction, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala index be2d352d903b7..684274ea5aad5 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.PartialFinalType import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -91,13 +92,13 @@ class StreamPhysicalLocalGroupAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecLocalGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala index 41fd057f5257f..7e5993875fcf2 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.rules.physical.stream.TwoStageOptimizedWindowAggregateRule import org.apache.flink.table.planner.plan.utils.WindowUtil.checkEmitConfiguration import org.apache.flink.table.planner.plan.utils.{AggregateUtil, FlinkRelOptUtil, RelExplainUtil, WindowUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.{NamedWindowProperty, SliceEnd, WindowReference} import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -125,12 +126,12 @@ class StreamPhysicalLocalWindowAggregate( override def translateToExecNode(): ExecNode[_] = { checkEmitConfiguration(FlinkRelOptUtil.getTableConfigFromContext(this)) new StreamExecLocalWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala index f962e2909b0e8..d39c77795795c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLookupJoin import org.apache.flink.table.planner.plan.utils.{FlinkRexUtil, JoinTypeUtil} import org.apache.flink.table.planner.utils.JavaScalaConversionUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptTable, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -77,6 +78,7 @@ class StreamPhysicalLookupJoin( (null, null) } new StreamExecLookupJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), remainingCondition.orNull, new TemporalTableSourceSpec(temporalTable), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala index 252af7c0a9254..c449762da57ac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala @@ -22,10 +22,11 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.MatchRecognize import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMatch -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.MatchUtil import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -97,10 +98,10 @@ class StreamPhysicalMatch( override def translateToExecNode(): ExecNode[_] = { new StreamExecMatch( + unwrapTableConfig(this), MatchUtil.createMatchSpec(logicalMatch), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala index 524a0c59f41d1..d2e88acb38674 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.MiniBatchIntervalTraitDef import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} @@ -63,10 +64,10 @@ class StreamPhysicalMiniBatchAssigner( override def translateToExecNode(): ExecNode[_] = { val miniBatchInterval = traits.getTrait(MiniBatchIntervalTraitDef.INSTANCE).getMiniBatchInterval new StreamExecMiniBatchAssigner( + unwrapTableConfig(this), miniBatchInterval, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala index ff1ea7654d2cd..3409e3380127c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecOverAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -57,10 +58,10 @@ class StreamPhysicalOverAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecOverAggregate( + unwrapTableConfig(this), OverAggregateUtil.createOverSpec(logicWindow), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala index b7eff1171a612..d946f708becc0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonCalc -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -58,6 +59,7 @@ class StreamPhysicalPythonCalc( } new StreamExecPythonCalc( + unwrapTableConfig(this), projection, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala index 434f163a37b80..6c91710a95f3d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala @@ -20,9 +20,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -74,11 +75,11 @@ class StreamPhysicalPythonCorrelate( } new StreamExecPythonCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala index 6099021572373..53fca11f65f27 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -79,6 +80,7 @@ class StreamPhysicalPythonGroupAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecPythonGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -86,7 +88,6 @@ class StreamPhysicalPythonGroupAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala index c77a5d766c963..bf6afa3c79676 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupTableAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -63,6 +64,7 @@ class StreamPhysicalPythonGroupTableAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecPythonGroupTableAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -70,7 +72,6 @@ class StreamPhysicalPythonGroupTableAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala index 3dc7043c7b1cd..ec623dee2f8bb 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, WindowEmitStrategy} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -71,6 +72,7 @@ class StreamPhysicalPythonGroupWindowAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecPythonGroupWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, window, @@ -79,7 +81,6 @@ class StreamPhysicalPythonGroupWindowAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala index 677644b691a90..7f62f9f0bf90f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonOverAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -56,11 +57,11 @@ class StreamPhysicalPythonOverAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecPythonOverAggregate( + unwrapTableConfig(this), OverAggregateUtil.createOverSpec(logicWindow), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala index 61879f5bbe4ea..77e536bdf7409 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecRank import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.rank._ import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -106,6 +107,7 @@ class StreamPhysicalRank( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val fieldCollations = orderKey.getFieldCollations new StreamExecRank( + unwrapTableConfig(this), rankType, new PartitionSpec(partitionKey.toArray), SortUtil.getSortSpec(fieldCollations), @@ -115,7 +117,6 @@ class StreamPhysicalRank( generateUpdateBefore, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala index fcc391442b5e5..af63d5be3c4e0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSinkSpec import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, RelDescriptionWriterImpl} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -85,13 +86,13 @@ class StreamPhysicalSink( util.Arrays.asList(abilitySpecs: _*)) tableSinkSpec.setTableSink(tableSink) new StreamExecSink( + unwrapTableConfig(this), tableSinkSpec, inputChangelogMode, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), upsertMaterialize, - getDescriptionWithUpsert(upsertMaterialize) - ) + getDescriptionWithUpsert(upsertMaterialize)) } /** diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala index d0df328623d39..47331fa27cccd 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSort import org.apache.flink.table.planner.plan.utils.{RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -71,11 +72,11 @@ class StreamPhysicalSort( override def translateToExecNode(): ExecNode[_] = { new StreamExecSort( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala index 346a77b5e72be..0e1d45ade0196 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSortLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.core.Sort @@ -90,6 +91,7 @@ class StreamPhysicalSortLimit( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecSortLimit( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), limitStart, limitEnd, @@ -97,7 +99,6 @@ class StreamPhysicalSortLimit( generateUpdateBefore, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala index c334be817e4bc..a82160be5495c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpe import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalTableSourceScan import org.apache.flink.table.planner.plan.schema.TableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode @@ -63,6 +64,7 @@ class StreamPhysicalTableSourceScan( tableSourceSpec.setTableSource(tableSource) new StreamExecTableSourceScan( + unwrapTableConfig(this), tableSourceSpec, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala index 4e1b359b4532d..933180bce029f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala @@ -26,6 +26,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.TemporalJoinUtil import org.apache.flink.table.planner.plan.utils.TemporalJoinUtil.{TEMPORAL_JOIN_CONDITION, TEMPORAL_JOIN_CONDITION_PRIMARY_KEY} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.util.Preconditions.checkState import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -115,6 +116,7 @@ class StreamPhysicalTemporalJoin( } new StreamExecTemporalJoin( + unwrapTableConfig(this), temporalJoinSpec, isTemporalFunctionJoin, leftTimeAttributeInputRef, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala index 1e21455b41020..e1709d6aad1c0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTemporalSort import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -62,10 +63,10 @@ class StreamPhysicalTemporalSort( override def translateToExecNode(): ExecNode[_] = { new StreamExecTemporalSort( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala index f45873573f7ae..fbc4a9611b8d1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecUnion -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -59,9 +60,9 @@ class StreamPhysicalUnion( override def translateToExecNode(): ExecNode[_] = { new StreamExecUnion( + unwrapTableConfig(this), getInputs.map(_ => InputProperty.DEFAULT), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala index 9b511a84845aa..07ed842d36326 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecValues +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import com.google.common.collect.ImmutableList import org.apache.calcite.plan._ @@ -52,9 +53,9 @@ class StreamPhysicalValues( override def translateToExecNode(): ExecNode[_] = { new StreamExecValues( + unwrapTableConfig(this), tuples.asList().map(_.asList()), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala index 6ccbecb436b0f..46fe743103d9b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.WatermarkAssigner import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{RelNode, RelWriter} @@ -70,6 +71,7 @@ class StreamPhysicalWatermarkAssigner( override def translateToExecNode(): ExecNode[_] = { new StreamExecWatermarkAssigner( + unwrapTableConfig(this), watermarkExpr, rowtimeFieldIndex, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala index eba1a5da1ebeb..8bdc707c7a454 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowAgg import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.WindowUtil.checkEmitConfiguration import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -103,13 +104,13 @@ class StreamPhysicalWindowAggregate( override def translateToExecNode(): ExecNode[_] = { checkEmitConfiguration(FlinkRelOptUtil.getTableConfigFromContext(this)) new StreamExecWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, windowing, namedWindowProperties.toArray, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala index 1b77b2d8ad264..1a80564f76656 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.plan.logical.WindowingStrategy import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowDeduplicate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -79,13 +80,13 @@ class StreamPhysicalWindowDeduplicate( override def translateToExecNode(): ExecNode[_] = { new StreamExecWindowDeduplicate( + unwrapTableConfig(this), partitionKeys, orderKey, keepLastRow, windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala index cd3af33e5aad5..d24f806a05ac4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -110,6 +111,7 @@ class StreamPhysicalWindowJoin( override def translateToExecNode(): ExecNode[_] = { new StreamExecWindowJoin( + unwrapTableConfig(this), joinSpec, leftWindowing, rightWindowing, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala index 16688e55f5e73..2aed05ad59eaa 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowRank import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.rank._ import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -94,6 +95,7 @@ class StreamPhysicalWindowRank( override def translateToExecNode(): ExecNode[_] = { val fieldCollations = orderKey.getFieldCollations new StreamExecWindowRank( + unwrapTableConfig(this), rankType, new PartitionSpec(partitionKey.toArray), SortUtil.getSortSpec(fieldCollations), @@ -102,7 +104,6 @@ class StreamPhysicalWindowRank( windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala index a2e2de5d6e927..4f438413fc1b9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrateg import org.apache.flink.table.planner.plan.nodes.common.CommonPhysicalWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -60,10 +61,10 @@ class StreamPhysicalWindowTableFunction( override def translateToExecNode(): ExecNode[_] = { new StreamExecWindowTableFunction( + unwrapTableConfig(this), windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java index 27997b6e96545..d2ae077e13283 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.FlinkVersion; +import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.planner.utils.JsonPlanTestBase; @@ -36,6 +37,8 @@ import java.nio.file.Paths; import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.api.Expressions.$; @@ -74,7 +77,7 @@ public void setup() throws Exception { @Test public void testCompilePlanSql() throws IOException { CompiledPlan compiledPlan = - tableEnv.compilePlanSql("insert into MySink select * from MyTable"); + tableEnv.compilePlanSql("INSERT INTO MySink SELECT * FROM MyTable"); String expected = TableTestUtil.readFromResource("/jsonplan/testGetJsonPlan.out"); assertThat( TableTestUtil.replaceExecNodeId( @@ -91,7 +94,7 @@ public void testCompilePlanSql() throws IOException { public void testExecutePlanSql() throws Exception { File sinkPath = createSourceSinkTables(); - tableEnv.compilePlanSql("insert into sink select * from src").execute().await(); + tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src").execute().await(); assertResult(DATA, sinkPath); } @@ -112,7 +115,7 @@ public void testCompileWriteToFileAndThenExecuteSql() throws Exception { File sinkPath = createSourceSinkTables(); - CompiledPlan plan = tableEnv.compilePlanSql("insert into sink select * from src"); + CompiledPlan plan = tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src"); plan.writeToFile(planPath); tableEnv.executeSql(String.format("EXECUTE PLAN '%s'", planPath.toAbsolutePath())).await(); @@ -315,6 +318,46 @@ public void testExplainPlan() throws IOException { .isEqualTo(expected); } + @Test + public void testPersistedConfigOption() throws Exception { + Path planPath = Paths.get(URI.create(getTempDirPath("plan")).getPath(), "plan.json"); + FileUtils.createParentDirectories(planPath.toFile()); + + List data = + Stream.concat( + DATA.stream(), + Stream.of( + "4,2,This string is long", + "5,3,This is an even longer string")) + .collect(Collectors.toList()); + String[] sinkColumnDefinitions = new String[] {"a bigint", "b int", "c varchar(11)"}; + + createTestCsvSourceTable("src", data, COLUMNS_DEFINITION); + File sinkPath = createTestCsvSinkTable("sink", sinkColumnDefinitions); + + // Set config option to trim the strings, so it's persisted in the json plan + tableEnv.getConfig() + .getConfiguration() + .set( + ExecutionConfigOptions.TABLE_EXEC_SINK_TYPE_LENGTH_ENFORCER, + ExecutionConfigOptions.TypeLengthEnforcer.TRIM_PAD); + CompiledPlan plan = tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src"); + + // Set config option to trim the strings to IGNORE, to validate that the persisted config + // is overriding the environment setting. + tableEnv.getConfig() + .getConfiguration() + .set( + ExecutionConfigOptions.TABLE_EXEC_SINK_TYPE_LENGTH_ENFORCER, + ExecutionConfigOptions.TypeLengthEnforcer.IGNORE); + + plan.execute().await(); + List expected = + Stream.concat(DATA.stream(), Stream.of("4,2,This string", "5,3,This is an ")) + .collect(Collectors.toList()); + assertResult(expected, sinkPath); + } + @Test public void testBatchMode() { tableEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode()); @@ -335,7 +378,7 @@ public void testBatchMode() { + " 'table-sink-class' = 'DEFAULT')"; tableEnv.executeSql(sinkTableDdl); - assertThatThrownBy(() -> tableEnv.compilePlanSql("insert into sink select * from src")) + assertThatThrownBy(() -> tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src")) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("The compiled plan feature is not supported in batch mode."); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java index 73f62285ed68a..4bb01a0756bb9 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java @@ -112,6 +112,7 @@ public void testDeadlockCausedByExchange() { BatchExecExchange exchange = new BatchExecExchange( + new Configuration(), InputProperty.builder() .requiredDistribution(InputProperty.ANY_DISTRIBUTION) .build(), diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java index 0c59dbe02acc0..9301f79e72516 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java @@ -20,6 +20,9 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -42,7 +45,9 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.flink.configuration.ConfigOptions.key; import static org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil.UNSUPPORTED_JSON_SERDE_CLASSES; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -104,6 +109,26 @@ public void testMultipleAnnotations() { new Condition<>( m -> m.minPlanVersion() == FlinkVersion.v1_15, "minStateVersion")); + Configuration config = new Configuration(); + config.set(OPTION_1, 1); + config.set(OPTION_2, 2); + config.set(OPTION_3, 3); + config.set(OPTION_4, 4); + config.set(OPTION_5, 5); + config.set(OPTION_6, 6); + + ReadableConfig persistedConfig = + ExecNodeMetadataUtil.newPersistedConfig( + DummyNode.class, + config, + Stream.of(OPTION_1, OPTION_2, OPTION_3, OPTION_4, OPTION_5, OPTION_6)); + assertThat(persistedConfig.get(OPTION_1)).isEqualTo(1); + assertThat(persistedConfig.get(OPTION_2)).isEqualTo(OPTION_2.defaultValue()); + assertThat(persistedConfig.get(OPTION_3)).isEqualTo(3); + assertThat(persistedConfig.get(OPTION_4)).isEqualTo(4); + assertThat(persistedConfig.get(OPTION_5)).isEqualTo(5); + assertThat(persistedConfig.get(OPTION_6)).isEqualTo(OPTION_6.defaultValue()); + // Using multiple individual ExecNodeMetadata annotations ExecNodeMetadataUtil.addTestNode(DummyNodeMultipleAnnotations.class); assertThat(ExecNodeMetadataUtil.retrieveExecNode("dummy-node-multiple-annotations", 1)) @@ -120,6 +145,73 @@ public void testMultipleAnnotations() { .has( new Condition<>( m -> m.minPlanVersion() == FlinkVersion.v1_15, "minStateVersion")); + + assertThatThrownBy( + () -> + ExecNodeContext.newPersistedConfig( + DummyNodeMultipleAnnotations.class, config)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest.DummyNodeMultipleAnnotations, " + + "consumedOption: option111 not listed in [TableConfigOptions, " + + "ExecutionConfigOptions]."); + } + + @Test + public void testDuplicateConsumedOptions() { + ExecNodeMetadataUtil.addTestNode(DummyNodeDuplicateConsumedOptions.class); + assertThatThrownBy( + () -> + ExecNodeMetadataUtil.newPersistedConfig( + DummyNodeDuplicateConsumedOptions.class, + new Configuration(), + Stream.of(OPTION_1, OPTION_2, OPTION_3))) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest." + + "DummyNodeDuplicateConsumedOptions, consumedOption: " + + "option2 is listed multiple times in consumedOptions, " + + "potentially also with fallback/deprecated key."); + } + + @Test + public void testDuplicateDeprecatedKeysConsumedOptions() { + ExecNodeMetadataUtil.addTestNode(DummyNodeDuplicateDeprecatedKeysConsumedOptions.class); + assertThatThrownBy( + () -> + ExecNodeMetadataUtil.newPersistedConfig( + DummyNodeDuplicateDeprecatedKeysConsumedOptions.class, + new Configuration(), + Stream.of(OPTION_1, OPTION_2, OPTION_3))) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest." + + "DummyNodeDuplicateDeprecatedKeysConsumedOptions, " + + "consumedOption: option3-deprecated is listed multiple times in " + + "consumedOptions, potentially also with fallback/deprecated " + + "key."); + } + + @Test + public void testDuplicateFallbackKeysConsumedOptions() { + ExecNodeMetadataUtil.addTestNode(DummyNodeDuplicateFallbackKeysConsumedOptions.class); + assertThatThrownBy( + () -> + ExecNodeMetadataUtil.newPersistedConfig( + DummyNodeDuplicateFallbackKeysConsumedOptions.class, + new Configuration(), + Stream.of(OPTION_1, OPTION_2, OPTION_4))) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest." + + "DummyNodeDuplicateFallbackKeysConsumedOptions, " + + "consumedOption: option4-fallback is listed multiple times in " + + "consumedOptions, potentially also with fallback/deprecated " + + "key."); } @Test @@ -128,15 +220,16 @@ public void testNewContext() { .isInstanceOf(IllegalStateException.class) .hasMessage( "ExecNode: org.apache.flink.table.planner.plan.utils." - + "ExecNodeMetadataUtilTest.DummyNodeNoAnnotation is not listed in the " - + "unsupported classes since it is not annotated with: ExecNodeMetadata."); + + "ExecNodeMetadataUtilTest.DummyNodeNoAnnotation is not " + + "listed in the unsupported classes since it is not annotated " + + "with: ExecNodeMetadata."); assertThatThrownBy(() -> ExecNodeContext.newContext(DummyNode.class)) .isInstanceOf(IllegalStateException.class) .hasMessage( "ExecNode: org.apache.flink.table.planner.plan.utils." - + "ExecNodeMetadataUtilTest.DummyNode is not listed in the supported " - + "classes and yet is annotated with: ExecNodeMetadata."); + + "ExecNodeMetadataUtilTest.DummyNode is not listed in the " + + "supported classes and yet is annotated with: ExecNodeMetadata."); } @Test @@ -178,103 +271,89 @@ public void testStreamExecNodeJsonSerdeCoverage() { @ExecNodeMetadata( name = "dummy-node", version = 1, + consumedOptions = {"option1", "option3-deprecated", "option5-deprecated"}, minPlanVersion = FlinkVersion.v1_13, minStateVersion = FlinkVersion.v1_13), @ExecNodeMetadata( name = "dummy-node", version = 2, + consumedOptions = {"option2", "option3-deprecated", "option5", "option6-fallback"}, minPlanVersion = FlinkVersion.v1_14, minStateVersion = FlinkVersion.v1_14), @ExecNodeMetadata( name = "dummy-node", version = 3, + consumedOptions = {"option1", "option3", "option4-fallback", "option5-deprecated"}, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) }) - private static class DummyNode extends ExecNodeBase { + private static class DummyNode extends AbstractDummyNode { @JsonCreator protected DummyNode( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } @ExecNodeMetadata( name = "dummy-node-multiple-annotations", version = 1, + consumedOptions = {"option1", "option2"}, minPlanVersion = FlinkVersion.v1_13, minStateVersion = FlinkVersion.v1_13) @ExecNodeMetadata( name = "dummy-node-multiple-annotations", version = 2, + consumedOptions = {"option11", "option22"}, minPlanVersion = FlinkVersion.v1_14, minStateVersion = FlinkVersion.v1_14) @ExecNodeMetadata( name = "dummy-node-multiple-annotations", version = 3, + consumedOptions = {"option111", "option222"}, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) - private static class DummyNodeMultipleAnnotations extends ExecNodeBase { + private static class DummyNodeMultipleAnnotations extends AbstractDummyNode { @JsonCreator protected DummyNodeMultipleAnnotations( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } - private static class DummyNodeNoJsonCreator extends ExecNodeBase { + private static class DummyNodeNoJsonCreator extends AbstractDummyNode { protected DummyNodeNoJsonCreator( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } - private static class DummyNodeNoAnnotation extends ExecNodeBase + private static class DummyNodeNoAnnotation extends AbstractDummyNode implements StreamExecNode { @JsonCreator protected DummyNodeNoAnnotation( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } @@ -295,15 +374,85 @@ protected Transformation translateToPlanInternal( version = 3, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) - private static class DummyNodeBothAnnotations extends ExecNodeBase { + private static class DummyNodeBothAnnotations extends AbstractDummyNode { @JsonCreator protected DummyNodeBothAnnotations( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); + super(context, persistedConfig, properties, outputType, description); + } + } + + @ExecNodeMetadata( + name = "dummy-node-duplicate-consumedOptions", + version = 3, + consumedOptions = {"option1", "option2", "option3", "option2"}, + minPlanVersion = FlinkVersion.v1_15, + minStateVersion = FlinkVersion.v1_15) + private static class DummyNodeDuplicateConsumedOptions extends AbstractDummyNode { + + @JsonCreator + protected DummyNodeDuplicateConsumedOptions( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(context, persistedConfig, properties, outputType, description); + } + } + + @ExecNodeMetadata( + name = "dummy-node-duplicate-deprecated-keys-consumedOptions", + version = 3, + consumedOptions = {"option1", "option2", "option3", "option3-deprecated"}, + minPlanVersion = FlinkVersion.v1_15, + minStateVersion = FlinkVersion.v1_15) + private static class DummyNodeDuplicateDeprecatedKeysConsumedOptions extends AbstractDummyNode { + + @JsonCreator + protected DummyNodeDuplicateDeprecatedKeysConsumedOptions( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(context, persistedConfig, properties, outputType, description); + } + } + + @ExecNodeMetadata( + name = "dummy-node-duplicate-fallback-keys-consumedOptions", + version = 3, + consumedOptions = {"option1", "option2", "option4", "option4-fallback"}, + minPlanVersion = FlinkVersion.v1_15, + minStateVersion = FlinkVersion.v1_15) + private static class DummyNodeDuplicateFallbackKeysConsumedOptions extends AbstractDummyNode { + + @JsonCreator + protected DummyNodeDuplicateFallbackKeysConsumedOptions( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(context, persistedConfig, properties, outputType, description); + } + } + + private static class AbstractDummyNode extends ExecNodeBase { + + protected AbstractDummyNode( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(10, context, persistedConfig, properties, outputType, description); } @Override @@ -312,4 +461,35 @@ protected Transformation translateToPlanInternal( return null; } } + + private static final ConfigOption OPTION_1 = + key("option1").intType().defaultValue(-1).withDescription("option1"); + private static final ConfigOption OPTION_2 = + key("option2").intType().defaultValue(-1).withDescription("option2"); + private static final ConfigOption OPTION_3 = + key("option3") + .intType() + .defaultValue(-1) + .withDeprecatedKeys("option3-deprecated") + .withDescription("option3"); + private static final ConfigOption OPTION_4 = + key("option4") + .intType() + .defaultValue(-1) + .withFallbackKeys("option4-fallback") + .withDescription("option4"); + private static final ConfigOption OPTION_5 = + key("option5") + .intType() + .defaultValue(-1) + .withFallbackKeys("option5-fallback") + .withDeprecatedKeys("option5-deprecated") + .withDescription("option5"); + private static final ConfigOption OPTION_6 = + key("option6") + .intType() + .defaultValue(-1) + .withDeprecatedKeys("option6-deprecated") + .withFallbackKeys("option6-fallback") + .withDescription("option6"); } From ec8e43d584054d3b9dc7783a4646f51cd81bcc10 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Thu, 24 Mar 2022 16:02:17 +0200 Subject: [PATCH 100/258] [FLINK-26075][table-planner][test] Regenerate JSON plans (cherry picked from commit b46065d51c7cf2ba2a504e96ad0a4651f3b7ceb9) --- .../resources/jsonplan/testGetJsonPlan.out | 6 +++++ .../testComplexCalc.out | 6 +++++ .../testSimpleFilter.out | 6 +++++ .../testSimpleProject.out | 6 +++++ .../testChangelogSource.out | 10 +++++++ .../testUpsertSource.out | 10 +++++++ .../testCrossJoin.out | 6 +++++ .../testCrossJoinOverrideParameters.out | 6 +++++ .../testJoinWithFilter.out | 6 +++++ .../testLeftOuterJoinWithLiteralTrue.out | 6 +++++ .../testDeduplication.out | 12 +++++++++ .../testExpand.out | 14 ++++++++++ ...inctAggCalls[isMiniBatchEnabled=false].out | 10 +++++++ ...tinctAggCalls[isMiniBatchEnabled=true].out | 14 ++++++++++ ...sWithGroupBy[isMiniBatchEnabled=false].out | 10 +++++++ ...lsWithGroupBy[isMiniBatchEnabled=true].out | 14 ++++++++++ ...thoutGroupBy[isMiniBatchEnabled=false].out | 10 +++++++ ...ithoutGroupBy[isMiniBatchEnabled=true].out | 14 ++++++++++ ...inedAggCalls[isMiniBatchEnabled=false].out | 10 +++++++ ...finedAggCalls[isMiniBatchEnabled=true].out | 10 +++++++ .../testEventTimeHopWindow.out | 11 ++++++++ .../testEventTimeSessionWindow.out | 11 ++++++++ .../testEventTimeTumbleWindow.out | 11 ++++++++ .../testProcTimeHopWindow.out | 11 ++++++++ .../testProcTimeSessionWindow.out | 11 ++++++++ .../testProcTimeTumbleWindow.out | 11 ++++++++ .../testIncrementalAggregate.out | 18 +++++++++++++ ...egateWithSumCountDistinctAndRetraction.out | 26 +++++++++++++++++++ ...estProcessingTimeInnerJoinWithOnClause.out | 6 +++++ .../testRowTimeInnerJoinWithOnClause.out | 6 +++++ .../testInnerJoin.out | 6 +++++ .../testInnerJoinWithEqualPk.out | 14 ++++++++++ .../testInnerJoinWithPk.out | 14 ++++++++++ .../testLeftJoinNonEqui.out | 6 +++++ .../LimitJsonPlanTest_jsonplan/testLimit.out | 9 +++++++ .../testJoinTemporalTable.out | 6 +++++ ...oinTemporalTableWithProjectionPushDown.out | 6 +++++ .../testMatch.out | 6 +++++ ...ProcTimeBoundedNonPartitionedRangeOver.out | 6 +++++ ...estProcTimeBoundedPartitionedRangeOver.out | 6 +++++ ...PartitionedRowsOverWithBuiltinProctime.out | 6 +++++ ...tProcTimeUnboundedPartitionedRangeOver.out | 6 +++++ ...ctimeBoundedDistinctPartitionedRowOver.out | 6 +++++ ...tinctWithNonDistinctPartitionedRowOver.out | 6 +++++ .../testRowTimeBoundedPartitionedRowsOver.out | 6 +++++ .../RankJsonPlanTest_jsonplan/testRank.out | 9 +++++++ .../testSortLimit.out | 9 +++++++ .../testOverwrite.out | 6 +++++ .../testPartitioning.out | 6 +++++ .../testWritingMetadata.out | 6 +++++ .../testFilterPushDown.out | 6 +++++ .../testLimitPushDown.out | 9 +++++++ .../testPartitionPushDown.out | 6 +++++ .../testProjectPushDown.out | 6 +++++ .../testReadingMetadata.out | 6 +++++ .../testWatermarkPushDown.out | 6 +++++ .../testJoinTemporalFunction.out | 6 +++++ .../testTemporalTableJoin.out | 6 +++++ .../testSortProcessingTime.out | 6 +++++ .../testSortRowTime.out | 6 +++++ .../UnionJsonPlanTest_jsonplan/testUnion.out | 6 +++++ .../testValues.out | 6 +++++ .../testWatermarkAssigner.out | 6 +++++ .../testDistinctSplitEnabled.out | 18 +++++++++++++ .../testEventTimeCumulateWindow.out | 12 +++++++++ .../testEventTimeCumulateWindowWithOffset.out | 12 +++++++++ .../testEventTimeHopWindow.out | 12 +++++++++ .../testEventTimeHopWindowWithOffset.out | 12 +++++++++ .../testEventTimeTumbleWindow.out | 12 +++++++++ .../testEventTimeTumbleWindowWithOffset.out | 12 +++++++++ .../testProcTimeCumulateWindow.out | 9 +++++++ .../testProcTimeHopWindow.out | 9 +++++++ .../testProcTimeTumbleWindow.out | 9 +++++++ .../testEventTimeTumbleWindow.out | 21 +++++++++++++++ .../testFollowedByWindowDeduplicate.out | 12 +++++++++ .../testFollowedByWindowJoin.out | 15 +++++++++++ .../testFollowedByWindowRank.out | 12 +++++++++ .../testIndividualWindowTVF.out | 9 +++++++ .../testIndividualWindowTVFProcessingTime.out | 9 +++++++ 79 files changed, 729 insertions(+) diff --git a/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out b/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out index 1bc32adf98a61..281372b28f5aa 100644 --- a/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out +++ b/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out @@ -40,6 +40,12 @@ { "id": 2, "type": "stream-exec-sink_1", + "configuration":{ + "table.exec.sink.keyed-shuffle":"AUTO", + "table.exec.sink.not-null-enforcer":"ERROR", + "table.exec.sink.type-length-enforcer":"IGNORE", + "table.exec.sink.upsert-materialize":"AUTO" + }, "dynamicTableSink": { "table": { "identifier": "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out index 9ac59d9ec43d8..4dcf71adccd78 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out @@ -208,6 +208,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out index c104d9a4a3370..6028828ca81f6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out @@ -85,6 +85,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out index 2ae314d98f9dc..13806b710dce4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out @@ -46,6 +46,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out index 63449c095d367..1f4ca9d8dd78a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out @@ -77,6 +77,10 @@ }, { "id" : 4, "type" : "stream-exec-changelog-normalize_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "uniqueKeys" : [ 0, 1 ], "generateUpdateBefore" : true, "inputProperties" : [ { @@ -91,6 +95,12 @@ }, { "id" : 5, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out index b89ceb2ee9656..81bd0cc7972b7 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out @@ -65,6 +65,10 @@ }, { "id" : 3, "type" : "stream-exec-changelog-normalize_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "uniqueKeys" : [ 0, 1 ], "generateUpdateBefore" : true, "inputProperties" : [ { @@ -79,6 +83,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out index ba4d03d672baa..89307808f00f7 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out @@ -87,6 +87,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out index c8def35787512..9e69b436938ca 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out @@ -91,6 +91,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out index c31983bf6ef73..033efb8ede049 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out @@ -101,6 +101,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out index 8f151698837f8..7c3ebe89595f5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out @@ -87,6 +87,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out index 9829b7cdd0e2f..f2c4a58820d1e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out @@ -154,6 +154,12 @@ }, { "id" : 4, "type" : "stream-exec-deduplicate_1", + "configuration" : { + "table.exec.deduplicate.insert-update-after-sensitive-enabled" : "true", + "table.exec.deduplicate.mini-batch.compact-changes-enabled" : "false", + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "uniqueKeys" : [ 2 ], "isRowtime" : false, "keepLastRow" : false, @@ -223,6 +229,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`sink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out index 04d059e3aac90..3982472c711a4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out @@ -232,6 +232,10 @@ }, { "id" : 6, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0, 3, 4 ], "aggCalls" : [ { "name" : null, @@ -281,6 +285,10 @@ }, { "id" : 8, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : null, @@ -316,6 +324,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out index 5a691039e310f..52aad805789e8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out @@ -94,6 +94,10 @@ }, { "id" : 4, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a1", @@ -240,6 +244,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out index d56ae50aa590f..bfef96f23e12b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out @@ -97,6 +97,10 @@ }, { "id" : 4, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a1", @@ -337,6 +341,10 @@ }, { "id" : 6, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a1", @@ -484,6 +492,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out index 54e1aa4fc9eee..4ca7313f0a283 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out @@ -99,6 +99,10 @@ }, { "id" : 4, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a", @@ -192,6 +196,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out index 6d0adeb903fc2..b31d031ffd0c5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out @@ -102,6 +102,10 @@ }, { "id" : 4, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a", @@ -159,6 +163,10 @@ }, { "id" : 6, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt_a", @@ -253,6 +261,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out index c8d7b01d1a088..ab5ecc94b196d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out @@ -104,6 +104,10 @@ }, { "id" : 4, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ ], "aggCalls" : [ { "name" : "avg_a", @@ -216,6 +220,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out index c53e46390e100..8e9e1e6e283af 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out @@ -108,6 +108,10 @@ }, { "id" : 4, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ ], "aggCalls" : [ { "name" : "avg_a", @@ -173,6 +177,10 @@ }, { "id" : 6, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ ], "aggCalls" : [ { "name" : "avg_a", @@ -286,6 +294,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out index cc12b898c21aa..1ce190c4f7c0f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out @@ -88,6 +88,10 @@ }, { "id" : 4, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "a1", @@ -182,6 +186,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out index 41687d09b8d7a..201e9547f11b6 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out @@ -104,6 +104,10 @@ }, { "id" : 5, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "a1", @@ -198,6 +202,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out index aef0a949c30dd..ab0ae9d10881a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out @@ -200,6 +200,11 @@ }, { "id" : 5, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -259,6 +264,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out index a0a44cb80049c..467b00f4df578 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out @@ -200,6 +200,11 @@ }, { "id" : 5, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -257,6 +262,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out index f41b8d274cb50..d30a69a2d5fd1 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out @@ -200,6 +200,11 @@ }, { "id" : 5, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$3", @@ -410,6 +415,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out index 1c9fbb79b63a2..68ecda4bd6e07 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out @@ -287,6 +287,11 @@ }, { "id" : 6, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -338,6 +343,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out index b44c9c4daf429..e07c84acb3004 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out @@ -287,6 +287,11 @@ }, { "id" : 6, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -336,6 +341,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out index d9c1ed0437798..bc5fd3fa7c035 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out @@ -266,6 +266,11 @@ }, { "id" : 6, "type" : "stream-exec-group-window-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1", + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -407,6 +412,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out index 76cc64fec94c3..a2bcf705ab28a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out @@ -102,6 +102,10 @@ }, { "id" : 4, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0, 2 ], "aggCalls" : [ { "name" : null, @@ -209,6 +213,10 @@ }, { "id" : 6, "type" : "stream-exec-incremental-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "partialAggGrouping" : [ 0, 1 ], "finalAggGrouping" : [ 0 ], "partialOriginalAggCalls" : [ { @@ -250,6 +258,10 @@ }, { "id" : 8, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : null, @@ -277,6 +289,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out index abb17ca57f619..c4a0f219d8ecb 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out @@ -62,6 +62,10 @@ }, { "id" : 3, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "b", @@ -110,6 +114,10 @@ }, { "id" : 5, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "b", @@ -187,6 +195,10 @@ }, { "id" : 7, "type" : "stream-exec-local-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0, 2 ], "aggCalls" : [ { "name" : null, @@ -315,6 +327,10 @@ }, { "id" : 9, "type" : "stream-exec-incremental-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "partialAggGrouping" : [ 0, 1 ], "finalAggGrouping" : [ 0 ], "partialOriginalAggCalls" : [ { @@ -375,6 +391,10 @@ }, { "id" : 11, "type" : "stream-exec-global-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "true", + "table.exec.mini-batch.size" : "5" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : null, @@ -421,6 +441,12 @@ }, { "id" : 12, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out index 05bdf0d776dab..4d61dede144bd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out @@ -656,6 +656,12 @@ }, { "id" : 13, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out index d20fb04851b23..94927bb53e7b2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out @@ -480,6 +480,12 @@ }, { "id" : 11, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out index fae9d801614d9..0355fb03137ad 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out @@ -136,6 +136,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out index a75de5cd4e0be..5d80fb2baac57 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out @@ -56,6 +56,10 @@ }, { "id" : 3, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ ], "aggCallNeedRetractions" : [ ], @@ -139,6 +143,10 @@ }, { "id" : 7, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ ], "aggCallNeedRetractions" : [ ], @@ -196,6 +204,12 @@ }, { "id" : 10, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out index 3bd92217d6de9..149408c5e1c60 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out @@ -56,6 +56,10 @@ }, { "id" : 3, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "a2", @@ -170,6 +174,10 @@ }, { "id" : 8, "type" : "stream-exec-group-aggregate_1", + "configuration" : { + "table.exec.mini-batch.enabled" : "false", + "table.exec.mini-batch.size" : "-1" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "b2", @@ -288,6 +296,12 @@ }, { "id" : 13, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out index 289eb09c4edc1..27df7d82b1ccd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out @@ -172,6 +172,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out index aa1c8c2c7be9d..361b644af179b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out @@ -61,6 +61,9 @@ }, { "id" : 3, "type" : "stream-exec-limit_1", + "configuration" : { + "table.exec.rank.topn-cache-size" : "10000" + }, "rankRange" : { "type" : "Constant", "start" : 1, @@ -112,6 +115,12 @@ }, { "id" : 5, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out index 4f4afabbd6f39..cba8ae4894a33 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out @@ -378,6 +378,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out index 17946aba99ea8..c3012c79471f5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out @@ -368,6 +368,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out index b921155e86d97..c7bbd666aba9a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out @@ -302,6 +302,12 @@ }, { "id" : 5, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out index 2541c41215928..4a91e46f98eaf 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedNonPartitionedRangeOver.out @@ -362,6 +362,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out index 461837d4bb880..15f5426ec9924 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRangeOver.out @@ -422,6 +422,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out index fba1d4819773f..5d722f46d04df 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeBoundedPartitionedRowsOverWithBuiltinProctime.out @@ -359,6 +359,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out index 6f7c396ea6489..c4a29465ec854 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProcTimeUnboundedPartitionedRangeOver.out @@ -393,6 +393,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out index 6c8d61cfd199a..4eaf0b63f03ee 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctPartitionedRowOver.out @@ -403,6 +403,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out index 3e8b12df51628..b7477b9ae92f4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testProctimeBoundedDistinctWithNonDistinctPartitionedRowOver.out @@ -493,6 +493,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out index cd7d26f2fe809..9eb9209956eb4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/OverAggregateJsonPlanTest_jsonplan/testRowTimeBoundedPartitionedRowsOver.out @@ -237,6 +237,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out index 5a3a6e6b26b53..3c0fde4021af0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/RankJsonPlanTest_jsonplan/testRank.out @@ -72,6 +72,9 @@ }, { "id" : 4, "type" : "stream-exec-rank_1", + "configuration" : { + "table.exec.rank.topn-cache-size" : "10000" + }, "rankType" : "ROW_NUMBER", "partition" : { "fields" : [ 1 ] @@ -126,6 +129,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out index 6ba5bf6933d18..c0a3251360a2e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/SortLimitJsonPlanTest_jsonplan/testSortLimit.out @@ -58,6 +58,9 @@ }, { "id" : 3, "type" : "stream-exec-sort-limit_1", + "configuration" : { + "table.exec.rank.topn-cache-size" : "10000" + }, "orderBy" : { "fields" : [ { "index" : 1, @@ -113,6 +116,12 @@ }, { "id" : 5, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out index 4dc91d95c80d1..c9d3d04531344 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testOverwrite.out @@ -34,6 +34,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out index 4318664e07ec7..4858fb995f639 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testPartitioning.out @@ -69,6 +69,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out index 00f9f79f525e2..60411d8a3d952 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out @@ -34,6 +34,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out index 0c8aa8b0edde7..eed66e54fd254 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testFilterPushDown.out @@ -53,6 +53,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out index 55037d9632d43..50c566df575f3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testLimitPushDown.out @@ -50,6 +50,9 @@ }, { "id" : 3, "type" : "stream-exec-limit_1", + "configuration" : { + "table.exec.rank.topn-cache-size" : "10000" + }, "rankRange" : { "type" : "Constant", "start" : 1, @@ -79,6 +82,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out index 772fffbcb36ec..9232850325948 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testPartitionPushDown.out @@ -81,6 +81,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out index 9de33800c77e2..98a2518124777 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testProjectPushDown.out @@ -43,6 +43,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`sink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out index 265707e32ce50..a8014cf440021 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out @@ -49,6 +49,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`sink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out index f8798e3788986..bf02aa522339c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testWatermarkPushDown.out @@ -113,6 +113,12 @@ }, { "id" : 2, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`sink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testJoinTemporalFunction.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testJoinTemporalFunction.out index 7c4fd06033448..6c093078332da 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testJoinTemporalFunction.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testJoinTemporalFunction.out @@ -324,6 +324,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testTemporalTableJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testTemporalTableJoin.out index 2bf45647b143a..7666686f2669b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testTemporalTableJoin.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalJoinJsonPlanTest_jsonplan/testTemporalTableJoin.out @@ -324,6 +324,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out index 6386343f939c2..a8b9d4c857c1a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortProcessingTime.out @@ -309,6 +309,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortRowTime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortRowTime.out index 1c5ffe17ca623..1bfbf4d324a68 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortRowTime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TemporalSortJsonPlanTest_jsonplan/testSortRowTime.out @@ -255,6 +255,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out index 1e050955c0a4e..246a01af6848a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/UnionJsonPlanTest_jsonplan/testUnion.out @@ -92,6 +92,12 @@ }, { "id" : 4, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out index a1a7ff0d2931c..fd83244d9dd98 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ValuesJsonPlanTest_jsonplan/testValues.out @@ -78,6 +78,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out index 15b2bb8e1bf7d..4ebd7ddefd6c8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WatermarkAssignerJsonPlanTest_jsonplan/testWatermarkAssigner.out @@ -104,6 +104,12 @@ }, { "id" : 3, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`sink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out index be85b4c02b87c..94423a788627f 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testDistinctSplitEnabled.out @@ -242,6 +242,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0, 3 ], "aggCalls" : [ { "name" : null, @@ -399,6 +402,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0, 1 ], "aggCalls" : [ { "name" : null, @@ -549,6 +555,9 @@ }, { "id" : 9, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : null, @@ -619,6 +628,9 @@ }, { "id" : 11, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : null, @@ -772,6 +784,12 @@ }, { "id" : 13, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out index 4d3c3dcca26e0..cdfeaf552f19e 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindow.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -282,6 +285,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -407,6 +413,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithOffset.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithOffset.out index 3915c80efb63d..aa5990662ab0b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithOffset.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeCumulateWindowWithOffset.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -283,6 +286,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -409,6 +415,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out index 4c8d84a27a642..74a780364d821 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -282,6 +285,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -403,6 +409,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out index 25d8c8f078067..fd02b9e0855f0 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindowWithOffset.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -283,6 +286,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -405,6 +411,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out index 6e258b3309337..9d9064218f709 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$3", @@ -463,6 +466,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$3", @@ -619,6 +625,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out index ea0bd3702b511..577951a3fe647 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindowWithOffset.out @@ -221,6 +221,9 @@ }, { "id" : 5, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$3", @@ -464,6 +467,9 @@ }, { "id" : 7, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$3", @@ -621,6 +627,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out index 0217355037cfb..b7c7cb2a05c55 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeCumulateWindow.out @@ -286,6 +286,9 @@ }, { "id" : 6, "type" : "stream-exec-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -377,6 +380,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out index 2973abc7e9424..0f5cbd6d71361 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out @@ -287,6 +287,9 @@ }, { "id" : 6, "type" : "stream-exec-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$1", @@ -377,6 +380,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out index 02faef6fec2ea..2b2369f3e4a15 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out @@ -266,6 +266,9 @@ }, { "id" : 6, "type" : "stream-exec-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "EXPR$2", @@ -360,6 +363,12 @@ }, { "id" : 8, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out index 617e8b844b93a..2a59a3f7a2db8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowJoinJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out @@ -170,6 +170,9 @@ }, { "id" : 4, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt", @@ -305,6 +308,9 @@ }, { "id" : 6, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt", @@ -650,6 +656,9 @@ }, { "id" : 12, "type" : "stream-exec-local-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt", @@ -785,6 +794,9 @@ }, { "id" : 14, "type" : "stream-exec-global-window-aggregate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "grouping" : [ 0 ], "aggCalls" : [ { "name" : "cnt", @@ -963,6 +975,9 @@ }, { "id" : 17, "type" : "stream-exec-window-join_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "joinSpec" : { "joinType" : "INNER", "leftKeys" : [ 0 ], @@ -1064,6 +1079,12 @@ }, { "id" : 19, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowDeduplicate.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowDeduplicate.out index ea72435dd9e67..a10937b0c33f1 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowDeduplicate.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowDeduplicate.out @@ -209,6 +209,9 @@ }, { "id" : 4, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -381,6 +384,9 @@ }, { "id" : 7, "type" : "stream-exec-window-deduplicate_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "partitionKeys" : [ 0 ], "orderKey" : 3, "keepLastRow" : true, @@ -470,6 +476,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowJoin.out index c98861677f0be..df66bb07961c5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowJoin.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowJoin.out @@ -168,6 +168,9 @@ }, { "id" : 4, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -453,6 +456,9 @@ }, { "id" : 10, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -573,6 +579,9 @@ }, { "id" : 13, "type" : "stream-exec-window-join_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "joinSpec" : { "joinType" : "INNER", "leftKeys" : [ 2 ], @@ -674,6 +683,12 @@ }, { "id" : 15, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowRank.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowRank.out index 5242939a4bc87..398af0349871a 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowRank.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testFollowedByWindowRank.out @@ -209,6 +209,9 @@ }, { "id" : 4, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -323,6 +326,9 @@ }, { "id" : 7, "type" : "stream-exec-window-rank_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "rankType" : "ROW_NUMBER", "partitionSpec" : { "fields" : [ 0 ] @@ -401,6 +407,12 @@ }, { "id" : 9, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVF.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVF.out index aea8c5010bac0..24430239300b4 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVF.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVF.out @@ -168,6 +168,9 @@ }, { "id" : 4, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -261,6 +264,12 @@ }, { "id" : 6, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVFProcessingTime.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVFProcessingTime.out index e3af3a1fbf818..f3a32187d22fb 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVFProcessingTime.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/WindowTableFunctionJsonPlanTest_jsonplan/testIndividualWindowTVFProcessingTime.out @@ -264,6 +264,9 @@ }, { "id" : 5, "type" : "stream-exec-window-table-function_1", + "configuration" : { + "table.local-time-zone" : "default" + }, "windowing" : { "strategy" : "TimeAttribute", "window" : { @@ -359,6 +362,12 @@ }, { "id" : 7, "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, "dynamicTableSink" : { "table" : { "identifier" : "`default_catalog`.`default_database`.`MySink`", From 0f475a42dfbcb1de0d2e796d01a70cabdf35c4ca Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Fri, 11 Mar 2022 16:26:23 +0100 Subject: [PATCH 101/258] [FLINK-26615][tests] Fix timings in BatchingStateChangeUploadSchedulerTest.testUploadTimeout --- .../fs/BatchingStateChangeUploadSchedulerTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/BatchingStateChangeUploadSchedulerTest.java b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/BatchingStateChangeUploadSchedulerTest.java index 2b13d911985a3..f0d9cf2d70c32 100644 --- a/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/BatchingStateChangeUploadSchedulerTest.java +++ b/flink-dstl/flink-dstl-dfs/src/test/java/org/apache/flink/changelog/fs/BatchingStateChangeUploadSchedulerTest.java @@ -220,19 +220,19 @@ public void testRetryOnTimeout() throws Exception { new ManuallyTriggeredScheduledExecutorService(); BlockingUploader uploader = new BlockingUploader(); try (BatchingStateChangeUploadScheduler store = - scheduler(numAttempts, executorService, uploader, 10)) { + scheduler(numAttempts, executorService, uploader, 50)) { store.upload(upload); Deadline deadline = Deadline.fromNow(Duration.ofMinutes(5)); while (uploader.getUploadsCount() < numAttempts - 1 && deadline.hasTimeLeft()) { executorService.triggerScheduledTasks(); executorService.triggerAll(); - Thread.sleep(10); + Thread.sleep(1); // should be less than timeout to avoid all attempts timing out } uploader.unblock(); while (!upload.finished.get() && deadline.hasTimeLeft()) { executorService.triggerScheduledTasks(); executorService.triggerAll(); - Thread.sleep(10); + Thread.sleep(1); } } From b46ece18d8a8b3a4972c35d78ed0affa593d0313 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Fri, 25 Mar 2022 17:58:03 +0800 Subject: [PATCH 102/258] [FLINK-26865][python] Fix the potential failure of loading library in Thread Mode This closes #19244. --- flink-python/dev/dev-requirements.txt | 2 +- flink-python/pom.xml | 2 +- flink-python/setup.py | 2 +- flink-python/src/main/resources/META-INF/NOTICE | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-python/dev/dev-requirements.txt b/flink-python/dev/dev-requirements.txt index bd75137d024bf..00ce207053924 100755 --- a/flink-python/dev/dev-requirements.txt +++ b/flink-python/dev/dev-requirements.txt @@ -27,4 +27,4 @@ numpy>=1.14.3,<1.20 fastavro>=0.21.4,<0.24 grpcio>=1.29.0,<2 grpcio-tools>=1.3.5,<=1.14.2 -pemja==0.1.2; python_version >= '3.7' +pemja==0.1.3; python_version >= '3.7' diff --git a/flink-python/pom.xml b/flink-python/pom.xml index ad39eea7b1681..b735df3ad7738 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -107,7 +107,7 @@ under the License. com.alibaba pemja - 0.1.2 + 0.1.3 diff --git a/flink-python/setup.py b/flink-python/setup.py index 6bd3f97ed986d..edad2f9e6bf61 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -313,7 +313,7 @@ def extracted_output_files(base_dir, file_path, output_directory): 'pandas>=1.0,<1.2.0', 'pyarrow>=0.15.1,<3.0.0', 'pytz>=2018.3', 'numpy>=1.14.3,<1.20', 'fastavro>=0.21.4,<0.24', 'requests>=2.26.0', 'protobuf<3.18', - 'pemja==0.1.2;python_full_version >= "3.7"', + 'pemja==0.1.3;python_full_version >= "3.7"', apache_flink_libraries_dependency], cmdclass={'build_ext': build_ext}, tests_require=['pytest==4.4.1'], diff --git a/flink-python/src/main/resources/META-INF/NOTICE b/flink-python/src/main/resources/META-INF/NOTICE index 8532b8f5dcfc1..008988b175eaa 100644 --- a/flink-python/src/main/resources/META-INF/NOTICE +++ b/flink-python/src/main/resources/META-INF/NOTICE @@ -27,7 +27,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.beam:beam-vendor-sdks-java-extensions-protobuf:2.27.0 - org.apache.beam:beam-vendor-guava-26_0-jre:0.1 - org.apache.beam:beam-vendor-grpc-1_26_0:0.3 -- com.alibaba:pemja:0.1.2 +- com.alibaba:pemja:0.1.3 This project bundles the following dependencies under the BSD license. See bundled license files for details From f858421d67a27c155accdd7da9117c29541f8939 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Thu, 17 Mar 2022 15:25:32 +0100 Subject: [PATCH 103/258] [FLINK-26708] TimestampsAndWatermarksOperator should not propagate WatermarkStatus --- .../operators/TimestampsAndWatermarksOperator.java | 4 ++++ .../TimestampsAndWatermarksOperatorTest.java | 14 ++++++++++++++ .../util/OneInputStreamOperatorTestHarness.java | 11 +++++++++++ 3 files changed, 29 insertions(+) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperator.java index a10c92e1e5ff2..570cb6b90b3e6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperator.java @@ -126,6 +126,10 @@ public void processWatermark(org.apache.flink.streaming.api.watermark.Watermark } } + /** Override the base implementation to completely ignore statuses propagated from upstream. */ + @Override + public void processWatermarkStatus(WatermarkStatus watermarkStatus) throws Exception {} + @Override public void finish() throws Exception { super.finish(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperatorTest.java index 555a1a698c2ee..b7f34619a3024 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/TimestampsAndWatermarksOperatorTest.java @@ -25,6 +25,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.junit.Test; @@ -56,6 +57,19 @@ public void inputWatermarksAreNotForwarded() throws Exception { assertThat(testHarness.getOutput(), empty()); } + @Test + public void inputStatusesAreNotForwarded() throws Exception { + OneInputStreamOperatorTestHarness testHarness = + createTestHarness( + WatermarkStrategy.forGenerator((ctx) -> new PeriodicWatermarkGenerator()) + .withTimestampAssigner((ctx) -> new LongExtractor())); + + testHarness.processWatermarkStatus(WatermarkStatus.IDLE); + testHarness.setProcessingTime(AUTO_WATERMARK_INTERVAL); + + assertThat(testHarness.getOutput(), empty()); + } + @Test public void longMaxInputWatermarkIsForwarded() throws Exception { OneInputStreamOperatorTestHarness testHarness = diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java index a078a08751284..a65667d602e9a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java @@ -30,6 +30,7 @@ import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.apache.flink.util.Preconditions; import java.util.ArrayList; @@ -225,6 +226,16 @@ public void processWatermark(long watermark) throws Exception { processWatermark(new Watermark(watermark)); } + public void processWatermarkStatus(WatermarkStatus status) throws Exception { + if (inputs.isEmpty()) { + getOneInputOperator().processWatermarkStatus(status); + } else { + checkState(inputs.size() == 1); + Input input = inputs.get(0); + input.processWatermarkStatus(status); + } + } + public void processWatermark(Watermark mark) throws Exception { currentWatermark = mark.getTimestamp(); if (inputs.isEmpty()) { From 3fed74d757b34f9537de1b48b0bab7840c86649b Mon Sep 17 00:00:00 2001 From: wangfeifan Date: Tue, 22 Mar 2022 20:11:41 +0800 Subject: [PATCH 104/258] [FLINK-26799][state/changelog] fix seek condition in StateChangeFormat#read --- .../java/org/apache/flink/changelog/fs/StateChangeFormat.java | 2 +- .../state/changelog/inmemory/StateChangelogStorageTest.java | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFormat.java b/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFormat.java index ba04f595bded0..ff2149301e42c 100644 --- a/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFormat.java +++ b/flink-dstl/flink-dstl-dfs/src/main/java/org/apache/flink/changelog/fs/StateChangeFormat.java @@ -91,7 +91,7 @@ public CloseableIterator read(StreamStateHandle handle, long offset throws IOException { FSDataInputStream stream = handle.openInputStream(); DataInputViewStreamWrapper input = wrap(stream); - if (stream.getPos() != offset) { + if (offset != 0) { LOG.debug("seek from {} to {}", stream.getPos(), offset); input.skipBytesToRead((int) offset); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageTest.java index d7ddf40713178..4adde98878a9b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/changelog/inmemory/StateChangelogStorageTest.java @@ -71,7 +71,7 @@ public void testNoAppendAfterClose() throws IOException { @Test public void testWriteAndRead() throws Exception { KeyGroupRange kgRange = KeyGroupRange.of(0, 5); - Map> appendsByKeyGroup = generateAppends(kgRange, 10, 20); + Map> appendsByKeyGroup = generateAppends(kgRange, 405, 20); try (StateChangelogStorage client = getFactory(); StateChangelogWriter writer = @@ -84,6 +84,7 @@ public void testWriteAndRead() throws Exception { for (byte[] bytes : appends) { writer.append(group, bytes); } + writer.nextSequenceNumber(); } T handle = writer.persist(prev).get(); From 28228acd3007d9d3e09c19a61a5670748c15faed Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Fri, 11 Feb 2022 11:58:30 +0800 Subject: [PATCH 105/258] [hotfix][connector/common] Fix typo of variable in SourceOperator --- .../flink/streaming/api/operators/SourceOperator.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java index aba208549aa35..5660bc1546c9d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java @@ -170,7 +170,7 @@ private enum OperatingMode { private CompletableFuture waitingForAlignmentFuture = CompletableFuture.completedFuture(null); - private @Nullable LatencyMarkerEmitter latencyMarerEmitter; + private @Nullable LatencyMarkerEmitter latencyMarkerEmitter; public SourceOperator( FunctionWithException, Exception> @@ -334,8 +334,8 @@ private void stopInternalServices() { if (eventTimeLogic != null) { eventTimeLogic.stopPeriodicWatermarkEmits(); } - if (latencyMarerEmitter != null) { - latencyMarerEmitter.close(); + if (latencyMarkerEmitter != null) { + latencyMarkerEmitter.close(); } } @@ -433,7 +433,7 @@ private void initializeLatencyMarkerEmitter(DataOutput output) { .getConfiguration() .getLong(MetricOptions.LATENCY_INTERVAL); if (latencyTrackingInterval > 0) { - latencyMarerEmitter = + latencyMarkerEmitter = new LatencyMarkerEmitter<>( getProcessingTimeService(), output::emitLatencyMarker, From c71582d5837b067f26f8aec610193bacf6ccb26b Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Fri, 11 Feb 2022 18:19:10 +0800 Subject: [PATCH 106/258] [FLINK-26018][connector/common] Create per-split output on split addition in SourceOperator This change could avoid watermark being pushed forward by records from the first split in the first fetch when multiple splits are assigned to the source operator. --- flink-connectors/flink-connector-base/pom.xml | 8 ++ .../base/source/reader/SourceReaderBase.java | 3 + .../reader/splitreader/SplitReader.java | 5 + .../source/reader/SourceReaderBaseTest.java | 130 ++++++++++++++++++ .../kafka/source/KafkaSourceITCase.java | 64 +++++++++ .../api/operators/SourceOperator.java | 48 +++++-- .../source/SourceOperatorEventTimeTest.java | 121 +++++++--------- .../source/TestingSourceOperator.java | 58 ++++++++ 8 files changed, 352 insertions(+), 85 deletions(-) diff --git a/flink-connectors/flink-connector-base/pom.xml b/flink-connectors/flink-connector-base/pom.xml index 1af4fc818f5fc..4f8ff0eabd1a6 100644 --- a/flink-connectors/flink-connector-base/pom.xml +++ b/flink-connectors/flink-connector-base/pom.xml @@ -72,6 +72,14 @@ test + + org.apache.flink + flink-streaming-java + ${project.version} + test + test-jar + + diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java index 416dbb2a297d5..18d49f3caca93 100644 --- a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/SourceReaderBase.java @@ -324,6 +324,9 @@ private SplitContext(String splitId, SplitStateT state) { SourceOutput getOrCreateSplitOutput(ReaderOutput mainOutput) { if (sourceOutput == null) { + // The split output should have been created when AddSplitsEvent was processed in + // SourceOperator. Here we just use this method to get the previously created + // output. sourceOutput = mainOutput.createOutputForSplit(splitId); } return sourceOutput; diff --git a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java index 4f2ff6af6d76d..550cb953e80ea 100644 --- a/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java +++ b/flink-connectors/flink-connector-base/src/main/java/org/apache/flink/connector/base/source/reader/splitreader/SplitReader.java @@ -50,6 +50,11 @@ public interface SplitReader { /** * Handle the split changes. This call should be non-blocking. * + *

For the consistency of internal state in SourceReaderBase, if an invalid split is added to + * the reader (for example splits without any records), it should be put back into {@link + * RecordsWithSplitIds} as finished splits so that SourceReaderBase could be able to clean up + * resources created for it. + * * @param splitsChanges the split changes that the SplitReader needs to handle. */ void handleSplitsChanges(SplitsChange splitsChanges); diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java index 4a5544d414802..d17e1cfcf8756 100644 --- a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/SourceReaderBaseTest.java @@ -18,10 +18,14 @@ package org.apache.flink.connector.base.source.reader; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkOutput; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceSplit; import org.apache.flink.api.connector.source.mocks.MockSourceSplit; +import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager; import org.apache.flink.connector.base.source.reader.mocks.MockSourceReader; @@ -37,9 +41,22 @@ import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; import org.apache.flink.core.io.InputStatus; +import org.apache.flink.core.testutils.CommonTestUtils; +import org.apache.flink.runtime.source.event.AddSplitEvent; +import org.apache.flink.streaming.api.operators.SourceOperator; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.io.PushingAsyncDataInput; +import org.apache.flink.streaming.runtime.streamrecord.LatencyMarker; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -48,12 +65,15 @@ import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; +import static org.apache.flink.streaming.api.operators.source.TestingSourceOperator.createTestOperator; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; /** A unit test class for {@link SourceReaderBase}. */ public class SourceReaderBaseTest extends SourceReaderTestBase { + private static final Logger LOG = LoggerFactory.getLogger(SourceReaderBaseTest.class); + @Test void testExceptionInSplitReader() { assertThatThrownBy( @@ -239,6 +259,83 @@ void testPollNextReturnMoreAvailableWhenAllSplitFetcherCloseWithLeftoverElementI .isEqualTo(InputStatus.MORE_AVAILABLE); } + @ParameterizedTest(name = "Emit record before split addition: {0}") + @ValueSource(booleans = {true, false}) + void testPerSplitWatermark(boolean emitRecordBeforeSplitAddition) throws Exception { + MockSplitReader mockSplitReader = + MockSplitReader.newBuilder() + .setNumRecordsPerSplitPerFetch(3) + .setBlockingFetch(true) + .build(); + + MockSourceReader reader = + new MockSourceReader( + new FutureCompletingBlockingQueue<>(), + () -> mockSplitReader, + new Configuration(), + new TestingReaderContext()); + + SourceOperator sourceOperator = + createTestOperator( + reader, + WatermarkStrategy.forGenerator( + (context) -> new OnEventWatermarkGenerator()), + true); + + MockSourceSplit splitA = new MockSourceSplit(0, 0, 3); + splitA.addRecord(100); + splitA.addRecord(200); + splitA.addRecord(300); + + MockSourceSplit splitB = new MockSourceSplit(1, 0, 3); + splitB.addRecord(150); + splitB.addRecord(250); + splitB.addRecord(350); + + WatermarkCollectingDataOutput output = new WatermarkCollectingDataOutput(); + + if (emitRecordBeforeSplitAddition) { + sourceOperator.emitNext(output); + } + + AddSplitEvent addSplitsEvent = + new AddSplitEvent<>(Arrays.asList(splitA, splitB), new MockSourceSplitSerializer()); + sourceOperator.handleOperatorEvent(addSplitsEvent); + + // First 3 records from split A should not generate any watermarks + CommonTestUtils.waitUtil( + () -> { + try { + sourceOperator.emitNext(output); + } catch (Exception e) { + LOG.warn("Exception caught at emitting records", e); + return false; + } + return output.numRecords == 3; + }, + Duration.ofSeconds(10), + String.format( + "%d out of 3 records are received within timeout", output.numRecords)); + assertThat(output.watermarks).isEmpty(); + + CommonTestUtils.waitUtil( + () -> { + try { + sourceOperator.emitNext(output); + } catch (Exception e) { + LOG.warn("Exception caught at emitting records", e); + return false; + } + return output.numRecords == 6; + }, + Duration.ofSeconds(10), + String.format( + "%d out of 6 records are received within timeout", output.numRecords)); + + assertThat(output.watermarks).hasSize(3); + assertThat(output.watermarks).containsExactly(150L, 250L, 300L); + } + // ---------------- helper methods ----------------- @Override @@ -379,4 +476,37 @@ private void shutdownAllSplitFetcher() { } } } + + private static class OnEventWatermarkGenerator implements WatermarkGenerator { + + @Override + public void onEvent(Integer event, long eventTimestamp, WatermarkOutput output) { + output.emitWatermark(new org.apache.flink.api.common.eventtime.Watermark(event)); + } + + @Override + public void onPeriodicEmit(WatermarkOutput output) {} + } + + private static class WatermarkCollectingDataOutput + implements PushingAsyncDataInput.DataOutput { + int numRecords = 0; + final List watermarks = new ArrayList<>(); + + @Override + public void emitRecord(StreamRecord streamRecord) { + numRecords++; + } + + @Override + public void emitWatermark(Watermark watermark) throws Exception { + watermarks.add(watermark.getTimestamp()); + } + + @Override + public void emitWatermarkStatus(WatermarkStatus watermarkStatus) {} + + @Override + public void emitLatencyMarker(LatencyMarker latencyMarker) {} + } } diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java index 71108d284add7..72690961e6b73 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java @@ -20,6 +20,9 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.accumulators.ListAccumulator; +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkGenerator; +import org.apache.flink.api.common.eventtime.WatermarkOutput; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -38,6 +41,7 @@ import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.operators.StreamMap; @@ -70,11 +74,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.PARTITION; import static org.apache.flink.connector.kafka.testutils.KafkaSourceExternalContext.SplitMappingMode.TOPIC; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; /** Unite test class for {@link KafkaSource}. */ @@ -256,6 +262,53 @@ public void testBasicReadWithoutGroupId(boolean enableObjectReuse) throws Except "testBasicReadWithoutGroupId"); executeAndVerify(env, stream); } + + @Test + public void testPerPartitionWatermark() throws Throwable { + String watermarkTopic = "watermarkTestTopic-" + UUID.randomUUID(); + KafkaSourceTestEnv.createTestTopic(watermarkTopic, 2, 1); + List> records = + Arrays.asList( + new ProducerRecord<>(watermarkTopic, 0, 100L, null, 100), + new ProducerRecord<>(watermarkTopic, 0, 200L, null, 200), + new ProducerRecord<>(watermarkTopic, 0, 300L, null, 300), + new ProducerRecord<>(watermarkTopic, 1, 150L, null, 150), + new ProducerRecord<>(watermarkTopic, 1, 250L, null, 250), + new ProducerRecord<>(watermarkTopic, 1, 350L, null, 350)); + KafkaSourceTestEnv.produceToKafka(records); + KafkaSource source = + KafkaSource.builder() + .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings) + .setTopics(watermarkTopic) + .setGroupId("watermark-test") + .setDeserializer(new TestingKafkaRecordDeserializationSchema(false)) + .setStartingOffsets(OffsetsInitializer.earliest()) + .setBounded(OffsetsInitializer.latest()) + .build(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.fromSource( + source, + WatermarkStrategy.forGenerator( + (context) -> new OnEventWatermarkGenerator()), + "testPerPartitionWatermark") + .process( + new ProcessFunction() { + @Override + public void processElement( + PartitionAndValue value, + ProcessFunction.Context ctx, + Collector out) { + assertThat(ctx.timestamp()) + .as( + "Event time should never behind watermark " + + "because of per-split watermark multiplexing logic") + .isGreaterThanOrEqualTo( + ctx.timerService().currentWatermark()); + } + }); + env.execute(); + } } /** Integration test based on connector testing framework. */ @@ -400,4 +453,15 @@ public void invoke(PartitionAndValue value, Context context) { } }); } + + private static class OnEventWatermarkGenerator + implements WatermarkGenerator { + @Override + public void onEvent(PartitionAndValue event, long eventTimestamp, WatermarkOutput output) { + output.emitWatermark(new Watermark(eventTimestamp)); + } + + @Override + public void onPeriodicEmit(WatermarkOutput output) {} + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java index 5660bc1546c9d..53621bd1b6e0b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperator.java @@ -67,6 +67,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -154,6 +155,8 @@ public class SourceOperator extends AbstractStr private final SourceOperatorAvailabilityHelper availabilityHelper = new SourceOperatorAvailabilityHelper(); + private final List outputPendingSplits = new ArrayList<>(); + private enum OperatingMode { READING, WAITING_FOR_ALIGNMENT, @@ -396,11 +399,7 @@ private DataInputStatus emitNextNotReading(DataOutput output) throws Except watermarkAlignmentParams.getUpdateInterval(), watermarkAlignmentParams.getUpdateInterval()); } - currentMainOutput = - eventTimeLogic.createMainOutput(output, this::onWatermarkEmitted); - initializeLatencyMarkerEmitter(output); - lastInvokedOutput = output; - this.operatingMode = OperatingMode.READING; + initializeMainOutput(output); return convertToInternalStatus(sourceReader.pollNext(currentMainOutput)); case SOURCE_STOPPED: this.operatingMode = OperatingMode.DATA_FINISHED; @@ -423,6 +422,15 @@ private DataInputStatus emitNextNotReading(DataOutput output) throws Except } } + private void initializeMainOutput(DataOutput output) { + currentMainOutput = eventTimeLogic.createMainOutput(output, this::onWatermarkEmitted); + initializeLatencyMarkerEmitter(output); + lastInvokedOutput = output; + // Create per-split output for pending splits added before main output is initialized + createOutputForSplits(outputPendingSplits); + this.operatingMode = OperatingMode.READING; + } + private void initializeLatencyMarkerEmitter(DataOutput output) { long latencyTrackingInterval = getExecutionConfig().isLatencyTrackingConfigured() @@ -515,11 +523,7 @@ public void handleOperatorEvent(OperatorEvent event) { updateMaxDesiredWatermark((WatermarkAlignmentEvent) event); checkWatermarkAlignment(); } else if (event instanceof AddSplitEvent) { - try { - sourceReader.addSplits(((AddSplitEvent) event).splits(splitSerializer)); - } catch (IOException e) { - throw new FlinkRuntimeException("Failed to deserialize the splits.", e); - } + handleAddSplitsEvent(((AddSplitEvent) event)); } else if (event instanceof SourceEventWrapper) { sourceReader.handleSourceEvents(((SourceEventWrapper) event).getSourceEvent()); } else if (event instanceof NoMoreSplitsEvent) { @@ -529,6 +533,30 @@ public void handleOperatorEvent(OperatorEvent event) { } } + private void handleAddSplitsEvent(AddSplitEvent event) { + try { + List newSplits = event.splits(splitSerializer); + if (operatingMode == OperatingMode.OUTPUT_NOT_INITIALIZED) { + // For splits arrived before the main output is initialized, store them into the + // pending list. Outputs of these splits will be created once the main output is + // ready. + outputPendingSplits.addAll(newSplits); + } else { + // Create output directly for new splits if the main output is already initialized. + createOutputForSplits(newSplits); + } + sourceReader.addSplits(newSplits); + } catch (IOException e) { + throw new FlinkRuntimeException("Failed to deserialize the splits.", e); + } + } + + private void createOutputForSplits(List newSplits) { + for (SplitT split : newSplits) { + currentMainOutput.createOutputForSplit(split.splitId()); + } + } + private void updateMaxDesiredWatermark(WatermarkAlignmentEvent event) { currentMaxDesiredWatermark = event.getMaxWatermark(); sourceMetricGroup.updateMaxDesiredWatermark(currentMaxDesiredWatermark); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/SourceOperatorEventTimeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/SourceOperatorEventTimeTest.java index 41b1bdfb3a8ee..959bc82402f6a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/SourceOperatorEventTimeTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/SourceOperatorEventTimeTest.java @@ -18,30 +18,17 @@ package org.apache.flink.streaming.api.operators.source; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.mocks.MockSourceSplit; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer; import org.apache.flink.core.io.InputStatus; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateInitializationContextImpl; -import org.apache.flink.runtime.state.TestTaskStateManager; -import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.runtime.source.event.AddSplitEvent; import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.io.DataInputStatus; -import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; -import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; -import org.apache.flink.streaming.util.MockOutput; -import org.apache.flink.streaming.util.MockStreamConfig; import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; @@ -52,13 +39,13 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; import java.util.stream.Collectors; +import static org.apache.flink.streaming.api.operators.source.TestingSourceOperator.createTestOperator; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertThat; @@ -159,6 +146,35 @@ public void testPerSplitOutputEventWatermarks() throws Exception { result, new Watermark(100L), new Watermark(150L), new Watermark(200L)); } + @Test + public void testCreatingPerSplitOutputOnSplitAddition() throws Exception { + final WatermarkStrategy watermarkStrategy = + WatermarkStrategy.forGenerator((ctx) -> new OnEventTestWatermarkGenerator<>()); + + InterpretingSourceReader reader = + new InterpretingSourceReader( + // No watermark (no record from split 2, whose watermark is Long.MIN_VALUE) + (output) -> output.createOutputForSplit("1").collect(0, 100L), + (output) -> output.createOutputForSplit("1").collect(0, 200L), + (output) -> output.createOutputForSplit("1").collect(0, 300L), + // Emit watermark 150 (from the 1st record of split 2) + (output) -> output.createOutputForSplit("2").collect(0, 150L), + // Emit watermark 300 (from the 3rd record in split 1) + (output) -> output.createOutputForSplit("2").collect(0, 400L)); + SourceOperator sourceOperator = + createTestOperator(reader, watermarkStrategy, emitProgressiveWatermarks); + + // Add two splits to SourceOperator. Output for two splits should be created during event + // handling. + sourceOperator.handleOperatorEvent( + new AddSplitEvent<>( + Arrays.asList(new MockSourceSplit(1), new MockSourceSplit(2)), + new MockSourceSplitSerializer())); + + final List result = testSequenceOfWatermarks(sourceOperator); + assertWatermarksOrEmpty(result, new Watermark(150L), new Watermark(300L)); + } + // ------------------------------------------------------------------------ // test execution helpers // ------------------------------------------------------------------------ @@ -186,9 +202,18 @@ private final List testSequenceOfWatermarks( final WatermarkStrategy watermarkStrategy, final Consumer>... actions) throws Exception { + final SourceReader reader = new InterpretingSourceReader(actions); + final SourceOperator sourceOperator = + createTestOperator(reader, watermarkStrategy, emitProgressiveWatermarks); - final List allEvents = - testSequenceOfEvents(emitProgressiveWatermarks, watermarkStrategy, actions); + return testSequenceOfWatermarks(sourceOperator); + } + + @SuppressWarnings("FinalPrivateMethod") + private final List testSequenceOfWatermarks( + SourceOperator sourceOperator) throws Exception { + + final List allEvents = testSequenceOfEvents(sourceOperator); return allEvents.stream() .filter((evt) -> evt instanceof Watermark) @@ -197,23 +222,13 @@ private final List testSequenceOfWatermarks( } @SuppressWarnings("FinalPrivateMethod") - @SafeVarargs private final List testSequenceOfEvents( - final boolean emitProgressiveWatermarks, - final WatermarkStrategy watermarkStrategy, - final Consumer>... actions) - throws Exception { + final SourceOperator sourceOperator) throws Exception { final CollectingDataOutput out = new CollectingDataOutput<>(); - final TestProcessingTimeService timeService = new TestProcessingTimeService(); - timeService.setCurrentTime(Integer.MAX_VALUE); // start somewhere that is not zero - - final SourceReader reader = new InterpretingSourceReader(actions); - - final SourceOperator sourceOperator = - createTestOperator( - reader, watermarkStrategy, timeService, emitProgressiveWatermarks); + final TestProcessingTimeService timeService = + ((TestProcessingTimeService) sourceOperator.getProcessingTimeService()); while (sourceOperator.emitNext(out) != DataInputStatus.END_OF_INPUT) { timeService.setCurrentTime(timeService.getCurrentProcessingTime() + 100); @@ -222,50 +237,6 @@ private final List testSequenceOfEvents( return out.events; } - // ------------------------------------------------------------------------ - // test setup helpers - // ------------------------------------------------------------------------ - - private static SourceOperator createTestOperator( - SourceReader reader, - WatermarkStrategy watermarkStrategy, - ProcessingTimeService timeService, - boolean emitProgressiveWatermarks) - throws Exception { - - final OperatorStateStore operatorStateStore = - new MemoryStateBackend() - .createOperatorStateBackend( - new MockEnvironmentBuilder().build(), - "test-operator", - Collections.emptyList(), - new CloseableRegistry()); - - final StateInitializationContext stateContext = - new StateInitializationContextImpl(null, operatorStateStore, null, null, null); - - final SourceOperator sourceOperator = - new TestingSourceOperator<>( - reader, watermarkStrategy, timeService, emitProgressiveWatermarks); - - sourceOperator.setup( - new SourceOperatorStreamTask( - new StreamMockEnvironment( - new Configuration(), - new Configuration(), - new ExecutionConfig(), - 1L, - new MockInputSplitProvider(), - 1, - new TestTaskStateManager())), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(new ArrayList<>())); - sourceOperator.initializeState(stateContext); - sourceOperator.open(); - - return sourceOperator; - } - // ------------------------------------------------------------------------ // test mocks // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/TestingSourceOperator.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/TestingSourceOperator.java index 5fbfd7c98b7fa..d25226e9bdd2f 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/TestingSourceOperator.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/source/TestingSourceOperator.java @@ -20,18 +20,34 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.state.OperatorStateStore; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.mocks.MockSourceSplit; import org.apache.flink.api.connector.source.mocks.MockSourceSplitSerializer; import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.TestTaskStateManager; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask; +import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; import org.apache.flink.streaming.util.MockStreamingRuntimeContext; +import java.util.ArrayList; +import java.util.Collections; + /** A SourceOperator extension to simplify test setup. */ public class TestingSourceOperator extends SourceOperator { @@ -100,4 +116,46 @@ public ExecutionConfig getExecutionConfig() { cfg.setAutoWatermarkInterval(100); return cfg; } + + public static SourceOperator createTestOperator( + SourceReader reader, + WatermarkStrategy watermarkStrategy, + boolean emitProgressiveWatermarks) + throws Exception { + + final OperatorStateStore operatorStateStore = + new HashMapStateBackend() + .createOperatorStateBackend( + new MockEnvironmentBuilder().build(), + "test-operator", + Collections.emptyList(), + new CloseableRegistry()); + + final StateInitializationContext stateContext = + new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + + TestProcessingTimeService timeService = new TestProcessingTimeService(); + timeService.setCurrentTime(Integer.MAX_VALUE); // start somewhere that is not zero + + final SourceOperator sourceOperator = + new TestingSourceOperator<>( + reader, watermarkStrategy, timeService, emitProgressiveWatermarks); + + sourceOperator.setup( + new SourceOperatorStreamTask( + new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager())), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(new ArrayList<>())); + sourceOperator.initializeState(stateContext); + sourceOperator.open(); + + return sourceOperator; + } } From 0be2a28d1d1eae03a8cf9d64ab3e7d68f5d87b64 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 22 Mar 2022 11:44:13 +0100 Subject: [PATCH 107/258] [FLINK-26783] Do not trigger global failover if failed during commiting side-effects during stop-with-savepoint If a job fails during commitin side-effects of the stop-with-savepoint and we restart to the latest checkpoint instead of savepoint, we might end up producing duplicates. On the other hand if we restart to the savepoint we end up in a situation where a running Flink job depends on the existence of the savepoint. In this commit we do not trigger a global failover in case the savepoint completed successfully, but the job failed during committing side effects. In that case we will finish the completable future with an exception that explains that the savepoint is consistent, but it might have uncommitted side effects and we ask users to manually restart a job from that savepoint if they want to commit side effects. This closes #19198 --- .../StopWithSavepointStoppingException.java | 51 ++++++++ ...opWithSavepointTerminationHandlerImpl.java | 14 +-- ...thSavepointTerminationHandlerImplTest.java | 20 +--- .../test/checkpointing/SavepointITCase.java | 109 +++++++++++++++++- 4 files changed, 162 insertions(+), 32 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java new file mode 100644 index 0000000000000..d41dbc81f8cc5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.scheduler.stopwithsavepoint; + +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.throwable.ThrowableAnnotation; +import org.apache.flink.runtime.throwable.ThrowableType; +import org.apache.flink.util.FlinkException; + +/** + * Exception thrown when a savepoint has been created successfully when stopping with savepoint, but + * the job has not finished. In that case side-effects might have not been committed. This exception + * is used to communicate that to the use. + */ +@Experimental +@ThrowableAnnotation(ThrowableType.NonRecoverableError) +public class StopWithSavepointStoppingException extends FlinkException { + private final String savepointPath; + + public StopWithSavepointStoppingException(String savepointPath, JobID jobID) { + super( + String.format( + "A savepoint has been created at: %s, but the corresponding job %s failed " + + "during stopping. The savepoint is consistent, but might have " + + "uncommitted transactions. If you want to commit the transaction " + + "please restart a job from this savepoint.", + savepointPath, jobID)); + this.savepointPath = savepointPath; + } + + public String getSavepointPath() { + return savepointPath; + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java index aec32d34981f7..f3416135656de 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImpl.java @@ -24,7 +24,6 @@ import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.scheduler.SchedulerNG; -import org.apache.flink.util.FlinkException; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -167,16 +166,13 @@ private void handleAnyExecutionNotFinished(Set notFinishedExecut */ private void terminateExceptionallyWithGlobalFailover( Iterable unfinishedExecutionStates, String savepointPath) { - String errorMessage = - String.format( - "Inconsistent execution state after stopping with savepoint. At least one execution is still in one of the following states: %s. A global fail-over is triggered to recover the job %s.", - StringUtils.join(unfinishedExecutionStates, ", "), jobId); - FlinkException inconsistentFinalStateException = new FlinkException(errorMessage); + StopWithSavepointStoppingException inconsistentFinalStateException = + new StopWithSavepointStoppingException(savepointPath, jobId); log.warn( - "A savepoint was created at {} but the corresponding job {} didn't terminate successfully.", - savepointPath, - jobId, + "Inconsistent execution state after stopping with savepoint. At least one" + + " execution is still in one of the following states: {}.", + StringUtils.join(unfinishedExecutionStates, ", "), inconsistentFinalStateException); scheduler.handleGlobalFailure(inconsistentFinalStateException); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java index cab4abeb2a37a..f0c2bee92df90 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointTerminationHandlerImplTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.runtime.checkpoint.CheckpointProperties; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; import org.apache.flink.runtime.checkpoint.TestingCheckpointScheduling; @@ -31,7 +30,6 @@ import org.apache.flink.runtime.state.testutils.EmptyStreamStateHandle; import org.apache.flink.runtime.state.testutils.TestCompletedCheckpointStorageLocation; import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FlinkException; import org.apache.flink.util.TestLogger; import org.junit.Test; @@ -152,10 +150,6 @@ public void testFailedTerminationHandling() throws ExecutionException, Interrupt createTestInstance(globalFailOverTriggered::complete); final ExecutionState expectedNonFinishedState = ExecutionState.FAILED; - final String expectedErrorMessage = - String.format( - "Inconsistent execution state after stopping with savepoint. At least one execution is still in one of the following states: %s. A global fail-over is triggered to recover the job %s.", - expectedNonFinishedState, JOB_ID); final EmptyStreamStateHandle streamStateHandle = new EmptyStreamStateHandle(); final CompletedCheckpoint completedSavepoint = createCompletedSavepoint(streamStateHandle); @@ -168,24 +162,14 @@ public void testFailedTerminationHandling() throws ExecutionException, Interrupt testInstance.getSavepointPath().get(); fail("An ExecutionException is expected."); } catch (Throwable e) { - final Optional actualFlinkException = - ExceptionUtils.findThrowable(e, FlinkException.class); + final Optional actualFlinkException = + ExceptionUtils.findThrowable(e, StopWithSavepointStoppingException.class); assertTrue( "A FlinkException should have been thrown.", actualFlinkException.isPresent()); - assertThat( - actualFlinkException.get(), - FlinkMatchers.containsMessage(expectedErrorMessage)); } assertTrue("Global fail-over was not triggered.", globalFailOverTriggered.isDone()); - assertThat( - globalFailOverTriggered.get(), FlinkMatchers.containsMessage(expectedErrorMessage)); - assertFalse("Savepoint should not be discarded.", streamStateHandle.isDisposed()); - - assertFalse( - "Checkpoint scheduling should not be enabled in case of failure.", - checkpointScheduling.isEnabled()); } @Test(expected = UnsupportedOperationException.class) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index acc0ddd8a1eef..a318136fd5d7f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -59,9 +59,11 @@ import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.messages.FlinkJobNotFoundException; +import org.apache.flink.runtime.operators.testutils.ExpectedTestException; import org.apache.flink.runtime.rest.messages.EmptyRequestBody; import org.apache.flink.runtime.rest.messages.JobMessageParameters; import org.apache.flink.runtime.rest.messages.job.JobDetailsHeaders; +import org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointStoppingException; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.runtime.state.StateSnapshotContext; @@ -143,10 +145,12 @@ import static org.apache.flink.util.ExceptionUtils.assertThrowableWithMessage; import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; +import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -292,6 +296,79 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { public void initializeState(FunctionInitializationContext context) throws Exception {} } + private static final OneShotLatch stopWithSavepointRestartLatch = new OneShotLatch(); + + @Test + public void testStopWithSavepointFailsOverToSavepoint() throws Throwable { + int sinkParallelism = 5; + MiniClusterWithClientResource cluster = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberSlotsPerTaskManager(sinkParallelism + 1) + .build()); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.getConfig().setRestartStrategy(RestartStrategies.fixedDelayRestart(3, 10)); + env.setParallelism(1); + env.addSource(new InfiniteTestSource()) + .name("Infinite Source") + .map(new FailingOnCompletedSavepointMapFunction(2)) + .addSink(new DiscardingSink<>()) + // different parallelism to break chaining and add some concurrent tasks + .setParallelism(sinkParallelism); + + final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); + + cluster.before(); + try { + ClusterClient client = cluster.getClusterClient(); + client.submitJob(jobGraph).get(); + waitUntilAllTasksAreRunning(cluster.getRestClusterClient(), jobGraph.getJobID()); + + cluster.getMiniCluster().triggerCheckpoint(jobGraph.getJobID()).get(); + final CompletableFuture savepointCompleted = + client.stopWithSavepoint( + jobGraph.getJobID(), + true, + savepointDir.getAbsolutePath(), + SavepointFormatType.CANONICAL); + + final Throwable savepointException = + assertThrows(ExecutionException.class, savepointCompleted::get).getCause(); + assertThrowable( + savepointException, + throwable -> + throwable instanceof StopWithSavepointStoppingException + && throwable + .getMessage() + .startsWith("A savepoint has been created at: ")); + assertThat(client.getJobStatus(jobGraph.getJobID()).get(), is(JobStatus.FAILED)); + } finally { + cluster.after(); + } + } + + private static final class FailingOnCompletedSavepointMapFunction + extends RichMapFunction implements CheckpointListener { + private final long savepointId; + + private FailingOnCompletedSavepointMapFunction(long savepointId) { + this.savepointId = savepointId; + } + + @Override + public Integer map(Integer value) throws Exception { + return value; + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (checkpointId == savepointId) { + throw new ExpectedTestException(); + } + } + } + /** * Triggers a savepoint for a job that uses the FsStateBackend. We expect that all checkpoint * files are written to a new savepoint directory. @@ -924,7 +1001,8 @@ public void testStopWithSavepointFailingInSnapshotCreation() throws Exception { // 1. task failure restart // 2. job failover triggered by the CheckpointFailureManager 2, - assertInSnapshotCreationFailure()); + assertInSnapshotCreationFailure(), + true); } @Test @@ -937,8 +1015,26 @@ public void testStopWithSavepointFailingAfterSnapshotCreation() throws Exception // two restarts expected: // 1. task failure restart // 2. job failover triggered by SchedulerBase.stopWithSavepoint - 2, - assertAfterSnapshotCreationFailure()); + 0, + (jobId, actualException) -> { + if (ClusterOptions.isAdaptiveSchedulerEnabled(new Configuration())) { + return actualException + .getMessage() + .contains("Stop with savepoint operation could not be completed"); + } else { + Optional actualFlinkException = + findThrowable( + actualException, StopWithSavepointStoppingException.class); + return actualFlinkException + .map( + e -> + e.getMessage() + .startsWith( + "A savepoint has been created at:")) + .orElse(false); + } + }, + false); } @Test @@ -1051,7 +1147,8 @@ private static void testStopWithFailingSourceInOnePipeline( InfiniteTestSource failingSource, File savepointDir, int expectedMaximumNumberOfRestarts, - BiFunction exceptionAssertion) + BiFunction exceptionAssertion, + boolean shouldRestart) throws Exception { MiniClusterWithClientResource cluster = new MiniClusterWithClientResource( @@ -1107,7 +1204,9 @@ private static void testStopWithFailingSourceInOnePipeline( assertThrowable(e, ex -> exceptionAssertion.apply(jobGraph.getJobID(), e)); } - waitUntilAllTasksAreRunning(cluster.getRestClusterClient(), jobGraph.getJobID()); + if (shouldRestart) { + waitUntilAllTasksAreRunning(cluster.getRestClusterClient(), jobGraph.getJobID()); + } } finally { cluster.after(); } From 52cc77df9f11b8ba21de80c445b1261959cf315a Mon Sep 17 00:00:00 2001 From: zhangjingcun Date: Wed, 23 Mar 2022 12:24:27 +0800 Subject: [PATCH 108/258] [FLINK-26814][python][yarn] Ensure PyFlink jobs works in YARN application mode This closes #19208. --- docs/content.zh/docs/deployment/cli.md | 16 ++++++++++ docs/content/docs/deployment/cli.md | 16 ++++++++++ .../flink/configuration/ConfigConstants.java | 5 +++ .../flink/yarn/YarnClusterDescriptor.java | 32 ++++++++++++++++--- .../YarnApplicationClusterEntryPoint.java | 13 ++++++-- 5 files changed, 75 insertions(+), 7 deletions(-) diff --git a/docs/content.zh/docs/deployment/cli.md b/docs/content.zh/docs/deployment/cli.md index 68ec237f4f1d4..c0af1a426a27f 100644 --- a/docs/content.zh/docs/deployment/cli.md +++ b/docs/content.zh/docs/deployment/cli.md @@ -418,6 +418,22 @@ $ ./bin/flink run \ --python examples/python/table/word_count.py ``` +- Run a PyFlink job using a [YARN cluster in Application Mode]({{< ref "docs/deployment/resource-providers/yarn" >}}#application-mode): +```bash +$ ./bin/flink run-application -t yarn-application \ + -Djobmanager.memory.process.size=1024m \ + -Dtaskmanager.memory.process.size=1024m \ + -Dyarn.application.name= \ + -Dyarn.ship-files=/path/to/shipfiles \ + -pyarch shipfiles/venv.zip \ + -pyclientexec venv.zip/venv/bin/python3 \ + -pyexec venv.zip/venv/bin/python3 \ + -py shipfiles/word_count.py +``` +Note It assumes that the Python dependencies needed to execute the job are already placed in the directory `/path/to/shipfiles`. For example, it should contain venv.zip and word_count.py for the above example. + +Note As it executes the job on the JobManager in YARN application mode, the paths specified in `-pyarch` and `-py` are paths relative to `shipfiles` which is the directory name of the shipped files. + - Run a PyFlink application on a native Kubernetes cluster having the cluster ID ``, it requires a docker image with PyFlink installed, please refer to [Enabling PyFlink in docker]({{< ref "docs/deployment/resource-providers/standalone/docker" >}}#enabling-python): ```bash $ ./bin/flink run-application \ diff --git a/docs/content/docs/deployment/cli.md b/docs/content/docs/deployment/cli.md index 983a941d16ba9..7a8e0ad04abf1 100644 --- a/docs/content/docs/deployment/cli.md +++ b/docs/content/docs/deployment/cli.md @@ -416,6 +416,22 @@ $ ./bin/flink run \ --python examples/python/table/word_count.py ``` +- Run a PyFlink job using a [YARN cluster in Application Mode]({{< ref "docs/deployment/resource-providers/yarn" >}}#application-mode): +```bash +$ ./bin/flink run-application -t yarn-application \ + -Djobmanager.memory.process.size=1024m \ + -Dtaskmanager.memory.process.size=1024m \ + -Dyarn.application.name= \ + -Dyarn.ship-files=/path/to/shipfiles \ + -pyarch shipfiles/venv.zip \ + -pyclientexec venv.zip/venv/bin/python3 \ + -pyexec venv.zip/venv/bin/python3 \ + -py shipfiles/word_count.py +``` +Note It assumes that the Python dependencies needed to execute the job are already placed in the directory `/path/to/shipfiles`. For example, it should contain venv.zip and word_count.py for the above example. + +Note As it executes the job on the JobManager in YARN application mode, the paths specified in `-pyarch` and `-py` are paths relative to `shipfiles` which is the directory name of the shipped files. + - Run a PyFlink application on a native Kubernetes cluster having the cluster ID ``, it requires a docker image with PyFlink installed, please refer to [Enabling PyFlink in docker]({{< ref "docs/deployment/resource-providers/standalone/docker" >}}#enabling-python): ```bash $ ./bin/flink run-application \ diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java index c14399ed271bb..e7c441ccc5194 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java @@ -1732,6 +1732,11 @@ public final class ConfigConstants { /** The environment variable name which contains the location of the opt directory. */ public static final String ENV_FLINK_OPT_DIR = "FLINK_OPT_DIR"; + /** + * The default Flink opt directory if none has been specified via {@link #ENV_FLINK_OPT_DIR}. + */ + public static final String DEFAULT_FLINK_OPT_DIR = "opt"; + /** The environment variable name which contains the location of the plugins folder. */ public static final String ENV_FLINK_PLUGINS_DIR = "FLINK_PLUGINS_DIR"; diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index c99fd9c704fa4..e90bf65c5ebfc 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -27,6 +27,7 @@ import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.deployment.application.ApplicationConfiguration; import org.apache.flink.client.program.ClusterClientProvider; +import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.ConfigOption; @@ -118,8 +119,10 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.flink.client.deployment.application.ApplicationConfiguration.APPLICATION_MAIN_CLASS; import static org.apache.flink.configuration.ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR; import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_LIB_DIR; +import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_OPT_DIR; import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -447,11 +450,15 @@ public ClusterClientProvider deployApplicationCluster( applicationConfiguration.applyToConfiguration(flinkConfiguration); - final List pipelineJars = - flinkConfiguration - .getOptional(PipelineOptions.JARS) - .orElse(Collections.emptyList()); - Preconditions.checkArgument(pipelineJars.size() == 1, "Should only have one jar"); + // No need to do pipelineJars validation if it is a PyFlink job. + if (!(PackagedProgramUtils.isPython(applicationConfiguration.getApplicationClassName()) + || PackagedProgramUtils.isPython(applicationConfiguration.getProgramArguments()))) { + final List pipelineJars = + flinkConfiguration + .getOptional(PipelineOptions.JARS) + .orElse(Collections.emptyList()); + Preconditions.checkArgument(pipelineJars.size() == 1, "Should only have one jar"); + } try { return deployInternal( @@ -910,6 +917,17 @@ private ApplicationReport startAppMaster( LocalResourceType.ARCHIVE); } + // only for application mode + // Python jar file only needs to be shipped and should not be added to classpath. + if (YarnApplicationClusterEntryPoint.class.getName().equals(yarnClusterEntrypoint) + && PackagedProgramUtils.isPython(configuration.get(APPLICATION_MAIN_CLASS))) { + fileUploader.registerMultipleLocalResources( + Collections.singletonList( + new Path(PackagedProgramUtils.getPythonJar().toURI())), + ConfigConstants.DEFAULT_FLINK_OPT_DIR, + LocalResourceType.FILE); + } + // Upload and register user jars final List userClassPaths = fileUploader.registerMultipleLocalResources( @@ -1134,6 +1152,10 @@ private ApplicationReport startAppMaster( // set Flink app class path appMasterEnv.put(YarnConfigKeys.ENV_FLINK_CLASSPATH, classPathBuilder.toString()); + // Set FLINK_OPT_DIR to `opt` folder under working dir in container + appMasterEnv.put( + ENV_FLINK_OPT_DIR, Path.CUR_DIR + "/" + ConfigConstants.DEFAULT_FLINK_OPT_DIR); + // set Flink on YARN internal configuration values appMasterEnv.put(YarnConfigKeys.FLINK_DIST_JAR, localResourceDescFlinkJar.toString()); appMasterEnv.put(YarnConfigKeys.ENV_APP_ID, appId.toString()); diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnApplicationClusterEntryPoint.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnApplicationClusterEntryPoint.java index d38ebaeb666f5..d619d370f5cfb 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnApplicationClusterEntryPoint.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnApplicationClusterEntryPoint.java @@ -24,6 +24,7 @@ import org.apache.flink.client.program.DefaultPackagedProgramRetriever; import org.apache.flink.client.program.PackagedProgram; import org.apache.flink.client.program.PackagedProgramRetriever; +import org.apache.flink.client.program.PackagedProgramUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; @@ -132,9 +133,17 @@ private static PackagedProgramRetriever getPackagedProgramRetriever( throws FlinkException { final File userLibDir = YarnEntrypointUtils.getUsrLibDir(configuration).orElse(null); - final File userApplicationJar = getUserApplicationJar(userLibDir, configuration); + + // No need to do pipelineJars validation if it is a PyFlink job. + if (!(PackagedProgramUtils.isPython(jobClassName) + || PackagedProgramUtils.isPython(programArguments))) { + final File userApplicationJar = getUserApplicationJar(userLibDir, configuration); + return DefaultPackagedProgramRetriever.create( + userLibDir, userApplicationJar, jobClassName, programArguments, configuration); + } + return DefaultPackagedProgramRetriever.create( - userLibDir, userApplicationJar, jobClassName, programArguments, configuration); + userLibDir, jobClassName, programArguments, configuration); } private static File getUserApplicationJar( From 372b82c841906c5b7f3808e9206a3a34d172d4b8 Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Tue, 29 Mar 2022 11:00:28 +0800 Subject: [PATCH 109/258] [FLINK-26421][python] Remove TableConfig from StreamTableEnvironment#create Following: 57742b85095147711070c566069244c40ed8e77c remove the `TableConfig` from `StreamTableEnviroment#create()` and allow configuration only via `EnviromentSettings.with_configuration()`. --- .../pyflink/table/table_environment.py | 29 ++----------------- 1 file changed, 3 insertions(+), 26 deletions(-) diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 6e7379922355e..e001f36d37d90 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -1653,7 +1653,6 @@ def __init__(self, j_tenv): @staticmethod def create(stream_execution_environment: StreamExecutionEnvironment = None, # type: ignore - table_config: TableConfig = None, environment_settings: EnvironmentSettings = None) -> 'StreamTableEnvironment': """ Creates a :class:`~pyflink.table.StreamTableEnvironment`. @@ -1664,10 +1663,6 @@ def create(stream_execution_environment: StreamExecutionEnvironment = None, # t # create with StreamExecutionEnvironment. >>> env = StreamExecutionEnvironment.get_execution_environment() >>> table_env = StreamTableEnvironment.create(env) - # create with StreamExecutionEnvironment and TableConfig. - >>> table_config = TableConfig() - >>> table_config.set_null_check(False) - >>> table_env = StreamTableEnvironment.create(env, table_config) # create with StreamExecutionEnvironment and EnvironmentSettings. >>> configuration = Configuration() >>> configuration.set_string('execution.buffer-timeout', '1 min') @@ -1685,27 +1680,15 @@ def create(stream_execution_environment: StreamExecutionEnvironment = None, # t :param stream_execution_environment: The :class:`~pyflink.datastream.StreamExecutionEnvironment` of the TableEnvironment. - :param table_config: The configuration of the TableEnvironment, optional. :param environment_settings: The environment settings used to instantiate the TableEnvironment. :return: The StreamTableEnvironment created from given StreamExecutionEnvironment and configuration. """ if stream_execution_environment is None and \ - table_config is None and \ environment_settings is None: raise ValueError("No argument found, the param 'stream_execution_environment' " "or 'environment_settings' is required.") - elif stream_execution_environment is None and \ - table_config is not None and \ - environment_settings is None: - raise ValueError("Only the param 'table_config' is found, " - "the param 'stream_execution_environment' is also required.") - if table_config is not None and \ - environment_settings is not None: - raise ValueError("The param 'table_config' and " - "'environment_settings' cannot be used at the same time") - gateway = get_gateway() if environment_settings is not None: if stream_execution_environment is None: @@ -1716,15 +1699,9 @@ def create(stream_execution_environment: StreamExecutionEnvironment = None, # t stream_execution_environment._j_stream_execution_environment, environment_settings._j_environment_settings) else: - if table_config is not None: - warnings.warn("Deprecated in 1.15, please use EnvironmentSettings.", - DeprecationWarning) - j_tenv = gateway.jvm.StreamTableEnvironment.create( - stream_execution_environment._j_stream_execution_environment, - table_config._j_table_config) - else: - j_tenv = gateway.jvm.StreamTableEnvironment.create( - stream_execution_environment._j_stream_execution_environment) + j_tenv = gateway.jvm.StreamTableEnvironment.create( + stream_execution_environment._j_stream_execution_environment) + return StreamTableEnvironment(j_tenv) def from_data_stream(self, From 1e679d43c5b472b57e223f7593902873694b6786 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Fri, 18 Mar 2022 14:41:02 +0800 Subject: [PATCH 110/258] [hotfix][release] Update the compatibility table for the release 1.14 and 1.15 This closes #19144. --- docs/content.zh/docs/ops/upgrading.md | 66 +++++++++++++++++++++++++++ docs/content/docs/ops/upgrading.md | 66 +++++++++++++++++++++++++++ 2 files changed, 132 insertions(+) diff --git a/docs/content.zh/docs/ops/upgrading.md b/docs/content.zh/docs/ops/upgrading.md index 73d4c3cf7604b..b12111e43bd9b 100644 --- a/docs/content.zh/docs/ops/upgrading.md +++ b/docs/content.zh/docs/ops/upgrading.md @@ -216,6 +216,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: 1.11.x 1.12.x 1.13.x + 1.14.x + 1.15.x Limitations @@ -235,6 +237,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: + + The maximum parallelism of a job that was migrated from Flink 1.1.x to 1.2.x+ is currently fixed as the parallelism of the job. This means that the parallelism can not be increased after migration. This limitation might be removed in a future bugfix release. @@ -254,6 +258,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O When migrating from Flink 1.2.x to Flink 1.3.x+, changing parallelism at the same time is not supported. Users have to first take a savepoint after migrating to Flink 1.3.x+, and then change @@ -279,6 +285,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O Migrating from Flink 1.3.0 to Flink 1.4.[0,1] will fail if the savepoint contains Scala case classes. Users have to directly migrate to 1.4.2+ instead. @@ -296,6 +304,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -313,6 +323,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O There is a known issue with resuming broadcast state created with 1.5.x in versions 1.6.x up to 1.6.2, and 1.7.0: FLINK-11087. Users upgrading to 1.6.x or 1.7.x series need to directly migrate to minor versions higher than 1.6.2 and 1.7.0, @@ -333,6 +345,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -350,6 +364,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -367,6 +383,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -384,6 +402,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -401,6 +421,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -418,6 +440,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -435,6 +459,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O + O + O @@ -452,8 +478,48 @@ Savepoints are compatible across Flink versions as indicated by the table below: O + O + O Don't upgrade from 1.12.x to 1.13.x with an unaligned checkpoint. Please use a savepoint for migrating. + + 1.14.x + + + + + + + + + + + + + + O + O + + + + 1.15.x + + + + + + + + + + + + + + + O + + diff --git a/docs/content/docs/ops/upgrading.md b/docs/content/docs/ops/upgrading.md index 8b1480d8e9181..fb3fd159416cf 100644 --- a/docs/content/docs/ops/upgrading.md +++ b/docs/content/docs/ops/upgrading.md @@ -231,6 +231,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: 1.11.x 1.12.x 1.13.x + 1.14.x + 1.15.x Limitations @@ -250,6 +252,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: + + The maximum parallelism of a job that was migrated from Flink 1.1.x to 1.2.x+ is currently fixed as the parallelism of the job. This means that the parallelism can not be increased after migration. This limitation might be removed in a future bugfix release. @@ -269,6 +273,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O When migrating from Flink 1.2.x to Flink 1.3.x+, changing parallelism at the same time is not supported. Users have to first take a savepoint after migrating to Flink 1.3.x+, and then change @@ -294,6 +300,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O Migrating from Flink 1.3.0 to Flink 1.4.[0,1] will fail if the savepoint contains Scala case classes. Users have to directly migrate to 1.4.2+ instead. @@ -311,6 +319,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -328,6 +338,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O There is a known issue with resuming broadcast state created with 1.5.x in versions 1.6.x up to 1.6.2, and 1.7.0: FLINK-11087. Users upgrading to 1.6.x or 1.7.x series need to directly migrate to minor versions higher than 1.6.2 and 1.7.0, @@ -348,6 +360,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -365,6 +379,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -382,6 +398,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -399,6 +417,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -416,6 +436,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -433,6 +455,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O O + O + O @@ -450,6 +474,8 @@ Savepoints are compatible across Flink versions as indicated by the table below: O O + O + O @@ -467,8 +493,48 @@ Savepoints are compatible across Flink versions as indicated by the table below: O + O + O Don't upgrade from 1.12.x to 1.13.x with an unaligned checkpoint. Please use a savepoint for migrating. + + 1.14.x + + + + + + + + + + + + + + O + O + + + + 1.15.x + + + + + + + + + + + + + + + O + + From 716fa7426e59e8466ad475c9b1cafb7021aa1e58 Mon Sep 17 00:00:00 2001 From: shihong90 <2572805166@qq.com> Date: Sun, 14 Nov 2021 21:18:31 +0800 Subject: [PATCH 111/258] [FLINK-24351][docs] Translate "JSON Function" pages into Chinese This closes #17789. --- docs/data/sql_functions_zh.yml | 95 ++++++++++++++-------------------- 1 file changed, 38 insertions(+), 57 deletions(-) diff --git a/docs/data/sql_functions_zh.yml b/docs/data/sql_functions_zh.yml index ae8670fd71281..f5c2a7992dba1 100644 --- a/docs/data/sql_functions_zh.yml +++ b/docs/data/sql_functions_zh.yml @@ -707,11 +707,10 @@ json: - sql: IS JSON [ { VALUE | SCALAR | ARRAY | OBJECT } ] table: STRING.isJson([JsonType type]) description: | - Determine whether a given string is valid JSON. + 判定给定字符串是否为有效的 JSON。 - Specifying the optional type argument puts a constraint on which type of JSON object is - allowed. If the string is valid JSON, but not that type, `false` is returned. The default is - `VALUE`. + 指定可选类型参数将会限制 JSON 对象所允许的类型。 + 如果字符串是有效的 JSON,但不是指定的类型,则返回 `false`。默认值为 `VALUE`。 ``` -- TRUE @@ -742,9 +741,9 @@ json: - sql: JSON_EXISTS(jsonValue, path [ { TRUE | FALSE | UNKNOWN | ERROR } ON ERROR ]) table: STRING.jsonExists(STRING path [, JsonExistsOnError onError]) description: | - Determines whether a JSON string satisfies a given path search criterion. + 判定 JSON 字符串是否满足给定的路径搜索条件。 - If the error behavior is omitted, `FALSE ON ERROR` is assumed as the default. + 如果要忽略错误行为,那么将 `FALSE ON ERROR` 设为默认值。 ``` -- TRUE @@ -765,10 +764,9 @@ json: - sql: JSON_STRING(value) table: jsonString(value) description: | - Serializes a value into JSON. + 将值序列化为 JSON。 - This function returns a JSON string containing the serialized value. If the value is `NULL`, - the function returns `NULL`. + 此函数返回一个包含序列化值的 JSON 字符串。如果值为 `NULL`,函数返回 `NULL`。 ``` -- NULL @@ -786,23 +784,18 @@ json: - sql: JSON_VALUE(jsonValue, path [RETURNING ] [ { NULL | ERROR | DEFAULT } ON EMPTY ] [ { NULL | ERROR | DEFAULT } ON ERROR ]) table: STRING.jsonValue(STRING path [, returnType, onEmpty, defaultOnEmpty, onError, defaultOnError]) description: | - Extracts a scalar from a JSON string. + 从 JSON 字符串中提取标量。 - This method searches a JSON string for a given path expression and returns the value if the - value at that path is scalar. Non-scalar values cannot be returned. By default, the value is - returned as `STRING`. Using `returningType` a different type can be chosen, with the following - types being supported: + 此方法搜索给定路径表达式的 JSON 字符串,如果该路径上的值是标量则返回该值。不能返回非标量值。默认情况下返回值类型为 `STRING`。 + 可以使用 `returnsType` 设置不同的类型,如下所示: * `VARCHAR` / `STRING` * `BOOLEAN` * `INTEGER` * `DOUBLE` - For empty path expressions or errors a behavior can be defined to either return `null`, raise - an error or return a defined default value instead. When omitted, the default is - `NULL ON EMPTY` or `NULL ON ERROR`, respectively. The default value may be a literal or an - expression. If the default value itself raises an error, it falls through to the error - behavior for `ON EMPTY`, and raises an error for `ON ERROR`. + 对于空路径表达式或错误,可以将行为定义为返回 `null`、引发错误或返回定义的默认值。当省略时,默认为 `NULL ON EMPTY` 或 + `NULL ON ERROR`。默认值可以是文字或表达式。如果默认值本身引发错误,通过错误就会造成 `ON EMPTY` 的行为,并引发 `ON ERROR` 的错误。 ``` -- "true" @@ -820,22 +813,20 @@ json: DEFAULT FALSE ON ERROR) -- 0.998D - JSON_VALUE('{"a.b": [0.998,0.996]}','$.["a.b"][0]' + JSON_VALUE('{"a.b": [0.998,0.996]}','$.["a.b"][0]' RETURNING DOUBLE) ``` - sql: JSON_QUERY(jsonValue, path [ { WITHOUT | WITH CONDITIONAL | WITH UNCONDITIONAL } [ ARRAY ] WRAPPER ] [ { NULL | EMPTY ARRAY | EMPTY OBJECT | ERROR } ON EMPTY ] [ { NULL | EMPTY ARRAY | EMPTY OBJECT | ERROR } ON ERROR ]) table: STRING.jsonQuery(path [, JsonQueryWrapper [, JsonQueryOnEmptyOrError, JsonQueryOnEmptyOrError ] ]) description: | - Extracts JSON values from a JSON string. + 从 JSON 字符串中提取 JSON 值。 - The result is always returned as a `STRING`. The `RETURNING` clause is currently not supported. + 结果总是以 `STRING` 的形式返回。目前尚不支持 `RETURNING` 子句。 - The `wrappingBehavior` determines whether the extracted value should be wrapped into an array, - and whether to do so unconditionally or only if the value itself isn't an array already. + `wrappingBehavior` 决定是否将提取的值包装到一个数组中,以及是否无条件地这样做,还是只有当值本身不是数组时才这样做。 - `onEmpty` and `onError` determine the behavior in case the path expression is empty, or in - case an error was raised, respectively. By default, in both cases `null` is returned. Other - choices are to use an empty array, an empty object, or to raise an error. + `onEmpty` 和 `onError` 分别决定路径表达式为空或引发错误时的行为。默认情况下,这两种情况都会返回 `null`。其他选择是 + 使用空数组、空对象或引发错误。 ``` -- '{ "b": 1 }' @@ -848,7 +839,7 @@ json: JSON_QUERY('{"a":[{"c":"c1"},{"c":"c2"}]}', 'lax $.a[*].c') - -- Wrap result into an array + -- 将结果包装到数组中 -- '[{}]' JSON_QUERY('{}', '$' WITH CONDITIONAL ARRAY WRAPPER) -- '[1, 2]' @@ -856,13 +847,13 @@ json: -- '[[1, 2]]' JSON_QUERY('[1, 2]', '$' WITH UNCONDITIONAL ARRAY WRAPPER) - -- Scalars must be wrapped to be returned + -- 必须包装标量才能返回 -- NULL JSON_QUERY(1, '$') -- '[1]' JSON_QUERY(1, '$' WITH CONDITIONAL ARRAY WRAPPER) - -- Behavior if path expression is empty / there is an error + -- 路径表达式为空/存在错误时的行为 -- '{}' JSON_QUERY('{}', 'lax $.invalid' EMPTY OBJECT ON EMPTY) -- '[]' @@ -871,16 +862,13 @@ json: - sql: JSON_OBJECT([[KEY] key VALUE value]* [ { NULL | ABSENT } ON NULL ]) table: jsonObject(JsonOnNull, keyValues...) description: | - Builds a JSON object string from a list of key-value pairs. + 将键值对列表构建为 JSON 对象字符串。 - Note that keys must be non-`NULL` string literals, while values may be arbitrary expressions. + 注意,键必须是非 `NULL` 字符串自变量,而值可以是任意表达式。 - This function returns a JSON string. The `ON NULL` behavior defines how to treat `NULL` - values. If omitted, `NULL ON NULL` is assumed by default. + 这个函数返回一个 JSON 字符串。`ON NULL` 行为定义了如何处理 `NULL` 值。如果省略,则默认为 `NULL ON NULL`。 - Values which are created from another JSON construction function call (`JSON_OBJECT`, - `JSON_ARRAY`) are inserted directly rather than as a string. This allows building nested JSON - structures. + 值是由另一个 JSON 构造函数调用 (`JSON_OBJECT`,`JSON_ARRAY`) 直接插入所创建,而不是作为一个字符串。它允许构建嵌套的 JSON 结构。 ``` -- '{}' @@ -889,7 +877,7 @@ json: -- '{"K1":"V1","K2":"V2"}' JSON_OBJECT('K1' VALUE 'V1', 'K2' VALUE 'V2') - -- Expressions as values + -- 表达式作为值 JSON_OBJECT('orderNo' VALUE orders.orderId) -- ON NULL @@ -908,15 +896,14 @@ json: - sql: JSON_OBJECTAGG([KEY] key VALUE value [ { NULL | ABSENT } ON NULL ]) table: jsonObjectAgg(JsonOnNull, keyExpression, valueExpression) description: | - Builds a JSON object string by aggregating key-value expressions into a single JSON object. + 通过将 key-value 聚合到单个 JSON 对象中,构建 JSON 对象字符串。 - The key expression must return a non-nullable character string. Value expressions can be - arbitrary, including other JSON functions. If a value is `NULL`, the `ON NULL` behavior - defines what to do. If omitted, `NULL ON NULL` is assumed by default. + 键表达式必须返回不为空的字符串。值表达式可以是任意的,包括其他 JSON 函数。 + 如果值为 `NULL`,则 `ON NULL` 行为定义了要执行的操作。如果省略,默认情况下假定为 `NULL ON NULL`。 - Note that keys must be unique. If a key occurs multiple times, an error will be thrown. + 请注意,键必须是唯一的。如果一个键出现多次,将抛出一个错误。 - This function is currently not supported in `OVER` windows. + 目前在 `OVER` windows 中不支持此函数。 ``` -- '{"Apple":2,"Banana":17,"Orange":0}' @@ -927,15 +914,11 @@ json: - sql: JSON_ARRAY([value]* [ { NULL | ABSENT } ON NULL ]) table: jsonArray(JsonOnNull, values...) description: | - Builds a JSON array string from a list of values. + 将数值列表构建为 JSON 数组字符串。 - This function returns a JSON string. The values can be arbitrary expressions. The `ON NULL` - behavior defines how to treat `NULL` values. If omitted, `ABSENT ON NULL` is assumed by - default. + 这个函数返回一个 JSON 字符串,值可以是任意表达式。`ON NULL` 行为定义了如何处理 `NULL` 值。如果省略,则假定 `ABSENT ON NULL` 为默认值。 - Elements which are created from another JSON construction function call (`JSON_OBJECT`, - `JSON_ARRAY`) are inserted directly rather than as a string. This allows building nested JSON - structures. + 元素是由另一个 JSON 构造函数调用 (`JSON_OBJECT`,`JSON_ARRAY`) 直接插入所创建,而不是作为一个字符串。它允许构建嵌套的 JSON 结构。 ``` -- '[]' @@ -943,7 +926,7 @@ json: -- '[1,"2"]' JSON_ARRAY(1, '2') - -- Expressions as values + -- 表达式作为值 JSON_ARRAY(orders.orderId) -- ON NULL @@ -956,13 +939,11 @@ json: - sql: JSON_ARRAYAGG(items [ { NULL | ABSENT } ON NULL ]) table: jsonArrayAgg(JsonOnNull, itemExpression) description: | - Builds a JSON object string by aggregating items into an array. + 通过将字段聚合到数组中构建 JSON 对象字符串。 - Item expressions can be arbitrary, including other JSON functions. If a value is `NULL`, the - `ON NULL` behavior defines what to do. If omitted, `ABSENT ON NULL` is assumed by default. + 项目表达式可以是任意的,包括其他 JSON 函数。如果值为 `NULL`,则 `ON NULL` 行为定义了要执行的操作。如果省略,默认情况下假定为 `ABSENT ON NULL`。 - This function is currently not supported in `OVER` windows, unbounded session windows, or hop - windows. + 此函数目前不支持 `OVER` windows、未绑定的 session windows 或 hop windows。 ``` -- '["Apple","Banana","Orange"]' From 29243e02eb7d3edef229d5da067af48108392b47 Mon Sep 17 00:00:00 2001 From: chenzihao Date: Fri, 18 Mar 2022 16:43:17 +0800 Subject: [PATCH 112/258] [FLINK-26634][docs-zh] Update Chinese version of Elasticsearch connector docs This closes #19101. --- .../connectors/datastream/elasticsearch.md | 53 +++---------------- 1 file changed, 8 insertions(+), 45 deletions(-) diff --git a/docs/content.zh/docs/connectors/datastream/elasticsearch.md b/docs/content.zh/docs/connectors/datastream/elasticsearch.md index 28d2f8fcdb7da..5688306bab392 100644 --- a/docs/content.zh/docs/connectors/datastream/elasticsearch.md +++ b/docs/content.zh/docs/connectors/datastream/elasticsearch.md @@ -210,72 +210,35 @@ def createIndexRequest(element: (String)): IndexRequest = { ### Elasticsearch Sinks 和容错 -默认情况下,Flink Elasticsearch Sink 不会提供任何传递健壮性的保障。 -用户可以选择启用 Elasticsearch sink 的 at-least-once 语义。 - -通过启用 Flink checkpoint,Flink Elasticsearch Sink 可以保证至少一次将操作请求发送到 Elasticsearch 集群。 +通过启用 Flink checkpoint,Flink Elasticsearch Sink 保证至少一次将操作请求发送到 Elasticsearch 集群。 这是通过在进行 checkpoint 时等待 `BulkProcessor` 中所有挂起的操作请求来实现。 这有效地保证了在触发 checkpoint 之前所有的请求被 Elasticsearch 成功确认,然后继续处理发送到 sink 的记录。 关于 checkpoint 和容错的更多详细信息,请参见[容错文档]({{< ref "docs/learn-flink/fault_tolerance" >}})。 -要使用具有容错特性的 Elasticsearch Sinks,需要配置启用 at-least-once 分发并且在执行环境中启用作业拓扑的 checkpoint: +要使用具有容错特性的 Elasticsearch Sinks,需要在执行环境中启用作业拓扑的 checkpoint: {{< tabs "d00d1e93-4844-40d7-b0ec-9ec37e73145e" >}} {{< tab "Java" >}} -Elasticsearch 6: -```java -final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); -env.enableCheckpointing(5000); // 每 5000 毫秒执行一次 checkpoint - -Elasticsearch6SinkBuilder sinkBuilder = new Elasticsearch6SinkBuilder() - .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) - .setHosts(new HttpHost("127.0.0.1", 9200, "http")) - .setEmitter( - (element, context, indexer) -> - indexer.add(createIndexRequest(element))); -``` - -Elasticsearch 7: ```java final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(5000); // 每 5000 毫秒执行一次 checkpoint - -Elasticsearch7SinkBuilder sinkBuilder = new Elasticsearch7SinkBuilder() - .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) - .setHosts(new HttpHost("127.0.0.1", 9200, "http")) - .setEmitter( - (element, context, indexer) -> - indexer.add(createIndexRequest(element))); ``` {{< /tab >}} {{< tab "Scala" >}} -Elasticsearch 6: ```scala val env = StreamExecutionEnvironment.getExecutionEnvironment() env.enableCheckpointing(5000) // 每 5000 毫秒执行一次 checkpoint - -val sinkBuilder = new Elasticsearch6SinkBuilder[String] - .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) - .setHosts(new HttpHost("127.0.0.1", 9200, "http")) - .setEmitter((element: String, context: SinkWriter.Context, indexer: RequestIndexer) => - indexer.add(createIndexRequest(element))) -``` - -Elasticsearch 7: -```scala -val env = StreamExecutionEnvironment.getExecutionEnvironment() -env.enableCheckpointing(5000) // 每 5000 毫秒执行一次 checkpoint - -val sinkBuilder = new Elasticsearch7SinkBuilder[String] - .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) - .setHosts(new HttpHost("127.0.0.1", 9200, "http")) - .setEmitter((element: String, context: SinkWriter.Context, indexer: RequestIndexer) => - indexer.add(createIndexRequest(element))) ``` {{< /tab >}} {{< /tabs >}} +

+重要提示: 默认情况下不启用 checkpoint, 但默认传输保证 AT_LEAST_ONCE 语义。 +这会导致 sink 缓冲请求,直到它结束或 BulkProcessor 自动刷新。 +默认情况下,BulkProcessor 将在 1000 个添加操作后刷新。要将 Processor 配置为更频繁地刷新,请参阅 BulkProcessor 配置部分。 +

+ ### 处理失败的 Elasticsearch 请求 Elasticsearch 操作请求可能由于多种原因而失败,包括节点队列容量暂时已满或者要被索引的文档格式错误。 From 5a5490a49866d02ab5c4761f59ddf94c06f42b41 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 17 Mar 2022 22:07:58 +0100 Subject: [PATCH 113/258] [refactor][streaming] Migrate Source(Operator)StreamTaskTest to JUnit5 and assertj --- .../tasks/SourceOperatorStreamTaskTest.java | 61 ++++++------ .../runtime/tasks/SourceStreamTaskTest.java | 93 +++++++++---------- .../tasks/SourceStreamTaskTestBase.java | 23 ++--- 3 files changed, 85 insertions(+), 92 deletions(-) diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java index 3d9087f55d2b4..0ea212a386d5e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java @@ -59,7 +59,8 @@ import org.apache.flink.streaming.runtime.tasks.LifeCycleMonitor.LifeCyclePhase; import org.apache.flink.util.SerializedValue; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.io.Serializable; @@ -75,29 +76,24 @@ import java.util.stream.IntStream; import static org.apache.flink.streaming.util.TestHarnessUtil.assertOutputEquals; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.lessThanOrEqualTo; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** * Tests for verifying that the {@link SourceOperator} as a task input can be integrated well with * {@link org.apache.flink.streaming.runtime.io.StreamOneInputProcessor}. */ -public class SourceOperatorStreamTaskTest extends SourceStreamTaskTestBase { +class SourceOperatorStreamTaskTest extends SourceStreamTaskTestBase { + private static final OperatorID OPERATOR_ID = new OperatorID(); private static final int NUM_RECORDS = 10; @Test - public void testMetrics() throws Exception { + void testMetrics() throws Exception { testMetrics( SourceOperatorStreamTask::new, new SourceOperatorFactory<>( new MockSource(Boundedness.BOUNDED, 1), WatermarkStrategy.noWatermarks()), - lessThanOrEqualTo(1_000_000d)); + busyTime -> busyTime.isLessThanOrEqualTo(1_000_000d)); } /** @@ -105,7 +101,7 @@ public void testMetrics() throws Exception { * operators. */ @Test - public void testSnapshotAndRestore() throws Exception { + void testSnapshotAndRestore() throws Exception { // process NUM_RECORDS records and take a snapshot. TaskStateSnapshot taskStateSnapshot = executeAndWaitForCheckpoint(1, null, IntStream.range(0, NUM_RECORDS)); @@ -116,7 +112,7 @@ public void testSnapshotAndRestore() throws Exception { } @Test - public void testSnapshotAndAdvanceToEndOfEventTime() throws Exception { + void testSnapshotAndAdvanceToEndOfEventTime() throws Exception { final int checkpointId = 1; try (StreamTaskMailboxTestHarness testHarness = createTestHarness(checkpointId, null)) { @@ -139,7 +135,7 @@ public void testSnapshotAndAdvanceToEndOfEventTime() throws Exception { } @Test - public void testEmittingMaxWatermarkAfterReadingAllRecords() throws Exception { + void testEmittingMaxWatermarkAfterReadingAllRecords() throws Exception { try (StreamTaskMailboxTestHarness testHarness = createTestHarness()) { testHarness.processAll(); testHarness.finishProcessing(); @@ -147,22 +143,22 @@ public void testEmittingMaxWatermarkAfterReadingAllRecords() throws Exception { Queue expectedOutput = new LinkedList<>(); expectedOutput.add(Watermark.MAX_WATERMARK); expectedOutput.add(new EndOfData(StopMode.DRAIN)); - assertThat(testHarness.getOutput().toArray(), equalTo(expectedOutput.toArray())); + assertThat(testHarness.getOutput().toArray()).isEqualTo(expectedOutput.toArray()); } } @Test - public void testNotEmittingMaxWatermarkAfterCancelling() throws Exception { + void testNotEmittingMaxWatermarkAfterCancelling() throws Exception { try (StreamTaskMailboxTestHarness testHarness = createTestHarness()) { testHarness.getStreamTask().cancel(); testHarness.finishProcessing(); - assertThat(testHarness.getOutput(), hasSize(0)); + assertThat(testHarness.getOutput()).hasSize(0); } } @Test - public void testExternallyInducedSource() throws Exception { + void testExternallyInducedSource() throws Exception { final int numEventsBeforeCheckpoint = 10; final int totalNumEvents = 20; TestingExternallyInducedSourceReader testingReader = @@ -176,17 +172,18 @@ public void testExternallyInducedSource() throws Exception { testHarness.processAll(); - assertEquals(totalNumEvents, runtimeTestingReader.numEmittedEvents); - assertTrue(runtimeTestingReader.checkpointed); - assertEquals( - TestingExternallyInducedSourceReader.CHECKPOINT_ID, - runtimeTestingReader.checkpointedId); - assertEquals(numEventsBeforeCheckpoint, runtimeTestingReader.checkpointedAt); + assertThat(runtimeTestingReader.numEmittedEvents).isEqualTo(totalNumEvents); + assertThat(runtimeTestingReader.checkpointed).isTrue(); + assertThat(runtimeTestingReader.checkpointedId) + .isEqualTo(TestingExternallyInducedSourceReader.CHECKPOINT_ID); + assertThat(runtimeTestingReader.checkpointedAt).isEqualTo(numEventsBeforeCheckpoint); + Assertions.assertThat(testHarness.getOutput()) + .contains(new CheckpointBarrier(2, 2, checkpointOptions)); } } @Test - public void testSkipExecutionIfFinishedOnRestore() throws Exception { + void testSkipExecutionIfFinishedOnRestore() throws Exception { TaskStateSnapshot taskStateSnapshot = TaskStateSnapshot.FINISHED_ON_RESTORE; LifeCycleMonitorSource testingSource = @@ -215,7 +212,8 @@ public void notifyEndOfData(StopMode mode) throws IOException { testHarness.getStreamTask().invoke(); testHarness.processAll(); - assertThat(output, contains(Watermark.MAX_WATERMARK, new EndOfData(StopMode.DRAIN))); + assertThat(output) + .containsExactly(Watermark.MAX_WATERMARK, new EndOfData(StopMode.DRAIN)); LifeCycleMonitorSourceReader sourceReader = (LifeCycleMonitorSourceReader) @@ -226,7 +224,7 @@ public void notifyEndOfData(StopMode mode) throws IOException { } @Test - public void testTriggeringStopWithSavepointWithDrain() throws Exception { + void testTriggeringStopWithSavepointWithDrain() throws Exception { SourceOperatorFactory sourceOperatorFactory = new SourceOperatorFactory<>( new MockSource(Boundedness.CONTINUOUS_UNBOUNDED, 2), @@ -271,9 +269,9 @@ public void acknowledgeCheckpoint( testHarness.waitForTaskCompletion(); testHarness.finishProcessing(); - assertTrue(triggerResult.isDone()); - assertTrue(triggerResult.get()); - assertTrue(checkpointCompleted.isDone()); + assertThat(triggerResult.isDone()).isTrue(); + assertThat(triggerResult.get()).isTrue(); + assertThat(checkpointCompleted.isDone()).isTrue(); } } @@ -304,7 +302,8 @@ private TaskStateSnapshot executeAndWaitForCheckpoint( expectedOutput.add( new CheckpointBarrier(checkpointId, checkpointId, checkpointOptions)); - assertEquals(checkpointId, testHarness.taskStateManager.getReportedCheckpointId()); + assertThat(testHarness.taskStateManager.getReportedCheckpointId()) + .isEqualTo(checkpointId); assertOutputEquals("Output was not correct.", expectedOutput, testHarness.getOutput()); return testHarness.taskStateManager.getLastJobManagerTaskStateSnapshot(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java index aa4b364cbc113..d16c5abc2a476 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTest.java @@ -70,8 +70,7 @@ import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.function.CheckedSupplier; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.io.Serializable; @@ -94,23 +93,18 @@ import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO; import static org.apache.flink.streaming.runtime.tasks.StreamTaskFinalCheckpointsTest.triggerCheckpoint; import static org.apache.flink.util.Preconditions.checkState; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; /** * These tests verify that the RichFunction methods are called (in correct order). And that * checkpointing/element emission don't occur concurrently. */ -public class SourceStreamTaskTest extends SourceStreamTaskTestBase { +class SourceStreamTaskTest extends SourceStreamTaskTestBase { /** This test verifies that open() and close() are correctly called by the StreamTask. */ @Test - public void testOpenClose() throws Exception { + void testOpenClose() throws Exception { final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>(SourceStreamTask::new, STRING_TYPE_INFO); @@ -124,15 +118,17 @@ public void testOpenClose() throws Exception { testHarness.invoke(); testHarness.waitForTaskCompletion(); - assertTrue("RichFunction methods where not called.", OpenCloseTestSource.closeCalled); + assertThat(OpenCloseTestSource.closeCalled) + .as("RichFunction methods where not called.") + .isTrue(); List resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); - Assert.assertEquals(10, resultElements.size()); + assertThat(resultElements.size()).isEqualTo(10); } @Test - public void testMetrics() throws Exception { + void testMetrics() throws Exception { testMetrics( SourceStreamTask::new, SimpleOperatorFactory.of( @@ -140,7 +136,7 @@ public void testMetrics() throws Exception { new CancelTestSource( INT_TYPE_INFO.createSerializer(new ExecutionConfig()), 42))), - is(Double.NaN)); + busyTime -> busyTime.isNaN()); } /** @@ -157,7 +153,7 @@ public void testMetrics() throws Exception { */ @Test @SuppressWarnings("unchecked") - public void testCheckpointing() throws Exception { + void testCheckpointing() throws Exception { final int numElements = 100; final int numCheckpoints = 100; final int numCheckpointers = 1; @@ -214,14 +210,14 @@ public void testCheckpointing() throws Exception { List> resultElements = TestHarnessUtil.getRawElementsFromOutput(testHarness.getOutput()); - Assert.assertEquals(numElements, resultElements.size()); + assertThat(resultElements.size()).isEqualTo(numElements); } finally { executor.shutdown(); } } @Test - public void testClosingAllOperatorsOnChainProperly() throws Exception { + void testClosingAllOperatorsOnChainProperly() throws Exception { final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>(SourceStreamTask::new, STRING_TYPE_INFO); @@ -254,11 +250,11 @@ public void testClosingAllOperatorsOnChainProperly() throws Exception { new StreamRecord<>("[Operator1]: Finish")); final Object[] output = testHarness.getOutput().toArray(); - assertArrayEquals("Output was not correct.", expected.toArray(), output); + assertThat(output).as("Output was not correct.").isEqualTo(expected.toArray()); } @Test - public void testNotMarkingEndOfInputWhenTaskCancelled() throws Exception { + void testNotMarkingEndOfInputWhenTaskCancelled() throws Exception { final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>(SourceStreamTask::new, STRING_TYPE_INFO); @@ -299,22 +295,22 @@ public void testNotMarkingEndOfInputWhenTaskCancelled() throws Exception { } @Test - public void testCancellationWithSourceBlockedOnLock() throws Exception { + void testCancellationWithSourceBlockedOnLock() throws Exception { testCancellationWithSourceBlockedOnLock(false, false); } @Test - public void testCancellationWithSourceBlockedOnLockWithPendingMail() throws Exception { + void testCancellationWithSourceBlockedOnLockWithPendingMail() throws Exception { testCancellationWithSourceBlockedOnLock(true, false); } @Test - public void testCancellationWithSourceBlockedOnLockAndThrowingOnError() throws Exception { + void testCancellationWithSourceBlockedOnLockAndThrowingOnError() throws Exception { testCancellationWithSourceBlockedOnLock(false, true); } @Test - public void testCancellationWithSourceBlockedOnLockWithPendingMailAndThrowingOnError() + void testCancellationWithSourceBlockedOnLockWithPendingMailAndThrowingOnError() throws Exception { testCancellationWithSourceBlockedOnLock(true, true); } @@ -324,8 +320,8 @@ public void testCancellationWithSourceBlockedOnLockWithPendingMailAndThrowingOnE * StreamTask} which, as of the time this test is being written, is not tested anywhere else * (like {@link StreamTaskTest} or {@link OneInputStreamTaskTest}). */ - public void testCancellationWithSourceBlockedOnLock( - boolean withPendingMail, boolean throwInCancel) throws Exception { + void testCancellationWithSourceBlockedOnLock(boolean withPendingMail, boolean throwInCancel) + throws Exception { final StreamTaskTestHarness testHarness = new StreamTaskTestHarness<>(SourceStreamTask::new, STRING_TYPE_INFO); @@ -354,9 +350,9 @@ public void testCancellationWithSourceBlockedOnLock( .createExecutor(0) .execute( () -> - assertFalse( - "This should never execute before task cancelation", - testHarness.getTask().isRunning()), + assertThat(testHarness.getTask().isRunning()) + .as("This should never execute before task cancelation") + .isFalse(), "Test"); } @@ -427,12 +423,12 @@ public void cancel() { } @Test - public void testInterruptionExceptionNotSwallowed() throws Exception { + void testInterruptionExceptionNotSwallowed() throws Exception { testInterruptionExceptionNotSwallowed(InterruptedException::new); } @Test - public void testWrappedInterruptionExceptionNotSwallowed() throws Exception { + void testWrappedInterruptionExceptionNotSwallowed() throws Exception { testInterruptionExceptionNotSwallowed( () -> new RuntimeException(new FlinkRuntimeException(new InterruptedException()))); } @@ -491,7 +487,7 @@ public void cancel() {} } @Test - public void testWaitsForSourceThreadOnCancel() throws Exception { + void testWaitsForSourceThreadOnCancel() throws Exception { StreamTaskTestHarness harness = new StreamTaskTestHarness<>(SourceStreamTask::new, STRING_TYPE_INFO); @@ -504,13 +500,13 @@ public void testWaitsForSourceThreadOnCancel() throws Exception { // SourceStreamTask should be still waiting for NonStoppingSource after cancellation harness.getTask().cancel(); harness.waitForTaskCompletion(50, true); // allow task to exit prematurely - assertTrue(harness.taskThread.isAlive()); + assertThat(harness.taskThread.isAlive()).isTrue(); // SourceStreamTask should be still waiting for NonStoppingSource after interruptions for (int i = 0; i < 10; i++) { harness.getTask().maybeInterruptOnCancel(harness.getTaskThread(), null, null); harness.waitForTaskCompletion(50, true); // allow task to exit prematurely - assertTrue(harness.taskThread.isAlive()); + assertThat(harness.taskThread.isAlive()).isTrue(); } // It should only exit once NonStoppingSource allows for it @@ -519,7 +515,7 @@ public void testWaitsForSourceThreadOnCancel() throws Exception { } @Test - public void testTriggeringCheckpointAfterSourceThreadFinished() throws Exception { + void testTriggeringCheckpointAfterSourceThreadFinished() throws Exception { ResultPartition[] partitionWriters = new ResultPartition[2]; try (NettyShuffleEnvironment env = new NettyShuffleEnvironmentBuilder() @@ -585,12 +581,12 @@ public void acknowledgeCheckpoint( (id, error) -> testHarness.getStreamTask().notifyCheckpointCompleteAsync(2)); testHarness.finishProcessing(); - assertTrue(checkpointFuture.isDone()); + assertThat(checkpointFuture.isDone()).isTrue(); // Each result partition should have emitted 1 barrier, 1 max watermark and 1 // EndOfUserRecordEvent. for (ResultPartition resultPartition : partitionWriters) { - assertEquals(3, resultPartition.getNumberOfQueuedBuffers()); + assertThat(resultPartition.getNumberOfQueuedBuffers()).isEqualTo(3); } } } finally { @@ -603,7 +599,7 @@ public void acknowledgeCheckpoint( } @Test - public void testClosedOnRestoreSourceSkipExecution() throws Exception { + void testClosedOnRestoreSourceSkipExecution() throws Exception { LifeCycleMonitorSource testSource = new LifeCycleMonitorSource(); List output = new ArrayList<>(); try (StreamTaskMailboxTestHarness harness = @@ -626,7 +622,8 @@ public void notifyEndOfData(StopMode mode) throws IOException { harness.processAll(); harness.streamTask.getCompletionFuture().get(); - assertThat(output, contains(Watermark.MAX_WATERMARK, new EndOfData(StopMode.DRAIN))); + assertThat(output) + .containsExactly(Watermark.MAX_WATERMARK, new EndOfData(StopMode.DRAIN)); LifeCycleMonitorSource source = (LifeCycleMonitorSource) @@ -642,7 +639,7 @@ public void notifyEndOfData(StopMode mode) throws IOException { } @Test - public void testTriggeringStopWithSavepointWithDrain() throws Exception { + void testTriggeringStopWithSavepointWithDrain() throws Exception { SourceFunction testSource = new EmptySource(); CompletableFuture checkpointCompleted = new CompletableFuture<>(); @@ -684,9 +681,9 @@ public void acknowledgeCheckpoint( harness.streamTask.runMailboxLoop(); harness.finishProcessing(); - assertTrue(triggerResult.isDone()); - assertTrue(triggerResult.get()); - assertTrue(checkpointCompleted.isDone()); + assertThat(triggerResult.isDone()).isTrue(); + assertThat(triggerResult.get()).isTrue(); + assertThat(checkpointCompleted.isDone()).isTrue(); } } @@ -767,7 +764,7 @@ public void cancel() { public List snapshotState(long checkpointId, long timestamp) throws Exception { if (!semaphore.tryAcquire()) { - Assert.fail("Concurrent invocation of snapshotState."); + fail("Concurrent invocation of snapshotState."); } int startCount = count; lastCheckpointId = checkpointId; @@ -780,7 +777,7 @@ public List snapshotState(long checkpointId, long timestamp) if (startCount != count) { semaphore.release(); // This means that next() was invoked while the snapshot was ongoing - Assert.fail("Count is different at start end end of snapshot."); + fail("Count is different at start end end of snapshot."); } semaphore.release(); return Collections.singletonList(sum); @@ -871,7 +868,7 @@ private static class OpenCloseTestSource extends RichSourceFunction { public void open(Configuration parameters) throws Exception { super.open(parameters); if (closeCalled) { - Assert.fail("Close called before open."); + fail("Close called before open."); } openCalled = true; } @@ -880,7 +877,7 @@ public void open(Configuration parameters) throws Exception { public void close() throws Exception { super.close(); if (!openCalled) { - Assert.fail("Open was not called before close."); + fail("Open was not called before close."); } closeCalled = true; } @@ -888,7 +885,7 @@ public void close() throws Exception { @Override public void run(SourceContext ctx) throws Exception { if (!openCalled) { - Assert.fail("Open was not called before run."); + fail("Open was not called before run."); } for (int i = 0; i < 10; i++) { ctx.collect("Hello" + i); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTestBase.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTestBase.java index 30656413d551f..b477796704a76 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTestBase.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTaskTestBase.java @@ -32,16 +32,15 @@ import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; -import org.hamcrest.Matcher; +import org.assertj.core.api.AbstractDoubleAssert; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Future; +import java.util.function.Consumer; import static org.apache.flink.api.common.typeinfo.BasicTypeInfo.INT_TYPE_INFO; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertThat; +import static org.assertj.core.api.Assertions.assertThat; /** Common base class for testing source tasks. */ public class SourceStreamTaskTestBase { @@ -49,7 +48,7 @@ public void testMetrics( FunctionWithException, Exception> taskFactory, StreamOperatorFactory operatorFactory, - Matcher busyTimeMatcher) + Consumer> busyTimeMatcher) throws Exception { long sleepTime = 42; @@ -73,26 +72,24 @@ public void testMetrics( OneShotLatch checkpointAcknowledgeLatch = new OneShotLatch(); harness.getCheckpointResponder().setAcknowledgeLatch(checkpointAcknowledgeLatch); - assertFalse(triggerFuture.isDone()); + assertThat(triggerFuture).isNotDone(); Thread.sleep(sleepTime); while (!triggerFuture.isDone()) { harness.streamTask.runMailboxStep(); } Gauge checkpointStartDelayGauge = (Gauge) metrics.get(MetricNames.CHECKPOINT_START_DELAY_TIME); - assertThat( - checkpointStartDelayGauge.getValue(), - greaterThanOrEqualTo(sleepTime * 1_000_000)); + assertThat(checkpointStartDelayGauge.getValue()) + .isGreaterThanOrEqualTo(sleepTime * 1_000_000); Gauge busyTimeGauge = (Gauge) metrics.get(MetricNames.TASK_BUSY_TIME); - assertThat(busyTimeGauge.getValue(), busyTimeMatcher); + busyTimeMatcher.accept(assertThat(busyTimeGauge.getValue())); checkpointAcknowledgeLatch.await(); TestCheckpointResponder.AcknowledgeReport acknowledgeReport = Iterables.getOnlyElement( harness.getCheckpointResponder().getAcknowledgeReports()); - assertThat( - acknowledgeReport.getCheckpointMetrics().getCheckpointStartDelayNanos(), - greaterThanOrEqualTo(sleepTime * 1_000_000)); + assertThat(acknowledgeReport.getCheckpointMetrics().getCheckpointStartDelayNanos()) + .isGreaterThanOrEqualTo(sleepTime * 1_000_000); } } } From d5d859efb98434081b6ac45dfa2e939b667d86f9 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Thu, 17 Mar 2022 23:08:46 +0100 Subject: [PATCH 114/258] [FLINK-25256][streaming] Externally induced sources replay barriers received over RPC instead of inventing them out of thin air. This change preserves the CheckpointOptions and properly integrates user-triggered snapshots and workflows with more than one source. The externally induced source now merely delays the barrier instead of being able to insert one at a whim which would never work in aforementioned setups. --- ...treamTaskExternallyInducedSourceInput.java | 24 +++ .../tasks/SourceOperatorStreamTask.java | 187 +++++++++++++++--- .../tasks/SourceOperatorStreamTaskTest.java | 79 +++++++- 3 files changed, 254 insertions(+), 36 deletions(-) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskExternallyInducedSourceInput.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskExternallyInducedSourceInput.java index fff008cfa69d8..ca0462eec19d9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskExternallyInducedSourceInput.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/StreamTaskExternallyInducedSourceInput.java @@ -21,12 +21,14 @@ import org.apache.flink.api.connector.source.ExternallyInducedSourceReader; import org.apache.flink.streaming.api.operators.SourceOperator; +import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; /** A subclass of {@link StreamTaskSourceInput} for {@link ExternallyInducedSourceReader}. */ public class StreamTaskExternallyInducedSourceInput extends StreamTaskSourceInput { private final Consumer checkpointTriggeringHook; private final ExternallyInducedSourceReader sourceReader; + private CompletableFuture blockFuture; @SuppressWarnings("unchecked") public StreamTaskExternallyInducedSourceInput( @@ -39,12 +41,34 @@ public StreamTaskExternallyInducedSourceInput( this.sourceReader = (ExternallyInducedSourceReader) operator.getSourceReader(); } + public void blockUntil(CompletableFuture blockFuture) { + this.blockFuture = blockFuture; + // assume that the future is completed in mailbox thread + blockFuture.whenComplete((v, e) -> unblock()); + } + + private void unblock() { + this.blockFuture = null; + } + @Override public DataInputStatus emitNext(DataOutput output) throws Exception { + if (blockFuture != null) { + return DataInputStatus.NOTHING_AVAILABLE; + } + DataInputStatus status = super.emitNext(output); if (status == DataInputStatus.NOTHING_AVAILABLE) { sourceReader.shouldTriggerCheckpoint().ifPresent(checkpointTriggeringHook); } return status; } + + @Override + public CompletableFuture getAvailableFuture() { + if (blockFuture != null) { + return blockFuture; + } + return super.getAvailableFuture(); + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java index efc594b6940db..3d2805599fbb6 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTask.java @@ -24,14 +24,12 @@ import org.apache.flink.metrics.Counter; import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.CheckpointType; import org.apache.flink.runtime.checkpoint.SavepointType; import org.apache.flink.runtime.checkpoint.SnapshotType; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.io.network.api.StopMode; import org.apache.flink.runtime.metrics.MetricNames; import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup; -import org.apache.flink.runtime.state.CheckpointStorageLocationReference; import org.apache.flink.streaming.api.operators.Output; import org.apache.flink.streaming.api.operators.SourceOperator; import org.apache.flink.streaming.api.watermark.Watermark; @@ -48,7 +46,12 @@ import javax.annotation.Nullable; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -57,7 +60,25 @@ public class SourceOperatorStreamTask extends StreamTask> { private AsyncDataOutputToOutput output; - private boolean isExternallyInducedSource; + /** + * Contains information about all checkpoints where RPC from checkpoint coordinator arrives + * before the source reader triggers it. (Common case) + */ + private SortedMap untriggeredCheckpoints = new TreeMap<>(); + /** + * Contains the checkpoints that are triggered by the source but the RPC from checkpoint + * coordinator has yet to arrive. This may happen if the barrier is inserted as an event into + * the data plane by the source coordinator and the (distributed) source reader reads that event + * before receiving Flink's checkpoint RPC. (Rare case) + */ + private SortedSet triggeredCheckpoints = new TreeSet<>(); + /** + * Blocks input until the RPC call has been received that corresponds to the triggered + * checkpoint. This future must only be accessed and completed in the mailbox thread. + */ + private CompletableFuture waitForRPC = FutureUtils.completedVoidFuture(); + /** Only set for externally induced sources. See also {@link #isExternallyInducedSource()}. */ + private StreamTaskExternallyInducedSourceInput externallyInducedSourceInput; public SourceOperatorStreamTask(Environment env) throws Exception { super(env); @@ -79,14 +100,14 @@ public void init() throws Exception { if (operatorChain.isTaskDeployedAsFinished()) { input = new StreamTaskFinishedOnRestoreSourceInput<>(sourceOperator, 0, 0); } else if (sourceReader instanceof ExternallyInducedSourceReader) { - isExternallyInducedSource = true; - - input = + externallyInducedSourceInput = new StreamTaskExternallyInducedSourceInput<>( sourceOperator, this::triggerCheckpointForExternallyInducedSource, 0, 0); + + input = externallyInducedSourceInput; } else { input = new StreamTaskSourceInput<>(sourceOperator, 0, 0); } @@ -112,20 +133,53 @@ public void init() throws Exception { @Override public CompletableFuture triggerCheckpointAsync( CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) { - if (!isExternallyInducedSource) { - if (isSynchronous(checkpointOptions.getCheckpointType())) { - return triggerStopWithSavepointAsync(checkpointMetaData, checkpointOptions); - } else { - return super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions); - } - } else if (checkpointOptions.getCheckpointType().equals(CheckpointType.FULL_CHECKPOINT)) { - // see FLINK-25256 - throw new IllegalStateException( - "Using externally induced sources, we can not enforce taking a full checkpoint." - + "If you are restoring from a snapshot in NO_CLAIM mode, please use" - + " either CLAIM or LEGACY mode."); + if (!isExternallyInducedSource()) { + return triggerCheckpointNowAsync(checkpointMetaData, checkpointOptions); + } + CompletableFuture triggerFuture = new CompletableFuture<>(); + // immediately move RPC to mailbox so we don't need to synchronize fields + mainMailboxExecutor.execute( + () -> + triggerCheckpointOnExternallyInducedSource( + checkpointMetaData, checkpointOptions, triggerFuture), + "SourceOperatorStreamTask#triggerCheckpointAsync(%s, %s)", + checkpointMetaData, + checkpointOptions); + return triggerFuture; + } + + private boolean isExternallyInducedSource() { + return externallyInducedSourceInput != null; + } + + private void triggerCheckpointOnExternallyInducedSource( + CheckpointMetaData checkpointMetaData, + CheckpointOptions checkpointOptions, + CompletableFuture triggerFuture) { + assert (mailboxProcessor.isMailboxThread()); + if (!triggeredCheckpoints.remove(checkpointMetaData.getCheckpointId())) { + // common case: RPC is received before source reader triggers checkpoint + // store metadata and options for later + untriggeredCheckpoints.put( + checkpointMetaData.getCheckpointId(), + new UntriggeredCheckpoint(checkpointMetaData, checkpointOptions)); + triggerFuture.complete(isRunning()); + } else { + // trigger already received (rare case) + FutureUtils.forward( + triggerCheckpointNowAsync(checkpointMetaData, checkpointOptions), + triggerFuture); + + cleanupOldCheckpoints(checkpointMetaData.getCheckpointId()); + } + } + + private CompletableFuture triggerCheckpointNowAsync( + CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) { + if (isSynchronous(checkpointOptions.getCheckpointType())) { + return triggerStopWithSavepointAsync(checkpointMetaData, checkpointOptions); } else { - return CompletableFuture.completedFuture(isRunning()); + return super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions); } } @@ -159,22 +213,76 @@ protected void advanceToEndOfEventTime() { output.emitWatermark(Watermark.MAX_WATERMARK); } + @Override + protected void declineCheckpoint(long checkpointId) { + cleanupCheckpoint(checkpointId); + super.declineCheckpoint(checkpointId); + } + + @Override + public Future notifyCheckpointAbortAsync( + long checkpointId, long latestCompletedCheckpointId) { + mainMailboxExecutor.execute( + () -> cleanupCheckpoint(checkpointId), "Cleanup checkpoint %d", checkpointId); + return super.notifyCheckpointAbortAsync(checkpointId, latestCompletedCheckpointId); + } + + @Override + public Future notifyCheckpointSubsumedAsync(long checkpointId) { + mainMailboxExecutor.execute( + () -> cleanupCheckpoint(checkpointId), "Cleanup checkpoint %d", checkpointId); + return super.notifyCheckpointSubsumedAsync(checkpointId); + } + // -------------------------- private void triggerCheckpointForExternallyInducedSource(long checkpointId) { - final CheckpointOptions checkpointOptions = - CheckpointOptions.forConfig( - CheckpointType.CHECKPOINT, - CheckpointStorageLocationReference.getDefault(), - configuration.isExactlyOnceCheckpointMode(), - configuration.isUnalignedCheckpointsEnabled(), - configuration.getAlignedCheckpointTimeout().toMillis()); - final long timestamp = System.currentTimeMillis(); + UntriggeredCheckpoint untriggeredCheckpoint = untriggeredCheckpoints.remove(checkpointId); + if (untriggeredCheckpoint != null) { + // common case: RPC before external sources induces it + triggerCheckpointNowAsync( + untriggeredCheckpoint.getMetadata(), + untriggeredCheckpoint.getCheckpointOptions()); + cleanupOldCheckpoints(checkpointId); + } else { + // rare case: external source induced first + triggeredCheckpoints.add(checkpointId); + if (waitForRPC.isDone()) { + waitForRPC = new CompletableFuture<>(); + externallyInducedSourceInput.blockUntil(waitForRPC); + } + } + } + + /** + * Cleanup any orphaned checkpoint before the given currently triggered checkpoint. These + * checkpoint may occur when the checkpoint is cancelled but the RPC is lost. Note, to be safe, + * checkpoint X is only removed when both RPC and trigger for a checkpoint Y>X is received. + */ + private void cleanupOldCheckpoints(long checkpointId) { + assert (mailboxProcessor.isMailboxThread()); + triggeredCheckpoints.headSet(checkpointId).clear(); + untriggeredCheckpoints.headMap(checkpointId).clear(); + + maybeResumeProcessing(); + } + + /** Resumes processing if it was blocked before or else is a no-op. */ + private void maybeResumeProcessing() { + assert (mailboxProcessor.isMailboxThread()); - final CheckpointMetaData checkpointMetaData = - new CheckpointMetaData(checkpointId, timestamp, timestamp); + if (triggeredCheckpoints.isEmpty()) { + waitForRPC.complete(null); + } + } + + /** Remove temporary data about a canceled checkpoint. */ + private void cleanupCheckpoint(long checkpointId) { + assert (mailboxProcessor.isMailboxThread()); + triggeredCheckpoints.remove(checkpointId); + untriggeredCheckpoints.remove(checkpointId); - super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions); + maybeResumeProcessing(); } // --------------------------- @@ -225,4 +333,23 @@ public void emitWatermarkStatus(WatermarkStatus watermarkStatus) throws Exceptio output.emitWatermarkStatus(watermarkStatus); } } + + private static class UntriggeredCheckpoint { + private final CheckpointMetaData metadata; + private final CheckpointOptions checkpointOptions; + + private UntriggeredCheckpoint( + CheckpointMetaData metadata, CheckpointOptions checkpointOptions) { + this.metadata = metadata; + this.checkpointOptions = checkpointOptions; + } + + public CheckpointMetaData getMetadata() { + return metadata; + } + + public CheckpointOptions getCheckpointOptions() { + return checkpointOptions; + } + } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java index 0ea212a386d5e..d3b9f64de26d9 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/SourceOperatorStreamTaskTest.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointMetaData; import org.apache.flink.runtime.checkpoint.CheckpointMetrics; import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.CheckpointType; import org.apache.flink.runtime.checkpoint.SavepointType; import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; import org.apache.flink.runtime.executiongraph.ExecutionAttemptID; @@ -61,6 +62,8 @@ import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import java.io.IOException; import java.io.Serializable; @@ -74,6 +77,7 @@ import java.util.concurrent.Future; import java.util.function.Supplier; import java.util.stream.IntStream; +import java.util.stream.Stream; import static org.apache.flink.streaming.util.TestHarnessUtil.assertOutputEquals; import static org.assertj.core.api.Assertions.assertThat; @@ -86,6 +90,10 @@ class SourceOperatorStreamTaskTest extends SourceStreamTaskTestBase { private static final OperatorID OPERATOR_ID = new OperatorID(); private static final int NUM_RECORDS = 10; + public static final CheckpointStorageLocationReference SAVEPOINT_LOCATION = + new CheckpointStorageLocationReference("Savepoint".getBytes()); + public static final CheckpointStorageLocationReference CHECKPOINT_LOCATION = + new CheckpointStorageLocationReference("Checkpoint".getBytes()); @Test void testMetrics() throws Exception { @@ -157,8 +165,35 @@ void testNotEmittingMaxWatermarkAfterCancelling() throws Exception { } } - @Test - void testExternallyInducedSource() throws Exception { + static Stream provideExternallyInducedParameters() { + return Stream.of( + CheckpointOptions.alignedNoTimeout( + SavepointType.savepoint(SavepointFormatType.CANONICAL), + SAVEPOINT_LOCATION), + CheckpointOptions.alignedNoTimeout( + SavepointType.terminate(SavepointFormatType.CANONICAL), + SAVEPOINT_LOCATION), + CheckpointOptions.alignedNoTimeout( + SavepointType.suspend(SavepointFormatType.CANONICAL), + SAVEPOINT_LOCATION), + CheckpointOptions.alignedNoTimeout( + CheckpointType.CHECKPOINT, CHECKPOINT_LOCATION), + CheckpointOptions.alignedWithTimeout( + CheckpointType.CHECKPOINT, CHECKPOINT_LOCATION, 123L), + CheckpointOptions.unaligned(CheckpointType.CHECKPOINT, CHECKPOINT_LOCATION), + CheckpointOptions.notExactlyOnce( + CheckpointType.CHECKPOINT, CHECKPOINT_LOCATION)) + .flatMap( + options -> + Stream.of( + new Object[] {options, true}, + new Object[] {options, false})); + } + + @ParameterizedTest + @MethodSource("provideExternallyInducedParameters") + void testExternallyInducedSource(CheckpointOptions checkpointOptions, boolean rpcFirst) + throws Exception { final int numEventsBeforeCheckpoint = 10; final int totalNumEvents = 20; TestingExternallyInducedSourceReader testingReader = @@ -170,15 +205,47 @@ void testExternallyInducedSource() throws Exception { ((SourceOperator) testHarness.getStreamTask().mainOperator) .getSourceReader(); - testHarness.processAll(); + CheckpointMetaData checkpointMetaData = + new CheckpointMetaData(TestingExternallyInducedSourceReader.CHECKPOINT_ID, 2); + if (rpcFirst) { + testHarness.streamTask.triggerCheckpointAsync( + checkpointMetaData, checkpointOptions); + testHarness.processAll(); + } else { + do { + testHarness.processSingleStep(); + } while (!runtimeTestingReader.shouldTriggerCheckpoint().isPresent()); + // stream task should block when trigger received but no RPC + assertThat(testHarness.streamTask.inputProcessor.isAvailable()).isFalse(); + CompletableFuture triggerCheckpointAsync = + testHarness.streamTask.triggerCheckpointAsync( + checkpointMetaData, checkpointOptions); + // process mails until checkpoint has been processed + while (!triggerCheckpointAsync.isDone()) { + testHarness.processSingleStep(); + } + // stream task should be unblocked now + assertThat(testHarness.streamTask.inputProcessor.isAvailable()).isTrue(); + testHarness.processAll(); + } - assertThat(runtimeTestingReader.numEmittedEvents).isEqualTo(totalNumEvents); + int expectedEvents = + checkpointOptions.getCheckpointType().isSavepoint() + && ((SavepointType) checkpointOptions.getCheckpointType()) + .isSynchronous() + ? numEventsBeforeCheckpoint + : totalNumEvents; + assertThat(runtimeTestingReader.numEmittedEvents).isEqualTo(expectedEvents); assertThat(runtimeTestingReader.checkpointed).isTrue(); assertThat(runtimeTestingReader.checkpointedId) .isEqualTo(TestingExternallyInducedSourceReader.CHECKPOINT_ID); assertThat(runtimeTestingReader.checkpointedAt).isEqualTo(numEventsBeforeCheckpoint); Assertions.assertThat(testHarness.getOutput()) - .contains(new CheckpointBarrier(2, 2, checkpointOptions)); + .contains( + new CheckpointBarrier( + checkpointMetaData.getCheckpointId(), + checkpointMetaData.getTimestamp(), + checkpointOptions)); } } @@ -262,7 +329,7 @@ public void acknowledgeCheckpoint( new CheckpointMetaData(2, 2), CheckpointOptions.alignedNoTimeout( SavepointType.terminate(SavepointFormatType.CANONICAL), - CheckpointStorageLocationReference.getDefault())); + SAVEPOINT_LOCATION)); checkpointCompleted.whenComplete( (ignored, exception) -> testHarness.streamTask.notifyCheckpointCompleteAsync(2)); From 42e978548f44e98fafbf5f62175e48acba8642e0 Mon Sep 17 00:00:00 2001 From: Arvid Heise Date: Fri, 18 Mar 2022 16:14:44 +0100 Subject: [PATCH 115/258] [FLINK-25256][streaming] Clarify the contract of ExternallyInducedSource(Reader). --- .../source/ExternallyInducedSourceReader.java | 12 ++++++------ .../api/checkpoint/ExternallyInducedSource.java | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/connector/source/ExternallyInducedSourceReader.java b/flink-core/src/main/java/org/apache/flink/api/connector/source/ExternallyInducedSourceReader.java index c88bc0f1a157a..f26eb843bc684 100644 --- a/flink-core/src/main/java/org/apache/flink/api/connector/source/ExternallyInducedSourceReader.java +++ b/flink-core/src/main/java/org/apache/flink/api/connector/source/ExternallyInducedSourceReader.java @@ -24,16 +24,16 @@ import java.util.Optional; /** - * Sources that implement this interface do not trigger checkpoints when receiving a trigger message - * from the checkpoint coordinator, but when their input data/events indicate that a checkpoint + * Sources that implement this interface delay checkpoints when receiving a trigger message from the + * checkpoint coordinator to the point when their input data/events indicate that a checkpoint * should be triggered. * *

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

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

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

Important: It is crucial that all parallel source tasks trigger their checkpoints at * roughly the same time. Otherwise this leads to performance issues due to long checkpoint diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ExternallyInducedSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ExternallyInducedSource.java index eafd3074d1554..537a56f63df24 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ExternallyInducedSource.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ExternallyInducedSource.java @@ -23,8 +23,8 @@ import org.apache.flink.util.FlinkException; /** - * Sources that implement this interface do not trigger checkpoints when receiving a trigger message - * from the checkpoint coordinator, but when their input data/events indicate that a checkpoint + * Sources that implement this interface delay checkpoints when receiving a trigger message from the + * checkpoint coordinator to the point when their input data/events indicate that a checkpoint * should be triggered. * *

Since sources cannot simply create a new checkpoint on their own, this mechanism always goes From 417f5263125f69232241201804db748a6afbfb51 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Mon, 14 Feb 2022 12:56:41 +0100 Subject: [PATCH 116/258] [FLINK-26134][docs] Added documentation page with Checkpoint/Savepoint guarantees --- .../content.zh/docs/deployment/memory/network_mem_tuning.md | 6 +++++- docs/content/docs/deployment/memory/network_mem_tuning.md | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/deployment/memory/network_mem_tuning.md b/docs/content.zh/docs/deployment/memory/network_mem_tuning.md index 7d80135752296..cccba3bff4be8 100644 --- a/docs/content.zh/docs/deployment/memory/network_mem_tuning.md +++ b/docs/content.zh/docs/deployment/memory/network_mem_tuning.md @@ -126,7 +126,11 @@ Flink 有多个本地缓冲区池 —— 每个输出和输入流对应一个。 在往下游 subtask 发送数据部分时,缓冲区通过汇集 record 来优化网络开销。下游 subtask 应该在接收到完整的 record 后才开始处理它。 -如果缓冲区太小(比如小于一条 record),会因为开销比较大而导致吞吐低。 +If the buffer size is too small, or the buffers are flushed too frequently (`execution.buffer-timeout` configuration parameter), this can lead to decreased throughput +since the per-buffer overhead are significantly higher then per-record overheads in the Flink's runtime. + +As a rule of thumb, we don't recommend thinking about increasing the buffer size, or the buffer timeout unless you can observe a network bottleneck in your real life workload +(downstream operator idling, upstream backpressured, output buffer queue is full, downstream input queue is empty). 如果缓冲区太大,会导致: - 内存使用高 diff --git a/docs/content/docs/deployment/memory/network_mem_tuning.md b/docs/content/docs/deployment/memory/network_mem_tuning.md index 4bedad5bfb9c1..9c537517b8905 100644 --- a/docs/content/docs/deployment/memory/network_mem_tuning.md +++ b/docs/content/docs/deployment/memory/network_mem_tuning.md @@ -128,7 +128,11 @@ The default settings for exclusive buffers and floating buffers should be suffic The buffer collects records in order to optimize network overhead when sending the data portion to the next subtask. The next subtask should receive all parts of the record before consuming it. -If the buffer size is too small (i.e. less than one record), this can lead to low throughput since the overhead is still pretty large. +If the buffer size is too small, or the buffers are flushed too frequently (`execution.buffer-timeout` configuration parameter), this can lead to decreased throughput +since the per-buffer overhead are significantly higher then per-record overheads in the Flink's runtime. + +As a rule of thumb, we don't recommend thinking about increasing the buffer size, or the buffer timeout unless you can observe a network bottleneck in your real life workload +(downstream operator idling, upstream backpressured, output buffer queue is full, downstream input queue is empty). If the buffer size is too large, this can lead to: - high memory usage From c4ed56a870f3fc58ac0d2af66dcd35eb5bf5b7d3 Mon Sep 17 00:00:00 2001 From: Ruanshubin Date: Mon, 14 Mar 2022 17:33:13 +0800 Subject: [PATCH 117/258] [FLINK-26629][runtime] fix bug in code comment of SubtaskStateMapper.RANGE --- .../flink/runtime/io/network/api/writer/SubtaskStateMapper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapper.java b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapper.java index a2fc87b371904..c9136cebc549b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapper.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/io/network/api/writer/SubtaskStateMapper.java @@ -89,7 +89,7 @@ public boolean isAmbiguous() { *

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

For all downscale from n to [n-1 .. n/2], each new subtasks get exactly two old subtasks From d630eaae12e743a3666f42c2e7f6c6ce9caa0d66 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Wed, 23 Feb 2022 11:16:47 +0800 Subject: [PATCH 118/258] [FLINK-26028][Connector/pulsar] add sink documentation; change some pulsar source documentation. --- .../docs/connectors/datastream/pulsar.md | 474 +++++++++++--- .../docs/connectors/datastream/pulsar.md | 586 ++++++++++++++---- .../pulsar_producer_configuration.html | 90 +++ .../generated/pulsar_sink_configuration.html | 48 ++ .../ConfigOptionsDocGenerator.java | 3 + 5 files changed, 1007 insertions(+), 194 deletions(-) create mode 100644 docs/layouts/shortcodes/generated/pulsar_producer_configuration.html create mode 100644 docs/layouts/shortcodes/generated/pulsar_sink_configuration.html diff --git a/docs/content.zh/docs/connectors/datastream/pulsar.md b/docs/content.zh/docs/connectors/datastream/pulsar.md index c14958a878866..3ba5989c6cfec 100644 --- a/docs/content.zh/docs/connectors/datastream/pulsar.md +++ b/docs/content.zh/docs/connectors/datastream/pulsar.md @@ -24,12 +24,11 @@ under the License. # Apache Pulsar 连接器 -Flink 当前只提供 [Apache Pulsar](https://pulsar.apache.org) 数据源,用户可以使用它从 Pulsar 读取数据,并保证每条数据只被处理一次。 +Flink 当前提供 [Apache Pulsar](https://pulsar.apache.org) Source 和 Sink 连接器,用户可以使用它从 Pulsar 读取数据,并保证每条数据只被处理一次。 ## 添加依赖 -连接器当前支持 Pulsar 2.7.0 之后的版本,但是连接器使用到了 Pulsar 的[事务机制](https://pulsar.apache.org/docs/en/txn-what/),建议在 Pulsar 2.8.0 -及其之后的版本上使用连接器进行数据读取。 +Pulsar Source 当前支持 Pulsar 2.8.1 之后的版本,但是 Pulsar Source 使用到了 Pulsar 的[事务机制](https://pulsar.apache.org/docs/zh-CN/txn-what/),建议在 Pulsar 2.9.2 及其之后的版本上使用 Pulsar Source 进行数据读取。 如果想要了解更多关于 Pulsar API 兼容性设计,可以阅读文档 [PIP-72](https://github.com/apache/pulsar/wiki/PIP-72%3A-Introduce-Pulsar-Interface-Taxonomy%3A-Audience-and-Stability-Classification)。 @@ -37,18 +36,15 @@ Flink 当前只提供 [Apache Pulsar](https://pulsar.apache.org) 数据源,用 Flink 的流连接器并不会放到发行文件里面一同发布,阅读[此文档]({{< ref "docs/dev/configuration/overview" >}}),了解如何将连接器添加到集群实例内。 -## Pulsar 数据源 +## Pulsar Source {{< hint info >}} -Pulsar 数据源基于 Flink 最新的[批流一体 API]({{< ref "docs/dev/datastream/sources.md" >}}) 进行开发。 - -如果要想使用基于旧版的 `SourceFunction` 实现的 Pulsar 数据源,或者是项目的 Flink 版本低于 1.14,可以使用 StreamNative 单独维护的 [pulsar-flink](https://github.com/streamnative/pulsar-flink)。 +Pulsar Source 基于 Flink 最新的[批流一体 API]({{< ref "docs/dev/datastream/sources.md" >}}) 进行开发。 {{< /hint >}} ### 使用示例 -Pulsar 数据源提供了 builder 类来构造数据源实例。下面的代码实例使用 builder 类创建的数据源会从 topic "persistent://public/default/my-topic" 的数据开始端进行消费。 -连接器使用了 **Exclusive**(独占)的订阅方式消费消息,订阅名称为 `my-subscription`,并把消息体的二进制字节流以 UTF-8 的方式编码为字符串。 +Pulsar Source 提供了 builder 类来构造 `PulsarSource` 实例。下面的代码实例使用 builder 类创建的实例会从 “persistent://public/default/my-topic” 的数据开始端进行消费。对应的 Pulsar Source 使用了 **Exclusive**(独占)的订阅方式消费消息,订阅名称为 `my-subscription`,并把消息体的二进制字节流以 UTF-8 的方式编码为字符串。 ```java PulsarSource pulsarSource = PulsarSource.builder() @@ -64,17 +60,17 @@ PulsarSource pulsarSource = PulsarSource.builder() env.fromSource(source, WatermarkStrategy.noWatermarks(), "Pulsar Source"); ``` -如果使用构造类构造 Pulsar 数据源,一定要提供下面几个属性: +如果使用构造类构造 `PulsarSource`,一定要提供下面几个属性: -- Pulsar 数据消费的地址,使用 `setServiceUrl(String)` 方法提供 -- Pulsar HTTP 管理地址,使用 `setAdminUrl(String)` 方法提供 -- Pulsar 订阅名称,使用 `setSubscriptionName(String)` 方法提供 -- 需要消费的 topic 或者是 topic 下面的分区,详见[指定消费的 Topic 或者 Topic 分区](#指定消费的-topic-或者-topic-分区) -- 解码 Pulsar 消息的反序列化器,详见[反序列化器](#反序列化器) +- Pulsar 数据消费的地址,使用 `setServiceUrl(String)` 方法提供。 +- Pulsar HTTP 管理地址,使用 `setAdminUrl(String)` 方法提供。 +- Pulsar 订阅名称,使用 `setSubscriptionName(String)` 方法提供。 +- 需要消费的 Topic 或者是 Topic 下面的分区,详见[指定消费的 Topic 或者 Topic 分区](#指定消费的-topic-或者-topic-分区)。 +- 解码 Pulsar 消息的反序列化器,详见[反序列化器](#反序列化器)。 ### 指定消费的 Topic 或者 Topic 分区 -Pulsar 数据源提供了两种订阅 topic 或 topic 分区的方式。 +Pulsar Source 提供了两种订阅 Topic 或 Topic 分区的方式。 - Topic 列表,从这个 Topic 的所有分区上消费消息,例如: ```java @@ -84,66 +80,62 @@ Pulsar 数据源提供了两种订阅 topic 或 topic 分区的方式。 PulsarSource.builder().setTopics("topic-a-partition-0", "topic-a-partition-2"); ``` -- Topic 正则,连接器使用给定的正则表达式匹配出所有合规的 topic,例如: +- Topic 正则,Pulsar Source 使用给定的正则表达式匹配出所有合规的 Topic,例如: ```java PulsarSource.builder().setTopicPattern("topic-*"); ``` #### Topic 名称简写 -从 Pulsar 2.0 之后,完整的 topic 名称格式为 `{persistent|non-persistent}://租户/命名空间/topic`。 -但是连接器不需要提供 topic 名称的完整定义,因为 topic 类型、租户、命名空间都设置了默认值。 +从 Pulsar 2.0 之后,完整的 Topic 名称格式为 `{persistent|non-persistent}://租户/命名空间/topic`。但是 Pulsar Source 不需要提供 Topic 名称的完整定义,因为 Topic 类型、租户、命名空间都设置了默认值。 -Topic 属性 | 默认值 -:------------|:------- -topic 类型 | `persistent` -租户 | `public` -命名空间 | `default` +| Topic 属性 | 默认值 | +|:---------|:-------------| +| Topic 类型 | `persistent` | +| 租户 | `public` | +| 命名空间 | `default` | -下面的表格提供了当前 Pulsar 支持的简写方式: +下面的表格提供了当前 Pulsar Topic 支持的简写方式: -topic 名称简写 | 翻译后的 topic 名称 -:----------------|:--------------------- -`my-topic` | `persistent://public/default/my-topic` -`my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` +| Topic 名称简写 | 翻译后的 Topic 名称 | +|:----------------------------------|:-----------------------------------------------| +| `my-topic` | `persistent://public/default/my-topic` | +| `my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` | {{< hint warning >}} -对于 non-persistent(非持久化) topic,连接器不支持简写名称。所以无法将 `non-persistent://public/default/my-topic` 简写成 `non-persistent://my-topic`。 +对于 Non-persistent(非持久化)Topic,Pulsar Source 不支持简写名称。所以无法将 `non-persistent://public/default/my-topic` 简写成 `non-persistent://my-topic`。 {{< /hint >}} #### Pulsar Topic 层次结构 对于 Pulsar 而言,Topic 分区也是一种 Topic。Pulsar 会将一个有分区的 Topic 在内部按照分区的大小拆分成等量的无分区 Topic。 -例如,在 Pulsar 的 `sample` 租户下面的 `flink` 命名空间里面创建了一个有 3 个分区的 topic,给它起名为 `simple-string`。 -可以在 Pulsar 上看到如下的 topic 列表: +由于 Pulsar 内部的分区实际实现为一个 Topic,我们将用“分区”来指代“仅有一个分区的 Topic(Non-partitioned Topic)”和“具有多个分区的 Topic 下属的分区”。 + +例如,在 Pulsar 的 `sample` 租户下面的 `flink` 命名空间里面创建了一个有 3 个分区的 Topic,给它起名为 `simple-string`。可以在 Pulsar 上看到如下的 Topic 列表: -Topic 名称 | 是否分区 -:--------- | :---------- -`persistent://sample/flink/simple-string` | 是 -`persistent://sample/flink/simple-string-partition-0` | 否 -`persistent://sample/flink/simple-string-partition-1` | 否 -`persistent://sample/flink/simple-string-partition-2` | 否 +| Topic 名称 | 是否分区 | +|:------------------------------------------------------|:-----| +| `persistent://sample/flink/simple-string` | 是 | +| `persistent://sample/flink/simple-string-partition-0` | 否 | +| `persistent://sample/flink/simple-string-partition-1` | 否 | +| `persistent://sample/flink/simple-string-partition-2` | 否 | -这意味着,用户可以用上面的子 topic 去直接消费分区里面的数据,不需要再去基于上层的父 topic 去消费全部分区的数据。 -例如:使用 `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` 将会只消费 topic `sample/flink/simple-string` 上面的分区 1 和 2 里面的消息。 +这意味着,用户可以用上面的子 Topic 去直接消费分区里面的数据,不需要再去基于上层的父 Topic 去消费全部分区的数据。例如:使用 `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` 将会只消费 Topic `sample/flink/simple-string` 分区 1 和 2 里面的消息。 #### 配置 Topic 正则表达式 -前面提到了 Pulsar topic 有 `persistent`、`non-persistent` 两种类型,使用正则表达式消费数据的时候,连接器会尝试从正则表达式里面解析出消息的类型。 -例如:`PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` 会解析出 `non-persistent` 这个 topic 类型。 -如果用户使用 topic 名称简写的方式,连接器会使用默认的消息类型 `persistent`。 +前面提到了 Pulsar Topic 有 `persistent`、`non-persistent` 两种类型,使用正则表达式消费数据的时候,Pulsar Source 会尝试从正则表达式里面解析出消息的类型。例如:`PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` 会解析出 `non-persistent` 这个 Topic 类型。如果用户使用 Topic 名称简写的方式,Pulsar Source 会使用默认的消息类型 `persistent`。 -如果想用正则去消费 `persistent` 和 `non-persistent` 类型的 topic,需要使用 `RegexSubscriptionMode` 定义 topic 类型,例如:`setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)`。 +如果想用正则去消费 `persistent` 和 `non-persistent` 类型的 Topic,需要使用 `RegexSubscriptionMode` 定义 Topic 类型,例如:`setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)`。 ### 反序列化器 -反序列化器用于解析 Pulsar 消息,连接器使用 `PulsarDeserializationSchema` 来定义反序列化器。 -用户可以在 builder 类中使用 `setDeserializationSchema(PulsarDeserializationSchema)` 方法配置反序列化器,它会解析 Pulsar 的 `Message` 实例。 +反序列化器用于解析 Pulsar 消息,Pulsar Source 使用 `PulsarDeserializationSchema` 来定义反序列化器。用户可以在 builder 类中使用 `setDeserializationSchema(PulsarDeserializationSchema)` 方法配置反序列化器。 -如果用户只关心消息体的二进制字节流,并不需要其他属性来解析数据。可以直接使用预定义的 `PulsarDeserializationSchema`。Pulsar 连接器里面提供了 3 种预定义好的反序列化器。 +如果用户只关心消息体的二进制字节流,并不需要其他属性来解析数据。可以直接使用预定义的 `PulsarDeserializationSchema`。Pulsar Source里面提供了 3 种预定义的反序列化器。 -- 使用 Pulsar 的 [Schema](https://pulsar.apache.org/docs/en/schema-understand/) 解析消息。 +- 使用 Pulsar 的 [Schema](https://pulsar.apache.org/docs/zh-CN/schema-understand/) 解析消息。 ```java // 基础数据类型 PulsarDeserializationSchema.pulsarSchema(Schema); @@ -163,25 +155,22 @@ Topic 名称 | 是否分区 PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig); ``` -Pulsar 的 `Message` 包含了很多 [额外的属性](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#%E6%B6%88%E6%81%AF)。 -例如,消息的 key,消息发送时间,消息生产时间,用户在消息上自定义的键值对属性等。可以使用 `Message` 接口来获取这些属性。 +Pulsar 的 `Message` 包含了很多 [额外的属性](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#%E6%B6%88%E6%81%AF)。例如,消息的 key、消息发送时间、消息生产时间、用户在消息上自定义的键值对属性等。可以使用 `Message` 接口来获取这些属性。 -如果用户需要基于这些额外的属性来解析一条消息,可以实现 `PulsarDeserializationSchema` 接口。 -并一定要确保 `PulsarDeserializationSchema.getProducedType()` 方法返回的 `TypeInformation` 是正确的结果。 -Flink 使用 `TypeInformation` 将解析出来的结果序列化传递到下游算子。 +如果用户需要基于这些额外的属性来解析一条消息,可以实现 `PulsarDeserializationSchema` 接口。并一定要确保 `PulsarDeserializationSchema.getProducedType()` 方法返回的 `TypeInformation` 是正确的结果。Flink 使用 `TypeInformation` 将解析出来的结果序列化传递到下游算子。 ### Pulsar 订阅 -订阅是命名好的配置规则,指导消息如何投递给消费者。连接器需要提供一个独立的订阅名称,支持 Pulsar 的四种订阅模式: +订阅是命名好的配置规则,指导消息如何投递给消费者。Pulsar Source 需要提供一个独立的订阅名称,支持 Pulsar 的四种订阅模式: - [exclusive(独占)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#exclusive) - [shared(共享)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#shared%E5%85%B1%E4%BA%AB) - [failover(灾备)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#failover%E7%81%BE%E5%A4%87) - [key_shared(key 共享)](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#key_shared) -当前 Pulsar 连接器里面,`独占` 和 `灾备` 的实现没有区别,如果 Flink 的一个 reader 挂了,连接器会把所有未消费的数据交给其他的 reader 来消费数据。 +当前 Pulsar Source 里,`独占` 和 `灾备` 的实现没有区别,如果 Flink 的一个 reader 挂了,Pulsar Source 会把所有未消费的数据交给其他的 reader 来消费数据。 -默认情况下,如果没有指定订阅类型,连接器使用共享订阅类型(`SubscriptionType.Shared`)。 +默认情况下,如果没有指定订阅类型,Pulsar Source 使用共享订阅类型(`SubscriptionType.Shared`)。 ```java // 名为 "my-shared" 的共享订阅 @@ -191,19 +180,19 @@ PulsarSource.builder().setSubscriptionName("my-shared"); PulsarSource.builder().setSubscriptionName("my-exclusive").setSubscriptionType(SubscriptionType.Exclusive); ``` -如果想在 Pulsar 连接器里面使用 `key 共享` 订阅,需要提供 `RangeGenerator` 实例。`RangeGenerator` 会生成一组消息 key 的 hash 范围,连接器会基于给定的范围来消费数据。 +如果想在 Pulsar Source 里面使用 `key 共享` 订阅,需要提供 `RangeGenerator` 实例。`RangeGenerator` 会生成一组消息 key 的 hash 范围,Pulsar Source 会基于给定的范围来消费数据。 -Pulsar 连接器也提供了一个名为 `UniformRangeGenerator` 的默认实现,它会基于 flink 数据源的并行度将 hash 范围均分。 +Pulsar Source 也提供了一个名为 `UniformRangeGenerator` 的默认实现,它会基于 flink 数据源的并行度将 hash 范围均分。 ### 起始消费位置 -连接器使用 `setStartCursor(StartCursor)` 方法给定开始消费的位置。内置的消费位置有: +Pulsar Source 使用 `setStartCursor(StartCursor)` 方法给定开始消费的位置。内置的开始消费位置有: -- 从 topic 里面最早的一条消息开始消费。 +- 从 Topic 里面最早的一条消息开始消费。 ```java StartCursor.earliest(); ``` -- 从 topic 里面最新的一条消息开始消费。 +- 从 Topic 里面最新的一条消息开始消费。 ```java StartCursor.latest(); ``` @@ -222,23 +211,23 @@ Pulsar 连接器也提供了一个名为 `UniformRangeGenerator` 的默认实现 {{< hint info >}} 每条消息都有一个固定的序列号,这个序列号在 Pulsar 上有序排列,其包含了 ledger、entry、partition 等原始信息,用于在 Pulsar 底层存储上查找到具体的消息。 + Pulsar 称这个序列号为 `MessageId`,用户可以使用 `DefaultImplementation.newMessageId(long ledgerId, long entryId, int partitionIndex)` 创建它。 {{< /hint >}} ### 边界 -Pulsar 连接器同时支持流式和批的消费方式,默认情况下,连接器使用流的方式消费数据。除非任务失败或者被取消,否则连接器将持续消费数据。 -用户可以使用 `setBoundedStopCursor(StopCursor)` 给定停止消费的位置,这种情况下连接器会使用批的方式进行消费。当所有 topic 分区都消费到了停止位置,Flink 任务就会结束。 +Pulsar Source 默认情况下使用流的方式消费数据。除非任务失败或者被取消,否则将持续消费数据。用户可以使用 `setBoundedStopCursor(StopCursor)` 给定停止消费的位置,这种情况下会使用批的方式进行消费。使用流的方式一样可以给定停止位置,使用 `setUnboundedStopCursor(StopCursor)` 方法即可。 -使用流的方式一样可以给定停止位置,使用 `setUnboundedStopCursor(StopCursor)` 方法即可。 +在批模式下,使用 `setBoundedStopCursor(StopCursor)` 来指定一个消费停止位置。 -内置的停止位置如下: +内置的停止消费位置如下: - 永不停止。 ```java StopCursor.never(); ``` -- 停止于 Pulsar 启动时 topic 里面最新的那条数据。 +- 停止于 Pulsar 启动时 Topic 里面最新的那条数据。 ```java StopCursor.latest(); ``` @@ -255,31 +244,29 @@ Pulsar 连接器同时支持流式和批的消费方式,默认情况下,连 StopCursor.atEventTime(long); ``` -### 其他配置项 +### Source 配置项 -除了前面提到的配置选项,连接器还提供了丰富的选项供 Pulsar 专家使用,在 builder 类里通过 `setConfig(ConfigOption, T)` 和 `setConfig(Configuration)` 方法给定下述的全部配置。 +除了前面提到的配置选项,Pulsar Source 还提供了丰富的选项供 Pulsar 专家使用,在 builder 类里通过 `setConfig(ConfigOption, T)` 和 `setConfig(Configuration)` 方法给定下述的全部配置。 #### Pulsar Java 客户端配置项 -Pulsar 连接器使用[Java 客户端](https://pulsar.apache.org/docs/en/client-libraries-java/)来创建消费实例,相关的配置定义于 Pulsar 的 `ClientConfigurationData` 内。连接器在 `PulsarOptions` 选项中,定义大部分的可供用户定义的配置。 +Pulsar Source 使用 [Java 客户端](https://pulsar.apache.org/docs/zh-CN/client-libraries-java/)来创建消费实例,相关的配置定义于 Pulsar 的 `ClientConfigurationData` 内。在 `PulsarOptions` 选项中,定义大部分的可供用户定义的配置。 {{< generated/pulsar_client_configuration >}} #### Pulsar 管理 API 配置项 -[管理 API](https://pulsar.apache.org/docs/en/admin-api-overview/) 用于查询 topic 的元数据和用正则订阅的时候的 topic 查找,它与 -Java 客户端共享大部分配置。下面列举的配置只供管理 API 使用,连接器也在 `PulsarOptions` 里予以定义。 +[管理 API](https://pulsar.apache.org/docs/zh-CN/admin-api-overview/) 用于查询 Topic 的元数据和用正则订阅的时候的 Topic 查找,它与 Java 客户端共享大部分配置。下面列举的配置只供管理 API 使用,`PulsarOptions` 包含了这些配置 。 {{< generated/pulsar_admin_configuration >}} #### Pulsar 消费者 API 配置项 -Pulsar 提供了消费者 API 和读者 API 两套 API 来进行数据消费,它们可用于不同的业务场景。 -Flink 上的 Pulsar 连接器使用消费者 API 进行消费,它的配置定义于 Pulsar 的 `ConsumerConfigurationData` 内。连接器将其中大部分的可供用户定义的配置定义于 `PulsarSourceOptions` 内。 +Pulsar 提供了消费者 API 和读者 API 两套 API 来进行数据消费,它们可用于不同的业务场景。Flink 上的 Pulsar Source 使用消费者 API 进行消费,它的配置定义于 Pulsar 的 `ConsumerConfigurationData` 内。Pulsar Source 将其中大部分的可供用户定义的配置定义于 `PulsarSourceOptions` 内。 {{< generated/pulsar_consumer_configuration >}} -#### Pulsar 数据源配置项 +#### Pulsar Source配置项 下述配置主要用于性能调优或者是控制消息确认的行为。如非必要,可以不用强制配置。 @@ -287,8 +274,7 @@ Flink 上的 Pulsar 连接器使用消费者 API 进行消费,它的配置定 ### 动态分区发现 -为了能在启动 Flink 任务之后还能发现在 Pulsar 上扩容的分区或者是新创建的 topic,连接器提供了动态分区发现机制。该机制不需要重启 Flink 任务。 -对选项 `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS` 设置一个正整数即可启用。 +为了能在启动 Flink 任务之后还能发现在 Pulsar 上扩容的分区或者是新创建的 Topic,Pulsar Source 提供了动态分区发现机制。该机制不需要重启 Flink 任务。对选项 `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS` 设置一个正整数即可启用。 ```java // 10 秒查询一次分区信息 @@ -302,29 +288,29 @@ PulsarSource.builder() ### 事件时间和水位线 -默认情况下,连接器使用 Pulsar 的 `Message` 里面的时间作为解析结果的时间戳。用户可以使用 `WatermarkStrategy` 来自行解析出想要的消息时间,并向下游传递对应的水位线。 +默认情况下,Pulsar Source 使用 Pulsar 的 `Message` 里面的时间作为解析结果的时间戳。用户可以使用 `WatermarkStrategy` 来自行解析出想要的消息时间,并向下游传递对应的水位线。 ```java env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With Custom Watermark Strategy"); ``` -[这篇文档]({{< ref "docs/dev/datastream/event-time/generating_watermarks.md" >}}) 详细讲解了如何定义 `WatermarkStrategy`。 +[这篇文档]({{< ref "docs/dev/datastream/event-time/generating_watermarks.md" >}})详细讲解了如何定义 `WatermarkStrategy`。 ### 消息确认 -一旦在 topic 上创建了订阅,消息便会[存储](https://pulsar.apache.org/docs/zh-CN/concepts-architecture-overview/#%E6%8C%81%E4%B9%85%E5%8C%96%E5%AD%98%E5%82%A8)在 Pulsar 里。即使没有消费者,消息也不会被丢弃。只有当连接器同 Pulsar 确认此条消息已经被消费,该消息才以某种机制会被移除。连接器支持四种订阅方式,它们的消息确认方式也大不相同。 +一旦在 Topic 上创建了订阅,消息便会[存储](https://pulsar.apache.org/docs/zh-CN/concepts-architecture-overview/#%E6%8C%81%E4%B9%85%E5%8C%96%E5%AD%98%E5%82%A8)在 Pulsar 里。即使没有消费者,消息也不会被丢弃。只有当 Pulsar Source 同 Pulsar 确认此条消息已经被消费,该消息才以某种机制会被移除。Pulsar Source 支持四种订阅方式,它们的消息确认方式也大不相同。 #### 独占和灾备订阅下的消息确认 -`独占` 和 `灾备` 订阅下,连接器使用累进式确认方式。确认某条消息已经被处理时,其前面被消费的消息会自动被置为已读。Pulsar 连接器会在 Flink 完成检查点时将对应时刻消费的消息置为已读,以此来保证 Pulsar 状态与 Flink 状态一致。 +`独占` 和 `灾备` 订阅下,Pulsar Source 使用累进式确认方式。确认某条消息已经被处理时,其前面消息会自动被置为已读。Pulsar Source 会在 Flink 完成检查点时将对应时刻消费的消息置为已读,以此来保证 Pulsar 状态与 Flink 状态一致。 -如果用户没有在 Flink 上启用检查点,连接器可以使用周期性提交来将消费状态提交给 Pulsar,使用配置 `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` 来进行定义。 +如果用户没有在 Flink 上启用检查点,Pulsar Source 可以使用周期性提交来将消费状态提交给 Pulsar,使用配置 `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` 来进行定义。 -需要注意的是,此种场景下,Pulsar 连接器并不依赖于提交到 Pulsar 的状态来做容错。消息确认只是为了能在 Pulsar 端看到对应的消费处理情况。 +需要注意的是,此种场景下,Pulsar Source 并不依赖于提交到 Pulsar 的状态来做容错。消息确认只是为了能在 Pulsar 端看到对应的消费处理情况。 #### 共享和 key 共享订阅下的消息确认 -`共享` 和 `key 共享` 需要依次确认每一条消息,所以连接器在 Pulsar 事务里面进行消息确认,然后将事务提交到 Pulsar。 +`共享` 和 `key 共享` 需要依次确认每一条消息,所以 Pulsar Source 在 Pulsar 事务里面进行消息确认,然后将事务提交到 Pulsar。 首先需要在 Pulsar 的 `borker.conf` 文件里面启用事务: @@ -332,11 +318,320 @@ env.fromSource(pulsarSource, new CustomWatermarkStrategy(), "Pulsar Source With transactionCoordinatorEnabled=true ``` -连接器创建的事务的默认超时时间为 3 小时,请确保这个时间大于 Flink 检查点的间隔。用户可以使用 `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` 来设置事务的超时时间。 +Pulsar Source 创建的事务的默认超时时间为 3 小时,请确保这个时间大于 Flink 检查点的间隔。用户可以使用 `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` 来设置事务的超时时间。 + +如果用户无法启用 Pulsar 的事务,或者是因为项目禁用了检查点,需要将 `PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE` 选项设置为 `true`,消息从 Pulsar 消费后会被立刻置为已读。Pulsar Source 无法保证此种场景下的消息一致性。 + +Pulsar Source 在 Pulsar 上使用日志的形式记录某个事务下的消息确认,为了更好的性能,请缩短 Flink 做检查点的间隔。 + +## Pulsar Sink + +Pulsar Sink 连接器可以将经过 Flink 处理后的数据写入一个或多个 Pulsar Topic 或者 Topic 下的某些分区。 + +{{< hint info >}} +Pulsar Sink 基于 Flink 最新的 [Sink API](https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction) 实现。 + +如果想要使用旧版的使用 `SinkFuntion` 接口实现的 Sink 连接器,可以使用 StreamNative 维护的 [pulsar-flink](https://github.com/streamnative/pulsar-flink)。 +{{< /hint >}} + +### 使用示例 + +Pulsar Sink 使用 builder 类来创建 `PulsarSink` 实例。 + +下面示例展示了如何通过 Pulsar Sink 以“至少一次”的语义将字符串类型的数据发送给 topic1。 + +```java +DataStream stream = ... + +PulsarSink sink = PulsarSink.builder() + .setServiceUrl(serviceUrl) + .setAdminUrl(adminUrl) + .setTopics("topic1") + .setSerializationSchema(PulsarSerializationSchema.flinkSchema(new SimpleStringSchema())) + .setDeliverGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + .build(); + +stream.sinkTo(sink); +``` + +下列为创建一个 `PulsarSink` 实例必需的属性: + +- Pulsar 数据消费的地址,使用 `setServiceUrl(String)` 方法提供。 +- Pulsar HTTP 管理地址,使用 `setAdminUrl(String)` 方法提供。 +- 需要发送到的 Topic 或者是 Topic 下面的分区,详见[指定写入的topic或者topic分区](#指定写入的topic或者topic分区)。 +- 编码 Pulsar 消息的序列化器,详见[序列化器](#序列化器)。 + +在创建 `PulsarSink` 时,建议使用 `setProducerName(String)` 来指定 `PulsarSink` 内部使用的 Pulsar 生产者名称。这样方便在数据监控页面找到对应的生产者监控指标。 + +### 指定写入的 Topic 或者 Topic 分区 + +`PulsarSink` 指定写入 Topic 的方式和 Pulsar Source [指定消费的 Topic 或者 Topic 分区](#指定消费的-topic-或者-topic-分区)的方式类似。`PulsarSink` 支持以 mixin 风格指定写入的 Topic 或分区。因此,可以指定一组 Topic 或者分区或者是两者都有。 + +```java +// Topic "some-topic1" 和 "some-topic2" +PulsarSink.builder().setTopics("some-topic1", "some-topic2") + +// Topic "topic-a" 的分区 0 和 2 +PulsarSink.builder().setTopics("topic-a-partition-0", "topic-a-partition-2") + +// Topic "topic-a" 以及 Topic "some-topic2" 分区 0 和 2 +PulsarSink.builder().setTopics("topic-a-partition-0", "topic-a-partition-2", "some-topic2") +``` + +动态分区发现默认处于开启状态,这意味着 `PulsarSink` 将会周期性地从 Pulsar 集群中查询 Topic 的元数据来获取可能有的分区数量变更信息。使用 `PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL` 配置项来指定查询的间隔时间。 + +可以选择实现 `TopicRouter` 接口来自定义[消息路由策略](#消息路由策略)。此外,阅读 [Topic 名称简写](#topic-名称简写)将有助于理解 Pulsar 的分区在 Pulsar 连接器中的配置方式。 + +{{< hint warning >}} +如果在 `PulsarSink` 中同时指定了某个 Topic 和其下属的分区,那么 `PulsarSink` 将会自动将两者合并,仅使用外层的 Topic。 + +举个例子,如果通过 `PulsarSink.builder().setTopics("some-topic1", "some-topic1-partition-0")` 来指定写入的 Topic,那么其结果等价于 `PulsarSink.builder().setTopics("some-topic1")`。 +{{< /hint >}} + +### 序列化器 + +序列化器(`PulsarSerializationSchema`)负责将 Flink 中的每条记录序列化成 byte 数组,并通过网络发送至指定的写入 Topic。和 Pulsar Source 类似的是,序列化器同时支持使用基于 Flink 的 `SerializationSchema` 接口实现序列化器和使用 Pulsar 原生的 `Schema` 类型实现的序列化器。不过序列化器并不支持 Pulsar 的 `Schema.AUTO_PRODUCE_BYTES()`。 -如果用户无法启用 Pulsar 的事务,或者是因为项目禁用了检查点,需要将 `PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE` 选项设置为 `true`,消息从 Pulsar 消费后会被立刻置为已读。连接器无法保证此种场景下的消息一致性。 +如果不需要指定 [Message](https://pulsar.apache.org/api/client/2.9.0-SNAPSHOT/org/apache/pulsar/client/api/Message.html) 接口中提供的 key 或者其他的消息属性,可以从上述 2 种预定义的 `PulsarSerializationSchema` 实现中选择适合需求的一种使用。 + +- 使用 Pulsar 的 [Schema](https://pulsar.apache.org/docs/zh-CN/schema-understand/) 来序列化 Flink 中的数据。 + ```java + // 原始数据类型 + PulsarSerializationSchema.pulsarSchema(Schema) -连接器在 Pulsar 上使用日志的形式记录某个事务下的消息确认,为了更好的性能,请缩短 Flink 做检查点的间隔。 + // 有结构数据类型(JSON、Protobuf、Avro 等) + PulsarSerializationSchema.pulsarSchema(Schema, Class) + + // 键值对类型 + PulsarSerializationSchema.pulsarSchema(Schema, Class, Class) + ``` +- 使用 Flink 的 `SerializationSchema` 来序列化数据。 + ```java + PulsarSerializationSchema.flinkSchema(SerializationSchema) + ``` + +同时使用 `PulsarSerializationSchema.pulsarSchema()` 以及在 builder 中指定 `PulsarSinkBuilder.enableSchemaEvolution()` 可以启用 [Schema evolution](https://pulsar.apache.org/docs/zh-CN/schema-evolution-compatibility/#schema-evolution) 特性。该特性会使用 Pulsar Broker 端提供的 Schema 版本兼容性检测以及 Schema 版本演进。下列示例展示了如何启用 Schema Evolution。 + +```java +Schema schema = Schema.AVRO(SomePojo.class); +PulsarSerializationSchema pulsarSchema = PulsarSerializationSchema.pulsarSchema(schema, SomePojo.class); + +PulsarSink sink = PulsarSink.builder() + ... + .setSerializationSchema(pulsarSchema) + .enableSchemaEvolution() + .build(); +``` + +{{< hint warning >}} +如果想要使用 Pulsar 原生的 Schema 序列化消息而不需要 Schema Evolution 特性,那么写入的 Topic 会使用 `Schema.BYTES` 作为消息的 Schema,对应 Topic 的消费者需要自己负责反序列化的工作。 + +例如,如果使用 `PulsarSerializationSchema.pulsarSchema(Schema.STRING)` 而不使用 `PulsarSinkBuilder.enableSchemaEvolution()`。那么在写入 Topic 中所记录的消息 Schema 将会是 `Schema.BYTES`。 +{{< /hint >}} + +### 消息路由策略 + +在 Pulsar Sink 中,消息路由发生在于分区之间,而非上层 Topic。对于给定 Topic 的情况,路由算法会首先会查询出 Topic 之上所有的分区信息,并在这些分区上实现消息的路由。Pulsar Sink 默认提供 2 种路由策略的实现。 + +- `KeyHashTopicRouter`:使用消息的 key 对应的哈希值来取模计算出消息对应的 Topic 分区。 + + 使用此路由可以将具有相同 key 的消息发送至同一个 Topic 分区。消息的 key 可以在自定义 `PulsarSerializationSchema` 时,在 `serialize()` 方法内使用 `PulsarMessageBuilder.key(String key)` 来予以指定。 + + 如果消息没有包含 key,此路由策略将从 Topic 分区中随机选择一个发送。 + + 可以使用 `MessageKeyHash.JAVA_HASH` 或者 `MessageKeyHash.MURMUR3_32_HASH` 两种不同的哈希算法来计算消息 key 的哈希值。使用 `PulsarSinkOptions.PULSAR_MESSAGE_KEY_HASH` 配置项来指定想要的哈希算法。 + +- `RoundRobinRouter`:轮换使用用户给定的 Topic 分区。 + + 消息将会轮替地选取 Topic 分区,当往某个 Topic 分区里写入指定数量的消息后,将会轮换至下一个 Topic 分区。使用 `PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES` 指定向一个 Topic 分区中写入的消息数量。 + +还可以通过实现 `TopicRouter` 接口来自定义消息路由策略,请注意 TopicRouter 的实现需要能被序列化。 + +在 `TopicRouter` 内可以指定任意的 Topic 分区(即使这个 Topic 分区不在 `setTopics()` 指定的列表中)。因此,当使用自定义的 `TopicRouter` 时,`PulsarSinkBuilder.setTopics` 选项是可选的。 + +```java +@PublicEvolving +public interface TopicRouter extends Serializable { + + String route(IN in, List partitions, PulsarSinkContext context); + + default void open(SinkConfiguration sinkConfiguration) { + // 默认无操作 + } +} +``` + +{{< hint info >}} +如前文所述,Pulsar 分区的内部被实现为一个无分区的 Topic,一般情况下 Pulsar 客户端会隐藏这个实现,并且提供内置的消息路由策略。Pulsar Sink 并没有使用 Pulsar 客户端提供的路由策略和封装,而是使用了 Pulsar 客户端更底层的 API 自行实现了消息路由逻辑。这样做的主要目的是能够在属于不同 Topic 的分区之间定义更灵活的消息路由策略。 + +详情请参考 Pulsar 的 [partitioned topics](https://pulsar.apache.org/docs/zh-CN/cookbooks-partitioned/)。 +{{< /hint >}} + +### 发送一致性 + +`PulsarSink` 支持三种发送一致性。 + +- `NONE`:Flink 应用运行时可能出现数据丢失的情况。在这种模式下,Pulsar Sink 发送消息后并不会检查消息是否发送成功。此模式具有最高的吞吐量,可用于一致性没有要求的场景。 +- `AT_LEAST_ONCE`:每条消息**至少有**一条对应消息发送至 Pulsar,发送至 Pulsar 的消息可能会因为 Flink 应用重启而出现重复。 +- `EXACTLY_ONCE`:每条消息**有且仅有**一条对应消息发送至 Pulsar。发送至 Pulsar 的消息不会有重复也不会丢失。Pulsar Sink 内部依赖 [Pulsar 事务](https://pulsar.apache.org/docs/zh-CN/transactions/)和两阶段提交协议来保证每条记录都能正确发往 Pulsar。 + +### 消息延时发送 + +[消息延时发送](https://pulsar.apache.org/docs/zh-CN/next/concepts-messaging/#%E6%B6%88%E6%81%AF%E5%BB%B6%E8%BF%9F%E4%BC%A0%E9%80%92)特性可以让指定发送的每一条消息需要延时一段时间后才能被下游的消费者所消费。当延时消息发送特性启用时,Pulsar Sink 会**立刻**将消息发送至 Pulsar Broker。但该消息在指定的延迟时间到达前将会保持对下游消费者不可见。 + +消息延时发送仅在 `Shared` 订阅模式下有效,在 `Exclusive` 和 `Failover` 模式下该特性无效。 + +可以使用 `MessageDelayer.fixed(Duration)` 创建一个 `MessageDelayer` 来为所有消息指定恒定的接收时延,或者实现 `MessageDelayer` 接口来为不同的消息指定不同的接收时延。 + +{{< hint warning >}} +消息对下游消费者的可见时间应当基于 `PulsarSinkContext.processTime() `计算得到。 +{{< /hint >}} + +### Sink 配置项 + +可以在 builder 类里通过 `setConfig(ConfigOption, T)` 和 `setConfig(Configuration)` 方法给定下述的全部配置。 + +#### PulsarClient 和 PulsarAdmin 配置项 + +Pulsar Sink 和 Pulsar Source 公用的配置选项可参考 + +- [Pulsar Java 客户端配置项](#pulsar-java-客户端配置项) +- [Pulsar 管理 API 配置项](#pulsar-管理-API-配置项) + +#### Pulsar 生产者 API 配置项 + +Pulsar Sink 使用生产者 API 来发送消息。Pulsar 的 `ProducerConfigurationData` 中大部分的配置项被映射为 `PulsarSinkOptions` 里的选项。 + +{{< generated/pulsar_producer_configuration >}} + +#### Pulsar Sink 配置项 + +下述配置主要用于性能调优或者是控制消息确认的行为。如非必要,可以不用考虑配置。 + +{{< generated/pulsar_sink_configuration >}} + +### Sink 监控指标 + +下列表格列出了当前 Sink 支持的监控指标,前 6 个指标是 [FLIP-33: Standardize Connector Metrics]([https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics](https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics)) 中规定的 Sink 连接器应当支持的标准指标。 + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ScopeMetricsUser VariablesDescriptionType
OperatornumBytesOutn/aPulsar Sink 启动后总共发出的字节数Counter
numBytesOutPerSecondn/a每秒发送的字节数Meter
numRecordsOutn/aPulsar Sink 启动后总共发出的消息数Counter
numRecordsOutPerSecondn/a每秒发送的消息数Meter
numRecordsOutErrorsn/a总共发送消息失败的次数Counter
currentSendTimen/a最近一条消息从被放入客户端缓冲队列到收到消息确认的时间Gauge
PulsarSink.numAcksReceivedn/a总共收到的确认数Counter
PulsarSink.sendLatencyMaxn/a所有生产者的最大发送延迟Gauge
PulsarSink.producer."ProducerName".sendLatency50PctProducerName某个生产者在过去的一个窗口内的发送延迟的中位数Gauge
PulsarSink.producer."ProducerName".sendLatency75PctProducerName某个生产者在过去的一个窗口内的发送延迟的 75 百分位数Gauge
PulsarSink.producer."ProducerName".sendLatency95PctProducerName某个生产者在过去的一个窗口内的发送延迟的 95 百分位数Gauge
PulsarSink.producer."ProducerName".sendLatency99PctProducerName某个生产者在过去的一个窗口内的发送延迟的 99 百分位数Gauge
PulsarSink.producer."ProducerName".sendLatency999PctProducerName某个生产者在过去的一个窗口内的发送延迟的 99.9 百分位数Gauge
+ +{{< hint info >}} +指标 `numBytesOut`、`numRecordsOut` 和 `numRecordsOutErrors` 从 Pulsar Producer 实例的监控指标中获得。 + +`currentSendTime` 记录了最近一条消息从放入生产者的缓冲队列到消息被消费确认所耗费的时间。这项指标在 `NONE` 发送一致性下不可用。 +{{< /hint >}} + +默认情况下,Pulsar 生产者每隔 60 秒才会刷新一次监控数据,然而 Pulsar Sink 每 500 毫秒就会从 Pulsar 生产者中获得最新的监控数据。因此 `numRecordsOut`、`numBytesOut`、`numAcksReceived` 以及 `numRecordsOutErrors` 4 个指标实际上每 60 秒才会刷新一次。 + +如果想要更高地刷新评率,可以通过 `builder.setConfig(PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS. 1L)` 来将 Pulsar 生产者的监控数据刷新频率调整至相应值(最低为1s)。 + +`numBytesOutRate` 和 `numRecordsOutRate` 指标是 Flink 内部通过 `numBytesOut` 和 `numRecordsOut` 计数器,在一个 60 秒的窗口内计算得到的。 + +### 设计思想简述 + +Pulsar Sink 遵循 [FLIP-191](https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction) 中定义的 Sink API 设计。 + +#### 无状态的 SinkWriter + +在 `EXACTLY_ONCE` 一致性下,Pulsar Sink 不会将事务相关的信息存放于检查点快照中。这意味着当 Flink 应用重启时,Pulsar Sink 会创建新的事务实例。上一次运行过程中任何未提交事务中的消息会因为超时中止而无法被下游的消费者所消费。这样的设计保证了 SinkWriter 是无状态的。 + +#### Pulsar Schema Evolution + +[Pulsar Schema Evolution](https://pulsar.apache.org/docs/zh-CN/schema-evolution-compatibility/) 允许用户在一个 Flink 应用程序中使用的数据模型发生特定改变后(比如向基于 ARVO 的 POJO 类中增加或删除一个字段),仍能使用同一个 Flink 应用程序的代码。 + +可以在 Pulsar 集群内指定哪些类型的数据模型的改变是被允许的,详情请参阅 [Pulsar Schema Evolution](https://pulsar.apache.org/docs/zh-CN/schema-evolution-compatibility/)。 ## 升级至最新的连接器 @@ -347,6 +642,11 @@ transactionCoordinatorEnabled=true ## 问题诊断 -使用 Flink 和 Pulsar 交互时如果遇到问题,一定要牢记 Flink 只使用了 Pulsar 的[Java 客户端](https://pulsar.apache.org/docs/zh-CN/client-libraries-java/) 和[管理 API](https://pulsar.apache.org/docs/zh-CN/admin-api-overview/)。用户遇到的问题很有可能与 Flink 无关,请先升级 Pulsar 的版本、Pulsar 客户端的版本、或者修改 Pulsar 的配置,Pulsar 连接器的配置来尝试解决问题。 +使用 Flink 和 Pulsar 交互时如果遇到问题,由于 Flink 内部实现只是基于 Pulsar 的 [Java 客户端](https://pulsar.apache.org/docs/zh-CN/client-libraries-java/)和[管理 API](https://pulsar.apache.org/docs/zh-CN/admin-api-overview/) 而开发的。 + +用户遇到的问题可能与 Flink 无关,请先升级 Pulsar 的版本、Pulsar 客户端的版本,或者修改 Pulsar 的配置、Pulsar 连接器的配置来尝试解决问题。 + +### 在 Java 11 上使用不稳定 +Pulsar connector 在 Java 11 中有一些尚未修复的问题。我们当前推荐在 Java 8 环境中运行Pulsar connector. {{< top >}} diff --git a/docs/content/docs/connectors/datastream/pulsar.md b/docs/content/docs/connectors/datastream/pulsar.md index d5442419658a0..11e62cc2acad0 100644 --- a/docs/content/docs/connectors/datastream/pulsar.md +++ b/docs/content/docs/connectors/datastream/pulsar.md @@ -24,14 +24,13 @@ under the License. # Apache Pulsar Connector -Flink provides an [Apache Pulsar](https://pulsar.apache.org) source connector for reading data from Pulsar topics with exactly-once guarantees. +Flink provides an [Apache Pulsar](https://pulsar.apache.org) connector for reading and writing data from and to Pulsar topics with exactly-once guarantees. ## Dependency -You can use the connector with Pulsar 2.7.0 or higher. However, the Pulsar source connector supports -Pulsar [transactions](https://pulsar.apache.org/docs/en/txn-what/), -it is recommended to use Pulsar 2.8.0 or higher releases. -For details on Pulsar compatibility, please refer to the [PIP-72](https://github.com/apache/pulsar/wiki/PIP-72%3A-Introduce-Pulsar-Interface-Taxonomy%3A-Audience-and-Stability-Classification). +You can use the connector with the Pulsar 2.8.1 or higher. Because the Pulsar connector supports +Pulsar [transactions](https://pulsar.apache.org/docs/en/txn-what/), it is recommended to use the Pulsar 2.9.2 or higher. +Details on Pulsar compatibility can be found in [PIP-72](https://github.com/apache/pulsar/wiki/PIP-72%3A-Introduce-Pulsar-Interface-Taxonomy%3A-Audience-and-Stability-Classification). {{< artifact flink-connector-pulsar >}} @@ -43,18 +42,16 @@ See how to link with them for cluster execution [here]({{< ref "docs/dev/configu {{< hint info >}} This part describes the Pulsar source based on the new [data source]({{< ref "docs/dev/datastream/sources.md" >}}) API. - -If you want to use the legacy `SourceFunction` or on Flink 1.13 or lower releases, just use the StreamNative's [pulsar-flink](https://github.com/streamnative/pulsar-flink). {{< /hint >}} ### Usage -Pulsar source provides a builder class for constructing an instance of PulsarSource. The code snippet below shows -how to build a PulsarSource to consume messages from the earliest cursor of topic "persistent://public/default/my-topic", -with **Exclusive** subscription `my-subscription` and deserialize the raw payload of the messages as strings. +The Pulsar source provides a builder class for constructing a PulsarSource instance. The code snippet below builds a PulsarSource instance. It consumes messages from the earliest cursor of the topic +"persistent://public/default/my-topic" in **Exclusive** subscription type (`my-subscription`) +and deserializes the raw payload of the messages as strings. ```java -PulsarSource pulsarSource = PulsarSource.builder() +PulsarSource source = PulsarSource.builder() .setServiceUrl(serviceUrl) .setAdminUrl(adminUrl) .setStartCursor(StartCursor.earliest()) @@ -69,13 +66,17 @@ env.fromSource(source, WatermarkStrategy.noWatermarks(), "Pulsar Source"); The following properties are **required** for building a PulsarSource: -- Pulsar service url, configured by `setServiceUrl(String)` -- Pulsar service http url (aka. admin url), configured by `setAdminUrl(String)` +- Pulsar service URL, configured by `setServiceUrl(String)` +- Pulsar service HTTP URL (also known as admin URL), configured by `setAdminUrl(String)` - Pulsar subscription name, configured by `setSubscriptionName(String)` - Topics / partitions to subscribe, see the following - [Topic-partition subscription](#topic-partition-subscription) for more details. + [topic-partition subscription](#topic-partition-subscription) for more details. - Deserializer to parse Pulsar messages, see the following - [Deserializer](#deserializer) for more details. + [deserializer](#deserializer) for more details. + +It is recommended to set the consumer name in Pulsar Source by `setConsumerName(String)`. +This sets a unique name for the Flink connector in the Pulsar statistic dashboard. +You can use it to monitor the performance of your Flink connector and applications. ### Topic-partition Subscription @@ -96,64 +97,65 @@ Pulsar source provide two ways of topic-partition subscription: #### Flexible Topic Naming -Since Pulsar 2.0, all topic names internally have the form `{persistent|non-persistent}://tenant/namespace/topic`. +Since Pulsar 2.0, all topic names internally are in a form of `{persistent|non-persistent}://tenant/namespace/topic`. Now, for partitioned topics, you can use short names in many cases (for the sake of simplicity). The flexible naming system stems from the fact that there is now a default topic type, tenant, and namespace in a Pulsar cluster. -Topic property | Default -:------------|:------- -topic type | `persistent` -tenant | `public` -namespace | `default` +| Topic property | Default | +|:---------------|:-------------| +| topic type | `persistent` | +| tenant | `public` | +| namespace | `default` | -This table lists a mapping relationship between your input topic name and translated topic name: +This table lists a mapping relationship between your input topic name and the translated topic name: -Input topic name | Translated topic name -:----------------|:--------------------- -`my-topic` | `persistent://public/default/my-topic` -`my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` +| Input topic name | Translated topic name | +|:----------------------------------|:-----------------------------------------------| +| `my-topic` | `persistent://public/default/my-topic` | +| `my-tenant/my-namespace/my-topic` | `persistent://my-tenant/my-namespace/my-topic` | {{< hint warning >}} -For non-persistent topics, you need to continue to specify the entire topic name, +For non-persistent topics, you need to specify the entire topic name, as the default-based rules do not apply for non-partitioned topics. -Thus, you cannot use a short name like `non-persistent://my-topic` and would need to use `non-persistent://public/default/my-topic` instead. +Thus, you cannot use a short name like `non-persistent://my-topic` and need to use `non-persistent://public/default/my-topic` instead. {{< /hint >}} #### Subscribing Pulsar Topic Partition Internally, Pulsar divides a partitioned topic as a set of non-partitioned topics according to the partition size. -For example, if a `simple-string` topic with 3 partitions is created under the `sample` tenant with `flink` namespace. +For example, if a `simple-string` topic with 3 partitions is created under the `sample` tenant with the `flink` namespace. The topics on Pulsar would be: -Topic name | Partitioned -:--------- | :---------- -`persistent://sample/flink/simple-string` | Y -`persistent://sample/flink/simple-string-partition-0` | N -`persistent://sample/flink/simple-string-partition-1` | N -`persistent://sample/flink/simple-string-partition-2` | N +| Topic name | Partitioned | +|:------------------------------------------------------|:------------| +| `persistent://sample/flink/simple-string` | Y | +| `persistent://sample/flink/simple-string-partition-0` | N | +| `persistent://sample/flink/simple-string-partition-1` | N | +| `persistent://sample/flink/simple-string-partition-2` | N | You can directly consume messages from the topic partitions by using the non-partitioned topic names above. -For example, use `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` would consume the partitions 1 and 2 of the `sample/flink/simple-string` topic. +For example, use `PulsarSource.builder().setTopics("sample/flink/simple-string-partition-1", "sample/flink/simple-string-partition-2")` +would consume the partitions 1 and 2 of the `sample/flink/simple-string` topic. -#### RegexSubscriptionMode for Topic Pattern +#### Setting Topic Patterns -Pulsar connector extracts the topic type (`persistent` or `non-persistent`) from the given topic pattern. -For example, `PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` would be `non-persistent`. -The topic type would be `persistent` if you do not provide the topic type in the regular expression. +The Pulsar source extracts the topic type (`persistent` or `non-persistent`) from the provided topic pattern. +For example, you can use the `PulsarSource.builder().setTopicPattern("non-persistent://my-topic*")` to specify a `non-persistent` topic. +By default, a `persistent` topic is created if you do not specify the topic type in the regular expression. -To consume both `persistent` and `non-persistent` topics based on the topic pattern, -you can use `setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)`. -Pulsar connector would filter the available topics by the `RegexSubscriptionMode`. +You can use `setTopicPattern("topic-*", RegexSubscriptionMode.AllTopics)` to consume +both `persistent` and `non-persistent` topics based on the topic pattern. +The Pulsar source would filter the available topics by the `RegexSubscriptionMode`. ### Deserializer -A deserializer (Deserialization schema) is required for parsing Pulsar messages. The deserializer is -configured by `setDeserializationSchema(PulsarDeserializationSchema)`. +A deserializer (`PulsarDeserializationSchema`) is for decoding Pulsar messages from bytes. +You can configure the deserializer using `setDeserializationSchema(PulsarDeserializationSchema)`. The `PulsarDeserializationSchema` defines how to deserialize a Pulsar `Message`. If only the raw payload of a message (message data in bytes) is needed, -you can use the predefined `PulsarDeserializationSchema`. Pulsar connector provides three types of implementation. +you can use the predefined `PulsarDeserializationSchema`. Pulsar connector provides three implementation methods. - Decode the message by using Pulsar's [Schema](https://pulsar.apache.org/docs/en/schema-understand/). ```java @@ -176,12 +178,12 @@ you can use the predefined `PulsarDeserializationSchema`. Pulsar connector provi ``` Pulsar `Message` contains some [extra properties](https://pulsar.apache.org/docs/en/concepts-messaging/#messages), -such as message key, message publish time, message time, application defined key/value pairs that will be attached to the message, etc. -These properties could be acquired by the `Message` interface. +such as message key, message publish time, message time, and application-defined key/value pairs etc. +These properties could be defined in the `Message` interface. If you want to deserialize the Pulsar message by these properties, you need to implement `PulsarDeserializationSchema`. -And ensure that the `TypeInformation` from the `PulsarDeserializationSchema.getProducedType()` must be correct. -Flink would use this `TypeInformation` for passing the messages to downstream operators. +Ensure that the `TypeInformation` from the `PulsarDeserializationSchema.getProducedType()` is correct. +Flink uses this `TypeInformation` to pass the messages to downstream operators. ### Pulsar Subscriptions @@ -196,7 +198,7 @@ The subscription name is required for consuming messages. Pulsar connector suppo There is no difference between `Exclusive` and `Failover` in the Pulsar connector. When a Flink reader crashes, all (non-acknowledged and subsequent) messages are redelivered to the available Flink readers. -By default, if no subscription type is defined, Pulsar source uses `Shared` subscription. +By default, if no subscription type is defined, Pulsar source uses the `Shared` subscription type. ```java // Shared subscription with name "my-shared" @@ -206,16 +208,15 @@ PulsarSource.builder().setSubscriptionName("my-shared"); PulsarSource.builder().setSubscriptionName("my-exclusive").setSubscriptionType(SubscriptionType.Exclusive); ``` -If you want to use `Key_Shared` subscription type on the Pulsar connector. Ensure that you provide a `RangeGenerator` implementation. -The `RangeGenerator` generates a set of key hash ranges so that -a respective reader subtask will only dispatch messages where the hash of the message key is contained in the specified range. +Ensure that you provide a `RangeGenerator` implementation if you want to use the `Key_Shared` subscription type on the Pulsar connector. +The `RangeGenerator` generates a set of key hash ranges so that a respective reader subtask only dispatches messages where the hash of the message key is contained in the specified range. -Pulsar connector would use a `UniformRangeGenerator` which would divides the range by the Flink source parallelism -if no `RangeGenerator` is provided in the `Key_Shared` subscription type. +The Pulsar connector uses `UniformRangeGenerator` that divides the range by the Flink source +parallelism if no `RangeGenerator` is provided in the `Key_Shared` subscription type. ### Starting Position -Pulsar source is able to consume messages starting from different positions by `setStartCursor(StartCursor)`. +The Pulsar source is able to consume messages starting from different positions by setting the `setStartCursor(StartCursor)` option. Built-in start cursors include: - Start from the earliest available message in the topic. @@ -227,14 +228,14 @@ Built-in start cursors include: StartCursor.latest(); ``` - Start from a specified message between the earliest and the latest. - Pulsar connector would consume from the latest available message if the message id doesn't exist. +The Pulsar connector consumes from the latest available message if the message ID does not exist. The start message is included in consuming result. ```java StartCursor.fromMessageId(MessageId); ``` - Start from a specified message between the earliest and the latest. - Pulsar connector would consume from the latest available message if the message id doesn't exist. +The Pulsar connector consumes from the latest available message if the message ID doesn't exist. Include or exclude the start message by using the second boolean parameter. ```java @@ -248,37 +249,35 @@ Built-in start cursors include: {{< hint info >}} Each Pulsar message belongs to an ordered sequence on its topic. The sequence ID (`MessageId`) of the message is ordered in that sequence. -`MessageId` contains some extra information (the ledger, entry, partition) on how the message is stored, +The `MessageId` contains some extra information (the ledger, entry, partition) about how the message is stored, you can create a `MessageId` by using `DefaultImplementation.newMessageId(long ledgerId, long entryId, int partitionIndex)`. {{< /hint >}} ### Boundedness -Pulsar source supports streaming and batch running modes. -By default, the `PulsarSource` is set to run in the streaming mode. +The Pulsar source supports streaming and batch execution mode. +By default, the `PulsarSource` is configured for unbounded data. -In streaming mode, Pulsar source never stops until a Flink job fails or is cancelled. However, -you can set Pulsar source stopping at a stop position by using ```setUnboundedStopCursor(StopCursor)```. -The Pulsar source will finish when all partitions reach their specified stop positions. +For unbounded data the Pulsar source never stops until a Flink job is stopped or failed. +You can use the `setUnboundedStopCursor(StopCursor)` to set the Pulsar source to stop at a specific stop position. -You can use ```setBoundedStopCursor(StopCursor)``` to specify a stop position so that the Pulsar source can run in the batch mode. -When all partitions have reached their stop positions, the source will finish. +You can use `setBoundedStopCursor(StopCursor)` to specify a stop position for bounded data. Built-in stop cursors include: -- Connector will never stop consuming. +- The Pulsar source never stops consuming messages. ```java StopCursor.never(); ``` -- Stop at the latest available message in Pulsar when the connector starts consuming. +- Stop at the latest available message when the Pulsar source starts consuming messages. ```java StopCursor.latest(); ``` -- Stop when connector meet a given message, or stop at a message which is produced after this given message. +- Stop when the connector meets a given message, or stop at a message which is produced after this given message. ```java StopCursor.atMessageId(MessageId); ``` -- Stop but include the given message in consuming result. +- Stop but include the given message in the consuming result. ```java StopCursor.afterMessageId(MessageId); ``` @@ -287,15 +286,16 @@ Built-in stop cursors include: StopCursor.atEventTime(long); ``` -### Configurable Options +### Source Configurable Options In addition to configuration options described above, you can set arbitrary options for `PulsarClient`, -`PulsarAdmin`, Pulsar `Consumer` and `PulsarSource` by using `setConfig(ConfigOption, T)` and `setConfig(Configuration)`. +`PulsarAdmin`, Pulsar `Consumer` and `PulsarSource` by using `setConfig(ConfigOption, T)`, +`setConfig(Configuration)` and `setConfig(Properties)`. #### PulsarClient Options -Pulsar connector use the [client API](https://pulsar.apache.org/docs/en/client-libraries-java/) -to create the `Consumer` instance. Pulsar connector extracts most parts of Pulsar's `ClientConfigurationData`, +The Pulsar connector uses the [client API](https://pulsar.apache.org/docs/en/client-libraries-java/) +to create the `Consumer` instance. The Pulsar connector extracts most parts of Pulsar's `ClientConfigurationData`, which is required for creating a `PulsarClient`, as Flink configuration options in `PulsarOptions`. {{< generated/pulsar_client_configuration >}} @@ -303,8 +303,9 @@ which is required for creating a `PulsarClient`, as Flink configuration options #### PulsarAdmin Options The [admin API](https://pulsar.apache.org/docs/en/admin-api-overview/) is used for querying topic metadata -and for discovering the desired topics when Pulsar connector uses topic pattern subscription. It would share most part of the -configuration options with the client API. The configuration options listed here are only used in the admin API. +and for discovering the desired topics when the Pulsar connector uses topic-pattern subscription. +It shares most part of the configuration options with the client API. +The configuration options listed here are only used in the admin API. They are also defined in `PulsarOptions`. {{< generated/pulsar_admin_configuration >}} @@ -312,34 +313,34 @@ They are also defined in `PulsarOptions`. #### Pulsar Consumer Options In general, Pulsar provides the Reader API and Consumer API for consuming messages in different scenarios. -Flink's Pulsar connector uses the Consumer API. It extracts most parts of Pulsar's `ConsumerConfigurationData` as Flink configuration options in `PulsarSourceOptions`. +The Pulsar connector uses the Consumer API. It extracts most parts of Pulsar's `ConsumerConfigurationData` as Flink configuration options in `PulsarSourceOptions`. {{< generated/pulsar_consumer_configuration >}} #### PulsarSource Options The configuration options below are mainly used for customizing the performance and message acknowledgement behavior. -You can just leave them alone if you do not meet any performance issues. +You can ignore them if you do not have any performance issues. {{< generated/pulsar_source_configuration >}} ### Dynamic Partition Discovery To handle scenarios like topic scaling-out or topic creation without restarting the Flink -job, Pulsar source can be configured to periodically discover new partitions under provided -topic-partition subscribing pattern. To enable partition discovery, set a non-negative value for -the option `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS`: +job, the Pulsar source periodically discover new partitions under a provided +topic-partition subscription pattern. To enable partition discovery, you can set a non-negative value for +the `PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS` option: ```java // discover new partitions per 10 seconds PulsarSource.builder() - .setConfig(PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 10000); + .setConfig(PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 10000); ``` {{< hint warning >}} -- Partition discovery is **enabled** by default. Pulsar connector would query the topic metadata every 30 seconds. -- You need to set the partition discovery interval to a negative value to disable this feature. -- The partition discovery would be disabled in batch mode even if you set this option with a non-negative value. +- Partition discovery is **enabled** by default. The Pulsar connector queries the topic metadata every 30 seconds. +- To disable partition discovery, you need to set a negative partition discovery interval. +- Partition discovery is disabled for bounded data even if you set this option with a non-negative value. {{< /hint >}} ### Event Time and Watermarks @@ -359,7 +360,7 @@ details about how to define a `WatermarkStrategy`. When a subscription is created, Pulsar [retains](https://pulsar.apache.org/docs/en/concepts-architecture-overview/#persistent-storage) all messages, even if the consumer is disconnected. The retained messages are discarded only when the connector acknowledges that all these messages are processed successfully. -Pulsar connector supports four subscription types, which makes the acknowledgement behaviors variety among different subscriptions. +The Pulsar connector supports four subscription types, which makes the acknowledgement behaviors vary among different subscriptions. #### Acknowledgement on Exclusive and Failover Subscription Types @@ -367,18 +368,18 @@ Pulsar connector supports four subscription types, which makes the acknowledgeme the latest successfully consumed message. All the message before the given message are marked with a consumed status. -Pulsar source acknowledges the current consuming message when checkpoints are **completed**, -to ensure the consistency between Flink's checkpoint state and committed position on Pulsar brokers. +The Pulsar source acknowledges the current consuming message when checkpoints are **completed**, +to ensure the consistency between Flink's checkpoint state and committed position on the Pulsar brokers. If checkpointing is disabled, Pulsar source periodically acknowledges messages. -You can set the acknowledgement period by using the `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` option. +You can use the `PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL` option to set the acknowledgement period. Pulsar source does **NOT** rely on committed positions for fault tolerance. -Acknowledging messages is only for exposing the progress of consumer and monitoring on these two subscription types. +Acknowledging messages is only for exposing the progress of consumers and monitoring on these two subscription types. #### Acknowledgement on Shared and Key_Shared Subscription Types -`Shared` and `Key_Shared` subscription types need to acknowledge messages one by one. You can acknowledge +In `Shared` and `Key_Shared` subscription types, messages are acknowledged one by one. You can acknowledge a message in a transaction and commit it to Pulsar. You should enable transaction in the Pulsar `borker.conf` file when using these two subscription types in connector: @@ -387,19 +388,399 @@ You should enable transaction in the Pulsar `borker.conf` file when using these transactionCoordinatorEnabled=true ``` -Pulsar transaction would be created with 3 hours as the timeout by default. Make sure that timeout > checkpoint interval + maximum recovery time. -A shorter checkpoint interval would increase the consuming performance. -You can change the transaction timeout by using the `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` option. +The default timeout for Pulsar transactions is 3 hours. +Make sure that that timeout is greater than checkpoint interval + maximum recovery time. +A shorter checkpoint interval indicates a better consuming performance. +You can use the `PulsarSourceOptions.PULSAR_TRANSACTION_TIMEOUT_MILLIS` option to change the transaction timeout. If checkpointing is disabled or you can not enable the transaction on Pulsar broker, you should set `PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE` to `true`. -The message would be immediately acknowledged after consuming. -We can not promise consistency in this scenario. +The message is immediately acknowledged after consuming. +No consistency guarantees can be made in this scenario. {{< hint info >}} All acknowledgements in a transaction are recorded in the Pulsar broker side. {{< /hint >}} +## Pulsar Sink + +The Pulsar Sink supports writing records into one or more Pulsar topics or a specified list of Pulsar partitions. + +{{< hint info >}} +This part describes the Pulsar sink based on the new +[data sink](https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction) API. + +If you still want to use the legacy `SinkFunction` or on Flink 1.14 or previous releases, just use the StreamNative's +[pulsar-flink](https://github.com/streamnative/pulsar-flink). +{{< /hint >}} + +### Usage + +The Pulsar Sink uses a builder class to construct the `PulsarSink` instance. +This example writes a String record to a Pulsar topic with at-least-once delivery guarantee. + +```java +DataStream stream = ... + +PulsarSink sink = PulsarSink.builder() + .setServiceUrl(serviceUrl) + .setAdminUrl(adminUrl) + .setTopics("topic1") + .setSerializationSchema(PulsarSerializationSchema.flinkSchema(new SimpleStringSchema())) + .setDeliverGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + .build(); + +stream.sinkTo(sink); +``` + +The following properties are **required** for building PulsarSink: + +- Pulsar service url, configured by `setServiceUrl(String)` +- Pulsar service http url (aka. admin url), configured by `setAdminUrl(String)` +- Topics / partitions to write, see [writing targets](#writing-targets) for more details. +- Serializer to generate Pulsar messages, see [serializer](#serializer) for more details. + +It is recommended to set the producer name in Pulsar Source by `setProducerName(String)`. +This sets a unique name for the Flink connector in the Pulsar statistic dashboard. +You can use it to monitor the performance of your Flink connector and applications. + +### Producing to topics + +Defining the topics for producing is similar to the [topic-partition subscription](#topic-partition-subscription) +in the Pulsar source. We support a mix-in style of topic setting. You can provide a list of topics, +partitions, or both of them. + +```java +// Topic "some-topic1" and "some-topic2" +PulsarSink.builder().setTopics("some-topic1", "some-topic2") + +// Partition 0 and 2 of topic "topic-a" +PulsarSink.builder().setTopics("topic-a-partition-0", "topic-a-partition-2") + +// Partition 0 and 2 of topic "topic-a" and topic "some-topic2" +PulsarSink.builder().setTopics("topic-a-partition-0", "topic-a-partition-2", "some-topic2") +``` + +The topics you provide support auto partition discovery. We query the topic metadata from the Pulsar in a fixed interval. +You can use the `PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL` option to change the discovery interval option. + +Configuring writing targets can be replaced by using a custom [`TopicRouter`] +[message routing](#message-routing). Configuring partitions on the Pulsar connector is explained in the [flexible topic naming](#flexible-topic-naming) section. + +{{< hint warning >}} +If you build the Pulsar sink based on both the topic and its corresponding partitions, Pulsar sink merges them and only uses the topic. + +For example, when using the `PulsarSink.builder().setTopics("some-topic1", "some-topic1-partition-0")` option to build the Pulsar sink, +this is simplified to `PulsarSink.builder().setTopics("some-topic1")`. +{{< /hint >}} + +### Serializer + +A serializer (`PulsarSerializationSchema`) is required for serializing the record instance into bytes. +Similar to `PulsarSource`, Pulsar sink supports both Flink's `SerializationSchema` and +Pulsar's `Schema`. Pulsar's `Schema.AUTO_PRODUCE_BYTES()` is not supported in the Pulsar sink. + +If you do not need the message key and other message properties in Pulsar's +[Message](https://pulsar.apache.org/api/client/2.9.0-SNAPSHOT/org/apache/pulsar/client/api/Message.html) interface, +you can use the predefined `PulsarSerializationSchema`. The Pulsar sink provides two implementation methods. + +- Encode the message by using Pulsar's [Schema](https://pulsar.apache.org/docs/en/schema-understand/). + ```java + // Primitive types + PulsarSerializationSchema.pulsarSchema(Schema) + + // Struct types (JSON, Protobuf, Avro, etc.) + PulsarSerializationSchema.pulsarSchema(Schema, Class) + + // KeyValue type + PulsarSerializationSchema.pulsarSchema(Schema, Class, Class) + ``` +- Encode the message by using Flink's `SerializationSchema` + ```java + PulsarSerializationSchema.flinkSchema(SerializationSchema) + ``` + +[Schema evolution](https://pulsar.apache.org/docs/en/schema-evolution-compatibility/#schema-evolution) +can be enabled by users using `PulsarSerializationSchema.pulsarSchema()` and +`PulsarSinkBuilder.enableSchemaEvolution()`. This means that any broker schema validation is in place. + +```java +Schema schema = Schema.AVRO(SomePojo.class); +PulsarSerializationSchema pulsarSchema = PulsarSerializationSchema.pulsarSchema(schema, SomePojo.class); + +PulsarSink sink = PulsarSink.builder() + ... + .setSerializationSchema(pulsarSchema) + .enableSchemaEvolution() + .build(); +``` + +{{< hint warning >}} +If you use Pulsar schema without enabling schema evolution, the target topic will have a `Schema.BYTES` schema. +Consumers will need to handle the deserialization (if needed) themselves. + +For example, if you set `PulsarSerializationSchema.pulsarSchema(Schema.STRING)` without enabling schema evolution, +the schema stored in Pulsar topics is `Schema.BYTES`. +{{< /hint >}} + +### Message Routing + +Routing in Pulsar Sink is operated on the partition level. For a list of partitioned topics, +the routing algorithm first collects all partitions from different topics, and then calculates routing within all the partitions. +By default Pulsar Sink supports two router implementation. + +- `KeyHashTopicRouter`: use the hashcode of the message's key to decide the topic partition that messages are sent to. + + The message key is provided by `PulsarSerializationSchema.key(IN, PulsarSinkContext)` + You need to implement this interface and extract the message key when you want to send the message with the same key to the same topic partition. + + If you do not provide the message key. A topic partition is randomly chosen from the topic list. + + The message key can be hashed in two ways: `MessageKeyHash.JAVA_HASH` and `MessageKeyHash.MURMUR3_32_HASH`. + You can use the `PulsarSinkOptions.PULSAR_MESSAGE_KEY_HASH` option to choose the hash method. + +- `RoundRobinRouter`: Round-robin among all the partitions. + + All messages are sent to the first partition, and switch to the next partition after sending + a fixed number of messages. The batch size can be customized by the `PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES` option. + +Let’s assume there are ten messages and two topics. Topic A has two partitions while topic B has three partitions. +The batch size is set to five messages. In this case, topic A has 5 messages per partition which topic B does not receive any messages. + +You can configure custom routers by using the `TopicRouter` interface. +If you implement a `TopicRouter`, ensure that it is serializable. +And you can return partitions which are not available in the pre-discovered partition list. + +Thus, you do not need to specify topics using the `PulsarSinkBuilder.setTopics` option when you implement the custom topic router. + +```java +@PublicEvolving +public interface TopicRouter extends Serializable { + + String route(IN in, List partitions, PulsarSinkContext context); + + default void open(SinkConfiguration sinkConfiguration) { + // Nothing to do by default. + } +} +``` + +{{< hint info >}} +Internally, a Pulsar partition is implemented as a topic. The Pulsar client provides APIs to hide this +implementation detail and handles routing under the hood automatically. Pulsar Sink uses a lower client +API to implement its own routing layer to support multiple topics routing. + +For details, see [partitioned topics](https://pulsar.apache.org/docs/en/cookbooks-partitioned/). +{{< /hint >}} + +### Delivery Guarantee + +`PulsarSink` supports three delivery guarantee semantics. + +- `NONE`: Data loss can happen even when the pipeline is running. + Basically, we use a fire-and-forget strategy to send records to Pulsar topics in this mode. + It means that this mode has the highest throughput. +- `AT_LEAST_ONCE`: No data loss happens, but data duplication can happen after a restart from checkpoint. +- `EXACTLY_ONCE`: No data loss happens. Each record is sent to the Pulsar broker only once. + Pulsar Sink uses [Pulsar transaction](https://pulsar.apache.org/docs/en/transactions/) + and two-phase commit (2PC) to ensure records are sent only once even after pipeline restarts. + +### Delayed message delivery + +[Delayed message delivery](https://pulsar.apache.org/docs/en/next/concepts-messaging/#delayed-message-delivery) +enables you to delay the possibility to consume a message. With delayed message enabled, the Pulsar sink sends a message to the Pulsar topic +**immediately**, but the message is delivered to a consumer once the specified delay is over. + +Delayed message delivery only works in the `Shared` subscription type. In `Exclusive` and `Failover` +subscription types, the delayed message is dispatched immediately. + +You can configure the `MessageDelayer` to define when to send the message to the consumer. +The default option is to never delay the message dispatching. You can use the `MessageDelayer.fixed(Duration)` option to +Configure delaying all messages in a fixed duration. You can also implement the `MessageDelayer` +interface to dispatch messages at different time. + +{{< hint warning >}} +The dispatch time should be calculated by the `PulsarSinkContext.processTime()`. +{{< /hint >}} + +### Sink Configurable Options + +You can set options for `PulsarClient`, `PulsarAdmin`, Pulsar `Producer` and `PulsarSink` +by using `setConfig(ConfigOption, T)`, `setConfig(Configuration)` and `setConfig(Properties)`. + +#### PulsarClient and PulsarAdmin Options + +For details, refer to [PulsarAdmin options](#pulsaradmin-options). + +#### Pulsar Producer Options + +The Pulsar connector uses the Producer API to send messages. It extracts most parts of +Pulsar's `ProducerConfigurationData` as Flink configuration options in `PulsarSinkOptions`. + +{{< generated/pulsar_producer_configuration >}} + +#### PulsarSink Options + +The configuration options below are mainly used for customizing the performance and message +sending behavior. You can just leave them alone if you do not have any performance issues. + +{{< generated/pulsar_sink_configuration >}} + +### Sink Metrics + +This table lists supported metrics. +The first 6 metrics are standard Pulsar Sink metrics as described in +[FLIP-33: Standardize Connector Metrics]([https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics](https://cwiki.apache.org/confluence/display/FLINK/FLIP-33%3A+Standardize+Connector+Metrics)) + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ScopeMetricsUser VariablesDescriptionType
OperatornumBytesOutn/aThe total number of output bytes since the sink starts. Count towards the numBytesOut in TaskIOMetricsGroup.Counter
numBytesOutPerSecondn/aThe output bytes per secondMeter
numRecordsOutn/aThe total number of output records since the sink starts.Counter
numRecordsOutPerSecondn/aThe output records per secondMeter
numRecordsOutErrorsn/aThe total number of records failed to sendCounter
currentSendTimen/aThe time it takes to send the last record, from enqueue the message in client buffer to its ack.Gauge
PulsarSink.numAcksReceivedn/aThe number of acks received for sent messages.Counter
PulsarSink.sendLatencyMaxn/aThe maximum send latency in the last refresh interval across all producers.Gauge
PulsarSink.producer."ProducerName".sendLatency50PctProducerNameThe 50th percentile of send latency in the last refresh interval for a specific producer.Gauge
PulsarSink.producer."ProducerName".sendLatency75PctProducerNameThe 75th percentile of send latency in the last refresh interval for a specific producer.Gauge
PulsarSink.producer."ProducerName".sendLatency95PctProducerNameThe 95th percentile of send latency in the last refresh interval for a specific producer.Gauge
PulsarSink.producer."ProducerName".sendLatency99PctProducerNameThe 99th percentile of send latency in the last refresh interval for a specific producer.Gauge
PulsarSink.producer."ProducerName".sendLatency999PctProducerNameThe 99.9th percentile of send latency in the last refresh interval for a specific producer.Gauge
+ +{{< hint info >}} +- `numBytesOut`, `numRecordsOut`, `numRecordsOutErrors` are retrieved from Pulsar client metrics. + +- `currentSendTime` tracks the time from when the producer calls `sendAync()` to + the time when the message is acknowledged by the broker. This metric is not available in `NONE` delivery guarantee. +{{< /hint >}} + +The Pulsar producer refreshes its stats every 60 seconds by default. The PulsarSink retrieves the Pulsar producer +stats every 500ms. That means that `numRecordsOut`, `numBytesOut`, `numAcksReceived`, and `numRecordsOutErrors` +are updated every 60 seconds. To increase the metrics refresh frequency, you can change +the Pulsar producer stats refresh interval to a smaller value (minimum 1 second), as shown below. + +```java +builder.setConfig(PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS. 1L) +``` + +`numBytesOutRate` and `numRecordsOutRate` are calculated based on the `numBytesOut` and `numRecordsOUt` +counter respectively. Flink internally uses a fixed 60 seconds window to calculate the rates. + +### Brief Design Rationale + +Pulsar sink follow the Sink API defined in +[FLIP-191](https://cwiki.apache.org/confluence/display/FLINK/FLIP-191%3A+Extend+unified+Sink+interface+to+support+small+file+compaction). + +#### Stateless SinkWriter + +In `EXACTLY_ONCE` mode, the Pulsar sink does not store transaction information in a checkpoint. +That means that new transactions will be created after a restart. +Therefore, any message in previous pending transactions is either aborted or timed out +(They are never visible to the downstream Pulsar consumer). +The Pulsar team is working to optimize the needed resources by unfinished pending transactions. + +#### Pulsar Schema Evolution + +[Pulsar Schema Evolution](https://pulsar.apache.org/docs/en/schema-evolution-compatibility/) allows +you to reuse the same Flink job after certain "allowed" data model changes, like adding or deleting +a field in a AVRO-based Pojo class. Please note that you can specify Pulsar schema validation rules +and define an auto schema update. For details, refer to [Pulsar Schema Evolution](https://pulsar.apache.org/docs/en/schema-evolution-compatibility/). + +## Known Issues + +This section describes some known issues about the Pulsar connectors. + +### Unstable on Java 11 + +Pulsar connector has some known issues on Java 11. It is recommended to run Pulsar connector +on Java 8. + +### No TransactionCoordinatorNotFound, but automatic reconnect + +Pulsar transactions are still in active development and are not stable. Pulsar 2.9.2 +introduces [a break change](https://github.com/apache/pulsar/pull/13135) in transactions. +If you use Pulsar 2.9.2 or higher with an older Pulsar client, you might get a `TransactionCoordinatorNotFound` exception. + +You can use the latest `pulsar-client-all` release to resolve this issue. + ## Upgrading to the Latest Connector Version The generic upgrade steps are outlined in [upgrading jobs and Flink versions guide]({{< ref "docs/ops/upgrading" >}}). @@ -407,7 +788,7 @@ The Pulsar connector does not store any state on the Flink side. The Pulsar conn For Pulsar, you additionally need to know these limitations: * Do not upgrade the Pulsar connector and Pulsar broker version at the same time. -* Always use a newer Pulsar client with Pulsar connector for consuming message from Pulsar. +* Always use a newer Pulsar client with Pulsar connector to consume messages from Pulsar. ## Troubleshooting @@ -417,13 +798,4 @@ If you have a problem with Pulsar when using Flink, keep in mind that Flink only and your problem might be independent of Flink and sometimes can be solved by upgrading Pulsar brokers, reconfiguring Pulsar brokers or reconfiguring Pulsar connector in Flink. -### Messages can be delayed on low volume topics - -When the Pulsar source connector reads from a low volume topic, users might observe a 10 seconds delay between messages. Pulsar buffers messages from topics by default. Before emitting to downstream -operators, the number of buffered records must be equal or larger than `PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS`. If the data volume is low, it could be that filling up the number of buffered records takes longer than `PULSAR_MAX_FETCH_TIME` (default to 10 seconds). If that's the case, it means that only after this time has passed the messages will be emitted. - -To avoid this behaviour, you need to change either the buffered records or the waiting time. - - - {{< top >}} diff --git a/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html b/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html new file mode 100644 index 0000000000000..044c4b228d47e --- /dev/null +++ b/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html @@ -0,0 +1,90 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
pulsar.producer.batchingEnabled
trueBooleanEnable batch send ability, it was enabled by default.
pulsar.producer.batchingMaxBytes
131072IntegerThe maximum size of messages permitted in a batch. Keep the maximum consistent as previous versions.
pulsar.producer.batchingMaxMessages
1000IntegerThe maximum number of messages permitted in a batch.
pulsar.producer.batchingMaxPublishDelayMicros
1000LongBatching time period of sending messages.
pulsar.producer.batchingPartitionSwitchFrequencyByPublishDelay
10IntegerThe maximum wait time for switching topic partitions.
pulsar.producer.chunkingEnabled
falseBoolean
pulsar.producer.compressionType
NONE

Enum

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

Possible values:
  • "NONE"
  • "LZ4"
  • "ZLIB"
  • "ZSTD"
  • "SNAPPY"
pulsar.producer.initialSequenceId
(none)LongThe sequence id for avoiding the duplication, it's used when Pulsar doesn't have transaction.
pulsar.producer.maxPendingMessages
1000IntegerThe maximum size of a queue holding pending messages.
For example, a message waiting to receive an acknowledgment from a https://pulsar.apache.org/docs/en/reference-terminology#broker.
By default, when the queue is full, all calls to the Send and SendAsync methods fail unless you set BlockIfQueueFull to true.
pulsar.producer.maxPendingMessagesAcrossPartitions
50000IntegerThe maximum number of pending messages across partitions.
Use the setting to lower the max pending messages for each partition (setMaxPendingMessages) if the total number exceeds the configured value.
pulsar.producer.producerName
(none)StringA producer name which would be displayed in the Pulsar's dashboard. If no producer name was provided, we would use a Pulsar generated name instead.
pulsar.producer.properties
MapA name or value property of this consumer. properties is application defined metadata attached to a consumer. When getting a topic stats, associate this metadata with the consumer stats for easier identification.
pulsar.producer.sendTimeoutMs
30000LongMessage send timeout in ms.If a message is not acknowledged by a server before the sendTimeout expires, an error occurs.
diff --git a/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html b/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html new file mode 100644 index 0000000000000..cd7f803534a1e --- /dev/null +++ b/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html @@ -0,0 +1,48 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
KeyDefaultTypeDescription
pulsar.sink.deliveryGuarantee
none

Enum

Optional delivery guarantee when committing.

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

Enum

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

Possible values:
  • "java-hash": This hash would use String.hashCode() to calculate the message key string's hash code.
  • "murmur-3-32-hash": This hash would calculate message key's hash code by using Murmur3 algorithm.
pulsar.sink.topicMetadataRefreshInterval
1800000LongAuto update the topic metadata in a fixed interval (in ms). The default value is 30 minutes.
pulsar.sink.transactionTimeoutMillis
10800000LongThis option is used when the user require the DeliveryGuarantee.EXACTLY_ONCE semantic.We would use transaction for making sure the message could be write only once.
diff --git a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java index 4602cb60af5ee..38b5121f4f45e 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java @@ -104,6 +104,9 @@ public class ConfigOptionsDocGenerator { new OptionsClassLocation( "flink-connectors/flink-connector-pulsar", "org.apache.flink.connector.pulsar.source"), + new OptionsClassLocation( + "flink-connectors/flink-connector-pulsar", + "org.apache.flink.connector.pulsar.sink"), new OptionsClassLocation( "flink-libraries/flink-cep", "org.apache.flink.cep.configuration"), new OptionsClassLocation( From 34afc9b7434fec1c9895a2a3ed52cce3830b2676 Mon Sep 17 00:00:00 2001 From: huweihua Date: Wed, 16 Mar 2022 09:49:36 +0800 Subject: [PATCH 119/258] [FLINK-26395][docs] Correct the wrong description of SQL function: RAND_INTEGER. This fixes #19257 --- docs/data/sql_functions.yml | 4 ++-- docs/data/sql_functions_zh.yml | 4 ++-- flink-python/pyflink/table/expressions.py | 2 +- .../src/main/java/org/apache/flink/table/api/Expressions.java | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/data/sql_functions.yml b/docs/data/sql_functions.yml index 80c4edf409596..8c01be4d5753c 100644 --- a/docs/data/sql_functions.yml +++ b/docs/data/sql_functions.yml @@ -221,10 +221,10 @@ arithmetic: description: Returns a pseudorandom double value in the range [0.0, 1.0) with an initial seed integer. Two RAND functions will return identical sequences of numbers if they have the same initial seed. - sql: RAND_INTEGER(INT) table: randInteger(INT) - description: Returns a pseudorandom double value in the range [0.0, INT) + description: Returns a pseudorandom integer value in the range [0, INT) - sql: RAND_INTEGER(INT1, INT2) table: randInteger(INT1, INT2) - description: Returns a pseudorandom double value in the range [0.0, INT2) with an initial seed INT1. Two RAND_INTGER functions will return idential sequences of numbers if they have the same initial seed and bound. + description: Returns a pseudorandom integer value in the range [0, INT2) with an initial seed INT1. Two RAND_INTGER functions will return idential sequences of numbers if they have the same initial seed and bound. - sql: UUID() table: uuid() description: Returns an UUID (Universally Unique Identifier) string (e.g., "3d3c68f7-f608-473f-b60c-b0c44ad4cc4e") according to RFC 4122 type 4 (pseudo randomly generated) UUID. The UUID is generated using a cryptographically strong pseudo random number generator. diff --git a/docs/data/sql_functions_zh.yml b/docs/data/sql_functions_zh.yml index f5c2a7992dba1..8fc1e4d663d27 100644 --- a/docs/data/sql_functions_zh.yml +++ b/docs/data/sql_functions_zh.yml @@ -271,11 +271,11 @@ arithmetic: 如果两个 RAND 函数具有相同的初始种子,它们将返回相同的数字序列。 - sql: RAND_INTEGER(INT) table: randInteger(INT) - description: 返回 [0.0, INT) 范围内的伪随机双精度值。 + description: 返回 [0, INT) 范围内的伪随机整数。 - sql: RAND_INTEGER(INT1, INT2) table: randInteger(INT1, INT2) description: | - 返回范围为 [0.0, INT2) 的伪随机双精度值,初始种子为 INT1。 + 返回范围为 [0, INT2) 的伪随机整数,初始种子为 INT1。 如果两个 RAND_INTGER 函数具有相同的初始种子和边界,它们将返回相同的数字序列。 - sql: UUID() table: uuid() diff --git a/flink-python/pyflink/table/expressions.py b/flink-python/pyflink/table/expressions.py index 2467db3645cdc..d870f825e20da 100644 --- a/flink-python/pyflink/table/expressions.py +++ b/flink-python/pyflink/table/expressions.py @@ -438,7 +438,7 @@ def rand(seed: Union[int, Expression[int]] = None) -> Expression[float]: def rand_integer(bound: Union[int, Expression[int]], seed: Union[int, Expression[int]] = None) -> Expression: """ - Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value + Returns a pseudorandom integer value between 0 (inclusive) and the specified value (exclusive) with a initial seed if specified. Two rand_integer() functions will return identical sequences of numbers if they have same initial seed and same bound. """ diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java index 2096f296286cf..b102fbcc91582 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Expressions.java @@ -444,7 +444,7 @@ public static ApiExpression rand(Object seed) { } /** - * Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value + * Returns a pseudorandom integer value between 0 (inclusive) and the specified value * (exclusive). */ public static ApiExpression randInteger(Object bound) { @@ -452,7 +452,7 @@ public static ApiExpression randInteger(Object bound) { } /** - * Returns a pseudorandom integer value between 0.0 (inclusive) and the specified value + * Returns a pseudorandom integer value between 0 (inclusive) and the specified value * (exclusive) with a initial seed. Two randInteger() functions will return identical sequences * of numbers if they have same initial seed and same bound. */ From 4e87586dfb0a7c208440d331751719be0be2ccaa Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Mon, 28 Mar 2022 12:59:42 +0200 Subject: [PATCH 120/258] [FLINK-26794][tests] Use API to access job checkpoints in IT cases Using dedicated API has the following advantages over looking for checkpoints on FS: - less prone to NoSuchFileException - less brittle because file layout is not a declared API - less IO usage --- .../runtime/testutils/CommonTestUtils.java | 35 +++++++++++++++++++ .../state/ChangelogCompatibilityITCase.java | 20 ++++++++--- .../test/state/ChangelogRescalingITCase.java | 30 ++++++++-------- .../org/apache/flink/test/util/TestUtils.java | 13 +++++++ 4 files changed, 77 insertions(+), 21 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java index 7e01ee4a5d9b6..678eb2bac1f7b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java @@ -22,11 +22,13 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Deadline; import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; import org.apache.flink.runtime.executiongraph.AccessExecutionVertex; import org.apache.flink.runtime.executiongraph.ErrorInfo; import org.apache.flink.runtime.jobgraph.JobVertexID; +import org.apache.flink.runtime.messages.FlinkJobNotFoundException; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo; import org.apache.flink.util.FileUtils; @@ -47,6 +49,8 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.function.Predicate; @@ -359,6 +363,37 @@ public static void waitForSubtasksToFinish( Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES))); } + /** Wait for at least one successful checkpoint. */ + public static void waitForCheckpoint(JobID jobID, MiniCluster miniCluster, Deadline timeout) + throws Exception, FlinkJobNotFoundException { + waitUntilCondition( + () -> + Optional.ofNullable( + miniCluster + .getExecutionGraph(jobID) + .get() + .getCheckpointStatsSnapshot()) + .filter(st -> st.getCounts().getNumberOfCompletedCheckpoints() > 0) + .isPresent(), + timeout); + } + + /** + * @return the path as {@link java.net.URI} to the latest checkpoint. + * @throws FlinkJobNotFoundException if job not found + */ + public static Optional getLatestCompletedCheckpointPath( + JobID jobID, MiniCluster cluster) + throws InterruptedException, ExecutionException, FlinkJobNotFoundException { + return Optional.ofNullable( + cluster.getExecutionGraph(jobID).get().getCheckpointStatsSnapshot()) + .flatMap( + stats -> + Optional.ofNullable( + stats.getHistory().getLatestCompletedCheckpoint())) + .map(CompletedCheckpointStats::getExternalPath); + } + /** Utility class to read the output of a process stream and forward it into a StringWriter. */ public static class PipeForwarder extends Thread { diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java index 3023e996271b2..ef4052c76a980 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java @@ -18,11 +18,13 @@ package org.apache.flink.test.state; import org.apache.flink.api.common.functions.ReduceFunction; +import org.apache.flink.api.common.time.Deadline; import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; @@ -37,16 +39,18 @@ import org.junit.runners.Parameterized; import java.io.File; +import java.time.Duration; import java.util.Arrays; import java.util.List; +import java.util.NoSuchElementException; import java.util.Optional; import static org.apache.flink.configuration.CheckpointingOptions.CHECKPOINTS_DIRECTORY; import static org.apache.flink.configuration.CheckpointingOptions.SAVEPOINT_DIRECTORY; import static org.apache.flink.runtime.jobgraph.SavepointRestoreSettings.forPath; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForCheckpoint; import static org.apache.flink.streaming.api.environment.CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION; -import static org.apache.flink.test.util.TestUtils.getMostRecentCompletedCheckpointMaybe; import static org.apache.flink.util.ExceptionUtils.findThrowableSerializedAware; import static org.junit.Assert.fail; @@ -141,12 +145,18 @@ private String tryCheckpointAndStop(JobGraph jobGraph) throws Exception { ClusterClient client = miniClusterResource.getClusterClient(); submit(jobGraph, client); if (testCase.restoreSource == RestoreSource.CHECKPOINT) { - while (!getMostRecentCompletedCheckpointMaybe(checkpointDir).isPresent()) { - Thread.sleep(50); - } + waitForCheckpoint( + jobGraph.getJobID(), + miniClusterResource.getMiniCluster(), + Deadline.fromNow(Duration.ofMinutes(5))); client.cancel(jobGraph.getJobID()).get(); // obtain the latest checkpoint *after* cancellation - that one won't be subsumed - return pathToString(getMostRecentCompletedCheckpointMaybe(checkpointDir).get()); + return CommonTestUtils.getLatestCompletedCheckpointPath( + jobGraph.getJobID(), miniClusterResource.getMiniCluster()) + .orElseThrow( + () -> { + throw new NoSuchElementException("No checkpoint was created yet"); + }); } else { return client.stopWithSavepoint( jobGraph.getJobID(), diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java index 85a30c8d76616..b3f70047537da 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; @@ -31,6 +32,7 @@ import org.apache.flink.configuration.MemorySize; import org.apache.flink.core.io.InputStatus; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.datastream.DataStreamUtils; @@ -55,10 +57,10 @@ import org.junit.runners.Parameterized.Parameters; import java.io.File; -import java.io.IOException; import java.io.Serializable; import java.time.Duration; import java.util.Iterator; +import java.util.NoSuchElementException; import java.util.Timer; import java.util.TimerTask; import java.util.concurrent.ExecutionException; @@ -80,14 +82,13 @@ import static org.apache.flink.configuration.TaskManagerOptions.BUFFER_DEBLOAT_ENABLED; import static org.apache.flink.runtime.jobgraph.SavepointRestoreSettings.forPath; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForCheckpoint; import static org.apache.flink.streaming.api.environment.CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION; import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.ALIGNED_CHECKPOINT_TIMEOUT; import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL; import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_MODE; import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.ENABLE_UNALIGNED; import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.EXTERNALIZED_CHECKPOINT; -import static org.apache.flink.test.util.TestUtils.getMostRecentCompletedCheckpoint; -import static org.apache.flink.test.util.TestUtils.getMostRecentCompletedCheckpointMaybe; import static org.apache.flink.util.Preconditions.checkArgument; /** @@ -149,19 +150,16 @@ public void after() { @Test public void test() throws Exception { // before rescale - File cpDir1 = temporaryFolder.newFolder(); - JobID jobID1 = submit(configureJob(parallelism1, cpDir1), graph -> {}); + JobID jobID1 = submit(configureJob(parallelism1, temporaryFolder.newFolder()), graph -> {}); Thread.sleep(ACCUMULATE_TIME_MILLIS); - File cpLocation = checkpointAndCancel(jobID1, cpDir1); + String cpLocation = checkpointAndCancel(jobID1); // rescale and checkpoint to verify JobID jobID2 = submit( configureJob(parallelism2, temporaryFolder.newFolder()), - graph -> - graph.setSavepointRestoreSettings( - forPath(cpLocation.toURI().toString()))); + graph -> graph.setSavepointRestoreSettings(forPath(cpLocation))); waitForAllTaskRunning(cluster.getMiniCluster(), jobID2, true); cluster.getClusterClient().cancel(jobID2).get(); } @@ -328,15 +326,15 @@ public boolean request() { } } - private File checkpointAndCancel(JobID jobID, File cpDir) - throws IOException, InterruptedException, ExecutionException { - while (!getMostRecentCompletedCheckpointMaybe(cpDir).isPresent()) { - checkStatus(jobID); - Thread.sleep(50); - } + private String checkpointAndCancel(JobID jobID) throws Exception { + waitForCheckpoint(jobID, cluster.getMiniCluster(), Deadline.fromNow(Duration.ofMinutes(5))); cluster.getClusterClient().cancel(jobID).get(); checkStatus(jobID); - return getMostRecentCompletedCheckpoint(cpDir); + return CommonTestUtils.getLatestCompletedCheckpointPath(jobID, cluster.getMiniCluster()) + .orElseThrow( + () -> { + throw new NoSuchElementException("No checkpoint was created yet"); + }); } private void checkStatus(JobID jobID) throws InterruptedException, ExecutionException { diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java index 8cbf3e32275a8..033f1780a29cf 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.checkpoint.metadata.CheckpointMetadata; import org.apache.flink.runtime.client.JobInitializationException; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -113,11 +114,23 @@ public static CheckpointMetadata loadCheckpointMetadata(String savepointPath) } } + /** + * @deprecated please use {@link + * org.apache.flink.runtime.testutils.CommonTestUtils#getLatestCompletedCheckpointPath(JobID, + * MiniCluster)} which is less prone to {@link NoSuchFileException} and IO-intensive. + */ + @Deprecated public static File getMostRecentCompletedCheckpoint(File checkpointDir) throws IOException { return getMostRecentCompletedCheckpointMaybe(checkpointDir) .orElseThrow(() -> new IllegalStateException("Cannot generate checkpoint")); } + /** + * @deprecated please use {@link + * org.apache.flink.runtime.testutils.CommonTestUtils#getLatestCompletedCheckpointPath(JobID, + * MiniCluster)} which is less prone to {@link NoSuchFileException} and IO-intensive. + */ + @Deprecated public static Optional getMostRecentCompletedCheckpointMaybe(File checkpointDir) throws IOException { return Files.find(checkpointDir.toPath(), 2, TestUtils::isCompletedCheckpoint) From f59404fe646590abcfbb4a512ea6762f43108646 Mon Sep 17 00:00:00 2001 From: Hangxiang Yu Date: Mon, 21 Mar 2022 11:22:54 +0800 Subject: [PATCH 121/258] [FLINK-26673][changelog] Disable periodic materialization when periodicMaterializeDelay is negative --- .../shortcodes/generated/state_backend_changelog_section.html | 2 +- .../shortcodes/generated/state_changelog_configuration.html | 2 +- .../org/apache/flink/configuration/StateChangelogOptions.java | 3 ++- .../flink/state/changelog/PeriodicMaterializationManager.java | 3 ++- .../checkpointing/ChangelogPeriodicMaterializationITCase.java | 2 +- 5 files changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/layouts/shortcodes/generated/state_backend_changelog_section.html b/docs/layouts/shortcodes/generated/state_backend_changelog_section.html index 405133dee789a..a5b41d308c349 100644 --- a/docs/layouts/shortcodes/generated/state_backend_changelog_section.html +++ b/docs/layouts/shortcodes/generated/state_backend_changelog_section.html @@ -24,7 +24,7 @@

state.backend.changelog.periodic-materialize.interval
10 min Duration - Defines the interval in milliseconds to perform periodic materialization for state backend. + Defines the interval in milliseconds to perform periodic materialization for state backend. The periodic materialization will be disabled when the value is negative
state.backend.changelog.storage
diff --git a/docs/layouts/shortcodes/generated/state_changelog_configuration.html b/docs/layouts/shortcodes/generated/state_changelog_configuration.html index 405133dee789a..a5b41d308c349 100644 --- a/docs/layouts/shortcodes/generated/state_changelog_configuration.html +++ b/docs/layouts/shortcodes/generated/state_changelog_configuration.html @@ -24,7 +24,7 @@
state.backend.changelog.periodic-materialize.interval
10 min Duration - Defines the interval in milliseconds to perform periodic materialization for state backend. + Defines the interval in milliseconds to perform periodic materialization for state backend. The periodic materialization will be disabled when the value is negative
state.backend.changelog.storage
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/StateChangelogOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/StateChangelogOptions.java index 588e6707b7422..ef2fc62794da0 100644 --- a/flink-core/src/main/java/org/apache/flink/configuration/StateChangelogOptions.java +++ b/flink-core/src/main/java/org/apache/flink/configuration/StateChangelogOptions.java @@ -34,7 +34,8 @@ public class StateChangelogOptions { .defaultValue(Duration.ofMinutes(10)) .withDescription( "Defines the interval in milliseconds to perform " - + "periodic materialization for state backend."); + + "periodic materialization for state backend. " + + "The periodic materialization will be disabled when the value is negative"); @Documentation.Section(Documentation.Sections.STATE_BACKEND_CHANGELOG) public static final ConfigOption MATERIALIZATION_MAX_FAILURES_ALLOWED = diff --git a/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java b/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java index e20c1090699ca..d51d7958cd3f4 100644 --- a/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java +++ b/flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/PeriodicMaterializationManager.java @@ -107,7 +107,8 @@ class PeriodicMaterializationManager implements Closeable { } public void start() { - if (!started) { + // disable periodic materialization when periodicMaterializeDelay is negative + if (!started && periodicMaterializeDelay >= 0) { started = true; diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationITCase.java index f8e063b420b39..fb4f8752562c8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogPeriodicMaterializationITCase.java @@ -49,7 +49,7 @@ public void testNonMaterialization() throws Exception { SharedReference hasMaterialization = sharedObjects.add(new AtomicBoolean(true)); StreamExecutionEnvironment env = - getEnv(delegatedStateBackend, checkpointFolder, 1000, 1, Long.MAX_VALUE, 0); + getEnv(delegatedStateBackend, checkpointFolder, 1000, 1, -1, 0); waitAndAssert( buildJobGraph( env, From b12f8efad939c1654142b893d8f630646956387e Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Sun, 27 Mar 2022 11:29:12 +0300 Subject: [PATCH 122/258] [FLINK-26092][table-runtime] Fix `JSON_OBJECTAGG` when emitting `NULL` Previously, when the Json aggregation is taking place, and id JsonOnNull.NULL is selected, which means that we still want to emit a `null` JSON node, .i.e `{.... "myField" : null ... }` when no values get accumulated, we used a null `StringData` object. When `state.backend.changelog.enabled` is enabled, the contents of the map accumulating the aggregated records, gets serialized leading to NPE, since `null` is not supported by `StringDataSerilizer`. To solve this, we instead create a StringData with an empty `byte[]`, which denotes the null, and when the aggregation ends and we create the final JSON result, we check for a `byte[]` of `length` `0` in order to write the JSON `null` node. --- .../BuiltInAggregateFunctionTestBase.java | 16 +- .../JsonAggregationFunctionsITCase.java | 272 ++++++++++-------- .../aggregate/JsonObjectAggFunction.java | 7 +- 3 files changed, 169 insertions(+), 126 deletions(-) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java index ba2f01ef21f29..fc65b91a67c2a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java @@ -18,7 +18,7 @@ package org.apache.flink.table.planner.functions; -import org.apache.flink.configuration.StateChangelogOptions; +import org.apache.flink.configuration.StateBackendOptions; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.table.api.EnvironmentSettings; @@ -78,8 +78,7 @@ public class BuiltInAggregateFunctionTestBase { public void testFunction() throws Exception { final TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode()); - // see https://issues.apache.org/jira/browse/FLINK-26092 - tEnv.getConfig().set(StateChangelogOptions.ENABLE_STATE_CHANGE_LOG, false); + tEnv.getConfig().set(StateBackendOptions.STATE_BACKEND, testSpec.backendName); final Table sourceTable = asTable(tEnv, testSpec.sourceRowType, testSpec.sourceRows); for (final TestItem testItem : testSpec.testItems) { @@ -175,6 +174,7 @@ protected static class TestSpec { private DataType sourceRowType; private List sourceRows; + private String backendName; private TestSpec(BuiltInFunctionDefinition definition) { this.definition = Preconditions.checkNotNull(definition); @@ -195,6 +195,15 @@ TestSpec withSource(DataType sourceRowType, List sourceRows) { return this; } + public TestSpec withStateBackend(String backendName) { + this.backendName = backendName; + return this; + } + + public TestSpec copy() { + return new TestSpec(definition).withSource(sourceRowType, new ArrayList<>(sourceRows)); + } + TestSpec testSqlResult( Function sqlSpec, DataType expectedRowType, List expectedRows) { this.testItems.add(new SqlTestItem(sqlSpec, expectedRowType, expectedRows)); @@ -238,6 +247,7 @@ public String toString() { bob.append(description); bob.append(")"); } + bob.append(", backend: ").append(backendName); return bob.toString(); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonAggregationFunctionsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonAggregationFunctionsITCase.java index 9209a496505bf..b0ffd6cc5b4a2 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonAggregationFunctionsITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonAggregationFunctionsITCase.java @@ -27,7 +27,11 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.flink.runtime.state.StateBackendLoader.HASHMAP_STATE_BACKEND_NAME; +import static org.apache.flink.runtime.state.StateBackendLoader.ROCKSDB_STATE_BACKEND_NAME; import static org.apache.flink.table.api.DataTypes.INT; import static org.apache.flink.table.api.DataTypes.ROW; import static org.apache.flink.table.api.DataTypes.STRING; @@ -43,128 +47,154 @@ public class JsonAggregationFunctionsITCase extends BuiltInAggregateFunctionTest @Parameterized.Parameters(name = "{index}: {0}") public static List testData() throws Exception { - return Arrays.asList( - // JSON_OBJECTAGG - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) - .withDescription("Basic Aggregation") - .withSource( - ROW(STRING(), INT()), - Arrays.asList( - Row.ofKind(INSERT, "A", 1), - Row.ofKind(INSERT, "B", null), - Row.ofKind(INSERT, "C", 3))) - .testResult( - source -> "SELECT JSON_OBJECTAGG(f0 VALUE f1) FROM " + source, - source -> - source.select( - jsonObjectAgg(JsonOnNull.NULL, $("f0"), $("f1"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("{\"A\":1,\"B\":null,\"C\":3}"))), - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_ABSENT_ON_NULL) - .withDescription("Omits NULLs") - .withSource( - ROW(STRING(), INT()), - Arrays.asList( - Row.ofKind(INSERT, "A", 1), - Row.ofKind(INSERT, "B", null), - Row.ofKind(INSERT, "C", 3))) - .testResult( - source -> - "SELECT JSON_OBJECTAGG(f0 VALUE f1 ABSENT ON NULL) FROM " - + source, - source -> - source.select( - jsonObjectAgg(JsonOnNull.ABSENT, $("f0"), $("f1"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("{\"A\":1,\"C\":3}"))), - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) - .withDescription("Retractions") - .withSource( - ROW(STRING(), INT()), - Arrays.asList( - Row.ofKind(INSERT, "A", 1), - Row.ofKind(INSERT, "B", 2), - Row.ofKind(INSERT, "C", 3), - Row.ofKind(DELETE, "B", 2))) - .testResult( - source -> "SELECT JSON_OBJECTAGG(f0 VALUE f1) FROM " + source, - source -> - source.select( - jsonObjectAgg(JsonOnNull.NULL, $("f0"), $("f1"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("{\"A\":1,\"C\":3}"))), - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) - .withDescription("Group Aggregation") - .withSource( - ROW(INT(), STRING(), INT()), - Arrays.asList( - Row.ofKind(INSERT, 1, "A", 0), - Row.ofKind(INSERT, 1, "B", 0), - Row.ofKind(INSERT, 2, "A", 0), - Row.ofKind(INSERT, 2, "C", 0))) - .testResult( - source -> - "SELECT f0, JSON_OBJECTAGG(f1 VALUE f2) FROM " - + source - + " GROUP BY f0", - source -> - source.groupBy($("f0")) - .select( - $("f0"), + return Stream.of( + // JSON_OBJECTAGG + TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) + .withDescription("Basic Aggregation") + .withSource( + ROW(STRING(), INT()), + Arrays.asList( + Row.ofKind(INSERT, "A", 1), + Row.ofKind(INSERT, "B", null), + Row.ofKind(INSERT, "C", 3))) + .testResult( + source -> + "SELECT JSON_OBJECTAGG(f0 VALUE f1) FROM " + source, + source -> + source.select( jsonObjectAgg( - JsonOnNull.NULL, $("f1"), $("f2"))), - ROW(INT(), VARCHAR(2000).notNull()), - ROW(INT(), STRING().notNull()), - Arrays.asList( - Row.of(1, "{\"A\":0,\"B\":0}"), - Row.of(2, "{\"A\":0,\"C\":0}"))), + JsonOnNull.NULL, $("f0"), $("f1"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList( + Row.of("{\"A\":1,\"B\":null,\"C\":3}"))), + TestSpec.forFunction( + BuiltInFunctionDefinitions.JSON_OBJECTAGG_ABSENT_ON_NULL) + .withDescription("Omits NULLs") + .withSource( + ROW(STRING(), INT()), + Arrays.asList( + Row.ofKind(INSERT, "A", 1), + Row.ofKind(INSERT, "B", null), + Row.ofKind(INSERT, "C", 3))) + .testResult( + source -> + "SELECT JSON_OBJECTAGG(f0 VALUE f1 ABSENT ON NULL) FROM " + + source, + source -> + source.select( + jsonObjectAgg( + JsonOnNull.ABSENT, + $("f0"), + $("f1"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList(Row.of("{\"A\":1,\"C\":3}"))), + TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) + .withDescription("Retractions") + .withSource( + ROW(STRING(), INT()), + Arrays.asList( + Row.ofKind(INSERT, "A", 1), + Row.ofKind(INSERT, "B", 2), + Row.ofKind(INSERT, "C", 3), + Row.ofKind(DELETE, "B", 2))) + .testResult( + source -> + "SELECT JSON_OBJECTAGG(f0 VALUE f1) FROM " + source, + source -> + source.select( + jsonObjectAgg( + JsonOnNull.NULL, $("f0"), $("f1"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList(Row.of("{\"A\":1,\"C\":3}"))), + TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_OBJECTAGG_NULL_ON_NULL) + .withDescription("Group Aggregation") + .withSource( + ROW(INT(), STRING(), INT()), + Arrays.asList( + Row.ofKind(INSERT, 1, "A", 0), + Row.ofKind(INSERT, 1, "B", 0), + Row.ofKind(INSERT, 2, "A", 0), + Row.ofKind(INSERT, 2, "C", 0))) + .testResult( + source -> + "SELECT f0, JSON_OBJECTAGG(f1 VALUE f2) FROM " + + source + + " GROUP BY f0", + source -> + source.groupBy($("f0")) + .select( + $("f0"), + jsonObjectAgg( + JsonOnNull.NULL, + $("f1"), + $("f2"))), + ROW(INT(), VARCHAR(2000).notNull()), + ROW(INT(), STRING().notNull()), + Arrays.asList( + Row.of(1, "{\"A\":0,\"B\":0}"), + Row.of(2, "{\"A\":0,\"C\":0}"))), - // JSON_ARRAYAGG - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_ARRAYAGG_ABSENT_ON_NULL) - .withDescription("Basic Aggregation") - .withSource( - ROW(STRING()), - Arrays.asList( - Row.ofKind(INSERT, "A"), - Row.ofKind(INSERT, (String) null), - Row.ofKind(INSERT, "C"))) - .testResult( - source -> "SELECT JSON_ARRAYAGG(f0) FROM " + source, - source -> source.select(jsonArrayAgg(JsonOnNull.ABSENT, $("f0"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("[\"A\",\"C\"]"))), - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_ARRAYAGG_NULL_ON_NULL) - .withDescription("Keeps NULLs") - .withSource( - ROW(STRING()), - Arrays.asList( - Row.ofKind(INSERT, "A"), - Row.ofKind(INSERT, (String) null), - Row.ofKind(INSERT, "C"))) - .testResult( - source -> "SELECT JSON_ARRAYAGG(f0 NULL ON NULL) FROM " + source, - source -> source.select(jsonArrayAgg(JsonOnNull.NULL, $("f0"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("[\"A\",null,\"C\"]"))), - TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_ARRAYAGG_ABSENT_ON_NULL) - .withDescription("Retractions") - .withSource( - ROW(INT()), - Arrays.asList( - Row.ofKind(INSERT, 1), - Row.ofKind(INSERT, 2), - Row.ofKind(INSERT, 3), - Row.ofKind(DELETE, 2))) - .testResult( - source -> "SELECT JSON_ARRAYAGG(f0) FROM " + source, - source -> source.select(jsonArrayAgg(JsonOnNull.ABSENT, $("f0"))), - ROW(VARCHAR(2000).notNull()), - ROW(STRING().notNull()), - Collections.singletonList(Row.of("[1,3]")))); + // JSON_ARRAYAGG + TestSpec.forFunction( + BuiltInFunctionDefinitions.JSON_ARRAYAGG_ABSENT_ON_NULL) + .withDescription("Basic Aggregation") + .withSource( + ROW(STRING()), + Arrays.asList( + Row.ofKind(INSERT, "A"), + Row.ofKind(INSERT, (String) null), + Row.ofKind(INSERT, "C"))) + .testResult( + source -> "SELECT JSON_ARRAYAGG(f0) FROM " + source, + source -> + source.select( + jsonArrayAgg(JsonOnNull.ABSENT, $("f0"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList(Row.of("[\"A\",\"C\"]"))), + TestSpec.forFunction(BuiltInFunctionDefinitions.JSON_ARRAYAGG_NULL_ON_NULL) + .withDescription("Keeps NULLs") + .withSource( + ROW(STRING()), + Arrays.asList( + Row.ofKind(INSERT, "A"), + Row.ofKind(INSERT, (String) null), + Row.ofKind(INSERT, "C"))) + .testResult( + source -> + "SELECT JSON_ARRAYAGG(f0 NULL ON NULL) FROM " + + source, + source -> + source.select( + jsonArrayAgg(JsonOnNull.NULL, $("f0"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList(Row.of("[\"A\",null,\"C\"]"))), + TestSpec.forFunction( + BuiltInFunctionDefinitions.JSON_ARRAYAGG_ABSENT_ON_NULL) + .withDescription("Retractions") + .withSource( + ROW(INT()), + Arrays.asList( + Row.ofKind(INSERT, 1), + Row.ofKind(INSERT, 2), + Row.ofKind(INSERT, 3), + Row.ofKind(DELETE, 2))) + .testResult( + source -> "SELECT JSON_ARRAYAGG(f0) FROM " + source, + source -> + source.select( + jsonArrayAgg(JsonOnNull.ABSENT, $("f0"))), + ROW(VARCHAR(2000).notNull()), + ROW(STRING().notNull()), + Collections.singletonList(Row.of("[1,3]")))) + .flatMap( + spec -> + Stream.of(HASHMAP_STATE_BACKEND_NAME, ROCKSDB_STATE_BACKEND_NAME) + .map(backend -> spec.copy().withStateBackend(backend))) + .collect(Collectors.toList()); } } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/JsonObjectAggFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/JsonObjectAggFunction.java index 8152f988fe32a..4be69c9259266 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/JsonObjectAggFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/aggregate/JsonObjectAggFunction.java @@ -57,6 +57,7 @@ public class JsonObjectAggFunction extends BuiltInAggregateFunction { private static final long serialVersionUID = 1L; + private static final StringData NULL_STRING_DATA = StringData.fromBytes(new byte[] {}); private static final NullNode NULL_NODE = getNodeFactory().nullNode(); private final transient List argumentTypes; @@ -107,7 +108,9 @@ public void accumulate(Accumulator acc, StringData keyData, @Nullable StringData if (valueData == null) { if (!skipNulls) { - acc.map.put(keyData, null); + // We cannot use null for StringData here, since it's not supported by the + // StringDataSerializer, instead use a StringData with an empty byte[] + acc.map.put(keyData, NULL_STRING_DATA); } } else { acc.map.put(keyData, valueData); @@ -135,7 +138,7 @@ public String getValue(Accumulator acc) { for (final StringData key : acc.map.keys()) { final StringData value = acc.map.get(key); final JsonNode valueNode = - value == null + value.toBytes().length == 0 ? NULL_NODE : getNodeFactory().rawValueNode(new RawValue(value.toString())); From b78aff52b00a43e6f9975a9b55847e1573f9f241 Mon Sep 17 00:00:00 2001 From: Anton Kalashnikov Date: Mon, 14 Feb 2022 17:26:58 +0100 Subject: [PATCH 123/258] [FLINK-26134][docs] Added documentation page with Checkpoint/Savepoint guarantees --- .../concepts/stateful-stream-processing.md | 5 +- docs/content.zh/docs/ops/state/checkpoints.md | 8 +- .../ops/state/checkpoints_vs_savepoints.md | 95 +++++++++++++++++++ docs/content.zh/docs/ops/state/savepoints.md | 8 +- .../concepts/stateful-stream-processing.md | 3 + docs/content/docs/ops/state/checkpoints.md | 10 +- .../ops/state/checkpoints_vs_savepoints.md | 95 +++++++++++++++++++ docs/content/docs/ops/state/savepoints.md | 11 +-- 8 files changed, 205 insertions(+), 30 deletions(-) create mode 100644 docs/content.zh/docs/ops/state/checkpoints_vs_savepoints.md create mode 100644 docs/content/docs/ops/state/checkpoints_vs_savepoints.md diff --git a/docs/content.zh/docs/concepts/stateful-stream-processing.md b/docs/content.zh/docs/concepts/stateful-stream-processing.md index 011c96bb128ff..d4748b5b59e62 100644 --- a/docs/content.zh/docs/concepts/stateful-stream-processing.md +++ b/docs/content.zh/docs/concepts/stateful-stream-processing.md @@ -310,7 +310,10 @@ mechanism for this. Savepoints are similar to checkpoints except that they are **triggered by the user** and **don't automatically expire** when newer -checkpoints are completed. +checkpoints are completed. +To make proper use of savepoints, it's important to understand the differences between +[checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) and [savepoints]({{< ref "docs/ops/state/savepoints" >}}) +which is described in [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). {{< top >}} diff --git a/docs/content.zh/docs/ops/state/checkpoints.md b/docs/content.zh/docs/ops/state/checkpoints.md index 9fb8e644f6f83..f72038810567a 100644 --- a/docs/content.zh/docs/ops/state/checkpoints.md +++ b/docs/content.zh/docs/ops/state/checkpoints.md @@ -31,6 +31,8 @@ Checkpoint 使 Flink 的状态具有良好的容错性,通过 checkpoint 机 参考 [Checkpointing]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) 查看如何在 Flink 程序中开启和配置 checkpoint。 +To understand the differences between checkpoints and [savepoints]({{< ref "docs/ops/state/savepoints" >}}) see [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). + ## 保留 Checkpoint Checkpoint 在默认的情况下仅用于恢复失败的作业,并不保留,当程序取消时 checkpoint 就会被删除。当然,你可以通过配置来保留 checkpoint,这些被保留的 checkpoint 在作业失败或取消时不会被清除。这样,你就可以使用该 checkpoint 来恢复失败的作业。 @@ -80,12 +82,6 @@ state.checkpoints.dir: hdfs:///checkpoints/ env.setStateBackend(new RocksDBStateBackend("hdfs:///checkpoints-data/")); ``` -### Checkpoint 与 Savepoint 的区别 - -Checkpoint 与 [savepoints]({{< ref "docs/ops/state/savepoints" >}}) 有一些区别,体现在 checkpoint : -- 使用 state backend 特定的数据格式,可能以增量方式存储。 -- 不支持 Flink 的特定功能,比如扩缩容。 - ### 从保留的 checkpoint 中恢复状态 与 savepoint 一样,作业可以从 checkpoint 的元数据文件恢复运行([savepoint恢复指南]({{< ref "docs/ops/state/savepoints" >}}#resuming-from-savepoints))。注意,如果元数据文件中信息不充分,那么 jobmanager 就需要使用相关的数据文件来恢复作业(参考[目录结构](#directory-structure))。 diff --git a/docs/content.zh/docs/ops/state/checkpoints_vs_savepoints.md b/docs/content.zh/docs/ops/state/checkpoints_vs_savepoints.md new file mode 100644 index 0000000000000..0b66b256ea9ff --- /dev/null +++ b/docs/content.zh/docs/ops/state/checkpoints_vs_savepoints.md @@ -0,0 +1,95 @@ +--- +title: "Checkpoints vs. Savepoints" +weight: 10 +type: docs +aliases: + - /ops/state/checkpoints_vs_savepoints.html +--- + + +# Checkpoints vs. Savepoints + +## Overview + +Conceptually, Flink's [savepoints]({{< ref "docs/ops/state/savepoints" >}}) are different from [checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) +in a way that's analogous to how backups are different from recovery logs in traditional database systems. + +The primary purpose of checkpoints is to provide a recovery mechanism in case of unexpected job failures. +A [checkpoint's lifecycle]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) is managed by Flink, +i.e. a checkpoint is created, owned, and released by Flink - without user interaction. +Because checkpoints are being triggered often, and are relied upon for failure recovery, the two main design goals for the checkpoint implementation are +i) being as lightweight to create and ii) being as fast to restore from as possible. +Optimizations towards those goals can exploit certain properties, e.g., that the job code doesn't change between the execution attempts. + +{{< hint info >}} +- Checkpoints are automatically deleted if the application is terminated by the user +(except if checkpoints are explicitly configured to be retained). +- Checkpoints are stored in state backend-specific (native) data format (may be incremental depending on the specific backend). +{{< /hint >}} + +Although [savepoints]({{< ref "docs/ops/state/savepoints" >}}) are created internally with the same mechanisms as +checkpoints, they are conceptually different and can be a bit more expensive to produce and restore from. Their design focuses +more on portability and operational flexibility, especially with respect to changes to the job. +The use case for savepoints is for planned, manual operations. For example, this could be an update of your Flink version, changing your job graph, and so on. + +{{< hint info >}} +- Savepoints are created, owned and deleted solely by the user. +That means, Flink does not delete savepoints neither after job termination nor after +restore. +- Savepoints are stored in a state backend independent (canonical) format (Note: Since Flink 1.15, savepoints can be also stored in +the backend-specific [native]({{< ref "docs/ops/state/savepoints" >}}#savepoint-format) format which is faster to create +and restore but comes with some limitations. +{{< /hint >}} + +### Capabilities and limitations +The following table gives an overview of capabilities and limitations for the various types of savepoints and +checkpoints. +- ✓ - Flink fully support this type of the snapshot +- x - Flink doesn't support this type of the snapshot +- ! - While these operations currently work, Flink doesn't officially guarantee support for them, so there is a certain level of risk associated with them + +| Operation | Canonical Savepoint | Native Savepoint | Aligned Checkpoint | Unaligned Checkpoint | +|:----------------------------------|:--------------------|:-----------------|:-------------------|:---------------------| +| State backend change | ✓ | x | x | x | +| State Processor API (writing) | ✓ | x | x | x | +| State Processor API (reading) | ✓ | ! | ! | x | +| Self-contained and relocatable | ✓ | ✓ | x | x | +| Schema evolution | ✓ | ! | ! | ! | +| Arbitrary job upgrade | ✓ | ✓ | ✓ | x | +| Non-arbitrary job upgrade | ✓ | ✓ | ✓ | x | +| Flink minor version upgrade | ✓ | ✓ | ✓ | x | +| Flink bug/patch version upgrade | ✓ | ✓ | ✓ | ✓ | +| Rescaling | ✓ | ✓ | ✓ | ✓ | + +- [State backend change]({{< ref "docs/ops/state/state_backends" >}}) - configuring a different State Backend than was used when taking the snapshot. +- [State Processor API (writing)]({{< ref "docs/libs/state_processor_api" >}}#writing-new-savepoints) - the ability to create a new snapshot of this type via the State Processor API. +- [State Processor API (reading)]({{< ref "docs/libs/state_processor_api" >}}#reading-state) - the ability to read states from an existing snapshot of this type via the State Processor API. +- Self-contained and relocatable - the one snapshot folder contains everything it needs for recovery +and it doesn't depend on other snapshots which means it can be easily moved to another place if needed. +- [Schema evolution]({{< ref "docs/dev/datastream/fault-tolerance/serialization/schema_evolution" >}}) - the *state* data type can be changed if it uses a serializer that supports schema evolution (e.g., POJOs and Avro types) +- Arbitrary job upgrade - the snapshot can be restored even if the [partitioning types]({{< ref "docs/dev/datastream/operators/overview" >}}#physical-partitioning)(rescale, rebalance, map, etc.) +or in-flight record types for the existing operators have changed. +- Non-arbitrary job upgrade - restoring the snapshot is possible with updated operators if the job graph topology and in-flight record types remain unchanged. +- Flink minor version upgrade - restoring a snapshot taken with an older minor version of Flink (1.x → 1.y). +- Flink bug/patch version upgrade - restoring a snapshot taken with an older patch version of Flink (1.14.x → 1.14.y). +- Rescaling - restoring the snapshot with a different parallelism than was used during the snapshot creation. + + +{{< top >}} diff --git a/docs/content.zh/docs/ops/state/savepoints.md b/docs/content.zh/docs/ops/state/savepoints.md index 87a2890ad03dc..11f336898d93c 100644 --- a/docs/content.zh/docs/ops/state/savepoints.md +++ b/docs/content.zh/docs/ops/state/savepoints.md @@ -27,7 +27,7 @@ under the License. # Savepoints -## 什么是 Savepoint ? Savepoint 与 Checkpoint 有什么不同? +## 什么是 Savepoint ? Savepoint 是依据 Flink [checkpointing 机制]({{< ref "docs/learn-flink/fault_tolerance" >}})所创建的流作业执行状态的一致镜像。 你可以使用 Savepoint 进行 Flink 作业的停止与重启、fork 或者更新。 Savepoint 由两部分组成:稳定存储(列入 HDFS,S3,...) 上包含二进制文件的目录(通常很大),和元数据文件(相对较小)。 稳定存储上的文件表示作业执行状态的数据镜像。 Savepoint 的元数据文件以(相对路径)的形式包含(主要)指向作为 Savepoint 一部分的稳定存储上的所有文件的指针。 @@ -35,11 +35,7 @@ Savepoint 是依据 Flink [checkpointing 机制]({{< ref "docs/learn-flink/fault **注意:** 为了允许程序和 Flink 版本之间的升级,请务必查看以下有关分配算子 ID 的部分 。 {{< /hint >}} -从概念上讲, Flink 的 Savepoint 与 Checkpoint 的不同之处类似于传统数据库中的备份与恢复日志之间的差异。 Checkpoint 的主要目的是为意外失败的作业提供恢复机制。 Checkpoint 的生命周期由 Flink 管理,即 Flink 创建,管理和删除 Checkpoint - 无需用户交互。 作为一种恢复和定期触发的方法,Checkpoint 实现有两个设计目标:i)轻量级创建和 ii)尽可能快地恢复。 可能会利用某些特定的属性来达到这个,例如, 工作代码在执行尝试之间不会改变。 在用户终止作业后,通常会删除 Checkpoint(除非明确配置为保留的 Checkpoint)。 - - 与此相反、Savepoint 由用户创建,拥有和删除。 他们的用例是计划的,手动备份和恢复。 例如,升级 Flink 版本,调整用户逻辑,改变并行度,以及进行红蓝部署等。 当然,Savepoint 必须在作业停止后继续存在。 从概念上讲,Savepoint 的生成,恢复成本可能更高一些,Savepoint 更多地关注可移植性和对前面提到的作业更改的支持。 - -除去这些概念上的差异,Checkpoint 和 Savepoint 的当前实现基本上使用相同的代码并生成相同的格式。然而,目前有一个例外,我们可能会在未来引入更多的差异。例外情况是使用 RocksDB 状态后端的增量 Checkpoint。他们使用了一些 RocksDB 内部格式,而不是 Flink 的本机 Savepoint 格式。这使他们成为了与 Savepoint 相比,更轻量级的 Checkpoint 机制的第一个实例。 +To make proper use of savepoints, it's important to understand the differences between [checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) and savepoints which is described in [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). ## 分配算子 ID diff --git a/docs/content/docs/concepts/stateful-stream-processing.md b/docs/content/docs/concepts/stateful-stream-processing.md index f8c2a77dcb5f6..7f139b021bb70 100644 --- a/docs/content/docs/concepts/stateful-stream-processing.md +++ b/docs/content/docs/concepts/stateful-stream-processing.md @@ -312,6 +312,9 @@ mechanism for this. Savepoints are similar to checkpoints except that they are **triggered by the user** and **don't automatically expire** when newer checkpoints are completed. +To make proper use of savepoints, it's important to understand the differences between +[checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) and [savepoints]({{< ref "docs/ops/state/savepoints" >}}) +which is described in [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). {{< top >}} diff --git a/docs/content/docs/ops/state/checkpoints.md b/docs/content/docs/ops/state/checkpoints.md index 63f0c0694dfb7..2d4f6ecd6a893 100644 --- a/docs/content/docs/ops/state/checkpoints.md +++ b/docs/content/docs/ops/state/checkpoints.md @@ -35,6 +35,8 @@ the same semantics as a failure-free execution. See [Checkpointing]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) for how to enable and configure checkpoints for your program. +To understand the differences between checkpoints and [savepoints]({{< ref "docs/ops/state/savepoints" >}}) see [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). + ## Checkpoint Storage When checkpointing is enabled, managed state is persisted to ensure consistent recovery in case of failures. @@ -156,12 +158,6 @@ env.getCheckpointConfig().setCheckpointStorage( new FileSystemCheckpointStorage("hdfs:///checkpoints-data/", FILE_SIZE_THESHOLD)); ``` -### Difference to Savepoints - -Checkpoints have a few differences from [savepoints]({{< ref "docs/ops/state/savepoints" >}}). They -- use a state backend specific (low-level) data format, may be incremental. (starting from Flink 1.15 savepoints can also use the backend [native]({{< ref "docs/ops/state/savepoints" >}}#savepoint-format) format.) -- do not support Flink specific features like rescaling. - ### Resuming from a retained checkpoint A job may be resumed from a checkpoint just as from a savepoint @@ -175,6 +171,4 @@ above). $ bin/flink run -s :checkpointMetaDataPath [:runArgs] ``` - - {{< top >}} diff --git a/docs/content/docs/ops/state/checkpoints_vs_savepoints.md b/docs/content/docs/ops/state/checkpoints_vs_savepoints.md new file mode 100644 index 0000000000000..0b66b256ea9ff --- /dev/null +++ b/docs/content/docs/ops/state/checkpoints_vs_savepoints.md @@ -0,0 +1,95 @@ +--- +title: "Checkpoints vs. Savepoints" +weight: 10 +type: docs +aliases: + - /ops/state/checkpoints_vs_savepoints.html +--- + + +# Checkpoints vs. Savepoints + +## Overview + +Conceptually, Flink's [savepoints]({{< ref "docs/ops/state/savepoints" >}}) are different from [checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) +in a way that's analogous to how backups are different from recovery logs in traditional database systems. + +The primary purpose of checkpoints is to provide a recovery mechanism in case of unexpected job failures. +A [checkpoint's lifecycle]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) is managed by Flink, +i.e. a checkpoint is created, owned, and released by Flink - without user interaction. +Because checkpoints are being triggered often, and are relied upon for failure recovery, the two main design goals for the checkpoint implementation are +i) being as lightweight to create and ii) being as fast to restore from as possible. +Optimizations towards those goals can exploit certain properties, e.g., that the job code doesn't change between the execution attempts. + +{{< hint info >}} +- Checkpoints are automatically deleted if the application is terminated by the user +(except if checkpoints are explicitly configured to be retained). +- Checkpoints are stored in state backend-specific (native) data format (may be incremental depending on the specific backend). +{{< /hint >}} + +Although [savepoints]({{< ref "docs/ops/state/savepoints" >}}) are created internally with the same mechanisms as +checkpoints, they are conceptually different and can be a bit more expensive to produce and restore from. Their design focuses +more on portability and operational flexibility, especially with respect to changes to the job. +The use case for savepoints is for planned, manual operations. For example, this could be an update of your Flink version, changing your job graph, and so on. + +{{< hint info >}} +- Savepoints are created, owned and deleted solely by the user. +That means, Flink does not delete savepoints neither after job termination nor after +restore. +- Savepoints are stored in a state backend independent (canonical) format (Note: Since Flink 1.15, savepoints can be also stored in +the backend-specific [native]({{< ref "docs/ops/state/savepoints" >}}#savepoint-format) format which is faster to create +and restore but comes with some limitations. +{{< /hint >}} + +### Capabilities and limitations +The following table gives an overview of capabilities and limitations for the various types of savepoints and +checkpoints. +- ✓ - Flink fully support this type of the snapshot +- x - Flink doesn't support this type of the snapshot +- ! - While these operations currently work, Flink doesn't officially guarantee support for them, so there is a certain level of risk associated with them + +| Operation | Canonical Savepoint | Native Savepoint | Aligned Checkpoint | Unaligned Checkpoint | +|:----------------------------------|:--------------------|:-----------------|:-------------------|:---------------------| +| State backend change | ✓ | x | x | x | +| State Processor API (writing) | ✓ | x | x | x | +| State Processor API (reading) | ✓ | ! | ! | x | +| Self-contained and relocatable | ✓ | ✓ | x | x | +| Schema evolution | ✓ | ! | ! | ! | +| Arbitrary job upgrade | ✓ | ✓ | ✓ | x | +| Non-arbitrary job upgrade | ✓ | ✓ | ✓ | x | +| Flink minor version upgrade | ✓ | ✓ | ✓ | x | +| Flink bug/patch version upgrade | ✓ | ✓ | ✓ | ✓ | +| Rescaling | ✓ | ✓ | ✓ | ✓ | + +- [State backend change]({{< ref "docs/ops/state/state_backends" >}}) - configuring a different State Backend than was used when taking the snapshot. +- [State Processor API (writing)]({{< ref "docs/libs/state_processor_api" >}}#writing-new-savepoints) - the ability to create a new snapshot of this type via the State Processor API. +- [State Processor API (reading)]({{< ref "docs/libs/state_processor_api" >}}#reading-state) - the ability to read states from an existing snapshot of this type via the State Processor API. +- Self-contained and relocatable - the one snapshot folder contains everything it needs for recovery +and it doesn't depend on other snapshots which means it can be easily moved to another place if needed. +- [Schema evolution]({{< ref "docs/dev/datastream/fault-tolerance/serialization/schema_evolution" >}}) - the *state* data type can be changed if it uses a serializer that supports schema evolution (e.g., POJOs and Avro types) +- Arbitrary job upgrade - the snapshot can be restored even if the [partitioning types]({{< ref "docs/dev/datastream/operators/overview" >}}#physical-partitioning)(rescale, rebalance, map, etc.) +or in-flight record types for the existing operators have changed. +- Non-arbitrary job upgrade - restoring the snapshot is possible with updated operators if the job graph topology and in-flight record types remain unchanged. +- Flink minor version upgrade - restoring a snapshot taken with an older minor version of Flink (1.x → 1.y). +- Flink bug/patch version upgrade - restoring a snapshot taken with an older patch version of Flink (1.14.x → 1.14.y). +- Rescaling - restoring the snapshot with a different parallelism than was used during the snapshot creation. + + +{{< top >}} diff --git a/docs/content/docs/ops/state/savepoints.md b/docs/content/docs/ops/state/savepoints.md index a24c4db4d941a..2b182e43e6c12 100644 --- a/docs/content/docs/ops/state/savepoints.md +++ b/docs/content/docs/ops/state/savepoints.md @@ -27,7 +27,7 @@ under the License. # Savepoints -## What is a Savepoint? How is a Savepoint different from a Checkpoint? +## What is a Savepoint? A Savepoint is a consistent image of the execution state of a streaming job, created via Flink's [checkpointing mechanism]({{< ref "docs/learn-flink/fault_tolerance" >}}). You can use Savepoints to stop-and-resume, fork, or update your Flink jobs. Savepoints consist of two parts: a directory with (typically large) binary files on stable storage (e.g. HDFS, S3, ...) and a (relatively small) meta data file. The files on stable storage represent the net data of the job's execution state @@ -37,14 +37,7 @@ image. The meta data file of a Savepoint contains (primarily) pointers to all fi In order to allow upgrades between programs and Flink versions, it is important to check out the following section about [assigning IDs to your operators](#assigning-operator-ids). {{< /hint >}} -Conceptually, Flink's Savepoints are different from Checkpoints in a similar way that backups are different from recovery logs in traditional database systems. The primary purpose of Checkpoints is to provide a recovery mechanism in case of -unexpected job failures. A Checkpoint's lifecycle is managed by Flink, i.e. a Checkpoint is created, owned, and deleted by Flink - without user interaction. As a method of recovery and being periodically triggered, two main -design goals for the Checkpoint implementation are i) being as lightweight to create and ii) being as fast to restore from as possible. Optimizations towards those goals can exploit certain properties, e.g. that the job code -does not change between the execution attempts. Checkpoints are usually dropped after the job was terminated by the user unless the job is explicitly configured to retain checkpoints upon failure or cancellation. - -In contrast to all this, Savepoints are created, owned, and deleted by the user. Their use case is planned, manual backup and resume. For example, this could be an update of your Flink version, changing your job graph, -changing parallelism, forking a second job like for a red/blue deployment, and so on. Of course, Savepoints must survive job termination. Conceptually, Savepoints can be a bit more expensive to produce and restore from and focus -more on portability and flexibility with respect to changes to the job. +To make proper use of savepoints, it's important to understand the differences between [checkpoints]({{< ref "docs/ops/state/checkpoints" >}}) and savepoints which is described in [checkpoints vs. savepoints]({{< ref "docs/ops/state/checkpoints_vs_savepoints" >}}). ## Assigning Operator IDs From e0c8df17f3b0469affeb4c07be5b7c03d3a94333 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 29 Mar 2022 17:31:11 +0300 Subject: [PATCH 124/258] [FLINK-25227][table-planner] Fix LEAST/GREATEST to return primitives Previously, `LEAST` and `GREATEST` functions would return primitive types in the generated code implementing their logic, producing issues for operators applied on top of them, and most importantly comparison operators, i.e.: ``` f0 INT, f1 INT SELECT GREATEST(f0, f1) = GREATEST(f0, f1) ``` would return `FALSE`, since the generated code would return `Integer` instead of `int`, as the result of `GREATEST`, and the `=` operator on `Integer` objects would return false, even if the actual integer value of them was the same. (cherry picked from commit 89cdc6e01f291e9ce5c1dfeb4bd883809e1eeaf5) --- .../codegen/calls/ScalarOperatorGens.scala | 18 ++-- .../GreatestLeastFunctionsITCase.java | 83 ++++++++++++++++++- 2 files changed, 89 insertions(+), 12 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 6288b966ad379..6d920b4787078 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -1381,10 +1381,11 @@ object ScalarOperatorGens { elements: Seq[GeneratedExpression], greatest: Boolean = true) : GeneratedExpression = { - val Seq(result, cur, nullTerm) = newNames("result", "cur", "nullTerm") + val Seq(result, tmpResult, cur, nullTerm) = newNames("result", "tmpResult", "cur", "nullTerm") val widerType = toScala(findCommonType(elements.map(element => element.resultType))) .orElse(throw new CodeGenException(s"Unable to find common type for $elements.")) - val resultTypeTerm = boxedTypeTermForType(widerType.get) + val boxedResultTypeTerm = boxedTypeTermForType(widerType.get) + val primitiveResultTypeTerm = primitiveTypeTermForType(widerType.get) def castIfNumeric(t: GeneratedExpression): String = { if (isNumeric(widerType.get)) { @@ -1398,13 +1399,13 @@ object ScalarOperatorGens { s""" | ${element.code} | if (!$nullTerm) { - | $resultTypeTerm $cur = ${castIfNumeric(element)}; + | $boxedResultTypeTerm $cur = ${castIfNumeric(element)}; | if (${element.nullTerm}) { | $nullTerm = true; | } else { - | int compareResult = $result.compareTo($cur); + | int compareResult = $tmpResult.compareTo($cur); | if (($greatest && compareResult < 0) || (compareResult > 0 && !$greatest)) { - | $result = $cur; + | $tmpResult = $cur; | } | } | } @@ -1413,11 +1414,12 @@ object ScalarOperatorGens { val code = s""" - | $resultTypeTerm $result = ${castIfNumeric(elements.head)}; + | $boxedResultTypeTerm $tmpResult = ${castIfNumeric(elements.head)}; + | $primitiveResultTypeTerm $result = ${primitiveDefaultValue(widerType.get)}; | boolean $nullTerm = false; | $elementsCode - | if ($nullTerm) { - | $result = null; + | if (!$nullTerm) { + | $result = $tmpResult; | } """.stripMargin GeneratedExpression(result, nullTerm, code, resultType) diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java index 178043841748d..d495c3f98e1d1 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/GreatestLeastFunctionsITCase.java @@ -81,20 +81,63 @@ public static List testData() { call("GREATEST", $("f6"), $("f7")), "GREATEST(f6, f7)", LocalDateTime.parse("1970-01-01T00:00:03.001"), - DataTypes.TIMESTAMP(3).notNull())) + DataTypes.TIMESTAMP(3).notNull()), + resultSpec( + call("GREATEST", $("f0"), $("f1"), $("f2")), + "GREATEST(f0, f1, f2)", + null, + DataTypes.INT()), + resultSpec( + call("GREATEST", $("f4"), $("f5")), + "GREATEST(f4, f5)", + "world", + DataTypes.STRING().notNull()), + resultSpec( + call("GREATEST", $("f6"), $("f7")), + "GREATEST(f6, f7)", + LocalDateTime.parse("1970-01-01T00:00:03.001"), + DataTypes.TIMESTAMP(3).notNull()), + // assert that primitive types are returned and used in the equality + // operator applied on top of the GREATEST functions + resultSpec( + call( + "EQUALS", + call("GREATEST", $("f1"), $("f2")), + call("GREATEST", $("f1"), $("f2"))), + "GREATEST(f1, f2) = GREATEST(f1, f2)", + true, + DataTypes.BOOLEAN().notNull()), + resultSpec( + call( + "EQUALS", + call("GREATEST", $("f0"), $("f1")), + call("GREATEST", $("f0"), $("f1"))), + "GREATEST(f0, f1) = GREATEST(f0, f1)", + null, + DataTypes.BOOLEAN())) .testSqlValidationError( "GREATEST(f5, f6)", "SQL validation failed. Invalid function call:\n" + "GREATEST(STRING NOT NULL, TIMESTAMP(3) NOT NULL)"), TestSpec.forFunction(BuiltInFunctionDefinitions.LEAST) - .onFieldsWithData(null, 1, 2, 3.14, "hello", "world") + .onFieldsWithData( + null, + 1, + 2, + 3.14, + "hello", + "world", + LocalDateTime.parse("1970-01-01T00:00:03.001"), + LocalDateTime.parse("1970-01-01T00:00:02.001")) .andDataTypes( DataTypes.INT().nullable(), DataTypes.INT().notNull(), DataTypes.INT().notNull(), DataTypes.DECIMAL(3, 2).notNull(), DataTypes.STRING().notNull(), - DataTypes.STRING().notNull()) + DataTypes.STRING().notNull(), + DataTypes.TIMESTAMP(3).notNull(), + DataTypes.TIMESTAMP(3).notNull()) .testSqlValidationError( "LEAST(f1, f4)", "SQL validation failed. Invalid function call:\n" @@ -115,6 +158,38 @@ public static List testData() { call("LEAST", $("f4"), $("f5")), "LEAST(f4, f5)", "hello", - DataTypes.STRING().notNull()))); + DataTypes.STRING().notNull()), + resultSpec( + call("LEAST", $("f0"), $("f1")), + "LEAST(f0, f1)", + null, + DataTypes.INT()), + resultSpec( + call("LEAST", $("f4"), $("f5")), + "LEAST(f4, f5)", + "hello", + DataTypes.STRING().notNull()), + // assert that primitive types are returned and used in the equality + // operator applied on top of the GREATEST functions + resultSpec( + call( + "EQUALS", + call("LEAST", $("f1"), $("f2")), + call("LEAST", $("f1"), $("f2"))), + "LEAST(f1, f2) = LEAST(f1, f2)", + true, + DataTypes.BOOLEAN().notNull()), + resultSpec( + call( + "EQUALS", + call("LEAST", $("f0"), $("f1")), + call("LEAST", $("f0"), $("f1"))), + "LEAST(f0, f1) = LEAST(f0, f1)", + null, + DataTypes.BOOLEAN())) + .testSqlValidationError( + "LEAST(f5, f6)", + "SQL validation failed. Invalid function call:\n" + + "LEAST(STRING NOT NULL, TIMESTAMP(3) NOT NULL)")); } } From ba63354de3d9a8837bbfb23cb87cf6c4af33e735 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Alexander=20Preu=C3=9F?= <11444089+alpreu@users.noreply.github.com> Date: Tue, 22 Mar 2022 14:24:00 +0100 Subject: [PATCH 125/258] [FLINK-26281][connectors/elasticsearch] Remove deprecated 'type', explain exactly-once semantic. (cherry picked from commit 9a3524008c4a2d955ccafb6e2ece39db37c2e3df) --- .../connectors/datastream/elasticsearch.md | 47 +++++++++++++++++-- .../connectors/datastream/elasticsearch.md | 8 +++- 2 files changed, 48 insertions(+), 7 deletions(-) diff --git a/docs/content.zh/docs/connectors/datastream/elasticsearch.md b/docs/content.zh/docs/connectors/datastream/elasticsearch.md index 5688306bab392..aea933e17a8c8 100644 --- a/docs/content.zh/docs/connectors/datastream/elasticsearch.md +++ b/docs/content.zh/docs/connectors/datastream/elasticsearch.md @@ -132,7 +132,6 @@ private static IndexRequest createIndexRequest(String element) { return Requests.indexRequest() .index("my-index") - .type("my-type") .id(element) .source(json); } @@ -165,7 +164,7 @@ def createIndexRequest(element: (String)): IndexRequest = { "data" -> element.asInstanceOf[AnyRef] ) - Requests.indexRequest.index("my-index").`type`("my-type").source(mapAsJavaMap(json)) + Requests.indexRequest.index("my-index").source(mapAsJavaMap(json)) } ``` @@ -220,23 +219,61 @@ def createIndexRequest(element: (String)): IndexRequest = { {{< tabs "d00d1e93-4844-40d7-b0ec-9ec37e73145e" >}} {{< tab "Java" >}} +Elasticsearch 6: +```java +final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); +env.enableCheckpointing(5000); // 每 5000 毫秒执行一次 checkpoint + +Elasticsearch6SinkBuilder sinkBuilder = new Elasticsearch6SinkBuilder() + .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + .setHosts(new HttpHost("127.0.0.1", 9200, "http")) + .setEmitter( + (element, context, indexer) -> + indexer.add(createIndexRequest(element))); +``` + +Elasticsearch 7: ```java final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(5000); // 每 5000 毫秒执行一次 checkpoint + +Elasticsearch7SinkBuilder sinkBuilder = new Elasticsearch7SinkBuilder() + .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + .setHosts(new HttpHost("127.0.0.1", 9200, "http")) + .setEmitter( + (element, context, indexer) -> + indexer.add(createIndexRequest(element))); ``` {{< /tab >}} {{< tab "Scala" >}} +Elasticsearch 6: ```scala val env = StreamExecutionEnvironment.getExecutionEnvironment() env.enableCheckpointing(5000) // 每 5000 毫秒执行一次 checkpoint + +val sinkBuilder = new Elasticsearch6SinkBuilder[String] + .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + .setHosts(new HttpHost("127.0.0.1", 9200, "http")) + .setEmitter((element: String, context: SinkWriter.Context, indexer: RequestIndexer) => + indexer.add(createIndexRequest(element))) +``` + +Elasticsearch 7: +```scala +val env = StreamExecutionEnvironment.getExecutionEnvironment() +env.enableCheckpointing(5000) // 每 5000 毫秒执行一次 checkpoint + +val sinkBuilder = new Elasticsearch7SinkBuilder[String] + .setDeliveryGuarantee(DeliveryGuarantee.AT_LEAST_ONCE) + .setHosts(new HttpHost("127.0.0.1", 9200, "http")) + .setEmitter((element: String, context: SinkWriter.Context, indexer: RequestIndexer) => + indexer.add(createIndexRequest(element))) ``` {{< /tab >}} {{< /tabs >}}

-重要提示: 默认情况下不启用 checkpoint, 但默认传输保证 AT_LEAST_ONCE 语义。 -这会导致 sink 缓冲请求,直到它结束或 BulkProcessor 自动刷新。 -默认情况下,BulkProcessor 将在 1000 个添加操作后刷新。要将 Processor 配置为更频繁地刷新,请参阅 BulkProcessor 配置部分。 +Using UpdateRequests with deterministic ids and the upsert method it is possible to achieve exactly-once semantics in Elasticsearch when AT_LEAST_ONCE delivery is configured for the connector.

### 处理失败的 Elasticsearch 请求 diff --git a/docs/content/docs/connectors/datastream/elasticsearch.md b/docs/content/docs/connectors/datastream/elasticsearch.md index eb493920fe015..d467bb391b22e 100644 --- a/docs/content/docs/connectors/datastream/elasticsearch.md +++ b/docs/content/docs/connectors/datastream/elasticsearch.md @@ -134,7 +134,6 @@ private static IndexRequest createIndexRequest(String element) { return Requests.indexRequest() .index("my-index") - .type("my-type") .id(element) .source(json); } @@ -195,7 +194,7 @@ def createIndexRequest(element: (String)): IndexRequest = { "data" -> element.asInstanceOf[AnyRef] ) - Requests.indexRequest.index("my-index").`type`("my-type").source(mapAsJavaMap(json)) + Requests.indexRequest.index("my-index").source(mapAsJavaMap(json)) } ``` {{< /tab >}} @@ -247,6 +246,11 @@ This causes the sink to buffer requests until it either finishes or the BulkProc By default, the BulkProcessor will flush after 1000 added Actions. To configure the processor to flush more frequently, please refer to the BulkProcessor configuration section.

+

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

+ + ### Handling Failing Elasticsearch Requests Elasticsearch action requests may fail due to a variety of reasons, including From 575a0083e73736df35b0986fb3487aa01b6f8351 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 30 Mar 2022 10:44:33 +0200 Subject: [PATCH 126/258] [FLINK-26923] Do not trigger global failover if failed during commiting side-effects during stop-with-savepoint for adaptive scheduler --- .../scheduler/adaptive/StopWithSavepoint.java | 12 +++++- .../StopWithSavepointStoppingException.java | 22 +++++++---- .../test/checkpointing/SavepointITCase.java | 7 ++-- .../scheduling/AdaptiveSchedulerITCase.java | 37 +++++++++++-------- 4 files changed, 52 insertions(+), 26 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepoint.java index 2c03e7454f1ac..de4584e004243 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StopWithSavepoint.java @@ -26,6 +26,7 @@ import org.apache.flink.runtime.scheduler.ExecutionGraphHandler; import org.apache.flink.runtime.scheduler.OperatorCoordinatorHandler; import org.apache.flink.runtime.scheduler.exceptionhistory.ExceptionHistoryEntry; +import org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointStoppingException; import org.apache.flink.util.FlinkException; import org.apache.flink.util.Preconditions; import org.apache.flink.util.concurrent.FutureUtils; @@ -146,7 +147,16 @@ public JobStatus getJobStatus() { @Override void onFailure(Throwable cause) { operationFailureCause = cause; - FailureResultUtil.restartOrFail(context.howToHandleFailure(cause), context, this); + if (savepoint == null) { + FailureResultUtil.restartOrFail(context.howToHandleFailure(cause), context, this); + } else { + // savepoint has been create successfully, but the job failed while committing side + // effects + final StopWithSavepointStoppingException ex = + new StopWithSavepointStoppingException(savepoint, this.getJobId(), cause); + this.operationFuture.completeExceptionally(ex); + FailureResultUtil.restartOrFail(context.howToHandleFailure(ex), context, this); + } } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java index d41dbc81f8cc5..5accea9a02d05 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/stopwithsavepoint/StopWithSavepointStoppingException.java @@ -35,16 +35,24 @@ public class StopWithSavepointStoppingException extends FlinkException { private final String savepointPath; public StopWithSavepointStoppingException(String savepointPath, JobID jobID) { - super( - String.format( - "A savepoint has been created at: %s, but the corresponding job %s failed " - + "during stopping. The savepoint is consistent, but might have " - + "uncommitted transactions. If you want to commit the transaction " - + "please restart a job from this savepoint.", - savepointPath, jobID)); + super(formatMessage(savepointPath, jobID)); this.savepointPath = savepointPath; } + public StopWithSavepointStoppingException(String savepointPath, JobID jobID, Throwable cause) { + super(formatMessage(savepointPath, jobID), cause); + this.savepointPath = savepointPath; + } + + private static String formatMessage(String savepointPath, JobID jobID) { + return String.format( + "A savepoint has been created at: %s, but the corresponding job %s failed " + + "during stopping. The savepoint is consistent, but might have " + + "uncommitted transactions. If you want to commit the transaction " + + "please restart a job from this savepoint.", + savepointPath, jobID); + } + public String getSavepointPath() { return savepointPath; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index a318136fd5d7f..92ddcdf757c53 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -145,6 +145,7 @@ import static org.apache.flink.util.ExceptionUtils.assertThrowableWithMessage; import static org.apache.flink.util.ExceptionUtils.findThrowable; import static org.apache.flink.util.ExceptionUtils.findThrowableWithMessage; +import static org.hamcrest.CoreMatchers.either; import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -296,8 +297,6 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { public void initializeState(FunctionInitializationContext context) throws Exception {} } - private static final OneShotLatch stopWithSavepointRestartLatch = new OneShotLatch(); - @Test public void testStopWithSavepointFailsOverToSavepoint() throws Throwable { int sinkParallelism = 5; @@ -342,7 +341,9 @@ public void testStopWithSavepointFailsOverToSavepoint() throws Throwable { && throwable .getMessage() .startsWith("A savepoint has been created at: ")); - assertThat(client.getJobStatus(jobGraph.getJobID()).get(), is(JobStatus.FAILED)); + assertThat( + client.getJobStatus(jobGraph.getJobID()).get(), + either(is(JobStatus.FAILED)).or(is(JobStatus.FAILING))); } finally { cluster.after(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java index 3852bd19c982d..96cb91e0f602e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.rest.messages.JobExceptionsHeaders; import org.apache.flink.runtime.rest.messages.JobExceptionsInfoWithHistory; import org.apache.flink.runtime.rest.messages.job.JobExceptionsMessageParameters; +import org.apache.flink.runtime.scheduler.stopwithsavepoint.StopWithSavepointStoppingException; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -70,9 +71,12 @@ import java.util.stream.Collectors; import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.util.ExceptionUtils.assertThrowable; import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.either; import static org.hamcrest.CoreMatchers.is; import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.fail; import static org.junit.Assume.assumeTrue; @@ -185,7 +189,7 @@ public void testStopWithSavepointFailOnCheckpoint() throws Exception { } @Test - public void testStopWithSavepointFailOnStop() throws Exception { + public void testStopWithSavepointFailOnStop() throws Throwable { StreamExecutionEnvironment env = getEnvWithSource(StopWithSavepointTestBehavior.FAIL_ON_CHECKPOINT_COMPLETE); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, 0L)); @@ -195,20 +199,23 @@ public void testStopWithSavepointFailOnStop() throws Exception { JobClient client = env.executeAsync(); DummySource.awaitRunning(); - try { - client.stopWithSavepoint( - false, - tempFolder.newFolder("savepoint").getAbsolutePath(), - SavepointFormatType.CANONICAL) - .get(); - fail("Expect exception"); - } catch (ExecutionException e) { - assertThat(e, containsCause(FlinkException.class)); - } - // expect job to run again (maybe restart) - CommonTestUtils.waitUntilCondition( - () -> client.getJobStatus().get() == JobStatus.RUNNING, - Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES))); + final CompletableFuture savepointCompleted = + client.stopWithSavepoint( + false, + tempFolder.newFolder("savepoint").getAbsolutePath(), + SavepointFormatType.CANONICAL); + final Throwable savepointException = + assertThrows(ExecutionException.class, savepointCompleted::get).getCause(); + assertThrowable( + savepointException, + throwable -> + throwable instanceof StopWithSavepointStoppingException + && throwable + .getMessage() + .startsWith("A savepoint has been created at: ")); + assertThat( + client.getJobStatus().get(), + either(is(JobStatus.FAILED)).or(is(JobStatus.FAILING))); } @Test From df3ab6c72f991e9b28f2c2566e95b23bdfb7cf9c Mon Sep 17 00:00:00 2001 From: Yun Tang Date: Thu, 10 Mar 2022 15:58:08 +0800 Subject: [PATCH 127/258] [FLINK-26575][checkpoint] Improve the info message when restoring keyed state backend --- .../org/apache/flink/runtime/state/KeyGroupRangeOffsets.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeOffsets.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeOffsets.java index 0d9046a66d582..7ba4ac8f63d56 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeOffsets.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyGroupRangeOffsets.java @@ -216,8 +216,7 @@ public String toString() { return "KeyGroupRangeOffsets{" + "keyGroupRange=" + keyGroupRange - + ", offsets=" - + Arrays.toString(offsets) + + (offsets.length > 10 ? "" : ", offsets=" + Arrays.toString(offsets)) + '}'; } } From 341deb239c7ea974af237f41e444cf42defc7bbd Mon Sep 17 00:00:00 2001 From: liujiangang Date: Tue, 29 Mar 2022 15:51:07 +0800 Subject: [PATCH 128/258] [hotfix][doc] Modify spelling error in elastic_scaling.md --- docs/content.zh/docs/deployment/elastic_scaling.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content.zh/docs/deployment/elastic_scaling.md b/docs/content.zh/docs/deployment/elastic_scaling.md index f58b36c634d43..b44c538a58287 100644 --- a/docs/content.zh/docs/deployment/elastic_scaling.md +++ b/docs/content.zh/docs/deployment/elastic_scaling.md @@ -164,7 +164,7 @@ Adaptive Batch Scheduler 是一种可以自动推导每个算子并行度的批 #### 启用 Adaptive Batch Scheduler 为了启用 Adaptive Batch Scheduler, 你需要: -- 配置 `jobmanager.scheduler: AdpaptiveBatch` +- 配置 `jobmanager.scheduler: AdaptiveBatch` - 由于 ["只支持所有数据交换都为 BLOCKING 模式的作业"](#局限性-2), 需要将 [`execution.batch-shuffle-mode`]({{< ref "docs/deployment/config" >}}#execution-batch-shuffle-mode) 配置为 `ALL-EXCHANGES-BLOCKING`(默认值) 。 除此之外,使用 Adaptive Batch Scheduler 时,以下相关配置也可以调整: From 849b6e34131ed1822eacef6c3ce7b96298078e1c Mon Sep 17 00:00:00 2001 From: Alexander Fedulov <1492164+afedulov@users.noreply.github.com> Date: Mon, 28 Mar 2022 12:58:37 +0200 Subject: [PATCH 129/258] [hotfix] Fix CsvRowDataDeserializationSchema constructor --- .../flink/formats/csv/CsvRowDataDeserializationSchema.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataDeserializationSchema.java b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataDeserializationSchema.java index 2d58007a0b0e6..7dfc5650c99c4 100644 --- a/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataDeserializationSchema.java +++ b/flink-formats/flink-csv/src/main/java/org/apache/flink/formats/csv/CsvRowDataDeserializationSchema.java @@ -72,7 +72,7 @@ private CsvRowDataDeserializationSchema( this.resultTypeInfo = resultTypeInfo; this.runtimeConverter = new CsvToRowDataConverters(ignoreParseErrors).createRowConverter(rowType, true); - this.csvSchema = CsvRowSchemaConverter.convert(rowType); + this.csvSchema = csvSchema; this.objectReader = new CsvMapper().readerFor(JsonNode.class).with(csvSchema); this.ignoreParseErrors = ignoreParseErrors; } From f6a49a5627acb10c5c32b7d7e95b79c3a2a1cdd8 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Wed, 12 Jan 2022 22:29:37 +0800 Subject: [PATCH 130/258] [FLINK-25440][doc][pulsar] Stop and Start cursor now all uses publishTime instead of eventTime; doc changed to reflect this change --- .../docs/connectors/datastream/pulsar.md | 8 +- .../docs/connectors/datastream/pulsar.md | 10 +- .../source/enumerator/cursor/StopCursor.java | 19 +++- .../cursor/start/TimestampStartCursor.java | 2 +- ...sor.java => EventTimestampStopCursor.java} | 6 +- .../stop/PublishTimestampStopCursor.java | 39 +++++++ .../enumerator/cursor/StopCursorTest.java | 105 ++++++++++++++++++ 7 files changed, 178 insertions(+), 11 deletions(-) rename flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/{TimestampStopCursor.java => EventTimestampStopCursor.java} (87%) create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java diff --git a/docs/content.zh/docs/connectors/datastream/pulsar.md b/docs/content.zh/docs/connectors/datastream/pulsar.md index 3ba5989c6cfec..8d44be88c71f9 100644 --- a/docs/content.zh/docs/connectors/datastream/pulsar.md +++ b/docs/content.zh/docs/connectors/datastream/pulsar.md @@ -239,11 +239,15 @@ Pulsar Source 默认情况下使用流的方式消费数据。除非任务失败 ```java StopCursor.afterMessageId(MessageId); ``` -- 停止于某个给定的消息时间戳。 +- 停止于某个给定的消息发布时间戳,比如 `Message.getPublishTime()`。 ```java - StopCursor.atEventTime(long); + StopCursor.atPublishTime(long); ``` +{{< hint warning >}} +StopCursor.atEventTime(long) 目前已经处于弃用状态。 +{{< /hint >}} + ### Source 配置项 除了前面提到的配置选项,Pulsar Source 还提供了丰富的选项供 Pulsar 专家使用,在 builder 类里通过 `setConfig(ConfigOption, T)` 和 `setConfig(Configuration)` 方法给定下述的全部配置。 diff --git a/docs/content/docs/connectors/datastream/pulsar.md b/docs/content/docs/connectors/datastream/pulsar.md index 11e62cc2acad0..0953753ee33fb 100644 --- a/docs/content/docs/connectors/datastream/pulsar.md +++ b/docs/content/docs/connectors/datastream/pulsar.md @@ -241,7 +241,7 @@ The Pulsar connector consumes from the latest available message if the message I ```java StartCursor.fromMessageId(MessageId, boolean); ``` -- Start from the specified message time by `Message.getEventTime()`. +- Start from the specified message time by `Message.getPublishTime()`. ```java StartCursor.fromMessageTime(long); ``` @@ -281,11 +281,15 @@ Built-in stop cursors include: ```java StopCursor.afterMessageId(MessageId); ``` -- Stop at the specified message time by `Message.getEventTime()`. +- Stop at the specified message time by `Message.getPublishTime()`. ```java - StopCursor.atEventTime(long); + StopCursor.atPublishTime(long); ``` +{{< hint warning >}} +StopCursor.atEventTime(long) is now deprecated. + {{< /hint >}} + ### Source Configurable Options In addition to configuration options described above, you can set arbitrary options for `PulsarClient`, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java index aaec14352dcf2..0bf46ce128274 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java @@ -19,10 +19,11 @@ package org.apache.flink.connector.pulsar.source.enumerator.cursor; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.EventTimestampStopCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.LatestMessageStopCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.MessageIdStopCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.NeverStopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.TimestampStopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.PublishTimestampStopCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.pulsar.client.admin.PulsarAdmin; @@ -63,15 +64,29 @@ static StopCursor latest() { return new LatestMessageStopCursor(); } + /** + * Stop when the messageId is equal or greater than the specified messageId. Message that is + * equal to the specified messageId will not be consumed. + */ static StopCursor atMessageId(MessageId messageId) { return new MessageIdStopCursor(messageId); } + /** + * Stop when the messageId is greater than the specified messageId. Message that is equal to the + * specified messageId will be consumed. + */ static StopCursor afterMessageId(MessageId messageId) { return new MessageIdStopCursor(messageId, false); } + @Deprecated static StopCursor atEventTime(long timestamp) { - return new TimestampStopCursor(timestamp); + return new EventTimestampStopCursor(timestamp); + } + + /** Stop when message publishTime is greater than the specified timestamp. */ + static StopCursor atPublishTime(long timestamp) { + return new PublishTimestampStopCursor(timestamp); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java index 88f3b5a0b4c74..eb4ea32ebb6b9 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java @@ -21,7 +21,7 @@ import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; -/** This cursor would left pulsar start consuming from a specific timestamp. */ +/** This cursor would left pulsar start consuming from a specific publish timestamp. */ public class TimestampStartCursor implements StartCursor { private static final long serialVersionUID = 5170578885838095320L; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/TimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java similarity index 87% rename from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/TimestampStopCursor.java rename to flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java index 534e77f54b78e..e425545de4412 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/TimestampStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java @@ -23,12 +23,12 @@ import org.apache.pulsar.client.api.Message; /** Stop consuming message at the given event time. */ -public class TimestampStopCursor implements StopCursor { - private static final long serialVersionUID = 3381576769339353027L; +public class EventTimestampStopCursor implements StopCursor { + private static final long serialVersionUID = 2391576769339369027L; private final long timestamp; - public TimestampStopCursor(long timestamp) { + public EventTimestampStopCursor(long timestamp) { this.timestamp = timestamp; } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java new file mode 100644 index 0000000000000..b598e7addd422 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.cursor.stop; + +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; + +import org.apache.pulsar.client.api.Message; + +/** Stop consuming message at the given publish time. */ +public class PublishTimestampStopCursor implements StopCursor { + private static final long serialVersionUID = 4386276745339324527L; + + private final long timestamp; + + public PublishTimestampStopCursor(long timestamp) { + this.timestamp = timestamp; + } + + @Override + public boolean shouldStop(Message message) { + return message.getPublishTime() >= timestamp; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java new file mode 100644 index 0000000000000..d003107793a84 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.cursor; + +import org.apache.flink.api.common.serialization.SimpleStringSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; +import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; + +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.Schema; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static java.util.Collections.singletonList; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; +import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.flinkSchema; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test different implementation of StopCursor. */ +public class StopCursorTest extends PulsarTestSuiteBase { + + @Test + void publishTimeStopCursor() throws IOException { + String topicName = randomAlphanumeric(5); + operator().createTopic(topicName, 2); + + PulsarOrderedPartitionSplitReader splitReader = + new PulsarOrderedPartitionSplitReader<>( + operator().client(), + operator().admin(), + sourceConfig(), + flinkSchema(new SimpleStringSchema())); + // send the first message and set the stopCursor to filter any late stopCursor + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.STRING, + randomAlphanumeric(10)); + long currentTimeStamp = System.currentTimeMillis(); + TopicPartition partition = new TopicPartition(topicName, 0, createFullRange()); + PulsarPartitionSplit split = + new PulsarPartitionSplit( + partition, + StopCursor.atPublishTime(currentTimeStamp), + MessageId.earliest, + null); + SplitsAddition addition = new SplitsAddition<>(singletonList(split)); + splitReader.handleSplitsChanges(addition); + // first fetch should have result + RecordsWithSplitIds> firstResult = splitReader.fetch(); + assertThat(firstResult.nextSplit()).isNotNull(); + assertThat(firstResult.nextRecordFromSplit()).isNotNull(); + assertThat(firstResult.finishedSplits()).isEmpty(); + // send the second message and expect it will not be received + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.STRING, + randomAlphanumeric(10)); + RecordsWithSplitIds> secondResult = splitReader.fetch(); + assertThat(secondResult.nextSplit()).isNotNull(); + assertThat(firstResult.nextRecordFromSplit()).isNull(); + assertThat(secondResult.finishedSplits()).isNotEmpty(); + } + + private SourceConfiguration sourceConfig() { + Configuration config = operator().config(); + config.set(PULSAR_MAX_FETCH_RECORDS, 1); + config.set(PULSAR_MAX_FETCH_TIME, 1000L); + config.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10)); + config.set(PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE, true); + return new SourceConfiguration(config); + } +} From 5a0f674f85cbbd909e22dd873e9beef79a658faa Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 22 Mar 2022 10:01:04 +0100 Subject: [PATCH 131/258] [FLINK-26798][runtime] Hardens test against unexpected heartbeat testJobFailureWhenTaskExecutorHeartbeatTimeout failed due to a heartbeat being processed during the test. The missing payload of the test implementation caused an unexpected error after the JobGraph was already running (and therefore, expected some executions being present in the ExecutionDeploymentReport of the TaskExecutorToJobManagerHeartbeatPayload. The heartbeatFromTaskManager call is not necessary in this test because we're simulating a timeout of the heartbeat, anyway. Removing this call fixes the issue. --- .../runtime/jobmaster/JobMasterTest.java | 26 +++---------------- 1 file changed, 3 insertions(+), 23 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index 50a97001b19ab..f2c0864b8ff05 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -98,7 +98,6 @@ import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway; -import org.apache.flink.runtime.taskexecutor.TaskExecutorToJobManagerHeartbeatPayload; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway; import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder; import org.apache.flink.runtime.taskexecutor.slot.SlotOffer; @@ -156,7 +155,6 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; -import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -1752,9 +1750,7 @@ public void testJobFailureWhenGracefulTaskExecutorTermination() throws Exception (localTaskManagerLocation, jobMasterGateway) -> jobMasterGateway.disconnectTaskManager( localTaskManagerLocation.getResourceID(), - new FlinkException("Test disconnectTaskManager exception.")), - (jobMasterGateway, resourceID) -> - (ignoredA, ignoredB) -> FutureUtils.completedVoidFuture()); + new FlinkException("Test disconnectTaskManager exception."))); } @Test @@ -1766,14 +1762,7 @@ public void testJobFailureWhenTaskExecutorHeartbeatTimeout() throws Exception { testingHeartbeatService, (localTaskManagerLocation, jobMasterGateway) -> testingHeartbeatService.triggerHeartbeatTimeout( - jmResourceId, localTaskManagerLocation.getResourceID()), - (jobMasterGateway, taskManagerResourceId) -> - (resourceId, ignored) -> { - jobMasterGateway.heartbeatFromTaskManager( - taskManagerResourceId, - TaskExecutorToJobManagerHeartbeatPayload.empty()); - return FutureUtils.completedVoidFuture(); - }); + jmResourceId, localTaskManagerLocation.getResourceID())); } /** @@ -1987,12 +1976,7 @@ public void testJobMasterAcceptsSlotsWhenJobIsRestarting() throws Exception { private void runJobFailureWhenTaskExecutorTerminatesTest( HeartbeatServices heartbeatServices, - BiConsumer jobReachedRunningState, - BiFunction< - JobMasterGateway, - ResourceID, - BiFunction>> - heartbeatConsumerFunction) + BiConsumer jobReachedRunningState) throws Exception { final JobGraph jobGraph = JobGraphTestUtils.singleNoOpJobGraph(); final JobMasterBuilder.TestingOnCompletionActions onCompletionActions = @@ -2024,10 +2008,6 @@ private void runJobFailureWhenTaskExecutorTerminatesTest( taskDeploymentDescriptor.getExecutionAttemptId()); return CompletableFuture.completedFuture(Acknowledge.get()); }) - .setHeartbeatJobManagerFunction( - heartbeatConsumerFunction.apply( - jobMasterGateway, - taskManagerUnresolvedLocation.getResourceID())) .createTestingTaskExecutorGateway(); final Collection slotOffers = From 2e7cc4e38797388161be574debccbfc7f042e377 Mon Sep 17 00:00:00 2001 From: Qingsheng Ren Date: Fri, 1 Apr 2022 15:56:04 +0800 Subject: [PATCH 132/258] [FLINK-26928][connector/kafka] Remove unnecessary docker network creation in tests and close network on shutdown() This closes #19296. --- .../connectors/kafka/KafkaTestEnvironmentImpl.java | 13 +++++++++---- .../connectors/kafka/table/KafkaTableTestBase.java | 3 --- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java index 75791e58c5e8c..adfbdce6c9557 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestEnvironmentImpl.java @@ -78,6 +78,7 @@ public class KafkaTestEnvironmentImpl extends KafkaTestEnvironment { private final Map brokers = new HashMap<>(); private final Set pausedBroker = new HashSet<>(); private @Nullable GenericContainer zookeeper; + private @Nullable Network network; private String brokerConnectionString = ""; private Properties standardProps; private FlinkKafkaProducer.Semantic producerSemantic = FlinkKafkaProducer.Semantic.EXACTLY_ONCE; @@ -356,6 +357,10 @@ public void shutdown() throws Exception { if (zookeeper != null) { zookeeper.stop(); } + + if (network != null) { + network.close(); + } } private class KafkaOffsetHandlerImpl implements KafkaOffsetHandler { @@ -397,14 +402,14 @@ public void close() { } private void startKafkaContainerCluster(int numBrokers) { - Network network = Network.newNetwork(); if (numBrokers > 1) { + network = Network.newNetwork(); zookeeper = createZookeeperContainer(network); zookeeper.start(); LOG.info("Zookeeper container started"); } for (int brokerID = 0; brokerID < numBrokers; brokerID++) { - KafkaContainer broker = createKafkaContainer(network, brokerID, zookeeper); + KafkaContainer broker = createKafkaContainer(brokerID, zookeeper); brokers.put(brokerID, broker); } new ArrayList<>(brokers.values()).parallelStream().forEach(GenericContainer::start); @@ -426,11 +431,10 @@ private GenericContainer createZookeeperContainer(Network network) { } private KafkaContainer createKafkaContainer( - Network network, int brokerID, @Nullable GenericContainer zookeeper) { + int brokerID, @Nullable GenericContainer zookeeper) { String brokerName = String.format("Kafka-%d", brokerID); KafkaContainer broker = KafkaUtil.createKafkaContainer(DockerImageVersions.KAFKA, LOG, brokerName) - .withNetwork(network) .withNetworkAliases(brokerName) .withEnv("KAFKA_BROKER_ID", String.valueOf(brokerID)) .withEnv("KAFKA_MESSAGE_MAX_BYTES", String.valueOf(50 * 1024 * 1024)) @@ -447,6 +451,7 @@ private KafkaContainer createKafkaContainer( if (zookeeper != null) { broker.dependsOn(zookeeper) + .withNetwork(zookeeper.getNetwork()) .withExternalZookeeper( String.format("%s:%d", ZOOKEEPER_HOSTNAME, ZOOKEEPER_PORT)); } else { diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java index d5810e2351687..20d03af74d297 100644 --- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java +++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java @@ -41,7 +41,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; -import org.testcontainers.containers.Network; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.utility.DockerImageName; @@ -63,7 +62,6 @@ public abstract class KafkaTableTestBase extends AbstractTestBase { private static final Logger LOG = LoggerFactory.getLogger(KafkaTableTestBase.class); private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; - private static final Network NETWORK = Network.newNetwork(); private static final int zkTimeoutMills = 30000; @ClassRule @@ -77,7 +75,6 @@ protected void doStart() { } } }.withEmbeddedZookeeper() - .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS) .withEnv( "KAFKA_TRANSACTION_MAX_TIMEOUT_MS", From f6a617b50f618af8f5284acca64d61328b4a7448 Mon Sep 17 00:00:00 2001 From: Tartarus0zm Date: Fri, 1 Apr 2022 11:33:35 +0800 Subject: [PATCH 133/258] [FLINK-26460][table-planner] Fix Unsupported type when convertTypeToSpec: MAP (cherry picked from commit 33e7c84fb0f6aadf9d228c41c0ba6808634a7e36) This closes #1896 --- .../planner/calcite/PreValidateReWriter.scala | 22 +++++++++--- .../planner/plan/stream/sql/TableSinkTest.xml | 16 +++++++++ .../plan/stream/sql/TableSinkTest.scala | 21 +++++++++++ .../runtime/stream/sql/TableSinkITCase.scala | 36 +++++++++++++++++++ 4 files changed, 91 insertions(+), 4 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/PreValidateReWriter.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/PreValidateReWriter.scala index 39c0aa5c000a1..3d930b0968b06 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/PreValidateReWriter.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/calcite/PreValidateReWriter.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.calcite import org.apache.flink.sql.parser.SqlProperty import org.apache.flink.sql.parser.dml.RichSqlInsert import org.apache.flink.sql.parser.dql.SqlRichExplain +import org.apache.flink.sql.parser.`type`.SqlMapTypeNameSpec import org.apache.flink.table.api.ValidationException import org.apache.flink.table.planner.calcite.PreValidateReWriter.{appendPartitionAndNullsProjects, notSupported} import org.apache.flink.table.planner.plan.schema.{CatalogSourceTable, FlinkPreparingTableBase, LegacyCatalogSourceTable} @@ -30,12 +31,12 @@ import org.apache.calcite.plan.RelOptTable import org.apache.calcite.prepare.CalciteCatalogReader import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeFactory, RelDataTypeField} import org.apache.calcite.runtime.{CalciteContextException, Resources} -import org.apache.calcite.sql.`type`.SqlTypeUtil +import org.apache.calcite.sql.`type`.{SqlTypeName, SqlTypeUtil} import org.apache.calcite.sql.fun.SqlStdOperatorTable import org.apache.calcite.sql.parser.SqlParserPos import org.apache.calcite.sql.util.SqlBasicVisitor import org.apache.calcite.sql.validate.{SqlValidatorException, SqlValidatorTable, SqlValidatorUtil} -import org.apache.calcite.sql.{SqlCall, SqlIdentifier, SqlKind, SqlLiteral, SqlNode, SqlNodeList, SqlOrderBy, SqlSelect, SqlUtil} +import org.apache.calcite.sql.{SqlCall, SqlDataTypeSpec, SqlIdentifier, SqlKind, SqlLiteral, SqlNode, SqlNodeList, SqlOrderBy, SqlSelect, SqlUtil} import org.apache.calcite.util.Static.RESOURCE import java.util @@ -391,8 +392,21 @@ object PreValidateReWriter { == desiredType)) { node } else { - SqlStdOperatorTable.CAST.createCall(SqlParserPos.ZERO, - node, SqlTypeUtil.convertTypeToSpec(desiredType)) + // See FLINK-26460 for more details + val sqlDataTypeSpec = + if (SqlTypeUtil.isNull(currentType) && SqlTypeUtil.isMap(desiredType)) { + val keyType = desiredType.getKeyType + val valueType = desiredType.getValueType + new SqlDataTypeSpec( + new SqlMapTypeNameSpec( + SqlTypeUtil.convertTypeToSpec(keyType).withNullable(keyType.isNullable), + SqlTypeUtil.convertTypeToSpec(valueType).withNullable(valueType.isNullable), + SqlParserPos.ZERO), + SqlParserPos.ZERO) + } else { + SqlTypeUtil.convertTypeToSpec(desiredType) + } + SqlStdOperatorTable.CAST.createCall(SqlParserPos.ZERO, node, sqlDataTypeSpec) } } } diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml index b7dea9f32617b..78d2822cd4a9d 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml @@ -748,4 +748,20 @@ Sink(table=[default_catalog.default_database.SinkJoinChangeLog], fields=[person, ]]> + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala index f760fb5b75dec..e8bfaeef453e1 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala @@ -771,6 +771,27 @@ class TableSinkTest extends TableTestBase { util.verifyAstPlan(stmtSet, ExplainDetail.CHANGELOG_MODE) } + + @Test + def testInsertPartColumn(): Unit = { + util.addTable( + s""" + |CREATE TABLE zm_test ( + | `a` BIGINT, + | `m1` MAP, + | `m2` MAP, + | `m3` MAP, + | `m4` MAP + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'true' + |) + |""".stripMargin) + val stmtSet = util.tableEnv.createStatementSet() + stmtSet.addInsertSql( + "INSERT INTO zm_test(`a`) SELECT `a` FROM MyTable") + util.verifyRelPlan(stmtSet, ExplainDetail.CHANGELOG_MODE) + } } /** tests table factory use ParallelSourceFunction which support parallelism by env*/ diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala index 20b61a3630ac8..a05687312cec8 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSinkITCase.scala @@ -154,4 +154,40 @@ class TableSinkITCase(mode: StateBackendMode) extends StreamingWithStateTestBase val expected = List("+I[jason, 4]") assertEquals(expected.sorted, result.sorted) } + + @Test + def testInsertPartColumn(): Unit = { + tEnv.executeSql( + """ + |CREATE TABLE zm_test ( + | `person` String, + | `votes` BIGINT, + | `m1` MAP, + | `m2` MAP, + | `m3` MAP, + | `m4` MAP + |) WITH ( + | 'connector' = 'values', + | 'sink-insert-only' = 'true' + |) + |""".stripMargin) + + tEnv.executeSql( + """ + |insert into zm_test(`person`, `votes`) + | select + | `person`, + | `votes` + | from + | src + |""".stripMargin).await() + + val result = TestValuesTableFactory.getResults("zm_test") + val expected = List( + "+I[jason, 1, null, null, null, null]", + "+I[jason, 1, null, null, null, null]", + "+I[jason, 1, null, null, null, null]", + "+I[jason, 1, null, null, null, null]") + assertEquals(expected.sorted, result.sorted) + } } From 4a3d6e52fe820e53b7f3d9bf301b4a1c7d14ab41 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 1 Apr 2022 15:42:03 +0800 Subject: [PATCH 134/258] [FLINK-26920][python] Handles the dependencies properly in create_temporary_view This closes #19318. --- .../pyflink/table/table_environment.py | 45 ++++++++++--------- .../python/beam/BeamPythonFunctionRunner.java | 7 +-- 2 files changed, 29 insertions(+), 23 deletions(-) diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index e001f36d37d90..07ed66c3c827e 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -1156,27 +1156,32 @@ def create_temporary_view(self, """ if isinstance(table_or_data_stream, Table): self._j_tenv.createTemporaryView(view_path, table_or_data_stream._j_table) - elif len(fields_or_schema) == 0: - self._j_tenv.createTemporaryView(view_path, table_or_data_stream._j_data_stream) - elif len(fields_or_schema) == 1 and isinstance(fields_or_schema[0], str): - self._j_tenv.createTemporaryView( - view_path, - table_or_data_stream._j_data_stream, - fields_or_schema[0]) - elif len(fields_or_schema) == 1 and isinstance(fields_or_schema[0], Schema): - self._j_tenv.createTemporaryView( - view_path, - table_or_data_stream._j_data_stream, - fields_or_schema[0]._j_schema) - elif (len(fields_or_schema) > 0 and - all(isinstance(elem, Expression) for elem in fields_or_schema)): - self._j_tenv.createTemporaryView( - view_path, - table_or_data_stream._j_data_stream, - to_expression_jarray(fields_or_schema)) else: - raise ValueError("Invalid arguments for 'fields': %r" % - ','.join([repr(item) for item in fields_or_schema])) + j_data_stream = table_or_data_stream._j_data_stream + JPythonConfigUtil = get_gateway().jvm.org.apache.flink.python.util.PythonConfigUtil + JPythonConfigUtil.configPythonOperator(j_data_stream.getExecutionEnvironment()) + + if len(fields_or_schema) == 0: + self._j_tenv.createTemporaryView(view_path, j_data_stream) + elif len(fields_or_schema) == 1 and isinstance(fields_or_schema[0], str): + self._j_tenv.createTemporaryView( + view_path, + j_data_stream, + fields_or_schema[0]) + elif len(fields_or_schema) == 1 and isinstance(fields_or_schema[0], Schema): + self._j_tenv.createTemporaryView( + view_path, + j_data_stream, + fields_or_schema[0]._j_schema) + elif (len(fields_or_schema) > 0 and + all(isinstance(elem, Expression) for elem in fields_or_schema)): + self._j_tenv.createTemporaryView( + view_path, + j_data_stream, + to_expression_jarray(fields_or_schema)) + else: + raise ValueError("Invalid arguments for 'fields': %r" % + ','.join([repr(item) for item in fields_or_schema])) def add_python_file(self, file_path: str): """ diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java index 86264ab90eaa6..da4e111cdfabd 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java @@ -232,9 +232,10 @@ public void open(PythonConfig config) throws Exception { if (memoryManager != null && config.isUsingManagedMemory()) { Preconditions.checkArgument( managedMemoryFraction > 0 && managedMemoryFraction <= 1.0, - "The configured managed memory fraction for Python worker process must be within (0, 1], was: %s. " - + "It may be because the consumer type \"Python\" was missing or set to 0 for the config option \"taskmanager.memory.managed.consumer-weights\"." - + managedMemoryFraction); + String.format( + "The configured managed memory fraction for Python worker process must be within (0, 1], was: %s. " + + "It may be because the consumer type \"Python\" was missing or set to 0 for the config option \"taskmanager.memory.managed.consumer-weights\".", + managedMemoryFraction)); final LongFunctionWithException initializer = (size) -> From 02c5e4136c809eac7b5d723be0d043b639ddf477 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 1 Apr 2022 19:23:43 +0800 Subject: [PATCH 135/258] [FLINK-26986][python] Remove deprecated string expressions in Python Table API This closes #19326. --- .../table/python_table_api_connectors.md | 2 +- .../docs/dev/python/table/udfs/python_udfs.md | 12 +- .../table/udfs/vectorized_python_udfs.md | 3 +- docs/content.zh/docs/dev/table/catalogs.md | 2 +- docs/content.zh/docs/dev/table/tableApi.md | 15 +- .../table/python_table_api_connectors.md | 2 +- .../docs/dev/python/table/udfs/python_udfs.md | 12 +- .../table/udfs/vectorized_python_udfs.md | 3 +- docs/content/docs/dev/table/catalogs.md | 2 +- docs/content/docs/dev/table/tableApi.md | 15 +- .../examples/table/pandas/pandas_udaf.py | 2 +- .../examples/table/windowing/over_window.py | 6 +- .../table/windowing/session_window.py | 2 +- .../table/windowing/sliding_window.py | 2 +- .../examples/table/windowing/tumble_window.py | 2 +- flink-python/pyflink/table/table.py | 214 +++++------------- flink-python/pyflink/table/window.py | 7 - 17 files changed, 99 insertions(+), 204 deletions(-) diff --git a/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md b/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md index dbf94fbd1eee9..6be2c1e727514 100644 --- a/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md +++ b/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md @@ -139,7 +139,7 @@ import numpy as np # 创建一个 PyFlink 表 pdf = pd.DataFrame(np.random.rand(1000, 2)) -table = t_env.from_pandas(pdf, ["a", "b"]).filter("a > 0.5") +table = t_env.from_pandas(pdf, ["a", "b"]).filter(col('a') > 0.5) # 将 PyFlink 表转换成 Pandas DataFrame pdf = table.to_pandas() diff --git a/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md b/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md index 124d1bbfcc7ef..f16c433decca5 100644 --- a/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md +++ b/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md @@ -128,7 +128,7 @@ add = udf(functools.partial(partial_add, k=1), result_type=DataTypes.BIGINT()) # 注册 Python 自定义函数 table_env.create_temporary_function("add", add) # 在 Python Table API 中使用 Python 自定义函数 -my_table.select("add(a, b)") +my_table.select(call('add', my_table.a, my_table.b)) # 也可以在 Python Table API 中直接使用 Python 自定义函数 my_table.select(add(my_table.a, my_table.b)) @@ -156,8 +156,8 @@ my_table = ... # type: Table, table schema: [a: String] split = udtf(Split(), result_types=[DataTypes.STRING(), DataTypes.INT()]) # 在 Python Table API 中使用 Python 表值函数 -my_table.join_lateral(split(my_table.a).alias("word, length")) -my_table.left_outer_join_lateral(split(my_table.a).alias("word, length")) +my_table.join_lateral(split(my_table.a).alias("word", "length")) +my_table.left_outer_join_lateral(split(my_table.a).alias("word", "length")) # 在 SQL API 中使用 Python 表值函数 table_env.create_temporary_function("split", udtf(Split(), result_types=[DataTypes.STRING(), DataTypes.INT()])) @@ -194,8 +194,8 @@ my_table = ... # type: Table, table schema: [a: String] table_env.create_java_temporary_function("split", "my.java.function.Split") # 在 Python Table API 中使用表值函数。 "alias"指定表的字段名称。 -my_table.join_lateral(call('split', my_table.a).alias("word, length")).select(my_table.a, col('word'), col('length')) -my_table.left_outer_join_lateral(call('split', my_table.a).alias("word, length")).select(my_table.a, col('word'), col('length')) +my_table.join_lateral(call('split', my_table.a).alias("word", "length")).select(my_table.a, col('word'), col('length')) +my_table.left_outer_join_lateral(call('split', my_table.a).alias("word", "length")).select(my_table.a, col('word'), col('length')) # 注册 Python 函数。 @@ -337,7 +337,7 @@ tumble_window = Tumble.over(lit(1).hours) \ result = t.window(tumble_window) \ .group_by(col('w'), col('name')) \ - .select("w.start, w.end, weighted_avg(value, count)") \ + .select(col('w').start, col('w').end, weighted_avg(col('value'), col('count'))) \ .to_pandas() print(result) diff --git a/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md b/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md index af6c7f60c15b1..10012d8ba4a53 100644 --- a/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md +++ b/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md @@ -101,8 +101,7 @@ tumble_window = Tumble.over(expr.lit(1).hours) \ my_table.window(tumble_window) \ .group_by("w") \ - .select("w.start, w.end, mean_udaf(b)") - + .select(col('w').start, col('w').end, mean_udaf(col('b'))) # 在 Over Window Aggregation 中使用向量化聚合函数 table_env.create_temporary_function("mean_udaf", mean_udaf) diff --git a/docs/content.zh/docs/dev/table/catalogs.md b/docs/content.zh/docs/dev/table/catalogs.md index 0b86fbd6769a2..05722d87f651e 100644 --- a/docs/content.zh/docs/dev/table/catalogs.md +++ b/docs/content.zh/docs/dev/table/catalogs.md @@ -241,7 +241,7 @@ schema = Schema.new_builder() \ catalog_table = t_env.create_table("myhive.mydb.mytable", TableDescriptor.for_connector("kafka") .schema(schema) - // … + # … .build()) # tables should contain "mytable" diff --git a/docs/content.zh/docs/dev/table/tableApi.md b/docs/content.zh/docs/dev/table/tableApi.md index 65a9f9d87a257..d2b198ca29c04 100644 --- a/docs/content.zh/docs/dev/table/tableApi.md +++ b/docs/content.zh/docs/dev/table/tableApi.md @@ -417,7 +417,7 @@ val orders: Table = tableEnv.from("Orders").as("x", "y", "z", "t") {{< tab "Python" >}} ```python orders = t_env.from_path("Orders") -result = orders.alias("x, y, z, t") +result = orders.alias("x", "y", "z", "t") ``` {{< /tab >}} {{< /tabs >}} @@ -1054,7 +1054,7 @@ def split(x): # join orders = t_env.from_path("Orders") -joined_table = orders.join_lateral(split(orders.c).alias("s, t, v")) +joined_table = orders.join_lateral(split(orders.c).alias("s", "t", "v")) result = joined_table.select(joined_table.a, joined_table.b, joined_table.s, joined_table.t, joined_table.v) ``` {{< /tab >}} @@ -1103,7 +1103,7 @@ def split(x): # join orders = t_env.from_path("Orders") -joined_table = orders.left_outer_join_lateral(split(orders.c).alias("s, t, v")) +joined_table = orders.left_outer_join_lateral(split(orders.c).alias("s", "t", "v")) result = joined_table.select(joined_table.a, joined_table.b, joined_table.s, joined_table.t, joined_table.v) ``` {{< /tab >}} @@ -2453,7 +2453,7 @@ agg = udaf(function, # 使用 python 通用聚合函数进行聚合 result = t.group_by(t.a) \ .aggregate(agg.alias("c", "d")) \ - .select("a, c, d") + .select(col('a'), col('c'), col('d')) # 使用 python 向量化聚合函数进行聚合 pandas_udaf = udaf(lambda pd: (pd.b.mean(), pd.b.max()), @@ -2462,8 +2462,7 @@ pandas_udaf = udaf(lambda pd: (pd.b.mean(), pd.b.max()), DataTypes.FIELD("b", DataTypes.INT())]), func_type="pandas") t.aggregate(pandas_udaf.alias("a", "b")) \ - .select("a, b") - + .select(col('a'), col('b')) ``` {{< /tab >}} @@ -2515,9 +2514,9 @@ tumble_window = Tumble.over(expr.lit(1).hours) \ .alias("w") t.select(t.b, t.rowtime) \ .window(tumble_window) \ - .group_by("w") \ + .group_by(col("w")) \ .aggregate(pandas_udaf.alias("d", "e")) \ - .select("w.rowtime, d, e") + .select(col('w').rowtime, col('d'), col('e')) ``` {{< /tab >}} {{< /tabs >}} diff --git a/docs/content/docs/dev/python/table/python_table_api_connectors.md b/docs/content/docs/dev/python/table/python_table_api_connectors.md index a7f31d30cfb14..b3ee2481ab843 100644 --- a/docs/content/docs/dev/python/table/python_table_api_connectors.md +++ b/docs/content/docs/dev/python/table/python_table_api_connectors.md @@ -143,7 +143,7 @@ import numpy as np # Create a PyFlink Table pdf = pd.DataFrame(np.random.rand(1000, 2)) -table = t_env.from_pandas(pdf, ["a", "b"]).filter("a > 0.5") +table = t_env.from_pandas(pdf, ["a", "b"]).filter(col('a') > 0.5) # Convert the PyFlink Table to a Pandas DataFrame pdf = table.to_pandas() diff --git a/docs/content/docs/dev/python/table/udfs/python_udfs.md b/docs/content/docs/dev/python/table/udfs/python_udfs.md index 745124b7bc99b..c33a51bf90038 100644 --- a/docs/content/docs/dev/python/table/udfs/python_udfs.md +++ b/docs/content/docs/dev/python/table/udfs/python_udfs.md @@ -129,7 +129,7 @@ add = udf(functools.partial(partial_add, k=1), result_type=DataTypes.BIGINT()) # register the Python function table_env.create_temporary_function("add", add) # use the function in Python Table API -my_table.select("add(a, b)") +my_table.select(call('add', my_table.a, my_table.b)) # You can also use the Python function in Python Table API directly my_table.select(add(my_table.a, my_table.b)) @@ -158,8 +158,8 @@ my_table = ... # type: Table, table schema: [a: String] split = udtf(Split(), result_types=[DataTypes.STRING(), DataTypes.INT()]) # use the Python Table Function in Python Table API -my_table.join_lateral(split(my_table.a).alias("word, length")) -my_table.left_outer_join_lateral(split(my_table.a).alias("word, length")) +my_table.join_lateral(split(my_table.a).alias("word", "length")) +my_table.left_outer_join_lateral(split(my_table.a).alias("word", "length")) # use the Python Table function in SQL API table_env.create_temporary_function("split", udtf(Split(), result_types=[DataTypes.STRING(), DataTypes.INT()])) @@ -196,8 +196,8 @@ my_table = ... # type: Table, table schema: [a: String] table_env.create_java_temporary_function("split", "my.java.function.Split") # Use the table function in the Python Table API. "alias" specifies the field names of the table. -my_table.join_lateral(call('split', my_table.a).alias("word, length")).select(my_table.a, col('word'), col('length')) -my_table.left_outer_join_lateral(call('split', my_table.a).alias("word, length")).select(my_table.a, col('word'), col('length')) +my_table.join_lateral(call('split', my_table.a).alias("word", "length")).select(my_table.a, col('word'), col('length')) +my_table.left_outer_join_lateral(call('split', my_table.a).alias("word", "length")).select(my_table.a, col('word'), col('length')) # Register the python function. @@ -338,7 +338,7 @@ tumble_window = Tumble.over(lit(1).hours) \ result = t.window(tumble_window) \ .group_by(col('w'), col('name')) \ - .select("w.start, w.end, weighted_avg(value, count)") \ + .select(col('w').start, col('w').end, weighted_avg(col('value'), col('count'))) \ .to_pandas() print(result) diff --git a/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md b/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md index 7b79eabab8eff..c3461b342f4f9 100644 --- a/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md +++ b/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md @@ -100,8 +100,7 @@ tumble_window = Tumble.over(expr.lit(1).hours) \ my_table.window(tumble_window) \ .group_by("w") \ - .select("w.start, w.end, mean_udaf(b)") - + .select(col('w').start, col('w').end, mean_udaf(col('b'))) # use the vectorized Python aggregate function in Over Window Aggregation table_env.create_temporary_function("mean_udaf", mean_udaf) diff --git a/docs/content/docs/dev/table/catalogs.md b/docs/content/docs/dev/table/catalogs.md index d0b7ab6be6416..f166ab5f61b5d 100644 --- a/docs/content/docs/dev/table/catalogs.md +++ b/docs/content/docs/dev/table/catalogs.md @@ -245,7 +245,7 @@ schema = Schema.new_builder() \ catalog_table = t_env.create_table("myhive.mydb.mytable", TableDescriptor.for_connector("kafka") .schema(schema) - // … + # … .build()) # tables should contain "mytable" diff --git a/docs/content/docs/dev/table/tableApi.md b/docs/content/docs/dev/table/tableApi.md index d4f132c5afbce..91ab6cf49c121 100644 --- a/docs/content/docs/dev/table/tableApi.md +++ b/docs/content/docs/dev/table/tableApi.md @@ -418,7 +418,7 @@ val orders: Table = tableEnv.from("Orders").as("x", "y", "z", "t") {{< tab "Python" >}} ```python orders = t_env.from_path("Orders") -result = orders.alias("x, y, z, t") +result = orders.alias("x", "y", "z", "t") ``` {{< /tab >}} {{< /tabs >}} @@ -1053,7 +1053,7 @@ def split(x): # join orders = t_env.from_path("Orders") -joined_table = orders.join_lateral(split(orders.c).alias("s, t, v")) +joined_table = orders.join_lateral(split(orders.c).alias("s", "t", "v")) result = joined_table.select(joined_table.a, joined_table.b, joined_table.s, joined_table.t, joined_table.v) ``` {{< /tab >}} @@ -1102,7 +1102,7 @@ def split(x): # join orders = t_env.from_path("Orders") -joined_table = orders.left_outer_join_lateral(split(orders.c).alias("s, t, v")) +joined_table = orders.left_outer_join_lateral(split(orders.c).alias("s", "t", "v")) result = joined_table.select(joined_table.a, joined_table.b, joined_table.s, joined_table.t, joined_table.v) ``` {{< /tab >}} @@ -2452,7 +2452,7 @@ agg = udaf(function, # aggregate with a python general aggregate function result = t.group_by(t.a) \ .aggregate(agg.alias("c", "d")) \ - .select("a, c, d") + select(col('a'), col('c'), col('d')) # aggregate with a python vectorized aggregate function pandas_udaf = udaf(lambda pd: (pd.b.mean(), pd.b.max()), @@ -2461,8 +2461,7 @@ pandas_udaf = udaf(lambda pd: (pd.b.mean(), pd.b.max()), DataTypes.FIELD("b", DataTypes.INT())]), func_type="pandas") t.aggregate(pandas_udaf.alias("a", "b")) \ - .select("a, b") - + select(col('a'), col('b')) ``` {{< /tab >}} @@ -2515,9 +2514,9 @@ tumble_window = Tumble.over(expr.lit(1).hours) \ .alias("w") t.select(t.b, t.rowtime) \ .window(tumble_window) \ - .group_by("w") \ + .group_by(col("w")) \ .aggregate(pandas_udaf.alias("d", "e")) \ - .select("w.rowtime, d, e") + .select(col('w').rowtime, col('d'), col('e')) ``` {{< /tab >}} {{< /tabs >}} diff --git a/flink-python/pyflink/examples/table/pandas/pandas_udaf.py b/flink-python/pyflink/examples/table/pandas/pandas_udaf.py index e4e8b9ea5222b..4b884dd480341 100644 --- a/flink-python/pyflink/examples/table/pandas/pandas_udaf.py +++ b/flink-python/pyflink/examples/table/pandas/pandas_udaf.py @@ -55,7 +55,7 @@ def pandas_udaf(): .column("f2", DataTypes.FLOAT()) .watermark("ts", "ts - INTERVAL '3' SECOND") .build() - ).alias("ts, name, price") + ).alias("ts", "name", "price") # define the sink t_env.create_temporary_table( diff --git a/flink-python/pyflink/examples/table/windowing/over_window.py b/flink-python/pyflink/examples/table/windowing/over_window.py index 982d6b860d468..5fd736b529a20 100644 --- a/flink-python/pyflink/examples/table/windowing/over_window.py +++ b/flink-python/pyflink/examples/table/windowing/over_window.py @@ -54,7 +54,7 @@ def tumble_window_demo(): .column("f2", DataTypes.FLOAT()) .watermark("ts", "ts - INTERVAL '3' SECOND") .build() - ).alias("ts, name, price") + ).alias("ts", "name", "price") # define the sink t_env.create_temporary_table( @@ -68,8 +68,8 @@ def tumble_window_demo(): # define the over window operation table = table.over_window( - Over.partition_by("name") - .order_by("ts") + Over.partition_by(col("name")) + .order_by(col("ts")) .preceding(row_interval(2)) .following(CURRENT_ROW) .alias('w')) \ diff --git a/flink-python/pyflink/examples/table/windowing/session_window.py b/flink-python/pyflink/examples/table/windowing/session_window.py index 5b40a7b9af8bd..49e4680938913 100644 --- a/flink-python/pyflink/examples/table/windowing/session_window.py +++ b/flink-python/pyflink/examples/table/windowing/session_window.py @@ -52,7 +52,7 @@ def session_window_demo(): .column("f2", DataTypes.FLOAT()) .watermark("ts", "ts - INTERVAL '3' SECOND") .build() - ).alias("ts, name, price") + ).alias("ts", "name", "price") # define the sink t_env.create_temporary_table( diff --git a/flink-python/pyflink/examples/table/windowing/sliding_window.py b/flink-python/pyflink/examples/table/windowing/sliding_window.py index 1b8bb150fd215..fc460c67df469 100644 --- a/flink-python/pyflink/examples/table/windowing/sliding_window.py +++ b/flink-python/pyflink/examples/table/windowing/sliding_window.py @@ -54,7 +54,7 @@ def sliding_window_demo(): .column("f2", DataTypes.FLOAT()) .watermark("ts", "ts - INTERVAL '3' SECOND") .build() - ).alias("ts, name, price") + ).alias("ts", "name", "price") # define the sink t_env.create_temporary_table( diff --git a/flink-python/pyflink/examples/table/windowing/tumble_window.py b/flink-python/pyflink/examples/table/windowing/tumble_window.py index dd3ba2ea408b9..c7787473451e0 100644 --- a/flink-python/pyflink/examples/table/windowing/tumble_window.py +++ b/flink-python/pyflink/examples/table/windowing/tumble_window.py @@ -54,7 +54,7 @@ def tumble_window_demo(): .column("f2", DataTypes.FLOAT()) .watermark("ts", "ts - INTERVAL '3' SECOND") .build() - ).alias("ts, name, price") + ).alias("ts", "name", "price") # define the sink t_env.create_temporary_table( diff --git a/flink-python/pyflink/table/table.py b/flink-python/pyflink/table/table.py index b7d8ae60673f5..ed46d8afe53d7 100644 --- a/flink-python/pyflink/table/table.py +++ b/flink-python/pyflink/table/table.py @@ -90,7 +90,7 @@ def __getattr__(self, name) -> Expression: % (name, ', '.join(self.get_schema().get_field_names()))) return col(name) - def select(self, *fields: Union[str, Expression]) -> 'Table': + def select(self, *fields: Expression) -> 'Table': """ Performs a selection operation. Similar to a SQL SELECT statement. The field expressions can contain complex expressions. @@ -102,16 +102,9 @@ def select(self, *fields: Union[str, Expression]) -> 'Table': >>> tab.select(tab.key, expr.concat(tab.value, 'hello')) >>> tab.select(expr.col('key'), expr.concat(expr.col('value'), 'hello')) - >>> tab.select("key, value + 'hello'") - :return: The result table. """ - if all(isinstance(f, Expression) for f in fields): - return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return Table(self._j_table.select(fields[0]), self._t_env) + return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) def alias(self, field: str, *fields: str) -> 'Table': """ @@ -122,7 +115,6 @@ def alias(self, field: str, *fields: str) -> 'Table': :: >>> tab.alias("a", "b", "c") - >>> tab.alias("a, b, c") :param field: Field alias. :param fields: Additional field aliases. @@ -141,7 +133,6 @@ def filter(self, predicate: Union[str, Expression[bool]]) -> 'Table': :: >>> tab.filter(tab.name == 'Fred') - >>> tab.filter("name = 'Fred'") :param predicate: Predicate expression string. :return: The result table. @@ -157,14 +148,13 @@ def where(self, predicate: Union[str, Expression[bool]]) -> 'Table': :: >>> tab.where(tab.name == 'Fred') - >>> tab.where("name = 'Fred'") :param predicate: Predicate expression string. :return: The result table. """ return Table(self._j_table.where(_get_java_expression(predicate)), self._t_env) - def group_by(self, *fields: Union[str, Expression]) -> 'GroupedTable': + def group_by(self, *fields: Expression) -> 'GroupedTable': """ Groups the elements on some grouping keys. Use this before a selection with aggregations to perform the aggregation on a per-group basis. Similar to a SQL GROUP BY statement. @@ -173,17 +163,11 @@ def group_by(self, *fields: Union[str, Expression]) -> 'GroupedTable': :: >>> tab.group_by(tab.key).select(tab.key, tab.value.avg) - >>> tab.group_by("key").select("key, value.avg") :param fields: Group keys. :return: The grouped table. """ - if all(isinstance(f, Expression) for f in fields): - return GroupedTable(self._j_table.groupBy(to_expression_jarray(fields)), self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return GroupedTable(self._j_table.groupBy(fields[0]), self._t_env) + return GroupedTable(self._j_table.groupBy(to_expression_jarray(fields)), self._t_env) def distinct(self) -> 'Table': """ @@ -228,7 +212,7 @@ def join(self, right: 'Table', join_predicate: Union[str, Expression[bool]] = No def left_outer_join(self, right: 'Table', - join_predicate: Union[str, Expression[bool]] = None) -> 'Table': + join_predicate: Expression[bool] = None) -> 'Table': """ Joins two :class:`~pyflink.table.Table`. Similar to a SQL left outer join. The fields of the two joined operations must not overlap, use :func:`~pyflink.table.Table.alias` to @@ -244,7 +228,6 @@ def left_outer_join(self, >>> left.left_outer_join(right) >>> left.left_outer_join(right, left.a == right.b) - >>> left.left_outer_join(right, "a = b") :param right: Right table. :param join_predicate: Optional, the join predicate expression string. @@ -273,7 +256,6 @@ def right_outer_join(self, :: >>> left.right_outer_join(right, left.a == right.b) - >>> left.right_outer_join(right, "a = b") :param right: Right table. :param join_predicate: The join predicate expression string. @@ -299,7 +281,6 @@ def full_outer_join(self, :: >>> left.full_outer_join(right, left.a == right.b) - >>> left.full_outer_join(right, "a = b") :param right: Right table. :param join_predicate: The join predicate expression string. @@ -320,8 +301,7 @@ def join_lateral(self, :: >>> t_env.create_java_temporary_system_function("split", - ... "java.table.function.class.name") - >>> tab.join_lateral("split(text, ' ') as (b)", "a = b") + ... "java.table.function.class.name") >>> from pyflink.table import expressions as expr >>> tab.join_lateral(expr.call('split', ' ').alias('b'), expr.col('a') == expr.col('b')) @@ -370,7 +350,6 @@ def left_outer_join_lateral(self, >>> t_env.create_java_temporary_system_function("split", ... "java.table.function.class.name") - >>> tab.left_outer_join_lateral("split(text, ' ') as (b)") >>> from pyflink.table import expressions as expr >>> tab.left_outer_join_lateral(expr.call('split', ' ').alias('b')) >>> # take all the columns as inputs @@ -528,7 +507,7 @@ def intersect_all(self, right: 'Table') -> 'Table': """ return Table(self._j_table.intersectAll(right._j_table), self._t_env) - def order_by(self, *fields: Union[str, Expression]) -> 'Table': + def order_by(self, *fields: Expression) -> 'Table': """ Sorts the given :class:`~pyflink.table.Table`. Similar to SQL ORDER BY. The resulting Table is sorted globally sorted across all parallel partitions. @@ -537,7 +516,6 @@ def order_by(self, *fields: Union[str, Expression]) -> 'Table': :: >>> tab.order_by(tab.name.desc) - >>> tab.order_by("name.desc") For unbounded tables, this operation requires a sorting on a time attribute or a subsequent fetch operation. @@ -545,12 +523,7 @@ def order_by(self, *fields: Union[str, Expression]) -> 'Table': :param fields: Order fields expression string. :return: The result table. """ - if all(isinstance(f, Expression) for f in fields): - return Table(self._j_table.orderBy(to_expression_jarray(fields)), self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return Table(self._j_table.orderBy(fields[0]), self._t_env) + return Table(self._j_table.orderBy(to_expression_jarray(fields)), self._t_env) def offset(self, offset: int) -> 'Table': """ @@ -565,7 +538,6 @@ def offset(self, offset: int) -> 'Table': # skips the first 3 rows and returns all following rows. >>> tab.order_by(tab.name.desc).offset(3) - >>> tab.order_by("name.desc").offset(3) # skips the first 10 rows and returns the next 5 rows. >>> tab.order_by(tab.name.desc).offset(10).fetch(5) @@ -590,7 +562,6 @@ def fetch(self, fetch: int) -> 'Table': :: >>> tab.order_by(tab.name.desc).fetch(3) - >>> tab.order_by("name.desc").fetch(3) Skips the first 10 rows and returns the next 5 rows. :: @@ -699,7 +670,7 @@ def over_window(self, *over_windows: OverWindow) -> 'OverWindowedTable': [item._java_over_window for item in over_windows]) return OverWindowedTable(self._j_table.window(window_array), self._t_env) - def add_columns(self, *fields: Union[str, Expression]) -> 'Table': + def add_columns(self, *fields: Expression) -> 'Table': """ Adds additional columns. Similar to a SQL SELECT statement. The field expressions can contain complex expressions, but can not contain aggregations. It will throw an @@ -710,19 +681,13 @@ def add_columns(self, *fields: Union[str, Expression]) -> 'Table': >>> from pyflink.table import expressions as expr >>> tab.add_columns((tab.a + 1).alias('a1'), expr.concat(tab.b, 'sunny').alias('b1')) - >>> tab.add_columns("a + 1 as a1, concat(b, 'sunny') as b1") :param fields: Column list string. :return: The result table. """ - if all(isinstance(f, Expression) for f in fields): - return Table(self._j_table.addColumns(to_expression_jarray(fields)), self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return Table(self._j_table.addColumns(fields[0]), self._t_env) + return Table(self._j_table.addColumns(to_expression_jarray(fields)), self._t_env) - def add_or_replace_columns(self, *fields: Union[str, Expression]) -> 'Table': + def add_or_replace_columns(self, *fields: Expression) -> 'Table': """ Adds additional columns. Similar to a SQL SELECT statement. The field expressions can contain complex expressions, but can not contain aggregations. Existing fields will be @@ -735,20 +700,14 @@ def add_or_replace_columns(self, *fields: Union[str, Expression]) -> 'Table': >>> from pyflink.table import expressions as expr >>> tab.add_or_replace_columns((tab.a + 1).alias('a1'), ... expr.concat(tab.b, 'sunny').alias('b1')) - >>> tab.add_or_replace_columns("a + 1 as a1, concat(b, 'sunny') as b1") :param fields: Column list string. :return: The result table. """ - if all(isinstance(f, Expression) for f in fields): - return Table(self._j_table.addOrReplaceColumns(to_expression_jarray(fields)), - self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return Table(self._j_table.addOrReplaceColumns(fields[0]), self._t_env) + return Table(self._j_table.addOrReplaceColumns(to_expression_jarray(fields)), + self._t_env) - def rename_columns(self, *fields: Union[str, Expression]) -> 'Table': + def rename_columns(self, *fields: Expression) -> 'Table': """ Renames existing columns. Similar to a field alias statement. The field expressions should be alias expressions, and only the existing fields can be renamed. @@ -757,20 +716,14 @@ def rename_columns(self, *fields: Union[str, Expression]) -> 'Table': :: >>> tab.rename_columns(tab.a.alias('a1'), tab.b.alias('b1')) - >>> tab.rename_columns("a as a1, b as b1") :param fields: Column list string. :return: The result table. """ - if all(isinstance(f, Expression) for f in fields): - return Table(self._j_table.renameColumns(to_expression_jarray(fields)), - self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return Table(self._j_table.renameColumns(fields[0]), self._t_env) + return Table(self._j_table.renameColumns(to_expression_jarray(fields)), + self._t_env) - def drop_columns(self, *fields: Union[str, Expression]) -> 'Table': + def drop_columns(self, *fields: Expression) -> 'Table': """ Drops existing columns. The field expressions should be field reference expressions. @@ -778,20 +731,14 @@ def drop_columns(self, *fields: Union[str, Expression]) -> 'Table': :: >>> tab.drop_columns(tab.a, tab.b) - >>> tab.drop_columns("a, b") :param fields: Column list string. :return: The result table. """ - if all(isinstance(f, Expression) for f in fields): - return Table(self._j_table.dropColumns(to_expression_jarray(fields)), - self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return Table(self._j_table.dropColumns(fields[0]), self._t_env) + return Table(self._j_table.dropColumns(to_expression_jarray(fields)), + self._t_env) - def map(self, func: Union[str, Expression, UserDefinedScalarFunctionWrapper]) -> 'Table': + def map(self, func: Union[Expression, UserDefinedScalarFunctionWrapper]) -> 'Table': """ Performs a map operation with a user-defined scalar function. @@ -811,15 +758,13 @@ def map(self, func: Union[str, Expression, UserDefinedScalarFunctionWrapper]) -> .. versionadded:: 1.13.0 """ - if isinstance(func, str): - return Table(self._j_table.map(func), self._t_env) - elif isinstance(func, Expression): + if isinstance(func, Expression): return Table(self._j_table.map(func._j_expr), self._t_env) else: func._set_takes_row_as_input() return Table(self._j_table.map(func(with_columns(col("*")))._j_expr), self._t_env) - def flat_map(self, func: Union[str, Expression, UserDefinedTableFunctionWrapper]) -> 'Table': + def flat_map(self, func: Union[Expression, UserDefinedTableFunctionWrapper]) -> 'Table': """ Performs a flatMap operation with a user-defined table function. @@ -843,15 +788,13 @@ def flat_map(self, func: Union[str, Expression, UserDefinedTableFunctionWrapper] .. versionadded:: 1.13.0 """ - if isinstance(func, str): - return Table(self._j_table.flatMap(func), self._t_env) - elif isinstance(func, Expression): + if isinstance(func, Expression): return Table(self._j_table.flatMap(func._j_expr), self._t_env) else: func._set_takes_row_as_input() return Table(self._j_table.flatMap(func(with_columns(col("*")))._j_expr), self._t_env) - def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWrapper]) \ + def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper]) \ -> 'AggregatedTable': """ Performs a global aggregate operation with an aggregate function. You have to close the @@ -865,7 +808,7 @@ def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWra ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.aggregate(agg(tab.a).alias("a", "b")).select("a, b") + >>> tab.aggregate(agg(tab.a).alias("a", "b")).select(col('a'), col('b')) >>> # take all the columns as inputs >>> # pd is a Pandas.DataFrame >>> agg_row = udaf(lambda pd: (pd.a.mean(), pd.a.max()), @@ -873,16 +816,14 @@ def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWra ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.aggregate(agg.alias("a, b")).select("a, b") + >>> tab.aggregate(agg.alias("a, b")).select(col('a'), col('b')) :param func: user-defined aggregate function. :return: The result table. .. versionadded:: 1.13.0 """ - if isinstance(func, str): - return AggregatedTable(self._j_table.aggregate(func), self._t_env) - elif isinstance(func, Expression): + if isinstance(func, Expression): return AggregatedTable(self._j_table.aggregate(func._j_expr), self._t_env) else: func._set_takes_row_as_input() @@ -893,7 +834,7 @@ def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWra func = func(with_columns(col("*"))) return AggregatedTable(self._j_table.aggregate(func._j_expr), self._t_env) - def flat_aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWrapper]) \ + def flat_aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper]) \ -> 'FlatAggregateTable': """ Perform a global flat_aggregate without group_by. flat_aggregate takes a @@ -904,7 +845,7 @@ def flat_aggregate(self, func: Union[str, Expression, UserDefinedAggregateFuncti :: >>> table_agg = udtaf(MyTableAggregateFunction()) - >>> tab.flat_aggregate(table_agg(tab.a).alias("a", "b")).select("a, b") + >>> tab.flat_aggregate(table_agg(tab.a).alias("a", "b")).select(col('a'), col('b')) >>> # take all the columns as inputs >>> class Top2(TableAggregateFunction): ... def emit_value(self, accumulator): @@ -930,16 +871,14 @@ def flat_aggregate(self, func: Union[str, Expression, UserDefinedAggregateFuncti ... return DataTypes.ROW( ... [DataTypes.FIELD("a", DataTypes.BIGINT())]) >>> top2 = udtaf(Top2()) - >>> tab.flat_aggregate(top2.alias("a", "b")).select("a, b") + >>> tab.flat_aggregate(top2.alias("a", "b")).select(col('a'), col('b')) :param func: user-defined table aggregate function. :return: The result table. .. versionadded:: 1.13.0 """ - if isinstance(func, str): - return FlatAggregateTable(self._j_table.flatAggregate(func), self._t_env) - elif isinstance(func, Expression): + if isinstance(func, Expression): return FlatAggregateTable(self._j_table.flatAggregate(func._j_expr), self._t_env) else: func._set_takes_row_as_input() @@ -1124,7 +1063,7 @@ def __init__(self, java_table, t_env): self._j_table = java_table self._t_env = t_env - def select(self, *fields: Union[str, Expression]) -> 'Table': + def select(self, *fields: Expression) -> 'Table': """ Performs a selection operation on a grouped table. Similar to an SQL SELECT statement. The field expressions can contain complex expressions and aggregations. @@ -1133,20 +1072,13 @@ def select(self, *fields: Union[str, Expression]) -> 'Table': :: >>> tab.group_by(tab.key).select(tab.key, tab.value.avg.alias('average')) - >>> tab.group_by("key").select("key, value.avg as average") - :param fields: Expression string that contains group keys and aggregate function calls. :return: The result table. """ - if all(isinstance(f, Expression) for f in fields): - return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return Table(self._j_table.select(fields[0]), self._t_env) + return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) - def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWrapper]) \ + def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper]) \ -> 'AggregatedTable': """ Performs a aggregate operation with an aggregate function. You have to close the @@ -1160,7 +1092,8 @@ def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWra ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.group_by(tab.a).aggregate(agg(tab.b).alias("c", "d")).select("a, c, d") + >>> tab.group_by(tab.a).aggregate(agg(tab.b).alias("c", "d")).select( + ... col('a'), col('c'), col('d')) >>> # take all the columns as inputs >>> # pd is a Pandas.DataFrame >>> agg_row = udaf(lambda pd: (pd.a.mean(), pd.b.max()), @@ -1168,16 +1101,14 @@ def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWra ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.group_by(tab.a).aggregate(agg.alias("a, b")).select("a, b") + >>> tab.group_by(tab.a).aggregate(agg.alias("a, b")).select(col('a'), col('b')) :param func: user-defined aggregate function. :return: The result table. .. versionadded:: 1.13.0 """ - if isinstance(func, str): - return AggregatedTable(self._j_table.aggregate(func), self._t_env) - elif isinstance(func, Expression): + if isinstance(func, Expression): return AggregatedTable(self._j_table.aggregate(func._j_expr), self._t_env) else: func._set_takes_row_as_input() @@ -1188,7 +1119,7 @@ def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWra func = func(with_columns(col("*"))) return AggregatedTable(self._j_table.aggregate(func._j_expr), self._t_env) - def flat_aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWrapper]) \ + def flat_aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper]) \ -> 'FlatAggregateTable': """ Performs a flat_aggregate operation on a grouped table. flat_aggregate takes a @@ -1199,7 +1130,8 @@ def flat_aggregate(self, func: Union[str, Expression, UserDefinedAggregateFuncti :: >>> table_agg = udtaf(MyTableAggregateFunction()) - >>> tab.group_by(tab.c).flat_aggregate(table_agg(tab.a).alias("a")).select("c, a") + >>> tab.group_by(tab.c).flat_aggregate(table_agg(tab.a).alias("a")).select( + ... col('c'), col('a')) >>> # take all the columns as inputs >>> class Top2(TableAggregateFunction): ... def emit_value(self, accumulator): @@ -1225,16 +1157,14 @@ def flat_aggregate(self, func: Union[str, Expression, UserDefinedAggregateFuncti ... return DataTypes.ROW( ... [DataTypes.FIELD("a", DataTypes.BIGINT())]) >>> top2 = udtaf(Top2()) - >>> tab.group_by(tab.c).flat_aggregate(top2.alias("a", "b")).select("a, b") + >>> tab.group_by(tab.c).flat_aggregate(top2.alias("a", "b")).select(col('a'), col('b')) :param func: user-defined table aggregate function. :return: The result table. .. versionadded:: 1.13.0 """ - if isinstance(func, str): - return FlatAggregateTable(self._j_table.flatAggregate(func), self._t_env) - elif isinstance(func, Expression): + if isinstance(func, Expression): return FlatAggregateTable(self._j_table.flatAggregate(func._j_expr), self._t_env) else: func._set_takes_row_as_input() @@ -1255,7 +1185,7 @@ def __init__(self, java_group_windowed_table, t_env): self._j_table = java_group_windowed_table self._t_env = t_env - def group_by(self, *fields: Union[str, Expression]) -> 'WindowGroupedTable': + def group_by(self, *fields: Expression) -> 'WindowGroupedTable': """ Groups the elements by a mandatory window and one or more optional grouping attributes. The window is specified by referring to its alias. @@ -1281,13 +1211,8 @@ def group_by(self, *fields: Union[str, Expression]) -> 'WindowGroupedTable': :param fields: Group keys. :return: A window grouped table. """ - if all(isinstance(f, Expression) for f in fields): - return WindowGroupedTable( - self._j_table.groupBy(to_expression_jarray(fields)), self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return WindowGroupedTable(self._j_table.groupBy(fields[0]), self._t_env) + return WindowGroupedTable( + self._j_table.groupBy(to_expression_jarray(fields)), self._t_env) class WindowGroupedTable(object): @@ -1299,7 +1224,7 @@ def __init__(self, java_window_grouped_table, t_env): self._j_table = java_window_grouped_table self._t_env = t_env - def select(self, *fields: Union[str, Expression]) -> 'Table': + def select(self, *fields: Expression) -> 'Table': """ Performs a selection operation on a window grouped table. Similar to an SQL SELECT statement. @@ -1311,7 +1236,6 @@ def select(self, *fields: Union[str, Expression]) -> 'Table': >>> window_grouped_table.select(col('key'), ... col('window').start, ... col('value').avg.alias('valavg')) - >>> window_grouped_table.select("key, window.start, value.avg as valavg") :param fields: Expression string. :return: The result table. @@ -1323,7 +1247,7 @@ def select(self, *fields: Union[str, Expression]) -> 'Table': assert isinstance(fields[0], str) return Table(self._j_table.select(fields[0]), self._t_env) - def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWrapper]) \ + def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper]) \ -> 'AggregatedTable': """ Performs an aggregate operation on a window grouped table. You have to close the @@ -1340,7 +1264,7 @@ def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWra >>> window_grouped_table.group_by("w") \ ... .aggregate(agg(window_grouped_table.b) \ ... .alias("c", "d")) \ - ... .select("c, d") + ... .select(col('c'), col('d')) >>> # take all the columns as inputs >>> # pd is a Pandas.DataFrame >>> agg_row = udaf(lambda pd: (pd.a.mean(), pd.b.max()), @@ -1355,9 +1279,7 @@ def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWra .. versionadded:: 1.13.0 """ - if isinstance(func, str): - return AggregatedTable(self._j_table.aggregate(func), self._t_env) - elif isinstance(func, Expression): + if isinstance(func, Expression): return AggregatedTable(self._j_table.aggregate(func._j_expr), self._t_env) else: func._set_takes_row_as_input() @@ -1391,7 +1313,7 @@ def __init__(self, java_over_windowed_table, t_env): self._j_table = java_over_windowed_table self._t_env = t_env - def select(self, *fields: Union[str, Expression]) -> 'Table': + def select(self, *fields: Expression) -> 'Table': """ Performs a selection operation on a over windowed table. Similar to an SQL SELECT statement. @@ -1403,17 +1325,11 @@ def select(self, *fields: Union[str, Expression]) -> 'Table': >>> over_windowed_table.select(col('c'), ... col('b').count.over(col('ow')), ... col('e').sum.over(col('ow'))) - >>> over_windowed_table.select("c, b.count over ow, e.sum over ow") :param fields: Expression string. :return: The result table. """ - if all(isinstance(f, Expression) for f in fields): - return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return Table(self._j_table.select(fields[0]), self._t_env) + return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) class AggregatedTable(object): @@ -1425,7 +1341,7 @@ def __init__(self, java_table, t_env): self._j_table = java_table self._t_env = t_env - def select(self, *fields: Union[str, Expression]) -> 'Table': + def select(self, *fields: Expression) -> 'Table': """ Performs a selection operation after an aggregate operation. The field expressions cannot contain table functions and aggregations. @@ -1438,7 +1354,7 @@ def select(self, *fields: Union[str, Expression]) -> 'Table': ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.aggregate(agg(tab.a).alias("a", "b")).select("a, b") + >>> tab.aggregate(agg(tab.a).alias("a", "b")).select(col('a'), col('b')) >>> # take all the columns as inputs >>> # pd is a Pandas.DataFrame >>> agg_row = udaf(lambda pd: (pd.a.mean(), pd.b.max()), @@ -1446,17 +1362,12 @@ def select(self, *fields: Union[str, Expression]) -> 'Table': ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.group_by(tab.a).aggregate(agg.alias("a, b")).select("a, b") + >>> tab.group_by(tab.a).aggregate(agg.alias("a, b")).select(col('a'), col('b')) :param fields: Expression string. :return: The result table. """ - if all(isinstance(f, Expression) for f in fields): - return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return Table(self._j_table.select(fields[0]), self._t_env) + return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) class FlatAggregateTable(object): @@ -1469,7 +1380,7 @@ def __init__(self, java_table, t_env): self._j_table = java_table self._t_env = t_env - def select(self, *fields: Union[str, Expression]) -> 'Table': + def select(self, *fields: Expression) -> 'Table': """ Performs a selection operation on a FlatAggregateTable. Similar to a SQL SELECT statement. The field expressions can contain complex expressions. @@ -1478,7 +1389,7 @@ def select(self, *fields: Union[str, Expression]) -> 'Table': :: >>> table_agg = udtaf(MyTableAggregateFunction()) - >>> tab.flat_aggregate(table_agg(tab.a).alias("a", "b")).select("a, b") + >>> tab.flat_aggregate(table_agg(tab.a).alias("a", "b")).select(col('a'), col('b')) >>> # take all the columns as inputs >>> class Top2(TableAggregateFunction): ... def emit_value(self, accumulator): @@ -1504,14 +1415,9 @@ def select(self, *fields: Union[str, Expression]) -> 'Table': ... return DataTypes.ROW( ... [DataTypes.FIELD("a", DataTypes.BIGINT())]) >>> top2 = udtaf(Top2()) - >>> tab.group_by(tab.c).flat_aggregate(top2.alias("a", "b")).select("a, b") + >>> tab.group_by(tab.c).flat_aggregate(top2.alias("a", "b")).select(col('a'), col('b')) :param fields: Expression string. :return: The result table. """ - if all(isinstance(f, Expression) for f in fields): - return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) - else: - assert len(fields) == 1 - assert isinstance(fields[0], str) - return Table(self._j_table.select(fields[0]), self._t_env) + return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) diff --git a/flink-python/pyflink/table/window.py b/flink-python/pyflink/table/window.py index 282660c648196..630455e3d5f4d 100644 --- a/flink-python/pyflink/table/window.py +++ b/flink-python/pyflink/table/window.py @@ -144,9 +144,6 @@ class Session(object): >>> Session.with_gap(expr.lit(10).minutes) ... .on(expr.col("rowtime")) ... .alias("w") - - >>> Session.with_gap("10.minutes").on("rowtime").alias("w") - """ @classmethod @@ -230,8 +227,6 @@ class Slide(object): ... .every(expr.lit(5).minutes) ... .on(expr.col("rowtime")) ... .alias("w") - - >>> Slide.over("10.minutes").every("5.minutes").on("rowtime").alias("w") """ @classmethod @@ -338,8 +333,6 @@ class Over(object): ... .order_by(col("rowtime")) \\ ... .preceding(expr.UNBOUNDED_RANGE) \\ ... .alias("w") - - >>> Over.partition_by("a").order_by("rowtime").preceding("unbounded_range").alias("w") """ @classmethod From 4f39b034697e811c30835c28fda2aef6ad66ae2a Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 22 Mar 2022 12:58:52 +0100 Subject: [PATCH 136/258] [FLINK-26797][runtime] Makes the test less strict We only care about monotonically increasing numbers. Unstable connections could cause a retry by the curator client. See FLINK-26120 where a the same error cause was fixed in the same way. --- .../ZKCheckpointIDCounterMultiServersTest.java | 16 +++------------- 1 file changed, 3 insertions(+), 13 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java index 5f303e1bcc897..c18408f08d575 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZKCheckpointIDCounterMultiServersTest.java @@ -33,9 +33,7 @@ import org.junit.Rule; import org.junit.Test; -import java.util.concurrent.atomic.AtomicLong; - -import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.number.OrderingComparison.greaterThan; import static org.junit.Assert.assertThat; /** Tests for {@link ZooKeeperCheckpointIDCounter} in a ZooKeeper ensemble. */ @@ -71,22 +69,14 @@ public void testRecoveredAfterConnectionLoss() throws Exception { curatorFrameworkWrapper.asCuratorFramework(), listener); idCounter.start(); - AtomicLong localCounter = new AtomicLong(1L); - - assertThat( - "ZooKeeperCheckpointIDCounter doesn't properly work.", - idCounter.getAndIncrement(), - is(localCounter.getAndIncrement())); + final long initialID = idCounter.getAndIncrement(); zooKeeperResource.restart(); connectionLossLatch.await(); reconnectedLatch.await(); - assertThat( - "ZooKeeperCheckpointIDCounter doesn't properly work after reconnected.", - idCounter.getAndIncrement(), - is(localCounter.getAndIncrement())); + assertThat(idCounter.getAndIncrement(), greaterThan(initialID)); } finally { curatorFrameworkWrapper.close(); } From 2083203d2420dc629204ebdaf75b2a33b83267a5 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Tue, 22 Mar 2022 13:03:42 +0100 Subject: [PATCH 137/258] [hotfix][runtime] Adds debug logs for FinalRequestProcessor to CI run org.apache.zookeeper.server.FinalRequestProcessor provides debug logs for actual operations being performed on the server, which might be interesting to see when investigating test instabilities. The following logs are part of the debug logs: Processing request:: sessionid:0x1003757ccd40000 type:getData cxid:0x11 zxid:0xfffffffffffffffe txntype:unknown reqpath:/flink/default/checkpoint_id_counter ...contains the type of request (e.g. 'getData') and path (e.g. '/flink/default/checkpoint_id_counter'). --- tools/ci/log4j.properties | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tools/ci/log4j.properties b/tools/ci/log4j.properties index bf831743e6f26..80a611405d7e4 100644 --- a/tools/ci/log4j.properties +++ b/tools/ci/log4j.properties @@ -61,6 +61,11 @@ logger.zookeeper.name = org.apache.zookeeper logger.zookeeper.level = INFO logger.zookeeper.additivity = false logger.zookeeper.appenderRef.zk.ref = ZooKeeperServerAppender +# FinalRequestProcessor in DEBUG mode enables us to see requests being processed by the ZK server +logger.zookeeper_FinalRequestProcessor.name = org.apache.zookeeper.server.FinalRequestProcessor +logger.zookeeper_FinalRequestProcessor.level = DEBUG +logger.zookeeper_FinalRequestProcessor.additivity = false +logger.zookeeper_FinalRequestProcessor.appenderRef.zk.ref = ZooKeeperServerAppender logger.shaded_zookeeper.name = org.apache.flink.shaded.zookeeper3 logger.shaded_zookeeper.level = INFO logger.shaded_zookeeper.additivity = false From 1bfbdc9c76de3b3fab4d70058257b7a307269570 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Sat, 2 Apr 2022 09:58:17 +0800 Subject: [PATCH 138/258] Revert "[FLINK-26986][python] Remove deprecated string expressions in Python Table API" This reverts commits 02c5e4136c809eac7b5d723be0d043b639ddf477. --- .../table/python_table_api_connectors.md | 2 +- .../docs/dev/python/table/udfs/python_udfs.md | 12 +- .../table/udfs/vectorized_python_udfs.md | 3 +- docs/content.zh/docs/dev/table/catalogs.md | 2 +- docs/content.zh/docs/dev/table/tableApi.md | 15 +- .../table/python_table_api_connectors.md | 2 +- .../docs/dev/python/table/udfs/python_udfs.md | 12 +- .../table/udfs/vectorized_python_udfs.md | 3 +- docs/content/docs/dev/table/catalogs.md | 2 +- docs/content/docs/dev/table/tableApi.md | 15 +- .../examples/table/pandas/pandas_udaf.py | 2 +- .../examples/table/windowing/over_window.py | 6 +- .../table/windowing/session_window.py | 2 +- .../table/windowing/sliding_window.py | 2 +- .../examples/table/windowing/tumble_window.py | 2 +- flink-python/pyflink/table/table.py | 214 +++++++++++++----- flink-python/pyflink/table/window.py | 7 + 17 files changed, 204 insertions(+), 99 deletions(-) diff --git a/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md b/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md index 6be2c1e727514..dbf94fbd1eee9 100644 --- a/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md +++ b/docs/content.zh/docs/dev/python/table/python_table_api_connectors.md @@ -139,7 +139,7 @@ import numpy as np # 创建一个 PyFlink 表 pdf = pd.DataFrame(np.random.rand(1000, 2)) -table = t_env.from_pandas(pdf, ["a", "b"]).filter(col('a') > 0.5) +table = t_env.from_pandas(pdf, ["a", "b"]).filter("a > 0.5") # 将 PyFlink 表转换成 Pandas DataFrame pdf = table.to_pandas() diff --git a/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md b/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md index f16c433decca5..124d1bbfcc7ef 100644 --- a/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md +++ b/docs/content.zh/docs/dev/python/table/udfs/python_udfs.md @@ -128,7 +128,7 @@ add = udf(functools.partial(partial_add, k=1), result_type=DataTypes.BIGINT()) # 注册 Python 自定义函数 table_env.create_temporary_function("add", add) # 在 Python Table API 中使用 Python 自定义函数 -my_table.select(call('add', my_table.a, my_table.b)) +my_table.select("add(a, b)") # 也可以在 Python Table API 中直接使用 Python 自定义函数 my_table.select(add(my_table.a, my_table.b)) @@ -156,8 +156,8 @@ my_table = ... # type: Table, table schema: [a: String] split = udtf(Split(), result_types=[DataTypes.STRING(), DataTypes.INT()]) # 在 Python Table API 中使用 Python 表值函数 -my_table.join_lateral(split(my_table.a).alias("word", "length")) -my_table.left_outer_join_lateral(split(my_table.a).alias("word", "length")) +my_table.join_lateral(split(my_table.a).alias("word, length")) +my_table.left_outer_join_lateral(split(my_table.a).alias("word, length")) # 在 SQL API 中使用 Python 表值函数 table_env.create_temporary_function("split", udtf(Split(), result_types=[DataTypes.STRING(), DataTypes.INT()])) @@ -194,8 +194,8 @@ my_table = ... # type: Table, table schema: [a: String] table_env.create_java_temporary_function("split", "my.java.function.Split") # 在 Python Table API 中使用表值函数。 "alias"指定表的字段名称。 -my_table.join_lateral(call('split', my_table.a).alias("word", "length")).select(my_table.a, col('word'), col('length')) -my_table.left_outer_join_lateral(call('split', my_table.a).alias("word", "length")).select(my_table.a, col('word'), col('length')) +my_table.join_lateral(call('split', my_table.a).alias("word, length")).select(my_table.a, col('word'), col('length')) +my_table.left_outer_join_lateral(call('split', my_table.a).alias("word, length")).select(my_table.a, col('word'), col('length')) # 注册 Python 函数。 @@ -337,7 +337,7 @@ tumble_window = Tumble.over(lit(1).hours) \ result = t.window(tumble_window) \ .group_by(col('w'), col('name')) \ - .select(col('w').start, col('w').end, weighted_avg(col('value'), col('count'))) \ + .select("w.start, w.end, weighted_avg(value, count)") \ .to_pandas() print(result) diff --git a/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md b/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md index 10012d8ba4a53..af6c7f60c15b1 100644 --- a/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md +++ b/docs/content.zh/docs/dev/python/table/udfs/vectorized_python_udfs.md @@ -101,7 +101,8 @@ tumble_window = Tumble.over(expr.lit(1).hours) \ my_table.window(tumble_window) \ .group_by("w") \ - .select(col('w').start, col('w').end, mean_udaf(col('b'))) + .select("w.start, w.end, mean_udaf(b)") + # 在 Over Window Aggregation 中使用向量化聚合函数 table_env.create_temporary_function("mean_udaf", mean_udaf) diff --git a/docs/content.zh/docs/dev/table/catalogs.md b/docs/content.zh/docs/dev/table/catalogs.md index 05722d87f651e..0b86fbd6769a2 100644 --- a/docs/content.zh/docs/dev/table/catalogs.md +++ b/docs/content.zh/docs/dev/table/catalogs.md @@ -241,7 +241,7 @@ schema = Schema.new_builder() \ catalog_table = t_env.create_table("myhive.mydb.mytable", TableDescriptor.for_connector("kafka") .schema(schema) - # … + // … .build()) # tables should contain "mytable" diff --git a/docs/content.zh/docs/dev/table/tableApi.md b/docs/content.zh/docs/dev/table/tableApi.md index d2b198ca29c04..65a9f9d87a257 100644 --- a/docs/content.zh/docs/dev/table/tableApi.md +++ b/docs/content.zh/docs/dev/table/tableApi.md @@ -417,7 +417,7 @@ val orders: Table = tableEnv.from("Orders").as("x", "y", "z", "t") {{< tab "Python" >}} ```python orders = t_env.from_path("Orders") -result = orders.alias("x", "y", "z", "t") +result = orders.alias("x, y, z, t") ``` {{< /tab >}} {{< /tabs >}} @@ -1054,7 +1054,7 @@ def split(x): # join orders = t_env.from_path("Orders") -joined_table = orders.join_lateral(split(orders.c).alias("s", "t", "v")) +joined_table = orders.join_lateral(split(orders.c).alias("s, t, v")) result = joined_table.select(joined_table.a, joined_table.b, joined_table.s, joined_table.t, joined_table.v) ``` {{< /tab >}} @@ -1103,7 +1103,7 @@ def split(x): # join orders = t_env.from_path("Orders") -joined_table = orders.left_outer_join_lateral(split(orders.c).alias("s", "t", "v")) +joined_table = orders.left_outer_join_lateral(split(orders.c).alias("s, t, v")) result = joined_table.select(joined_table.a, joined_table.b, joined_table.s, joined_table.t, joined_table.v) ``` {{< /tab >}} @@ -2453,7 +2453,7 @@ agg = udaf(function, # 使用 python 通用聚合函数进行聚合 result = t.group_by(t.a) \ .aggregate(agg.alias("c", "d")) \ - .select(col('a'), col('c'), col('d')) + .select("a, c, d") # 使用 python 向量化聚合函数进行聚合 pandas_udaf = udaf(lambda pd: (pd.b.mean(), pd.b.max()), @@ -2462,7 +2462,8 @@ pandas_udaf = udaf(lambda pd: (pd.b.mean(), pd.b.max()), DataTypes.FIELD("b", DataTypes.INT())]), func_type="pandas") t.aggregate(pandas_udaf.alias("a", "b")) \ - .select(col('a'), col('b')) + .select("a, b") + ``` {{< /tab >}} @@ -2514,9 +2515,9 @@ tumble_window = Tumble.over(expr.lit(1).hours) \ .alias("w") t.select(t.b, t.rowtime) \ .window(tumble_window) \ - .group_by(col("w")) \ + .group_by("w") \ .aggregate(pandas_udaf.alias("d", "e")) \ - .select(col('w').rowtime, col('d'), col('e')) + .select("w.rowtime, d, e") ``` {{< /tab >}} {{< /tabs >}} diff --git a/docs/content/docs/dev/python/table/python_table_api_connectors.md b/docs/content/docs/dev/python/table/python_table_api_connectors.md index b3ee2481ab843..a7f31d30cfb14 100644 --- a/docs/content/docs/dev/python/table/python_table_api_connectors.md +++ b/docs/content/docs/dev/python/table/python_table_api_connectors.md @@ -143,7 +143,7 @@ import numpy as np # Create a PyFlink Table pdf = pd.DataFrame(np.random.rand(1000, 2)) -table = t_env.from_pandas(pdf, ["a", "b"]).filter(col('a') > 0.5) +table = t_env.from_pandas(pdf, ["a", "b"]).filter("a > 0.5") # Convert the PyFlink Table to a Pandas DataFrame pdf = table.to_pandas() diff --git a/docs/content/docs/dev/python/table/udfs/python_udfs.md b/docs/content/docs/dev/python/table/udfs/python_udfs.md index c33a51bf90038..745124b7bc99b 100644 --- a/docs/content/docs/dev/python/table/udfs/python_udfs.md +++ b/docs/content/docs/dev/python/table/udfs/python_udfs.md @@ -129,7 +129,7 @@ add = udf(functools.partial(partial_add, k=1), result_type=DataTypes.BIGINT()) # register the Python function table_env.create_temporary_function("add", add) # use the function in Python Table API -my_table.select(call('add', my_table.a, my_table.b)) +my_table.select("add(a, b)") # You can also use the Python function in Python Table API directly my_table.select(add(my_table.a, my_table.b)) @@ -158,8 +158,8 @@ my_table = ... # type: Table, table schema: [a: String] split = udtf(Split(), result_types=[DataTypes.STRING(), DataTypes.INT()]) # use the Python Table Function in Python Table API -my_table.join_lateral(split(my_table.a).alias("word", "length")) -my_table.left_outer_join_lateral(split(my_table.a).alias("word", "length")) +my_table.join_lateral(split(my_table.a).alias("word, length")) +my_table.left_outer_join_lateral(split(my_table.a).alias("word, length")) # use the Python Table function in SQL API table_env.create_temporary_function("split", udtf(Split(), result_types=[DataTypes.STRING(), DataTypes.INT()])) @@ -196,8 +196,8 @@ my_table = ... # type: Table, table schema: [a: String] table_env.create_java_temporary_function("split", "my.java.function.Split") # Use the table function in the Python Table API. "alias" specifies the field names of the table. -my_table.join_lateral(call('split', my_table.a).alias("word", "length")).select(my_table.a, col('word'), col('length')) -my_table.left_outer_join_lateral(call('split', my_table.a).alias("word", "length")).select(my_table.a, col('word'), col('length')) +my_table.join_lateral(call('split', my_table.a).alias("word, length")).select(my_table.a, col('word'), col('length')) +my_table.left_outer_join_lateral(call('split', my_table.a).alias("word, length")).select(my_table.a, col('word'), col('length')) # Register the python function. @@ -338,7 +338,7 @@ tumble_window = Tumble.over(lit(1).hours) \ result = t.window(tumble_window) \ .group_by(col('w'), col('name')) \ - .select(col('w').start, col('w').end, weighted_avg(col('value'), col('count'))) \ + .select("w.start, w.end, weighted_avg(value, count)") \ .to_pandas() print(result) diff --git a/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md b/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md index c3461b342f4f9..7b79eabab8eff 100644 --- a/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md +++ b/docs/content/docs/dev/python/table/udfs/vectorized_python_udfs.md @@ -100,7 +100,8 @@ tumble_window = Tumble.over(expr.lit(1).hours) \ my_table.window(tumble_window) \ .group_by("w") \ - .select(col('w').start, col('w').end, mean_udaf(col('b'))) + .select("w.start, w.end, mean_udaf(b)") + # use the vectorized Python aggregate function in Over Window Aggregation table_env.create_temporary_function("mean_udaf", mean_udaf) diff --git a/docs/content/docs/dev/table/catalogs.md b/docs/content/docs/dev/table/catalogs.md index f166ab5f61b5d..d0b7ab6be6416 100644 --- a/docs/content/docs/dev/table/catalogs.md +++ b/docs/content/docs/dev/table/catalogs.md @@ -245,7 +245,7 @@ schema = Schema.new_builder() \ catalog_table = t_env.create_table("myhive.mydb.mytable", TableDescriptor.for_connector("kafka") .schema(schema) - # … + // … .build()) # tables should contain "mytable" diff --git a/docs/content/docs/dev/table/tableApi.md b/docs/content/docs/dev/table/tableApi.md index 91ab6cf49c121..d4f132c5afbce 100644 --- a/docs/content/docs/dev/table/tableApi.md +++ b/docs/content/docs/dev/table/tableApi.md @@ -418,7 +418,7 @@ val orders: Table = tableEnv.from("Orders").as("x", "y", "z", "t") {{< tab "Python" >}} ```python orders = t_env.from_path("Orders") -result = orders.alias("x", "y", "z", "t") +result = orders.alias("x, y, z, t") ``` {{< /tab >}} {{< /tabs >}} @@ -1053,7 +1053,7 @@ def split(x): # join orders = t_env.from_path("Orders") -joined_table = orders.join_lateral(split(orders.c).alias("s", "t", "v")) +joined_table = orders.join_lateral(split(orders.c).alias("s, t, v")) result = joined_table.select(joined_table.a, joined_table.b, joined_table.s, joined_table.t, joined_table.v) ``` {{< /tab >}} @@ -1102,7 +1102,7 @@ def split(x): # join orders = t_env.from_path("Orders") -joined_table = orders.left_outer_join_lateral(split(orders.c).alias("s", "t", "v")) +joined_table = orders.left_outer_join_lateral(split(orders.c).alias("s, t, v")) result = joined_table.select(joined_table.a, joined_table.b, joined_table.s, joined_table.t, joined_table.v) ``` {{< /tab >}} @@ -2452,7 +2452,7 @@ agg = udaf(function, # aggregate with a python general aggregate function result = t.group_by(t.a) \ .aggregate(agg.alias("c", "d")) \ - select(col('a'), col('c'), col('d')) + .select("a, c, d") # aggregate with a python vectorized aggregate function pandas_udaf = udaf(lambda pd: (pd.b.mean(), pd.b.max()), @@ -2461,7 +2461,8 @@ pandas_udaf = udaf(lambda pd: (pd.b.mean(), pd.b.max()), DataTypes.FIELD("b", DataTypes.INT())]), func_type="pandas") t.aggregate(pandas_udaf.alias("a", "b")) \ - select(col('a'), col('b')) + .select("a, b") + ``` {{< /tab >}} @@ -2514,9 +2515,9 @@ tumble_window = Tumble.over(expr.lit(1).hours) \ .alias("w") t.select(t.b, t.rowtime) \ .window(tumble_window) \ - .group_by(col("w")) \ + .group_by("w") \ .aggregate(pandas_udaf.alias("d", "e")) \ - .select(col('w').rowtime, col('d'), col('e')) + .select("w.rowtime, d, e") ``` {{< /tab >}} {{< /tabs >}} diff --git a/flink-python/pyflink/examples/table/pandas/pandas_udaf.py b/flink-python/pyflink/examples/table/pandas/pandas_udaf.py index 4b884dd480341..e4e8b9ea5222b 100644 --- a/flink-python/pyflink/examples/table/pandas/pandas_udaf.py +++ b/flink-python/pyflink/examples/table/pandas/pandas_udaf.py @@ -55,7 +55,7 @@ def pandas_udaf(): .column("f2", DataTypes.FLOAT()) .watermark("ts", "ts - INTERVAL '3' SECOND") .build() - ).alias("ts", "name", "price") + ).alias("ts, name, price") # define the sink t_env.create_temporary_table( diff --git a/flink-python/pyflink/examples/table/windowing/over_window.py b/flink-python/pyflink/examples/table/windowing/over_window.py index 5fd736b529a20..982d6b860d468 100644 --- a/flink-python/pyflink/examples/table/windowing/over_window.py +++ b/flink-python/pyflink/examples/table/windowing/over_window.py @@ -54,7 +54,7 @@ def tumble_window_demo(): .column("f2", DataTypes.FLOAT()) .watermark("ts", "ts - INTERVAL '3' SECOND") .build() - ).alias("ts", "name", "price") + ).alias("ts, name, price") # define the sink t_env.create_temporary_table( @@ -68,8 +68,8 @@ def tumble_window_demo(): # define the over window operation table = table.over_window( - Over.partition_by(col("name")) - .order_by(col("ts")) + Over.partition_by("name") + .order_by("ts") .preceding(row_interval(2)) .following(CURRENT_ROW) .alias('w')) \ diff --git a/flink-python/pyflink/examples/table/windowing/session_window.py b/flink-python/pyflink/examples/table/windowing/session_window.py index 49e4680938913..5b40a7b9af8bd 100644 --- a/flink-python/pyflink/examples/table/windowing/session_window.py +++ b/flink-python/pyflink/examples/table/windowing/session_window.py @@ -52,7 +52,7 @@ def session_window_demo(): .column("f2", DataTypes.FLOAT()) .watermark("ts", "ts - INTERVAL '3' SECOND") .build() - ).alias("ts", "name", "price") + ).alias("ts, name, price") # define the sink t_env.create_temporary_table( diff --git a/flink-python/pyflink/examples/table/windowing/sliding_window.py b/flink-python/pyflink/examples/table/windowing/sliding_window.py index fc460c67df469..1b8bb150fd215 100644 --- a/flink-python/pyflink/examples/table/windowing/sliding_window.py +++ b/flink-python/pyflink/examples/table/windowing/sliding_window.py @@ -54,7 +54,7 @@ def sliding_window_demo(): .column("f2", DataTypes.FLOAT()) .watermark("ts", "ts - INTERVAL '3' SECOND") .build() - ).alias("ts", "name", "price") + ).alias("ts, name, price") # define the sink t_env.create_temporary_table( diff --git a/flink-python/pyflink/examples/table/windowing/tumble_window.py b/flink-python/pyflink/examples/table/windowing/tumble_window.py index c7787473451e0..dd3ba2ea408b9 100644 --- a/flink-python/pyflink/examples/table/windowing/tumble_window.py +++ b/flink-python/pyflink/examples/table/windowing/tumble_window.py @@ -54,7 +54,7 @@ def tumble_window_demo(): .column("f2", DataTypes.FLOAT()) .watermark("ts", "ts - INTERVAL '3' SECOND") .build() - ).alias("ts", "name", "price") + ).alias("ts, name, price") # define the sink t_env.create_temporary_table( diff --git a/flink-python/pyflink/table/table.py b/flink-python/pyflink/table/table.py index ed46d8afe53d7..b7d8ae60673f5 100644 --- a/flink-python/pyflink/table/table.py +++ b/flink-python/pyflink/table/table.py @@ -90,7 +90,7 @@ def __getattr__(self, name) -> Expression: % (name, ', '.join(self.get_schema().get_field_names()))) return col(name) - def select(self, *fields: Expression) -> 'Table': + def select(self, *fields: Union[str, Expression]) -> 'Table': """ Performs a selection operation. Similar to a SQL SELECT statement. The field expressions can contain complex expressions. @@ -102,9 +102,16 @@ def select(self, *fields: Expression) -> 'Table': >>> tab.select(tab.key, expr.concat(tab.value, 'hello')) >>> tab.select(expr.col('key'), expr.concat(expr.col('value'), 'hello')) + >>> tab.select("key, value + 'hello'") + :return: The result table. """ - return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) + if all(isinstance(f, Expression) for f in fields): + return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return Table(self._j_table.select(fields[0]), self._t_env) def alias(self, field: str, *fields: str) -> 'Table': """ @@ -115,6 +122,7 @@ def alias(self, field: str, *fields: str) -> 'Table': :: >>> tab.alias("a", "b", "c") + >>> tab.alias("a, b, c") :param field: Field alias. :param fields: Additional field aliases. @@ -133,6 +141,7 @@ def filter(self, predicate: Union[str, Expression[bool]]) -> 'Table': :: >>> tab.filter(tab.name == 'Fred') + >>> tab.filter("name = 'Fred'") :param predicate: Predicate expression string. :return: The result table. @@ -148,13 +157,14 @@ def where(self, predicate: Union[str, Expression[bool]]) -> 'Table': :: >>> tab.where(tab.name == 'Fred') + >>> tab.where("name = 'Fred'") :param predicate: Predicate expression string. :return: The result table. """ return Table(self._j_table.where(_get_java_expression(predicate)), self._t_env) - def group_by(self, *fields: Expression) -> 'GroupedTable': + def group_by(self, *fields: Union[str, Expression]) -> 'GroupedTable': """ Groups the elements on some grouping keys. Use this before a selection with aggregations to perform the aggregation on a per-group basis. Similar to a SQL GROUP BY statement. @@ -163,11 +173,17 @@ def group_by(self, *fields: Expression) -> 'GroupedTable': :: >>> tab.group_by(tab.key).select(tab.key, tab.value.avg) + >>> tab.group_by("key").select("key, value.avg") :param fields: Group keys. :return: The grouped table. """ - return GroupedTable(self._j_table.groupBy(to_expression_jarray(fields)), self._t_env) + if all(isinstance(f, Expression) for f in fields): + return GroupedTable(self._j_table.groupBy(to_expression_jarray(fields)), self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return GroupedTable(self._j_table.groupBy(fields[0]), self._t_env) def distinct(self) -> 'Table': """ @@ -212,7 +228,7 @@ def join(self, right: 'Table', join_predicate: Union[str, Expression[bool]] = No def left_outer_join(self, right: 'Table', - join_predicate: Expression[bool] = None) -> 'Table': + join_predicate: Union[str, Expression[bool]] = None) -> 'Table': """ Joins two :class:`~pyflink.table.Table`. Similar to a SQL left outer join. The fields of the two joined operations must not overlap, use :func:`~pyflink.table.Table.alias` to @@ -228,6 +244,7 @@ def left_outer_join(self, >>> left.left_outer_join(right) >>> left.left_outer_join(right, left.a == right.b) + >>> left.left_outer_join(right, "a = b") :param right: Right table. :param join_predicate: Optional, the join predicate expression string. @@ -256,6 +273,7 @@ def right_outer_join(self, :: >>> left.right_outer_join(right, left.a == right.b) + >>> left.right_outer_join(right, "a = b") :param right: Right table. :param join_predicate: The join predicate expression string. @@ -281,6 +299,7 @@ def full_outer_join(self, :: >>> left.full_outer_join(right, left.a == right.b) + >>> left.full_outer_join(right, "a = b") :param right: Right table. :param join_predicate: The join predicate expression string. @@ -301,7 +320,8 @@ def join_lateral(self, :: >>> t_env.create_java_temporary_system_function("split", - ... "java.table.function.class.name") + ... "java.table.function.class.name") + >>> tab.join_lateral("split(text, ' ') as (b)", "a = b") >>> from pyflink.table import expressions as expr >>> tab.join_lateral(expr.call('split', ' ').alias('b'), expr.col('a') == expr.col('b')) @@ -350,6 +370,7 @@ def left_outer_join_lateral(self, >>> t_env.create_java_temporary_system_function("split", ... "java.table.function.class.name") + >>> tab.left_outer_join_lateral("split(text, ' ') as (b)") >>> from pyflink.table import expressions as expr >>> tab.left_outer_join_lateral(expr.call('split', ' ').alias('b')) >>> # take all the columns as inputs @@ -507,7 +528,7 @@ def intersect_all(self, right: 'Table') -> 'Table': """ return Table(self._j_table.intersectAll(right._j_table), self._t_env) - def order_by(self, *fields: Expression) -> 'Table': + def order_by(self, *fields: Union[str, Expression]) -> 'Table': """ Sorts the given :class:`~pyflink.table.Table`. Similar to SQL ORDER BY. The resulting Table is sorted globally sorted across all parallel partitions. @@ -516,6 +537,7 @@ def order_by(self, *fields: Expression) -> 'Table': :: >>> tab.order_by(tab.name.desc) + >>> tab.order_by("name.desc") For unbounded tables, this operation requires a sorting on a time attribute or a subsequent fetch operation. @@ -523,7 +545,12 @@ def order_by(self, *fields: Expression) -> 'Table': :param fields: Order fields expression string. :return: The result table. """ - return Table(self._j_table.orderBy(to_expression_jarray(fields)), self._t_env) + if all(isinstance(f, Expression) for f in fields): + return Table(self._j_table.orderBy(to_expression_jarray(fields)), self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return Table(self._j_table.orderBy(fields[0]), self._t_env) def offset(self, offset: int) -> 'Table': """ @@ -538,6 +565,7 @@ def offset(self, offset: int) -> 'Table': # skips the first 3 rows and returns all following rows. >>> tab.order_by(tab.name.desc).offset(3) + >>> tab.order_by("name.desc").offset(3) # skips the first 10 rows and returns the next 5 rows. >>> tab.order_by(tab.name.desc).offset(10).fetch(5) @@ -562,6 +590,7 @@ def fetch(self, fetch: int) -> 'Table': :: >>> tab.order_by(tab.name.desc).fetch(3) + >>> tab.order_by("name.desc").fetch(3) Skips the first 10 rows and returns the next 5 rows. :: @@ -670,7 +699,7 @@ def over_window(self, *over_windows: OverWindow) -> 'OverWindowedTable': [item._java_over_window for item in over_windows]) return OverWindowedTable(self._j_table.window(window_array), self._t_env) - def add_columns(self, *fields: Expression) -> 'Table': + def add_columns(self, *fields: Union[str, Expression]) -> 'Table': """ Adds additional columns. Similar to a SQL SELECT statement. The field expressions can contain complex expressions, but can not contain aggregations. It will throw an @@ -681,13 +710,19 @@ def add_columns(self, *fields: Expression) -> 'Table': >>> from pyflink.table import expressions as expr >>> tab.add_columns((tab.a + 1).alias('a1'), expr.concat(tab.b, 'sunny').alias('b1')) + >>> tab.add_columns("a + 1 as a1, concat(b, 'sunny') as b1") :param fields: Column list string. :return: The result table. """ - return Table(self._j_table.addColumns(to_expression_jarray(fields)), self._t_env) + if all(isinstance(f, Expression) for f in fields): + return Table(self._j_table.addColumns(to_expression_jarray(fields)), self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return Table(self._j_table.addColumns(fields[0]), self._t_env) - def add_or_replace_columns(self, *fields: Expression) -> 'Table': + def add_or_replace_columns(self, *fields: Union[str, Expression]) -> 'Table': """ Adds additional columns. Similar to a SQL SELECT statement. The field expressions can contain complex expressions, but can not contain aggregations. Existing fields will be @@ -700,14 +735,20 @@ def add_or_replace_columns(self, *fields: Expression) -> 'Table': >>> from pyflink.table import expressions as expr >>> tab.add_or_replace_columns((tab.a + 1).alias('a1'), ... expr.concat(tab.b, 'sunny').alias('b1')) + >>> tab.add_or_replace_columns("a + 1 as a1, concat(b, 'sunny') as b1") :param fields: Column list string. :return: The result table. """ - return Table(self._j_table.addOrReplaceColumns(to_expression_jarray(fields)), - self._t_env) + if all(isinstance(f, Expression) for f in fields): + return Table(self._j_table.addOrReplaceColumns(to_expression_jarray(fields)), + self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return Table(self._j_table.addOrReplaceColumns(fields[0]), self._t_env) - def rename_columns(self, *fields: Expression) -> 'Table': + def rename_columns(self, *fields: Union[str, Expression]) -> 'Table': """ Renames existing columns. Similar to a field alias statement. The field expressions should be alias expressions, and only the existing fields can be renamed. @@ -716,14 +757,20 @@ def rename_columns(self, *fields: Expression) -> 'Table': :: >>> tab.rename_columns(tab.a.alias('a1'), tab.b.alias('b1')) + >>> tab.rename_columns("a as a1, b as b1") :param fields: Column list string. :return: The result table. """ - return Table(self._j_table.renameColumns(to_expression_jarray(fields)), - self._t_env) + if all(isinstance(f, Expression) for f in fields): + return Table(self._j_table.renameColumns(to_expression_jarray(fields)), + self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return Table(self._j_table.renameColumns(fields[0]), self._t_env) - def drop_columns(self, *fields: Expression) -> 'Table': + def drop_columns(self, *fields: Union[str, Expression]) -> 'Table': """ Drops existing columns. The field expressions should be field reference expressions. @@ -731,14 +778,20 @@ def drop_columns(self, *fields: Expression) -> 'Table': :: >>> tab.drop_columns(tab.a, tab.b) + >>> tab.drop_columns("a, b") :param fields: Column list string. :return: The result table. """ - return Table(self._j_table.dropColumns(to_expression_jarray(fields)), - self._t_env) + if all(isinstance(f, Expression) for f in fields): + return Table(self._j_table.dropColumns(to_expression_jarray(fields)), + self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return Table(self._j_table.dropColumns(fields[0]), self._t_env) - def map(self, func: Union[Expression, UserDefinedScalarFunctionWrapper]) -> 'Table': + def map(self, func: Union[str, Expression, UserDefinedScalarFunctionWrapper]) -> 'Table': """ Performs a map operation with a user-defined scalar function. @@ -758,13 +811,15 @@ def map(self, func: Union[Expression, UserDefinedScalarFunctionWrapper]) -> 'Tab .. versionadded:: 1.13.0 """ - if isinstance(func, Expression): + if isinstance(func, str): + return Table(self._j_table.map(func), self._t_env) + elif isinstance(func, Expression): return Table(self._j_table.map(func._j_expr), self._t_env) else: func._set_takes_row_as_input() return Table(self._j_table.map(func(with_columns(col("*")))._j_expr), self._t_env) - def flat_map(self, func: Union[Expression, UserDefinedTableFunctionWrapper]) -> 'Table': + def flat_map(self, func: Union[str, Expression, UserDefinedTableFunctionWrapper]) -> 'Table': """ Performs a flatMap operation with a user-defined table function. @@ -788,13 +843,15 @@ def flat_map(self, func: Union[Expression, UserDefinedTableFunctionWrapper]) -> .. versionadded:: 1.13.0 """ - if isinstance(func, Expression): + if isinstance(func, str): + return Table(self._j_table.flatMap(func), self._t_env) + elif isinstance(func, Expression): return Table(self._j_table.flatMap(func._j_expr), self._t_env) else: func._set_takes_row_as_input() return Table(self._j_table.flatMap(func(with_columns(col("*")))._j_expr), self._t_env) - def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper]) \ + def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWrapper]) \ -> 'AggregatedTable': """ Performs a global aggregate operation with an aggregate function. You have to close the @@ -808,7 +865,7 @@ def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper] ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.aggregate(agg(tab.a).alias("a", "b")).select(col('a'), col('b')) + >>> tab.aggregate(agg(tab.a).alias("a", "b")).select("a, b") >>> # take all the columns as inputs >>> # pd is a Pandas.DataFrame >>> agg_row = udaf(lambda pd: (pd.a.mean(), pd.a.max()), @@ -816,14 +873,16 @@ def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper] ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.aggregate(agg.alias("a, b")).select(col('a'), col('b')) + >>> tab.aggregate(agg.alias("a, b")).select("a, b") :param func: user-defined aggregate function. :return: The result table. .. versionadded:: 1.13.0 """ - if isinstance(func, Expression): + if isinstance(func, str): + return AggregatedTable(self._j_table.aggregate(func), self._t_env) + elif isinstance(func, Expression): return AggregatedTable(self._j_table.aggregate(func._j_expr), self._t_env) else: func._set_takes_row_as_input() @@ -834,7 +893,7 @@ def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper] func = func(with_columns(col("*"))) return AggregatedTable(self._j_table.aggregate(func._j_expr), self._t_env) - def flat_aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper]) \ + def flat_aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWrapper]) \ -> 'FlatAggregateTable': """ Perform a global flat_aggregate without group_by. flat_aggregate takes a @@ -845,7 +904,7 @@ def flat_aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWra :: >>> table_agg = udtaf(MyTableAggregateFunction()) - >>> tab.flat_aggregate(table_agg(tab.a).alias("a", "b")).select(col('a'), col('b')) + >>> tab.flat_aggregate(table_agg(tab.a).alias("a", "b")).select("a, b") >>> # take all the columns as inputs >>> class Top2(TableAggregateFunction): ... def emit_value(self, accumulator): @@ -871,14 +930,16 @@ def flat_aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWra ... return DataTypes.ROW( ... [DataTypes.FIELD("a", DataTypes.BIGINT())]) >>> top2 = udtaf(Top2()) - >>> tab.flat_aggregate(top2.alias("a", "b")).select(col('a'), col('b')) + >>> tab.flat_aggregate(top2.alias("a", "b")).select("a, b") :param func: user-defined table aggregate function. :return: The result table. .. versionadded:: 1.13.0 """ - if isinstance(func, Expression): + if isinstance(func, str): + return FlatAggregateTable(self._j_table.flatAggregate(func), self._t_env) + elif isinstance(func, Expression): return FlatAggregateTable(self._j_table.flatAggregate(func._j_expr), self._t_env) else: func._set_takes_row_as_input() @@ -1063,7 +1124,7 @@ def __init__(self, java_table, t_env): self._j_table = java_table self._t_env = t_env - def select(self, *fields: Expression) -> 'Table': + def select(self, *fields: Union[str, Expression]) -> 'Table': """ Performs a selection operation on a grouped table. Similar to an SQL SELECT statement. The field expressions can contain complex expressions and aggregations. @@ -1072,13 +1133,20 @@ def select(self, *fields: Expression) -> 'Table': :: >>> tab.group_by(tab.key).select(tab.key, tab.value.avg.alias('average')) + >>> tab.group_by("key").select("key, value.avg as average") + :param fields: Expression string that contains group keys and aggregate function calls. :return: The result table. """ - return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) + if all(isinstance(f, Expression) for f in fields): + return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return Table(self._j_table.select(fields[0]), self._t_env) - def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper]) \ + def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWrapper]) \ -> 'AggregatedTable': """ Performs a aggregate operation with an aggregate function. You have to close the @@ -1092,8 +1160,7 @@ def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper] ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.group_by(tab.a).aggregate(agg(tab.b).alias("c", "d")).select( - ... col('a'), col('c'), col('d')) + >>> tab.group_by(tab.a).aggregate(agg(tab.b).alias("c", "d")).select("a, c, d") >>> # take all the columns as inputs >>> # pd is a Pandas.DataFrame >>> agg_row = udaf(lambda pd: (pd.a.mean(), pd.b.max()), @@ -1101,14 +1168,16 @@ def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper] ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.group_by(tab.a).aggregate(agg.alias("a, b")).select(col('a'), col('b')) + >>> tab.group_by(tab.a).aggregate(agg.alias("a, b")).select("a, b") :param func: user-defined aggregate function. :return: The result table. .. versionadded:: 1.13.0 """ - if isinstance(func, Expression): + if isinstance(func, str): + return AggregatedTable(self._j_table.aggregate(func), self._t_env) + elif isinstance(func, Expression): return AggregatedTable(self._j_table.aggregate(func._j_expr), self._t_env) else: func._set_takes_row_as_input() @@ -1119,7 +1188,7 @@ def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper] func = func(with_columns(col("*"))) return AggregatedTable(self._j_table.aggregate(func._j_expr), self._t_env) - def flat_aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper]) \ + def flat_aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWrapper]) \ -> 'FlatAggregateTable': """ Performs a flat_aggregate operation on a grouped table. flat_aggregate takes a @@ -1130,8 +1199,7 @@ def flat_aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWra :: >>> table_agg = udtaf(MyTableAggregateFunction()) - >>> tab.group_by(tab.c).flat_aggregate(table_agg(tab.a).alias("a")).select( - ... col('c'), col('a')) + >>> tab.group_by(tab.c).flat_aggregate(table_agg(tab.a).alias("a")).select("c, a") >>> # take all the columns as inputs >>> class Top2(TableAggregateFunction): ... def emit_value(self, accumulator): @@ -1157,14 +1225,16 @@ def flat_aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWra ... return DataTypes.ROW( ... [DataTypes.FIELD("a", DataTypes.BIGINT())]) >>> top2 = udtaf(Top2()) - >>> tab.group_by(tab.c).flat_aggregate(top2.alias("a", "b")).select(col('a'), col('b')) + >>> tab.group_by(tab.c).flat_aggregate(top2.alias("a", "b")).select("a, b") :param func: user-defined table aggregate function. :return: The result table. .. versionadded:: 1.13.0 """ - if isinstance(func, Expression): + if isinstance(func, str): + return FlatAggregateTable(self._j_table.flatAggregate(func), self._t_env) + elif isinstance(func, Expression): return FlatAggregateTable(self._j_table.flatAggregate(func._j_expr), self._t_env) else: func._set_takes_row_as_input() @@ -1185,7 +1255,7 @@ def __init__(self, java_group_windowed_table, t_env): self._j_table = java_group_windowed_table self._t_env = t_env - def group_by(self, *fields: Expression) -> 'WindowGroupedTable': + def group_by(self, *fields: Union[str, Expression]) -> 'WindowGroupedTable': """ Groups the elements by a mandatory window and one or more optional grouping attributes. The window is specified by referring to its alias. @@ -1211,8 +1281,13 @@ def group_by(self, *fields: Expression) -> 'WindowGroupedTable': :param fields: Group keys. :return: A window grouped table. """ - return WindowGroupedTable( - self._j_table.groupBy(to_expression_jarray(fields)), self._t_env) + if all(isinstance(f, Expression) for f in fields): + return WindowGroupedTable( + self._j_table.groupBy(to_expression_jarray(fields)), self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return WindowGroupedTable(self._j_table.groupBy(fields[0]), self._t_env) class WindowGroupedTable(object): @@ -1224,7 +1299,7 @@ def __init__(self, java_window_grouped_table, t_env): self._j_table = java_window_grouped_table self._t_env = t_env - def select(self, *fields: Expression) -> 'Table': + def select(self, *fields: Union[str, Expression]) -> 'Table': """ Performs a selection operation on a window grouped table. Similar to an SQL SELECT statement. @@ -1236,6 +1311,7 @@ def select(self, *fields: Expression) -> 'Table': >>> window_grouped_table.select(col('key'), ... col('window').start, ... col('value').avg.alias('valavg')) + >>> window_grouped_table.select("key, window.start, value.avg as valavg") :param fields: Expression string. :return: The result table. @@ -1247,7 +1323,7 @@ def select(self, *fields: Expression) -> 'Table': assert isinstance(fields[0], str) return Table(self._j_table.select(fields[0]), self._t_env) - def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper]) \ + def aggregate(self, func: Union[str, Expression, UserDefinedAggregateFunctionWrapper]) \ -> 'AggregatedTable': """ Performs an aggregate operation on a window grouped table. You have to close the @@ -1264,7 +1340,7 @@ def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper] >>> window_grouped_table.group_by("w") \ ... .aggregate(agg(window_grouped_table.b) \ ... .alias("c", "d")) \ - ... .select(col('c'), col('d')) + ... .select("c, d") >>> # take all the columns as inputs >>> # pd is a Pandas.DataFrame >>> agg_row = udaf(lambda pd: (pd.a.mean(), pd.b.max()), @@ -1279,7 +1355,9 @@ def aggregate(self, func: Union[Expression, UserDefinedAggregateFunctionWrapper] .. versionadded:: 1.13.0 """ - if isinstance(func, Expression): + if isinstance(func, str): + return AggregatedTable(self._j_table.aggregate(func), self._t_env) + elif isinstance(func, Expression): return AggregatedTable(self._j_table.aggregate(func._j_expr), self._t_env) else: func._set_takes_row_as_input() @@ -1313,7 +1391,7 @@ def __init__(self, java_over_windowed_table, t_env): self._j_table = java_over_windowed_table self._t_env = t_env - def select(self, *fields: Expression) -> 'Table': + def select(self, *fields: Union[str, Expression]) -> 'Table': """ Performs a selection operation on a over windowed table. Similar to an SQL SELECT statement. @@ -1325,11 +1403,17 @@ def select(self, *fields: Expression) -> 'Table': >>> over_windowed_table.select(col('c'), ... col('b').count.over(col('ow')), ... col('e').sum.over(col('ow'))) + >>> over_windowed_table.select("c, b.count over ow, e.sum over ow") :param fields: Expression string. :return: The result table. """ - return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) + if all(isinstance(f, Expression) for f in fields): + return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return Table(self._j_table.select(fields[0]), self._t_env) class AggregatedTable(object): @@ -1341,7 +1425,7 @@ def __init__(self, java_table, t_env): self._j_table = java_table self._t_env = t_env - def select(self, *fields: Expression) -> 'Table': + def select(self, *fields: Union[str, Expression]) -> 'Table': """ Performs a selection operation after an aggregate operation. The field expressions cannot contain table functions and aggregations. @@ -1354,7 +1438,7 @@ def select(self, *fields: Expression) -> 'Table': ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.aggregate(agg(tab.a).alias("a", "b")).select(col('a'), col('b')) + >>> tab.aggregate(agg(tab.a).alias("a", "b")).select("a, b") >>> # take all the columns as inputs >>> # pd is a Pandas.DataFrame >>> agg_row = udaf(lambda pd: (pd.a.mean(), pd.b.max()), @@ -1362,12 +1446,17 @@ def select(self, *fields: Expression) -> 'Table': ... [DataTypes.FIELD("a", DataTypes.FLOAT()), ... DataTypes.FIELD("b", DataTypes.INT())]), ... func_type="pandas") - >>> tab.group_by(tab.a).aggregate(agg.alias("a, b")).select(col('a'), col('b')) + >>> tab.group_by(tab.a).aggregate(agg.alias("a, b")).select("a, b") :param fields: Expression string. :return: The result table. """ - return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) + if all(isinstance(f, Expression) for f in fields): + return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return Table(self._j_table.select(fields[0]), self._t_env) class FlatAggregateTable(object): @@ -1380,7 +1469,7 @@ def __init__(self, java_table, t_env): self._j_table = java_table self._t_env = t_env - def select(self, *fields: Expression) -> 'Table': + def select(self, *fields: Union[str, Expression]) -> 'Table': """ Performs a selection operation on a FlatAggregateTable. Similar to a SQL SELECT statement. The field expressions can contain complex expressions. @@ -1389,7 +1478,7 @@ def select(self, *fields: Expression) -> 'Table': :: >>> table_agg = udtaf(MyTableAggregateFunction()) - >>> tab.flat_aggregate(table_agg(tab.a).alias("a", "b")).select(col('a'), col('b')) + >>> tab.flat_aggregate(table_agg(tab.a).alias("a", "b")).select("a, b") >>> # take all the columns as inputs >>> class Top2(TableAggregateFunction): ... def emit_value(self, accumulator): @@ -1415,9 +1504,14 @@ def select(self, *fields: Expression) -> 'Table': ... return DataTypes.ROW( ... [DataTypes.FIELD("a", DataTypes.BIGINT())]) >>> top2 = udtaf(Top2()) - >>> tab.group_by(tab.c).flat_aggregate(top2.alias("a", "b")).select(col('a'), col('b')) + >>> tab.group_by(tab.c).flat_aggregate(top2.alias("a", "b")).select("a, b") :param fields: Expression string. :return: The result table. """ - return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) + if all(isinstance(f, Expression) for f in fields): + return Table(self._j_table.select(to_expression_jarray(fields)), self._t_env) + else: + assert len(fields) == 1 + assert isinstance(fields[0], str) + return Table(self._j_table.select(fields[0]), self._t_env) diff --git a/flink-python/pyflink/table/window.py b/flink-python/pyflink/table/window.py index 630455e3d5f4d..282660c648196 100644 --- a/flink-python/pyflink/table/window.py +++ b/flink-python/pyflink/table/window.py @@ -144,6 +144,9 @@ class Session(object): >>> Session.with_gap(expr.lit(10).minutes) ... .on(expr.col("rowtime")) ... .alias("w") + + >>> Session.with_gap("10.minutes").on("rowtime").alias("w") + """ @classmethod @@ -227,6 +230,8 @@ class Slide(object): ... .every(expr.lit(5).minutes) ... .on(expr.col("rowtime")) ... .alias("w") + + >>> Slide.over("10.minutes").every("5.minutes").on("rowtime").alias("w") """ @classmethod @@ -333,6 +338,8 @@ class Over(object): ... .order_by(col("rowtime")) \\ ... .preceding(expr.UNBOUNDED_RANGE) \\ ... .alias("w") + + >>> Over.partition_by("a").order_by("rowtime").preceding("unbounded_range").alias("w") """ @classmethod From b3428b345a230446d7a502ef5e6827a2ab826d4e Mon Sep 17 00:00:00 2001 From: Mulavar <978007503@qq.com> Date: Thu, 24 Mar 2022 21:42:20 +0800 Subject: [PATCH 139/258] [FLINK-26738][state] Mark StateDescriptor#defaultValue as deprecated with clear docs This closes #19229. --- .../docs/dev/datastream/fault-tolerance/state.md | 2 ++ .../docs/dev/datastream/fault-tolerance/state.md | 2 ++ .../flink/api/common/state/StateDescriptor.java | 12 ++++++++++-- 3 files changed, 14 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/dev/datastream/fault-tolerance/state.md b/docs/content.zh/docs/dev/datastream/fault-tolerance/state.md index ac260aa429eb5..8e2049054c5e1 100644 --- a/docs/content.zh/docs/dev/datastream/fault-tolerance/state.md +++ b/docs/content.zh/docs/dev/datastream/fault-tolerance/state.md @@ -383,6 +383,8 @@ Heap state backend 会额外存储一个包括用户状态以及时间戳的 Jav - 当前开启 TTL 的 map state 仅在用户值序列化器支持 null 的情况下,才支持用户值为 null。如果用户值序列化器不支持 null, 可以用 `NullableSerializer` 包装一层。 + +- 启用 TTL 配置后,`StateDescriptor` 中的 `defaultValue`(已被标记 `deprecated`)将会失效。这个设计的目的是为了确保语义更加清晰,在此基础上,用户需要手动管理那些实际值为 null 或已过期的状态默认值。 #### 过期数据的清理 diff --git a/docs/content/docs/dev/datastream/fault-tolerance/state.md b/docs/content/docs/dev/datastream/fault-tolerance/state.md index 9cc3e23e31b38..5af6e2635af2c 100644 --- a/docs/content/docs/dev/datastream/fault-tolerance/state.md +++ b/docs/content/docs/dev/datastream/fault-tolerance/state.md @@ -432,6 +432,8 @@ will lead to compatibility failure and `StateMigrationException`. - The map state with TTL currently supports null user values only if the user value serializer can handle null values. If the serializer does not support null values, it can be wrapped with `NullableSerializer` at the cost of an extra byte in the serialized form. +- With TTL enabled configuration, the `defaultValue` in `StateDescriptor`, which is atucally already deprecated, will no longer take an effect. This aims to make the semantics more clear and let user manually manage the default value if the contents of the state is null or expired. + #### Cleanup of Expired State By default, expired values are explicitly removed on read, such as `ValueState#value`, and periodically garbage collected diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java index 4af9b44233da1..309abfb391917 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/StateDescriptor.java @@ -103,8 +103,13 @@ public enum Type { /** The configuration of state time-to-live(TTL), it is disabled by default. */ @Nonnull private StateTtlConfig ttlConfig = StateTtlConfig.DISABLED; - /** The default value returned by the state when no other value is bound to a key. */ - @Nullable protected transient T defaultValue; + /** + * The default value returned by the state when no other value is bound to a key. + * + * @deprecated To make the semantics more clear, user should manually manage the default value + * if the contents of the state is {@code null} + */ + @Nullable @Deprecated protected transient T defaultValue; // ------------------------------------------------------------------------ @@ -260,6 +265,9 @@ public boolean isQueryable() { *

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

If enabling the TTL configuration, the field {@link StateDescriptor#defaultValue} will be + * invalid. + * * @param ttlConfig configuration of state TTL */ public void enableTimeToLive(StateTtlConfig ttlConfig) { From 8f502b808eea89cc53f6aed917c12f42e0661280 Mon Sep 17 00:00:00 2001 From: Zakelly Date: Sat, 2 Apr 2022 11:33:52 +0800 Subject: [PATCH 140/258] [FLINK-26998][state] Activate default log-level options in RocksDB state backend this closes #19335. --- .../streaming/state/PredefinedOptions.java | 17 ++--------------- .../state/RocksDBStateBackendConfigTest.java | 3 +++ 2 files changed, 5 insertions(+), 15 deletions(-) diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java index 25e5fa04d5908..d69b54314f0ac 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java @@ -22,7 +22,6 @@ import org.apache.flink.configuration.MemorySize; import org.rocksdb.CompactionStyle; -import org.rocksdb.InfoLogLevel; import javax.annotation.Nullable; @@ -54,15 +53,9 @@ public enum PredefinedOptions { *

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

The following options are set: - * - *

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

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

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

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

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

    Note: Because Flink does not rely on RocksDB data on disk for recovery, there is no need @@ -172,7 +160,6 @@ public enum PredefinedOptions { { put(RocksDBConfigurableOptions.MAX_BACKGROUND_THREADS, 4); put(RocksDBConfigurableOptions.MAX_OPEN_FILES, -1); - put(RocksDBConfigurableOptions.LOG_LEVEL, InfoLogLevel.HEADER_LEVEL); } }); diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java index a1d93df798a8e..8a4af6b7b89d4 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java @@ -102,6 +102,9 @@ public void testDefaultDbLogDir() throws Exception { // set the environment variable 'log.file' with the Flink log file location System.setProperty("log.file", logFile.getPath()); try (RocksDBResourceContainer container = backend.createOptionsAndResourceContainer()) { + assertEquals( + RocksDBConfigurableOptions.LOG_LEVEL.defaultValue(), + container.getDbOptions().infoLogLevel()); assertEquals(logFile.getParent(), container.getDbOptions().dbLogDir()); } finally { logFile.delete(); From aa3bb951db745f94070f2ef6ecb62ce207bda520 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Fri, 1 Apr 2022 13:44:41 +0200 Subject: [PATCH 141/258] [FLINK-26987][runtime] Fixes getAllAndLock livelock This livelock can happen in situations where an entry was marked for deletion but is not deleted, yet. --- .../zookeeper/ZooKeeperStateHandleStore.java | 10 +++-- .../ZooKeeperStateHandleStoreTest.java | 40 +++++++++++++++++++ 2 files changed, 47 insertions(+), 3 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java index c4388d410bf1b..0217ffdac9bc2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStore.java @@ -391,7 +391,6 @@ List, String>> getAllAndLock( final String rootPath = "/"; boolean success = false; - retry: while (!success) { stateHandles.clear(); @@ -411,8 +410,13 @@ List, String>> getAllAndLock( final RetrievableStateHandle stateHandle = getAndLock(path); stateHandles.add(new Tuple2<>(stateHandle, path)); } catch (NotExistException ignored) { - // Concurrent deletion, retry - continue retry; + // The node is subject for deletion which can mean two things: + // 1. The state is marked for deletion: The cVersion of the node does not + // necessarily change. We're not interested in the state anymore, anyway. + // Therefore, this error can be ignored. + // 2. An actual concurrent deletion is going on. The child node is gone. + // That would affect the cVersion of the parent node and, as a consequence, + // would trigger a restart the logic through the while loop. } catch (IOException ioException) { LOG.warn( "Could not get all ZooKeeper children. Node {} contained " diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java index ddf6fc7b2071a..eff53576fc183 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/zookeeper/ZooKeeperStateHandleStoreTest.java @@ -867,6 +867,46 @@ public void testGetAllAndLockOnConcurrentDelete() throws Exception { Iterables.getOnlyElement(actuallyLockedHandles).f0.retrieveState().getValue()); } + @Test + public void testGetAllAndLockWhileEntryIsMarkedForDeletion() throws Exception { + final TestingLongStateHandleHelper stateHandleProvider = new TestingLongStateHandleHelper(); + final CuratorFramework client = + ZooKeeperUtils.useNamespaceAndEnsurePath( + ZOOKEEPER.getClient(), "/testGetAllAndLockWhileEntryIsMarkedForDeletion"); + + final ZooKeeperStateHandleStore + stateHandleStore = new ZooKeeperStateHandleStore<>(client, stateHandleProvider); + + final String pathInZooKeeperPrefix = "/node"; + + final long stateForDeletion = 42L; + final String handlePathForDeletion = pathInZooKeeperPrefix + "-for-deletion"; + stateHandleStore.addAndLock( + handlePathForDeletion, + new TestingLongStateHandleHelper.LongStateHandle(stateForDeletion)); + // marks the entry for deletion but doesn't delete it, yet + client.delete() + .deletingChildrenIfNeeded() + .forPath(ZooKeeperStateHandleStore.getRootLockPath(handlePathForDeletion)); + + final long stateToKeep = stateForDeletion + 2; + stateHandleStore.addAndLock( + pathInZooKeeperPrefix + "-keep", + new TestingLongStateHandleHelper.LongStateHandle(stateToKeep)); + + final List< + Tuple2< + RetrievableStateHandle< + TestingLongStateHandleHelper.LongStateHandle>, + String>> + actuallyLockedHandles = stateHandleStore.getAllAndLock(); + + assertEquals( + "Only the StateHandle that was expected to be kept should be returned.", + stateToKeep, + Iterables.getOnlyElement(actuallyLockedHandles).f0.retrieveState().getValue()); + } + /** Tests that the state is returned sorted. */ @Test public void testGetAllSortedByName() throws Exception { From 7c308e8073278c90f0768c59427e3c62cb7851f1 Mon Sep 17 00:00:00 2001 From: Sergey Nuyanzin Date: Thu, 31 Mar 2022 15:42:00 +0200 Subject: [PATCH 142/258] [FLINK-26961][connectors][filesystems][formats] Update Jackson Databind and Annotations to 2.13.2.2, Jackson Dataformat to 2.13.2, Jackson Core to 2.13.2 and Jackson-BOM to 2.13.2.20220328. This closes #19339 --- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 12 ++++++------ .../src/main/resources/META-INF/NOTICE | 12 ++++++------ .../src/main/resources/META-INF/NOTICE | 8 ++++---- .../pom.xml | 6 +++--- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 6 +++--- .../src/main/resources/META-INF/NOTICE | 2 +- .../src/main/resources/META-INF/NOTICE | 8 ++++---- .../src/main/resources/META-INF/NOTICE | 8 ++++---- .../src/main/resources/META-INF/NOTICE | 6 +++--- .../src/main/resources/META-INF/NOTICE | 6 +++--- flink-kubernetes/src/main/resources/META-INF/NOTICE | 10 +++++----- flink-python/src/main/resources/META-INF/NOTICE | 6 +++--- pom.xml | 2 +- 15 files changed, 48 insertions(+), 48 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE index a61e1eddbb234..00e24d6d2f510 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE @@ -55,7 +55,7 @@ This project bundles the following dependencies under the Apache Software Licens - com.typesafe.netty:netty-reactive-streams-http:2.0.5 - com.typesafe.netty:netty-reactive-streams:2.0.5 - commons-logging:commons-logging:1.1.3 -- com.fasterxml.jackson.core:jackson-core:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 This project bundles the following dependencies under the BSD license. See bundled license files for details. diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE index de1f4f377a249..f2e61f6fd243d 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch6/src/main/resources/META-INF/NOTICE @@ -6,12 +6,12 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.2 - commons-codec:commons-codec:1.15 - commons-logging:commons-logging:1.1.3 - org.apache.httpcomponents:httpasyncclient:4.1.2 diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index fa04fce164d88..718480db4e420 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -7,12 +7,12 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - com.carrotsearch:hppc:0.8.1 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.2 - com.github.spullara.mustache.java:compiler:0.9.6 - commons-codec:commons-codec:1.15 - commons-logging:commons-logging:1.1.3 diff --git a/flink-connectors/flink-sql-connector-kinesis/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-kinesis/src/main/resources/META-INF/NOTICE index fdb21a08d573e..477b69ba68b17 100644 --- a/flink-connectors/flink-sql-connector-kinesis/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-kinesis/src/main/resources/META-INF/NOTICE @@ -14,7 +14,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.commons:commons-lang3:3.3.2 - com.google.guava:guava:29.0-jre - com.google.guava:failureaccess:1.0.1 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml index ed4a6c80c8f0a..31cc067fe431e 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml @@ -72,19 +72,19 @@ com.fasterxml.jackson.core jackson-core - 2.13.0 + 2.13.2 com.fasterxml.jackson.core jackson-annotations - 2.13.0 + 2.13.2 com.fasterxml.jackson.core jackson-databind - 2.13.0 + 2.13.2.2 diff --git a/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE index c2a59b5105ae2..63516ed042709 100644 --- a/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-azure-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -6,7 +6,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 - com.google.errorprone:error_prone_annotations:2.2.0 - com.google.guava:failureaccess:1.0 - com.google.guava:guava:27.0-jre diff --git a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE index e55f90db980cd..39269ee8e1751 100644 --- a/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-fs-hadoop-shaded/src/main/resources/META-INF/NOTICE @@ -22,9 +22,9 @@ This project bundles the following dependencies under the Apache Software Licens - com.google.guava:guava:27.0-jre - com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava - com.google.j2objc:j2objc-annotations:1.1 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 - com.fasterxml.woodstox:woodstox-core:5.0.3 This project bundles the following dependencies under the MIT (https://opensource.org/licenses/MIT) diff --git a/flink-filesystems/flink-gs-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-gs-fs-hadoop/src/main/resources/META-INF/NOTICE index fd302700df886..aa1afa70fa786 100644 --- a/flink-filesystems/flink-gs-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-gs-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -6,7 +6,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-core:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 - com.google.android:annotations:4.1.1.4 - com.google.api-client:google-api-client-jackson2:1.32.2 - com.google.api-client:google-api-client:1.33.0 diff --git a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE index 02d7e43585a3f..a38114f6ef79f 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-s3-fs-hadoop/src/main/resources/META-INF/NOTICE @@ -9,10 +9,10 @@ This project bundles the following dependencies under the Apache Software Licens - com.amazonaws:aws-java-sdk-s3:1.11.951 - com.amazonaws:aws-java-sdk-sts:1.11.951 - com.amazonaws:jmespath-java:1.11.951 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 - com.fasterxml.woodstox:woodstox-core:5.0.3 - com.google.errorprone:error_prone_annotations:2.2.0 - com.google.guava:failureaccess:1.0 diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE index 65ee90d99e0b0..037a9f881e3ad 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE @@ -22,10 +22,10 @@ This project bundles the following dependencies under the Apache Software Licens - com.facebook.presto:presto-hive-common:0.257 - com.facebook.presto:presto-hive-metastore:0.257 - com.facebook.presto.hadoop:hadoop-apache2:2.7.4-9 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 - com.fasterxml.woodstox:woodstox-core:5.0.3 - com.google.guava:guava:26.0-jre - com.google.inject:guice:4.2.2 diff --git a/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE index bfddc9b0f8e80..48394cfcebf36 100644 --- a/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE +++ b/flink-formats/flink-sql-avro-confluent-registry/src/main/resources/META-INF/NOTICE @@ -7,9 +7,9 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - org.apache.avro:avro:1.10.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 - org.apache.commons:commons-compress:1.21 - io.confluent:kafka-schema-registry-client:6.2.2 - org.apache.kafka:kafka-clients:6.2.2-ccs diff --git a/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE b/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE index dcf6dcc9a540a..0096914beaca8 100644 --- a/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE +++ b/flink-formats/flink-sql-avro/src/main/resources/META-INF/NOTICE @@ -7,7 +7,7 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0. (http://www.apache.org/licenses/LICENSE-2.0.txt) - org.apache.avro:avro:1.10.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 - org.apache.commons:commons-compress:1.21 diff --git a/flink-kubernetes/src/main/resources/META-INF/NOTICE b/flink-kubernetes/src/main/resources/META-INF/NOTICE index 1adda9dcd7562..08f96e407167f 100644 --- a/flink-kubernetes/src/main/resources/META-INF/NOTICE +++ b/flink-kubernetes/src/main/resources/META-INF/NOTICE @@ -6,11 +6,11 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 -- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.0 -- com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 +- com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.2 +- com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.13.2 - com.squareup.okhttp3:logging-interceptor:3.14.9 - com.squareup.okhttp3:okhttp:3.14.9 - com.squareup.okio:okio:1.17.2 diff --git a/flink-python/src/main/resources/META-INF/NOTICE b/flink-python/src/main/resources/META-INF/NOTICE index 008988b175eaa..877a483f5380d 100644 --- a/flink-python/src/main/resources/META-INF/NOTICE +++ b/flink-python/src/main/resources/META-INF/NOTICE @@ -6,9 +6,9 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- com.fasterxml.jackson.core:jackson-annotations:2.13.0 -- com.fasterxml.jackson.core:jackson-core:2.13.0 -- com.fasterxml.jackson.core:jackson-databind:2.13.0 +- com.fasterxml.jackson.core:jackson-annotations:2.13.2 +- com.fasterxml.jackson.core:jackson-core:2.13.2 +- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 - com.google.flatbuffers:flatbuffers-java:1.9.0 - io.netty:netty-buffer:4.1.70.Final - io.netty:netty-common:4.1.70.Final diff --git a/pom.xml b/pom.xml index 5e4498125db7c..711c631e097ee 100644 --- a/pom.xml +++ b/pom.xml @@ -544,7 +544,7 @@ under the License. jackson-bom pom import - 2.13.0 + 2.13.2.20220328 From 1f83c5e528c69221144998c3cc5d5d9177ce3ec2 Mon Sep 17 00:00:00 2001 From: wangzhiwu Date: Mon, 7 Feb 2022 21:43:56 +0800 Subject: [PATCH 143/258] [FLINK-25797][Docs] Translate datastream/formats/parquet.md page into Chinese. Co-authored-by: Jing Ge Co-authored-by: wangzhiwubigdata Co-authored-by: Zhiwu Wang <2827873682@qq.com> --- .../connectors/datastream/formats/parquet.md | 221 +++++++++--------- 1 file changed, 106 insertions(+), 115 deletions(-) diff --git a/docs/content.zh/docs/connectors/datastream/formats/parquet.md b/docs/content.zh/docs/connectors/datastream/formats/parquet.md index 87c1a1b3dd794..16b0249c72bb8 100644 --- a/docs/content.zh/docs/connectors/datastream/formats/parquet.md +++ b/docs/content.zh/docs/connectors/datastream/formats/parquet.md @@ -26,21 +26,22 @@ under the License. --> + + # Parquet format -Flink supports reading [Parquet](https://parquet.apache.org/) files, -producing {{< javadoc file="org/apache/flink/table/data/RowData.html" name="Flink RowData">}} and producing [Avro](https://avro.apache.org/) records. -To use the format you need to add the `flink-parquet` dependency to your project: +Flink 支持读取 [Parquet](https://parquet.apache.org/) 文件并生成 {{< javadoc file="org/apache/flink/table/data/RowData.html" name="Flink RowData">}} 和 [Avro](https://avro.apache.org/) 记录。 +要使用 Parquet format,你需要将 flink-parquet 依赖添加到项目中: ```xml - org.apache.flink - flink-parquet - {{< version >}} + org.apache.flink + flink-parquet + {{< version >}} ``` -To read Avro records, you will need to add the `parquet-avro` dependency: +要使用 Avro 格式,你需要将 parquet-avro 依赖添加到项目中: ```xml @@ -61,83 +62,78 @@ To read Avro records, you will need to add the `parquet-avro` dependency: ``` -This format is compatible with the new Source that can be used in both batch and streaming execution modes. -Thus, you can use this format for two kinds of data: +此格式与新的 Source 兼容,可以同时在批和流模式下使用。 +因此,你可使用此格式处理以下两类数据: -- Bounded data: lists all files and reads them all. -- Unbounded data: monitors a directory for new files that appear. +- 有界数据: 列出所有文件并全部读取。 +- 无界数据:监控目录中出现的新文件 {{< hint info >}} -When you start a File Source it is configured for bounded data by default. -To configure the File Source for unbounded data, you must additionally call -`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`. +当你开启一个 File Source,会被默认为有界读取。 +如果你想在连续读取模式下使用 File Source,你必须额外调用 +`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`。 {{< /hint >}} **Vectorized reader** ```java - // Parquet rows are decoded in batches FileSource.forBulkFileFormat(BulkFormat,Path...) - // Monitor the Paths to read data as unbounded data FileSource.forBulkFileFormat(BulkFormat,Path...) - .monitorContinuously(Duration.ofMillis(5L)) - .build(); - +.monitorContinuously(Duration.ofMillis(5L)) +.build(); ``` **Avro Parquet reader** ```java - // Parquet rows are decoded in batches FileSource.forRecordStreamFormat(StreamFormat,Path...) - // Monitor the Paths to read data as unbounded data FileSource.forRecordStreamFormat(StreamFormat,Path...) .monitorContinuously(Duration.ofMillis(5L)) .build(); - - ``` {{< hint info >}} -Following examples are all configured for bounded data. -To configure the File Source for unbounded data, you must additionally call -`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`. +下面的案例都是基于有界数据的。 +如果你想在连续读取模式下使用 File Source,你必须额外调用 +`AbstractFileSource.AbstractFileSourceBuilder.monitorContinuously(Duration)`。 {{< /hint >}} + + ## Flink RowData -In this example, you will create a DataStream containing Parquet records as Flink RowDatas. The schema is projected to read only the specified fields ("f7", "f4" and "f99"). -Flink will read records in batches of 500 records. The first boolean parameter specifies that timestamp columns will be interpreted as UTC. -The second boolean instructs the application that the projected Parquet fields names are case-sensitive. -There is no watermark strategy defined as records do not contain event timestamps. +在此示例中,你将创建由 Parquet 格式的记录构成的 Flink RowDatas DataStream。我们把 schema 信息映射为只读字段("f7"、"f4" 和 "f99")。 +每个批次读取 500 条记录。其中,第一个布尔类型的参数用来指定是否需要将时间戳列处理为 UTC。 +第二个布尔类型参数用来指定在进行 Parquet 字段映射时,是否要区分大小写。 +这里不需要水印策略,因为记录中不包含事件时间戳。 ```java final LogicalType[] fieldTypes = - new LogicalType[] { - new DoubleType(), new IntType(), new VarCharType() - }; + new LogicalType[] { + new DoubleType(), new IntType(), new VarCharType() + }; final ParquetColumnarRowInputFormat format = - new ParquetColumnarRowInputFormat<>( - new Configuration(), - RowType.of(fieldTypes, new String[] {"f7", "f4", "f99"}), - 500, - false, - true); + new ParquetColumnarRowInputFormat<>( + new Configuration(), + RowType.of(fieldTypes, new String[] {"f7", "f4", "f99"}), + 500, + false, + true); final FileSource source = - FileSource.forBulkFileFormat(format, /* Flink Path */) - .build(); + FileSource.forBulkFileFormat(format, /* Flink Path */) + .build(); final DataStream stream = - env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); + env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` ## Avro Records -Flink supports producing three types of Avro records by reading Parquet files: +Flink 支持三种方式来读取 Parquet 文件并创建 Avro records : - [Generic record](https://avro.apache.org/docs/1.10.0/api/java/index.html) - [Specific record](https://avro.apache.org/docs/1.10.0/api/java/index.html) @@ -145,62 +141,62 @@ Flink supports producing three types of Avro records by reading Parquet files: ### Generic record -Avro schemas are defined using JSON. You can get more information about Avro schemas and types from the [Avro specification](https://avro.apache.org/docs/1.10.0/spec.html). -This example uses an Avro schema example similar to the one described in the [official Avro tutorial](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html): +使用 JSON 定义 Avro schemas。你可以从 [Avro specification](https://avro.apache.org/docs/1.10.0/spec.html) 获取更多关于 Avro schemas 和类型的信息。 +此示例使用了一个在 [official Avro tutorial](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html) 中描述的示例相似的 Avro schema: ```json lines {"namespace": "example.avro", - "type": "record", - "name": "User", - "fields": [ - {"name": "name", "type": "string"}, - {"name": "favoriteNumber", "type": ["int", "null"]}, - {"name": "favoriteColor", "type": ["string", "null"]} - ] + "type": "record", + "name": "User", + "fields": [ + {"name": "name", "type": "string"}, + {"name": "favoriteNumber", "type": ["int", "null"]}, + {"name": "favoriteColor", "type": ["string", "null"]} + ] } ``` +这个 schema 定义了一个具有三个属性的的 user 记录:name,favoriteNumber 和 favoriteColor。你可以 +在 [record specification](https://avro.apache.org/docs/1.10.0/spec.html#schema_record) 找到更多关于如何定义 Avro schema 的详细信息。 -This schema defines a record representing a user with three fields: name, favoriteNumber, and favoriteColor. You can find more details at [record specification](https://avro.apache.org/docs/1.10.0/spec.html#schema_record) for how to define an Avro schema. - -In the following example, you will create a DataStream containing Parquet records as Avro Generic records. -It will parse the Avro schema based on the JSON string. There are many other ways to parse a schema, e.g. from java.io.File or java.io.InputStream. Please refer to [Avro Schema](https://avro.apache.org/docs/1.10.0/api/java/org/apache/avro/Schema.html) for details. -After that, you will create an `AvroParquetRecordFormat` via `AvroParquetReaders` for Avro Generic records. +在此示例中,你将创建包含由 Avro Generic records 格式构成的 Parquet records 的 DataStream。 +Flink 会基于 JSON 字符串解析 Avro schema。也有很多其他的方式解析 schema,例如基于 java.io.File 或 java.io.InputStream。 +请参考 [Avro Schema](https://avro.apache.org/docs/1.10.0/api/java/org/apache/avro/Schema.html) 以获取更多详细信息。 +然后,你可以通过 `AvroParquetReaders` 为 Avro Generic 记录创建 `AvroParquetRecordFormat`。 ```java -// parsing avro schema +// 解析 avro schema final Schema schema = new Schema.Parser() - .parse( - "{\"type\": \"record\", " - + "\"name\": \"User\", " - + "\"fields\": [\n" - + " {\"name\": \"name\", \"type\": \"string\" },\n" - + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" - + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" - + " ]\n" - + " }"); + .parse( + "{\"type\": \"record\", " + + "\"name\": \"User\", " + + "\"fields\": [\n" + + " {\"name\": \"name\", \"type\": \"string\" },\n" + + " {\"name\": \"favoriteNumber\", \"type\": [\"int\", \"null\"] },\n" + + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" + + " ]\n" + + " }"); final FileSource source = FileSource.forRecordStreamFormat( - AvroParquetReaders.forGenericRecord(schema), /* Flink Path */) + AvroParquetReaders.forGenericRecord(schema), /* Flink Path */) .build(); final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(10L); - + final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` ### Specific record -Based on the previously defined schema, you can generate classes by leveraging Avro code generation. -Once the classes have been generated, there is no need to use the schema directly in your programs. -You can either use `avro-tools.jar` to generate code manually or you could use the Avro Maven plugin to perform -code generation on any .avsc files present in the configured source directory. Please refer to -[Avro Getting Started](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html) for more information. +基于之前定义的 schema,你可以通过利用 Avro 代码生成来生成类。 +一旦生成了类,就不需要在程序中直接使用 schema。 +你可以使用 `avro-tools.jar` 手动生成代码,也可以直接使用 Avro Maven 插件对配置的源目录中的任何 .avsc 文件执行代码生成。 +请参考 [Avro Getting Started](https://avro.apache.org/docs/1.10.0/gettingstartedjava.html) 获取更多信息。 -The following example uses the example schema [testdata.avsc](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc): +此示例使用了样例 schema [testdata.avsc](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/resources/avro/testdata.avsc): ```json lines [ @@ -218,17 +214,17 @@ The following example uses the example schema [testdata.avsc](https://github.com ] ``` -You will use the Avro Maven plugin to generate the `Address` Java class: +你可以使用 Avro Maven plugin 生成 `Address` Java 类。 ```java @org.apache.avro.specific.AvroGenerated public class Address extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { - // generated code... + // 生成的代码... } ``` -You will create an `AvroParquetRecordFormat` via `AvroParquetReaders` for Avro Specific record -and then create a DataStream containing Parquet records as Avro Specific records. +你可以通过 `AvroParquetReaders` 为 Avro Specific 记录创建 `AvroParquetRecordFormat`, +然后创建一个包含由 Avro Specific records 格式构成的 Parquet records 的 DateStream。 ```java final FileSource source = @@ -245,12 +241,11 @@ final DataStream stream = ### Reflect record -Beyond Avro Generic and Specific record that requires a predefined Avro schema, -Flink also supports creating a DataStream from Parquet files based on existing Java POJO classes. -In this case, Avro will use Java reflection to generate schemas and protocols for these POJO classes. -Java types are mapped to Avro schemas, please refer to the [Avro reflect](https://avro.apache.org/docs/1.10.0/api/java/index.html) documentation for more details. +除了需要预定义 Avro Generic 和 Specific 记录, Flink 还支持基于现有 Java POJO 类从 Parquet 文件创建 DateStream。 +在这种场景中,Avro 会使用 Java 反射为这些 POJO 类生成 schema 和协议。 +请参考 [Avro reflect](https://avro.apache.org/docs/1.10.0/api/java/index.html) 文档获取更多关于 Java 类型到 Avro schemas 映射的详细信息。 -This example uses a simple Java POJO class [Datum](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/Datum.java): +本例使用了一个简单的 Java POJO 类 [Datum](https://github.com/apache/flink/blob/master/flink-formats/flink-parquet/src/test/java/org/apache/flink/formats/parquet/avro/Datum.java): ```java public class Datum implements Serializable { @@ -287,8 +282,8 @@ public class Datum implements Serializable { } ``` -You will create an `AvroParquetRecordFormat` via `AvroParquetReaders` for Avro Reflect record -and then create a DataStream containing Parquet records as Avro Reflect records. +你可以通过 `AvroParquetReaders` 为 Avro Reflect 记录创建一个 `AvroParquetRecordFormat`, +然后创建一个包含由 Avro Reflect records 格式构成的 Parquet records 的 DateStream。 ```java final FileSource source = @@ -303,14 +298,12 @@ final DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks(), "file-source"); ``` -#### Prerequisite for Parquet files +### 使用 Parquet files 必备条件 -In order to support reading Avro reflect records, the Parquet file must contain specific meta information. -The Avro schema used for creating the Parquet data must contain a `namespace`, -which will be used by the program to identify the concrete Java class for the reflection process. +为了支持读取 Avro Reflect 数据,Parquet 文件必须包含特定的 meta 信息。为了生成 Parquet 数据,Avro schema 信息中必须包含 namespace, +以便让程序在反射执行过程中能确定唯一的 Java Class 对象。 -The following example shows the `User` schema used previously. But this time it contains a namespace -pointing to the location(in this case the package), where the `User` class for the reflection could be found. +下面的案例展示了上文中的 User 对象的 schema 信息。但是当前案例包含了一个指定文件目录的 namespace(当前案例下的包路径),反射过程中可以找到对应的 User 类。 ```java // avro schema with namespace @@ -324,10 +317,9 @@ final String schema = + " {\"name\": \"favoriteColor\", \"type\": [\"string\", \"null\"] }\n" + " ]\n" + " }"; - ``` -Parquet files created with this schema will contain meta information like: +由上述 scheme 信息创建的 Parquet 文件包含以下 meta 信息: ```text creator: parquet-mr version 1.12.2 (build 77e30c8093386ec52c3cfa6c34b7ef3321322c94) @@ -349,45 +341,44 @@ favoriteColor: BINARY UNCOMPRESSED DO:0 FPO:92 SZ:55/55/1.00 VC:3 ENC:RLE,PLAI ``` -With the `User` class defined in the package org.apache.flink.formats.parquet.avro: +使用包 `org.apache.flink.formats.parquet.avro` 路径下已定义的 User 类: ```java public class User { - private String name; - private Integer favoriteNumber; - private String favoriteColor; + private String name; + private Integer favoriteNumber; + private String favoriteColor; - public User() {} + public User() {} - public User(String name, Integer favoriteNumber, String favoriteColor) { - this.name = name; - this.favoriteNumber = favoriteNumber; - this.favoriteColor = favoriteColor; - } + public User(String name, Integer favoriteNumber, String favoriteColor) { + this.name = name; + this.favoriteNumber = favoriteNumber; + this.favoriteColor = favoriteColor; + } - public String getName() { - return name; - } + public String getName() { + return name; + } - public Integer getFavoriteNumber() { - return favoriteNumber; - } + public Integer getFavoriteNumber() { + return favoriteNumber; + } - public String getFavoriteColor() { - return favoriteColor; - } + public String getFavoriteColor() { + return favoriteColor; } +} ``` -you can write the following program to read Avro Reflect records of User type from parquet files: +你可以通过下面的程序读取类型为 User 的 Avro Reflect records: ```java final FileSource source = FileSource.forRecordStreamFormat( AvroParquetReaders.forReflectRecord(User.class), /* Flink Path */) .build(); - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(10L); From 948f06374d0c502437b56e25f6b3ec237fe7c720 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 21 Mar 2022 17:54:37 +0100 Subject: [PATCH 144/258] [FLINK-26712][table-planner] Metadata keys should not conflict with physical columns This reduces the likelihood for name collisions between metadata columns and physical columns. It might break some connector implementations that used SupportsReadable/WritingMetadata and name-based column arithmetics. We don't recommend using name-based column arithmetics but index-based ones. This closes #19236. --- .../file/table/FileSystemTableSource.java | 34 +- .../abilities/SupportsWritingMetadata.java | 5 +- .../abilities/SupportsReadingMetadata.java | 5 +- .../planner/connectors/DynamicSinkUtils.java | 9 +- .../connectors/DynamicSourceUtils.java | 13 +- .../PushProjectIntoTableSourceScanRule.java | 2 + .../factories/TestValuesTableFactory.java | 3 +- ...ushProjectIntoTableSourceScanRuleTest.java | 4 +- .../file/table/FileSystemTableSourceTest.xml | 4 +- .../plan/batch/sql/TableSourceTest.xml | 4 +- .../testWritingMetadata.out | 2 +- .../testReadingMetadata.out | 10 +- ...shWatermarkIntoTableSourceScanRuleTest.xml | 4 +- ...ushLocalAggIntoTableSourceScanRuleTest.xml | 444 +++++++++--------- .../plan/stream/sql/SourceWatermarkTest.xml | 4 +- .../planner/plan/stream/sql/TableScanTest.xml | 25 +- .../planner/plan/stream/sql/TableSinkTest.xml | 19 +- .../plan/stream/sql/TableSourceTest.xml | 4 +- .../plan/stream/sql/TableScanTest.scala | 19 + .../plan/stream/sql/TableSinkTest.scala | 29 ++ 20 files changed, 375 insertions(+), 268 deletions(-) diff --git a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java index 5fc8dd1e71827..448a39618ed0d 100644 --- a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java +++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java @@ -64,8 +64,11 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.Stream; +import static org.apache.flink.util.CollectionUtil.entry; + /** File system table source. */ @Internal public class FileSystemTableSource extends AbstractFileSystemTable @@ -109,20 +112,14 @@ public FileSystemTableSource( @Override public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { - // When this table has no partition, just return a empty source. + // When this table has no partition, just return an empty source. if (!partitionKeys.isEmpty() && getOrFetchPartitions().isEmpty()) { return InputFormatProvider.of(new CollectionInputFormat<>(new ArrayList<>(), null)); } // Resolve metadata and make sure to filter out metadata not in the producedDataType final List metadataKeys = - DataType.getFieldNames(producedDataType).stream() - .filter( - ((this.metadataKeys == null) - ? Collections.emptyList() - : this.metadataKeys) - ::contains) - .collect(Collectors.toList()); + this.metadataKeys == null ? Collections.emptyList() : this.metadataKeys; final List metadataToExtract = metadataKeys.stream().map(ReadableFileInfo::resolve).collect(Collectors.toList()); @@ -225,16 +222,27 @@ private BulkFormat wrapBulkFormat( List metadata, List partitionKeys) { if (!metadata.isEmpty() || !partitionKeys.isEmpty()) { + final List producedFieldNames = DataType.getFieldNames(producedDataType); + final Map metadataColumns = + IntStream.range(0, metadata.size()) + .mapToObj( + i -> { + // Access metadata columns from the back because the + // names are decided by the planner + final int columnPos = + producedFieldNames.size() - metadata.size() + i; + return entry( + producedFieldNames.get(columnPos), + metadata.get(i).getAccessor()); + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + bulkFormat = new FileInfoExtractorBulkFormat( bulkFormat, producedDataType, context.createTypeInformation(producedDataType), - metadata.stream() - .collect( - Collectors.toMap( - ReadableFileInfo::getKey, - ReadableFileInfo::getAccessor)), + metadataColumns, partitionKeys, defaultPartName); } diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java index dd344950c1dc5..858bc05ef6c5c 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java @@ -70,7 +70,7 @@ *

    {@code
      * // for t1 and t2
      * ROW < i INT, s STRING, d DOUBLE >                                              // physical input
    - * ROW < i INT, s STRING, d DOUBLE, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE > // final input
    + * ROW < i INT, s STRING, d DOUBLE, $metadata$timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE > // final input
      *
      * // for t3
      * ROW < i INT, s STRING, d DOUBLE >                                              // physical input
    @@ -115,7 +115,8 @@ public interface SupportsWritingMetadata {
          *
          * @param metadataKeys a subset of the keys returned by {@link #listWritableMetadata()}, ordered
          *     by the iteration order of returned map
    -     * @param consumedDataType the final input type of the sink
    +     * @param consumedDataType the final input type of the sink, it is intended to be only forwarded
    +     *     and the planner will decide on the field names to avoid collisions
          * @see EncodingFormat#applyWritableMetadata(List)
          */
         void applyWritableMetadata(List metadataKeys, DataType consumedDataType);
    diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java
    index dcf20353af4fb..ac462b668a992 100644
    --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java
    +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java
    @@ -78,7 +78,7 @@
      * 
    {@code
      * // for t1 and t2
      * ROW < i INT, s STRING, d DOUBLE >                                              // physical output
    - * ROW < i INT, s STRING, d DOUBLE, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE > // final output
    + * ROW < i INT, s STRING, d DOUBLE, $metadata$timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE > // final output
      * }
    */ @PublicEvolving @@ -129,7 +129,8 @@ public interface SupportsReadingMetadata { * * @param metadataKeys a subset of the keys returned by {@link #listReadableMetadata()}, ordered * by the iteration order of returned map - * @param producedDataType the final output type of the source + * @param producedDataType the final output type of the source, it is intended to be only + * forwarded and the planner will decide on the field names to avoid collisions * @see DecodingFormat#applyReadableMetadata(List) */ void applyReadableMetadata(List metadataKeys, DataType producedDataType); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java index dd5a362b417db..7a906ffa9bd26 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java @@ -84,6 +84,9 @@ @Internal public final class DynamicSinkUtils { + // Ensures that physical and metadata columns don't collide. + private static final String METADATA_COLUMN_PREFIX = "$metadata$"; + /** Converts an {@link TableResult#collect()} sink to a {@link RelNode}. */ public static RelNode convertCollectToRel( FlinkRelBuilder relBuilder, @@ -642,7 +645,11 @@ private static RowType createConsumedType(ResolvedSchema schema, DynamicTableSin final Stream metadataFields = createRequiredMetadataKeys(schema, sink).stream() - .map(k -> new RowField(k, metadataMap.get(k).getLogicalType())); + .map( + k -> + new RowField( + METADATA_COLUMN_PREFIX + k, + metadataMap.get(k).getLogicalType())); final List rowFields = Stream.concat(physicalFields, metadataFields).collect(Collectors.toList()); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java index e531da669bbaa..13dda6d39c672 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java @@ -71,6 +71,9 @@ @Internal public final class DynamicSourceUtils { + // Ensures that physical and metadata columns don't collide. + public static final String METADATA_COLUMN_PREFIX = "$metadata$"; + /** * Converts a given {@link DataStream} to a {@link RelNode}. It adds helper projections if * necessary. @@ -204,7 +207,11 @@ public static RowType createProducedType(ResolvedSchema schema, DynamicTableSour final Stream metadataFields = createRequiredMetadataKeys(schema, source).stream() - .map(k -> new RowField(k, metadataMap.get(k).getLogicalType())); + .map( + k -> + new RowField( + METADATA_COLUMN_PREFIX + k, + metadataMap.get(k).getLogicalType())); final List rowFields = Stream.concat(physicalFields, metadataFields).collect(Collectors.toList()); @@ -315,7 +322,9 @@ private static void pushMetadataProjection(FlinkRelBuilder relBuilder, ResolvedS .getMetadataKey() .orElse(metadataColumn.getName()); return rexBuilder.makeAbstractCast( - relDataType, relBuilder.field(metadataKey)); + relDataType, + relBuilder.field( + METADATA_COLUMN_PREFIX + metadataKey)); } else { return relBuilder.field(c.getName()); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java index a4044bf45b05d..58af37cbcbf50 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java @@ -60,6 +60,7 @@ import java.util.stream.IntStream; import java.util.stream.Stream; +import static org.apache.flink.table.planner.connectors.DynamicSourceUtils.METADATA_COLUMN_PREFIX; import static org.apache.flink.table.planner.connectors.DynamicSourceUtils.createProducedType; import static org.apache.flink.table.planner.connectors.DynamicSourceUtils.createRequiredMetadataKeys; import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapContext; @@ -308,6 +309,7 @@ private RowType performPushDown( final List projectedMetadataKeys = projectedMetadataColumns.stream() .map(NestedColumn::name) + .map(k -> k.substring(METADATA_COLUMN_PREFIX.length())) .collect(Collectors.toList()); abilitySpecs.add(new ReadingMetadataSpec(projectedMetadataKeys, newProducedType)); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java index db9585ea4bd4f..1c93fe10e022a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java @@ -402,8 +402,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { Collection data = registeredData.getOrDefault(dataId, Collections.emptyList()); List> partitions = parsePartitionList(helper.getOptions().get(PARTITION_LIST)); - DataType producedDataType = - context.getCatalogTable().getSchema().toPhysicalRowDataType(); + DataType producedDataType = context.getPhysicalRowDataType(); // pushing project into scan will prune schema and we have to get the mapping between // partition and row Map, Collection> partition2Rows; diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java index d98ac9414fb22..3f38bae8ab52a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java @@ -349,7 +349,7 @@ public void testProjectionIncludingOnlyMetadata() { equalTo(Collections.emptyList())); assertThat( DataType.getFieldNames(appliedMetadataDataType.get()), - equalTo(Collections.singletonList("m2"))); + equalTo(Collections.singletonList("$metadata$m2"))); } @Test @@ -375,7 +375,7 @@ public void testProjectionWithMetadataAndPhysicalFields() { equalTo(Collections.singletonList("f1"))); assertThat( DataType.getFieldNames(appliedMetadataDataType.get()), - equalTo(Arrays.asList("f1", "m2"))); + equalTo(Arrays.asList("f1", "$metadata$m2"))); } // --------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml index 6a2c8f88d9b5f..479eb8ad78795 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml @@ -50,8 +50,8 @@ LogicalSink(table=[default_catalog.default_database.MySink], fields=[a, b, filem diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml index 4c1344574125b..037458a3857f5 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml @@ -113,8 +113,8 @@ LogicalProject(id=[$0], nested1=[$1.nested1], results=[+(+($1.nested1.value, $1. diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out index 60411d8a3d952..53107af6c8869 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out @@ -69,7 +69,7 @@ "abilities" : [ { "type" : "WritingMetadata", "metadataKeys" : [ "m" ], - "consumedType" : "ROW<`a` BIGINT, `b` INT, `m` VARCHAR(2147483647)> NOT NULL" + "consumedType" : "ROW<`a` BIGINT, `b` INT, `$metadata$m` VARCHAR(2147483647)> NOT NULL" } ] }, "inputChangelogMode" : [ "INSERT" ], diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out index a8014cf440021..81bc950fda9fe 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out @@ -40,11 +40,11 @@ }, { "type" : "ReadingMetadata", "metadataKeys" : [ "m" ], - "producedType" : "ROW<`a` BIGINT, `b` INT, `m` VARCHAR(2147483647)> NOT NULL" + "producedType" : "ROW<`a` BIGINT, `b` INT, `$metadata$m` VARCHAR(2147483647)> NOT NULL" } ] }, - "outputType" : "ROW<`a` BIGINT, `b` INT, `m` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, b], metadata=[m]]], fields=[a, b, m])", + "outputType" : "ROW<`a` BIGINT, `b` INT, `$metadata$m` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, b], metadata=[m]]], fields=[a, b, $metadata$m])", "inputProperties" : [ ] }, { "id" : 2, @@ -88,8 +88,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` BIGINT, `b` INT, `m` VARCHAR(2147483647)>", - "description" : "Sink(table=[default_catalog.default_database.sink], fields=[a, b, m])" + "outputType" : "ROW<`a` BIGINT, `b` INT, `$metadata$m` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink], fields=[a, b, $metadata$m])" } ], "edges" : [ { "source" : 1, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml index deab0da75fd68..b0874d69e861c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml @@ -111,8 +111,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2], metadata=[$3], computed=[$4]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml index d97e11194e1e3..abaf03610ccc8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml @@ -16,13 +16,18 @@ See the License for the specific language governing permissions and limitations under the License. --> - + @@ -30,7 +35,9 @@ FROM inventory LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) +- LogicalProject(name=[$1], type=[$4], amount=[$2]) - +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) + +- LogicalSort(fetch=[100]) + +- LogicalProject(id=[$0], name=[$1], amount=[$2], price=[$3], type=[$4]) + +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) ]]> @@ -38,14 +45,19 @@ LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) Calc(select=[EXPR$0, name, type]) +- HashAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[name, type]]) - +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[name, type, amount], metadata=[], aggregates=[grouping=[name,type], aggFunctions=[LongSumAggFunction(amount)]]]], fields=[name, type, sum$0]) + +- LocalHashAggregate(groupBy=[name, type], select=[name, type, Partial_SUM(amount) AS sum$0]) + +- Calc(select=[name, type, amount]) + +- Limit(offset=[0], fetch=[100], global=[true]) + +- Exchange(distribution=[single]) + +- Limit(offset=[0], fetch=[100], global=[false]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory, limit=[100]]], fields=[id, name, amount, price, type]) ]]> - + @@ -62,109 +74,185 @@ LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) - + + + + + + + + + + + + + count(id) FILTER(WHERE id > 100), + name +FROM inventory + group by name]]> ($0, 100))]) + +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) ]]> (id, 100)) AS $f4]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[name, id, amount, price], metadata=[]]], fields=[name, id, amount, price]) ]]> - + + count(id), + name +FROM inventory + group by name]]> - + + group by name]]> - + + id, + amount, + sum(price) over (partition by name), + name +FROM inventory]]> (COUNT($3) OVER (PARTITION BY $1), 0), $SUM0($3) OVER (PARTITION BY $1), null:BIGINT)], name=[$1]) ++- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) ]]> (w0$o0, 0), w0$o1, null:BIGINT) AS EXPR$2, name]) ++- OverAggregate(partitionBy=[name], window#0=[COUNT(price) AS w0$o0, $SUM0(price) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[id, name, amount, price, w0$o0, w0$o1]) + +- Sort(orderBy=[name ASC]) + +- Exchange(distribution=[hash[name]]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[id, name, amount, price], metadata=[]]], fields=[id, name, amount, price]) +]]> + + + + + + + + + + + @@ -193,6 +281,28 @@ Calc(select=[EXPR$0, name, type]) +- HashAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[name, type]]) +- TableSourceScan(table=[[default_catalog, default_database, inventory, filter=[=(id, 123:BIGINT)], project=[name, type, amount], metadata=[], aggregates=[grouping=[name,type], aggFunctions=[LongSumAggFunction(amount)]]]], fields=[name, type, sum$0]) +]]> + + + + + + + + + + + @@ -222,7 +332,7 @@ LogicalProject(EXPR$0=[$2], EXPR$1=[$3], name=[$0], type=[$1]) Calc(select=[EXPR$0, EXPR$1, name, type]) +- HashAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0, Final_MAX(max$1) AS EXPR$1]) +- Exchange(distribution=[hash[name, type]]) - +- TableSourceScan(table=[[default_catalog, default_database, inventory_meta, filter=[=(id, 123:BIGINT)], project=[name, type, amount], metadata=[metadata_1], aggregates=[grouping=[name,type], aggFunctions=[LongSumAggFunction(amount),LongMaxAggFunction(metadata_1)]]]], fields=[name, type, sum$0, max$1]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory_meta, filter=[=(id, 123:BIGINT)], project=[name, type, amount], metadata=[metadata_1], aggregates=[grouping=[name,type], aggFunctions=[LongSumAggFunction(amount),LongMaxAggFunction($metadata$metadata_1)]]]], fields=[name, type, sum$0, max$1]) ]]> @@ -254,46 +364,39 @@ Calc(select=[EXPR$0, type, name]) ]]> - + + min(id), + max(amount), + sum(price), + avg(price), + count(id) +FROM inventory]]> - + @@ -301,9 +404,8 @@ FROM ( LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) +- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) +- LogicalProject(name=[$1], type=[$4], amount=[$2]) - +- LogicalSort(fetch=[100]) - +- LogicalProject(id=[$0], name=[$1], amount=[$2], price=[$3], type=[$4]) - +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) + +- LogicalFilter(condition=[=($0, 123)]) + +- LogicalTableScan(table=[[default_catalog, default_database, inventory_no_proj]]) ]]> @@ -311,188 +413,86 @@ LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) Calc(select=[EXPR$0, name, type]) +- HashAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0]) +- Exchange(distribution=[hash[name, type]]) - +- LocalHashAggregate(groupBy=[name, type], select=[name, type, Partial_SUM(amount) AS sum$0]) - +- Calc(select=[name, type, amount]) - +- Limit(offset=[0], fetch=[100], global=[true]) - +- Exchange(distribution=[single]) - +- Limit(offset=[0], fetch=[100], global=[false]) - +- TableSourceScan(table=[[default_catalog, default_database, inventory, limit=[100]]], fields=[id, name, amount, price, type]) -]]> - - - - - - - - - - - - - - - - - - - - - - - + + avg(price), + count(id) +FROM inventory]]> - + + group by name, type]]> - - - - - - - - (COUNT($3) OVER (PARTITION BY $1), 0), $SUM0($3) OVER (PARTITION BY $1), null:BIGINT)], name=[$1]) -+- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) -]]> - - - (w0$o0, 0), w0$o1, null:BIGINT) AS EXPR$2, name]) -+- OverAggregate(partitionBy=[name], window#0=[COUNT(price) AS w0$o0, $SUM0(price) AS w0$o1 RANG BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING], select=[id, name, amount, price, w0$o0, w0$o1]) - +- Sort(orderBy=[name ASC]) - +- Exchange(distribution=[hash[name]]) - +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[id, name, amount, price], metadata=[]]], fields=[id, name, amount, price]) +Calc(select=[EXPR$0, name, type]) ++- HashAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0]) + +- Exchange(distribution=[hash[name, type]]) + +- LocalHashAggregate(groupBy=[name, type], select=[name, type, Partial_SUM(amount) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[name, type, amount], metadata=[]]], fields=[name, type, amount]) ]]> - + 100), - name + sum(amount), + name, + type FROM inventory - group by name]]> + group by name, type]]> ($0, 100))]) +LogicalProject(EXPR$0=[$2], name=[$0], type=[$1]) ++- LogicalAggregate(group=[{0, 1}], EXPR$0=[SUM($2)]) + +- LogicalProject(name=[$1], type=[$4], amount=[$2]) +- LogicalTableScan(table=[[default_catalog, default_database, inventory]]) ]]> (id, 100)) AS $f4]) - +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[name, id, amount, price], metadata=[]]], fields=[name, id, amount, price]) +Calc(select=[EXPR$0, name, type]) ++- SortAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0]) + +- Sort(orderBy=[name ASC, type ASC]) + +- Exchange(distribution=[hash[name, type]]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory, project=[name, type, amount], metadata=[], aggregates=[grouping=[name,type], aggFunctions=[LongSumAggFunction(amount)]]]], fields=[name, type, sum$0]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml index b6fb8db11bd87..88d4eba4d3909 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml @@ -144,7 +144,7 @@ LogicalProject(a=[$0], b=[$1]) @@ -182,7 +182,7 @@ LogicalProject(a=[$0], b=[$1]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml index 8759c6acf50af..7c252f7ae3116 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml @@ -192,8 +192,8 @@ LogicalProject(a=[$0], other_metadata=[CAST($4):INTEGER], b=[$1], c=[$2], metada @@ -209,8 +209,25 @@ LogicalProject(b=[$1], other_metadata=[CAST($4):INTEGER]) + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml index 78d2822cd4a9d..41f48c1cc4e87 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml @@ -489,8 +489,23 @@ LogicalSink(table=[default_catalog.default_database.MetadataTable], fields=[a, b + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml index f8009ff219748..648a40ad94e62 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml @@ -79,8 +79,8 @@ LogicalProject(id=[$0], nested1=[$1.nested1], results=[+(+($1.nested1.value, $1. diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala index 63051cba01d30..c380c7ac9c50f 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala @@ -99,6 +99,25 @@ class TableScanTest extends TableTestBase { util.verifyExecPlan("SELECT * FROM MetadataTable") } + @Test + def testDDLWithMetadataThatConflictsWithPhysicalColumn(): Unit = { + util.addTable( + s""" + |CREATE TABLE MetadataTable ( + | `timestamp` TIMESTAMP(9), + | `metadata_timestamp` TIMESTAMP(0) METADATA FROM 'timestamp', + | `other` STRING METADATA, + | `computed_other` AS UPPER(`other`), + | `computed_timestamp` AS CAST(`metadata_timestamp` AS STRING) + |) WITH ( + | 'connector' = 'values', + | 'bounded' = 'false', + | 'readable-metadata' = 'timestamp:TIMESTAMP(0), other:STRING' + |) + """.stripMargin) + util.verifyExecPlan("SELECT * FROM MetadataTable") + } + @Test def testDDLWithMetadataColumnProjectionPushDown(): Unit = { // tests reordering, skipping, and casting of metadata diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala index e8bfaeef453e1..f568041691f96 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala @@ -429,6 +429,35 @@ class TableSinkTest extends TableTestBase { util.verifyRelPlan(stmtSet) } + @Test + def testMetadataColumnThatConflictsWithPhysicalColumn(): Unit = { + util.addTable( + s""" + |CREATE TABLE MetadataTable ( + | `metadata_1` DOUBLE, + | `m_1` STRING METADATA FROM 'metadata_1' VIRTUAL, + | `m_2` BIGINT METADATA FROM 'metadata_2', + | `metadata_2` DOUBLE, + | `other` STRING + |) WITH ( + | 'connector' = 'values', + | 'readable-metadata' = 'metadata_1:STRING, metadata_2:BIGINT', + | 'writable-metadata' = 'metadata_1:STRING, metadata_2:BIGINT' + |) + """.stripMargin) + + val sql = + """ + |INSERT INTO MetadataTable + |SELECT `metadata_1`, `m_2`, `metadata_2`, `other` + |FROM MetadataTable + |""".stripMargin + val stmtSet = util.tableEnv.createStatementSet() + stmtSet.addInsertSql(sql) + + util.verifyRelPlan(stmtSet) + } + @Test def testSinkDisorderChangeLogWithJoin(): Unit = { util.tableEnv.executeSql( From 171026917f1365bd69b458e755065400e02275c3 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Fri, 1 Apr 2022 09:49:27 +0200 Subject: [PATCH 145/258] [hotfix][runtime] Adds write method for consistency reasons --- .../apache/flink/core/fs/local/LocalDataOutputStream.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java index 1503b4bc2c9c8..14eaf70c15958 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java @@ -50,6 +50,11 @@ public void write(final int b) throws IOException { fos.write(b); } + @Override + public void write(@Nonnull final byte[] b) throws IOException { + fos.write(b); + } + @Override public void write(final byte[] b, final int off, final int len) throws IOException { fos.write(b, off, len); From 5fdaf79a31bb1eccdd3cac4b1b8b52e585e89b0e Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Fri, 1 Apr 2022 18:09:45 +0200 Subject: [PATCH 146/258] [hotfix] Fixes typo in NonClosingOutputStreamDecorator --- .../runtime/state/SnappyStreamCompressionDecorator.java | 4 ++-- .../flink/runtime/state/StreamCompressionDecorator.java | 6 +++--- .../state/UncompressedStreamCompressionDecorator.java | 4 ++-- ...mDecorator.java => NonClosingOutputStreamDecorator.java} | 4 ++-- 4 files changed, 9 insertions(+), 9 deletions(-) rename flink-runtime/src/main/java/org/apache/flink/runtime/util/{NonClosingOutpusStreamDecorator.java => NonClosingOutputStreamDecorator.java} (90%) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnappyStreamCompressionDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnappyStreamCompressionDecorator.java index 5b024149d1f85..ff3318b55cb58 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnappyStreamCompressionDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SnappyStreamCompressionDecorator.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.util.NonClosingInputStreamDecorator; -import org.apache.flink.runtime.util.NonClosingOutpusStreamDecorator; +import org.apache.flink.runtime.util.NonClosingOutputStreamDecorator; import org.xerial.snappy.SnappyFramedInputStream; import org.xerial.snappy.SnappyFramedOutputStream; @@ -42,7 +42,7 @@ public class SnappyStreamCompressionDecorator extends StreamCompressionDecorator private static final double MIN_COMPRESSION_RATIO = 0.85d; @Override - protected OutputStream decorateWithCompression(NonClosingOutpusStreamDecorator stream) + protected OutputStream decorateWithCompression(NonClosingOutputStreamDecorator stream) throws IOException { return new SnappyFramedOutputStream(stream, COMPRESSION_BLOCK_SIZE, MIN_COMPRESSION_RATIO); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamCompressionDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamCompressionDecorator.java index ad9e52c64e276..e32a51b97bbc4 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamCompressionDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StreamCompressionDecorator.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.util.NonClosingInputStreamDecorator; -import org.apache.flink.runtime.util.NonClosingOutpusStreamDecorator; +import org.apache.flink.runtime.util.NonClosingOutputStreamDecorator; import java.io.IOException; import java.io.InputStream; @@ -46,7 +46,7 @@ public abstract class StreamCompressionDecorator implements Serializable { * @return an output stream that is decorated by the compression scheme. */ public final OutputStream decorateWithCompression(OutputStream stream) throws IOException { - return decorateWithCompression(new NonClosingOutpusStreamDecorator(stream)); + return decorateWithCompression(new NonClosingOutputStreamDecorator(stream)); } /** @@ -64,7 +64,7 @@ public final InputStream decorateWithCompression(InputStream stream) throws IOEx * @param stream the stream to decorate * @return an output stream that is decorated by the compression scheme. */ - protected abstract OutputStream decorateWithCompression(NonClosingOutpusStreamDecorator stream) + protected abstract OutputStream decorateWithCompression(NonClosingOutputStreamDecorator stream) throws IOException; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/UncompressedStreamCompressionDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/UncompressedStreamCompressionDecorator.java index 05ac250ea38bd..7fa053e069bf2 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/UncompressedStreamCompressionDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/UncompressedStreamCompressionDecorator.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.runtime.util.NonClosingInputStreamDecorator; -import org.apache.flink.runtime.util.NonClosingOutpusStreamDecorator; +import org.apache.flink.runtime.util.NonClosingOutputStreamDecorator; import java.io.IOException; import java.io.InputStream; @@ -36,7 +36,7 @@ public class UncompressedStreamCompressionDecorator extends StreamCompressionDec private static final long serialVersionUID = 1L; @Override - protected OutputStream decorateWithCompression(NonClosingOutpusStreamDecorator stream) + protected OutputStream decorateWithCompression(NonClosingOutputStreamDecorator stream) throws IOException { return stream; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutpusStreamDecorator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutputStreamDecorator.java similarity index 90% rename from flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutpusStreamDecorator.java rename to flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutputStreamDecorator.java index dee7d7d005731..c40577375252a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutpusStreamDecorator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/NonClosingOutputStreamDecorator.java @@ -25,9 +25,9 @@ /** Decorator for input streams that ignores calls to {@link OutputStream#close()}. */ @Internal -public class NonClosingOutpusStreamDecorator extends ForwardingOutputStream { +public class NonClosingOutputStreamDecorator extends ForwardingOutputStream { - public NonClosingOutpusStreamDecorator(OutputStream delegate) { + public NonClosingOutputStreamDecorator(OutputStream delegate) { super(delegate); } From fafeb7f9534c684b76db14b5cbd26c44251c8647 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 31 Mar 2022 13:49:24 +0200 Subject: [PATCH 147/258] [FLINK-26957][runtime] Removes flush in FileSystemJobResultStore The writeValue calls close by default internally. Calling flush afterwards could cause errors. It's also not really necessary. OutputStream.flush does not guarantee persistence according to its JavaDoc. In contrast, calling close does guarantee it. --- .../highavailability/FileSystemJobResultStore.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java index 5f05180c05c75..010ce77e74cc7 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/FileSystemJobResultStore.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.rest.messages.json.JobResultDeserializer; import org.apache.flink.runtime.rest.messages.json.JobResultSerializer; +import org.apache.flink.runtime.util.NonClosingOutputStreamDecorator; import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; @@ -137,8 +138,11 @@ Path constructEntryPath(String fileName) { public void createDirtyResultInternal(JobResultEntry jobResultEntry) throws IOException { final Path path = constructDirtyPath(jobResultEntry.getJobId()); try (OutputStream os = fileSystem.create(path, FileSystem.WriteMode.NO_OVERWRITE)) { - mapper.writeValue(os, new JsonJobResultEntry(jobResultEntry)); - os.flush(); + mapper.writeValue( + // working around the internally used _writeAndClose method to ensure that close + // is only called once + new NonClosingOutputStreamDecorator(os), + new JsonJobResultEntry(jobResultEntry)); } } From cb0da8f2817bb51a01d168b70fdac99e7f34d94f Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Thu, 31 Mar 2022 14:14:30 +0200 Subject: [PATCH 148/258] [FLINK-26957][runtime] Adds invariant to LocalDataOutputStream to verify that no operation is allowed on a closed stream --- .../core/fs/local/LocalDataOutputStream.java | 18 +++++++++ .../core/fs/local/LocalFileSystemTest.java | 40 +++++++++++++++++++ 2 files changed, 58 insertions(+) diff --git a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java index 14eaf70c15958..a4da43416fdfc 100644 --- a/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java +++ b/flink-core/src/main/java/org/apache/flink/core/fs/local/LocalDataOutputStream.java @@ -21,9 +21,12 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.core.fs.FSDataOutputStream; +import javax.annotation.Nonnull; + import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.channels.ClosedChannelException; /** * The LocalDataOutputStream class is a wrapper class for a data output stream to the @@ -35,6 +38,8 @@ public class LocalDataOutputStream extends FSDataOutputStream { /** The file output stream used to write data. */ private final FileOutputStream fos; + private boolean closed = false; + /** * Constructs a new LocalDataOutputStream object from a given {@link File} object. * @@ -47,36 +52,49 @@ public LocalDataOutputStream(final File file) throws IOException { @Override public void write(final int b) throws IOException { + checkOpen(); fos.write(b); } @Override public void write(@Nonnull final byte[] b) throws IOException { + checkOpen(); fos.write(b); } @Override public void write(final byte[] b, final int off, final int len) throws IOException { + checkOpen(); fos.write(b, off, len); } @Override public void close() throws IOException { + closed = true; fos.close(); } @Override public void flush() throws IOException { + checkOpen(); fos.flush(); } @Override public void sync() throws IOException { + checkOpen(); fos.getFD().sync(); } @Override public long getPos() throws IOException { + checkOpen(); return fos.getChannel().position(); } + + private void checkOpen() throws IOException { + if (closed) { + throw new ClosedChannelException(); + } + } } diff --git a/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java b/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java index 2a7bcc6c5d539..51bc888440b55 100644 --- a/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java +++ b/flink-core/src/test/java/org/apache/flink/core/fs/local/LocalFileSystemTest.java @@ -28,6 +28,7 @@ import org.apache.flink.util.ExecutorUtils; import org.apache.flink.util.FileUtils; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.function.ThrowingConsumer; import org.apache.commons.lang3.RandomStringUtils; import org.junit.Assume; @@ -39,6 +40,7 @@ import java.io.FileInputStream; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.channels.ClosedChannelException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -389,6 +391,44 @@ public void testCreatingFileInCurrentDirectoryWithRelativePath() throws IOExcept } } + @Test(expected = ClosedChannelException.class) + public void testFlushMethodFailsOnClosedOutputStream() throws IOException { + testMethodCallFailureOnClosedStream(FSDataOutputStream::flush); + } + + @Test(expected = ClosedChannelException.class) + public void testWriteIntegerMethodFailsOnClosedOutputStream() throws IOException { + testMethodCallFailureOnClosedStream(os -> os.write(0)); + } + + @Test(expected = ClosedChannelException.class) + public void testWriteBytesMethodFailsOnClosedOutputStream() throws IOException { + testMethodCallFailureOnClosedStream(os -> os.write(new byte[0])); + } + + @Test(expected = ClosedChannelException.class) + public void testWriteBytesSubArrayMethodFailsOnClosedOutputStream() throws IOException { + testMethodCallFailureOnClosedStream(os -> os.write(new byte[0], 0, 0)); + } + + @Test(expected = ClosedChannelException.class) + public void testGetPosMethodFailsOnClosedOutputStream() throws IOException { + testMethodCallFailureOnClosedStream(FSDataOutputStream::getPos); + } + + private void testMethodCallFailureOnClosedStream( + ThrowingConsumer callback) throws IOException { + final FileSystem fs = FileSystem.getLocalFileSystem(); + final FSDataOutputStream outputStream = + fs.create( + new Path( + temporaryFolder.getRoot().toString(), + "close_fs_test_" + UUID.randomUUID()), + WriteMode.OVERWRITE); + outputStream.close(); + callback.accept(outputStream); + } + private Collection createTargetDirectories( File root, int directoryDepth, int numberDirectories) { final StringBuilder stringBuilder = new StringBuilder(); From 0cfee81048b1d56e18b1c7aa80f78bb7be92c2bd Mon Sep 17 00:00:00 2001 From: huangxingbo Date: Sat, 2 Apr 2022 15:55:55 +0800 Subject: [PATCH 149/258] [FLINK-27069][python] Fix the potential memory corruption in Thread Mode This closes #19368. --- flink-python/dev/dev-requirements.txt | 2 +- flink-python/pom.xml | 2 +- flink-python/setup.py | 2 +- flink-python/src/main/resources/META-INF/NOTICE | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-python/dev/dev-requirements.txt b/flink-python/dev/dev-requirements.txt index 00ce207053924..b061a67a20d28 100755 --- a/flink-python/dev/dev-requirements.txt +++ b/flink-python/dev/dev-requirements.txt @@ -27,4 +27,4 @@ numpy>=1.14.3,<1.20 fastavro>=0.21.4,<0.24 grpcio>=1.29.0,<2 grpcio-tools>=1.3.5,<=1.14.2 -pemja==0.1.3; python_version >= '3.7' +pemja==0.1.4; python_version >= '3.7' diff --git a/flink-python/pom.xml b/flink-python/pom.xml index b735df3ad7738..43b00cc6e1dd3 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -107,7 +107,7 @@ under the License. com.alibaba pemja - 0.1.3 + 0.1.4 diff --git a/flink-python/setup.py b/flink-python/setup.py index edad2f9e6bf61..36facf77f4bbf 100644 --- a/flink-python/setup.py +++ b/flink-python/setup.py @@ -313,7 +313,7 @@ def extracted_output_files(base_dir, file_path, output_directory): 'pandas>=1.0,<1.2.0', 'pyarrow>=0.15.1,<3.0.0', 'pytz>=2018.3', 'numpy>=1.14.3,<1.20', 'fastavro>=0.21.4,<0.24', 'requests>=2.26.0', 'protobuf<3.18', - 'pemja==0.1.3;python_full_version >= "3.7"', + 'pemja==0.1.4;python_full_version >= "3.7"', apache_flink_libraries_dependency], cmdclass={'build_ext': build_ext}, tests_require=['pytest==4.4.1'], diff --git a/flink-python/src/main/resources/META-INF/NOTICE b/flink-python/src/main/resources/META-INF/NOTICE index 877a483f5380d..182971f352073 100644 --- a/flink-python/src/main/resources/META-INF/NOTICE +++ b/flink-python/src/main/resources/META-INF/NOTICE @@ -27,7 +27,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.beam:beam-vendor-sdks-java-extensions-protobuf:2.27.0 - org.apache.beam:beam-vendor-guava-26_0-jre:0.1 - org.apache.beam:beam-vendor-grpc-1_26_0:0.3 -- com.alibaba:pemja:0.1.3 +- com.alibaba:pemja:0.1.4 This project bundles the following dependencies under the BSD license. See bundled license files for details From 6588ae7249f7c40ce362bff8d35d3c1220f02c37 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Wed, 2 Feb 2022 16:26:03 +0200 Subject: [PATCH 150/258] [FLINK-25897][docs] Update gradle quickstart quide to gradle 7.3.3 (cherry picked from commit 9b2d03a2a4e075d9e307644af36bbd888be7f4ed) --- .../docs/dev/configuration/overview.md | 27 ++++++++++++++----- 1 file changed, 21 insertions(+), 6 deletions(-) diff --git a/docs/content/docs/dev/configuration/overview.md b/docs/content/docs/dev/configuration/overview.md index 4ba7f83c36edd..0ac9dbadf3c03 100644 --- a/docs/content/docs/dev/configuration/overview.md +++ b/docs/content/docs/dev/configuration/overview.md @@ -76,7 +76,10 @@ $ curl https://flink.apache.org/q/quickstart.sh | bash -s {{< version >}} {{< /tab >}} {{< tab "Gradle" >}} -You can create a project with a Gradle build script or use the provided quickstart bash script. +You can create an empty project, where you are required to create the `src/main/java` and +`src/main/resources` directories manually and start writing some class(es) in that, with the use +of the following Gradle build script or instead use the provided quickstart bash script to get a +completely functional startup project. ### Gradle build script @@ -94,25 +97,31 @@ plugins { // artifact properties group = 'org.quickstart' version = '0.1-SNAPSHOT' -mainClassName = 'org.quickstart.StreamingJob' -mainClassName = 'org.quickstart.StreamingJob' +mainClassName = 'org.quickstart.DataStreamJob' description = """Flink Quickstart Job""" ext { javaVersion = '1.8' flinkVersion = '{{< version >}}' + scalaBinaryVersion = '{{< scala_version >}}' slf4jVersion = '1.7.32' log4jVersion = '2.17.1' } sourceCompatibility = javaVersion targetCompatibility = javaVersion tasks.withType(JavaCompile) { - options.encoding = 'UTF-8' + options.encoding = 'UTF-8' } applicationDefaultJvmArgs = ["-Dlog4j.configurationFile=log4j2.properties"] // declare where to find the dependencies of your project repositories { mavenCentral() + maven { + url "https://repository.apache.org/content/repositories/snapshots" + mavenContent { + snapshotsOnly() + } + } } // NOTE: We cannot use "compileOnly" or "shadow" configurations since then we could not run code // in the IDE or with "gradle run". We also cannot exclude transitive dependencies from the @@ -139,10 +148,9 @@ dependencies { // connectors. These must be in the flinkShadowJar configuration! // -------------------------------------------------------------- //flinkShadowJar "org.apache.flink:flink-connector-kafka:${flinkVersion}" + runtimeOnly "org.apache.logging.log4j:log4j-slf4j-impl:${log4jVersion}" runtimeOnly "org.apache.logging.log4j:log4j-api:${log4jVersion}" runtimeOnly "org.apache.logging.log4j:log4j-core:${log4jVersion}" - runtimeOnly "org.apache.logging.log4j:log4j-slf4j-impl:${log4jVersion}" - runtimeOnly "org.slf4j:slf4j-log4j12:${slf4jVersion}" // Add test dependencies here. // testCompile "junit:junit:4.12" } @@ -156,6 +164,13 @@ sourceSets { } run.classpath = sourceSets.main.runtimeClasspath +jar { + manifest { + attributes 'Built-By': System.getProperty('user.name'), + 'Build-Jdk': System.getProperty('java.version') + } +} + shadowJar { configurations = [project.configurations.flinkShadowJar] } From e0dd646429484e2172154be2868a9fd71e999f29 Mon Sep 17 00:00:00 2001 From: Yi Tang Date: Wed, 6 Apr 2022 15:37:20 +0800 Subject: [PATCH 151/258] [FLINK-25238][table-runtime] Fix ArrayDataSerializer#copy for customized types --- .../table/runtime/typeutils/ArrayDataSerializer.java | 4 +++- .../runtime/typeutils/ArrayDataSerializerTest.java | 12 ++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java index 078f01a46d3d3..26a6b0bdea1c1 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializer.java @@ -84,10 +84,12 @@ public ArrayData createInstance() { public ArrayData copy(ArrayData from) { if (from instanceof GenericArrayData) { return copyGenericArray((GenericArrayData) from); + } else if (from instanceof ColumnarArrayData) { + return copyColumnarArray((ColumnarArrayData) from); } else if (from instanceof BinaryArrayData) { return ((BinaryArrayData) from).copy(); } else { - return copyColumnarArray((ColumnarArrayData) from); + return toBinaryArray(from); } } diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java index 558bbbb0ca281..29fcbda37f29f 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/typeutils/ArrayDataSerializerTest.java @@ -29,6 +29,7 @@ import org.apache.flink.table.data.writer.BinaryArrayWriter; import org.apache.flink.testutils.DeeplyEqualsChecker; +import java.lang.reflect.Proxy; import java.nio.charset.StandardCharsets; /** A test for the {@link ArrayDataSerializer}. */ @@ -88,6 +89,7 @@ protected ArrayData[] getTestData() { createArray("11", "haa", "ke"), createArray("11", "lele", "haa", "ke"), createColumnarArray("11", "lele", "haa", "ke"), + createCustomTypeArray("11", "lele", "haa", "ke"), }; } @@ -108,4 +110,14 @@ private static ColumnarArrayData createColumnarArray(String... vs) { } return new ColumnarArrayData(vector, 0, vs.length); } + + static ArrayData createCustomTypeArray(String... vs) { + BinaryArrayData binaryArrayData = createArray(vs); + Object customArrayData = + Proxy.newProxyInstance( + ArrayDataSerializerTest.class.getClassLoader(), + new Class[] {ArrayData.class}, + (proxy, method, args) -> method.invoke(binaryArrayData, args)); + return (ArrayData) customArrayData; + } } From 1d7b2361392e1ce8659e3c9cef20468efbde3357 Mon Sep 17 00:00:00 2001 From: zhangchaoming Date: Thu, 24 Mar 2022 15:43:55 +0800 Subject: [PATCH 152/258] [FLINK-26835][serialization] Fix concurrent modification exception --- .../java/typeutils/runtime/PojoSerializer.java | 2 +- .../runtime/RuntimeSerializerFactory.java | 18 +++--------------- 2 files changed, 4 insertions(+), 16 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java index 30f48803fdaa0..28218a35dfc6c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java @@ -1040,7 +1040,7 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE } cl = Thread.currentThread().getContextClassLoader(); - subclassSerializerCache = new HashMap, TypeSerializer>(); + subclassSerializerCache = new HashMap<>(); } // -------------------------------------------------------------------------------------------- diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java index d9a84f65e50f1..9b8f541daa32d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RuntimeSerializerFactory.java @@ -36,8 +36,6 @@ public final class RuntimeSerializerFactory private TypeSerializer serializer; - private boolean firstSerializer = true; - private Class clazz; // Because we read the class from the TaskConfig and instantiate ourselves @@ -62,7 +60,6 @@ public void writeParametersToConfig(Configuration config) { } } - @SuppressWarnings("unchecked") @Override public void readParametersFromConfig(Configuration config, ClassLoader cl) throws ClassNotFoundException { @@ -71,12 +68,8 @@ public void readParametersFromConfig(Configuration config, ClassLoader cl) } try { - this.clazz = - (Class) InstantiationUtil.readObjectFromConfig(config, CONFIG_KEY_CLASS, cl); - this.serializer = - (TypeSerializer) - InstantiationUtil.readObjectFromConfig(config, CONFIG_KEY_SER, cl); - firstSerializer = true; + this.clazz = InstantiationUtil.readObjectFromConfig(config, CONFIG_KEY_CLASS, cl); + this.serializer = InstantiationUtil.readObjectFromConfig(config, CONFIG_KEY_SER, cl); } catch (ClassNotFoundException e) { throw e; } catch (Exception e) { @@ -87,12 +80,7 @@ public void readParametersFromConfig(Configuration config, ClassLoader cl) @Override public TypeSerializer getSerializer() { if (this.serializer != null) { - if (firstSerializer) { - firstSerializer = false; - return this.serializer; - } else { - return this.serializer.duplicate(); - } + return this.serializer.duplicate(); } else { throw new RuntimeException( "SerializerFactory has not been initialized from configuration."); From 2c89b6463f4cbecf9c3772377df82793319832ed Mon Sep 17 00:00:00 2001 From: Jing Zhang Date: Thu, 7 Apr 2022 15:16:44 +0800 Subject: [PATCH 153/258] [FLINK-27086][docs] Add a QA about how to handle exception when use hive parser in hive dialect document --- .../docs/connectors/table/hive/hive_dialect.md | 3 +++ .../docs/connectors/table/hive/hive_dialect.md | 2 ++ docs/static/fig/hive_parser_load_exception.png | Bin 0 -> 212874 bytes 3 files changed, 5 insertions(+) create mode 100644 docs/static/fig/hive_parser_load_exception.png diff --git a/docs/content.zh/docs/connectors/table/hive/hive_dialect.md b/docs/content.zh/docs/connectors/table/hive/hive_dialect.md index 3101f0db990aa..d591457ce70ec 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_dialect.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_dialect.md @@ -416,3 +416,6 @@ Flink SQL> select * from tbl cluster by key; -- run cluster by - 虽然所有 Hive 版本支持相同的语法,但是一些特定的功能是否可用仍取决于你使用的[Hive 版本]({{< ref "docs/connectors/table/hive/overview" >}}#支持的hive版本)。例如,更新数据库位置 只在 Hive-2.4.0 或更高版本支持。 - 执行 DML 和 DQL 时应该使用 [HiveModule]({{< ref "docs/connectors/table/hive/hive_functions" >}}#use-hive-built-in-functions-via-hivemodule) 。 +- 从 Flink 1.15版本开始,在使用 Hive 方言抛出以下异常时,请尝试用 opt 目录下的 flink-table-planner_2.12 jar 包来替换 lib 目录下的 flink-table-planner-loader jar 包。具体原因请参考 [FLINK-25128](https://issues.apache.org/jira/browse/FLINK-25128)。 + {{error}} + diff --git a/docs/content/docs/connectors/table/hive/hive_dialect.md b/docs/content/docs/connectors/table/hive/hive_dialect.md index 38c0845b2e624..f8d2e675cbbc5 100644 --- a/docs/content/docs/connectors/table/hive/hive_dialect.md +++ b/docs/content/docs/connectors/table/hive/hive_dialect.md @@ -430,3 +430,5 @@ The following are some precautions for using the Hive dialect. location is only supported in Hive-2.4.0 or later. - Use [HiveModule]({{< ref "docs/connectors/table/hive/hive_functions" >}}#use-hive-built-in-functions-via-hivemodule) to run DML and DQL. +- Since Flink 1.15 you need to swap flink-table-planner-loader located in /lib with flink-table-planner_2.12 located in /opt to avoid the following exception. Please see [FLINK-25128](https://issues.apache.org/jira/browse/FLINK-25128) for more details. + {{error}} diff --git a/docs/static/fig/hive_parser_load_exception.png b/docs/static/fig/hive_parser_load_exception.png new file mode 100644 index 0000000000000000000000000000000000000000..4d7fc3cf71ea1776d843ba2eef3023ff406a52bb GIT binary patch literal 212874 zcmdRW=|9wM8@JF<){-U5(1r*jB74#%q!r07B(j!un2;rFktoYll1sLbkYz|2yCVBC z7)-JZCNmhubGYu$^P>B?|AEJ=7UQ>^$9Wvz?ezHarSn|7g?BSCF>&c%ICGVWiKB#x ziMfOw0snGUnPn#vlLV9g8QtrC>2o7%Nv_V8WIBzKU*=WVlK;LfKVKkCU%0rQsYfRWt=?cf%wk;Wrv97p3`A02eU4y8j0We z@ovV})WIQ6jc3+>fhVtb@J8T!+rC0ua;NMYJD%jS2r29V-?>hThfvsQ;##lE1<%!ZYy|M}IVWvJ`(KR>=l?b_~=nE(Bu7gwKHp820& z9nV7N4*t(CuXp=y3;xe5JyptRU?inY2)b{j&AHm)A5K zhXcHy(=x|BqP*kBJ^pF=NE2E~$x>{5(n;DC`p~X8N-<)<^_(dFPe9F~1bchVDX$l3 zZR5NiW*e0>MP$Ojz(D1q?O+l$w5~RAik+Y@O$G)W92QU=t>_j|Dd;Vf6M&3Ty4L zQIS{`{kmCo?1^pNdE_lrKU>J&)z9^$hcAu3jFnEdx&{cbh;S=~`rRXUrip5`xz#

    Iacr{zNsk<+b=o8{s;({Y!&ptojAFu}gMude&Tjll zU`Ve7WYDTfTKi(os%LO#T#8Hx{*&eE?vDMvb}@S|8BNl-X>LAY{5bz?$-aj@MnsT{XC&#GJJTT2leoL3R2_v8CvVH$kB z0^QVbLa4*ovs%s5wJ!8lkCU3Xm=q;J)*bMvfSQg-oekUfWio!h?yGOqi*M`@5fO0; z&`JsVuZoUJ{~69xl2%o5F zeQc^^#Y0KgYqjUQCIl>c_!I)>_VRvlZMC=azEif!;ZS5reD5?EyF<_7ME}x}s)UBO z{6AR2a(AX4|1h^W^;TysJ>;c9=$H~piTz;Mi^D;K37WB7j!eNr;&M&M(8rv*4X5`E zd&MgFTeRvS(Q3Kq&b9`ojh{Ab5jO0>oJ))B{7Z8EZ}j8)lvlqK?Zd=a#4x4;KU2E- zS$8z1=C|Z}bCb0^(P}D0(P_5|9ed93(;-)L&y)_lwUD=utyfDK>2ogWy(*vb0*lT0 zap$s3a$#PmYix7v`ldF8Lg9>T(94r;9dW{7oK@0PIZJYac2+pi4a^@`U(NmQs?toN z{GkU;oRnk_JtU4dHz+fAu(b`z&OY)5_4dadR9budgLH~l79+7k!5Py}P@vLMG!zLV z?F-LW6gc%@OTx;0`*yaPCgjqQjkBgMN&VfDy{-zXH@O`Q3rY?_q|zc+*M>C5gxapf zzq&VOELu8p$b?ljqQYiLAid0wctG6#VA2NT*^4@x*#or;2Ns299}65v_miwYvfFV# z>I#!uXMVaxd8>x&LVD5cvlr}hE_KKJLw#?d+j;xsv|0J)!rxIzS(?0I6dO_0a;p2< zTkGJ1jw)Jjts`s}Sbp1VSu$~q$p|oiu>Cx4=N|S=Pe8PPf2wvr?m?=`VjnM$TM#l$ zsbAF$-O!-0^D-iWw~%ddt1Z>%dZMM9e8K&fty8IKX)HEQPLVVBTh`atvv*gGTnNBi z*{7(3J$5N&i4m*M99U@)?^S7-ef9Ow{MBOuWWNvHfeGGhqysK;;iq@F$k_@x>e4Ob zeT{b}O$Ch7XH~S!TwoDsXpCYb@7XcF-(LPr>2bK5kW!3$}@F&l#1dzH2 zA94;sJEwmv2XYK@NCpc&5&BZGBu7B4A7hwEQT`5m<&5kH4sHI z{RZl*>r?J^8A~2c{&60uxvDJ9AAmaCGpKd>Piko8`exd$`f09O?%G|R-L`iT`SpG_ z)wH)EHLm}3CdIy1qso*A<}3E!^$J=V6G$H5dP2L*EWq`wo~PchXxDVi@WZm_m6dAI z-wv(>h+X*dykY!FTwI(1|NW{xE^^5i!|jSMWc^^0CvRr+7ZjAS)h@7-L^p%6BO{i* zeSJsw?PjYLXWbEZE=)Q)>J7U~dr0(PC3J40FV7^tVT8ITmF)u z8yP)x?_9KB?C|)xHKcltozc_qslm92aaq%Y70be$if*whY$gx*$80_5%pX@Qs-lcX z`0?6n6PGAsruDZ{HRoTBE!WX z4=t9b^&9_u`&n1pepr`U&Q>9xT4&$O9NMWntTU0c>rDLX%hHRz<+aI#{BCv8Z&_K@%96>K_H!c8)Kcsbc4lPYT=6V(7<3=>7>oF; z0oUcRFrLir8!cyFGrT?_!ggR*OfhGn(>!>{&k`5;Y<@mGBoos8f={q(WYE8&%L0XqYqkXS=YzjAqzEHZ%VqLnrR%1s|Vpm6e zMwp1za=7uQ`WXt}E=OFGI6CiMPY#_I9+nH8YYe@KT0N3rt$f+weI@>m0vNlD`&CTemBxV8BF=G$L9PTtl;<}nxigG*N=iE3zQv^ka@ zyHHhId#R|eYTu1d2}9P`qc(55Rn(fc+^;gF}_WBzWu2c;ey4V$wie6jYM@QHog zZZ$wYw1`xt_>n!cI7I!T>wO;@B{&Cij!9|ns*v?S+p?bJNHCjlVDr|G8Y%P-jj!K) zjlZXkCWIZzZCv1TEK;w*^gi6dS8=#)L9@euqP{&Zi*1UDQQ$D>)t`NsiEQ=f(JX@E znUY@Qvgk%mZ)~6-Tk7KR3W043!sr2;Ex{P_sf~m>;qT{y3??5{(*muf4P_6en26%@ zi;7B%i|Ypmg`YgaWq7wau0N6Rl5+m9soqrc`zm6<6G2${s-f7n(B&1axGFR;(;n9D z)sbJ{#&RR^{cq&2!2X@($rMQaq$GB;C?meN+A=VPPOyZKpk~Xak=k~tD z-us5C3DD&E$`|Py@!_6gqs=)4mMwZrc8sL?&K)(&Xrk+$InxnM-5Hf$a5DZeG4TcbDLY4&PhFW#3 zb@Gckb`xbnrNnW`7lNwGZM6&RDa>sR0)6-U#hI7u(l~wEZLF3Dl@E9^Hwwin2I8dX zHR;G3mH#hos?<2z6g)b&$J)1zdGx$Cp}nN_fK43&!Rbe;55s1DIg$RgtS`uX-cy4! zm6yG4&NBMNk>o$dl+Up{r|V2LCC~4*O96LD1{7Z$Vb;4}NsC zxt(a=Qi*Eh1q>pTUp7$0)S*^Q`wApPn@(}gP>p`0*D0my8+2#WRd3EYN}HXVd+cL^ z1)Y(N!BhoFbLzz^KC(^Czz24^sx$;*?gsrihUN1tkB<~-S8&D|5%M$?ac@bZn9-k% z;EFADPLB@Rr2K9tigQ<}#B9}h&!2snJ#;TC%D~N&32P3RzyOD4nr1c}w#^;eihF@EGPCr)y#y?6y zxJ~5Pr~P#~PAC+r=6vD>!85|smE>A_6Nz#??*`zjtGQz}PQJb=04AoVC6k&T#l>+s zmhSgN=Vfq2woZ9XdZ{howt{zuGnbij*>hG^RT)b16HwDo~BFmn;mACq7 zPP6H(?nr8${1ngA7dgLhKbc&*nr^6xmFcuhRQbt0v8O0VH#&IxzX>FVjyQV%4> zV9@%F^qZc$PdiC4W0~gW=1?yT&8)4Rj=%U&TIwBbOO2sxr$egnj4WZ%JENhx))2SnLWi1PVGC5)5oM>gJ$ovBS@E@Shf zm3AJt^!%NK~{mQL> zBd9dZ;*fmqpM@Ai^eI-!!66Bci1`;L&qu_Ys)oWYKY4T)px)J7V71O==KFT<1bX`T zO_Wz9=c!5yNA(NRM3T5`H1J&3 zo+qrqZmO&I%Rko|Ex3wgZ@x1-=18^|-Y}Fso@An# z-mAISfx+E;QqyzS0iz=vCzOEajMM|RSUNCwI`V6w7*F`=10p;y&Q2X?bg4Nj*Qz7C zd}{u2Q^rkl_`j#nL+AWtl3Q}y6`r_yedu-ERLbZS5ER_i@I*R#ag?Ckq%F$*;!@54 zjr;)b+-V_yzRW!PYObhoybC-16?#RKDNB$bQUb@mv!ANElhmEiR z1rz?kuW1%PhV=o?Pm#(TSWsk<-0>at9hJIu=+U>g5q=eJkGIH2yE{0skv7^Ae(q64 zfx`*Hq0FluWF4BA5StIHBfbIqb(OmUgTTyAgh@>tGCnEJTchwrRU()vsxDs3HH?0U zP0Xe#_t7`wFGgBK4E1XjqMLO$!b*e}X0G>hSAqV|o3lz!4^$IQ%Ql|1k+i9uUCcvb zVt4;``@KXs@xdO&7o&NNwfWn$w4$P^im>T+w@etCiga`dlim6(<9nueJSZu2iGOdboK6G-;x$}|9 z7H0>gqy462t((7mVb-B}>zpqzX_bk6mmG7=In{NmEcV=?+NC)6?{4L5lTteRMh6}PI1-0_YZ zb_5;d=u*5d3hrrRVb%rP;}mmLaF1-8 zT3UAZ+)c)A4qzQTJ)ic~Qp+YOYx!E;7su&&cXKeA(0u4~r5}gxX|MlCD5H|Urx@=c5C6}In7er?x`wa!+JO!Hw^S)twM zcwy+I((Nm?PnQB`>}Hf;tPDt{D=RA^9ZsbA!4;q2VC^*PE-rMA8t$#`IJDv5nUvti z@Ule+tLK0rUD@YwYiE50EOuD@41;(7iHm%QSzph z3Zv`8Mj33SH_{U!Y{*wX?kv{QY7*Ag{9!l3MYdMOw+_L}l-eC7xO*oOVExN>N!+q5 z4md7pdj>(R=xj9TRL@ig^#`O;@teePWo-;>L!|%Y9{65zKws5A*JYSMr9fgfJV7wR zZf%f&Y~Zy&{Y|*i<#!Hr#_SXiO{1;;K0#0l?7?&Cvj?&M)bwHP=ZiSQ?4kCG`FP8csL|;yTr&h+$?JJzk_RQh-tc6=_4&^F zE*F(X-U!=X_lSlAun~Y7PX+@^1{45gH!*P#+5+fD0s;b2Z=`2AoE?5aiSv zumM5JBriEU+q-huh)!ZqE zS$Aap_|?BTj_T`ha<qW;P4J_euB+0ha=onn%A7T(a{O5M7fJ0=+9%v|o) zmT(+`BZg!0jYV(fjJk%ZYRU~^)yBoS(oRRpd@x(1b=k`dcZ^QtkT_;EFu&}Wy!_R? zo+zS_i=4N&H}pxqd%{KurbT_u%EEy9dV70I$|ldv`~6%|NojSMoxh@kK}tf$5D@sM zZ}ym+oP)7O3yA`MfWcif#4i2U96CtS)1`V(INC(m;Mg2D$7txWr5J86UWMVxEgN41 znsxki)aI#2E}?qny*_j`(8dveSVZ%6bfEB5@T2jx7C}pqTas_z{uU(x9=byTvQVzcCWO+eB zLH|EL&gYusRnJZi+=*)nKAI{I;1BBBVyhR6m zVp3#LAJ_@w0u~OIw)TlrcG(vINSCEW*)J0=%BK*y-2GTUt+d|ku?CIv07qfa>@C8Z zzS*TO913GKIVO2ma}$fyy{Y5Le$xa+&Wb{ zaC`gXJF{Q+D_d|ZJ}<8hKN;^o`Sd(D*Wm|MmY{wpLYE?}HmX9im2##IndJ4XZW6XO zk6r(~#kiQ=^?;_D*1kG}Wz5VmsB8$D>FWibD)jGbSi7L0&F3`1vwIXUYpqZU=*dIk zwlHI$t;Z@F=J0`}t5CgoTHuUw`v-L5KYP3N@} zcOHsUbiH>E>SJJZ6iUZxt;IC4bqYEuDF4Cqa!i^}RmP3rd7mn&-RQtVlR0fURZ*Vo zgWb)~q_F7YB0L?HuA0?r?e2iG7OCa9xL^!`&)ONklM3fwSTj}zoOs$4Vq#*-wQ#Ju zr=VY16;Y~Sv3}03un-T=W3zrlwYM8p)7m;alR8N^Z{9qR(h9TR_x#ZGakUt#p)68| z(x&utuxz#i2R%kU#opF-{3n@gLK(Q7*yf~AyLyngbTUu72BU4DSM2jPF%_3A{7kbT7#gATp(*s)`m^LqTY=mg`OZd-5fB%iAG!NI{0RN(s8G;$C* z@VF@OY40&27)-gzM>+=!=v?w7rqC(o{N1$AYYS7-Y(E< z*!6uiR%Ek2k+XY>g63UHbCC16!al|?7uZPvI_$-+{mq9LVhx0E)0&gd(`BxUFX1m0 zc&*M~D6(g+|6FeYc;vOMfEs$wKbBj>oW{^xVd?#7mW?-&{q|10g(5=2W;;1k)9atr zrzFkVk4hd`RztBYmZu#V%+nU{O>9{#+y&A-xjvwuNu};1*L_X5uayz~JE~c_S*{&V zy_dl8P%Y_tyEQA{lk0^%oZ|rxMFJ*X8!%6uZ@>TY?Y$s&?F%yRh#@Z`XLhyZRMl{? zxslhqTec3QO6PV3o17TvzFb7h1k`!)%Qrhkj>!=t{=x3CzTsgD0l(d$4`=RQ%;|nj zAeVmr+~R?<0ez7qN?b}sYbVe$Prqi;&b|G)s;brIyUs_su9k9;fTyOW4xhO$|6^pn zi%h$D>sHp#ydy~@?%Z`5eic>K7ino(Rz*%bZjwYn)t6xBc0C4s<_5OAkGwT%!2j*b z|IGq0KG?p#ToIrU8TBR#)Mst1PTvQ+01WI8=+TzG7>}~wp4wO`=;igbe*znYIT0kW z)x)oXp3zHt_ny5XaM0S^{0)rtwDyuJVI-KkPwFH-iN-Hq22P}|$o4(0tmPsMc zj^-#XxSv2G0*Qq~^IlUe$t*-$aOl8osIkHi-b#q1culT3qz>oMGMjIE-J%$L9^dXg z=dzyOnC1DxB&Y!|1a`vBEyJ+Bu=GbCf)1{LEiS!}y+XNkK^2H<3-?BQLoMr~svds&?x|;EE*Tlu* ziLSx$#_gS*XZk4y=8Dx;MWxl%ZGb4k?RJng=Y2G8iFYmOWL(Kv(wYl z@XG&HOf|ndao^gCFRdl(xXGtf@y^oQ-_EONTmp;@6Mf<&Sl~b0%1dc{O)93aylg#Ojc>pk(097xlHCo}1Y?j!a0n@mRAU-%5I4DtK($ zs!gZL$9wMX*5!?4ah`TZN)vIr3tY@Dp&j1iJv3Y3y$~@Q?aQc}KK)-NvUSSwCT$L} zV6}*cSv$M6gm~KOio}+j77nvHvi(wb6i+*1I}v0!U^8mFXq1o}rZd_-$unO1eJy{2 zJy`3)mr$O@{{C@B07A@Q!y!i8Z{t2Wrv+JcyLl6fFr~G<7cix!ikGIIMY+WL`GmeY zc-VFxz1OZBw`U?Pb0ERru!HZCU4KL~Uh^H-v~^va8jV+jNy$Uu^oLy=S7I%xt6L`` zv8oklCa$qC0Uag^7fOsB$=3H^6&V>hK$H1Nua=U3s326mgQ)eVp0@deW$CDs1VZQj zsx9X?iQH9ftm_37HQ#Pdzv6I##O6v7Mk~)xO-H9QkU+P3*F9r-=EJikc3sPUJ)u0A8cj?M#$mcC*~#u{Ko=OZxQ?!_t9uU) z&Mz+|VA9LVPGknJJD%2LnF`8~k!=%0YYiX#xyJf1(JXX?URut#ai&E1RI?v=$D?daNIEoT7i2|4EW>L8{e!! z&}#5F`Bg2YEcaiN3K{h0%JY;I|JGZh%pR@ZO#%v>ZSKLgqT#kS2Oa>A)!v*jF0HNY z*k)r6#O-$f-n1qf9G7jg`(F?p1!O9C+9^0f*0gXj*Es06S#m2a@{sXtjZU*4d+0+8 zdHlAcerS6Kzg=@EI?ixzQ)mu7>Z4Le_du{qjPK9@I9C1 zIYKl3olhz){e%hjC(uXnU0r*@d2^%nQVf9D<>q9n|JbmL1(d%CWW4KcncqOG)wyRe zLVws?U0oNkTNDgb#P^x|CGE_kbXPa7P>EZVENMFjhpEq=nDiC$hZU`$qF6;i#~|Du zdrkn!+g4ItVCSs$R=~u`=_S1QcJ(L(%@_bNeAj)nQ3&D-uqXt24WTvj(UoHY{)_c& z+gXNhf>yWHjTnk$n2pWyEh=H**J=zA_IDjH_Y>v?;Lt*1fkdgEl5@8OUu(PAw3j7I zL4Rm>vC_-yfqM$STXwO0D0{?>^7EQ;F)Y^Y<7$iRf0{XF{G^i&Af@teRr*u~wJw3B zzLr5)69dU^y9_S?nfJ!hMSBPB3(BFbNM-tk0$SLXDf8FYL56LX!cMj%%d!>T`g=lm zbY^pw(Cdh^rao9!kPshRTFu4B zZ4?lQ68_lq5!01Xmg30%G2`#L)odCDjb|he=YDkWP`Rv})()Z}C?M(H+Xxj9Tg%jP z$nJSyhW}x7C?SiM-Sdq?AW*;<&%3Rj)($eStD9R%aj|FjWl9uKQO-38-i!|ZKDicI71rURtszS% zrG$rvAL>*9ntO&~pgt;_yc_mAFiS~dZ$odd0EjQpC%4ruGdcR8{n;ze?%f^`->coR z?whx=IY?956fMVGx=83oagu-0(@cKVs&Lk935Q95hBKDlx^6Xiz7L*@6LC|=Fl=kY zEyb&{4gElS$N~m-;jOI>uZ?e>ZQxq~GCCPw+Rl?)xeZ>?+L*WPtj=47J$He~=9dAt zbyfOOhQDA9F#j2vk$*ikHc3t99{yGwo4t_=ayfkCao;VHDDYlNP zGP?6Or%kMQrt34!HI3mofM^)EWh2(b+r9Cro7;L|s3G_|owla5XYno%i^tYj@jVud zeN>g3f=x}|CuHhj{yHgw=hq=*>Er>X@=3*OF}p7Dswf^ zA-OeT-&?OUyF}y=upo)>$#RpN)D;0c4;@ET6_T#1c%KtJ zVmX5HkmLBoTsxO|DLsRCn5k{?3GarHf>5aTnV^RJ$s{S}w*75&b6R3!uBXFC0~?8o zfZ2?Lmy_V_Xj5Kv!nj(4e(VgL-fz2Mxyey7Y1P~LlFqG-&Qd_Dg0+h~0ebmQJeHC- z3isJ|K{5f*DoR#u{kZe2tlEB%^P~G zDzd8+i@)i9K5LIPNcYWCUT9hspXk5xs%<6pTSvgw5M`?r18SGn|&%aFEyXY%!v>YqJY@5XEg>{;|Kb9{Jh z+xtL2nL$kN_FQhOl*$!N_$HE2jEJp_W64}RPJM6ZQ)L+{Q6~hzYI|13;_XdLOr(=9 zf^aEfatAY$og#lDoC>5x;_}^sBb<8gd+wTC=~GMDsqa25O%r0(HPP)hZ*^6%cX5d+ zXqN*NDAzC!b}Q)ZHYgiqVbCQa%Y2Hy+*vz?n!S^JHb@}fh~*;a?~$VoXFc-h2C^Tp z(@^Q)ih&vT{P}Y;a6~!v0Cl9LrQKH-12HneAKiIH5clz5m3REo?Ka4706GDMr3Tyg z@=^}iwx37jlP~ThiGoQdy-#}I?wqTZG@pcGj3%qD#{*5*jQg60`F&-RTf%d>gG?wf zWzVIfLFxlF<(Oy%KuYMda4isYEPqw5N_VwTo%w$!T_i=OX@yU?xpitNVJ$Utba4H* zAf~r-%e>(EJ%QgeIwykVKOs*DFH9QosLo=cSohnP>?Q+YY_O|uxNyKVLNNm3 zPK}L?1**PJ(b`6RIKaXlFdH9?64UKZiX#6(VND8jI#(cWc;k{W-L%p|l2h+)+2GU6 z)WlkaGV{2*WuIvu6H*J>-(m>=Bv@)7fo}x{(wne4Qc3z$)`-eAwNQx*GR@-NHAbq2 z@ID6TeW>KvKLoK9TfY3ibo{282`|KeRNXb?Ib{6aIZu>`Mn6KS60Q46TmY!hYW=FPuNc?r?2$yW%b@Mj^ps)Lg_H;vR_eXbg%z| z3;e~@skZSbM#PTbj$run&iXtCet)S@6S zxl#1a1-3#q+i!N&M~prP*4;|{JtNhI_nY<_tmGyBgH`s1mz24 zO*}WVhZgpohU-$!}Ja_a0YOJ-_cPph$b28xm7BouO%< z2TnY&9j~gMc0klw!{B;;w}QC)~xFKInRl0CzdygoRvLGQ6-%m~v@u z=@n63o*#hPoY!971JC|2t;kmDgb?TIgdh&kIYil3PMuiTG-lp!+^9(zaWZo$TB_^} ziZ3WA5Q*r&)`u8~GYVaE_E@j>BSRmDEx0t9WW3F2y3 zYG?}1Co1sRbfH#$=m|R(MOWSYm@2yZ4zJ+P4)zm)xi?7xHvh-b*v4AN4zaWg#U3&HAjDEC=DQedj|#LtnJBaRgtgz@^32lnYNOf` zRn1YlI%)hV!l1C=do9RpP{6hHg)A_WCfuyUftn9v+Z~38{Hcu$GpBLeX!cq}3URgydTSJ@%Wql(uAD*|!J!`;umX1^TB=$Kl>50ip?Pk9UK5(A8PE7!B zhI_o4`z(Vcu~MS|87xI7-Te1-0eAucZk%p(CCJ7T^Ajn6H{AR(t17iqGGSXa{qHcBw1KMJM=UF5Q_KK81NRYZ_1srO&Pn94mp_hh)JOkg|9 zvhbBSkAh&ug&p348qlm~=O4hRfb_$6dks=att)CkE90kgkn6>~dG~wwAVK5VvFGd_ zkTAwQQsY`hR`$EdLCz9Xc0dFrMo+`)CGwVQF(W5kYvlIr9`LK^3Gg0X^4Pb#@V}xk zuAoFAsa)umJ>~Xob9aua616&x*v=c^$#nIzpPZHw)ZXPOStw|q_H^e_3VV^ggCi`4 zQ|0*iJWjXz>ztCz6x)R6hPfNSD@WpN8>HQ+WGKU(}RWc>gt|P)ORz(9Va=`-fBN z6_{|MUk)xdAie6s4!E)Jllyd!U0d^fxoq5!Bhz@&uxv zScBOI{+Ofm$sliJ>Fi<^0uvH}XY8KGP@OR6z{pX;u5rQ}IFIx{ocG-d%#|MSbXfQY zw-(5Zg)cYmVA~zH<3HfmEhZPC!$BC@KAZr0h|cmMolx)i?bZV$$SPnaJ3n-aN>$nV zW4!e&{3ka@T7+#A&*pcYA9u?Ddfr_91X?|~<*lGi9tt1+Hxhq)`~b;90Ih!aq-=r5 z@0y0dor4JnAsRvZ+*e#ua#}v+P`v$rh$Hk+&+4)Tv@vDP zI5O`;zrdeVLm7}vRBlT_40II@`M1>RkKEOS!x>~-O(f*J6O;?VR$N1Ci|}kG`4M3p zSDtR3qqG`td^Lt_kbh5})dQR0?H&%S5nP*PkWJzBfDPZYE_JwPl`U)sNHa8W2<(r) zNdG2BxMC~hwY5px_K4to#)UF-tskCX({HU8ZGnh9T}jHUD!jKqrB?qye;T!Ng4Wni z83-;feV#t8em`$*$~1V&v}A?)!5g7lx4i6CvUa#dOnuw`0Z{I*=0DNmadIqN&}s}o zT8jKv?&i-*LY)1>!;>-A8wWx4rCSa9xh4taL$fALly+%jynB-hYBp zjY)aN!f=?xR#{lA;!{##c^^ZKvmuXoz<1S;_b)B>X)Ad*qLzjAsl5W7PL?hnG>=w4+(N-3Bx9a5A9&*WYrHuYK+B0>PF+V|dBcZ;1!Q><0)#dBkz)zB>g+u$|&UJW~5Pd|S#WF*Jz0=m8|eb~;fEMDc!D zSM@vPLsBoTqrXjrP}AET2d=Tmd;eNDcNSXlkv((%%KdIBzTx@#N{qJB^l$WYgt@T_ z)<1rwo^tN|$!ve;t&PN5oO7x_NG87n0-MT2eS<4Mc~yF6{Cb8AJ`z_QC_xHJi2WrAlf|Hy3t$pQsBKCOduBEx0b@V6PBlcS@KS zM}54f*Zd0#Ox%$0@STuuMRdckca<9;pM>sfORWi1N#($eM*odUXR6Y5sey3=vXAA5~)y!TJ>5d+6tvx+{}CPU`cim5d%?Y_xEXnezSe9-;q0`t}K zF%cxigUKIN#9aOhRbGoxUO~gX zx8v*QLuc0YWJ;uRX)I|%9hZe)!5Ox3W5%R0kP%JBDiQpprFx!V6vxodyC+6rw;#{A z%PUD?vFvV>d_ z4JGt>#BFaOnOgE>_KJ}AiZhpKUMb-o%f|ie5aKqy-O-^2D#pOzpt{Mcg$imUs{7QW zrwZn6eLJ|@un3-zCV@Pv#rxuho}~{ZC5bn@09yMASkEV&PyLY&A&H<^{YE`e{J&VD zUM$Euo;bKR1;9@jp5b3V)Ff}|C7YF={QA~FRkyf7A-#=4@SZp~tVJtmlJQKEe!B15 zo}(9^41Jn=-Sb^plS?(f_&}CwL!7_j=DQGowL3OeZQ|d5WvX%+sv_bP%?|~R?U++> z(9E2|t3ip>nh{nSwep4i|rSeL{0W|ojQmx0kj)@UCy}l2ZI31-D zSIUO8x;FNn-2=Pda%j<9m8;Kz~3sc@ooQ-O|^WQ*dZE{Let+m+=WkiHEFlur?q~(b|cUmTcIw{jUeHt4@2Oj=xAxip`vmP%Lm5T`Oj9Gzt<%GhO<0u zI|qtBQ#wd<*Eh8Bf^$J-L%AC@&aapq8*!$VnyivslAluLRJm{H9DC8niAKzvyZ8jr zKb3J!0dZP_S!OH^+0EwTU|KDX$w=_^nr%FFr?E(*V-@nJ?4fV2<|;-&PG-5T7dSfjBH;04X!wHOYiMLNpX+!t z_j*ip4fNaWY|oQHAT*&kVNu@B)MWn&laTmOpNVNLiG{;Pda@e}+ye83k6yCPtEj{a z{!?q+<6CK}?bd)gCL<)Je>(9np1Wm=KRdx%cWYHuOvs)+YcX^4Qum)OcdI5+_(-}p z2nV1<6pzZfx9WFBQa%wjfK&jr3Y<=nV2_sR!e>~2S*#((lFkslY3Lw+e=@kQAg4|= z3rRg?E3X<^2-!i5tkmIo*?H->!X#y2&00v@c$6F{6W|mNx5z?D2^-ABoqz5S*`d&w z8W>M==q$?_Pj3(^bw9P9U(Q0O^#rRDEI(2P-Cn*sdv^V9S+qBp+^H{L!a81+NsdlC z7_0d6_X)rtfVLoLmUwvkG$Hn<*RvJstJ${Uk-lT+)x`hyEhg*VNmPk;I+yTsqj}82 zCKKwfORQ_zH1Zh>{iC}`tN$dou%%qF!b#AXQYjL4YWf^M)_OF1jfoz6z|y0l*%LLQ zbFY{WLAjS8GN(MeLo8A*u!+;mqZ^@gJatUS8N<Uh!~vzVoKei*`u?^^jE8B^ly z_MBioIc54sym+zS%72RgT~3Za&|5GOT;w2$BFh_PdDk6fzol=2FETTyiQ-!)p|u3mT&l4mC1E!X!%yM~UJL`l zpRs~t^ubRg@5%|>K4^EkjOFhC%>w)h(6SL_l=g0Ki*3KD%gQPd8l!XJ`JZ5~V@W1t3)x@-x7$+`9c{%QW=mMD(cJaaWQkz@BZ_La?@sMVY%qM4SA!aN{gh1`|4RuI)tOb zqdT4y6-fRduB^+uPqi!ERHT{)-M35G0o;yG(sJ}%Z$BTFY&rsR9HdMUeT9}pJ@Iv3 zK0ZG5pR7(raD3nOjvgP>TVV-Y&p4DI83jDJl-AL7 zqz?8H*iu3f5C__4ts%sDq5HmZ@{g18wCjJ`Wye9Y{G?L;i$)i`>ilP+Wz_%joNcl7 z)N-pERx!e>(&EuL<^2#zPDh8r6pR$A{-@aINkc~@&|Cuo{$GRWCgd&ulkT)Cvd$e_ zaq~a1UM*hqo-JF)*bcYWCT!5yW7etQ5uv zVa>!)L~*&v*BJ}Vk%dB47Rj13=OXmm^!OcBAI69+a9vOp^=X8&7=`;UHYPqF<=|T9 zY`Yl2r72&U`u_FC9%JX0#52$L`M%W*c2RL8&vTvq+YhUS%hOK?H}#@{L& zhx}%Pk{*@91dT38uuG@3YOepvg466n?zo@><88`DF(~O%x!2!Z5Ab`QQGL+uxdZWH zp+*nc_z=%S5`S=<;@NN0S=GLk7Ta-HJ7?#|f>wSZxw%KbppH?OUOgOj4t2P)?@Cdh z|MUwdGYgBu(FXw->1>Si0K%_fkPG%`Emg)4Ms&^}idGOd>% zj8&6Tl#TxVT@ej&Ji`*v2e;!DKu5O8SI7NZw4q34qoq+n-9Jf%Y$01k6#Fts$R+<_Qh6lj{s!f3j2U!cS1MJ#;XCK*` zv8MY*rY{#knB=hj>6h8hLihHX!%H1;qn(za29P~F_>>qmT{T?tq(H zrt=9h@v{u6L3g>%PCK)s`(~TM+AJlyY7L#yInX=EGW3*9m_#Ch9u2=MVECHux^1dR z{Oo%w>E=G8Q6;SB5@kSlxi8~W9IZ{k*<7@ol5U+YPGI^m?}P7)+8!T}O8qxHrQRnJ zRZY8n)5mevTAOFOFK8uT(~R!*J#aw^ODOzp;hjl&O{=YF_71Evs2O$e@=63`@#4h+ z{X{z?90G$jobLVAvkg3l*RyLktlTN~nMLKEqIt)w|E691@as)(ObzJ!kT9gXj|SU2 zw$?Tp5?BV#`xmfXs}{IWUtZ_tw;F_s;oS&Iz&@J$T;CqDcKUJG@d|~;w`u!tn=1Kv z?QIk`JXCi=bh3V#d-W1hM%=p?zr|}-j$-axJe+i-tBsi1)EGj=hT^P4^k$3s^=8zX zo@>f4qgZM4&+?}{Sn4c;3AI?658`JN58uAoxSb&0KF7kLP+bRGO;y_?>mM6oydsRt zqGs7L$D57wL*>mSvEQ=Kwj_Hx3~Bf-Y-4FadzA}S+tF0h>jFFtPaZ$swZ6&7MnraA zyy9Fzzy}&%-kqABuK9reFF2G*KHi}Q$A(qiVr3UhWca_JhRa_%LxL54l@N@aMc=Ij zf`gCGbMpe~C@7;3r>+e8{CsGdeRbS@vjF!Erfv*J3`{C$A0WSP8-EaB!P3pDuz6nE zb5cCAs={2=+lOzg@BG?2Ofh)w0DrzFpMZB_Jh=H>QA^866i<&3D=7PSQmW$JH+$W; zj~SK@RF~Lx#d!75m9T7=gjdn-+IeFoOK6|X^#@;2)gRX8+CZ^ldNCJ+88pga80S+!a zj@L&%9332-DC@Jbx3iltD+3qtY{$7_wgc){C7_d%lauKt*KqG=?+Ha{Qv2<}dx6cV zpSZX_3-QG`W<&Vi*tIgoh8!ANvzS|Q2;-^m z3^24qmDf83Z`Bs_Dl^YL`lhV;v+@7%_MTx)bzRq}6af(d5u^uDkSaxxBB6^4B1n@W zy-P3Bn-oDh0t(WVj`ZG(RHZ~jdI=C9N{bL6Kmwcd8efhuZ*jF<64~Mfu%O zkK)VYskjGu{$FBA+r^(~gBD!7DCMKkkrDGaYY=Y%0H~eee1cb!e4K+NXqafqP${q; zwzRESFZSbZ0g;oT&i!^%c0RbtV)p0tokIglSbpHB6j(rE-L4wShd?R{ye2?hTVG#q zD{x;^hXslE`P{k%;!v*}ZgIDVcH$8X818_LsJE9GpbwBmfivU+Sbtz`jc6?^j4=mG5mj3e@N1x&@l;id!Qjouw^HEHBX4lbaFw&n%;(>$_FYXpFbE=J{U`z5EQ(Y3_peeYwEJT=O@w0%7mm;@! zeIIuh3|i>)=e)5;$S?Gqz)tZng{zfh2YtQO@zNO&*MfWM9Nl#_A~nY0Eatuv-yOj>e3`(rBVasI%BRg*%_9m(8-mb zvCMaRJv+tsY3(%3;D_=<_Z~%=F57Igxi4_yChJIM)~fN2qs}ZuqlaX|_u4E^Pc0I!0BW#qZaZ)ezIkwzEO&Mdw0Mr&;aHi%!a|?3VlIMU47K>lHok{5wm|p~yh|cXaYr$oM=?raHZRhz(@E6|+>BMs4%kxx-^X{OqzDv4 zI2=+AF2P_O%R!-pw~$?MUTE=L`?A$Khs?|~63kJ-CUGua53Q*ZGxOPx`34q1!gYrE+m=fv#Gh7Q}VUiEjm)SaH9B>oK30xP= z`S|QJ9eU=k(+@yeJTF99qN1YsmI^44 z_x8s%#(_T&v~%&gH&6rtY0ze6XONks4Asd4&EC8OS->NJ`?UF8*gw6nujO4KFSr+Y zaZglK)V0cBJXCRUK}QB3MIsOLNl-E>NwgAwwt{ARv0p)b<#M*}0)X!-3YXQX1gyaz z^LsT{;<_Uv3p?Ltd3cRM&6mZ(77}npjW{1no~V6c zHomlA9OogwBI027*o-E3xXC@x-fYyY8gVoEPEb1mv%yw4*K^l=G=y115IyaYRxyBy zmwIsfD5*${jiuEv-+RfAGUwRMbumFH3-Ew|*ap&gpi`OE4hi)4&*(P$Bmc|{e!KG4 z2fgq%uWcq!7aSaTl7uN-Eps1TZWk(jV(0)g(aPr_Q-2ZW^k7s;0(q0NOd#Ew5!g3D zfdd590Q%e56i+#TWnd9&4iKqx<8vLykw#|X?rha1P*ek02hss31Rj=#Zh~Ei+#bB60EX<%opH@w(C5tsV@ee}0su%7;7{sNf=s7o_F^c^mR=1Mk@&KO_y z^YXfG25Wwg3I^7}7cWYR(||S$L}rf0-0=zegw9g*aG~dTHp?I8^j(3 zy)zWT1`HZF0K)(v1FM10d4E{!o%?HmD*(z{#D2Q7H-`PSvYra6y1$N1dW|tMRRXX8 zm6hQD^QXtdj&YZm!RAv_R|gJFFf;^w$+CJMO{={#I&Jh)`#A{qMp zy4vzP~IwIBBj9~ihY^FEColi zIJdsTLCYI|67=nuRE5HTnLfNv%2kfE)lnE;mG8jl?BkMqV_P~@NJ-d5SVMANpef4j zoVe4$Y_7{2*cqG%I8tPdLRB7_3Qw0tMtASa;HY;PC0{@+;n)V!=!oDKll6ht8($AM25mM*vs)+i0@5KNntTha&H0W%Y)ENWM~*PX zaTlp<7w!vGL*{^1C8?^Rq2E%lkcGGI-CcpC1!NS$PF; ztQ-R!;{Xu)bxdfZqf79<^xYU!D^Ml_@7QHEHMKwdoqLQclFOiYE`+1dfT+y>Womf^ z?STecxx%#5gt)jy08!q!_-VnJjn^(ik7*IKWC!HVL;#3{9{kh>*#}K^zATpyV(Td? zI0tCIz$l(Ye~^%ipM8ez_Ff}acp9(+h|-zr(h!sIg>8RCoP@rS;c5xD2E3y!bQgpM zIpB52M{S^I2XjsS;B||jyhFp}Ov5{YG9Pe+Hz|~W*ayTSxsyCO{768_clIP0=E2U znJV}ZzE4@T!wz16$NL}`WCx!e`#akKD!uLOWD@{Sc%#;)HDFQa(7W`etFu37086UF zKk~qn;DIh+07j)i2>~pcglq1{YN6q_cTEJp>7rn5iYIZ* z5}=B2Dw0E^JAjXVJm+jb+am}QaM;i60;X8#4BLX=6oHU|%=WMFF=mM93wS1&sv(oJ zZR>4!7=-BaH;qlj`K<~dQ=$2?a-HWmcl7!Qc2vmVJ)YP~?bt84u?p=JHZ4;!J^Dib zC@CC-ci@f%v6WZxxRq!t0lG&vIlFQzf66pCYuVKYE(@(Z&^RRTNyOtEJR0fi>+>J2 z>>2G0^FaYD0_cQ2VE2y%vQ*jQ5)G|#otO+T{t=8Xh@3Lv7JDDOBH<70|Kdth)6e(SxU0-4gYgot9um4yO)T5c0EjFi0tIo zaEdTPF*w=_4-B2`bV?BcgCV;yg9~iy%ouw;t&~lVNDgc9yT~K+ba%J-Dceb2DV!*i4f@R}?x9qBJIJ$JpoB<4c09AyoukUe4 zMOOzKa=yv^MJA{`0aFN=9+3Nx{R(GREWc(hrLWVe1JVAA^rS*wz^sFI;MnHD&T}qM zx`C1l$fFo6K?9b-vRGmx+`Jd0ps;Q;mKDY;jQax$UvY(@AX9we0n!sNyfNMqkPC0W z$oj4!K(+#+;K|ufTKlW3oFL5uyWq)b&{g5|^mM)khjK2^aq!R^>DYcgJJ(TAP!Kqm zs>dl)9O%YEKj7rusry+M3={&9L3!y86(=Du>j0AuK%}7S3NSWksGtHhXYPXePyFfc zLum@36>0aF;=mX^BpB&IOZTxym=iSG_I7|?6+m8q^FJlw465*46?l#1ungT|WRLmP z@6a;iRA1TEJzpW7cU6}dz?F!o107x514MT8?;_9lh_z2f6K7)5n=fVG0}Z|F==P;}Ae7SlJON~9q$JkyT>`L{2j>6HMc4EfAH#8 zrJjQ+MXwsEr}~X;nXp#%Scu%VS?|}dt+RG2msio<{=)Bm`1rR5O*wytn2xZwXuQ}b zeZ%Z~)NnJc>Zq{iwjWODT`u|I+J(+5`_O{5B}cwRwp3C%!g10atD~QY>qK@8Tj*D! zi8F4~-$Gys9|=tFm_ZGGnJu}Uzjos}{Y=uh_O+pAcN2r?^6u_qJcIkDi8PsFj$RN+ z5T=ocw$=taL*;9-M3(OeKZl^}DOxR-geVNAwx1uL-dHm)W&@*N0b0bHs{l*ti_G$d z9FcFeO5tUIKniba4j>oat_Rp(P${T_ROjwalK{U&<2ygPI}}#oOTCRB3j#I-{?#AH zf74DRq3AnQlEU#tR~vd%8pu5XCI44uo0bOtcM@RF*VmE#<~dp0BnlUv5>Iqs7f?Tb zzx+$q104ij;krGl&@rwc|u($VqwJFFQ z0Ed8&tn4`9!{2d~KfK3qsK~qiye#v;r%a4QGxBXz8>w#k)yG^ihi8LiN9T^o#z;kE_OG80}o$zgQ*O_ei0*&?d|Q&_K>35 z0oz)HfdXF)gAM~mXF+bzw<7qC4c=rJ3+uE$$cC{U;G_pZ2??aQw?H-Wci!;d(P@C( zaS6uGf&O00!cJ7em^GejM&rQ_kV)W+X8=+9V_i}jCcy|TJdsd5VUHhS{5_fAA5izc z;MhCL8hk59}dsEdq@xSWxf8Yl`Kd2W*;RA({EkxLEQYHT&hR%NVd z+1<{Js@oS@uufKbX68e0<0)hrF*EI^m-7)F6mtYN+zC$&t7B{Oygy;guq3rkmu3mV zNosehGS;(8_kE>9orK?Id0qHTk|u}^KCL+QR7$AM&UF%5q^=_@-VW%X`XFuieSO}_ z%!2HhfYnmjYdfQI7NTjlN6~Jza0ir~5y65ZqZ-WYy^PJo(iBSta;a)}rHi{OeIWm- zy}_fdyU>)=qu2&{OBOTCg%S=sqUR*7TW0Uv-UPZ1R%6UaNXB2A%_yQ$yj2|if0&T; zWVxQJzw#Ah0??EIQVO>Anv+xG8~}TO@;mTrFBWGc21R9vn-6d0jSqU7tiVbK+NzV2 zAc=tL`)%Rxk93^qjEk`%ppOpr8(69M)@jNQb?aX^2lUbhkBZ_%xP3ifPRxLZ0(8tR zb=QaSqKhk;9zcVeY|RKX?tcg9&5!Rui)S_uetET)RhH9NwwLZqb+j4&@!yUCK^REb z8t1H?oVL3TrI6s=!2W4N4G*hnX5BqcmyD0$W&rO77Hf3$9B}NJR6YcDzAt0mE--(X zofuz-Gw6Zm3=x@nldjeX+*C<2+QFfilKz zBYrpk)ezG=bnE|U0q(H@D!87{#+QVB8O_GA7HE@=itX zJl@#y%mIwpYDIdqGFk%3r9gIWd%`D z4GE+shjkM0qVZ87-(o1-B)t#_%0Ti27H#ixB`|qfB`NVQ0ibMsfH3jTnmX}*3xoh* z+`znvVE-;COR8ONeH%~>IxGJ>#M9RX2@i_*J6mw+L?7^UmaqrUg6Z?JV}5mWV9)vX z9Rt7C#s(jN&{9$c&?_5t8vmC?{%x%#z!@-sugT|xXTy*Yr>^@X*W|yG345^=qCapO zYwXg$^QeDnuTKWl$gcVm(nzmp2AV{+ur2e!ye z%$&)ELMxsp#A!?6r*>f8%q?N!yV8`Z+>jiFmaGNlFXGfcZ{HH0-t);A*htL^VxK-1 zMUf)El2GP^FlXil!$MhuQ_=};iCK+9__|1;9TeTfPYLxPdTBOuN-Mj}5K+Q$%6>ZR z9E4^tLYm_+uYrp}4@@Xx;bQmnz*m&O6UcExG3>BaA~M#yGTae*B@xB0Hpjpysm%~8~Wwh`f%~g zI*aPq3mx4DQqIhb0m!OAmnaDQ=wvw70qYT=<0epH0B8$XSrF;<9l(?uFa(26V|sAF zKZz6YP(Z32^x9#nVmAY%d%#Ku;Sg-&pk&lL-!R3?vh%%6e}W9hy$x$~_n;BP1icw4hvwTn_6^n#UJn`1%RHwS$zd%BW?Mh*`#Tv=hYyrA3P3F+AV96Gsp%#Q zH))j#npKfaKz1J{2e5fVHsQ*EO9Loqb=9i+Y>nU?p>y1c#SLN6lFOTD=s6`A(LHvC zEBt_aL3}*!rtTjg=+O>Zy#Uz&O!N&P3jElkq2XZ)P^sem@veT36FwFHN`($)lWsxsnVI5$bU@#v>&eQL1^O0j z1MSR8VFFKdG};sss3=Y=`1R`7?qE20Q2rcC^YqsOrhrm5%M5sWC!Pv8%il{lKz+)8p$aJQ=%d?IFTmvb(UijtY3qA`nWP?F6iE3x<>!+qaV?I zBqTZV*Zpv11JX(_zF7!Z5k+DUQ(xDaiyEJrejQtT3!A8;Kw~i`8zwkzJuHMmZeDJl74!>O)gT2uNpO+me*jtl zi|dvK*Z<`xzzd>QIS>)qTN&{`j9&SUA&{)=*9c_An=X+$TtC-?G(hV#)Bba84&)xV z)Kt@;4}e_(rRAtp2RaMoiX3J2YhP@J>-;^x<^OakBDKLgn_&@Os3hsU1qx|JV zVU|iX*md_6S=xm^oOdNCYxG!kUZoFEP)t9bcl#UQ-d_7VfEw`YAjtr7a8wF#_y3Vd z1Ck}qW2eSj_a7VP{!?(9mfsO#^8?h;ECB zNILvE8gdT#t27yWYv2HCx6P?ad{+x7V{Qjs)%6kG0D~)3DJO!T^V@;fTg5J|287c8Scn{fuq3Idq0U20PWWl_>Tlx z+1{Sqk^;nFfg!+;`k)SK$^Vf|-aa;q{b*8Y8QYBXCJO6d^dtX%V*9 zb>fPP?0ywZ8~tWb>TwdWB?VAf($cx>^6LHbY2LrTda48ta?O`f+w~u|MmDP!GAx#>f^s2Uv>FkkFPS~`PbvC6#w=3D*Ast zzN-1@|8JM1+p@(fNl6qwX%WX!cipnfOnu*E)E}BXE!7s;+g!U~;LF3<#iT?_%#dp} z5;c%JknEd)rnY5V^cN%lXVCu7Q(Zk>3L&qFoZS2P`vsHcASrixO4zGGreP6}-ckRC zv+<0Z2N9bDAVzPI~k&~S14a9~at zVHWUJKUV5R`8Fm$=sX%lBHT|)*B%!32$6SP#WJJ$dE>HXJg+%sRTeZeJBv~UYT}%9f zMan{K7_RComFs+gn49dzVkO5?gTfb)VFlOl(c;u=Kq=2F!&x zy{lJeOwW2hsJ=j+2#{6X>df{-6>4_YRunBRdrf*b)(ZNdR1%tQZFN9~8y!T5Cs$ML zB~ei^o82#M{U1KRvOTJup(+jM*0=Pk!0dD0y88C>(jdGOC0UuH@sAidxaOZ5x|OWi zE9ZO2=q$Dg6`x^@70!X-w?BWWnXfiR=+|+tb~}LcI{5LEc2dCTW1gFO}F?s5M4aVu8?pF z=4IqZp6wkr&3Q=AJ=SPy57X}tSv1sNd3`KZ$ZsuM-y|4z%dW{#$usmcMEuWdpub+zi4hY3x198?}{l4YOgOc~99vFhJzC zF&%Ar25$`d(z*Z%-W&IxR!@qVcd|E+(yK5<8n*Y-%i1~%5h&fmq@Twq{YZlfJB{wLYxD;!i2u$WFsLHZx5vFY;s#}pty{5iB8R>1V8BCZ^6WkVM5V9|NK;+HDDg>ND1shIeM{V#Xj0x14e4sH3eQ zS&sh+zJGt}^$^inHTDY|$ETI+r17R+3Ukw+B>KR79*r*6ZO+74jb1Z++;r^?4BiQO z9gsVgYs91nmo;eLcB}u!LpB|6>Fn)~x;PWZK~4zn%x2uZ1L1jW%$zZ9RLEtZ+g-z! z_NpyooG|++%bbeh2Fn5`muI|!A4#kc-`S$7Q25xh$E0bwfi>>!(^HB(oF79-uF%>_MRe1_|YMwaF}(b7C+aK zC1O$jkw(9?;MI4C(@P(@v)89h7RQT!{MT7Dd2?U(Vp`O?)jcP_y5FzYSs@y^CHn48 z!d>R!l!4&)j|tYaF35Y+iVoj-AZRJ7uFo1jEikaZl4g?RUIdqOKbRzacURaTKJ9jD z^P*jBk7vPB7c+ZXq$D%kN<>?5JTCbfT+M^&y3$Pw5X+junOxeu*Ytk@R&B1KOM7@HXT>#;{U?1OwXG{N&Mma!(6JF?oGcSnIpSAby0F($4_5hHV-4N zjqJw7)iZ@GT&gMJoy)tL+>op|wk!gTX+boO`s7PsNn zc1TpjV3f=j65g{YEA^a7XuunVL2=m(z1zj0q#r-(Nb(UqV}$W|qE4Z!6#r}K41lF0 z_h=o;-%e2N@iLhq_>r(UIy+9%hF|^Y@EhOIsjM4*$p@oKxSPky(4QqL#>pP1C8q=k z`Dh`VC$@K`D}!D?pEG7bP{R<~PbwssaFsa$QOC2RLQYg=V`^8w3qcCDSw3K1Mu=?o zTs8GW-`?3j2{=7$-5-Ama{wzSKzh-?Epk{|3uV9-`z!VVt~*0c%GKZda`{^I*GphM z9h;h*xafl%Q;`C0XuZxCCbHos#rgpRG{DD(94XV89QbBLbNC#+uF(X(5UB**f?s{d z_q=t^7X8Xe_WI7fcxUdXiQ56P=%T^HWaC(!qr#V00%6(mX`yea$ zEGkvMRQCf8l(7Ciw7c6*sQZ^K(^2t=oprPoy}UR1<$}k zOXD0?J2(oCI++QF9|Q)ReON(q6yc7;|B(WXZc`~IF8B_d*xi>?F<_>oEdR6BW@{<- zDI>p(lEg9A;tqGwhA!qiiH<4(8rYHQ zHx{P|M%(oag1m*Nw@#Fu*JZAm5BI7o4<*$P>wHVG@X#puhHw^wzgc)^@-4H7F+z=E zfNlcS*p6wCru-~7TkO$3wgkK?ish=eDTMnIr4o9^HLkV6OYCeZ;&^&@#9AN`AnKWk z3_s&zpqNu>E4qVS*yiVJ+GY`I{9KbUqnmdd{-8q$ms7}K3DmXi zT6}Kh&w||NO1D`cl)eZCr7EK37)TO#1H zCiOSWqcqpkDs_Jqy%;;E5=EW|A1*>4)vR`)uvY7>rHh|8c0Oc($o?&L{jwSTbe~|= z1%f}+TtC_r6^6{EwyLb2M=B*1BtCjxR6bN=$FHXH?5X(gDmsq0?|Ginym;=V|B3ZF zZ}caXtDJ8xy6q6g80-x1&p%{4Fqx1RJ$=|wTCr2nVdH1x=Y_5v8*dNP=@?EOK{R zwQw#QiP0~v`>F&HZ7+Nh>5pnwCx!iIQ-8Zcuiip%OtVmt3}!@NedC2*GgD!gE-9?~ z3Ji!K{YB{gF2!ixm#f=N6&DAhQxYc%UFJu=+kQ)rqi90qXMF^B11(^4p2lngVaTMk z%PL=9n+*R}j~q;Xl`GEVbQnV_BX~$d9C}TYnCyqsF8g%-XM`_j2MoOAnkLz_q&Q7< zTZQq1w-3!GkH)`OJcOMcd7xtu!jAp6%kf8B9Wk&&EA+99Maf(iv~jLy|5 zY==P7blW`Z=;@GKc9moSgxml65zmGq=fLKc9+fl-)+QC=zKzVq9=a{v>5rbd8`0)$ z7t2xU(2_lSfQfl9Za3n`?`AC84 z5R~AdGHiMR?)wxZoO$5syU8*~3VnXC4o}4)0-9Q`PKmn-i!H+fuXWZdz~@1;x*>IHTPW85ZBb3|qoEJbT0ie;IuWVQr*$IkK=%3rOM z?M6Q;@{tJQG>UTSN6Ps&H7fGIFGQC3qM=qrk&-KpDO{hGyUQJYR!j1)MAW<(+>Cf9 zR^UjvzO5^M1x2CdOcnX_WNGiitex|fNZKb#VG0*M?mtyb42N{248JDGt`eLn^@N}c zmMnz88OHgF?!xpn25T1)dxWorxX|XWXp+8bH$dxyeO1ZTJ*6|Z4ZDAUeI_42UhV+y z(}4g&!Nk@2X2H)Z^aV`=e`uSOtyT5O&^ zi~VF1`MIn|lrcRixjQu;dr-6btv~lP)1qK0eHwEq|AtPqJWq7t z@~CTX*rzn#Y3)nhznuplc8&-ISRvbo-Z1{(3W)9t)F4JOZ*z&Q+1(^al)pP?Nd0X4mlSX zI}&xdoxy4N<06&YzpM(4RPWb`r*7IWM%L=@KKl{dxi>S_EImxM4}fgJ-Fk6ScPchC zyU@``Q*0*PLF@yues*de-n0b5>(W<)e-%)9fPFot9do}tIbaKmi^p^(IE1AH2OLY3 zFpcEScFyMsy0jV0V~B#zV}x8wa$H;HJ|cL-sZ*Zxtkru~6fvH!Rkn|AFDD+<+-Lri zpUXyUWWweut8x@5)ZP`qP`BnAaE?xGdTWiteeY&^W)=wvICO0M zw1JHT2ecxqT9pG@jY_M$`BgU#-JnNGohQvza zZ&uiQpvVGEj<*!IImvS}^%v+~_Cax6m9whI5l6)@i~*K zxT1BIa5E^#`(GY&0u{HF+=YJml5)${5y!#pANn`%qZk#awKMXhoCp3Aqd{Xv!# z-Vbg%Q%24Y-*HZGFD?AgR89A!XX06|Y#GRFp2o@LqF5ukD(crD>`aZH_7O%_=49g9 z=#Tbof1)%QFTaYf{b@cMC)vwekW-YvA(xKZ+DO_eB zbhvRB{4?BR7fSDc2?jg3loA*hkKAP)LN@()?CT#Grn$gka4PF(|CT*!clJbuBMR#B z*q?IqLq=O%Z9Wzu$ysC?gI{4&-;&s{E2+&jB1`2Ho;9V5 zwUvyy_UWu}4@s6ttJz(&k2)Jz^$du|U$NLg!Gk1~0FuP1yXPDIPbw%#C`$Jp-QP+* z0f6e7Y*W=;2=O!ze-Mt}#oap5U5IU)naboBS6eV2WXm~*G{(jYDmiMxVSDy-m*pcANVK=c({gs>}SnA4u>20;BNQ%57hg11lOK z9#JDQ5A$g0&=|9`FW+4)9IwAm&8xg>U|aK0lY=!MT3rt1B}SVM1~Aa@-o47gNc3)U z)pI0k?2U9!reOwas_m64cf>bt~K$YlaEJdnm4MZ*ow>D;<`;b%X{+RM9fqq0^uBQ8}CBOMqWZC@1^@BE>s?0?OC=Z3^7La z{`AXVUFVUIUs#-TlPgbnY3%1d_QmIHcP^Gt)e(|*QAsrW;q0+5E>Py69Wkg06Dnn0 z`V`2XX@YSqZtjomCH z%)JDjhj8+$p0%r>A~BkIMg#ysO1`P3r4&H3ovW@FPB?CAR!mbXQ-Q<$R$kHmUY)&X} zKKj;9YQ&l!usbT_O!-hcb25R3i@4|>D;SV{lUx1!<$zJ{ORD4%x_!kJ$GCPPmCedxKWY9L78%z#0VQ`*NW9uCK#o%IgnSED= zu%PLC{6uY*U^cZKyG{~j4sC1pSCf`;mfd`L*#r%#jV2Q`^*7z22l zzd!y(Dw`we8C`)K+m+!fKi=42Vz&OOw0{3%BjEj4NnkGy~3 zZ5?T(iJ?(G8%e-wFVvhrw4`o*uf;Rw>{}R?;&b#w+pMk{=WDZyZElZ`3ACXPn3?hz z>rSno>XJfGs$_m26YMXl_jl!%7T4Ld^I+&|**s4vqc2#%Ztg)%^wWyyiQ9g@{~Vt0 zDCR;%lCS%f`Sa)4(sa^13!7%t+A;dYLMQj@se8pYd*2RY!BYg`&=Fb;!ZR+{Emyr~pVg#7z%*aW)U3y5$6II-_0bHeeWwkHmc8 zd`V2!yUyDf_!Id&8yz)N*Z<^O)$@EUO^3modx{sO zEwlc2`3iQF%s*Fa9m*(-_wI znRvgeoY1d}-S4l+bFg26CUtQ;vcmfW?-cYr3vTo7&^ukc>{I^8LzW^fgN8A*Jz-P% ztD&ZFsl&Gcb_>jG+{Uq&WpK$H*Dgpq{`&zr=yqZ*;=C3M{V6C9KZ*70?!>7ppjb3? zh!7P&ET~#(e9AkDp&{hDRA8$;ftAf`v5_nu!|ubX#~oLFGoxC4el%#7Uun$GEzXGg zZYDGwF9cz;B(RtJvs+*#OL0l2?{$#kL*LA8uifzb%wd(B+<9-Bh2=+?Of*zdELrdC z`kz}U7Q6{;O>tV~@?8Mm(?0LRWyO2hD;9F*f@DoUQ3f1gMa6Zu9PGuvMJs3~>C$;W zY=G02E5V%|Q8~6P#w!iMw>n_2AO6ZAU0V>RyOg`>f238zpxm471Ph`D(Se%pHSM!U zI?O$}Wsg3G)|FnhN_oQn2rj;uPrV=ZD!xcEoSB|OYXe%kHH*Xgp@F-h{arcck1t(k!Gygf=?A`)XbqxMZN)S@B z*_Ss@#6`@Ff80$sAuK6e%+@NtDZ9BT2?u5iZc1i-QQ0ZOjp+G4arKs;8fA&?;~!X5 zi|Za-Qh7EJYp9veeNlP5N;4&cR10zp>?xlDGhES}0_>foKv;T|6 zlkzKWOLLwb0xx1e?AJ>F7@hJRf6A7;>+=&m_Cm9A;3?}un1K;;~ywBcyw*< z#P*PE6T2TUdD-#s^TK-Vrz}`fWE#w-oGP`UjH*NC7bjPt)8~g^k0HSxV@45f?l0~_ z1@~7}g2B^c!*d65V%AUJ&9kiy+{*U+Zx~&26rZ;SeQs8clj~e>%~Bz^uTp%+4}zMw zmVNo$%f06x*7DD))+Xk@UAF3qq+f6Pxw*{0IqQrkkb+J}jVqk0Qg4kEFw(a zx*P}c5*x!U1^1h$QCI6KDrzf@tKeQo?@v5iGdEsh9ASB87GmBMw@;uP`q7r7+#>WR zBFH4lMIC>zkdUZ(L{?mi1)Ez}UP^Gw+_FZ%zV7=KN_y-wVa7TOdjHP8Uf#~{=?|DA zJbUa-{$Li5m_WfB-0YrJR?*lNR7}ez-9C8CxvtT6NY0%{7DD%QQB4%omvYDFr~J-W z9{QWu3jH}Y77BQ^#<}b-csu!Ma{Ep>-`R{be_1|nFea|D`dJEga)v>Kk=v4^SMte; zrjtLme0--wy%%-*{c`4SFQ46B=tr$&9ki+%>!quNrn9Ig(t8QPhjl4i*(n0tB10-) zOiw2@+*{@poH6At=7E&l z;cFoenW3XNY45MzAM&6alA^zJDqRcbmKEi0C4Os(CRfO%eG|Ev(<-vHv=Qu1&`J5w^Ld8u zmc-z5tYA$hCoUc-^y6}XUEJ2^!sQ(vWTYI1*+@flv8l7g(uv7dP=CYgxUzG5JTt~p z-P7qkVTh;YId^I$U|w-ym4SJ5{;&$Tx2YtoxoN0#YO>Os_+T!s8gZ;hyN8wCCj`6* zFc+$^_z)w4_|{gJ-ivtMo|2ytv$Gs@;J{s29Q}#9D0txKW#Y&?yI1K*;g#i3 zBxMJ+3t!sXn2qn0fX<;2&cv#_8zsu`Ta3?^9t*jbUUR#*v%;&uzSXj$QoUAXyAU1s zxSV^XFRo_gJP1n;h&qk>)0aXv}@EdQn z9+_MI>}S}CcX!TLI@=Noj>wpCU1w>u4h&u<0%TM>dsngr%@kWuuddzP7s};Qad5@o7w-`Od66BP-MWbJ_qLf%{va^GN z7KgAafaj2*db}-hc3--Zp>QStp+39_Sqn#hAvM@~*(64fNXQwm zgL9kJ8e!dP~sdLYCBW5{-lqi@P#OE~ENuV`8l5xxau*CIRVa4~KJ$%HC4L`#C!v!|*? zU61Lhf9L$oR0*#&=2ULOfz9}$hBvl#A!Vg1RKE-h<2cIsethfLIdn);r1pcnEwYI| z7}#p$?Z^vqM;$`_;#1pH$rQHydpk6~JC|x$cD1CjR<~~HCsOnFxzhkz$-N{8Faqus zwpl5j{3%RqJk($A4URsUe-P)HzIpIev1&T%&rnjF`)r98wQuw^MhbA;6TdHPJ1{BD zh)Ihd+KPDj$zFX8b`5{xMH1`2C?2b)@1JLGVXUVL1NP6BFDc~Hf@VQMh11md0wuj+ zQTMJBOXJ%Y*1=lShqESeMYq!r^EtJmYu!?7-MmZLR5@M6cao;A(EH20PZmV3qz!66 zKlyy2eqVx>=PD+fL}N7kxr+7&wu-PPXqo<1uHU(y*M zFIfB89eONmB}nlm&}Arq`y$9<0SxXRQ`JtEoyE#CR`c$CVWT5_15GfYcI8l!gvS7G zq277H>!7qQo6M)IYCeZWE1jQ6@Zg%&T((x1eYLJyeq8$mzT&l8_^R;zSHvS;@W5vo z0Ql5)XjiWLoWFFR)(r&lfTXI-UMHzgRf*!m#htZYh%e-neMyl9Q08hI^bpO{2E?V1 zN={pZU`*DVnD+2B$#q4ckDLCn+!=-rbEa`w8hKtLkjfBUqMnwO6ZS%pS#paEZs}n4 zgvb{UQO9;3P_-OO=`9}y+vu4gf_)WuCwGW=#KfoE9PD$}(Q49EKaj9#@q|UZGm)<&&^^o1f=*Qj|<*t^&xonB* z=MC(6J}+{67PpO_i7GjEOkbFcyp~1H)whnW8%qYpqNW3L|8@uljcqc~ux~>}(uTek zHWP`S^^u(2L%)T)cF>PAh&BiRw{{gve7l?Cp>1LXa(6CA8dd;~^IHORdiwPzJyUv9 zO)YQtM*Cl7^l;Qq^3-d8lU6-fLdh0%R(A(j)l8uGER_2j)EW zHQK;Dw^i5Q3deOPv<>>vGTv%+boY`60EwLms?W=y%+ zO6So}Zb|xq%&2l|?!;%k74@EzMnadFEWh`YH=K0PL5Wp&sB76Sho?zy#l2>@Y(AKC zHvLe9LDq9eZL^n~d&g&1*e#n%<~4QQ>8&MFlfbCW--(+B-wwnOHt(PGQnVhzp1D@U znYR~%7K=kqNZK%IEpvO2}S&ni3R<#UnN3l?yse~z?9`IYl^ zY|flWWBf&nLZ{sNPfMtv|A(&kj;H$l`mEQ_Ns``u}60H z$liOetdqSR92^|S!Ra`T`MdhwkNdCt^ZVBy$9bRkb-k|FbG)7x76}@Hi(Xd7Rpz@f zh=V7H#D+gA$KB;^!=R$W$RC;~SJ`;^`ln|DHRqnmBV%1%mcKM@!J=>fvFEc~DzP1K z{ggwv$(%h@M(be0$!ODlaibf9F;p}+7Uj2EV$FNc8h+fVV`9efrZFaOQ8844j+WX5 zZAh^nDs@PekF+9bej=3qY;4U1zdXOMC`B@+pK za8XL$64i!gH9*Y0P#S$`bg@(KpRxdQ9*=smH+@nx zgl}u*9OSOQUgUTi{*~_)u}tahT4BJ6JN~RI!4lpJ989gT0#s|83Ze+I>KR~qbj*Of z-k%L^?SpwONoMw(Au`VhteWm`K0wCr5{pr?3C}k@KniPyQN;qhNz5O%ZFS%X(za}m zn#yubD<@rDwoQT`XyCY5wwfmmvz z>e+m@SZg=wfd98Nte~?et%rw+C&aAtCk_&IcsEv3{6x7wjm1$Ep?I4l4N9fcoO2y4 z*JPwkALL_)a*E4?A>Hp}%O<;7E2sl{-@frvY?7w{IsA9qIo?R=Gk=qRfC*y9^yEA& z-NL>6z^X#e_i1@8yxe9#Z><+#+X>8^$=N!m?meiLgWm_zcaZ$$XPI-?{W0ndAbr~s zG$+LjYs6vksgA23a`e1||COhVBh0CK6z;rdH>SV~HK_zT$zWS6KgLJDc_KKIZ@p}4 z8)5RI_}ubeltjNTJy0`1R}vrpQL@}!aOrw=@_5^Hsr4pNnc4iDqgY*}jJ(QNA-g*!&l_TI8YEiOL0MYq8S?uQ(1Z0ACbaNX^GSpy-CvpR-o?eD3`Zl@z? zaVCy#!Ob&rCO`kxY`~s_djJ!N6?r%FeaA=V?pkQ)|B)NbkxZV&)F39uLeyzDT&vkXt+b=s^!t zBe{C4K?Ue6r7k?X8GrZ>C-N6UeHFrw=%$5q2)VtD;%07t*7(~UJ&O}vLahLmc0cCY zD6XX1CzmFJVzSiPy_Jh7eqBCgmAKQ*P4b=Z6&WRDg}&X0M!m|E^>hSPPN|};6sevg z`Iq2Qc!A@Rl+*1{fRLEh#gGezqWu=UN0NXTy&D@G=Ws6Cb>g6)^c%pxImaa6@X%Py zVoe$oS%E~}vxj|?aN6|p(#N5d#^}1`)vI>yF5cfb*3hqPj7F_8@rxC@Ra*5A zVLLz76oan8cf|&ctS_6@eupS3#qFxEb-K*CYD*NZSBf(pi$-5N`RsmEdTASUrbLsX z;=%d$f+E$d-A$YGrZ0+wv~tvCqebd69fLWP6$L}gv?^2XlGZiCnBJegDHpAxMR)BU zh=IF_rP?&BSwZ8NPP^zzB7-$)rr ze72m9QK?iZ;i+K5x-49m;3DZb;+!fAqw;kov)Dah-CZs1dXZ-rOqawn|5Ei!4^GLC ze6DUK0PD;5h)!}7KM|Mho+EPF6BN5){FI_#kQ_US@m&@a@9Fl%LMu;CY}*{pgi>B`Z!%Q{I%Yx> zEk~>%>CnM^v(lQ-7WQ!|LZV6wsRajuVMFHJ8aq&P6_jK46Ci0#o!nmfG1-8P4{5mr>e2W(KtW*Ta=PgBG%Hp4Y!7RUe_Sn1Wex} z=+?bje#3>?m3dR=_iF*j`Fn?i#y&Bif9`dBRmd(&1Z)q;`aKswDva^^CHv-#y_Zk| z)Xekeyqx^f$2&u5-(_>-KT1TQn;M2o?R$8a8+xZ?26fQVqOnLxTl;W#ai)#jz*_{H zS#sXjzDY*qH4J6qj}-uS59mRn1X&XP-=fYu42Q6Qi=oB)5M7?|BMLG=pTaH{<$#9; zhRGs-cK2p#*FCqE++8@5>Y8j8;7+cY_R=!Q{gytP7ij=ewlv!CgYFE0a^)oIu=^jB z?bH64mCHgEi|1NNRm_ zKds-QyroBDhNP_Yab?SksolXl z8RRyOcRZZg|0_>WJoCvE zl)r?b4Y%g~l${tB5D*rp{;H5>98CW)3qpVGIG z291QS@+jpAbvK!}mSi+2HaPYNak95g^BdPXwizQ!#87XakMfyS*iAXE`8tq-eb%Zz zDsL1cEaYRmE&(~kfh&MF59gnIhP=G!Lz=9}#)1o|{l5YU*JS<(bsS!894dW&phNeL zw%zOep$PD8T-KHYpN8(xkP(97>PHEse$AfGl2@Li;>)~`z4&-ks*2f&TkapM{QT=F zS{2p3Sb3~gbB@r<&yi^J-X;Yd-uWLVxm|6(`b3?@fsiw+_P;zp)1Nl5g1PYcqRhTl z5{cPX3qy87ltoDM*Ot!vm^+S35&$OmKnq-yynQFYrLwotJA6id$Yoe=0^%LrgaU?2*+5f-jmTj}p4MT$S?{U1x{`!v1Ye2u=v0Fh)cN1?X2 z3kRyJhwN+L3e-j6N`KoitzgLIemAcxNC_-?$o+r40Ituf=zRe#R5NBzLo2tmf7JJ> zr0kzEKA?HF!0g3*{>QkQ_UDOx4cw_IAaHjo3LyCTm$tH|e@ZsQ0^wL*$-}e8yRvlG zIJ{STBq{D8tPleNUmV^{kgl##=mB#RKZ*1jG% z9r{Bf&X>`9Vn5ly6X?$Pwurl7lGPv15boXO%Pk`bkgGO&Vj>>@C7VZHUkTnNd_mOg zO@AtN*N5M2WACY-Y5Jy*%nNO;5{PR^_Cu$X>&?lJ%+j;u-66In(573b088mcTByFF zJlFpF)@H<3YHixN2lzyqbNoUJ9pmW9%0juJH5;_zL)5w=x*R2TsG1-FUBXdiyGJ>m zbAzyraD&g5rL|MgI$j<+@hUW`1MjtfSqgQxaJaux*QjgjfPOiqOSqFWAjI;Oh1d_M zU%(-znLheT(Mq3iI(04@c!&~Y2G1o5##8+Z;oQJ?0MdEmY41aGkP%@5*3$NX6Xi|` zDVG!tSQDxonn{%~WZ+>HrIbtC`v^Ls$G!JcIj+h~S%V$)t(^2(8QD#6tW(Qef5E%& z9gX;PX#Nh+l}a}kKekcD$h!?fQYY9 z&28;aV6!|@b1DwhhHP#+4-B$=;20tMBTZ`r%iPJtCSK^-b(h*thD)4H-!Pm%r4Kn) z*1w9FjTxh>sD}xPvP>*5+B{NVx8$s=uf!E38C}VNk4TShkgA=bqV%`mvDa9LkLN zMJBQZkH0AWuaci?Cs7)9-9@UQ>P!FVqs$)mHnsJl=>%=Jz<4(+>zQxO50HlqIRRbM zPTX~6skdTipWM$6*kIk^thAdqqySsQHGhHC`u;kk^2>;LH>F4wtIonRG^(mjiIr%+ zc2S4w_PaMPORs@zQbk8TXhzCH=?M> zwe=UgoDIAQ!w6wdh|I`~dr|sqJ^A9hqVmnjAHavseLofOzF$J*wrB~jH6IRuL*rsa z-=aFj{}9pz=DQc(kKZT*f73qp7T&9xM5*?Km=4&D{48k@9?(_NEF;y>#-ypY&T)pg zJIpjbT6Fu?3JQ>!5!5g#xWZw94k)!X#kxuY$R@*?zxQB$H&C-qHTTtOE9n)1-skJB zx^`-C5-({DLVQb)%1yl4F^B-Se&FD%?5JlZ5WYj-AEYsk5rKDdPMZ{;6!lrDr8&eM z>8APbBRqZI=GhA3+>kwOGnmuJ3wp=COdG9AW15oOxftMmu$4*`L`-#-}!_TWItp|o#H|4HP20`k(; z{owbFnay3D+7!7JfTS+lEz47R_eU7p`?sgXhlU}n;+fbeZr6!$y6Gj?asY5pD~(8 zezxG>eNb74U==f#ny!wikzMAS-T681cqT=cP(u3wnR~Le;V#kZnPIZ2!Ml2n@o();%? zJn(1f%>qmK$Q##wxQ5e~^6i?*dApf)FkcL)818cVIh&1!dwk8EPDy}#2N0Ruf75Mo z0DzJkSf*`gf0w{-%_$=$I#yCWy~8M&PwKCr_)BbQz21}Xuoc&LGbML9+C7kD^`S2l z^~cI7w7=KV7_0`zx~g3I1&m9VcDt*K0K9V6!{FrX9-?^N5(jmX^z0*%`P{QGFO80L zy-1x2yskDM^c)_*#~cF8gC_6A!tgUsA5))B`#<|-7jQIBGBXk1ez|E-l2l!-`TYjh zKU0XD#jaUMfgBWvU*c`cjA&)z%>GH+P8Z_d3#5_N9hMCN5hs_xc+J

    A+vUd9F2{ zy?*4ZSV+U!MZs}HgJi7(E}oRsIL-iMGOmRq>+fblfMAPfJ_k;*S(W1m*x!5qaEP*B z(Q!u?qt)8e%inh8KOw928IpDgS!se27QfQMW9GKMTA*0BwAQI=YYBYSXzf72fK>AD zK-sdN=9M%(uPQou!uT`b{y(`a0|8!SLT}t-4>rFdzMJ9lZdN}qGt#&lpkDh~I0UI1 zp}C8BQbL?cN^8lQQm?IXK6!8QB>Z|5Va1p7OuPPWwKlx_bi`71?bdW@;M+Y$aIi(T;ao-p}P$|aV`ewG~7K3L=!%0By1Jt}@}LZtw-RufjL|6Hv_<^Dm)x#-MR zAIQg4I#p#EY0_=!d@tvm$J<}EbFIG1usysc%S{j(*Dvi$L=c0Z=H_`O*jG~-lvQYV zSZqEixZX6Ul}qGdL^p+vL__90&wydi+bp>OQcw2$;vsVE`vK=?YCs&G&GH!TaHOGo z(cr>?>%V!)3*^@|*}gjo*+@IT)&U9BFyV%^_!mnIio4&oy*( z~m0lb=_I6kfekaZz z`B@3YoEk>GEmC@zS;E*ZeIN%kTi(TmbG*O8<-2P^fCd3hTp&+rNVy@8VoK0ac2WYC zo}q!Cf5P$F!BSH-_g)(7e2=vLU$G)TY!zVZ8jHl8bESrqF9&u2k8#;;VK8W*;^xN1 z`nx#X%m|a$jHB1amO(B5n~yI2SGXvyNwCY)Z>}#p?((U!mT}wioE#o0MoHe{;81Z1 ztr{!+^ZJiEiHVY`r~RPk#Pb~3V94SK6{dT3uY!&gf_=;AXXqaDvD%ka6hSUOKborpi|&pZM@*eDsyVyw0P6Emy%$ zXKHYVTyJGv*+|6b<${%0Yg79nb!^cxdM-|aYXLlU85_cq#SG)cYomMxaBl;Glz$F* zwh*m1EE{d-8`~})XYzts-i5mz|L&F{&izZ@T&gg~j2q`FB>QdpET{ z7>yHh+sVo3J208@&3X*%*oI>AC@2(_e!degn7i;cZwjolFx8#OE%n-bd#>q(%p9}4 zB=8Nai_`jt=w#^}0f{$zT^{-2w{l9Q{x0uduX&=~rfNF5G+V?KkDRo+d2AvC!ZmtCucI3_=kaN z_+Bgnow`Jznjh>I_Iqn+TqdAwBNZ@Ge+Z&+zam#-*JZK~{G@Ou!LG3jTfjU^3 z*y2TNl{{#EU$l@lIo~&8j=}pgitij?PMrrzh2h`*12UhO!yL}&uS3Qb8RlGKB!`cw zzchAamY@~HGLwzkGR~VV1>EMRH?kn^QI`MPW(xx6sD|kWOz2boJ@VRV;Gc*=e16Nt zt@sPr%^Jk-y&*IEFUyIu=utoM9C^p=Sj|{UZT75=7nlMVWPv+2>$UrQYESN?j>ViC zH{-1@RbLA$^=I3qE}!w94O%q{D}osq`7!Ikb3>RxLZ?@d>Xo*4hP$}>aIpg64-o!% z_@R`*Qzw4~@0-@a`Ew(;G<6juxyd<~sfcK<)uLX2|L7vUI%(kmeQB=W-%%-WI+0X~ z>Y*#-pOie2r!##W-{z%mI`)iQYD9IS3C2n+#t%<%WoW4)_DnlW6G8YhD{@+nLR7QJ~MrIzi+?m|)(kj7lpHzIG%H9;w-!B7Q#is=Ll#x0~tbaz^;t7xcDBe~gZDRgLW3pA|cQSKnPxx7LDs_okZxr_-2V zdB*GHmmYB*9Ej`vN8HerY;9?(6kL6oOXDG570Pt+d!hX|Ac)Er8-541%&tu~RtGSl z8D^Yzo|9>{i{reb@0K<Ebz< zrTF`61>sKq+qaq48zr~$Z-bs+o?`dM-P@+YFBDH*=w310(ode7_EKi3E@<9u4$HPV zUTke%ZkL zyn8qAQd4gNB>s%zToa#~Onj%=sH#au$pzdZ>6I?fX9;d?NltDC{m2Y@X~VHu>btr^ zi89kJ~ zan5bX*zn>0y$DiIcY9dSoGMzor>(tuH_idjnUHA#k@SQbTvyyt0OPE`J9#bzdz-=< z-hklnUiVMHe{&=VpOTc-0t4}T$;BqKrv%4c&r7~4>#<(&Q{}!c`}@ssERUk6O@Zq8 zqkq%@hN7!RZg!Xi%!lleDhLwp!p);D#T^CbogWM}EH}kD?Xr^D%m>uAZWIVeeqRtc z^2>MEy!TRn4!_TL?q;EV`EKG|leJK+A+KWgaJxakqmZ(a_j{vx5aRM){&QAM$yl4fp2sQ5Jw6pTowUo&pz*^4{|V6DbZ9{s8*_7HV< z~Q`>;t2~)6urKI>4~iyQj#zj+ut!uH>M{kxT7Wyij3)Vak6-QP!=JA z4}RGKzzAkHGAzLq+%5HQoa`C#T#aHkdH+%cS0wxAQY=Chj-s=KC-pV$PHW>0$kq3f zJFUp$DFn!`PWC0@o+|TI@yvJls8`*iAuSGXID9yY&xn4N(eFw{%>`x#W&1VPwcZ$u zeBVjwvkq}Tk1ji|=GVLgBteHE9pBxX1j=s+=7l{fE6>i=N6(KsdaC;-GEDmB3INGb z0)x6CKeh^VUgh&BWODJ0UnnsGV?@Dy;yauyqS+@ml-$tflcmWS$VYcH34fn$ zh~DcP-Lj}j#-2G^$P2hnp0J?RI+o#TfBH@d!=<~G&Kl28x?LXsrv3r-*iM9F%^Ym~ zw{{z)yvS?+w}_n=FkK!Qn_f=ykNIn*^sf{ACjo`Kr#cXDrFgo?75KAw+K--lDu_>n zt6A_(qWYG87XK;r`up=eFmSb;lXVy8Glu})+DSSvvCPRSiDE?sNJ_`k0tLEz9u1O= zOby>r3vSher%#4dKKer)scsR)=Q1DrA;-BB;d2{6no3L>zCl~-c)vSe`1*$EhKE;0 z{QgAX%q>5kPLIz_C7T?-yc-cW_u-Dv&{fe>PCH=dzQNF|&8hXzDw%Wlq%_CAyLD_U zruUUHNh18mMy|M$3YkY7eiU49yILX1SSeCFy-5Gzqn{Wi1)MP)#I}#l@h(|Qml>i* zPM!%8q*)p*9y>H(Jcf;<{@ZMXz5v&wHfO|B9m4I`3fuhVI~Iu(8!jJOAL9*7E*|L! z-z=fsC~LQxrO{XAX?O*0h;i8b{N!A7VA=-yR^*?OUC*2yXgfQW0l(0*?$%cyV`O9K zN%VCeK1f)@s-dK?S^Ji0-C_T>*DQb@W(hIty z;`@3M`JmwrJQ)=NF141AhwMOCn;CtqA2ZpL4hvsyKQc`K1X$oPc4%=pBzJ00HGKjb zAz{DyhTrJhv3Ke%d?<_?BBrQMX#(51nb-j2Qf~qgJo7 zaQe4xp3<@WvIA!;C+zrMjK1p8Kge0*E)c&F&bGy3rTtme_MZ#Ba=J&oob$s}6Upyw9UA!QT-~GfJ@@UcXZEbV za|*S`zX?jAL}=?{j^+Wwdwsd7eGpROKb_okli#CRI7EGEKSIRO^@HjCN!$<{&}lIk zx4+jfSCKQ}iHL3rH6^yZhc1<|v6|`M%(R%TFC<<|=pxez&FcPlkn0ZJd0%= z0%K4Cs}#uYxPSl=*}Y_LDC9qfuOh9O|2ICUTQI0@C31MCOn&wJcXn)~=~4(y-) z*riYB-?K_1&XNf3ww<#Ex?(|p-O^c9WD)gVcX*p6>qA*-r(awIJm!y5!5i7#;W{sw z_;H!pDiT#{{~LWOrR84d3vFai-A#D!+3vsGg0mESpaLILj1;CbUbzx`f8mP-z4LpP z7?f^h49lk{4{ohVoA-EJVW>z@Or+!?W9+s|>!{#RNqnqwPbnvPY}glLx%_Omb?$v! z%fj5RLxptTMx(MfRdeS%*IQOUig-hdG}}+ki@m^wdfm2()hQcvlOtG zXVB^ln(JWN4sfg!45pt>gW`*RvJszct}}-5mi%cH4oN)RPT&wJcwQsXJE9tjZDEOi zdDxV15)XkNO>{p$Q%z}~jmsGyoL;n2?;*2FGC3MgW!Uu^E>3Bg_9Y6aj|fv39cToRWZIpREzO5Ah+HuG#S_rGwv)9Ily}zet&A#OOr?DHW7i4pN6+;K!IIQf`dILi=dyABB zr!ka{3JNDQ4hU-w>dcs#=zDJHuzrIyJbK{Ggv=PcKX*xc3v-$yyxB ztV~UGrJlMn-4*lcS3#75`vIVs!tZI5qXP3C@Rpy{uRWS@x$7tIWFf1o&WhMzA2tql z0keVEG!TmqADNe`vJDbohDLK~LN>xC(mS6D)mxmNM zUY^#mJ+0G+BW&c!)`H?ggZ{oH^MmJycMzFjRWO!xS9c`>@-#m&50TM7fzX+VJ-Nl*aJL8-jHEskZm@eR>YB`jN^0cY> z+{8aMqkEuQ&b5`a@!9-^HKVPF&t%{J{mq@Hp4Qw1_hD9#h=}rfl46@EhxVr0Qs10P zu+~i(EVOk?at190fB5?M2rZwzQi`H}!A|=m?+5Yq{2WuA^p?!LMQ8@hB{24NIFzzr zED({d%&>R+=3{cri?#Bzdv5AInSL1eJF&qqf49%ME?&12)~Vgx3?1t6Wnb+Ucd&Sz zCtB0a*Htc;-3{g3vL(9b_eV2!+DmHoZ&W1%rw%4Rg3=^y>LDIo?qnexYf`ZlYF)O$7xYaNVD7)F9D)~j>Yr(1WrnfE3mLvWaq!-*3V$mU~-Le0^Hk%_pm&xjTVM)DKKcF z4K?Aj*p@RmXjh7J$fRLOrZ59G4$f|3ze%M{Xxyd^7UrEd9ul4s+8y6A{TO@lX-RNg zgGpolwogof^{GeN-|%a5L^0LHl%tL|dvU{rQ^B`CKd7vhRI)@o#^L2eM)BRnYaZol z+Eb6QKH$t4z}Rrh?ea+2qaF(TE;|@%qv#LTaH*G`6$+G=P&-iu_DI4$++y|wOQqPfRNFhIOly@rkv|LCsD%|V|RszRT&NPl~@AvIKmMPg!eb8|JWX8a@tiB2E zQ_Bzh2b$g;_B;e7@W$9LDyRXbTTdS<->lEdUDt0Oney)EIpfcW@?I#ef-uV<-dbY4_It1bl+%F!5)zL<<5WH z&m{H}zQ*kSt$tABTcz94=o@lVIUx3p=9u-pqtm@>RgD2DVGkaq-o4QLEPddkAR~Bf zlCsIgQabXJK1pW{Gv9pzZBp}LuNHfNljO|qcapy#?zE?Q#vlhjnuAt3`ZCixA%GMa zLSJcR^y&P`fj8^^Hq*HJtRn$QGTBQ4d*t=cv=L>w^gK2m*#$4sLOwZ?xHqg-ac zArnKn12++YVa?}P@l2^tJ+>C>&d(M9=-YJrWT3s7+u~b%RtU7-QP63+6D_57o6M=`1@YzViNZ?vb@6CVCuZy4oaZD0(^?for!Zop|es&yfubOb=@3z5Aw&bdG+ z)q~KPOL)4c-fJo;Uo6Rtct}L4^S2Pb(!NNt1mQ*I{NRl2XDOWNXO z2Pq?vRNc-q{fx>)tI%R6D8q+RZ%+pZZd&AcaPU!9>|+lq%dTh`uIu*Gdi}F9JDo89 zs6U$UkyNFX1OFKW7{pZ@9dT`*LGWG+JKd^6;haoeiwq?bYC!6~C^;ankJ|^B;e&Ngw zeAbMGI3^i_?Os~Vz_OCy{Y9Ld^QP83M@%@5&JWJil?|BNa3|OFv<|`=8;!85!!u|o z{$$|41n6!MtpFvc<(E(vVB4rnMAZ04CG%Mua+k88ZJXuPXV^bOe?IzM{F~R5`~hMX z;{y?>=H$>xLAqbdT#ZE1ThJxac2_NWysCNi=ivJDHvcSN3XK<>``fz|P4pJ%O2Lql zvfsbLtslRaKE8oal7b>AZ=_tLjnkp^HgcjgcS28f5Afaib0}dB>$7apkzhWYOH1joC8s?zo42}O z=nr1wHcQ256LZ4Vit83780WSJgu1KW*%qG(A6W=UiO8iI+?boEt15O(i|~}T7r1be z9VS7(KYiRlbnkj&8E1H~Jk(5R$m<8$Htj}{{f}tK&`E8>h##W>!&eI#Y+?J=Yai^s zaB{?G4EOWb3cn7$`n8Ncv01Hjw%UJ$**3@FG=y_T3Ta3pSIS*uiJT_kW;v2Hhe~Hr z3yR*wV`}=k0}hBs`hHRGwOfV7bldHI0Q*Mq&)`xZ1s4l%2K4f6e)>M?%PW#WcOK0Vo zjwmA$D!z5-_ej{NVFi{v?gU-qWxQ#HNH0a=uo9e7YbWPLQ#gyJV002RvWa;8=ZSMp z(jS|hAl{9J>Ap6F!u6gL=S?Ke_LBPeKf~4-p;D>?7o@gZS5=K#Fl$w*b4vBKqu>t3C0bjwH7ZL3R*m^M&cmk7A)a4X z{0_%K%*MTg+O_68WmU13e%J#GN9@Vjg19Gr{M7o9nOuu3kwILF=3QC$=cr%bH&xrC zVg>Q@o4GHzrTpXkJ22&Aj}P}5!blwXIX{}a!`meGdu^3vL%Y5|{o?0#eULQ9JO5e8 zMrU+_*`OMh&`V?A)33R61-ESYo;4ER&FuSi`_HIvLi8}F!!-{4GDE~;ci%IlLA~cj zeQ8?X@x;w0ZzU0~3L7xhoj_wqbgilLeg8700}Ot8=WfH8((&k59z)yOItM0klBeyv zP{o#3tkU;+5!##ZBaky8EBlk3cl0%D!bRR_8bQhuMi1xOm)G?z;+;p z_vqrhM>_EGrK2jxS;r9XII-Ug3dP#SfXsH=88kUFWCm>|c~SG(*m&&rpabV$Mbn2w zv$Nfw{dxlMuZhe=6d8lUu&YB(0!x#jU|>6{#S$4rVGjM@PV0CE|L)vaxMK> z`AAEp6({QV%B+g_?oKBnC?CmHM)~Gbxz4E1n|^I-t;#is2UK_w?yf9joLoW!C*!j-}JED9nM!?1X=cYQennU7?)vLMgi)ZJ#A6P1FtR|?c2Dw&=O{{im@!B=6)f=AiMej z>04p4d~YN4PW=In9%-p>S3QLPLrtZ|q%-bXN7sGa6PaCyU!c}Llc0J$8qCVjd%^HD zGNP1+Qb+$uS>1x6tz4n}`@|U`&~AfO&A;FEku_&PM40?eeIP(qQzFQEQ9ogi}zO6xR*7_=fJ z6s|0_{DhxA#@YukXlr41Q7tpnu`jrN*Hr0t8DIs%7ko5L`G0&;L$rvRi7n$JZZso~ zKF8Ojo>k4RCyh*`)cM^ZM9?LfkoAv>3-Gx%>dN9|j&hFfJrl!V94PqA{9dpby^AN}yK4Nf7GLrRAbP&&FvVy^1=}XhFuk4$?ckebVj~PZ9cy*EV zKM&|2Z@+j1s#ROX**9f|aa$)5xlacR3oi|w_$y~LM{ePZB+HVtY1^G~QX;k>eiNnz#*wbBLc7P%^~)f=K_|uu(n! zIC>@;J6}LInfTQV>Sthsd_*FsM( zIKNbgum->A(p_|Z_gU6~l*d~>&3kLpjpACn3q?7`$5ly z5`FHswY0c8-xg<^km(3Km`+{%v9mAB$qJxNMRs#}d8u20$V%V^gja)pB=gC#Ilt0z z+1Fn+{;8MwXP=laj{6&Cm=9|qPd$D76NQ^B_)mL|+@BvSo0HbWLvb0hr{l$&Fjy-T zRYX<}HKx0jajnP;)kQjCHc)TWI9zI+$t$VPDxN=4*clmCkS?ueC&5UUnc?~999LrY z)rwMJ%tHb6czfg_i_D}&kcZ%&URrJotF#(d z&u!&AdntbJCc}cATxa{){+*dMVOXOx1ep%V%+8~%*e^qEGy}46S+cgctX)Hr&0Zb`0@aBiN$%ek5;HxZAAEGV;I!O6yCG#g)}9MO}3Z zk;NBCO8;ZYGN=sg>-*eNlo=93eW5B)+8pY&lmO}qG@A3GcN%+|2vaj+Dq{z<0ENGM1DnmoCrpdOd*xNmmAeVv-->znfX$FGhGq@$?>8!nMZxj zm&+e*j;=bHWkehx{tI!ts5*lgWFn-!-k(v)#EJL;&EeV{7f!%MMz8^Ql`PFOu3sRSZ8CY-rv;I5YP88XN4`%C6G<^l<`A}n;+zMrz{YvCQ58-dRFER z)~8GfWAWGv$HL`l2zB3E`j-yh-2=$|s*1Bq{3W<4iS#oR)=mu_Yk}j&F(b#bp^jwt z0I4a{v7sFip&q=*Mppze6MLhlDlqtPx-W7=TuOoXGbcwSg3@~m`m4C734WA$&cuOxnN@x(EdaV)-#!P>xnV0oCT+4gg)g+%(yV=RI zv25DWZKj+jZ0Z8dHNQ0R10F;2oWA8)a12xbVpZui(YN1yEyv=c}?+@=#Wuf0+grk{UhK<(5;f+H&OlR_e z-=B+A{tyj216U|`?S=L5NXKa2E?q1{pp150yDu75x#~Iz?Jj03wfj3OKL7f-t+nox zFw9LE?lc)GqvhasTnkqmJCX!oFhKoK6gt*NA&mdP8mXsf4%l(ad;Kd8vpf*Q~(PAe~B&oJh;O zp|Pg&8!X@fO|To^%Ib3N&rAJkDMmf~jd*|FUi)cniEl#ums@7vz(lx-e=?{ZAXyneTG3?`&-$!uYkTvC zPt09Z({kXrA^Oq}fwW)0!DACn^#!JH4504uWL|_t=Oe#Vo(Va6P#L%KjeMm0p1%8inshKW&=YYLOkCM$Io-7*suLqnbtUy1M>NLV zJ(Z|Mgq9TJv8UvXwPSFM$el*eO0V}eI-DOraB3M1>WmKUX7xPrUp6wxDdcQF#09U| zGDfA-^OSn?(Sfk2c0aTfGq?~i2&ByZ)I3@=OT;bwI8Bx;Q3mTjr`A2hXcVLg#tFhy|2`-ddQ z8OVr#NBLKA&!X?EBaLF4Z~%HH4GhB!3~d#>#>1}%^DBB+sYaakqGbLNdcW*Q!^cU~ zAU0CazQ3<&_CfmeJ(X6eK#WTJn|iKB{sd)WVfw-z4obG#t2!;|6&{r@4gIGl8@xJ*rc@K2Pl-{#YOiR%1B>B*o(Ry9F(K&n z83ImF<(1Jj-VT;2U)e3&EHFv5IiY8GC+~=bR;f#eOC9hc$9Y9g&-qZzp0h_rtzBRF z))!KamXJ8_w2g&;r=_+57kiRrH4A=6J0Q%580KmtU;+&W*#JdqRcU~@1FbgHNkVsd z;~UTCNc6;^gj9e_;tTGo;)kV=+Rm)M9;U5BCz!~6u_i;B;6WeRc?V>ldxGZA?!_*6 zzC&8JJg!6IamO$f~w6aR+c5wk&bi;uTo*O1v}ko4j(p~>7cX2HJb1RmDm4y-YMQ-% zPT6TW1p>N2Q0OAB z$mUMmh#37WBpc`Rat}+m(1%l1bvT*Uf|!_V0)*%c#{6+G*TZH*uSCTrQvGzhZ*>5kp`7=EzP_LYr~TYNK#YxJ+H+HVMTk^+Gt=Msd%2vfoH$SapNhw8 z^21Up3?I=Br)U1}#1jMo@HP<0&RWDSv_Z{dDI6$Kj3I#2u1ZrzI`B3`1zmmQ;Kj#C z#idlv1%_`2G%Z4h-g1NANW)G)V}{6UbkC=IoC#($DTE7sc`a|i%1(k?UE^o)iuo_O zBB6+MIu3%i;lf%OA#lZQdhl0vFs*aUVOx3|>D_6Y2$;~14E5EhvLFS!t+*PrcfVr@G7aFK)dOg)VmK@r}=a z2Q>HMEm{upke9M~x?4S5HgenH4<}u z34jY{(VCg@n--*UMsYeqK~?6u+VG1zZx-&r=|~OH9}6LymIdAaxXDG!vJY}UO#3*Q zUWqUAs&%=&A=x630^%_s<7sK&(7G-Oo~p9%%bOCVp9A#9?JiOu@vLpWyUt1{4V|ZR z{P_HD_!kDv`(-r-k00`be!|Ztst8(4LQx}ueSPclXpQgu{ijt?DNWgp%0EhP$NM7u zGfEu}=AI}E3X4GC0%1zmQVmFd-X{G8psJPkkE-z=gt4^+c;clNmQvfprianf^iqFm zTvjT7^bnc8y#t?RgtWilcKk@?$MM^4q@v__Q$DG{oES2K80w}mQBIto;kc2sK&h-e z-=^Ro7@_p#|KshgqpEDb_d!Iuy9Fr$0qGJrC@3i)9n#$?T@r$HcS?%V-Ca@#>8?Yl zbR0V7=KaiP=J%a>XV&~R%eBsO0q30ixu5;)d++PI_TH}?TI0t?jL_?bhJa3 z(=%~Mb7Rs~o}}wXyf&}~a7)de7u&PCvYtD$tZ!B$K+z#G?9TblBVk@UKhltW2t8rr z&fpUhV=_h2TYoT1V5pQtt+a8`N@+5!hC1Tfq5RB4{`z@dqwnuC#)Wwe>qrByQS!~^ zNV;tWzfX7p8fJxfh=|2$O_a1qi@zC|X*H?y^VRrVL#ceWKGH`t2~Bk-$N6gg;=o8C zB7$EIWJ^84eG5bKoIP@w2CfOs2K!5b%Wu$YRC(cSg$4^KHe1L0-W2SG3N+y_dngm| zL2YS6Q!x?x&HrkPImL57PJvnZD*M= zrjIisjV2!5aY~nTeEdony|jcY_fY63GgSjGxmA;*Fw&Q8ZhRI+`PJwrrpgoTto1d6 z*7v{fdc1mNcXMR~se0L%&ET>x+6MYr2q$Q|A zmRj-Gu6vwLCpZbB=mQ2PntIlD3brF0wBP3#CODp3ARW>dnO*PNWX(fMueG2Y{=&o~1&sf8^jQi2B|~W5`!hP^`27hFi%nF^Y?$Ud0$_fT zJPkhl;1Tm=dQDj^AL9)X0u5{|2+QJJotL~9mv!1TTyK5V;kuL~o_d_b8~Kr}-cP68 zjgMrutE=M%mTrMY0-MXV1%Uq_YiITE zkeYo-Gw!pty)d&6Y`&$v?=HUuL%%$|Gqb&~tpUt?r+odPaomoJ_{#NWWc2X(l&E}k zIA)6P{Mv^%cow%0F=j0OBO(ngRv#LQ=}$4Q`^R>0Pw&fL7}3?n zF|WO->l2N~j!T&KNnW;h-ledY&egjLj}|ZW8E;b;>_^^C;U`=UbQo!M+b5`c-u_k7 zH(xdM>O6|@pX95(EoZv~Ed@L{sg9wL3x{11-G5ku+_KBY4{jjBI`o+#{o+IONty-X z3@0dP8!#`|eNEY{)K3`j=jkgx#F&z@3wTI0!o_Z_JgV*j#F;x%1XWY1kz9DD+oXP3 z=4cLG#GCeVb{LC8UjFF#l-EP;#DMdSu{2`#E0!`h&`1`zOSHf@NUEoz)!eY5U~ zagFLh9E_P`Qs}RrGHR(2%EQkl)kT=(T8a-K0A6GQ7PJ@lyH-|;+qcoe&Nmt z{7z1{8%r(o5PWw@segk3Y|Rg(uH93v;bSVyEDf_!U%aDfZa30FsC&874rf%ay6HIk zS?D9?5On4-Quw{)duTr0*^RDC>v-@KC>gk!P6j+KItAARB`HS)Y(trKy&nQc2mM@$ ziPh-Scr@|j2|!J&k}`p`j4hU%m*|5;>MAeF$We#eo~k#Ef?$e(BO@uI1k+^H0KSDzWo`SllnJ!K%0cH4SyF^<@YSG%A7%n^Pb zl(<7%?=ZgomT&(I^`>j7*{i7|o4+lt#mR8qj_yqk*G*$_{cgjkoNTGq5I1!`Zko<3;x zyxk!)KlNUk1w70^8MPvh1@-iOct*m{_p^06r*9XSQX=bS&MSnT@ov45J(GLhQZs$O(sHznkiYSJ^a*a2x{!e4!Jo{b){WRR6 zFt0$`wuA)j=%t^fm=*mW@f-@45Oex=cOdSIJE47B5Ni?YVg&U_Fk9R|Z=9fsnSp;5Afh_emGa4g zo%!&7+?~{}PBUYERPO~wXUt$&dQh{kNTB}&Dlygf6YK<61ujR^o(v`$05%5wnKnEU zYRy^)eiLu<_Ycf#hKMjCfo#_*Pl=oMecY?(h{Y+2HY-Sx61r4K7vp+<8`>IHBa(Vm zNt=s*=iFs}?7g_6V76Kk99c--s=^;+HheN2!SxO~@?xPHXS4Q~Dxg|eAD}EHz%iy0 znM(O;wBDlT<9+m|fk^<=TUk2d^X^hE$d~DU?C6fBPan0d?;CuI3$Rn-bW%?i2gEs@ z@!LHzR-OJ=rX?z%6FLtYE4BIMMI*%HdQ^bEsFzE4d^7(-hlr~!PMwpT5p(1xkPDL; zf01A{OARmiVYJjhWlw~>a}8p=^JF#R+bSEX`ZZ7U30)m@kOEC^yr3+q<#HEy;tMVP z!OlM;YLh|OD?^meWU$fWJ;=h?GI9%^}&bcZEnsl05~wwfc{Z{o{4 zE5<^B$Rla{YmmVU3@(%xK_IbUH4%u6(Vf5H^prFOkGwM7W+ZG=1h zB0FTaeZp)~eD2H5%Qi7rj7)5sTUBuYT*%J*QiE4%;Lj=#9NF#gc@;`2D8&M+*6rG% zOLw0lTgamQ@qx@@)Cn}Q7bD|6qU$fJwaLWA35&BD@^C69;r8^#xWfm@s z;zbvpnGM+$>o-_DF@Q{fZoG_%tkD8-Sg;T_(S>)$sPSo)o|od9H~NYxM)YUTdfi-(h-?c{`Tm<2k*X+wuu81^6rfdy`_~|7 z_dc8FOTeq~VSN>^%uV?DW`Gj(<&nQ^Z$Ik1nEk`lx(p6`^D@1+4C++oz02xAqt$x! zvffkF*V`v>({Ru#ZJgEV?pps^1Vmrae%A-9+EhH*H9&cR4%8^Hg>qfQT{(0Aigo_S z+LYAWLSjdWZVgKZC*_cqxAhft4c!J2YZX=@-4JMj()IV#9F-N^HnEP3lbLiN7)Q1& zU-!GYAx#MvznhrRH-^}q+|H27Akwru=6U&?AM;3sa=0&ae`1d#R@4uaf#kJp$3N%cH z;h}+8c%`}lATzEy2w8ZV##W*v6hL<-1_y&k!{X$W27o7Z-@}}uus-^?2*>~2s4@&3c zIWR#F(~DVapx;#aYA>L75k@lD_rmoZTBl}%imJNJH6hH1(F+W#2p8Lw;EDA9+M52!dM*#s-1ZLFS@5+dF=4AS{5c z;9{DU)Wfi;n?}bY_CqW6z0;qw21TaYgAjec$T?1v=M`|%gqiuQNJ?7Qn)14I4d0VB z?jg%TG2er9wxT~h4P3LDJ_%+wwH)4|;iX^1F1Pvtu;1oo=-&`ZNLyksD24Y8VYm=Y zDuRDFK|SR>af;7QV1JQ*RN_u#*S+Bm3AXlT{RHjSZQjCIdM8a!$;m4(x(qv91szOp zk%<>0zzR+_v&oz?Q>ox1kPPS`1H}Rt0&;E*!eqy|D?ZVQY%s^HgEUG@V@)VVRn?Z? z6?|nwdvQzAZ(Sb1c`!mf$&*g|bU_;4jYKn1jti*+z~Sj-``s*~mI<-ja=oNIrt3 zxly@pxh$Jm%#-mSXZW4&3usCcv>o)(0aixGL{S7Ip@+GXxcu{@vedH+^S6Zed6$~% zY7b;4Ku{jQ$0LW7D1mTl-sg%&H=Eig1yarZE0TJL0__(I09CmWIw1iMC=-V+RrVt1 zpgz59DHxVpB{F;wTHe;JZP%Nc{DJ*$jFp3X$vc~gQQ+m1SD*=Y7LnJ>4am@?sVMYM zGGnD**U*ibjA`10Kaj=xvp?SZIWM41{L{=>LW>*}ERs@}R%8?cSQX-Jb?GdoU?wsN z(xT_}o~;riP>j;c`FRw}QkG|5eB??!{EPzuBwtgK%_8s{RdDJ&7MAl8+-iBB z!$i^I-(ls?ra>PeK9nli=$-Bruk9t#oo=o26)0v1xgU*92>`(3)u7WCV06^~-8s5z zgR)hU8vk06qD)T^?uE9-dx!vquC{tYEp_U16<#n1{>tb0N$xIc_vFiN>cg%d{K2&% zOvgm^o<6drlxwxO=i(n6^?xyL=pJ?HNur6th%y0C1q_|jhMeNEQ{LR;_!)TVP~5b{Q%Wg09> zL*?k)GMRuFX%6j(7wVaKn)rWnYA6w#dQG-fZFgl*D*ojAf%N9cCqOcW84q@(VkNAgF&7Ejek?qLe6u?AI3!TrvWn?BqM1;uROr# zEsJ;!t6>d_{F>W%@LH)dcDwF={M3ZsK!ldQUsfpzgu2BwhQ%{^`bop;YqNWLcA+!c zl)^94U9rP~G_!JY=Y(0_Ly)Q}mUMEYFQ%o8od1Y>{`42CTvAhA*=e}biZ2+h^22Ip z5giE7Ps5zjuh)NjhFGqETjs0EO^Q7ItC~EzYOX>#{mCAyb+GAAFlH~07q&LmH>wAN z@}9TEnW;(&z=s{I2cz!C;-`I725rT7UWVqEl)iF&pM*uh^ed$M;{tqC?oMRy$bs(2 zqvuW-5R)=HVZyhM61h_Po`@dX+#RP!n#&?c=U1Eyr8UFD36=zF&%y%0Mu2b!V7|b= zSmB@APOwr6FhZoJ+4#|_#uUbRleX(^fRlT!N^0vADwgWvOsASNbau;7^**V>jNmr5# zkXy2e`%xqLSj`r>3ov8AGC|r=Qg|7=)u70IcXb`n4hFndVFl_6sLpfFSp7N{6=&&R zw@hCFBG~H5jbvbqJ07nmGw>Xn>l^pEUTIM#rF%g}&O}EZAbl(2ooP_9R8?m?vXh3| zP0K30Wnaq=FpAi!lQC&6|59DCKS!d>hDy<}kIHu<#g_)Z|$-iZmDUE{z>|*Al<`K7(?lEkaGiO={$o}=wBR^gJ2zf zVNsFdyHLXjLr$`YC11s~OTn$8wtKGzp{cb#)rj)lp;al%K7BAG0XUM2&kN^d)Z?XI zOv-45V44<&c9xsbi-_rhkuWdWQv6TlL|H_`!D}xBHs{GeO0IUIe3;{P7Gx1+mbBoV zF4e`$N>-WMjLGC7N?VWaF9}qi#9MZ^Y>XvCGoJ_A#&q5hynNkJuirSJx_Z)eN+c_N z^_h6o$M+H+Pz5tH)8zb!p~}`(-{;|o@SJTea${s4rc)7cuya13Y=3AjM1c+{hF75l z#hp|75)(9V1X@IIQ&YH31H(bDbVkYr#6M0af-bIOdm@>G=LwRxlU;C9k@OA1W6(=2 z@cVjrwE7$~M%$fxrlH!AAe7m^2SXB;K6rV+FIWT>X}r?s5NYDIOG@OiHh z(^uFvASvaBPC))!jD1j6KEa9s(Oh_1YTe}F-}kbPmE|{5XM>Nrdv0-r*&a(Mmk-NP zzQKzy*9f=Qro6bQ65k*1%OO-Az+_(cpZ6Q#JT_h%)`m3t2*aG8ABHBZ7yJQJLSen5 zlc7D^zRaM$Wfv~&ulU=eg;x~xLvDfaynR~A%s0b#P~rkg7!S(rK{^IBa^q>QwID~u z1Wcb1a{FlAo?(I19q3CzrLz670zON$G)qlodUvqehycG01dx`>lYnK(wbRL!va6Iu z+tU8kfD%~|&=Mz0>ABen`?zx}kJ?%GGS6R6>AfW@Lbhs=Yu!P?AIq0*(0-10b4Lw% zJ_mez%_}wbvm1#@*Mopb&O2H`-`z^YRC+U{(lWX6E4hdupl`2fEY1F7=_hN;JUP9Y z{6u21x$@?dVUK}qdIc{LDkeyF&Amc+=}G0Q=PznM0QG&nZd=dC)sy*Q-q@lP^sdTu zUsEjHknTcV%G0p%ty+*^LYv^{LsfG@m10>$VAwZK3FiI~X<;h5cpGex@kOonO%|zL z3{vN8s3>^uK&~*;eoE6YLE~W0X~W;UM{Q1Dw|UME-;w#%2j)S!@^l1U{??*J?#YUd z3#i)$1=BS&pzC$l$9D@Sjj0T8WVtr4(RRJTbR#?LV9+mN;Xw#cxj0#%r}f_2xg^au zgd%=fbn-nzR=Q*?w!mHJ(;t(Zt4gt zz#jT@IEK>*HS#52I`5s{60t15OeZ*N(sOPa|7|%3H~>D6;N%<=S0I1_z0;h05U zTfWHTmv!UKNmg!)%R`gVqcs9W8av*VAz9&ih^RMIlD7{FOiWY8M=*W&&vd6x0)zwa z8l8h6J1Zf(wZ3BESHAh)>Pg-WzBk@=_C6_Lmet!NKga+9#J@?Sj%x74@%oRp{;Yg(h}!)?CKQpyPmfxaTQ};e7{DLgMYfL} z46^u6a((+Zw!1^nkZvMHG^@u?R<@m3O`fCeVrsMe*=7v1{&mh8(aR>xLE4%RDTc2IlOP`~lqIF2p zO(yb6AW$~e*aa)kC!h4?1m$a}W9@(l0cz(iJwYE@A-SjZlRFsbu*fI{8HUvs=!Tj$` zn3<9|u}mI*twvRtr-pvM=GXIC`7W=n2Sw-I#`vs3N@Cl#pG^F<=)E$&7d3V8E1-pZ zrf?NmZJ{)iKXo@cyiMyDCAbF$6&!hF+zGSSjrOZf*aLMGOBkq@KxDLJiH^*v5O+2H zRhlAw~qL# zuMToqyJ|xdu`GsmG0X@m)6P0YM9MbB9HKjiB5Q6$>zOZEu+` z8K}%}mH<~}`AR^cFAEP1DHQDSGRfNZ*3G6K45=mF@c!pCa=-10#{dUoNIKIT_6#c1 zI&8W#ujaro?MYKx_e3_X!g|S;c^-CY?VO0vA93gLp}(AY8bl&wSbNX%oSAb)IsQRc z%u}H_8Z?55|A$Fa_}%ev-~H>^J4!zqyL6;_7$iFS@|1{xYhTV7Ykn4%S<6MFxT@*B zoU$>1b7;W10lgnDhY&{X3X=;R2am{IyqD=o${V!K`_=a{Ba-T`WPdUZ{H6FOj=k8W zwiq-+{pp=P`PmehP^cXk(TEhVW#PbrMCQeIJMnQ$ltq$l$~EchRT_w_T{2 z_h29K@yGR(9IwN~H&t`(2q>(|(`18JbAR+Zw@wzx%Rv}eC`$1yyR%_wzD=n;)AiaW zBW$bb>=ib^TUP(!99*t%hG*JF`<$5-{FE77y}gv}(wQQ! zlqfsMWR3azg+wa9_z)Cv-L*VZsDBwunKK3#d;!Hs*3XLAZuJos3U^0$E$NorDX;k+ zRl9%1>`ZdxF@3pf391{l+r{J;|2;<&q=2^4HxrGadlv@A4DC&S)oK0P$7RJ(v9yC` z-2dl2`QyI#M9?J8Sq%R@rE#lM@sy><)&!&!o4|8I1^`HgEB03;Viy_X@E*Q!rX)(y*N#j&7V483akcFcy;6L@Zm7&F7 zdn;MTC5eJ%5p3w$WS-|$Smd?UssXS2es1;C%eiq#=nl4+o~V_m2L4%39XytwurD4? z_efe1+IAadxx3j^${P4!M;$uq``&S-JTE-R9l^p#@aLVZH8a8-;}VuiR*&!=`epo9 zj(1Y_&ToOdlz8@$9f2YXo?*v#;?PZxtnR)e@4}3~uEO(V4nmpvXU?dj$)X(ai4X_&tI1>uRK(f*$= zfzB)hbB`_8yzH0WB$!>4!PQKP*f5ebeEYS6#e5qgm((%!XXo1p!vAdCAQ`7Pv#Z)t zt~AyKYxQ@kB+6d%UQQq-eP&zf!f@GTRSrGu6letXE=SL(ur1js4HCx)>xN2a`2IK+jd$>j?7v@qW zIpy6!q;nV<$+qUHkGt~I3O4D{NbpT-5FWw){#1vljBIK6!xigR`8OM?@YuNm&2WCdtezx_&U9%j9`C-B0m%QA67WF?jjnMu(!Cd0> ztQ^5^opy>$iL3IVho5?x{JNJIxGxF3?{d}3_)^pjnpU{K#Xg@p(x*rrpS5L_KBm&@ zKfr&uW}UaG%I>Q6pB{mm5@U(H66X*EbR{-@Yad8*9u@F%m*PvKE6NVfl2{ezrzicd zHuT^16!xexR?UnP0l9yD1!Jb7!P>sO zhA8#4HI255XIna`)`ke-C(gGmsw_5=T&^G3z2QG!oN1#oRuh87ZO9JQ^aTp`x#sz| zUxxSvASO#x$dYclaAqe2FOYm&cTqeZ=)d7ELDFre@+Yvg7ud>$*R zfq_ej&3i&cEZ0SBN_S1}hx3lki+-eX%ECZQ=8vkY)ALfkah!vukg$#vj0#B-=|T_`|HOmxAtinm>av=PLd2VZ z5+@QePJx~oB!AWMh_i21Rt1L@;8&~X{)NO=Mp8U(_ z;$_c}5wz(QiV+AN-i8&A5@Q|N%J2w5jfGC zt3>~gyV3aI5iZXz|ERA+IsR1dS%nIP@c-h6{`>BMk2!B#|Fg`2Nf!ITfB6$M+KJHU zYwZ7N|Nnjef3~+#n>jY~|K$$`q?PFY?c4u*b7Eq1|GSF+{^);x8vWbHq1M;9cCx|8DF5{YaDi|DQMG zzhC^{HA~7y{d2p7$RBr*1@;P&Jw`GfLvShx_hYpv{n~c}Q8ad}b_;tnhQM8r(|;9( zMr-o&{k6=z_Fa>2q{ayK(bMZN& zH$Q8boP?Ff}lSNls#;DMp;3|DV=^ z^RSkvm<$Q-V2>GpgF;#qgaZF}tRmk9zHw0#jFO4ViF4v8VxaZmYntVFR8w@tBl1LH zV`J;jTWpb2;qX>dMN?p#XiHdQ_kkW(o#c;&H~IJw3ixt)-!k?1aJ+{ z@(Y^pJ&Cg{i?RO9G&0>Ws9)~HJ$fZoxWHSV9l*tjBHi~0#i1gQ5MqGhwyl))luqdJ zV`0J9Z&)pS?y|k8Gah|4l~9oO5o>GNFlOh#`eZL7-G@J~el0vfVX-OXR9Z11oAa!K zDterE=G<||#$J;Ke#i^6P#l;Nhi7Ia_X;5yABG)8GP(=%Bfg6;oaf;2SRN!!n3g`F zqG%u}>c%n>Au0GXAjTS&Jis6*Lc<~y!ol7kzN*d_#A+&h)Ehq?dpYoiWtZtsXb<(=!@bNMEIUQhAI z{xMw#S=4ZOF;7zRP}K3x3l1xN$>@hCU6)~76Z3Hi62$jdqI|XKfVP5x_tPyyuNe_6 z#P9svQLHTbx>WCF0GH|$O24xuMZ=%xN0o`Dsy3L;Wtg2pyGCvrLM*X6fj`=Fd3)`b zmS3^UkG*rgNa}H0(vj+|GLZ4zT`xt6F~PW&3Ao?eb93MbJ1TE;B(3FrGQgw8hADQq z>s3jDa#6qRgGs!AXp9xf*1b(_Yn(>{+t&AavM{psO3z;Nc~Yd<`S64$I-<9c53#># z7QKSXjhNg1x*icszXgtS7H6_(pZL+Cpa<1{vBPN<#vyhO8E(d58;yPb+0FI5lDzlg z540_;w9jBrzqjgWd->X~B&hSvq!{dg)F?bn^x;pV z@raG;?szl52DqywE+b@eH__?p@HER+SJC{DXM#~2IJm{m6-xQ5HYUcZj&2WpC|JMx z<+st0{~(F%7`tA`ZSg*6e&>go68ak|I^PHObX(8oNwk!Xp10h(@XT}t)y0Ls`AVSH z0?Ia3sm@yXhlSY)cy?<`|dqi4DaWm&T|iqSOCUa zYYL+1lQPs!G8@A<0wh*`X3`LgkE2p{l3c!fk>v*UQ(k%zS7Le9ZGQJWipG6)w1QHs zLr0f;{SBo37xs|*w<2I&!6rn&vd0%vce1%DRd`DA8CiUqWbarv1M7nL2U`7M?3x3 z^V`?opjzZN1_Vg*Blkm-L(7R5`n1Vb)3&<^5M|27WO+9Io?BG>%yPYu>nXBJ|l3o&S zV)7K{t3sj;vO!E#2@9Uad&G~uQM3FOkkbwxR@k+zy3o9s}!1f zoe*U%!XJ4wZYmIUXQ=aXo8Des97O7NYOLQh>x47|xtN!3TUvCBakl7Tyn7dL>T-!& znooIfpHY{mVIJclkyUN_cToR`{%?P;Ou z5yxH*USGi_!D#_xj+yROR}U7#R+_2DCYKId+fo$J`k8A{Rh#BpZ`RkJK1xHSPy$I$ z{KsT5p+8LPc1@k(LqKOqFrZ3F8%c2VKAPr3b1d zf$2CKy+}#O;WSi3Wi^rsCO*l@GTagFch9$HbUz>cT63?E-5VNvh$uJEMgyJdeHGah!NqT7(aLtKqJ_?l>{ zQu(SoMdr&{K@bzN&d*1j49`R888aUF*}>HJsj%5X&qs&5NDwhz#cjJ=yP<-IQIAT| zms|KT1FB6Y-Yh{Id#8bGN=RMgSe+E)NO%yO`spTA)~`R+b{tS3ppRA%Tu2{#)1kIS z;kn;;(M><6y&K0|2@;xlWd2NvS*4>@un+fk^uU-tta4)a{=1C6tatxkjOi|f`+aAq zQJv^W3!H{!4m!Y8)n9{pEa`(x7rjK-SS-X&KM zBp&DJ2rkWldcxL_*jeEw_cm{c^I-D}1s77RJ$Q(v=0W>+r-I4}tFJ~2c#9>JiLj_+ z>*~)4+f<4a*dl*mA8q3md+_H;5I>Nth;^438<(3(L{tclxjR=39xJmC%6a^dU}|QT zxJ^oy;T#60zcr(qc%-y_)y#adL>9P;H8k0Vh5S#L`~0Qy=xpbP5Rc4w#uJ4FI?_%A zq~6g>;OA>OJ4#?rW2$_#_gt$(8K6fEzuuG$mxJI{X84r~Youq=x?}ZcX8B*zDm5ik zlu92rp4)Z`nms{{b8b*h$tk|bA~$}Vy*NhkWr9d4jZSv_%uCVvbE|s5EpKoBws(|0dW zZ#-omdzB(Zmrnl^x)6QTxGqjWF@BZCx?4vu8CN41>CKJaSMj@1^K|j7+|u@$)Q_mh zoT)Cm9|H~tkIk{d%fcz!q6<2@OJE2H+8$9-a1;?1$)=R~>bCGevQXz>kA6|rl@aFz zN-Go@n&%v*e(A(uc*5Y&(Hk3o)^wpB)!pK--{19f+=^Tw*+g-R+Gk}pah-Wr3eOTZsG?Rial8SFWTz(1hP>57 zDuo6TE8h1|ZsyWhX5?3-x1cf#C#oaPViRM=n-tF0Ob{ANN*IC01*?6wcymAm#{x9h=mC1^n3 z;T6?*E;1LN_L4H$`&TGCU2xnjhD}e;+MSh#kC7`Xgg*|^zvx@sI8-g~q&I1<*x=au zs1Wb0)za^*&6k8Q{fQ0adttP3bh8G(GjbGDo2Xy*BTnR!V}(9SshnNgta6Hnt@^=0I z06@k>nk09_r7k5Qv~AbL=_ziy?&2xPf8H7X_|*e#J@Z%$H}>BAsUU2CgcG;MW*7H8 zT`Nk5m7x)l?2uL8xB7wybxW62?B_z;sOxbH&9O8-y6yEsfv|MwpBvy5F8TShXX9 zDX@#jDbBeGnAGxiw!7~?s{itx*LkYIbF;PA>^^B>1~w}N3D?edT(?`69>mL9=OnUj^oq-#TRx>2D`{0#Th8w=n2)b_b*;qj z|7JcHWIsbs>Z0s(uc5l~GX)^66ez$lZ+b%X`eYp$8`uSH?BZrWq>6(rTyWz*%%AJg zI6sSFiA0`x+go+`l`{O>EPoPWd+T9>UhL106w$G-$uvX+tv0P=e5Iuui$%Ne(la<@3+|Lw<%;vn4rZ*Xr)O~QN6T4cO zgL~bf4DffBi_xQ6dVOEkFFTu_XK$~;&t0$MOfN?4vD*BY{()?a8Ogd`pxw%=UyVMX z~o1pF9E1_6599-VveRJ$pbZsfU@+{MDj4Tv(khsiW zjmooj{IS-bg=^8&7J@er;k*x7rp(YTB2)ZluBSC@|Ku5gV|6*T!M#Gy>moa>bmy#1 z6mxm!3Hg|&#hCKq(Fcm13Jx3cZcAqQZAS5n$oZc&-10Ixn|v83Dix86+t!^KMepO> zo}G2eXe&?KhGx(U>(p}?z zJJ%>ZlvEClut;l5d-`=c2t$i`dE-Q+cz6Ve7(Jq)Wtqdihh88{b7k)Wt~qr)VZiq% zTxBH$|3KM?i77T>E%?BY@vxEvsR7^pumEX$OFjJ^i^#W`$^}`6X7%oPGho;BCfc;J z%0YFXciLnuIJR-9rz!UqmR>Zs^&8Wt;nGf6>Kv_pOFW4c@W!25Z$zkm+PWZHu_Soy zy&=Q&{>d<|)!%#1v^(y%&t|^uT${S6J;a-;_l1;^D@7{hXWG`->EU0??9YI01ica+ z39h1sw7L88f^bq!A}@B+!bA(&oiu!4ZecPZq%Xb)H(*&+P6ALA>dpl;zW!wSMfdic z_3hGn##KA*KcFHxy&{IUjVZdokHqyRV#2+*mSOt$^9v0s;fIY0De)kEGY;i;YOed3<{>#=zRk~M z6WS|GYTt>LmzR1aiC{(d7tXWa1+K5WD7KtO6AKB02FBj|F!irJ5IBe3D>ou`>Jp1- z=xbqt3}?6MVW{wzcl$;6Wa*9(3tspHiS+_I2^uqZ2ljrC19;q(13ELB5=kiU(P;BG zWJ2xrb^A)K3U*urnB|Y4D_0czyST29DlgxdKGxPTRZ8-t(W`w7rC|x~$+)#iH}L^~ zrSp1|3X2BlG;hCMoYdbl+9B8qUsc@tThjDfO*#_@!ss1=Rb2zndIUvrAH z`C_Hk5MyA&e`5>_|ugucMVI7Dmjx zx^DDT?X!`bXCYznQ$acLl<2hm1QCXX8PU5bsRM~!>OO)ev6J`s+|;`8R_j|scYr4t zMdY%BX4q_($u0 z>614ew8;~u?AQQ#CHV(=O+o8hmG+0reu2wclpQ{I4J?vfi_CnoxEvkxZWg_I!|FYA zpgds;7ZDTs&U#P45Jf0X>hxAMJ4@-;nl|2MWptL>W2$nH&g7QSA31G4VLkcC7k>tjEy4Eo)pf*QMU-@8)_|iT_qqk%aumnF~p` za3~-H23EPoiocoEfFS^(aJ9vfAa_?^9Ly}tIY;hkySUY4FlKUbxqPiOQ?$o&nkTe1 z|51L(7QRKvYc!Sz@4-r3q9GyeI|gHvT~oAxII&HBy`&RIS#$8I7qq6qi2)AFW*vp7_IARB_vxtIxpA z+rH!?a^I7GcAJ8Kl?2<9UDmiZ-PkNiZ1VgJ zcZ%Ppm#5o@3;n_LrKaav|K>coXKu-G+g<7BaY%D)YAc4De&%kJu($nfW1b;R!?yW{ z4p)kdx+2C!60)K!@A1uSRX;h1!40M>X=h)n=G91Mn1N$b+rzhz;moneJdoFNb2I-) z_9^?UPEEgbAZOW~;rnNs*TR+}OLM-NrXoJm7bPrL zUutm|8(|?+!R&KfI;q&Wb}0Cy<|{SdljxZC0I>S%NbzX+OjVvhPB^zdKne7-PkG3M zwX;>{4$d!~`!*d~7+xCX8TQBp^2)${W_KQGAag*Z=g~sA<)joy@B}P;q`uwz%Uf^ttQ=KXzmRMlEyA(>xc1pUu zRN9=NHh)eyhiu%i<$=Z)O1ibJRLNfQnHI_XTM-L|R`5(+c1}`$gYFMnA!Da-11z30 z?4RV{lduuK#9J4#BA{`6AnZ+!HtXv`Vk^s+ zl|Od@W+=!{W#}5=sl1xDBhDH8#?nOFuzBiD8f%Ybm`*B&21LJ1^%_a>2bzKT6UqQ* zL7i8RzSU|EBmmlFJ{yoa$Vi8VJBWS-?E=l$(RkwtBt*H0pkGNd2eUT^Uw9?R^JNH z+io_v{#u4*n!St-3{uE=R!8NjLWe8S> zKP;#|J15mtLjU(JT!@R)q*1br7+m)|mN$P)7_>QcZvyw9T@}pBLvR4jKo@rmROfPQE2q)w64~|Z? zVAW+ufeD9wMA>j~oTsigno(h9aBY15vJ$qvW4<_o;I0g8WXy5+k4yj{`L(sZ6-x6D zX8lxQ?RrN&C!0GNJXyc@?MH%A#yL;Om`>}lFPMq^@4MC4(@C>!6IKA*C(uw0qeh-sf8BCoqhzYmQiZBs2 zw4mrvM6!k%UFX=oEa)_T^%UE<4bFB1cN{!ZaS9HDePlmN6f=**ee|1{gWUtdVE&%M z0hNjadz@kFe0eXGB|DP$t?1C<74r1svM?tIcMX!@gsu$7Cn0XeJOlWrynwKH2ZdewEp65S#a%|2eSjQ>dQ9r_95G& zQWxlqPYjU?~e9pi5Pk)1WAIE0R|ofR3tctrfTlXvzIlDq49 zDnEYF%KQCt+OUKM{MV2SKvOP*+wr=a1=SnCe?@R-;f}tiK@mMz)`Dc~>%5w(JG$vw z?3fndF!pjKz;^y*lwdYnCV(d$`WE_`Ii8`mPjazCa#7?OeW679#;)8sSpeZlLgwoY zLih*BkUYu*?0llwBI?JHN?x96`Xnb=?^eG3Xz1I_b@QNH-lLT`&5ycY*3UvP6G^#G zZ<@vhr&Urd^VjLB{;LJRD%{sRQ|b*2(eK|RLHM|NF57vO)u@G()bpKGTmoiCm6uiM z?&Vhgq&eBq&qB-n@`{d%c5K)Q018kM_fMWlxv|BeBSoS=CmE`S=nga4pU=k~(AT`n zd0)v^0^}ThV*|rI@OXx|>@r`!g2!FNl0TN^^A!tCK7xUB)D-6@CgkUP9gJ@Q|D%va zuY!@q2*_|i9e`w8wKhPOyn!=HHMN5=P@G)`e470ZU*V<#H^CWDeTswje`5m5o zK}-xuJ~VNT086q2Krzt4#66vP7AvZPp=d%FDIy^X{rb#gJgYM!!ZZsC(o7N&)Ut#J zMi1TJT5rzbxrr=&g?!)>QfTr9MzOMc&radms5A5bM$>tR!_|FnUqtVsiyk#<^e&=B z^dNc<61_w(qjw{E4MFryl;~ab7F`SmBia~!7`*5Cz1R2ubDfzvXU|@1-S=m$ofXng zf$o@2(z5B+7BNRFm7yp!o~&5f_s!kO7cjC#1{0j2ADXS7nUqB8aSJ#12XptC|0fQ* z^Vh!6k(fhWJeeDvw(ssam$vYJ<7Xv6s5jJNI3}riZd|Cj|ea$`n82>*RzoE2mW9YN6X7KMATMG^4dE z`zXsCehy8Ox^e1j_<*~K7uwsnhT9&UP9IQ8-hu(Rc(VymvT(Svml*BNYg%us&r$Q2KYYBEPYlTJOxPP(|+hilO@4}F~IWA6zSsKZT;2VYuRzGZAEZLGI4D8VX5J9WKqc#DZv9Xn$gvH~` zVQ4f>kHyuMXAv}3C@nK}b=rL5P0yQ3;T8-r_#FX|U2zct3RSbPdT%_=Wzqho#Zz4$YabM#%?KLvNJ>6sp#@41G# zJvkB#c={SP+q2x;dwOcIrTXd*?(|U)nzUNc8IjHJ(E6wkjH3KAY+7w8rEh7pd`9e< zP5ZVAC=2Kr~ePEM12q4C9pZ3mc@`Z>KCpyZrvM z=6;Ts?a=8&DJ`N>9paeQ!`^{S8v9Dh>M!P6y@eYK4BG# zrZyEer_C5R=XV)FjhhQ8d174N|2+HUUmke!bFO=>sVWYrMyh@0bp66MjbJbC{yl=( z8*?Av(uk8^x5e)OClqS;Dwr7S592pmnKFYQxPKz9$r1gcFVE8L@>L1Mx1YDW=N1>j zXt_$-YB<8}8a>ZB%;g8}w4*#siiZYlB{0`l*FNypv;JK4Wie5j8@YikMCn74Kwak< znRM>f&zy~w{8nh8x65rkVw)b-(JG(gSf~3!3m4_eiXUXUi!R}UX_>)i98p^=&}*y7 z)_fpeKC|SYUc8v}Dqz%?hUYcN5?ISt1v38K z)g=lH^4vT&N)8mCx~gWR4S7#Hxjtz44X*ufoAnW8Z37dGrK5I|)iFTw96;Q6n*xb} z%{uSZ4LziTSNtOa`NxT@Col-&gdwGd#bE#XlTqn?`OWcdN%L@vQ=y+aN^t5j*i70- zXgVJLrr2rsB4lh_U$7F@K5D3CYxbD)rPz3y%VOl;zP%Y=p~$IW4tzz8jmK)Q?#+0A zCF5(y5KkZvYgltCO5XkbGi(TRvHyFQ%cAB-E#37b1)y>lEt2%8*&=9x>`23#wdy}{MFIqB6P?r~4yI+1FDOg{ex%d0L$1IWNW7C z==l@$Vj3}2Gt%~4=_UYl@v?<>GQZB%9Sr2bfX(&Lf0y4!2pMPsvdRd~T7QAi&#;O% zTarJrEf%?t(`ti(aK|KV)56{vS>7d6 z%pf<7`bAp6SA6X12ueflcVUjo38`lIM&>0y`%#_W zLxo3Xk!^Ut7(cDJBL*bCo9H?asa8(cIMS+-+ORf?Ie4#_v7Zp)XuwtS->>Mor~?sC zN{;}fhV+L4v^w(LdERk*1T3|vKp>S#12r%W!UgoSB?j38ZrpZ8+t-bo?KfJor!2t5 zjBVX$24mhbpFc{p9B7~>s_LNTSxkY9RKogqlwp8Oo9#orRRW6$@iN8_LA5W)J z$aTh8VqGE3ff!?o7z}R%KQ_L|H@pgxUIKGWnVW-M<&uCe_ep4KDHv>AN)D(Z1if~= zVQBT_Ik-NXn?EiAv={E;#3D%^oFyJz>|o++Wkm$oCzN6wR6@f5d=Tj3_#7}ZIP!mQ zRCQLT??zJWMmB{G*+&;xX$@tu)F6!+(Dp`#4_H#cm#U5(>0+|o*rwKu%Oq0Z@ZA}-HPj-fXj_JlKOZ0 zN!&D;Ue59hV#Np;lG_3CA_IsMc+j;5fNXJ&832KumJmf%5JaD=lOV z)}7SzIP2Q?ZWHRC95b4=a&i9L6*8<$zEu#k>p0@V$1r$Pm%LkEMk9N^$$T>O>fw&R z+ubSXls6V`YBAuj3B``{Em|ejO0&uI*cdS7^7WVA!g2r6ql6GTIU4aL5j#Dv%YJA{ zmDwx}E9K?Axy9u-s-RD`p#eJdAVez&JCSt3M+GNvGm1+~?qN)=Z7*A6z>?sd zqN0>`}Ko7CYfHteF-3$juERu{})Wb~Z znNJ@M;{hNJbx9m31d#d!>`(54$pK^lAr|}y*_7{c-@dC}rtE&qzP1!0K$x|DAT{3m znFC0@CXnS`;BhT&+~9ee+|y49IE@Ue(VusuFoH8D()()#@To18OlDO&g?Rw|G7rSH z+B9`5Tm8^gaK<%e)@a;XH+9TY@CbcwF=Q8b=HIJ7fY$cDmupX&ND3afQx5$rbgd#E zlIfb2Yj(8#N3w5%nXcl)Z;Gh}#&2FSWlJ|P^r*$bnsXvYfIs|faOd&&dNsOe-)1}3 zBe*qhNzgNJ_j;)#Xf&-q?9f_-;FMVQij)T#d`B){5?W(NCTB8;09q$b?pnkE!9UO$ z)6pvAaT96(%D_oc?TpU#_yl+cr#`ZmM$!&>>pg^PK!)T#CD>zP-jh#p{?o4^*I;09 zsK?XC)9KKrIGy2t1U4;(kGG zd?p>LUrb_m?zWt1ACLdMG&UXQm7Y8IA4@|v4Z;u$>NIHAO* zCW8*to7fhCMCU`&8By8A9gGVI0}z}zE)e{nWq18!(5JetWrX+}Vmwz1up*UiItm_+ zl0Ae@E+5pnoc>tjxS3wdyfuGlXZQrjQp)KTVycG~)zAV}pA2NRw@ZrXJ)~^9KL3UI z$h%D&;9Yt$-n~tR4bO~CCRQhIUzw`l1&*H)-efG8k8rs<-xT!RiZY0qw9${0i;F(2 z#`9awdTvqdzI9B6-va+Wz=Iok+?)scEB=E?#zP?u4E7#IqE|20lb+xFTEfbF6vQF> z_It|G+a-g8{r}h~1s+#^XzD&Pb;Qx?fDfh4I zJZ1ocaXd7sSz!NGW|aCvINBx80HA3A=9Z(?jL2I~5G&m9>nA%2z?^o^Vd!uymZq_0 zBdFiWg>{|;81j3+PnT;AoZbayuK{}%~DY{4zOu?Rr`$m`cqdf+v#rF_I{yF@MQ@L?OO8+8}O}H z>($fw8sY{no4$5z5{hWX4zX(x;?%rsGpY!A6;%g6exciLpZyPp9P=>dBhO6qmYZIR z|45J_u={1!)4l1LwFpkVId1$s@#Nh^+~1a8q6OjHnAC=M7nQ=;2S#y|#v!IvS9Jx0 zraD_i)<(cxEk!OGo?*C^?B50&B7Qj#{s2e0j$Xl#yUwwu|MCqMv!7%sF0?c-U>R0L z0GDUYWBd?+n^bs5J3LdU7NnpPicK_GAn7cJ=RJ|CL9p8hW<5c9K^%QV!#qvTgLtrm z?^a7SQ(p1h$)?SH+BsP6v-_`iXEL45=9h7ep|}7Cn)u$!j;sEFYXU8!PGFqaJOO@w zH0LB(CiYq!pjO^kN(dBK+%?UX@|qF_3O99{$r@;J7rdk5xQU!W+xUu}{eh|*uL89} zB}1jw?RxN;+LD)G>#Ys}fyl|wWzjkRY8Gk6LZd}mo{vVvq$H6Y-Sbekj$2ZqsF_9q zN2z1(B+a6q!@()}{zXuu+vu|%B~rJ7=KG^S=TV?r?E60B8=kxFe~c0PH=fu*)pOWV zlz0Ca{k!b0A3c|c#@B5f<)jjY@_!3rVl9F>T=ZWWzTCzoE2Hl;hV1SF0vlAO1>2wA z6~zJ+hPDTm*S@@}?p?*4?tmREzO0p!Ul#peM~abYg&lkASQYNLu9CM2uD6Nm^gJ(^ zn;(#17q%5`&zHV5yeeJS>z|@8pbx$e%?Ua4@a^6zYsG27uX}iNHtIc zx0N=z0+krGHa$w+-0jBFG=cv;zO+Aj9KO8&CZ5lyMY&`bCd932MVc2HYPBfJG5e7= zD6uSH*j~wS01e<@jOh_86;N#3)tSq8IB32KW1)lgu#%rxHtfndUQWxVA+KI@*=kY} zKjo}JT%9lomb9<3^-WGn19|zORY5|BRVR^9j$O`Gy+@x zaLXRj^(YLG*}bYT0IeB8kPV2FzVoEY?#$yCmbcCRIaYmMq$RN25sBJv$Z1e@6axlK zd+fq5yc*6O3VHCVnVdF}1o$$$ATdNlqSDfF9ONA#H=wS*uDy(`HD#n6rbbs|spnZ% zN_oRUm?iGeTlNm@(48I%7-sSv-MNXH+3!OZ-=1~a)A>@{@39v8E%m)4`FUu|tNIEq zS}rnIeEZ!`zggNLS)|1jIK|BWKF^Ca7l_ zRqNntxbH$7w;YSw^pxq$h8@FwsV`mG1P*0AYoGSg zQGVf%<0yR~tm={J?!;)*e+BJHk6X&W{v%$FZTqO3k;!UMHmvq-I9=|QUfB@;H_w$$ zx=~`E?#m*?;OFz20dqOp-M~KaJs(V)dd!hdXpZE-7@=-83KBZp0gZ#f=v&2`uPqe} zk*43K4W{gQ+xfQf7o!KxlMm_7le@+^X3S)(V!gHlKb8r#+NayB3JHA`%N_HO*tn|J z+P!8lqy)O*etLug;KOH8QhS=56-!ZZhA9qXt}~ImZEwo&9o>03T3^Zo`P%pNJ`c}m zb_*6r9Ujdy0C*>=U9rmUy6MOuPn)+KfILe0wC(*Hd~lqzsO@Bj=7@hb zQd-8!<0ar$`;n?$8YhhlMgqq&d_S|mvMeH7;blJUMoLsiJ$YChhmXA91n>`eyp3+z z&vn^F>997lHGoaf9}j!M>Y`sYH1!M!x~EhYzLWu~VfJ=aK-(gs%C*yPFV%mK4jTmB zr2-1~3)2OA`l~c21smu zaUoEGX@d{MeUm5I1rLkF03nk;BWBP`o+>-}X9B$ZK1<#Lp=GHm&N2uIMBXpQu%H%< z6Mz~YP|$vydC5XRSA_;ZY9w&zEoV_W@ln2}EhXGy6FGL)NPcspC5H+vdVwc3Rvh(G zvf)mL9w-4nG(D-A)CUAkLwxSNAHD)@rkiF94h=$&7np+*BYjChn{1zI5L@Zbm0k?3 zm3`w%^nO!qpZ$IOCA)m^P6^DNzY-cuX9YBbxbRBOtpJi#F}IpwUBpe2d0^rGAL7~* zpg9VDbX=GY`(#JM!*@EdZtDJwlVlkka(-g6to1CSW$-U&zufKF%HiF20W?0bRFWQ4 ztm}ve3Hj~>fiD{j$PCQ-xX^&43w6dJ3syh0dHT90v_| zvdrjd`E2iW=dGk@5l%rsTcmQ%CBf@x+gVQpKI0wd2L+*j7VM-rF6hrkC8X8RCyqAH zO9nYyYnbN&?l-?|H5K+PmvXGm&=F|Q=&uW2$wNDrlF_`_FS|cFV)Ur#`+|6&8!;{L z8GZ@%hh>( zQ%0uzAQ>8(9TufnAQhsSBSGQ*%w)y0quZXVzaOV-REYhj##h2)aLJ9CDzx zY_MVeuAY|Kd8haSV7$*NL_WTQ-sJ}${J9D)OW$M!{hRcDT$(s<2n3CQ%Os*V34VQ8 zcKq>aPvS(eIUMmb&9{(Ge?t3v=Xvnj!_TgPpG4lt65@r?gumn`i=uGx{Mg*K3rPtCiZQmKx@rn%8~; z7>z~nmc)8xgzxn_0nmUuMKdQoV&HT2UxSOi#XarG$R2AZK*=luJ_hJe0WJpZaR8e9 z(I&WxjC!DQ%!t|p0t~21ie2ac;K~7Ox11r4V}lPGD;z%LCzby-udh|S8)hcf0Eigd zy3p9E8>dkpgf1hJi4|&y+huJ&G7+f-w>=6wBq ze>!OFr7!>Tu?LOcPL0LlY_8wY1}t13b{pi)*`+LB=cpWZ0K>1QS1<2_l!6K*&QU8! z82*$21YkKs9r4@rD|$&)#+za}CSF2VqvV0~flhYZB^$hst&5(gqfn5W@AxQtk?RTKhA@(u)r zrn!kn&*5GEarzBadSNj53Qao&KJO!|xo0jxlmf4l{ zoQoiMcjVd~sqEb*>A`2v<)2J+NJC2)pn^(st>Ld;N{_hyT^T$!{&PJ|8)1IgW$M)Z zDac^xc0em=uh~0DuJgey194|Q7xwZ%cVz=_YOPg2{kgj4oMYgH-YeNNo~F@lmO~q0 zvI0A=Y{$O8Z!HDGpF?e9D6tshCg5MGVCy}~Hn!U>9d zcn=|?=Q4l;(b_()rey3E(OZ*UX@pI>M^c))U1Hw8xQjP1RzTenyd)-J~x@_oy>#{&-S|%op?VuFqsOq_^&rngX!@?t#m~PA&0Cmf$YEjF#;g87tqxnI6k5Ba8O`c{TSdgo1Z=!aakneND&=7)h>!L zX6P-r+B~oLIqvcD&|)MHy2&jiRWCkLtW4HXg8@hvMHxS2h{2Lo=9hmd>1azWIEfkm zi50v}-xbn0-Am;FC`O{OGSQ*>2^DwV(C+8M3OT?yiI*(k{uIg8m(}_Mr)B0!X@kb z6RSur{QPiYezOq(L%W#SIok@Xkl*VeV95>w@)b54GpW>Czgku6Co>a5K>xoTd2zmN zn_2w#v^PVITdKIzjhm{onpydt^vhwbTI+;7_=yuopZ2e9$n%=(RO^=QV7_Uk0GU#w zjQ{KXkPDRyyQO5@k^3RhZb8ph!0|URpsqJ}7kphbkzStkocD(xm}B>~Quh3hj6gya z93EfIrISotBbY#6*e1|i$u$TdfR6d`XO+K|GnnTqePtv6ZX?@7rIHVR?#r=*VZ(GmTbwS z9QC}IF+5W4N6S(491Ub|lMCg3D#zuC*B6OAqsUcokD^N%pqVl^S?ccUaE0w8d7cDa z*IZ`YnRiv55bxI>92ofAA3L%y2mFOpa>0}0s#6PMivH;Gzs{NPo6M zKd3XSSDscxs*FQ35PSH-N}XJZll-|zLx0|TbWH1KR#^f()L7L5{rP_b%eeUUgek5o z_1Eh@xR>emcOJb}7axhF>P`O^8%#HlM7AFn)^7E^2tcR_lTo%efHQ=RSVW&Yq&S_PfE^jHeba+h~)2^X-6p`I?y z;f$~zB{?(DeW%Ut6)QMf^**dazd^II=(<=dEYX4OkWLS`?^{Z!fT}><;ir9OqHn?h z(x{S8jW2>Hh21SQt?LeGzf^|Ee`%v{XSkxLk`ip#*8CMiPa=4vIoj*6GWfT0<^4pO zM)kxj1@rI4w&{P8#vi!t3Ye|ma6ngnPBvD0&%gxEgz6Y^w3b?ulP{k#Z=>FTG6bLI zDu^?_vS~Y2a6{8KI|V?)8{zWcEOwV>R2p@} zcQkdP%8y~W+5hm-y7_}cK|^OCbMbhl>UR5nNly<;4u$ow`$;rTDhABvmV18FClaRv z)a+#4WlXAi-r}ZU?yU1hTj%_?PHGN!foj@!*e~rz^7Jemi*C=APiZrrb|+ zki*x?OmL!gtw2j+QkR9stbinbIk{pV1y-iswtRh;@C-fjeU?X)g2m|`-cT08Y3w!L zumoQ?TKgNsjQ96vVH|sh+OaqroN1O7dT-JMs`b?s1(|+lo*aH00hLccL+?xg`_je@*3O7Smbusu_5EenZPS8rNKz9f%tlh=@R`b9lJ^#`R1i z8C{yZwvS{6R=s@6=FFvbN{icrdFDH|HXWX*!iaZ}Kr7mFUe)Iy&D9;WkIC!<4KV27 zd6~t`>@ZyKEZ`PPdG}hp0HiH>PCRf6hlTPV3+f05*S(AbA4FxC!SzjNPZ)R4ZStOy zY7LJinPmpIc_H1vPGj&wSm6)KanN~GAnf&?1!DjM*goWuz0!3a0$1Z_*_fB-ul)3p zPwiF>KTo-ks&7a`y(rk)n0gSjM1k3Et^R7m*eU3D^-fY8m`Yv2e5p3(dN8%c$2CDl z?sk$$t$;az~)lT+uTz{eN%cH-~Gumu^buPEqDzIUwn4@vy zZH=fd9ZMB|XY45;($V^Z^Fq5rB5O=*bzfJM6sP0On4>;T_9Hc#Se%s#DPK5=XB7BX z3iAae>D8gbF9JER(8fMCx_oqLsdXXiF;({AN1jW@oU8Hb--kzjKWE}G6Sd{}tr{-g z3)}H-2vyuly4Q)lC|H#?KlJ%5qtns-O%$sMn~U?`aYZ~x6d#As-q zQ5hn%+HbVyz1&{k^vs#>;F2^NKJ>6hi{kj{r+Wt#tIcb4n>&oo;Mag?sDw;@*OF{W z-|sgu_S)6Nu{B{R(7rdPgYJft9Z3*9L>qKRl1+zm7!YWaWaY$7SJzn;8~1^3XNjSj zqiE^G_xQQb#oeS)vjb0^Ohmhb{G%*;cjDe~E`AI#XD0tZzP$wTr3wItgC3>=VkAa5 z3^E&ZYBuHWmR0J@sV6P=zSTYr-S0N8%DUl~V`jhFsL5zL*`M7}Pl8X@>TIUn zPe%go-ck<;#p5rR>z!HzEphC8acYn>H&Dv_OI6^HZjzDs$h%arc>j=4Z89297U=g# zLLA-JQ+_&41r)7;$FyPEL%EM*~V@qTj;|m4+2~Bv z+vVIyfez*k>wb^==zRe%e4!{XGV5#qy!|a+c^H)Q1`!&GxnA`Fk(qxdaPnay&zejF z8wB5VwOndJ%x#f&ia&{V`y+udG}{_X@X$ly}N#;eHJXIBW0mZkR`S<{DMX$;(9-=$@4^MLRql>!y+*HI?^!B7 zLArJAk1gFQZbK1`BSX>tdLzU9Guzf!vK0~enT41W58?LlIMeg5aw^)#H^pza^g6{+}g2ik5wkDL|EsG`lt-Cd_cb#Lj6%|ni| zg9`Sib@hxr4=(9_tZ z%Ewyh{pDHNlRj(<8S&wK=qAX2*F+p~PYG(Tj*Vpuznw?+D&Co}VF>8O*<<~jUcHY_ zdH2fx?OnA~94~PWPy5W^{<=-2TIC)O4&So&a4WJ-5b8g{+}Y85SsJaG4SD=a*R#aY3m@No-xMRG zc{;bHsuj#A8oc-{#_2qMIFTx$9}bKl8OK&x_Ay1x?6JRm>FQSyoVPi2uizxYvuq`- zJHH(I(_M>9NyRK7Jj?S)n0C2=C!2?!aXz6{_T{J6%#)=`!@BJ3E^;Cg$o&>}Zks{P z=CsTm!Cpxhs{^jDs6I?GdB|c8q)Q6&vt!=%F~;ekQYv^H1seSu^w5(vbpGypu3l|= zF(s;;i1Lt3HLyUFL%#m~%kE3`EJ^wtoir)oh0e}!-N?pqWT)&li7Hf?WIJ=i_jy1| zIKW&dCp zZcR4pi^|_+^g!iXvp(MA`TxTE9DKwtd(G^$Sn2Cgn<0|OmQcuiUtGX2E7wl&;*Xc} z!R?kPY~UN`aL-rNHL?Ne`h@tvdVb|^T((A~D+Vdpl-MGl`5`y08?&}FGm}X3v>v+) zI685Cd=@|X;_TNQ5V`(lmltGF+A(iP5(ZcsBa6GQr9*xD6pxXwe;M$oHRI$xbY z@}6}`<6WU%Um#O{Oc#sZns$c=EZ_y7rB2&$6>DKadfcgi$7>N_G6O4w5}8iqXWHwH zH9;D!)pNNnFQM?&Gy+X2aGW*_NNIy647l+40C&jrV%}T6S z;XZY>o3)>}|HfnSrT!QNS@;)O1xE}VF>Esa13ddV=cP!AkqIhoCopCmKh7k_7s-pM#3lnF1u>QYB< zB8O*x4EmcmSvCfW3srlfd5@OqjseTrwjbEICrb=&nZCe+*q1Gdf4UQ|~m-iVi1&x)K zwFxI(qqS})!CxuB_al1W5eVGzTgz2FIgV~IhBP|U!v2wUDU~zye zOT7#Xo#o!g?0+}n%4{&Nv8;KYG2e$%6w#NYG;&e9Yq{3wc7$x3@DljrMQ9(Ilq0*0 zEKU4^e)%^pR%CuKCC?u}#)VQUq25tqx!D$ z<*}B#3IuYUtDTN`n(#u0O}hGi5(QcDBtlUxu58IzCk?RCH&=aSpUD#yCEG^wAqY?l>6RdY0OP+%&96^$ltTQt zWI2OM$yyzAz(Y=#2G4_tjO^Yr-@Hu6zjoAj(tBC{>7{UGh&ww+Lx=Bv+j0ZF^v`dO z+}7BNK3j!{qu*QE7q_+X(n)wS2tMH{>3g3%u}*2=(} z{gBYH6n>Of|4{cie9b;0=3HNx@Y<1LBJ%#}5-zO#pJX0qetph4A(7N1N&J zcU(kfRJ0`m3wY&oc{ssK!pQ>QT@M=bJj*8Ip)O-g5wr19nnB!w3K;bFrQ7CMEVr3c z-=D|a)CUNl2G(+FJSMr*`7tm^YBE1q=v-mg6mgl(s8IzJEd_)dCK3>FDr#mS6aZs! z@iPKqqzU=(MgJE52F#N_?pD%0dEUW={Ps=`RHzpgoQN`y$p1GCcK}0P4A;ZO=eK+k zp7T@ZLujaSEneotMYn{u zQQzlq1=f5YXKY;9=|}1JCDvz)<%r)xkdb7V&)^ID(Voe}Vn9#e`FioWb(VuslTn5i zN!NhuEdomtx&LQaEtKj0_5Az3?>96W?e4*ZEomH`c`u#}!g84nMre_HbLHpyVrozK zgp|sUFs>@RjB(pG_jR9l!IHmx8$R}Gr#UG&^ea{ul19`Qzoo=hzqU0Wg8Njl|8zl} z05h;ocF}8)(d9251!8YpdftRwp_X$#+dU-L=ZX{IpC*;oFxoB1a=&CWXMN<&AmE`b zNq9sH5pst#O@NdR*5vnhH}U*eT6ma>sW3}RZ65;6mh6GjHHUg7m?nIx)LBcN8}wy? zj7!p7K*#SGTYP-nQC+T^yB2x1c3wQsF$897d5a~;kgZu=Q~uJSo&8L+{rj{${z}Ja zle5Y1KhEP=P&%zC+^$7^$U0i4Y#L%yRWz7FSL;V%smNh$scOM1d+{^rG!cEuZI*bS zCBmoYR;(==jxX~B4v?$!8nxrLRy{w4t`lNA|M`*hCST!?IPKk)$@44OEN;uBR@QYS zBmb;l|J&OtRC%MRWfQxuq4jDLt*5+($lJk_$QvVuCQjJQF4%MY9Sae6Lp zbAdbqb&w0+lqT3zd^J`XW`*N5HG`m%JL3^O)R*oOh_0+1RpcIU8R*MD-gE_y)s+51 zmxp{}E!%PUbxr;W4=vWid6HZ-wn%a(e??L1GEn^;?_+edgo94cxWzA|g~J36S^3a0 z!nSZVIKrP~>coJqR-Z4oIH48;);>irOCwJp(sCoVn^3g?;s&F-Z6Ti}?L|Ejz7J1j zI_hKMD&;eE3GV(DFmm`JPcfugWG?P)x5Is+j>dB0_?H<~Q!CQ%I1*J2A@+6csk z$pdOcFH_i8D)|s9Vj}m=Hp5rrGdoSXoP7=p5!25&e|au^A$;r;f{%pz6XM&NVOfYD ziTA_dM?j=P?A|}k^A8>;4-rs3z5OSV$23@Y*so2_Lv7tJT7p4gHUtwDt=Dh*FOoz7 z79C;u&zSpOcSP#rt=9MV4JYc8ABOc$iM!!KhC-7W5GijN)D0tw)Z1Bwy+$f&oH`Si z$_mZY;MGO5y!Wlw!;QmjC&8P*-Ax>9aP&ePCmsul62>6~nNG^jFK2kC6K&g{c)yo1 zm4ALo`Gp>I+>t-h)){J2X1}spa@g-jvI*}( zy`#Kfnq$frBS4KXKBLbnfULh((rt=m=;>&RmsuMIuy`!KyDxuA$ndHK-t;S-5^{bB z_VCL$Cp+_d(8?v!=S&{*!blVtoCe)V{kN>fxi1E;t09v|xMn)1g-E5Wce-+>*e+El z4E8eD=qJfofw%Xz!Yp0dn2g4T5f$eAUFy$u`cJQhF#B1DW1six2~VWwa1X1+g;9OT zsMixbJ==OEo!urVLO9LOZ`Jg{QS&2=g%jDFs_P_j1g>~}e9@>QV(XzkGydavy6Uy| zU`oTkq6Ld=6@9Y#vKJYaty`8$-lsslWg62Xs1YnjjE=hI2X$3Ee)JsbwAxgWLp^cQ z$>npP8 z)ASnX;0`Hw#dTDNVS6?|30sj${Y&ebgV0 zg3RY#I0TW~{~{#- zJ9v~S*GDvzu&r}tenjA^{1zuNV+39A8r0jY*dRe%w<}~lVi7Whx^aHLtaXyfzMNet zuG=;WWOi1+lUIH1PJ?xWT#D?zuetV~!6_yRx*H*8v~F6gaFy$6k{*fUe zU{w(B?R_W7VGHZ_3&azPbN3@`Inw|BH8!Rts{0FcBb#cQ1!oqg_{}r-d2p+-fzy2l zZU|kQAujLqV)0HW0qyD?eYX6)VUTbxZcPo8` zGI1u}gC>LSErKxdD2C*6=z6u^N(&NvtzSu0@enZ7iLd~j(;qF1K3t<;%ocy0MvgNE z5al+%Tr82(5CYxm-OO9uVS)XTHosn*@=FzR%so{NdW&m`O&ScXse*Py*Ty%;We*Xp zLM#TUy?IC#ZpI#%fp^wf-;ySh^Ffc$A`tS_$>;+h=NjoFFuBU+AhvmccYxYDVQ6_S zS}l+#sYTEa2R#PJg^M(;Ka-RB?>`Ms8iq&Sz$N@ukF9mQXTRB>J2FlvLG}Kc)eLC~1$bw9q$G_c#-`KXJL)(pv5t-| z>{-~iC_;40U#4Sm9k`_3vq~hJ1ziu4Q{iZgSv}#Gl`Dd1UrBnk_4nUjsaKmjpkML} z839sl+)zrWRLSDhj|kkN)_xq0{Ak_JojyXno#LVNO5z?tY5L7B{wM^TNe;2FtjmbI5$zqRDVTdZPV}C1xF%+O^509HCyl2f%*O3pg4TWR}^koP}| z@D$g<;BYw3V_{kuY(ObGzglTa!(^Jza_AQaOEX5-W=U_DYxgt9tpX2(Lb<*e$5Yv* zUz9Ct-fQAtP3+d^FokLjiL)Fs9Q}7Bo}4>@5Ao)E8lCLc)&AeyJgK z#QEk0-qxV<__-AwP!DHX#Q>GJkw9MHVI|`a6uIOVVPrP@aMV_+o?P4L_gm&()9DWQ?I?RDj2;z|i(EKpd!$k#*YESDrHwOzxZbfT*_8vQo`?yIO z=ZT)STci!xyh}MvVaAXozI5YCRoM@xgRid}`lb-WJ{V1*u>868YGf^^fCw1SM8vUA z(%x~7qgj`#grNbte>C_7`;UuF`bce_U6h=H&d2YntOegV`@5l(+mDyc93PV(zh6a= zjbuUFog|N~wSJn}1*AlK_G)1C9IqBxm69s|y>uqA)O1~FP5V9sf;+;am?^huwy#Mi z!qXsBYLlZ8d7c6|!L0+nV<=g>PRqB!TL1X%2s&C4Q!3FF)%SiD;{f{H@;M26lhH?9 ziAk>J&@;=f_QB^dG-ZK(!{>*EnEt7zgUH(pxD7s$mIU>>mAScpj+R! zAg8pj{6bW;co^AG&@Mh&c9WXhPCIASJD+5p6_<&6odk627n*4dE;=jj;c=LbTCqrbQ?<^|wTJ_nm+YVA_ z9Hex(7CVCvj9XS)Qs-|1yYmD3;2_WLe=gU0`3;{dQt(gRUz`B*?lP-s5!s-)o9|?C zfscVJAuI|>Y40a!X!pXl&T{?yT5N!HhA!`}RX1Pb8T3qw=}=_<@}n*db~Ui9aV(Wc z*Ez!+K56eLel#mIa7mlxR8rMx(4+L1x;I^w_S^9j_vlJUvR5^Se3{4z=4|giahJXH zW|L_k1*Z;{Rbm8h8Zyb^ZIJI<2@$4$j*^q(xR z{_ej#d})W7H}pRi;4QVegzSIm@1rlwjINC#FV9U?d?fZz;Yr_kG>0J%P1ceZhH6-5 zlBQx&4x4t28-`dVMc~rv%M*QW&XQDi=4X{XH+yT#jnx_YI%1ePMVtSIFoD5nr`>+9 zk~P`%ynO{tfd<0e$<~I2dXtFrA54;~S3rt#18Qk;-$t5sY_klX4l6xU2g z-AV<1>BjqBxQ~5@8#Mm-*F8 z4MtUO#*Y;dsTiP6P{=i{_q5CKQ9D@EeGl8ixa&wemD_h+JfljF0V-<4d7Gdht_qxo zini^jst-8MUk<*o;AD-f7NemNY1IC`uh8lWi7VaywX2k7ODYwfl?B@Ozsco~Q$4Et z_H96`wkM{G5l4}OgKb{|8BD%&Igonp31s261;@F zz9;|*6fXz&qHNE$&s?k=HaOlQ@SYH`FK(I`18IRK<{W-5JlU=mbhH%L!5wn3xKmKq zJwppG{-ZyL(LQ*l!H9)eq3}jA_faLrQ0V>^>1Xu-NxxFe+f)N2a@v`E@yn;{qoYWo z02WY1f9w=`Qv^wydj#a~#sILmTU))AXIja9CY~L&^>k0J#jfn`bb~o%qu*@EY9Q?l z@#V|N8I4KKgm!)2;Di|t7Pjy>dZ0FT{YSu51fy`SxkTIuMR5dr_8&?znPMYe>uB|m zuorvPwzr>z$yMQ=lA@M{#{2q?(4cv zyl|d(m5IXOGo)u?39_|!U$ns)<^J zH4kA|WO~|Keekt(09fYG)g1C4@nB(!s?Nu*REi|*X#wd%15Kvg7Sv|8DR6M0O+`%e zMSr1Im1V1aVZpImM^YBI@;6^vvH*IW*^;<2YYxwJDOarfIo`(E?Z!r*Mt>m5UHZLD zhT%7n#8Eg}SB9SKpI|P#)q7vhNXo~HnG+AEZk>?)GLE5%%>j0QWu@=AZtqM`k~5=0 zQmDD@-LVzb0T@s*j4g(hkU&Ik#`W^qPO|Dyw>N;tluVLb#%}m2(pK%+TSdM$>q!*9 z?4`{D)sJ3uCY_t*;^`F4D%8PGN=S{unw_=Dh2U7KI#B{!nQSu1wKX{M^=Z6A@MXL? zEY;#AE?m$?RG19bcp+}|A{ipIpzrH!z>Z1B*(8W~7&L5#Ay$g>al9q|-z|LiIWd7^ zKcHt~^tZerZTf`w?s#Av7FVPKn7vSejzPGsBlxZYQQyk`YwM*rPzQ_K>f)_}a`?R7 z2nGudpWB9?X0`hd1yefB{>MVxtUV0)L;(~GiD(~2{d`A2F2Y-jowlc?&*e6OXd?Nx ztn!nlYv*jyiJ^IYd^T2KMM=Rj&sK6keFF`ITr@O5au;K|yTVvXxY~ZIHwq;m zQjfyHVxf&71Cg=>)&?E%IVgoM)!6}kj=a8f8a2pKKg&}XYe*DPRPvk3++gxZVnrZE z4oP7n^vOPqo+`aW)nEpv>!EWX+P{uwawW&@CxB64Dy58uRspeUz??6kxwpB-?)J9O z*I#cJg|s3D*Hx-OA@zm>(hR<}KeRC17s@bymEk+Ojn8wi0yC=fORRn|m9}a%P9ca> zNFi2jfw+e`g3r*9^KY2Gp7^2WTEFrbP9?jlUIOBOujaoc*^ z6C8}XqPY4@z3l1Db!TO(Y!0y??~=Yko252*PKgz19LcQ23Ow;iYVP;iYs#tRbvkM5 zFKnuz-m1$NyWZJXD>)bMg60x}L9k`_Z{yR3k`9?#n-xtZ6TGA>lB~Z+{Qj$89hGH; zQ((=M9ei^5$mF(m_;=YhS`r!LlfmFG>{C6M2S?uqa@Ji~KPG=()FFI0VE7aiQ~&}j zZ=NTrhZlm?92w_l4FwajFB`PKp7DB6=bNb?9)5IF?72y*`&{~Padp$JJ?eM$%U=2j zgl;|j+;KAgKFrGuNDJrh-ssf^a@FMWFjT|eGyOYK*8e~SiLyUEp4-aTq_Lmk|GDTq zOe!pY2+g0A;`NeZ;0kHlg*JcV6B%-pNpfd5^*luIDL{C+xj%4`=n@|YORj<_i}xg9 zswU_imLX;F-)_!mjvD_s?3H7|&99Oy~WOe+PCU#-!ZmA5ZVm7o0lu z52flN6Wo|dhP){6xK15R}O32E$@1WS#!uYD&Qz11`ti4U)WAeKUmR2AxNuO&56_Osrv zAvXpyT)xg`h?$MA-(6Xtc7I0M*|fY6B}b^s0nxr6K$zutvY`Miq=SK;QKhjmjf@9j z=ws=|GY=3=SpwIwxd%8gCyRY3Avov@HvMkg1k2{Ws+K!)IpfV17;n;vJPA?`@oOOM z=VJuI_WzKwlHUp!L;NWRmqi~aW*N8P|2tkOSV_~NBw&9e*W(rKtdb4Ik2X>GMnV!H zk4X(~Q&;h6K9Ts!1`v_V*w{e8ABWN+qe@`DL|p6uC@X;W1q-e85*^^1oV}rP6L_T- zt=C#Epi_e6f?t0N+3KRXhsNhVO^8|*G#a>^E0ab$Wx)c=@+h{5@kPV=(h&~6NOgF< z#q(EaHnJ77XWxPfuQz}qOmhjl|9C}(ow<|E5vX)OaX=&#tmP0y8`z@*0^rf%QqZ0F z>t#~r(j)_o78lB63W4(HG2;>Dof&}eYqUP^lvYq!#JUy&s;bYEEXu5*1>akj(l}3b zf#S!9{)t1@HA!AF@0)UrKdrb5|7*%`c5(#*Koq5u~iZpgle#3UJO|99me^i|7H;%K(rvB&~#-l}@* zAEVvF;p`n={sEsW2|j9ag?b?YvGV&f@}z~0%gYk>O&fq5u&Fl^YMogq{~3wG2^41a zB@+N70G@9#m1mredJ+`-{Uu$Br5Pt`|Hf*cf9`fQjm~O^$RYAD((*9wF)PK>N=fZ| zj0AZJEA3ch{WGr*cD1+v#B0FD+`Zw-Hen14`N@unkAZ@X8<$%=2|EcZ%X58kB+P19 z7gy4XZ_-Yj*9VcFwrkYPG;~Z&aq~D#f+Z(un-b-!yW{!Oe|Hjo>p{<%Ci^|<;KBY@ z_Dek5=07(lP^jRjp@)w!d22x3!|-9O@7NUw!ew_C@GD_?lRsN%AY!mA)dsR?M=ly5 z0Fhcm3h>xQWOr1D>7tFn!Jhi0#67i*ib?f!Kx1<*2@MR{+$%J-ov-vfp8%#z*I0h) zdhDXBHPe2aAF<*Isj8}Y>}vP8$}~Zv@<5H)8=pkOw_8#f8ufX!4`}=HcgRWe;-kJS3Xh{o9yK<;+lzc0{)*auEwuVC zY<|NT0;|~y-Xy^oreeEzSmb;}_ER6yjAJ%sm6CP3UO`8dq-sksmV+pu2MMob&LYO! z;&bC~H2B~HL8ATW0AxLPH809;^HeC!ZqTFqTDJWi2Obj)8d`r9T=mLydS?zz7>VDi z&L{E{!`pU(KrZ$b`tQJKzhMDdCGGj#`N`m zg7iQ1do$?8q!5MTYk6Sl!3jW%=V)tvkyXDsH=}BhWU7b+yWEhH&e~{Wlahz-zfJY~ z*>c-vJ^KGVn6*$a5Rt0;|C;FMp5>d>;OGI7Rp?wBcH>vl!y!L!m&>M}4&XSph-RDtv;9U6$jt z-CFQFiqQ}J2X{7vfGolCCXBD2P7%%U`_-|R&2bB;I#%ACJe{CeV=-xLx5YR=v!G*z zMv)U+{_?H)?$#xkxW7(j=L0riru_)9ZFy;r%t8#_nu=mw1%(m~f71?QqNRg@R-FBV z<*@c8!;rBeG%~Ab>O2UL-IuGK|Ls2u`zGVEtl9OIA2m=|3T=UShG^L8FwC`n7<9`J zS;H1TB^q&v03Vn2v>C{@(}zs z)p&esG6HmZpKi&E_(rIXIU&91KOld_xEwY2FIr?FcLUoFPTK*!=f$z%CrA{NGZzLm zM@*PB&fDte02g)&ei-{(I33gLQd)?3JD<*;sJ<=Qh0GN5c(hg z;eOMBi*-h5MYY+?i)xa!7P!je)z%Rw1$=0(93HX79xI4$rfA34C5G^Glm-C36D!10 z5w)&vH~^Bo>~}iYC~IpL-EsQ`fiQE4a{2tJdMe@Rq75lSfd9bcb1Oa_#|Q7Kwra+S zd*{q;TuF>}<7DLDumct94;qFKuZD53WMPPbvq*Q$EypOqTqv~!mtFXy#XDF+NSw>0 zqw(O0P4}tL>cF9fh&Y?)`1{1PE?DWlY6p0f`y*l$IHScJ1fEjuYukyE2fSq5UOWsa zrjrkH8A-7YQUe4qqaXli$}6z~8WZEha66jdFPb>X>bOxf&>A=i+?l3>P(Pc zXbeFtpC%>NWV-&j% z{Oam8Iv{B?u$KFO=sch`_AQ4gF8%>)DN$E;Nw&$(HZg@j#npEXi4qcy82=Mfj6$sL zR)`dWHDYscDuuO{3`eY1jm5ZJ&%U)g*a>jf#Oj0dVvn>##1J&(X{vtiO0eab(adip z^Y8G%chCpm(}X8gLK7lUX#I0zKJ+G9b81mRp^FCOQ3UY7`BWv3I)@ZxVGQu03E{(6 z3*7$A^F6Q@h{A@TsmQCMPvhkpBh#);+4JDE>4RNLG=WGU51n{r1a%pSZk4

    dxN`<`~w-{pwBoQ`^o|CJ1mtuj!@rT(3#9a6qkUiw*B_M?~_@d zj0)W>AD~SOT$t_B2-Af}5Htrf2l*wWRHn6%pvA@P-)GGzFUn~iIjEQKpm&{2h8#Cr zK#tk@dvp}jmfJAFym3o=aLzF-KZuZ8Nm0wl_5le(8%0h4a}Pi{MI|bTpv>#jlMIrQ zB=j56KP|x%Q$C`j4XDOYXBoD?o=UOCFL-apmN-a59{eDqz3l5fMMMhpT5Flk0gPXD z&?Z4+%}PpB2vAz0!Cq@W@~5NWngHzk&CZT4Chk?P6sJ00ePofMo2Px>ZCo+X3&tc#!z+>Ysra za#Z2?CsI|Ebo_+0R+^8g-8a}Z!xc_`>rfVmseq3X7dR} zE?*^*TK0K}D)T)zwpG=}RJo7u-Vgl+_6e-Hg5A0C$-)1tUZ#Q|^)xNfYz_u0r^#6{ zK3zCGy~LlF3mm(9ZQ}lz#Du#u(ET|Zc@gI$uWgxxLLV=iLvlPTJ3;zJ(EAmqh|=c$ zBK=pb4x`YgPD?hlELaL_@5Pk;|EgO9NvK%yg>l5-R($SzO3LwrE6bBD@FMySI z%N$GpVz6H)CIC>wUHQe5_KVI=>6au2DU})EfiR`iYA}i?-S+X}-)?(>E>{!6TH{Z} zGCw&kzfb*kTDjRe^W1%`F#>YZ1RI+L*N&T2j;GoOYeTBe#7w3^+fr##L8|ruNOIQe zL$r9jR||6obR}UbmEdgq^ICc#qq`HXw!@DYle--FVvF&)TBcopYq?U}>m8 zG@(>keHrTrle#qkJ14tvgF8kpH+?Ow;e(9EO9O|aEgps6Rrt%iR?!p0RfC@P?vEM@ z+{xeLHQ4(XmKPi*D7vV~fWZb(F8~4(;TE=B+=k4GNV2x>fM20$MZ_GTu>C!2a;9}L5iy4Fu zE%HW+T5^Vgswq6-*DbD`uzJV7PBY9c)j7-cKiH;|@H=B*--{1UzdM|3?mzZZT+F(C z{skeHYS?R!8)v2-c#$oGu zQg|kA?uX3W5X1>r&w@wNo{j%KgmM`ty`My$(Ed=uB|tg@NxB4X-?jJ${h3Uw%RgdC zAX2RU=6_$5YEw%$4e|HOJq#oY<2>D6105-M%mBr1F?MV=-wFm4wJjyFUIX1i|s&k%|6w z;dtk;_iaWm*&|cLZ?MT#?925Y1MdVj`;ox^dMA_4@qPqEEq48GvteUm@8Jgnt9h=| z51b9(CER-5u6o_FQ*R&rFq7jv&(;$5_1`hN~0D7hhK|#F7$oU2Hs5=#q+UIzvxL zfNITszs}wKceRUCml@_nt5}o=^B><%*mejPoc9pSt|x)X(s!POu#&KV{v>Pn#ik`E zkJC5TT?P$0fb;N?Ane@>mXVukrDS)!(@=KHXQ>xi~msh-aixp+!-_6IPJHSS<{(3O(?rc!1d zpSEF7IHW{z!bGd@bL78O{Q)|Qygk3hR*{0Gfw{?u^;*)-UM}y;AsaMge-@t1{Cm+5 z8SyytXQ>Rs^`~PjcK2e0io|lyQ$$+%aSr!Lxk_XyKAvN@#$c}BzSZilv@%O?9%EPx zW9N{stG_bJ4nMv3N#_hUnQi6z-mHKw@^uB?MCR*z3oQQ!BM|l{oNNO_>*N<3NMOXN z)n|OHn(5>!rmq8h#asx!KQ}So+Id&MmgE}Wo&C0~s`2CBWx}nL6;gA=_vq+^0{oj>L|-s&K(O<&%nVds)PHB-h*yPDr3 zm&a~&`a(|FF?&O-bgtmGwBtb=>@&)|=^5A_ds<7SsW&6qBkr?P zD>LLq&9Aye%KtxH5o3r(P}?8jP+%mDp-2+arzs;?`F9O_1!~0}=F(K|M*2`e zxJrs@&#^fWq}dRyi5xnaVI@?q2#l}^!M-3SAkbq!BWFKMSD6o#C2w4#B4MGOY3^z|$O6zLy!9bk~z)P2xZJ zeHxL8fVTBd4+e4{s?%(ao#u<;0+pGG>i^RM^zh{|$nC>8K{!#;!f6@9&_%*K_k2UC zaL_rD6j%QyK)WMYe|Wzcdav0=ry9i}6}b%vWmrGcQfV?&5cX2P9vK{9Vm~B`1-*8% z+PRYo(*zD(Xeao07(9VnFEO2a{IC#KE#IN z7wC(OJ{*j|W0@pA1(F-bI`Ug2JH+ldy=X5r5a)Im*CWgAe$^Jq3nN~CFJwhu*Rw08 zAGI+iwO&s{gCI@r%yA-6!LpRQp}vUYQ1z}A2(&&@kxI9qw6t7{vXsfFv zbftx(-#>0Ln|$#ZQ8=S16zY=K>C)qsVwfNKI9E?g@$IpzgD2B2N}r)f_7wRw@08XA zRV2#29W!zOY}-`=D|FtA`6a&1A)*;u0P~PdfoyN>8R=BSfFFXvoJsQo!loo5u>xqL z%j(Oconir#Yb#k_%JnWg+;GpkkFG*VDwyln>tfMuf7M~PRX)*2XE!k8?=KC8;C{FL zpFy7b3FE-w3iat3O3twmJA_WXE2`JEV z3zgwq|BWyYQIQo23>Eixb&F6*Oc3-(T=dVX1xHgD6C=^&kpnoXx?+EpEv;T<8X#-_ zYQ@=<1%&TOzb!wEF_ z;#__6DHH|JI>Tmp54bpO1)Egz9{**D?zdTNt;%F$YrC4n=p+LI^k@Kg=tnaB!Ak;s z&ao4I79Rx|8DnckVXy-;0OU3v#+1UIHGwn_W~}ag$;jnTt#4jRXILjH&ao? zEeqbXpBwO%9X5QaKKL(q>#}x5n=S;2e20JfjmaJx?)E9CO)M`?OR^W_NGR2ULP^~k z0fSTwpc!1=xcqX_j3PHO8$0AlLmAJa{iPb3wc3@DH%@3uGGYE{JRDDmf%LLEc1TDB zf5(630xU9{7*%+@!&}<#E)v}SW$ZY{K!9M z{YV7jAANv^1nw3N8zwk|iaZI^@=aMzX6Tx`8KtWW$F3 zPjTSbtlFzA_<;!wHTh4Kx)+=RlqP>i7<%{sx;>2>Dp?&CZ0(0i*A=HZIq{!p=xt31 zq_TP^$Zi2R(CBeIKYV9CIB{Uf47G>ES_wWb_}}VY>#s%kYhH}~Sl#$|9~}LjHUS;i5AOT3uXp2pTc7hD4)oT@5 zx5&_?QPz$i!dDI44lkzwwg})K61$f7%T)wk80ZcHeP#D!cC#ZpZNIX>y7!?s@V->0 z*ME!WsWO(DJqRlUlHjE~Zor!PxE|B4#UoGO$g8PeXw5Qh@Qg{g@ypWso1Z}T0; zpD)6eRy5z)T)>-L6d>f68L|SC_!}(v@c7PA+1_kM#uDwtone5~XqW5#>%Y z{3RQ`<_ogN8d07l!z};!m$eC6yjW5R2EVW4k%Q{<#ov?a^8S9{JJAx{)kb?Un)j3R%vhYFn8;Y-dnPT%d*#Aq zLW*QEO&lT8;PB7X%^`NVxThNE&$k>R@L~uA2YJZ=h1EwEV4#{Qm4R%o946z<;VwsiCKy3<1RtpU<&6g*ZqQ zekAsiv}pS)w9AfY63n9UZ0uLzip|j3K~&#OoWH#PhN(Bz!Cl*1J6BH7pFT-J;$g%c zBQml6t}+q~$N-C-e?%B%8AA~f3uZ>ge|#2`z^bTIQ_bOMDy-hh;}?TEjm%wWxr>B=DfSL8Qi?FQeVv6c-l;jgb}AsJ>v6My>Es zHrq@ixv#tf$MN+Q1x57YHOZ|?jx5o_j}JvMRhJPNU|E8Rxu;_ms|SO~)i6Em=Xf@5*l0SWrnmT?=zEi#{X8D3T2l zi)WKSv|)hD=hgE!Nd3R@ry*MQZ>j{I?Hn{ogl}{Cgr@9_iqhmjPlz~aXt>t&&8MN_ zRA(vHlxTytoTK~awa%sasj@Ly;)tWyX#vF7w?rQ4!UJW=^yyFmmh&9uia>1W(#G#W|4W+&fCvnp62YPfV3=2Ki}b7=lsM4c|2^> zg0xqER;p7^tjVIvYytx7&kb#`KiPK1@Kd&bOZmHxHC@h=FhCd3fuPB4DUsL{{MIpM zoB&xgzQ@Kc+hS`8v4bK3*2KTAxadEow@md@&i6~9vlnKKoY8*Ssz-7@vaMYJ5;0J< z3vC}I1VU(A8@81nI}uOXNJ0Y_)BAUKaolge*7*vTW_Q!Nfp>^Gw}^*x;_55<;!GtF zgDC0feyYJHL&)zrmo!6APytZ#sh1GZnS;RdlzchFN4Bdkm18lO8ab3Ug)B4KKMnVd z*UoiEWGf5{phwhwTlL^B(>|^(OH*$Vze1_G`EhyYXWwE?5Ol8MDHIA9(X8`}KY2Lm z&(yh^kLTpxwzeIuWd1Ps=frzb9LmE)D!P>|C5r2an7m|%8+;U;g!HUiEc3(+PU92< zyefHL{#8mLYA9kyyDi?1yoBk?`&{(w-G)M^A330UB8X+_;0A(XHt7-9ngO?=x0%n9KD#|M! z171!%zt;4Qu^SFx0X%gCz?W)c$c3>gVk?rC2iaR&k2V6Lv0tj9V*$7gFd`97`>;zu z1zl_Mv(A9~Dplkj(^jogi!J(GHWst(Gol1tNCgISy4hDTK*|m@M!RUFI5ME&gp2$t zrqA7lDzM7qlj7fS(khT*R3%zXJhD{=46R5ETGP%pI>>~>W2o#eE`aQDTY`-bo0~~pO4oLJ>=Do_xHZA z+(E#K-B#k!Sz`ZiFo^i>=|CoM_er>=C7uB%o6*zUnA9aSK)JdaaB^S8>?9F!o^hKL z^s^bGTo`cm5cGF?D}tCX{@%Nsc+{EAUB(n*dtSMgFyjoOi)Vkk<{0nleyeyh4|rf6 zh$I+pGV4{wdd;o@mPyPpz8}*GhXTdZCyIacgD?BXTHK?LOsu~D>2A9}u6Z3fOnTg= zZ%2B!Brvn+ty|4Puz-^`mS)G{=$Icnfv2U@!bITXDSI8e4JsMkZqm4&K|eZx->Xs+ z0kn6EWtz1{1VEo7%@3%a7y8N0NsNZ^o{e#$`;WYyo+dsf*E5G?ewsb_efVsA!en)> z5o!P?zoq9GSjbHLAeP%rgF%eI6Lppu`5`J7Z>B{op8<*Z2kQ6o`zCsz;uM})60PJi zmhp8I0~R?7WCo@g7O~DQ^O}%QrxI)HT%x?0`9PHCN#_FDxex2l=s&>E=Y|RNvg-5d zs_+)piB^lw1Je0;q=?4WNMS+99&Qxhd#HX(MXA5N=f$tsO@Kk8x^e*{^1^;+(l z%@gW(6d*J8IDxxf-poZehRr@`$N%gcS>qb$I#VVpG|{o3qt|5&q_XF?xCSsj)5z*wiyY@Ke0zyRc0yU(u^ZEMHMjy@z5uykuNSwT5NM}O^8^6 zq6>Tn$M~1II)Y4UL%t+}Os`XYck+62Hd{nMWuy3S#u5C&vx?;0O!$VhES2ol%SonZ zfzF`ID6(XK(+Ho&0iz=`^hUEgGEpP{`{TMF*JN2((nRz8G>j-CM)5E@ho)^M)zw<; zAm91er14zkrbE{wl)c8^v<9D*2dn8(LsdC5Y?e`(VtyBY&GOe&Z`+Kpw!H?E#tiQv zZS=$a`o|!;e24F|QKNKG79Z?`%5Lu09dgy;{T0zBD3M1{dfgGZXU{Atj7WWOJ1rUv zR4Q1S6S|Y>l!f7?AJ52S!!gc;Ic6jq&Gq}prV>x7DwxOt3my-(3n-Vs=MUr>#CBOm zH#e$9#h(2984`lo{EJ+O76Ney`@7`{DbKNRAL8D!d4!k&Gqe>?RWLi;3ftvrAJnSL zz{L~JlEG7|@v&)5OujG=%ZuaE$<~|YLbnGg38f57PL{)9J`e9D`*gRdgx&SEU$Z^V zi~&i1k4IJ2W`=G?cn7`f0;hEMytXOMmCuJSQ-7l~Y=j!s9i4gp`}%VUR_w_eu)A@w$%RbKy^CH(j2P!0DjH@{}MgSNZT5lDjFMQ+qLMg<%?aFWO zY}CIoKZjoK%JQ^fXTPP?7@7OzNr)VJZzf+7Le}L%1W~KzY~-l18d|4+!};NgzY+tt zO)6~NF5Cgul4iu2-)|IVgP`e8lk1)}+o|96F)z=lIAM}iF`N>ntUYbJ+}RZqoV1#1 z^V&L1TARG>0_k?oQojJ=*aUt0Vmizv4R@@SX}Y@JuN;b=KSL^uu>!dyKSz?cD=lyy zo?z_T5@`Zda0`oPW*|awEjG*|q8K$`3)6|iF8(2vQIJqHoal{r;D{p5Db&kx*sUXpOKMcgS@jg$s@^A0!<0Lr&T4D&$Sp@(>zW!99NqZhFm!@Ee-^F`w7M8 z>T3`AwSIZGXqWf!+{#+`)f>Vm3yedAskpn9snS(Xl9>1*$7bWM3Ix6SpJ2=3VIzH# zjZC!a8>yam;V^X_t^7hCf*u>&gGH|3qnz`HsMxNb9_vD`yj)*ANhabe!YK@-TuH=< zK`{u|o-LG#2H%*m_7Egf98C7W!UL%dxxy7SbA<9-R0!0}*zX1cg|lG|*95zMETk2W z(s3fwrQ?BWn>ia0(6!jXo6$RRO93N%wn5^I*t|kQ>2Go)A!Da&Y%#b=@yf-w@0dta zt&;)ywi#twc*HY5*7hKp{m~==`ywlW{8A&h>CLweoT24N{OA-YUAh?dO6}rfxZ85t z|G5Tgc#{njl*rhCE^1Cz&?l@AH1RgmgIa^qU}w@RHzJuoa>JA}Ce`g_{sdU=AQUco}$U z-w84wywmlwd-aCznq{Ee#Gqo;Fr#ABFsH6+1nDdU**Lgm;rfxKzGZSz#g)q2OZUm* zuj^dh*4dOe6|A@#Z$U8r43QijQmhFiO5w2W|B*mrhedIfhq-A`7!WoNQJ%b5#oN<^ zQ3BP}YwT3m!cK#Q#mi*o$L+bcph#|avB3HcZCV>MUu2SxBV3j(^!^P*8n<@I$Tk~Ddx~3JZ#wZA9>8!1WDaq3g7V57(IK!apr|h0mc+fuwmUxy0>9kA^+~Z; ze}ms$@F88A3M#uk%t8I!sJ|{$^KH{*Qo3S}issV1dWj*D#VTe8ZhndFVF$)`$MzB^ zOsKFcswp&BPBe4A_;Y3s_!sY*Ot~42l79f_+X6{@W}MnF|kwDm&tNu!j*K0)=ti+Qdz?xVg}O_zL4-lvVP3qUpKSE z>trp$KQ9)LRFC$LlkNkY7Q;pj&@ZIh-_PJN1xKe*qnC0yrJfg}DBl}C2bXuLjB|iI zX~T33RvG?`GM}9{jMhCu{^HBQTktt7dyA->-+&)^PzYGMJ&=|5p3>jw(Ec1JNXDlC zMsT`1DxNH)PC;r{?945(<5hAAHThDR&RqxnBl$V>SlT$)JR@r{%1uzjkmcS+c>C?y zX)bK(H+ny(&E0HSWmjLXYu7`d;AGVBcauD;`|h#tR6g+~k2^jOSG`l1{@dr+mTEKG z)J=ng8ggjDAJfBi+7DME{%72$Z~;mg-@)Cs>5sj0>Q(AOudTYgwW^H0TT)Yt`cbN{ z`vMoHof^_Y5p-Q$S_mDj;2%<1Q~AHNE8=r+#oXe~ z!!99!8yASs{dmke)9qHviZz(=!*GFeFW!gT~Q^?kL(e(j&TnaPecSm7k4cuP; zkD$0(;M@Izyxn3}I1awAdl(!WljE)QsDKxLpiM>yXue0URR<2e`yS3-Q^L+`hk zSkJwKde1#`cZBf9L8dp-wW`Ax`m@}>k((D}#EODX96vpmEfo#Rh6@}7^%d$z9$H zfFC|Hg+sWh<%jxY|9C40b9HwZ6Aiq`!{4aIs+UKf_m4367~0pOJE7a?4~?Jv+uhwa z*T{>AP$0N|_ric0;Z+j;_fb0B`E7y^XP7Ik@-Pvn?(tli1{1q>5!Yo8ZZsbv`3MIx z5{d+@I?JOnPFO#YIL*AfH%Q^CS_gnP~ z{mN7e-E{kcf(;=E?@qM2Xae27pR!CxMOsCQ{IKj#K4P5e zQihL z{djsH0a~;UuLHm_F|U0pEHq$U;|gGc#9#FPu`)mZy#g zMrLn!ybNZO4)J)Yrj`10HI)U*;B&BQi~g^IobLg0tHA+_>s+$Zo*yTpdk^%nwR^k(nlpp!vNY-9SVDK<4EJBt<`|kS^@H3ld%$FA zr2J{@ay7lOZ=2XZ5dZ}VvP84E&RwI!PyaeZxd06V%7kO%Y2R)~8mMXC%g9Kp(z{F& zF*s2Y>L>f#>FCc0fjLGzeRz-k3N`}q3LFB0Xvl)Nq+3XcS5J^%8ux4+cxsnU5}z2p zGAE2rfW=>7Ln1HxMr|Y^OK)GnFi5SDdG(rcaPA06XFf!&Y>-{GI6jLF(X66&-l6q7 zgC$*MI>ZVpSr(*^F~S7FzO8(&FE^^^-@Xb>NSxY4Me`a|Hp!w?-~A+R$lv!iAuc1O zYDjk=(a(_$5hF{&Xm3wUO%dufP;hw(1B1Um!3l{@@}R5&`-AzUnW-~mBky!jRT+q- z*!O|s%Ak}Bd4ILL-!gxBXP`MBQ+F!VvtNQ=Zw0eBw*Na5Jqv1TTowlQ1sVa;xu|;3 zaF*T9gkGuN*K|EnDa`vI1l_LOC3yCZZ}VLr&oBs%V>!tfIH1e@V!(+gi%0M^_3suBIlGG}camI;$@Skqg8D z9gaCqxlR>tlTB~1|AgC?<{iO!&r2)5r~nXEX;9qa{K(}+$YOByH4b|McOhz*2IL~@r^(=hn`AodBO7AN?bi3PBZ)QA4{7-T1|7ihk zHk%Zkh+y(@?)wnRlVysSX~mO2V*}@{n-dl}=gm?czlctC$|Wvn3AczUhEVQs&5uB} z2sivDg^kDS*tp#{7D+?if*Zd3f z^ZMBBn{mm7&72(joAikfV!h&qjT01NacSXVdY6yRhP>VLIxz=$C=#p?JBroKWa)D> zjClTpnijp29^!RrTeBH$JmN;4JfJfNZSm!EBnFZSj=?npUz2gqw%#zGf}trfQhv8M z1+6t%vTAt9mt-WaM4WNzkjy>S8$ZX!f}QO~Yg{qe)$*nlkF^1`-}V@;gaSW51X)|b zO`sz1{1NM5nmn_bZ6zp}JM#71%s5`mz5V05c4?Lu^ZV0y{_czbOD-@c*Rki#cK)^3 ziJeK6hv}*J{6K}c?$3Iyrmq?Yr%%!oBky0cKR5F(L%+LbK=Oj~Di@B~8l6;AikQfu zzf4(qv;^m;e7Fm?5AurS%yP;8)Mr)DdK|%8s!d+Zb$jsgGidt~rjC z?XdUvfeOuD03Ez1d=a{klkp6ZjfIWzjOpMuh`UQ{P>slj{stGQzftZ; z@7hZsX;K1holHzm2{iMvTlc(3c*h#&0XzZkth@y3T199$@dJabPJia&uA?HtFDOQu zD|g1tggxI;G#ObGO729*9{q(*qDO@lmWOjvyep0?YFvet8$A4I1h735t}WgO=HJ~o z{DcJMyK&;@cKQ%}L@)f8RoIeZPWhyrq78j+?5ey--$kDUfGp4ltRdNl%hA>8`F7(1 zfa{R?7%dks@n6fd+$k<&M^qX=vz6rGh*Qa4wqJmM#CVecNbi;kfm_z^be!_!mIX_4 zJSfBU#i27kbdg$jWL$ahRkiWv%t8%$3e)3Q%*j`|ULM6Xh1?--0^kmktG*rukwEl# zY_u%r6NoXOc%L_ZPkVd^+8B%KNXdmz8)uw+CG%CVr5*0c9l!t1c(W6ZG@boW0*0rZ z2ReGDqOYk1I*^ZY47Hi711hy>-)HD9$P=(c{2ye?PCvBua*sEoDN?>rVLndJ(-IK) z?@oN`P+<4Gcez=b9rFiXhIwNk(;n7)9N9CZkkw}!iLx|Wg$ee9Lz`zt4SC#S*=X?H zyuf1F%VPsPx)aVY36cEpGwHc|bsW2zZD*I7obP#gd;C}WOm9DzSX4B)a)7nHDjrv4 z8@i-YuIg=2GBnJ{wS5{oX)$2vH$iFcKmz`&(+kc3Xtjsc^%#MlAQ$4DMVW#%P8}e| zl&@LGOpw*TPb|h@yKu62edql!Xq1#Mt;kHXE&}T7m%{G8jB(foYJf@b^m#Q!B_>v`&pu0mnn~#rbF~;d->~yY$RJ-utUW;!pL7~hOID%vC&}JT>ev%;;V1D? z-zDp#{u=Z=^oDTNUnknv*e_-?eTLrGh;P2R-g0yv={)tu?x-)_8>@XlZ2Q;`LRZeV z1%HbD?VJ3(7GPn!>jE5vV}bWqR)J4RV<3P{Jz zuzT2d>uzXuMk_$ReXN`mDy&&FV;11^A(0+sN85n`nvRd>GY{o#j76XBd3yR}1!$3$ zrpTBKFw0YPa8L6+nXw4M6+ke|SCATLX#==pE%0vYqrkW27hmF!;qJ5$5K{1unQBp> zWAkeepD)Wo&!nIoMEH9QFgEWxg-(GsQD=_np0ya~Z%F`|*EWwT!_|)RT1a4^#nGGh z`usi6gZO7{%j@b6zH9FVErq@6a;66m<#YuC8?p~YM5pJ=H0plqr(M*eN3+|Jtv@w2 zp`&{Mw@~I;LG%=zDU@$d)C}mX3YCQ;B;;OaIhAdsmCLIFWDL`vi^uH*EqShdgl<`X zVv!M(8ka5&uy$2AKs~nlevD=8f?pZlo^ySkStu#$R{jrjZygq8xAg&ogp`0Vgn}R< z4Fb|gqm;nFfYRO4ARQxuBHbNBBhua75+hyG-8s}S@ZCPodERr*_rBl%-*s^z;LM(V z@4eRAYpvgJtxa%!0Arle2zyiTY2bmnqBSdW^XY9@gz5P?(@QU%zPIM~Y6Zil_H#m| z;^q1m#}*e6NKy+kUF2rYzQJ?MmYT{y6jukEMnVWCMA`S%urhtyeV`xt35pbTKGGl3 z);c$P&!o+uX^fE*aMYf#Kku(65hWYwh*+4oRm?p6g$^w_+^nvBIVVnu7I-QJv<1@B z&*_i^83OTaq5YGDgqRhwuOj;O!RoBu8$B*^x!@V9H%%Qz4o`{iLC+m1BrKYuIUd)0 z4+XbG{JhPDtaNHWnDfOr+!Gf*eVui>gUngkcBRT>Vy-#I{<^I8a{pCGx;x+R#oBtm z&3KCAvlU?M3XzZZ%$RVFiEh{6W%~kUzUmkzZIvM*e?$ik^CA~uHFJ+oQ!2s^U%_m> z84@VhFJ0~Iq`#DhG+8iX(ePv3tXMC0?SCM<>$DIH1A=nu7=Py{jrH+C@Yb7q{2#=8 zm#IpPIBmL3=8s`X-B4Ns{ZD{EJqR!Sql^6sk#b*TqMya>xZhZ419@9Bhn$xocRLTf z5$)2vB=aIKciaaDqw|cCJIq8vt~{2VtMBeH+S{AZ9c^S|>;=-blvf5;{|Kb0^ywW8 zZn!&AgT0Q#E1LQs7a)=RguGdmh4#V?g}pE%E)j{1F`#XYA}BFhX)GyOii>0LzMh?s z`WgTc41NZTPx=a=$Rna_=&tj?2VT`_XU&v zcf82KJ4LtsQ(>;zl>Ww>Da4*LD;L>z06fTU)^_n$k5N)_Q$bBB-qO-a&Q=T=g>MGs$V{@4{e&A2O z%va({x6sppApD1|l;pkOOd6)kW+Lp;`-92ig8Y;fzvIQ&UvikUi1$_czl0QY2(E)S zG`-akM>7*C%sPyeAJY%V1cnO^ZI4HCgx-qep?E~==p%XojDgdBaqReP|Gj9hARUdG z*3A!l%U>S5@)~a$ zxB%?s;(IZXCi=VNJ-!FfbGnk_rZ1*8N-hXihYTpWn-M1OTXoC)$g`Qi54@1`x7JWz zBMTQcyC8qKE>d%KrX1+x{p2{R|IvHh`v!eGCcf`!@hgJHQ9C9&e6%_eMC@ zz-sIJKprDJmz??QI&YaSNl(KcoQB?Z^__}Z6wxXLYn^U-T{SA~V9)w91GanF;Jgn` z3nmdYAde1Tvlu&s{(=C;?Bc(s}+KW?>)r&8l%;t?{XsspB zT?;y0g_g3{o)ojfYd@fN-_h;{em*#!#h1Qmy*OAN)+H17wjk4=%txzAm*r3>8zVT9 zox3R879b{wEB;84b_=~%uyYK49z`On`q`hWLuS4=zO-~K z&HPsh)3aRQMs^^o)TYKT?Czh5wvuAWut`+;B!)usX^>{-T38 zr6t0))z3Vc+F3Ig(T#h+cY=7AnITXZjxbAU?*F*zz{+6EaLgQfcKzAL$LR?1C$I3; zkYM`E2d%AGh=|NA%VwvaY@h@`_tP8M`%-|j0ny3|x4!yGdRiVSh=Wo0RS3bRbSm!( zAefXGuJ`6K%a*p(jGSv86o7O)Uq53|$jf}fGi>A(o*V7D!Y)nm%AeQovlKJtXLSOQ zch#=+WaaOxP3fJ^!7Sl0qsHH!HECqz2)|k`G9VIuuS+4xD&yp&b?A((OX;9gIPIJR zNv!3FWhYIyU1kpFe&E*7T0}6Jnwi1$%z@lw%pPbkmi66znEG0qsQ@(DgokE0;u73X zfhrnvyxJ3)AD}dJUU2m}oKg8DZz?Wdfh*9aUCrk+Zb+@6@PdRTvjU~Yj1}Xk3um-^ zX?dR?M$oU7?=yW~jT*QBr|hP1Xgh78{bCaHv$+G^Ym^Mg7JP9I7V5gyx}hGxr)cUF z+me3bNxN@NyA#HbbeN1vX=bWmLwnczqU3EX8W3v(wiUqcm|ritxE#V7CD91ysf|JK zcbT#KFtf3}R_MKNkW7TZ7H;m?w0PhN6FZee0#KmaytH>-gK~yVz@`P;VS;R{v*Jv) zcLAdU)YPwC;f!P66_TQj4Jh7LpR<#WhP5{awm>&a&V5)u1pxtY?Q+HJnJDtgJUV=0 zA--YF>;(mkirKHp(3%uDu>HE zCUR@?rNy-iDv8$18}3n)9o{PyMSkt0EiyT<&@+>h=u~qUI=ZlNy&wm%;X8ZXJNJ%= z>}{rL5{C~3hfjh+yP@z%G8i`$;ujdEyLdmQTz_pb20w0pih0s=ZCZL8S+mG|Hoqcb z{dv+U9!HCJFV?nDdNJ;&y;M-()I}4K)97{h09h+OOF3--7iF2&)PtP?hWiX->}yT{ zA~;O`y_@FdIab?KvxPppCFN?oDc+@IMiM$l%E0fvY4?vAt!SU0#npO_HrYjwO!aQu zU!rEt9)?ynmA z2g!?~BL)q7A6YCouXyjr!p$@WEbN%HPqb*=#Z&m8M-@EySM4!W-a63HlEc(;W6C!| zDLKLMODMG+R*3?<^UUZoJvxh9j<2G;6Z zUG#Dg4v8ONlRCLPMJ{$jl|8J0{8nj``sp<+6G0b8QVyENHyK~4W8l=gqUhy4vz5=E z=-t^rj*92@uiY_utIW^*TnStfp+%=VCNdKDq}C@f>C&eo@-{83<)lN1-L7p7+wNUe zli=3_x+5EWasioVoMWQc0-{b~q7ipaB8#3amzg}JCBnYe7CBcmsl~p$EapS7Lk3QT z?MvveP)O@Qu*>kh?4wOt2ADa&B0q>F_Hl`Lj!ulZd|puH7#-~E%+FedAo^b6aO8e-LF{S{99^Vg&6TMd;5#VST==yIW_7P)`yE z7Z`GVrAkkj7fzkfcwu;J1Dq48)yaNl>4gR#2vD5+gCFU=o-s^Xy8uAPk=xxR*r8ds zW6|^K&2Q#}pH%jM8?+Qrd)N++UJ35B zf<%zs?A9)H#zo^}f1UUMRM<98@T00Hh&vKlNqD^^zZsw==Z<1PVD$k_$9WS{pcb!G zg1RdtEVpa!zAR3F1iPs$hKPU5*GvCS>7ps0fC5T)&k0q~Dy;QdNw+li_zi-@&XcUE z>UVN#EZf&xqA#!IY{oxKTkXWK+*c(i0F}fmT=wU~y1dUD7=a7fi|@_Efn1a~rKUEQkZk{=TEu2x#LD3M7@g+LL*LzbB8SqEHd-6I%=9ixMils*7681qCj=ad zAw3kCe_X=>wzOTI(_y8&($vMGiE>iR%Q*W7En+d1;TGwyWE+9!y1d>6h#r@_0Q*g# z8L#DTl0#jQG_`W?>ZK;Y_-U-mqc><($CKpO+DQQ#6P}^A!OdU!Zx-F_{OxJCp6vl_ ztk&yti}E$$M4%G@F*^nW<#$c->hQ&L}z3EM1LUXL? z`$UUc-Pz0?svx7vgH<>2OB{q)g=e()1`>gs@qO}qGX@M~1Uu30&UQ(ff2P_nypoFK%Y*v-${1t7$dWL^4D9{_vYA+}bUa5oHEa?0KxgW1vEqBh8H z)b={%x#H8xOl@OOb9p;fL@|XeZkFi@;XYtY# zOIMt|z{c77y@#dt-K0{Q2^_u& zztogIdj{X#f?LDQrrN>c7wiBH@Up%7ow^PFdw!+u`rJ2ixko2IU!wVV5n%Y+u1&SP zL}MYO_uE&S&k-e(I{{8u?YoU)y=hvohOG!q)#1>zzmv$f~`7M zW(6u9*gD!&LVX6oBu&R#5BYa*3#8xnBylEwx%5enynP@G_VCz#1MpOie@CYWY$sw< zQ!*!MJ$V{JaUxBf-FJaj+|+3nSUchT>GX;d2)gafZ!yKCp>mDl$3C?s(#huL&S3pc zW*|Zi( z!5=S!ZkcxvT2>u~NG{DMdn**H`NssjYIHw5!U2+e@teyQ>0a1x!~H#`Chzlb_WO!j znGcDj2FfGAsnzM)FO?h!r$bF#hOOsx9u2Z9hK?1ryu^T?vJ;ryDVI(8AEabA2(2}t zAgepj{-kXgSF~=H1f-TjgK8l?`MvagfKEL9m5{naRkv z9MCpY;kD;@!kNu1Y}sAT{tZ<9m9DCFnZhN(CGyvwFBZl^NInmDhe46_ebjdjyy{dS11cDd=*pLz%&$P?LX7IiwPE3pPZ9l- z*Pl;I!IA>60Y)E$A!M_WJE1?KfN91>7x@6BwnmZ}+zLa}VRz-* z%SWuRy=}LR3m27T_4#rp2{NfXuuv00Y?lF)Jf!3_w|qUy`K#0?23&q0l{+2CMdZ)eF=C}03vpu9&cE*UfDf{&yNNILrDEFZb*-uU7kA+ zoC-AA9ZZ(*P&XW~r}irL3Nz6i04%U`aAcS9?vekS>7RQ)W#(o1Quuo-HeYQy?=!(9 zQoQ+--G5mU=H%xCsq$tUPVG`8z{)624h%Ny7*;l#oe_&c8aUTgEuI9$YC5G6A|n?GN!~U?OQt>=*(9RWSWt>Pj-ecjvXvZy1N(kaS494^fwO4m znY0`z9)QZFdXNp>-!{lSQ=jN$mO_Dqgskp#9c&U)$zl%Ym^?o2!Ga`s=C=+Q!v48k8saBk$n`S(cSDK99(6y+1+uzAR{hczAkW-F(Yz} z+%_LS8NEg2LcrpbYp0ez5i0yjHsy7)!ubW7r``*vef{vKUriyIGw*g-+SAvOIkOaj@wIX|ygYY2;_C0gBt=s@)8|wG zBe_dNI{6CqClBxm;rRqnclxtGoB04hZ@Zxd>eK9Js=cy|n{OP2BcxAR{2{EoE0|%= zZM&hX!M%9U6?c(IdjWMkxwzrlweuQF2gpk=>}w$);LH1wdq~xLrU&M76y!9fe>&z? zu+)nip5@Mn5G7A)$-kFw)xq~*dgj9XQE^v})Z0aC)z8finm!b^L%W^p_&1un$oNGG zQTaB?0mj)gI0ZE{t^b-cM&cQ44+KC?RD4f3Z7;<(*zx1`e%J(3y^+rR89a55B#fEk ztR*UE_C??__f@KC%RAOD{&HY~-`|nM3P$byH$0VO5iDDD6y>qgKq?J*-pt6lw0T`1 z!M<$vfac0&IMDIXy@`tZqevdnC2Rv$jeJYhqQ7|FLy|->V5!z4VXC>XZEgtFnwN6gsw!NU+PQQCeG&T(zg3Zj_$h zEOnZbB1KRAwY=AMFD>Wnb@6=eD@Wvqc*<_j5s9E=q!|ARgZ^0 zdMXxed#tM0E1O#+W=acf<425ID!AYnzIus+-i=8#!*r!ZaG1 zm0kK)J=PiS_VC7{6OPp`IY%s4FQV3UDm_^8t9_--LN(|Y+1`l=d^gZiggR)Z3+VWK)^-pg#Dp>36YL3 z27r-`d-E#bCXYVp_eBZv5F)w4ZX2vFzyv2E#e=lfv%BvVkOIYX3KQ7&fICU0FA()#N5+)pCmIH;4#)rD0;ql*yBO)a&b)cv4m1_0 zuCW`By*9x18-l|l6iH$smb{OpZBP6=9{IOZZandMH``>}&iz0{NIW9gY`J6;pwT1L zUoqQ@2}{r`0W}iD_(9o%lgeqtOpoyh^=_I0qW#{^HL(H{7*Zfp)4tiAETnUvsqh`q zyMd07fr0BC!%m53SPR0!pJC74@0N&-SRFv!iOx1R8k}MgK*~h?e4O-+Ne(U3Wi74k zQnu%l)+5AJIju~-@7HLnEaRlhbSeRWf`8dHVWt_ns!E(e&Tu)pK@M;)bRIyzr9EDZo6+6H(c#zUp@3!8#1ZMkYs!?P%aIb5e;NzW zJP8}bm>N5T3}7*j>Q74!b2ifkz!Now&H!N!IDH)i#zOKp&fYoIJ9?JpJ-J0>KhD|{ zX2=A6R)6}g!ZNx_Ffd84eOd^WK=;lwTLCe`9l5i#**R+249=S21UFh;c}LsX1GkcZ?UQj|JrK)VEh`eaU*PK5lgm zQE588^jV@t4xMgG3t*dgwz9a0TgI@+-OiDQVN9aH`tHWH2D$(n zL7nnKpVijdKS|fq&)Wu{>70JQfX5`}pa%N)0ry{Y!;Pq+_1u``#A)`L)4zF02;B4x zJy8*iES}}puADww`>ZZc{gDB#d|DcY>kSoJErT{E2DC)HN~iI;F{~zhkG z&b7H7Xa1%3{8^4x44^piJkVwzTThrmB*0YSI${(L>)^3u=NRc9y@eCIc6UeB=> zhR;xQfwDN9pIYEE*IJzVjr> zn>@#Bi+yC}D|Dz8L{Vpvi2=ssI7rJ&@LMmDbI+gW8Pp+Kk%2Zye&fDuLaTDF9j|o) zeBj5SQ1c2myymC_QkYJP?^ip!^JFCQI585>{2awm-+PLl{qvA;2fVi1qRO%gy}CW@ zPOE24*Bcd>W!}QrzuSl2JO;90-eySjWE1x3@MOnbWX4s3K;w={HTQV$$z>nk0}+?6 z*VAQJ`Hma-FRVpxa^NFx@6d{&cL8UIudlnI;1`40#z}R%?~J&F#vgrO@Pxhwf-{d1jX_#yl1o(l4agQ{j_64}FzR^fGbMi>EP-@6NShU%gk%Xs+1& zv3hyr^U&NQ4UX-5I>}@ihA2H5Yd#J5`cQH($Q@aszziqZrdIQZABf&%S1^ zL25~C_{-%PbDb46a;9&GhnkBt-(H`hw zeO(+;UCe23Dt>sa{44?1xmQ&U%W+V|z!p&9R!USU3u9MhGIey~6HlVeX`U*E;g-1& zimFIB@(me*U^ksUx4yo?`AIot4=@;4kJ$0aVmkQ<>b~L@fLuT04DoWD-2y?EdLIuK z<&*2kw2wN}QDPtIV?PtG=)yfTPjJggy~MYyR?&`_9}gjwrNvHTs?k0$@VD z_1)k_l;`i{o?$fic-#=GBU1E%CHh(-(J2u+)GMny_7rSgdVABuW+)?19W;Fd3_5Wm z^w{17OT0Q=J!A6Z4o96nXSBL=a(X5Bq16j8Q#%%QnCJYPMaK#P{@4EbS5rNhEv>1J z)2+VNG_`;1QhGepm8SKB$_cK;X70RLEs~+LMaJu9icurgKRnc6YCOjh>J z%HwYS)sJW9&U{usZ#T9-V$H+C*0U^N!u^b&b;#5_oP$b`kJ?4579m6x)nXO#-`e4x zhKG3!NC1)L41X*H$Rz`D4(-J5_c!I~rs;W>snMG0>MeW*aD7T3dm{0z$Zww27tSPq z?U==MRx@+KHI(-#jgH3NV7Kn3@*L$sayy!nSknSl5CdoB`*4K<2uz$PmWvho%%1Ydh6+G#>WeN@1jO$bwAn>a zWxcq!I(gB4%DEYn;CmJlG05E|J0Hqp;Gng;+`Lr^z25lBxV>0#7Dqv!K+$GMUH$ec zXUxXSb)CjVFWswjt6rk!I>oE`RL6;T%EpjgDs{0hkT>+mG8A3JD&WVN%P%qk8vQZG zKKOX{Drj?)mPf{lHZPZ`QiTYkXg>7o>{H48bSs5S%j7*+OliNW@V>{tw1AC%T zPsX;Pj{@*{qME6&eZu_X^m47aeaQUOXWqF$b_!k0HJ7}MxZ?GcezsSz*~EfRz zYaiu8Hp!J^whIqu!J*7sh4d#qr$*a}TjzSPHU+T}B_RF&ShEy)c|&_ccOdXMO@j`t z`;JQ>d$(7?E!Xr?;ZaiVp*{7vy|(V8`?aG< z^zq{Nm2x|;Mu-!7?%hF;DtYT&;P#wO@gcULL(rLMqxLPy4$|z*prPM{Jl{I4K7DOB zEP$DjJCp3uAG(r-w1_!RprJ5tht@0JKIMM=qOk=Aije=3uGCKv z7HqXrSy=1PsND@cEzk(w?hlG|zc|97a-kK!3p8NKd#t~9==@tzM?X)CmMY&T5eq5s zVis|D=(7`N)4(MQxVECdt}mQ(pwf;>>+3Ihl%U3=hzl#}+neqzm^W)~%FT-uZ;~!~ z5j!Lzr=H;4Z>Qk3K;`V66ANR7TXx2iSh=kr5xTCA;=hO;u-)U|=^`<<3hxC#y5Ci! z_=&Nf46+Lo##7&SuK5yd=9kT&QVBCaG#E3O7`&t;|)}eufC(m*Zf<3j&FP zStF9=OG{I{o4YUE{mxc+tNbS5e4tf>spn^YPltb2t*c4SaTtEQ7UV#3g{C!q*X=<$ zKR$PjzU|^oI=_8Mn70qp#IiAdzUJXHLFG|7+>t5E1?llmG=)rJMuc znb)nqTQXN8ME{~LzlV(uB>9J)bDYy;iI2r6SWWi!Skaod--abCqfYTqm4bl~&!5|N z#L>5Degn*$5g21&wFS zC?3Hpj$**#L1a_iG0vxv3duxzvKqx|NYL%5`1vW$KVRk{?B&V5hoWCOi7_Sre06C$ z5MaRST(JuEv7o2jn~uf0QWF)KvFWK2}D0*Y{a+pkfX1m$@48G)?J|r)&WO zc|kzU&i)Uv%mYelY>Vq_>OZszz}XvL!5rGVUh94G;|uBEbNi3)A~Zlv_ZX%HL3n#R zB?kPq%aDL!plkm};Jeb#anT`fdeJRB4aWgk4=_GJF42xy;`>aJvJ5eWpAo{oz1(Q+ znj+@*;Xeb4gsadaL0pub0KM9?{xeu#M{14dxT7};C#dCa^FB~M`AYKBQ>6$cm)6j-8UxQ@Dr*$< z@wUvr78v-2gA%=Uj>{4yc1hgZ zvbuC~j#wako+gsAne+jQO?byZ+UB6QEpmvS7=-PM2qvHPz`mOiT2UFNR$>fy* zoeKaU?s4kxu7==pmaup5eI&Mb3a6TQ8GFxe$U{ZN5D!iI;W&l_J0{>w1L>FBiMhjN zU!FD%T9Ma0O2m1_$!}9xKw+-*|1r_OrQV$#;2KhpQdZq^Sx_Gsa=JcVVR)+ zJ>qnSRmyVjCgtyMu`mlh!JgMGT0SMYM8%d-Wyy6v))yZuXI zQz;&=<&ob4+B}=l z#s6fKvy*b%r!9wnZ-0}gkCTeb`(m^k4G?);>FEkLRAh zm?O~pVwloqxScY%kdO>5_skuN`$bbuE|&($cR<-J|r!x2dEv{W>N!6azN~W!7mRBm@ z0?^Pf$54V4Bv@6Ckp=CrKN|9lN8)cO3#I)ZDXZ%*?H%5=!<1Ac{10v2c8=O{^q!^t z^zY;3e`(vlbgn5w=KnGs|NTbeEEW1!|D#v`Uq5iGeveA~?>GLn9RK~%iOl1D|Crdn zRptNst_b>I`qvu%A0zzxBN*}cD*pEg^lx+i_sFDQ9?$$O2>(5ke~2*YrC#qo+AynFE0NP(}26-*gk&9Nic9s55uDDBLZVTuZ^NMRjp`YwzCV zGw0PL&c{R#?DMD7N8JNjV{V&*-o44S6(0V3pl+vz^7E3vCo9`MOZu;g|LdO=MKiia z(64iEY8K1-7Z98-8fZ3pE}xaE3X2kdH;xxcy}nK=`f|ZR&#L&*xRbk2WOn!rVPcmjIHbe1rwT11q8VvtcpAVa;vsUi{yanXlmDn0*`n~%rxO!o`@7mzc|VGvEkKpXxVw!naS#UJ_Xfb zxn_;=deXo4ZJn1p?#rGne6v>UN}oI?aq;8&OIvuSvzwfVVd`L$y3Lx<-9?JMBmK}N z(_)=+K6ChK4M)5ima`w6)_vI{KAN0X*e$%z#W8KjFmg1KjG?wc?{>WOhE<^Q#f8n% zgmIFBE{m3eG5IPv^g5e%!<+5dxh`*66h%?qBiN-|Yu8Yb*J$na^fFGn;?gOs`9f6J zruwpG{q5kT7`{;eJ!Z=oNdFrL7Fir5=S_R7x7Qcts4-4f?ete<=a9M6%YC>=zZ1zR z5D|-3`4FL>{fy@P-7=uSw8|uQ$8d0tB&}ai9Xa>*ol`qB;SRb=9~gL^eqTZxbW>%4fBAE_5O4F9X)k}!bt!vAz>n0MrAiy{J#KiCg(&Pg5pWt2P^gTHJy{%+^ z%a9JY_d4-mXXI$f*N5JQ?}9T%CJmCF6}4KGE2hV9p3Dyn=mzWD)Z{N{*t15T6$U39 zmN?Q6T*6w<%nlIudp!*T^$v+nQlXbUI=}n&JZo)E>ZThbUgdS#O0CuFhU(Nop3P`u z7GPRFKxJjA2O}h5O_1Na2AR?EiiNCgS;{aET=5h$b#9eBJ0 z_Gn9k%*}>+(&?wi?^B}5^h7&g?MqhIp+9>Dz)$!WOv-KkW9O^>>^!8+X33cmM^l-^ z+&iL;FO8xnlW#3<<<_uGia2ruD=-wLO~aFH8n-EztdcLIrjCad`}3esoEd^sDb^V| zupcmedbDP)0|Jy!e`P%F3p@G1*)uNr+ddbq554R?+~u2z5|bI24@$~*-A4tbxM;E2 zG0`<$!ySoauY@rGa1ZR&6U z(qp>_`gAc6?6yKYwlhwPMjhfkL(xsMGmg+TZ0wDvo#NunnXV8#m{jk)btLNAD>wD^mwu7ld-}7 zHm!jDX5lbhB)Vs!iJv)gPl^tgEwIl8SC4|bU5adAJ7>m<+Uo^)czF3$kI+D2aOpf$ z;yvhh-%F|{v8QpPMMT(xmw7rWV|?x)1BD#oB7|pAT9zF%`9R}CSvlS9$V-}_!)IJi z+AO>y9xc}T&CzGYUa@$2|OcdaR;u+y(>6EMRvp(V^IHq&-tC&K*xT}E5~ykQmGFvCuKhm6F3Q=KmW2vQy`;dbgOD_Y zb60$k0MhNoy=fEiZWQ64uG%bmF6w_P<5>5W8oc|xdvTnz^j_fA_MC`IqQ!Sgb+Wc; z#Ct2HTu|I-Y(DchoAM^Mf~m41#P^a`hA$Gc%#*JlN&t$aXV=SKYqHV#i+b~WSd;GV zgTBJx#^!dFa0NQc%{(+|HlMw~3XTB^XWR{9ySGC0hv-6%HoXaEsbN>zD6Ia zn)Zc)1}SgTLPWJ1h{fMb=AV`|-- zZABMkcU9j5IzFfXI^--)X|F2C4>;pKu@HnUhKanm-EU0m6}>au59)Ww+4a$BYPw3# zxw>;JO-*yPY3g-)Tz)e6FW1AE8*neY^82p5N?X6sW5K-FtA+jQ`lTpOf{4W65bD#R zgkjQpH6ell+%YH7mW!V+6XtLiy$~;hEW$1KKv;KY#n!rDwGtQMwRgT9Zk`R_V^?I7 zmvM#o<#btxhIm)a<5fS*&G#4!De=D9LnCl0ePF5-Jbii@QKG-0@M;;EUiUMJ3-Cfd z&YPi>OW_gF+qZ!Fwlvg;DNeVWbH_0anfv?tbouDB5G#IZ9`DD$g=-jW>Vll|r%ZY- zj&r5o$KHh0mO3|1ZTi}|98?(z;%jRD!v)yS?TjK95=R2{bW%tCrKva9mDB{g_S)>d z{xnw?@^LHFrpJ)5nIN3>1U8pa0XXE-MpM12y60`gS!tdwfMqrNw!7%@M|Ar2@B z!OY=(qZqYGTt|4fJYm*g*6(FOTH^Ndc9x=ba9$s#uQhAzIuND8GBj@&0onE$;?Ctu zduEZ%>~3b8mSUA4_0H>V*(4ZYPfJ8|apPaj#OSYg81Fdk9e$<%`p|x=DLSDoh!lE z=N10aTC9gSm?k!*z_jc_aZ0XnWYY&eM zxiTnY%0A#t)k5vHnX5Ns;vEa;);*KyQicO98{PzsUT3a&hpf*B^vvc1LBX+EsyC?; z+Zqdncjp{cd0?@U&7;6SOJJpX1Z_Tcc|RI^w>gWarTUZ@2fZY=E4{atQ{R|7{7Muq zHUzlra(+w{V1^3TN^goY+r?NHT8yO4p^nS!^`%kXbeDD0rFbr53&xWxNj2rIHkG*xy z8ooAc3**gJ19igcA-_&U78c~b$C?)}%HkzH@Xi5YhD)!1c+taw{i7pp(yNC0LGI-5 z9ak+S9tEAg4;~38c75WSULF5foWHisjIAbe{$%=@ObdHV_IUCHZ+D!rja2S<@C2_3 z-$0M6BJirQ{`27&wFSY`@nsyzC1G8w+Io=~8y4@rxD~eD-KOTKXpITnh1_mqAy!@; zTQzH#HhX+;^2qQ87wU5g#G@aX9ftI?(OS}~N;AhsA< z%j%Jzvqd7V_q|@BI1RKT;=Ch}hL|6j`+vQ=441xt;$VGkboK9xEs5qsPM-##p+u`+lN;5pL z=VD7PhQR({u?ar3sXd84Gxlk6qZtS*3wYy3`#iljHh&eedYbXonEW)d+^9d4TkC81 zWm#n|xELZWCaXokw#Nq+m`c%NDv9m9XmCkmu9`$@GiCGc{wfr&eZ+*|i+$h8R}#3m z$SR(fv>}v&Ko7TE$iA}39YT&HE|U)nqYnALMZrm>Y+6dW;q6pW)lfAm&De%t)Q9I~q z&KWXCP}$3^6q)!@#58#g!JApd$o4jexM^&kq;5DK!Gu+%n*g}<)~$A4sjf`nA4I8T zX!)Enmdma!d2eS1!4|w?eQT?*_pP)KBGg27p96gCYU0lhrp<@^;wi1Ph)!4JjlTo2 z56{oZ8~-dw1C7S%Y5yi|I3AU_mz&}0dd#xp1kr=uOnbLVcycQ&$H!Iue2KT`KDZIu zXb`TQuJw=VHX7m2T==0$Z3E{!JY94Ehz@T$QgPp#G)OgFWk~1ud6@0f%x4)k8-%)K zyLR!emcg^ZsIcJDRm-S1LQ;EJXEH(8X;T zV*S2?hD{6lY}IPRW3OJMwM=5}0tnKmS+)Qttzs?2ZE9-`%iay}Fsct<8=qcPW=61K z@uY88#BG?D*b-o?yBKyo{9LIjw^#3dw5J6GcZEE*x-2ZL-5HRk`f^EaP*Z~|a6CJe zPR+S-DWL^zuDl{lu$X;aGLZZI3mYaL>0UkH5wg9>?<_7q351L1n|^KkveGL-k?5%|tVc>(p+9$t zV6S8cc;J=5o)n840(){btaokp#^7C`g1NH7qB1E&Vz2&rx?6D@>hwOL(+-5C+yViRsRfj^NIauBOLGsJu`96V@pgfUF!n`8lb{ zVq#(Uo!baXa-RioMXs+LVfIfp@F8s@W>hPg5OSA!9Niu0#W(2U5g(;-IAFE=Zl zox4f~>Zkk2eu3hQ%|0K)v*Tf7B(x%Sd35Or8x+kGy8gn>-b&@@<~5gE_N!6u?o#+T z&{Vb3;_K(k6d5&Avoq&VP!?y;;8dzCntLH-&O7 z)Qk8(t%Mg~CEO&@IN;qe(UnGWd&zB!p@+`ZY2%GvN*NjB$;$eXd|5bB*!H90CP!*&~LR22-cp+ z?{D6j=RZFQzhq(4TdN67&&UNK4hBb}C;bSdvea zFuF}}wt`=PW zO;A=0#c$d*ARTytMCoBzDM{q*d$~hi(qdwkbgm}_^f}g#e^|!ju~#UT_g(~%uL;HR zf5FOy6xI89zLT&0p>kjDGd06dgJsX=+rTf(Pd|ov|36f{cQl)C{6E~SXsbnyT2&Oa zYtNL@)GjS*@4ZUw5Jgq(sy$O%Yt`P=-cmDm)GVr^cjS1d^dMY}5?$_Ulwlkvo z6TgY-yZ~P9))2{TAA#Ym#$RkC1GLLRwJfuF({SK}Hy~{`b2YlIuPNvZZ;>Yr*#TPm z4ikCbjVlv-aG#HA-er(^=Q$xgC?+vo$yQ*07JtET33y8ay~~|jHY%O6^7+V^m|24V zJ)DjObr)f>;b!RmOT5qg{j_zx$wF&c&4_xZp~RQ5ENe&~jek--IUUvpymtYX_)Uha zR$E++3(}wT2gf>|^X1iEWNQ`o$g8saU)&kq&G$dwh+8*dUR77!J?|@e_Rznuu}A_S z_wvdlB;#Av8H(ASvV%@r-P&#i@uuGme1cqhZf=F-@|342|A&s!GW+Fm`{M9^0Avx| z*-lKxOPx5ek%m;+!==8$*iK1Lb5Ze``PJOd%Gs9K5?{f5N<(A2X6I!G5snE+LKXL; z%lbb*&8Yeiv|hf)?qEh7Ti1h2_J!ih54J_I+&BDfn+L>uI|(?pxXeED^dA?=O{T<$ zh=eTimv&*27xeT9vDQa=w1NK zEjC!U9|Y8(s8BP#1iFosGy7bzZG}~MKRZZUM!_#j!(487ff_NCoJakX$};0P%}@V_ z*u_ugzM3LM)PaGSq(>7@QL#A9gcz_Ga9Y@{E9>t73y>W_)bb3JQe5TddTRdA|K6k7 ze_MSG|I(~liC2zHDHM18Uk1@k;a@qA%KN^d|8dYRDE>VulIj#q467jQ{v=;#i~Hsn z1(*M~b&P)d7`2BF`gXe$(>EMH7%b(+Y6$XUwG|J-UwxYIPL`av_!7n1qcFL!TSH<$ znj|zWpSvOxWz8zTzi|Riiy1t#Fh`=@mCidxiJRLP_tS2V0nz`}ZRnzHI{>_G>fkSk zg7~NRHufI);Z--oKKiVz4UBRueoWvoJ2yq-^;cKs{sV6y6VW~ zp(|=6T>sN=DVNGc(|m&@ui)n!O z3jgG-Ylb8rfVL@=I`**$jrDCQue(U%oYYJX%TR%U|q22#^FFm<@x*1lvGN z_5$IVk<7io{=eZ6`=(hZ%ynQSoa0#!WS-8XV_v%<0giEedt~a^=AYznaS+?O0&l}f z@HzIOO1DPEfWcOcyIXN4Byy9Gk?A-zDwZ#&tW(E#gMKXyTmA)fv`T3b!%;E9+0B_- z7VV@s#X=IhZX3t2m+}yOarKf=YHlk&dKPS;KRD#M>&^Mq4`6IZF7er@J}c_ClF4Al z#$@mFadYCs0NcUSj)VWvDN+ifdvetOHFk5{W6C(4Eqw;N7?ZCk<_-QU3g6PI1fAe> zl1#E40T%k)6c8%Tc~;ZIrLQ4bSi`rzl!Fh&RV)9B_|0=tw|ly0CyT9hT<(q`W!-?! zKPO6Q;-EsTD9%}Tp{-tw8^tW(##1YPd}AePLJGb=Vzk$%4cx&$11<=+xqmFI@NA$L z%g-$WxRkM3OxA&mmnnb98b(5>ll4GZ|x)2?Dvj| zoM<{Oc$;@7lVKmQUxD64-fZjZww%u@j!Ji~N2IR$+54Nui&RQ#+P?I^crIBj2#0R5 z;5YV9Db82EGzhEW`SyZ!SUsl zT=f0MvDt#;abGTfJuPpY8jc%V`Cke-H!Fh({Y?xFPEAd3t1{9L01$29*50j_fk`yS zd(Yt4w$6Q?I%==i!qj4%LNzCH1s=k?!Q8?;%zHlL7az zYw%0zQpZ=t@44aB(9K4__aJ4H>{tx*R>Jz$i�ZE6K;JX|Nfx>av1GwIA}Q8VTgc zx4d}A4J{PzPjcTO>op4gU~%y)yy%PR7Y=lvf)nEMq%Y^REmvOXQ$ntB{Ksi8pF;z! zsNWtZ<*3tB(!JvUt=kiMQfb5(YlXnS`~pZXXF8?FUVCISC&~>aK1uM&CzTDeG}^$g z7BU7Bup4O}cYg-RC)LSgaL%&$hP$zsz$YOJp)lqAeI7y2lBG8TK%|$}C`q{aGnI~? zXSfz;C?1$1OFE8P;QmpR`HlMuRz2KpmXbeg^&m1-kjgS##B`OHAdaCdPWdF^bt6Cd-7$FSMKzxF`dwT z=(4cjbE8ikD8Cs1sW$PfD=l_;+X(dgIbeOPgE#fUC*JVXSRTWt15WMDgA-%rSK(lR z|7i=08m@AsuuUT-!fQ*fueup9$L2KH$);?#-;4)3VsF>n>}BEcFTW*P3%(h`b<%#R zxwW{g0}=lhN6POwb_9scVEi(7VhI=#b1XJAx~kL4Ow5t-M8)Fif&&)q3Wz2;_?>UQsMU3Ye#ArJpMYsgMqMo@HL;x z2J}{vQB{VINBCQMD+$Q@_z@L&w6+IP`&q5iDFZ_wjdtW;3L*~@m6)wtiBcC4YnzAo zkogK>e6Ddy;K`y3v-WAA&9n`TGKn9mrmd=$9{JR|U(>gsnBokblX+~; z<0_+3x6qvrYm>9*U_<#US%sbh>gTX(>xMmGLg*D)s$SpG$OIm=4ZDkJF*e?fO*}eP z?}9a&1(nZtBX{jrT91i~3V?ZmjsSd&rw?SleZBIT<)HAp;L0qq4CWXW6|mtolDR2g zFP93;DIXu#L10K#l)M z-ke;hbY)mI^cAGg6ajA(*O;Jq@n!T%$wUA{QqzPR$EOuEl>|@nGKMocq7Nx?^GWm` zT~1QBf9}rjhd8Ds{>?dJt6gAr>+F-_S03TNAtQ3~OyR(C0rfX^K|}zNf6rN$H@3?Q zcT4f{h>2&x71tac{gg5{`_h&Nhe{vTeq5Itzd0nrh1oSgku|3t}77E)zJX>1A>^X8)n5I%@^*4JLv4(I{X>P&P^; zWccQ5bmh!G2PGR~0X65>bdDcw#wkpK0_d6^h1tK}A+|SRYK2Yd-VeG4$O>^qPo!`8 z)TG+Q&zglU;krGk*QCi(;r1tA;l>pLT&b%Jv71gUqrFmAB5n_)#y-#>O5()voeK@H z*}Zvir_}k}EVg}eR)r$*KK)B5YYeM~FHq^#VEg5$-Zvcwmlu2H0A~QU)OInaBBa7I z@ofnPMIWb$?PYskp4%9l`+qwMFo1T6N?81u{v&xEV$(Dz1u#;O!tBSs7*uiGXh1*DCfwE*@T3^?p!O5V*hR4pN%CN9$;O<=f)Kam zL7YPW+LwqdIs%^afzth{USZj`(Oh{D-glRp>c`&Fct7o+WK8TyZhCVEoAGtm5EpA5Kt-dC6&JKA^g zB#18bW6$Z1(hJ3vX28#}u4lVyNGN*?`LrHLn=byae*0IM7cMAHM`(bQbz z4#(yl7zw>Qg?N)}J1>Qf*7i#-5lMydqRuZkT{(-x4CSdmF zljAIz5Kbw!@W9n@dqd5Z#n-ZB1&or_F@VwI@@x;)rArN|7i6AdDE8>X?u=e{+tbRp z7IP4UcwRUWcu82UvA<#;^cDLS0H#3W{E={vw;~BZ#Q>0xZ#}*i?J?4%|B-L~7vzJa z&|qoQG(Sh>NP=x#l1Uk*^H$gu@NG3)ip3|0!i@u=uh0p=ZzL3WfdVtb` zaGc^8)9FD?0zH|G1QNoWW0+>o8+pFQhJQ@I_Jy+iHZq}mZH(=h@|IrjLiXLHnGrwM zkTp+lyP|!YEB5&x+5AY3tn1R+qoW_%5;2pN`@t}Y5wfcwVlMrkN6V6+mn%}<*X{+V z=&{{3V81bPvvYQJ%0r9=nzEV(!X{AFq}A@+BV9AkP>$8jP;+Ko4{L^YkiQ@C>$C%R z@n&<3LQS-+NXK9hFyr;OSjcosgj7{4P-7H>Y`&LVmlDmHyHd@y!n5-mVRoI)PSXq+ zFnZldw=0rN)bj8Q2H0hM_+b4FJE1pAj1vRHe>BgksDi#Gi)~Q(JlCI$!F8^%p9{;h zgU%DEJVfHss79e4;mo)2e74PSu}lMIc7TXrwv_3j3QOQHUvt0=Qyc8T|q+ivwp6qZgQnj5iOHK$%)+8ls_2IPeZ-kxSv0bc? z<*b>|imX95MZ-&QW47)StyHhnUbBAA1atv*P6PFK|dG9>v zZVD2NVa2xqW8&o=v^`N5ZA$aOHz0^av~T|n+(>x5%((~|S>4QfGp7K=%+z>`^w?e` zE<(|P_l9Cy#We#~L$aOEOS_S7I(cFMlkicy^^m#LvA#D)%H?^5|ABq}SGJ48X#NHo z1=zMyP|V5Z#p3?S`7r$Q3E(E7Uv7F?$rCUB8!8w8bd!0rBhCKRajO{BIrY0OI}tK- zI?rl&?LkKnS*vwt<-FK`5nnrwG4ZQvgH{t=5YBcn)i|?qHjVz`Jv{qneB1GV@5wf= z0*YxXc0K($XS{!q>Twdd8t@X@UP9~c`u$1Qo&VznaQTYt{{_`OS|`ed@OkL;Ec*p- z6s&K>$sG|E;4qL|UwK4R0LQN7Ze-Z^I=ivcSQr>P9=(Ou+RZ1ucjY5j{Fff5pEGL+ z+wavG24FOK4XcLkn>X(sy1Tl#;M})14^8`LY1)TRRw+UzC|P#b2lV2S@Z%Oi!n{!iVIfESbNZk zCEZ#YWtD0YQqZ>?1u?7`#99hm*5e(fb8WM65All4G{%@18Q=Df zK9q|CftaNZ*IoRV<~NhwdE%!Z!_C5cH8)94WTEfJnj!ZFiuIjYrd(X)T@zQt3b4yj zCg_kTO7;yID=+T!cJ!t~@H&?$2%hCm(>V5F5jv_}u1UY_+iaRNLmV1iFv(UuOHrra zQPO=amwFIt zlJ(M6$6NT!t63q$O&7m?615yKv5~E&)~dyxzGnXj#Z`r9cD+;+*^S8!F+_o%XcW&=ckBXGwW4+x2xXZOj@|!9*HbZ9Cx<4;E*JMn4d$Z1 zP4#=k-FjH~krB1~Nr6ozE4B;&PxEt2yt@tKQl41IwL*>WFK*vT_OD=4jN{;>&}`s& ze?awb_fL@R{2Mh$C6c?67AeCb$KJHW^k)!)xAk6e>jZGIg9?A2o#bT`59bF>i|^*? zABUYX#s$=u_X~Im9iR#cXlj4f1hlXLTe%e5t+TjwPg%w5Ra@hHqmg8UHg0eR;p%6s zyugTD+XB-YUU8GRdjax~j^l1RjUJf0r~t-T!Crx}h9^ee5kHmmlGJKu5WGL5>WY57 za5Uj;_TYU5z)n;wb39~3#yS-HsY0=)?Ef7t}>I#)j9OE^9J!T32+LARw}y@bDr-B3G)=ga(oFwu z#$(;wXJenP8tUA+u9Lq;$s)~+>XicQVbiLQH9MZG{5TD`9;N2d0Sng6#5jKYd)CMH zMqx8uEitQwp`kG=ZA@)X=5bMNQHfYXOAy``aL#1JMPB5L(=?Iqrl_r{N0i)q;kpjU zMEkQIN-j+#eg8f2BO~5@Z^ApZzqcc|82tCtX&Sxm^G_WGDBK2Q+R`7hp4qS11h~kD zRjssYZU-u}$hD!+?$bbbI*=Oq%wDxM1Ez>aHV~%4;80-*CeUU*oufD9wC(NVGhJp7 zG3-$+uf#Vc>L$S)$g#?*tj>&z-_ytCJV~rOrr_6Ow-keV?7l#H8XEKEb7_5j)8Q4g z010^X(@OAgZUEwDYsbPy8?t>IRADPqWnLj$#ofMroR^`L3WTQ(Q_+q?u~WdgFD$bx z_Cl_yD*2O6x+mY7PV~>uzruKijV@ULR$fi|!-i!U^xTV=ZW%zdIz zaBdjHxRNLbLdJL%9&562JM=HF(30or*_P@6Vrf>jgFbn6tp02z29$@AHfhyTGM(Mf z-Fs+U-cl;j(U2JLPkR(lStX7sQx~8?G|xt*VI(ERXouuFV1}M;iQ_`*ZwKP+d@NQC zlPAvNUh=TT**9^o|JO$h)TZHU_oX+nbY0|qmEm&hL@@)0s5#0)Al~qxX^f751iS;5 zuN!!Pwxzhg8xX7+!|7$Z8TR{MQ2-=wBEy7t|2SZA8+U4J?>-7DXjJlSLCSiO{z|2% zoEhfR3-*evCheRi#@NTM{W_UYT-})bei;`Par|=eIbM;%+Q6YV`zdN%QP`nyJuEFn46Y<7Loie7;uMVB*Z(xZ^m`Maz*b1&Ds;1%o3?OW<<^ptAA=JnEXfx zY>ce&iVnqfe$=YE3_M$34f}ibD40Di>0iQWgU=9gp)F@y=l{wk8@~YA~@*EHd3WV={*^mWy|s&e==Rg!4~by#!nrh2l)tY03!r zR}p1AXZWzYO2`dR@D91%4m9Q3=C%0q-%PBwXC-TdMBJIic$;-!s_lOo3VY$Y&&(2z zkun#5CulYF@>^d;d%*iOSNPV08y=J|I zfUrAxv?b~~^~RP(b8xD*ahT75BbvIrd{Xb-zS}H)q@J!6veb-{=+9i*!_DzfPW-eH z6MG~rhzi79Ggu%O4cRVxvh(DGEG&!3F%@(&e3_GJm`Yana*;hk-ur zPe2KVY9eX44WcBGY>&fN>Y(V`B`r;k0UY3peKK&`%0=+S?j%3}Wx0mquwvOp?H|yB zrG|a~xuzHQaG&8q4{GI75R|f zn`?LCbFO3E#`>Q=c>FO|aSYLcxvO6j6HJ_u)VYcG=?ZXe22Qo#&wq|;PfaB)BWJZ0)SLyUL#}pv($a%ahyYFCc|xk0g>i{AWZE%B#j-IXaReDx}gk|yQ z4mjmC7=i$|KLd448OJ%K&@1KaMV0SFTPN zq1(z^IVzuVPl{6K_kSu<>yqmEj|45Qxcga^l8=c4>TDL$sv3`7q}rpjQGpS_mC$ZK zdMMgH;l+P{^eiVRV}o;3($w@Rrd1$2>DAWq2MwTKfzCqg0csT6g`vgGI3JbLZ;^JwCv-InZxVZ{XLMIK0s}v)CU7 z8#6x}+4A_bXoUn4QJ1}vZ2)t*Nq`52o(OhMgKa@4J#JW0FK6g#XcaeLygedXgl}9E zJp~9p*lsu5y63mFvA&#WA0>H0M%e_)~5PH zOj}?i7D<&;lMkB}${4*D5Z~o?BIhte7DU!dq^2^?BX2SE8(}9Q+lqPQd zCK^w!^)u2{Ns@-NVcWL*UW77aq_J?n>E&jr!Fx3;*cJi4cxTlaThW4@%rpl-fC)IKhY?8~}n}JnWx}U;~Zd^nvzF=5|Lt z3~oGULmX8dtZ6$Y{4YbuYA*I2Z+H5(JLFqTVVl{x95o6Vdu;0X`_omyqVF3*iR+=Fw14|}%^KS;p0 zELXil)()#ExBL+rnWn?(;@)I>z2#-Q^?e!DFsQHn~)t5 zX%39(orRn~qm#L^^JXEi4DYd&)6|Dtm+MUK5P$E zbWkZxzPqE*1>Lg$&s3u}*<)}0=0nTE+N7j=x@HuCVT}_QzXof25!Zl*#yfMs{Sfh} zKYw1G5SywUMv|p?ba5C4?9Okn;F^;LG})pr7o^8q`ZF7dpiP5V*uN8=q(?a)t#`?H zCY@Mv($ae&oZ-4&!=Yr{X$JCwmnDB2_i8Ro4^R+m`1yysxX%UU$OUQ?3+(OMV=0>S zz;X^oGlDRt^Kw1Nz`rH&XtHJV(u%HY^b{$?rvRb2SXh!$`@cqXW;){mg-{O8Eu--d zse6HY1%P2Yg^TLwDnK*>lwhWzFM|sua|Ak@mRG~GL3dv2(QPYa;hz7<)wJmfVa!f9f!%l#K1 zxTJv#{HokIIv{nf^tTS#2lr#hQ zfPgra9XQy~#e+IrIC6Cs>jZ{i-PH>LM>CLs1YGL++zovSR17|i*FUqUXdHJ%FpckYRH@*b)p-*wx4B%j!GL2T22g%!o zMn#v?h5cMDb(T$Kk5LzB((H#Tj?G*A1}ySw8J^{dd9|C7a|)4LuGuef!EV9fal*1w znS!JAYqzyCY{0p+ZPixdqwiHkfnW-N-6F2C2`}B7xfL`|sz=`y^83RLGBn|pHurUF zosIh3dFRiux{47)Q~kFhphp=VoNgy)9#k15x!m#G46{HJQT7{OetQing>mfMaJ-{@|tTHs4e)6%GE2L%?juT6nrnct`P%z*JkB5Ce4psngigCfKneGQSo z1B5dEd|95pL- z>jWjTrRfs&O=sIxe>{|&h|&E~)tslBQ3r!Bw84Tn|B+fCni*=D*Wj{kq?aJ|uu!t| zk4;@$Qd*2KF{skYYt@C)t*xuWQ=E3D;+}w8-jX*?(|)ap25KAkk9DsHrmM31n-0qR zGhV{CKn%}&`;b|6HQr}7Gp0>U3&yGpib;fTl6*GVg9L9FeN4(!jgxK5sS1$vB30OaPr))u3Xl=3=d9&-kvVC6 zp%_w^f_disDbo?LY&rY#jxZQ9F#!(ds$?^w!ih|GOf7b9QQ&IssRFICOzg``dO&vu*&H)wUyeBu66^dy?P@cu>%R&W-SNygZ0W)Ygz{IALbEfor!Y@ka`)4ZQ^^R*_~J8d-}>R>l-3*DY8*uV+H=84t| z0uujK?MkHbC!uW1ex%YizNb3IIsQIE#_d9w+&`#oG9-Xir9$1r_YLSZ;1o+bd?hn; zAP!=Cc7G~!%?wTTS}wkfXFHWE7kvk(v@Vks{iNBpX`>)&PikV_nlAL~Z;wOfHK?QQ z@l@YqZh@%gGxy+VYwGieM2n0yd*@<}4){AcXYQvds6{{zqCMYEJc+O0k`=6d5(ti4 zmz4mvT0ZVxV$pJUIDR?F8}JsCwfJ;s?I`|{Pc=g9jlR?B$s7p~f`%g`Ps;mfs{!ZR z*&!kNGo;2Kxp=-~3EjjR{(mEf_o4-C*m8jsXR3A(4G_B>0mE2grV4C3JKEa2fa+1~ zz6|7DJ8TUc&GcG*+_PQ2VznC7SEQas3f|PEutIILq@*+gVd>xox7fD&q;HCbM?>?E zkP^DUPWeC#%mfA8R}dYkKtm-4@Bt7VMgfe-4B7RZblL90M)Ae;Wun;Z3-E4b_5M#tGzsGa)sA#h~@)heDem;T7)o!$Qjhi^)g_L17kMz*~Uwvvqv` z4<$eeNKp^}*VNwr15iHqP<5?6(>`50B`{}o$f9+U{M zE~wxOf&H!P)p?wrk}aOD+XIa+g@Yvy&pWB)hJpVmN?F611DsS}I8o-7b_D^y9NB70 zur0%pB^kMsuehj+#Vn#zI48*2L&6`gP<5Qi0qs8Jn|}80eSmlnYK8nq?3iD6RbsR% z`}-cVJ=N}mnJuMv%1yTAvuR2~8c5zD>xv312d(%=K;OAsWkSvlME{k61_fo4g(-B@ z288j6lB25qJp86|gjkkA9WRE(Gp%0rpLVkmuK@ERV59yco_&`<#te7c_bvGND;<>2 z5eu-(?bRc|@}un$o)Q$8f+o0$BLFptJ@oA(by^XU`=IaUDJ{3E0G1b!(lT!t2T^HO zLl)()WiAHUo{aem8|uH{{k^ccvc9=TUkez=8Y>AQKF)-IeN$Cn+Gda%UA3JucTbrO;{SWr)mZM*UG%4KC`<*$q);u5mmhX?97 zar8F!UtiIw7q-P}gr1>}AN{)d{^9q5^Xu?nEk(gy+bJ2x7Tqm(VZfZwQf2`Kc<=Ns<7a$8NFTvJl5qA^(Uhlm=vC@} zyC4LfP%bWZn$JB6bB40+c=L}uVz6BVcB@O>Xy2GCuZ<8||21%3EP?Mlu2-d^ZW8>} z-i|YV)xIt$9yJ~#bnL+rV&jP^+&c;YZu1KSXgE3j6Ze)q`jnd&NE&ch12Ya5zC4GL zyPY_70QLBw`tR{LAI^sou51NEh#za1g_7bm8s3Aix7nSc@FBFbVFr9)jx5q(w=B9Y)^u` zX;tqt7y1p;iW=#l3dq6QY|O}|f-W)fT7h2$b)vd`dX@sRL8sHG6_`Q8l>Y+SjdXlg z91twCR^#?APyyI@5Bp=_&UPoDWJOyJWhL{y$GsWw=J7aBAUFVpv3#_krHYfyx zs@&W4u-C#6Ypf8S>jMcK;5cS!&-Jv2{_IeS5FLX#e!7?mxA`54k-_Ec@4Y^v-x+(k zr`W_^k@J7iq$~@dt=eqr=hiK>81;7a)8XEJ^hqPvl3N&ARlx4Iri^>`#;K;sINs{S zNZ`cSb$;5$VwH0OIxuC8?=KeTG^(Phon;hl*H9@Du?cjpko64gKXsqqnaTUo@&h>P zsAXyH#!yANLw*7Bi`KK;7Twh%fwANnIj8J*^$yApz?K8Ac5yt$Jg)P$te;)?XCCb~ z^t-R$WhVQ~?|3>I|5)$FuMzCMj8Fmo@ALM}`ymsMp~%$U}-C1RzzU`46wg=myVp)$o&U72yCNiE&N*=FgG zb`&lLFrXm7L-((hOKq2!)vnIc=I^+~6tV0ldq(nYJij@tCChc2y4-K-IhtfCYQgOg zE5IYAexRrooL`N5B`KX*?r&k z4&)2(%vq2pvt#ddeRY!EmA`T$i8=F|E3I8MCYiavL?lUF$`MIs(Tusvi2WXGrT$ui zvd@g(l}Xa|0LYGJw;!A`-37KEe&uKz>iab7RwSSMUhQ`3iUs0vT+ro)J5>@x zroPVJp$~tSgB;U*ndXmT1PNU(joZpHYp_Lq9(gz z{NFlNc?HpOPAZ|Ypf}UVIq5mcTK4v=n*{feK>)SX!_@!d1rS;Jc=P2Md9z=F>aHQ} z_!p}g3O9RKb@q~G!Rd$dH7_v5vI9Nr>)!g5*;$7~+bm56|9oxBPba z%W{>lXZJf_pKs3%(#l#t#5GBt3zc{1_4&y!<0d~cNzC+rCpIw#_I2`RjXwmJ$ANUO zUyO@frf02y%B8XJmlrSTgXOge(%abG!!Q~-#3cT7Bwq7K-yz0)e7>Z>C*E251ivnC z7dR7c6_V$QJ2Whp%6*_B=OCs%YOlG;76&r6CtTf|BX*nYdH->)6t@#F<%wCV)K)0n zZ-gxp7<(f9K_T@rgXZ?kr&=|t7!j*Qu(;Zz~J;DG5sgcyNk@9kNHt+gJ}o?GGVU?g#j z`Wn~>vIIA?lasm~Y7P65;KP@d%*J0#{QeJh$(KIn6_@l{!?%z#b_+g3^m%Z?|G(Aj zf6ChAV<8)ES2kxu-~rky)pL7Ce7D#8J(-g@6BZG;RRUWr_#kPd+J*FG#WYUu>mWvr zx>Sq+3!js+NC&=Cz>Q_Pr2l~)vM`%RsW4zl!bo+*eK~kw&nT9F`~FhqRlu)+q~9fX zs2-X7v4l{~*U?C?ybIuKCtLlcp?qAGE-oKxc20S0>JvXzx*h`cGjo&tRkJt_;rOIp9A|WkKa_@SD?yRsy{Cr+TD4!R1jc&@Bp}rU>g?)S$^6y z<#Uuq@0AW?v*pJg%FURj`g}W_-)9BNic+!#d8b{~Z@rF@t$IO^QK7ZD>ipmDdX$qmT$|^lU%VGhtWL@TP_RIC`(?;)i2kgGD?n1s{{8QEw>Bk^7MxDvm z3Tc&I&sVLE_QyBrm16fhuP%&QxT$emAl_Q<;*j;eO~&qr2$X+(_JfmVR>nNJOsJ?9 z&+F*WTA7{|@?ht}o>sbAt6q17v}HEiorm>FCj7~_SJw+bLzt<*Vwf8?L*kRYG;v%_ z@1GqSD4a45#f_HIkLipgJ*uH)Psd7vN1*av-L1xX`FS$`2p5^tiquRhzcQ-e`9y*+VcSq5N6rz{ zvi1&JbIkK~G@#85D&K*#Sd9diOKefz9EyA4?7G^?IP+3g=hcr={VjuKf`tv~5VyBX z{EL?DKku#_8)c$h1%)(MBT~_+g1UEHYo?q1LbeC?r^Fy%nc$n~_ZPt_fe>0cM?{Vae^FS4?p1KcDqPv|B;zVoh9(J#x|BKHxx2g! z`ku{WRro;`4IHWx>n)h7mc6Q9{;*DX9j;Zyy4j1arRimAfi6}|pC_M!R{lN?o^r9| zIP1GXD#_*~b}kFH?9rJAY@d%I1=#0~)ZfXcUW#mhF2{M#5D6s3+eVSA4R-64)r4%W zWmoPfB)4A<9TsyCIk78o&B|3D`Ab&3_rk6f68UK*grcPR9)0x9_N^h?3W7q8ZGPSM?Zq$FFabRqctPet z%|}C2ZxzFD*;B&lnZ;x>$vL zG%T&JZ8YJn(ofS4oC#{P?Fn1^^WUvYVD)v1eTxm#N0LynQ3wcHPcT(|c7nFbxm=ne zzgVUtyFq^V|6t64gtgiM$~kET5z}8WYF{$zTXqA@Ukv&H3r0Ruuza2tXm%>sdR*Y` z+n5v7E~#Ag!dd300JJ^TJ8GQ zo25d3Ncko8uwg$kUfFS9Qo|X2;ggyFV9-48_;=#zQn*HG=mbz4=X@8Yy1rEJy{RtO z60z+JErl^6s~Zh*yz>&}JVnVr;haX}G1f_&Q<;O>&kOFK#l^yNJ|e#VnabmFm6U1U zYnu2lbx-;@K=RI~D`1A+`$DGIL(@EBQPkz5sst~ZJb1w$BVStI{VDWR-j|n1aW&J? zr_d0?{#SFueZDMeahjd=4^|s*%j*K$%UTpOe$D#)-8elU-x2Dv8y0RosO1qwVio z)c2LE~;2m4)X@|E_(kE6JoR0 zxOV5HVF~eqVzR;o`Z?@_QGWnSd6Q1oT#8&4s*URSr>*ZJ=;R?FF_K=>ZoJG*6#` zHK*WuX0`8@2wuYMZ(=2>_ygoMqLm$4#U3p>GjN@#EFj+V0()(Q#?+n=qy`EJx zd=6seJzqXAI*T=$yL&d6ZCP@Bf5^u7iM2AqTH%I%2kY~Vfr!|TVeBNHv@SMV;#SB) z5uHjBM*fDoopJX(02=Z^O7d0HI!c`Vy443ZnQ zsER`5|KagK=6|jG!-R*~Wt$AexhBEaI3LCf6V$o-?P?qVB2xbAS36VckOQfArt~Ng zk;wab`N#7b(}(e!A4&xK9qv@SFOh;Kh?T(^W7Wat5}OZ%V%LUKn?P;+Uc@lS1ZXN{ zgN^Q^-Ue8nFsY*~{k-0H8Dn^fqhq1E7bg81*>)+(1`}Sm96phZrz$)-a37NhwqIHs zUC7z^e<#QR52N+)-87m>p)!M5-7f+Oy2rHdaACZ=j8;}L zERTci>14`%mvaTzLTLwZ&-L3KyKSiqHy`B#;4K&b;8`z}j4TjLvAh@@LwhaSrCXF; zXY*6QpvT!%I3-ZjdEQqSmwad8bK3iY^z~^nEuCalQo7fy?a}&$8=tMxS;Ma|sbi!z z_=f!pU;`9b@hKnMGs-5!_5~XC@2r8+0~+s#Dm&gY64;+&B8D}zk1iQ?^U&W7naJq& z%I{d%Oz)x(vCkYOMXepqMQ1LS=m#X-wJvNjcP?7{@4&jO($#7KPUNrx@T31zkoS-B zv%lG4*WIRfAl%+9tO6Fhz()5+zL)B)6WyIdYhU<{@Z%Y}Z z#do;gIRZ=P$neYN@Vu9RFErnH^fM zvmb>$EA(HKz@Ad2$Ol7uT9MbidKEW2ikjh*v)YqQDw%$R)ZDxF!CQa@DPm6olxpT~ zA}12rmg6VS8O5U-FnzB@L6XnGi(j>~S|(g6<-Q3Iox>|k4_dJ};%@MEwy$=Lx zBecHNuKc6&@$sK^Iif@bBm&&j0W-^kBC98in{BL{xE?3V9-$y8j8u|&NkpQWMuC32 z)pDi{Rx>j(Uu&wPKzX}a+mRqCeg>yODfg}RG5f9XEh(U35`8-^4gi>&c50`bW^cuu z-JbcATg$AVjm{DMtxWWdqziA;KEdR4c-9n3m!R8P)4`zNJQc)>4GJBLoMEI}c(Plk z+(w`ckv z)mGnRNnsJrtADvlDG9)DOawGu=2@f4!e+cVkgRqJ|LF{DYj)r6ZfnZ3K2$k!?rtdY z`vUc!%P8$N6o>6SC8wZ8$MZ76sqI=VhFW7RwR^Jrl;QHt-+lVG z8GhXwpaLq0gd!md(%m_NbV|d3Al)ThGb$j`NcSiy9Yd$IQbS0`$Pgn8H4H-!+}GcC zzx%Db?p=#NWZ_yb^YT3B?6dbi=k%5yh&iOU$C<}vyG zk3}$K&f6<@c}R886(85|ZJfER%CXp<#xIMq$~b;W+sx5Ybk_fiQ3Psy=6u)uOi&6MS zshdWh2(epSokuux9AzN~ShSuWWmhjh-q2)H;xTsL(>94;^?SHZY4z3i%WJtgobuiy z#dDuk@7!kT@1b}R#Sg4uG+sb9$Or(43 zo}=d{9b$(OV|{*&>y9P~n=d}?&C00JWm@-zvg_bDte8k%)kV~xzEZ#Aw^|mwE5y9i zo4B3Xd5NvpZDrsTF{0Fco9o;KE>Yq*0)_ZPGR3 z898!LF@J(p6^-s&Mh#t$$bAv(N1Nk*Wc}chVA9Wdl3@Af+S^ksp7X12%G_(#oLBKC z$2SVUF|Y}*rkmE&kx%a0!~^tZq@?8nP=m-40@Fi-_8zZF6UJ*#5GJUlOL<@2a|ge^ z*5Y-t4TAIl^K@p!$cy7z`J=}*4nZ=1CFqc>w|w6l--3x8Y;5d*!8Oj@tenJOtP$Nb z={Ptc#-MWyGtt`^-68<87(MjKVss(2Lhk7{&+)7ITd8H}Da z3UfTD$<;D=BaXQY$w*gjBa+q&4%lEKZP3ZxV_elG^xUW}z4#82y$P%y(8X({S(wJf zAxJ~8CQ_>3<7Z*!ct{gRfnnXlp4Q=LCbPavq)B(%j>&(h;s46;#fs)X<0LpPvMo03 znin0Q#&ejCW~Pz=vtT#wP-B3ifPCoJStk(Y|Bt*#@^*~EcLKApzP0I@P) z4*8(qc)1+`Qjm!`!3P}&j5+MbBOwt4LvbuTfbK2HuNHKTIr|~|2FE>R9T%-(?Izy1 z?$7xi{42I3T@r}=jc{Xmel;@HJ6|ZNA5@5EI^^67WhKh$yOGsrg*c%soO5lMb!~W| zyS>@j8KW=YB&`vgV6$8u_DA>IkHS%XfL5>X{V z;sKwT9yQRbJC>g(*^3@zilWqd`s6>Thj*M#Mgix8JZq!`tZ1bk8$@xV2Glw9Qy^Knt@`P!%B2cwLuk?^_Ro&Po|Ah;#tWFj|({WOVfv7;L z@fcLfEZlRCZL{TXq?{zSLM}BvZar;`>=)1r{GU5pdzxUFJr;xd0IMlI3Cy@-(A{`U z{ypLW5)!N;6c_N@?LjZe-prP8-e)Ogk=5IIpH)h@3r!0`)hZEN54(X1($}=&8*z?ZzE$nrr(91F z`e83Vu=b^6MJ7`}w0PZnR8gC@q%5ThtOKuoSf|C}9qNW)$%S)~`S5#PrWu6i9^JZ| zJWXHsD${cqsHqIIJ<1I89}iHuMtpSDtL5S5V4oLFO#tekif>y@!fokU)J)2xmB~X* z6Dafcn!_7eQ_KwOW`cjlS*<5IJsBT%6gNNr;Hhz+Q`ZK}5Y!O2O8YETpZgiV0A{6St>dI%$;9TO;2 z710sgh&Fw%8}+&n^Gza9#+#PW{xz3>r&jdJsajh1-!PbnxW(g3uce<8T`re-Znk>k zxS7bbDr9bjBqRMXMAqeS{ruqpe<*4xU_EQ;SjOp+W(5gz~f@xnP@lI%D9I+j9L}hg&cFaLh?nw*_YK)t~+CDKFvM#R;qM7#!4-2Arb)`DX^#^ zG?+%wG5AF@n0GaH?%z!ir;uE_%JN9_(}hqNHeUG|^1J`vS0I75)|wIqrq_vHWn3q% z7*RU%h5v$GHD1L9#opRsc4dm_N>ETzG$QTF#Gs^xLbm5l!l=j?EXsw?cYNLRQaRPU zE%d5BCjZst-h8pU5kRYx=N|-x`H|m0-B^Aut^Te81=ra>c_`-YQDeLYYN`$26Zdb> zsZ7=V(<%zK4c}xF&5W(EwMH9rJTf<`nvK5?sJGQ#@l`*neE0B77N`Sn=*lh3>a}7ILL#kK)JG zlP;9bGTyMo-P4rU9J zrm-$;*vsM`?@nVb@X)Js;KMuV91hdK5t9%%8fj57;&R>~8B&=+F}+NFbrs&!C^K&A zz-^aKXgYDr9zyu%4%)_e?IPz^^k+5~jxUa48QX=D(X568oJqK7)6wq0pOOmROx317 ztw$!~IaPO5q`>cWGke(A__MbO$@0}Y`?Ks+k~Ns8SuzfnwKvL*yF%@fU>A7gnt8ii z&|vTh_blAD6NbqZYYfJjb*(O@2YWpLn5xBX7;WlE#5QjRxrDPrA4%u=+XyK@KiUd4 z{rxPzp;CmwGu}~w75JkCv}3d<>lwR`6c7vS8?@r+fK5x^B`gNF!HfQ&bAg6!qd-G? z2QX4inYix*HCX0N{DHrpD3GgQf8bW(&O_GW-hV!#AB3$ubUdbYtb+TizFnSFuAm93 z__aNdN5LA^cyuh`5^*uUZ0+<75rL>!RVBNI^gKaDBWjlJu(#f~UDybmLR*!5W@U}k z-4@OS4qAY#eOfF2mStkOi?scNoY1?DTFew)OVwMwD}fCf+(Yg!sF8yLQls*aRu%8_ z2q0s}8L-K|uj?6kxXm=G1`$>bKDwFK$Sx&Hyg;v!sQV}AzJ3HLVkCNbl9Uq36jU zO3E0QtZ#OkgUEKEAI@S#5Cl6CH2JrRJZhhQ$SDq;hk<`waHX zt|Z#dv;&0Q+6&h>`skO_fx`Ml_22izlj+Hf=%!f6f33eGsSrsUvQ7T& zZhEX13~s+AB?J!?xk&varu(aS8nHf)v01e{ehU6bLR(p`4??(*>&yG0ux6d{mvA=sm5$~@s+kOR?-?D1H+!$2fC%31@4afKQ z+0PIJsb^Nq=wPS0Qqr=~&$^HH37d+2p%3qQ-%b5Rem}F*F?x4pv&J5<#`HJ1A^zJ7 z@ZW4_HcbGt6oqqN4w!+1$kPZjyRA@bubzuEel?#cc6{Urvg|b#?I=a_h6KxhW;A$4 z!BNth9mA!Y$*yB#Gz^_*)IYZ22chs}Ua{pjUbJ@!=ja z1xAV>6z_asWl1AY#<8AW;GGvsr%uEg_khUQx3u>7WjWPxmL{OHp4N+)J>UMwLu{HN z=vq;&OBqbX%{oPcxY9aQP6uw`V{5Crv%e;!~u12qQ==|}NaA|!W$uU#= zPd}jb5iM2W5WGMl_Vok!H8?c?NRHSSEM7+80V&vEvNCC)5AVoHeeQj z`<2V3Wj}_qn;u?3kewrF{N?cEuDh4?(%8$dl)jnQJ1>^~_8B5NFJ)jWRwXK3smAi} z`uJ+hJx+&^1diR`TNiwt=RUq!_MJUy1;XR+kGkWi#D#pJ>5b4?ZW=7N^|2t5pL7W( zty+e6B=k7)N6+leYAc{U6dE$VEFjiLk=(}mZ7g9b)Vw36`0{PXoN)hcZ#pSMdT1i7 zeInx^@H5I-5t;b$9%9|$WAJ)D8sWKge5!{)+5j)p!L&%q^e70>)Z%5-qw9n>7E;4L zvid{Xqw0vznsx)_*c|GRnd#BAh?*u^Stvxj!DJ%@*;-!2nS-o8$z+1tFP948H6aabtLVDWqF= zyr=!p(6-9DqG`e9DNk3lFZGNdLm0gy_E$9REIZ9<@m>e{irFU#is*fEZ7HbCT|7JL zKPAW%X_dLS^0=tLM|)LSPs=1bdcA6v8*;xIBy&PCb%y_y;L|!gCCCMFp19LliNPAL zZ7IVm8Xg*LMQQ(A2Me}Or=wc&lBr<;ukN82hp6Y4^}*gilSUttNB=MxjBgodg3amf z!{gMad`^wAMmv*bw4~=f6Ki7+4&ENmqluKF}dn}J{*1= znWc3j8|Gyr`KgZ0vNPD`fmKZZ*I36w5kMEQlww(}6l4VgiLHz`hlFVOSY}J%U%~Om zMuDOBbI4I^nKl12%it)KT)y0RL%^8Jv#>w!nt{Dy4arp8SzIO0&I@%P6pE0CzQA64 z7O2a8K-lh8SA)Gyh4*b^C)&2rBFF(hhNuRS+3DEkW>whh4W{WAhvY*_1Cag_f(BLG z@Q(c_N5Xlnz#g8TJ2%~^&3oE2bv4q~dKnw+KF?h=WE7lKbyxQ(CeH8noDf&N{aez% zu(i`Cc*wKb8@SV0_qo%J+U>a}Hu(1w+c{%NrbL#c-_UX*d*VH&1jaMptv{rS>dB!4mex|5&xR^k0_Ak$y{TFk?I70|>sv z&H};Hc^5r%>s3DJW7-c2&c2~EiVH>518*=uJrYYwu&cR~Zf}{Xka6MM__+EBk^Oue zOyn=~6=ZtXQdgP>`v;zgyg$Ep&04B7cPJCB3;qfIkV%#8&TSvaaW^0#75hQ3CxqaSc&(WE4OWW!)?f(?e)>uF&|CWD}$liDR zwlaHRlVQc5J6>DO4hzATgSKu{oNYC5WeSi9C*_w30CsVJr}ZQ{EXN!&brgi60x4gs z-w};g1l%<F}~%gL8qsdB~%KD~Lzd&^je_dy5)~_W4aNky_uj!gW8`wJn~TQg^eg z)GsM-H!fWKspmS0J8iKy2|nv4=A}_L!-i1ai&siO0+?F4(qaW5rAwl>BlWwb&Le7%_<#vhU>eqi{x7xp zPsu*51$c(!I72e74)U2+-j#&yNy;b~6g4@MMe-ChV`7%lztnR`iC&H4#F|VhJVy9{ z`bd(RIbtD2GLY{F?%h0IE#=&>dItweY2VGKm8D) z*Q8i?T5ePKl63l9G5eKL%f7obQ>GpVf>>M4XeJuFM9T+wtT$4#FJ>A_NE zm%ip0<^lKz6-T!jr@F+js~t?0XFdv-1|s~Une~|!54yoA>V?m)M(ky~0PXGyhjFhK zTfMcLht!TOdY~wLw#v#OZo#b!6EK&ofHv)(Uo^*ebg#dQEM2t|b51b4DJlPF%o!Ba zpX&Bi<72HgE+25qnRt*Mi;O|ctMoM@nBSdCQ~>fWO)aCY4XHs~LJ4RmXsB*i=RTCq zg9b&Sr|&^Ob8kxl?l3HrizEXlPX@C->90GDG6^Cs5Z?_J{rklmfdtgp-bfU9N}C*g zN3Ed;U1cudxx1U6Qa(5N_V+mle*gwu<$}$54&~iL*WyML=i#I6mVVB%rs3K8~tM3-AO^-)u?RYfYWfp_7>k{CV>60e079V?aKrQuE-~?Fe=w>Go`B@NDRa59s8sxRA%%uX?N~}S z=+{Zaj2~hHGVnAz<-&npsn4FT+`v<>)lIdlQ1^>rV!=%I6<(J_!25Gjt_-xqxU$Ei4g`p1=hPR>hVaqP+) zNwwI`4r~njXGG@Gn%$SSD*PVtNrRr~I+aUnz8hIAF8I-TZcD$DyJb06fRWK+|2Q0wt1fw zoWY*b2_C=cxz=uPhHQ@dok8{=%jJp0oAPj0CULfkE6owZRoA5eFOecZ;e20G!j}l# z(Kv4?#k{LAwh`jW{d;x+I9S*m9&m^pwwqJceNv*I{ET|2CK)G8dI4C=wsdTMvs1?A zKtjsQP)^Omg}RX=R@a^N&zm@w9~-IBtJfNE>s0fIsw;9c-!YI+Pp@B$dkA+F{KWrS z^0_f87e;-9^dFgHaxA@N*}Lldq);T+R;SLjGBsAW%kJA4b8_7-?L!e?Aw8LNYTY}W zz)Xy;Rd63#tvSZ|aUn|uvrf@_d$v2`4iUe*f4WfE66r3A$-a1k1ni4|i4mOEZG3z} z)c*n>&o^G0@BjO=byQ7k8m9qJtw|UJ0vrh;|CNHx_R`n6#Dq(8JA^Q>O!9l-GhkHa zrC|XND=}?UpWgZNR~q-P+TDh|fHZ3O_P6Or@fAqBC92GhjN_kV&>)P_bcAcwe3ZA>nM4SUOrok$9^UE7;1k66_P-#6p%EU+ol&wFd)lKO9=?;01ne~rtukTR<; z@0ZaF|GVut`$AkA=lwywQKlkS!XZu!6cYpzBUAk7APa1Q!;KWq!bvX9^Zx6~{xspf zBTk0xE&uz5e855skhJiD?>A9Pik1{tbDy7~R8JB9;#ZE4-IFr5i&xFb=6)%cgkS#> zY94ivTy&!All7_DUGGG}zPI;hE(D!8Wrwz?&YUWgIq|%MplpN7z;Q#Jn>#Gu9xF?e z&I4-%V@u;F@j{g;zB3wRJ5RpoRDwufZP}0cqSG1eMFxVwC^=&-h}1ocjys3jehNw% zQVV)WmbVB%sb>T7$SAWpq6o@T)RMdzOuK z`tRKYKc|`$9x5A;OGYi_)9IY;-32PQpUwBC+ix(*Kyynm*wIUfwk})v4>${8VqGz& z%%KVCXoRYKrvk?A0GON$C|UqIDw!cIkP64Q90UVQzsHPj4;8RKH09Or@TLIycu2al zq2Hi0h9@WzbB=#;jJcO6l<6?~P>vqAaLlxSxx#D_xFSK9(v-iAQt7fAU+sk8pa0_l zK^;eut#U8(qaOiS5EQ&a2e{4wJYWl;1!+g0JE8f$!=S)(nSb7A#g&Nx+7~3qz&h>c zF%EqKY8;cCn(vc8;y(^Sk5~iWHx#HG7%GXZdjad0>PxB%Q6kLbU zd4#j(W2g$@AaW+Ytr-dPKP7gX8b;SAg=w43uBkN-Ylon$%{XZ!V5LQNx81hut|=7| zns6wU zItP5u6)0RJ8G5pz{Ucuf10Qs2)#di}U1u6Uj8Dgr2Rh8Ekce{=4V^i?XEu~a%|_%o z{3IxzMvZ9m!M1fH=xmRG0NC2zdKTg#)H$yOs&%wNdo4aHp%zi*DM7r?`fhR|5hp=I?lZaiK?R1f1KvIj{CPO$(o)NqOZTQ3wC6hrRnu*jA$P zyRVnGdrS^Jd&O6~67cXe!eE8{?&)cV|8flr@dE0+(hSuKcHI@(T9mMrw#e#+9CgA< z_XrnkWVNtj?A}m+-b_!me|NB6xqj>$^11KXw`sDFq7(RiV1LKcM{X80B)}0cu6kE* zwowdPXul)^XZ1`0Z04Rv<8nHGF6^nr>DUkeB2T5SpzKd3y$=F-@P)XP+R6)uB4yy9(V6HeE;~ zO7eZW%awc7DJBlhcP}0Ed_Ea+|KkMFd6ssT%#%-SGFSg&>87iIO*UJNhC|4Pt)-HX3o#k;r}fmAaaK_McPT zp8;Jy*VeOMw?FX{cYWd8GBbJS5qy+(SA!&5O*n-TNfFAc0=3$z6>&1CHL=&Y;3Af_ zeM_!5!9U}5y45WBteCDZ`OQG~4_fd3G>4H`wQz7U4NbC|8GZ<0Fg`McbWX2yL|McG;8l?>y<)$|0Q;w_-Cq=yOCIxmXy)kiTi$F) zFsk=QY?QW?4&WecGUN0tG(!b<^&}SkEqd^nvVKeAXEHNwH-%ru;5a+r%9gj_C|)yd zA^D^7D--h$jsi#pxXw=%ZfjcddFFV)Iw(rF=q~KQHlXIC=6t^=0ch8F8s#XZs?4E4 z`hknpXz*sgk=B5b@ahnl_SxGJuj*Q?Wk32bmM;iO-XrRVxfe#!R%b49q)NvciR-&@ z$Lo#y9p5L{<21UO8qib%t+*+N{xpQzlF zJgUrr+LiMxjG$@VdMm z$Th3iz)TyBY1#9l`osc8$mBBNAx;~|5IdZ()1@7D4{l|69$+}#Q#&58NH*vQ1j8+% zG^AaEeDz8|HeXz{MLKt`J>)b>XLypTVp(%uj(8<ds-hlTo@o*`45@;@n)zZ>a3RvHGU0guQx$%Ui9@ha~; zkygjzFvCO&R;NFbx?uQ|x>?tXBB8CM`^+)j&^`fa-A#@a1Wh2zCAgKDBHvs!$E&;ud zsP$kmW2ZhTgAu4I#5Y3opXaeKkH0O9thYAtPh~NTB9Dt}GIHNP${%3p3m8ZQ?$&x> zpZ3(=pP{LEN-oU`h6fyd5Vd*6FlIGpZVt~v4YFld-W0OgM6ICC9Hk#i;80}1enKn0 ze`4yUVPp^g+Fl5T2WSd`79KhRNJn4hwh=gZrrF2btnA?6OcLPg<=W%q^?Vn#KJLy5 zva$(-$>?-?t2&Nl8=Xa#XO~tz%a*(nLree%G9+Ivo#kSFCus5eS@6o4!C)Lm9j7r3 z;P`!6E_!-7Wa?1i>H5U<&h(_b3I|Z-L-tngUqc3+TFvw~0lm5DT~2ERKvf6%+eQOH z{|T{}!HFo0==D2p$l95R+mzEqcjGdO-zAw#1d1{A#0@85;e%e0h4UiQhNl&KAXE4Kuo@p|L6C5mtLAl{!vGT*6c{# zC6)976(5|d!tu41;-~Twp!*vO|G)_0_5#q3t25K$%tqqAy;C#k zasZa>h|jjzs~h2&qXC^=+ymOvP|16HgPdn;{-bupQHIJfHgiu^xxaFO1u^ynd%qEOV>qr@@FkF`5b3x9msFyEcQ zD0PJdMJQMNO8&<7f#$}N(5-PDV3O5l3t=1?yaPD5PcHhj?bN32;$9pnSB9pXEf~`4 zCD5!3T(Z9!uRUEg0{o&_ljH2egl^9ggYndheSF!XY^iqL|Bj_JxwhD5b=uFo+327~ z-UogU`fgVM&Iq2%-Nt1rpj{0Bo84@~$MryuI<$JEvHhm#MBd*kR0=tKxA{HCt@949 zw17f}caN0*0qF?$kf8FspKUvM{<~3WvMm{X>L1k$Z$NiiL7Ef;i;|MhG*(&RIv{ED z`Fmjad4%3PVw;2H2TfvGiXg4JA_S}4@v?x*a*Ii=q1k*&1e!LfdH>NKlbzx(-;;P4 zG)@Iy*;!5ugzwWvzg_ItxiurUVWUe_v24Gc6DVY6U7b%xJDt7ba3VTPwcAZ7o!ugX zB^TjwKuGpZE2VM$1x=qvbY3aLJ7En8W+lnZ{BJJ@=PpM6Anp6gk8t|f3Hs?CPLFab z4w}?c9ziHj_%gOIh8gEfgaAEht44fY`6#%#2X+FVW_wdn`JiW}(z8|6lp4x66o9d(@JAspwyyzSd}jm?jpS9fv8Og)8>)5u)OgP<23-I7u*>?tNBM z?(e6QZU;z5A0jO@t}FLP+y#^8Xn7FF+&p=w$%ttfdLKo0+06zTJe9v*hUkQD7ScKZ&kcVV{9&n&_7Ma!0g&ngoI7|` zS#^)8)pVZp+e>Jn->B7Xm{;@z6B&@uCe7zjkHd5>yR@R^ zq5t|qK0jP#(w}(12Vj>KFmN&-V7e|Xr!IU9%jZwsYqG7&fw&0hhXJpvu*<8@yQ5HQ zg|m7#FDQsq|7lRGfU!u3QCeBz?5RR=wBb0UEHAF z9DW+%=|6G8_%Ddo^y8!A6~2yL=uh?9Cv9t|xn!vW#-6cWYZRQDu}ug2MLutA(KyJb z6l071^PRB2{nhAb8y;r~^nO;HnQ^C`oNr!QWFLg~@j}MFP(n!|eyOKi3w9wD0U3`- zWpKxAZst9H*{h+ndK-<%*}3&kuz8{LU;qEy^ycYz@%!^1qR_sg2$8WEtGMoj5%~{K zwLWoHg$-uwx6e2{CE6ys{&qgoMEAANQu3q5$mBc+(@*RPl%{H<)FUJT1Aun?&Vf?c#>Cj;`9yL?5Aikl?RCPOTT)O})gE3#Kn3Tof# zI07~6`Y98+I^ta)_vQO0R_Ab5*l;FgkSbBI)uF=jT&)31Y}K_NEtoItic>zHldX8v z7sG$luow-=x&@Mhvou$I^8lmXyUuZ-bIDQb>-3p0^LR^sop+--+c>FuCVEYc%m{3T zW&NVtv`KPYi$SUUva1=Bz%z36SxQ-TPHg~n>r!Co(>NpD-dtwGC_}6199Cjw%R8Ur zCf}@eI8KLO)omXl; zTmms4`^&EYq~E#xBlTmJvn)6?&G+T^ru*Pma{`JvT7$E_B{yc=#x~gR&biUC_7Ceu z9YRPOG`r}&lK%AkWmZILss2l(GiD;wn1iB-x{>NehPjyPOwd0Cp}y%&m7|8e@_%~) zysDPJK2~NEy`~kTeaB5dM7Ut^ZqZMro74k4-!;7Ziyt0+6jiqD^`&@AraGb6c1J0m zPbU-b*U0|>Tmxff70G~7k!zb=K1xgKa1Sl5nOD#3ix;RdRYAfP904_^8d&1IK*rUv z6{F>S>Hamb|0$H^z0+1aXI!*z|M_#NoEbwau;^`7dFO1<*2f%(c@cQ-Uj7~H`6gM- zkFL1SX+dr|m8NFw88#x=URmw|HXq&f8beap9`cJL4f>so(sY-)x__if>qU557W5@p3 zt2VqQA_ICSMKG&(b8k@JoLx0!faTKCFx@lD$H=E$Yo+7{ou$pR1wdwJNL$Ql?x0)H z%!Q{H+_~f_c{U5HxD?gZ)9vPW7m%pKF zTN-IdPxQhfVA0K*1Re>wakImF{&a!ef3~FJ*K~%?p!#W=-mt}S%i&~?ZmeD08xn&@>J=ywaU*l@^#JUtiRZlk zvE#-orT?%D{6m4&@FqP+c@cjZdM4tBR4z&9Re1>^ELSG`V|1d@GKp7hhN6QB4bn$& z01{ey313O@FH=|VQU{*gK-AtzTJ>-A_{hg^{m2dkvn!uBKRz(Icdf_Xu@%Rdvj>Tn z3{KTL6Ug=nTs z)|Rj~50~?wz;*tcL-SvHe)Pbk!OvxbpJNOtC*j<6E7Onv=1MSD*i32x&Cu;Vt>?Uo z3w5z)2+##HFr=V#lVl;!TBkQ}dsZe=|ESqAj|7zt3`D4YNOW<~id^}+1cW~srw?U6 z^7_Va7FrD}Y#Jke z)ZLt->sA<7qclg(t@>6hJBm5Y$JU>sk&v{q4g{^6S^idw3r&>q1Pgke(yTVx;ZH5E z&o0M#u-E9Du_$Zvg^=DJyCe$@YLVUPCpns__QYRw0eT1fb^K2!VFzhhroYApx)cDq z*xTM+N&^~4HIFolVrx|YcMmd+5AUX1T&t!~=#JADfXf4Nz8tcpBfQ^kCJI1ZP%w@G z=z#$E6i!>n{uZ^`o1>R@Rz>n(*hkFAtbi6`95OIEkje{=f}5~Yu#(z3kF`wh@@Y{e z@qhWI+g1-%NqD3<2~kkPUankkjN7V1*`OwOBX$xx3G+;~2E15X0;kc1U{x*%sq>m& zA?fdhlp0lId>k;!;$o4h9bn31C;oX)2P^`=y{etneV4C$ohaAUx=$dd)S+UMm)vos zmH(*kyY7wI91+*2l4|5Y42Nt2qmDYq|IF+y7Se33{BjBk^mdKNBeG$Ty{hpS!>&U@*<<)fHCmER6KxvDYQGHq2 zo2i-wU(-KYE`C;tPgN9b%%47|PrUZweK-{fNhB>hWgEXS$ zF>vd-%S)N$pwd!(BOheR)~Nj~PN=*u8owQKByl`^ydCoCoKWY6+zr~BJAXWVaRIx^ z+@F#Okfz_Pf)L2POLS&{eVjaRT*D70)lg1PR-K~d9~bIyp2jLZ-FroNv|-7-Erl@j zX!I}+MGOJ_0##F6h>%+|9C9k3it+nhpy!PjcG|JY8iLj zL|lwvwhAo2v}W|4B{f!J<{ZlFvBrL_GKA20=-1u*wm#-Nc;AM&G{`1-dxi5(Lr3sV z{k~v=Na>u4SgPc_e+>NCUsclh>TAa3%J|!YPHfKc(GtXMi-|Ld%PKZaKzAfew6)0L zz-J01$f;I55E^<+ep@7<8r7~N>^P+%{wdHVrT=IQmtt{ofiB(g>bcpjL|4Wf{zmK2 zYW1{2bkM%Ub5m5tMX>)ncaE&NDcKI1EL9u6emPZqyw`Vr9Lh8lT@T}vHJ`T+scv;Q z34Wavkvf^i5rTKlf?`VFe$@m?+ehaj853M zS%|nrUd*BGrdhjBcisRSW1$u$yRf=uf}nxYTJaR&e6Qo-Miy*S*r&uCS~*O`n_NG4 z*7p6#1#{srEj_>&fqk~R1n*$1O;U{WLpy@-mb$P)3J)K@_A=!jG^PguJrZIzh08Ov z)FaOL`Lbp}rf2zL-Dl`^rRO$tF&C8>*tiX0GU98YA53~Y^zt?IyuEz})Yt%Pc& zac?gntIxP(zVBm=@fRy^K65+|&<6}|-e$lNz?eWUa~i$tmnIPAtwn}YTG}0;FUhm9 z=i-}wIpiV1d_(x1Vy>3Sg&-Mc=d1la=nBIdXED52uO0qzvi9ArbDh%C4#%Hr^z7p351|K#kctrn?VTJ=QZFqjUsY#K z)-A8L&20IXcb<1ev`o8U(52`}T5^1F%kS3s+%WXyUP$%MiAyVbWhmlARudKxv3G77 zBuy@s8S|VCamC_XphT%TQzCkqV-q8Df$My=*x9;N$$0BWu=2X| zAeIMvx421WVXJ)j>e{ty{GejNg!V&ogwk^nlrbb6yj~S0^}Hm%Ed@8F+B3 z@|PE|K0sf*YGJ7|Frbo5Q8Th$t^A&QG-ul!qu|ELrfGF(+wZ3c;9^g$uBedTzUc!u zEA*qIE(8{&3=Z+Dz}hx1Y=kr2jEc5|VE2_vmA~wI2bCI`n#er5E6lreSYeJ9TfWn| zfGM%yQ93*@(^h_7Ves&K01OEq_Yb@X*c9d_UBZ0gskG-iCP1JqJSey9w3dL%6wjj8 zCG!*mZu;|{uGiWT&znOutVq##8*%;39N-b0?KAamrWGtMB5pA{PCFa7SEd4v;PTF^ zHL?tM5B)oEu8{kQnFT6yl*p(S;}lyX~fwz zd-In^0r1vjcOnEO=51?tscLJN_wX?1hYizkS`iboMERn~iJ&FZ-Lf1Og9hDvRr;Wf z?1PC-Ip|lm9?SH)wD3ydCzFE0Su3^P=HbWZoI**f4)K&S^kISw%cJBlA>hUA;qi`8 z{t<_2rb4ekK|0k$;FAyL%!CKNi4`}jD7Dx9E|uX$msaRjb$sX{{yWD_yKGeh&#r3| zv+El~XRv?>3=cf`UfR>j5!__qY|wvON>58Z=br7hCG@L({%bAZ6oH2X?;X$X`o4p7 zNOpDy0AT_!6|hc*Hvvul&t*DQv#>Vex;pjN&1Ari5(PX<@b32X*Cb*~^sDL8iGX{x z`>(elomCM%E4Uq>szygs#l6_w<1@~qrd?f!svwifDIkysh4|pi>@vHY&_{s{pq3d- zN%m$uFJlvEYZL2nVg{|tdem?t|M>Z>8E_%AMvSsQl$q030MUOHlg}Y&zE+NimtzH zmzH)hUlOd$WK8O=j~+JmE92>PuR8v%DrONuFrNvipOMO7-~%qYzJHZ7#eWL3u!n0m)UwIGMxm%1|Tz3>f;A*(9T4j*s|ds?P+m4$*hU@&qzT> zgZo><{mu$N>ANw)k<1nh_P~k6p|7-?ytuWs{k2uIKEI;w_D-rlN&(SL8(x zD)5#%gmGUCY^Xj{DO}Ix*!=1)hVse!?r5Mk`K@!)t3|i=Kfu4+1C9C88(>ON8nYSJ z6Og!}+AXHR5IlMGHnP2YlzEAJrrk|Dlt&J@e@UK`elIS;Pl(3Dl};p?6$Xoc(LRG*?h!d7 z+TZaDZejJ^*rnxxP!gIUV6#~2%@NKF24p`=H1yolYC4E&7w?5Ab}qTacNW|-Gc!4k zTZLDwc1cBCbmSdQdd|tgBD7`gk4}-jYe8zXmm|?YcKSF9&`;tw6rWU7>2U zM?godV17V~_;V;QG&H0v>V;n}L+o3s4Qsd%d#Q@$qv^ zcdB-ch@=tO^=Q}|b3-ZB!=rH4sr+h&WZXplib`nS@8fAh5debBGqH22-#d^G_uE4E z9iBh4?lw$Jij42hstxx8XP9Vw);L+7&2)SkC2aDDzSP6g(`VANXtSS1hxtdlht`s7 z!+PexsU{7_!zT@~?Z&{h6kP~E4xDA$ZvN}SQlF^cS#P_&)|F)B|HxdjDg}EK`dKavTWal%wUutUnEWY`d(rZkRgG}3Ep+n>JvQe8!)*52 z7cNV76{$)3_3RV%>#3QZd8RuJK%g&VvC+%q7GB;YFM4>Sqw^QItL5HX8z%UK6B}-J z23qHsXVw6aI(BYuv8dwXD@$Ps%wICJiD^;ve~MqS=h~ue&efmslm5*QKJ&64iIp_#1orP5E*i7oPStPfW@P($Zg&~|wC*2iE;dS-yDj4ris zdLSd!spK-p4`M7P@YUfC1$$W3C^Q>PC?=8HHdqn41Zb*qKt~4JH z2Wwgcujf%Zc@_l`mOm`}HI0wDVN*RE?ZZu0;QDP# zjtz~bhP*G2ogX6uN zXmoKGGw{C%TNe zLe%%lkYZ=5kZQ8fKnfNhldiW z0j1YOYUmvT1c;On2qe6V^Pe*_=lo{AzaM6QK@-?}?R(wzy05iX>t|NuuG>@0_@+tB zj4T8iiR;{rpH#_4W*Tex{oEN^PU&w9cAYJ`6Xsba9czQ4#B#0BnxzOu zB&(M+5={v2(>;9@P`T5$8IX5Jv0>H@Jdn`Z@!7ojrlPrWD(sa!#K>^8u&}ovcl^U< zW4dViL9k4jwHI`kL!J{$(SkUr;(uMmO<=gk@$ z`)`x@N41C@f=y+q=|4z`4>@XRJNZrz$-VbOo%MsaFsbO&&u@?i#zAPIpb3O*XCpGO zD-BS9ECb+E=zhy4M*1B6lxVw010VK={E~`FHwNPn@R~Wo-`UPd7;=Xhx?A#KpXT!Wf1vgNjpKCHn3y+X8Uhb19{|Cga>`V>T)nkHUmZOcVPwmvGYb2GITQx=XN>lANta&|J{+kwVaZmfn~wwB^g)(AiDYj{Ltu}c}Wp=o-s@<`jN zbu4gzb4)9xkVdh%%WXXS$CR#M-%}+ubbi9I4bH&}qkzrR)O=IZ^)z}bF z8H9^O!fv%RqE+EFwD3;2SLyAp$lFzJANgFIQ;Jr+Bx)?%EROf&uWaFDk8_9t6{$}H z$BXx&)`rc$J>?4(qm%RJ3cfg*#AiguuKLI31gHyFsl^|#e3U!glTN`rLLT@@OvNT| zW_76zSrEwWj(7f!8&S8oFRmphnE33V`#9VQdGr;jf3OE1-kreM$ss+=sa8Z}Hz&IH zmx^Q5_yUPuC7QFzU?ytmrA(NQ@AW>doW-M~)$GY?;W*bq0~~~MK{s?IdZ91s?b~_DttC>BwX=8xeHG6{X$w;;8~laZj3o)k72)xKL9PZ*sTsI}x2OL&&fIaqJx zM7XKsV=i?`B|E-ZTAGPtI*w*;l%mAQ4J~%rh95Nt zW$?4))M^_T_jO|Kw?!_>Wl z9NLYr@6Pube(;H>OxLTxDU5|#XM6=)vyXEWN66>@j%f@)ST>2qlFTdIzgcyBxAKo) zMM0Pfh9`qEa!G-McT8EK4{QU28b6WPgl&d#)i#!xjh+B&C~C{P4q!&9`_=gk%_}tN zb3p(O8VIQFjp9{7Dp?XI8|P+G0ehS<+r7IjOv%*^EW((&uEssr_>DGMCNu$Uh-JOA zP}heB@Jxzb!P8^eOkAn&0kv&C+_|%|`7N>7*=v#z+p@1{>O6AWm{FLsqI6xowxH#t zu|;Cwd>{dF!*%_9UhXb-yc?SR#a*4qd`C7);m@xlqnC8{V*Gg#Mx(a$MOWqj7%|$C zBP`}&ZgS!7lK^Uz=ZoTZeZs}}DRIi?qpW?GTTrzE9MEfbKtTfMYk(L%3nZh$01?P` zssVH5+FI%Rvp8?)1fiQ~5u9|3!xhVSGyVm+Zq=bf+zj*9UpPSSGhV9sMdTGX2QUzzu65~$XD+Q|&U8h&} zBvI9?6OgoygURQ>t4-QvgLmW$tQ&YcbM*Ji+;O##1PQK~I~qMIlB+@rs_oF{wdbXO%>GD%Dc#vL!TfQ$m! z|Hvnfnavtc_{>nAk|}xvwod8dV3=>4QwdN2KUt>Y{jSW)J7>2fBP*Z_*(7_U94FN9 zssy82oyQuCi%)SY6@+^k=D8CnU4o(C-TkDQ989F;Sgz~%Rg5s?#f_FB&VEPV`1<`g zU5=O)_;MPmmwPz)be6nfW@~-Wwa{EsHre{j1jt;y6g$@f%F6Xq`2%+nAI+Wg0AIBa z4PHoR)q_gQ(Nl>DsL3%|i!=>;d99j_ z)wNZIv{H&)p=FY~dP(*V;#9F;mGp{W4X_bS21Bs~10|&}_lE&RqnhQ)ZCTqm0-6zFKCMcs24z&9A44(Qdj_W2G>tY_A_01 zV?awNz{?Lh4CD)lZ}`Cmf6LGzi|!~!mgzd?p(Z_jv7UiNJSxJS-FJD!^*TRoBul@;sufLml_&+V*mh$DB<=fK!j!1MLuyPwC$Sy)=r95)>M!-|bt`FBSb+%e%o;jv=b`U^_Ic!7#rcZbI*pmi$q%2%2G&#l7)Q zE&ygY$ZV+vL?~+pup-}lMnif2i7-W{Aa*^!C{jrAtQs>mPC`05Gf!C5qN*-0e8Mqyw z94$H4;rCKcseX?^rT~Of+^z|>th6v0-7tsY1@ zNcJ<%BJc)K1!Dj_U-;% zcFg(rUsL*mNF1o3LYscKk>>M|;`2?Yzq!BGYk4<9E5*#&)U+UdU|Ql8`M~IJ|6}w{ z=v22-?nsb8jyipF@v3I;mryHkGdwd7FdI#%-Z!N~z5w?eQNM}%Y@(!l&);z4z)xd_ zFLnE5NRZ++IaxnBCiKiC`l6#Y$^L$=v~WGAee&tVtF!ykkD{MQ8)xL9@JzyACG{&gqvvDd$U_EUkTo9f>`G~fm<>)$`5R0NUk z->%nLz}WunGcrEff4aW^`sQ8OnOFI@>+e_pH^_gAbeEu(HaSVz&)bErH_OHS62 z=EB0KD_QQlg56;j?y0N#eg_bS7UCc2`9EJzQuSh$<-r~RXQu9e4S za`lVC4Rekd-OucSUPoygZn^HTXV9~bN>27jC{)AHus9WcvHC*zbKKnUen7;AYi-V_J7~1*3s@P)|X;r-cwjICO!;q85sVNr&Ab&ju#VV!VG;K}k z>z}nWKDGsts~K{5#_Pn1*>_f!3JLq4MCsRg!)5i?Td`cYF?to8Bxq?Q4eN}h*nJ!U z>oDQN)q{uK@B3mF{SK}a%8NQg0VJ{802C3GF#FC3&^Iwwa~fUpVJIB`bSt`7L5w## zQVW!y)(AVfz7-1biw7Pf#6eA=_!O1WaRb<}Rf_eLznw`ExN)y!PKMyPTXM25p1LbA z=)b?(&!+Ur$6mZV5egrBL3N^9!ef~W;{ydAPNlZQ%~0>0LRVl7moeyPP-X^epV3Fw zCFFC|Q@Wos+zD$YtwjJa69iih$}?h=j!&9&PA5}c*6bVeAVGT{rB$*yf5==m^H*7F z1!LT&ZoyZ|uj6s4kh&7*4AxN|mLfkUj9Nu=F>hHuX;CHCkOlSM21D zZv8A8C$t-{id}*0ucrah#VHZ=bU>#c(u5Du>zZ)Oq;g%{Du56YE8rzu$D6S+sv;3M zcT@#<{~|LVOn6ONeq`xrod;oTiK}4_i)jhMr}{}V2=3QGy}AsccLpJ8BZU7hMlz;E zZTfC!&OFUrAVt4|GOzeG)e<(29GL8IA$Q{KdV+5j6#9kDlR8eUVZ0ENXrrxf{L*vi zS-dEps;SfU#ANl#8D8?k%MQ6KM#Z0DDdn|OMyj_)7oXdN$5PNcrR#NBrdOaw`rMi4 zF?SHp?b|D*?TRs5NdgiZkvY!+#{)rm8}YIl#}d|Nf-G&w0ALHQDj`r#qqzHSdI`!S zOK%u}XW*GLMjM3*xzc78yuwNd7lgY@fka11>I$Gfm1Lqtf=O>9%F>%XL?@Jg_VEg) zr?n#Jf0m{bax^tJ%pD4|JtAYWo>U9l-;Y4(Zv6q%6$b-E6&C8ty(UG4lk}_Bo=iAm zU9Xx|6hfca2EYRS)sg>MRvFLqGUZxI!7GreGYKm2p*K1qV@e2rP=TB-ADGTy4)!r~ zzGl2u3bVS7rRB9o4YT&@7qXl1kzg~kh71Nolj7pUw76c5st<>e3qXS9$~LTX{oYM1 z+)o;*#E?xrFw){ZTnf}d9A{r$xsILkEd>@-H3Z+M*TT2;fQj5#srNG!Kr2xAZL5|= zixjby-_MH|BfW9u2)%T}c8uDy^?Js(Q1>KXDeb^QN?v z50HoS)=emMM3vI{E!{3}}h}SnvHJjIx9mP zPa*|I80_2QN;%(>wo6l22${3TuaX%(jg;EyYD$+vk*uT_i^Mz@>j}L$|sTlQWPI@@`V5TNclQ#n32&0Rj z@dW}6yY0$>Dtg~TRO=t~v5nlKn+{^#lxKd|k*O+jI~lYmZyVrVETKt3uV_KB{}H$Q z`1`Dx9>RgZeh%eMVz=B^&B;O;z2wg;N>O?9Kn)F(M(kUO4$U}goW1PU!B?juMkOr^ z3E@@$^}POX4pE>I2fqvu1^a@CQX$ox=fE=bgp=;`-8FVZvF42xXVNuze3d)RV6FCUdyr~&6qC0m*G`Ad-N?K$toa_I*9QX6qU}rkx<0~ z9U5s{c-59YF6Q&)FTMFcYq733KJy8t62iZ`S+A$DP=E9j4JU>COZaxZ+)WW%KMLt(x6zV$1Ly!S|Wy zsHUd0+SAzlfjDdsb2rQ7&a9)(;OVPGORNv#gl1gRbZ_x4aW-=Z(A}L z8h-!Y%x==+raSi*O{I$ln-^Ai$rYc^z{DjMl~-%lxujwu67WI!tRlnGKc>*E?VCF+ zKOlhXQU*ye+3N{R`O2tb-w~n3*vY(FpD8_!&mH!oBV0OiEe|j9Ca-VFYA&&_eQLV1 z+uOV0-}g2@P)aJMv1xM*atNs1f6sYOqa`Yxj!H?3+ys1RO{RGOO$^6u4sqH=?h0NY zpgc~~YGB{}K1kP{SNA?YJ-F571YL?)Y6>1Y=|ihqI9T7$8}*SefmCRYLeKwKj`Xfg zMmJ;=Fo9xCn+uK4T=5$Mde$G%(Benbm+bzS@kip8Dw?uCN9Wt`91r>Ip+QeoQqM73 z8kWLtNt=2jYLFTIFil$mNqxJbaCfQV-TFBWk$5AwQjif3?4d_gZo&h1(g3bGe%Cs0xa*8MoR8Pj z6YZyzV~nBpVvS{E5EoWTX1orIH!!PneX=Dwi1RXz6JuDdzpR)AXy4ireSlF6ngzWx zB;%uQAeiCbE;?I&g1i9q#JSbUmlM3{N|p{6^F9M70kT5wNtQ z(go*B-~_IL-C^`OyDye{#AVZ?KMr^mK({PTS0dkzjTutejE&0my2%^OK4Y*^NYm9Y z^ajv?)y>0`QOU`G&ooCnAM_17F&We>G8L4hK4*M(`~4YPEe zGiMl(7}mH1%=AQnq%D910VZsO6b?l9h(t#&{ucCOf+f6y9&E6tDg6KgrM<}vb#-PHcW zf0 z<6&BmSqIG}yF(`uO12-0RH3i?U04LHXwa^NrlVo{+r+ph+shfAuH!_9f6fmh5{Dcoo2? zilF+uR3R#0_{Qm8iB9rg2U`6B>m43Y3WatTR+M^#U)?pP&AIVHvd5@ll+*H!@3UX& z$3@t1p3KHkZ4O-=o34EsSlnXY8RzUdxse;;?AfZOP+1A;Z@VVYr+O0SU_cAfp2#5w zojVWH%zVi9)fXRr!JFK!!y8o>>D}zLl~M744}%K)qauZsS4Zd*!l`TCgpOlr8eWwh^bi}89MvZZ+~xXeq#5MsUdXQ7S~_?Nye!$B17t@JhoUkYCd}KMtno>^d;X54 z1mALVLy&w$mJQbN!Q>HO`Ga%DdTvLJpoOfE|0#nAaZVOHw|TTx8<3EvBN=ai_yx&f`IFbl*R`-wAP|6r+|Qwk^ieX#|G z`>{itxRVYfyhts&0T^1MlHSg;pAnyuH0`o0lFz0Ew){ zb*Z8JM+yNo=ckQ!5c9MLp8u;M{n$#gBmUsM?Bx1ZHjo%7GvOIh8WC6T=Oo%8L`* zRkK}B?ZSKy%nH`kw}8%KzpfHj}1Y;I&J5CxWx30&@id&E3(t z25PMeQAR=^guvkkoA@1_mo)U9kTm{hW;VPbU8(>qmWb5i&8Z|E_=6!aK7LD@FVC{qFf@8UuOrpfYH?)6>*J3`ePZTrX+|$$2#g zpW~)W_&J1g%bzvRAqAtw;tE=J9j*VGS|a`B*R zO;2alZz+y(N0dfIl}0t4)R;*=KW(ljo^Uh_*$+g~XShJj03xyCP*IfFUVM7G3HNoh zHVm4>OQE;1v-rAd1J06hf8?0laM@rt!&KEB|K+H8D=P!HC`roKZAzQtRLh((%#1PD zFLqlzSK5uG>7%^%Gx>o7)s|qKwM~xeA(N_upJGq)wzF&X@WLlW?2CVL z0hC*Ib-|W;EG-=tl?HrwKK~Ctl5%|KC?@i5)WO<3dGC6c=onEe$|j0uM}^o7-3FTu zMTs<`*_3tra4oqv`oq}V2Q$d{N@x`*^3P{_WeoH8S%Nd#*y@Kq8AMqo#eXZhM;7&G zO{r0YHum~m$zpNYJYA#S3cSH3lTfy)_22yt$1vGm16!_v)4-T z^1~zco65qsF83?4m4C66E(*Ek;$bqAZS^2>jW&$w9^3 z>CTywO`2-&2neNpuMT}A!hlqsF;*?_%y+ZMo3LVwF^Gy5X&K|dGe*AKf2HFCpLj;` zt5K%$tE$=6U$r!U>T3cr9aQ??iY)k5+f7%P4!_dk7NdQ3C2HR8#$d#>Bo*~VlGaZ> zuSyHA5Sn)g3HE7%Vx5*SilUxD!j;Nru>6LJljtke1ig({l@_|F9IYa0fn#i--X620 zUQh$>q=iA^#ZzSwbD@~}G0~Ws5$~cJ1qTtqpA-_8&et}(YCfspGjQnAF>Cr;vM+F z8+}eb%~f)lTGeNAEsT6zJI!|mJA~j> zR(Ye#svH(l5na!ap=zr2bOk|%LX2TnFviA0X3v7qcOS8AVUI6_U|hwVzvw6SAT2i{ znc2R_zPN4_X_?>p%TLFxaac@!UL(JWwg0|BtO*+hb9e`*O%~WH3J{e()#=F`#~>iD zCCT&dR?JYx;(vB^j3D!|?))kS&Yf)pdU+j)?u=Ek50qGKu|~#sDSS*rUsa)Nxofme zC@OW##@hH3<9=TxX5fLEN@&Npq(v;$@`lUsYwNLGhk=XlT;>-2z4++A z7g$4eUA<6hE1>05fD{t90I)VqOe6N2^mo)z#O09O^-SkL))8b_=U+kFWwx=#kVeJI zieEny_L^^0sj+>d z2(ZcdLRY|LsDAT=#$lu>bX=-IuG+6m;}ppEpQBLgJGU76_{DG#eBOuAZfxD-dd$f@BJjtlV3roHAtD_ z@`B`Zr7YM3)ctf}G$Zwoam3p@OFrDIOp-!sY!oI(&!z`imB_Nk-mg9F3^byOE*QV{ zchaOJa(kIx)sDj`K4)iIP)`=|v|j6N9?dQ!7+t%h+G8>F)19+fl>MOQ(T4V1-1Bo) zG5(8*}33l;~zJ&!gQ&!bYV1n;hBJfaH|g~M&)n~HsG zZNp28*OKpNc=(vxNB;Eu!`a^4MlD<2>YR~qC?FxjOhUK>Cv0t6FulBVLzTn%xDpvC z@!8Gt@U}9;;DO2At>gM5rS?;te=rgX7*P~IJKe20w)RlX`wZaOF3wKZF$3=gPfmE} zNKPhp&t7~aaiAMH)RQyIZ}4>rQM!e;9Ddig&q}4)WXNi8I}u1+R4-6&DIQ-dcvJSB zBH(!XukHQf3TZ~yFwZ$!J@c3%C(bdH_n|Y4 z@^w-KWsgTV_WXJKjv8BVFUO=!gUs870&jHXajpgf(eSm?=rbt&+Jb(o$SE zVV6^rzv6o_rcbp*S7x-;k@im)WQSeG7qOSX@9foZLabm-&m_D{;LLyjjx`8pUGvel zz^Lu;e6FIdjH5oqFV}SXyrIZOIE$(SX=ba@#N~bfdQq1lR(K3uv4X)`H*O{Cd=xZ--AeaKfyiU5_Ggw z>e(>8T~j~0{^|A$++5qdH$c4YYNB*4uNXpu%sGb%m5P15e$wUe&mOlqL5Z*NwowVxd6LqW*; z9{s*yp?4$;su|DQVN~!Gs!N;L8W+#&cEL)dgfrNY1SDjVLMo zhu7iRe;LzmTG8KXcJuePH!lYJWM+Q~GGytq(|Dc6#F){ibN7*ab*XSAz zT}mG&;KF+>n5xq_fzG*We&V>#@UrD0&)3DE6iuDTEh7Ui7)_jg)IG7CbUBH(3m=Uh zzHBM28(ojrk5t&ICznnH&2~*7RXojs^4#&jaW*RBf_vDBq1s;W8Jv%A?cq;_h$$$gDqW z?2ptll|D43ahj;zv}yg)L--J)>hNPY3K}@{xP@;$`F8Wt@2lr6kYX`Ew0`f)<-S|7 z0b3>gc5@B;tBE_V`4P5PQnyP|D+-PceP$Z-#l)%o9nFvPFw3GMy3Ropjxb+rMzxv8 zp=%J-?%T;%E!S5;F$nzdYJv9WfeLX3@X$Mpm-*+ERYSB=6!abU3HkChGW&f--Bd5E zl)9mI;KY?~H~SD5^;+&h+#cd1n|}6X{e+K2ylsaRH~bM5p-2lgLzH*nWV1)*Az`RH5q0&!@Xda8sFnv<7d z`6N?^2XF2j$PQJpR}@=jGAuk7VfhQ;{iJ}K`(cMtiYAX(nl}GjsfZ=AoW4PHPEA_@ z)$aF*Oqs$)VToe;kIDOKskZ^=-KC$cHN3ct^W*GzR9-*X!jeu$2V=Ru7@S9QJ-w$w zls%iU9AcuYql((ASp4<4yIi#VE1#%n$kL-*(b)T<$LRSdIQJjrneTHp3Ub%6qZ+mb zkV~lei+NI-M2|e0?a&7YCMG-+1C(#`#y4?AUt4KzBIEePECYDrV6XijTYuuyRbEyt=3 zYo@#3y!43Xqiu7u%;G+7@2?}SOEm%RH-84ug2d%1xl7({-@LlvtwCNj=O0tcs`PA) zH}-j}VzEL=AE>7D)B>4xz{8UPE)Y_H(*Aqzg z{)(hr&;M$PNq7+~C7b`eWPrU_Qlegh>$BW)UF?%@eJ5Z+Lq0TQT<65AXc^0ya!>-VrU-OvU{^n zrT@j`*>1o3`#G)393#MSJ?AxCQ|$idXyyjN{sMd?hIP6}N< zmtmwOX}7jTnwBME(#=!)>S1fOM&;!8A8*0m_0#U0g6c0T(>PCxlI0N~@1te6r!^)< zwd_pl1K0j%#UGPv9o9)9XO@v^Ej8gU|E}BfNKQ?y^rgqqdhP5J)gHfEZ#IjAkT3W# zXT~hCJ+~;{7uBIRj+v9AFGyG=3JX+mL^_z0MAB@%{pI|u&7XC0Ab$imbM-%q?M8;L z9;@tWpVtmSX8Nq>l!~Ux&e-3q-+tBJoW3%>b#R#3ypNe)Kd3{g@qCpkBa5QYy~<%h z%@kf>e9I>$_f;g+@b3h=nt)iCGo3QU%{fsbXcvl2C^zSO0xn?9~b0 zvWWrrZT#_O(xvp`XO$>2(c70#Dr4c@IGa{S4kmHQJnNohtkc`Z;gwS#qJ!hXajtd5 zWzP2>ZJDZ)o1uvh z%N08DZtOnln)6GBIVT5`5dL3{_SSjb`R)q^A0%YHp1jg2XaF&;SZ=J~ zzlGYaZeIG+I&!h>CtBOR`bAiHEOpc;;!zQgp3xQLi)>deEd33Y9t`&iNTZ6g{SLUM-05rrGaEOYRtcB9%pz?y*) zWp}@ab$W9OzR)(3bkwRp^UzKiop_MChMi!JP^pHatG%~Xt(O<8rw>aiR=F;*#IM;Ue`$}%zPpl(X`7b{ z$F!Ye`XO&J5>VHTGrdL{iOw)`VDc5mxj$a6JBbP7Efu|G6rw!eZC>5()dCc}%j_^Q z7@6*zoZevW>Uv548{T0I@?>oUv1qNB^>FfNN8Q#h-lRQfacw7U>sW>xVjMb6m$D@d zL1p|QRh#LaY|MCqy*>g?ahUdUmQI7$izj(9c!L?I;T}p%r}O_-qTlJ`@in8Eh|cl* zsWT5*eqX$P=YD$jUuH?m#TJcTdaRyZ39X+(9-L>@XTW0yEA)-rDw1zDIP=$7a9h}2 zI+x)4WYhOqhMI&Rl_#%9?rZUD-#sY`vp(zZ7zQ zIa*EY|B}4wz5CvHOQ2So8Mvt*guo%K>)1J7HLtxiXPT&pC)7jX`mopb99!69BQEao z9+3x>neL;n({|uWOTN~e8AeZy5^MI}rxtudM|9r1X!P!@Kl?$n=!#8aTbX(n#koXw zSJRr0UKt5K0@vpxEv}lU9iX#6IcqLb_}c79@x*-T7t78nA?nP~{#7G&^qLD=U?Zp)ZHEK}hP zsdxm3#xuzAzl2X&e)Z39oI>+^0LYEh-9-hQQXJ5TGYs(WoRJI%8U z{kf-$V^oV794aSrUHZ}a+II`a%SeaBgDVr5hWZ4wM_;zrJBjy4Nmvn9#ePK^D1RCX z*0?j{9GjX>RV!@@oW*MTC@RJzeAJ84R6X~ zXX{HmW&X%2ioqpO!3R(E<^Gbrwv=opWJ8n zxF0IElp0_>>PNTV=FHe|Y*Ak-e?6VfY89A58pOI_gE(*^*TP@w#_B@gMBJVx+f7f7 z$e|Rkxav{lM%w1v>ftQoud5?r4V8H-$1lR41PVw0jEm1lGwNFcSmI#S{}s|(nIb=Q ze(eY%j8wL(*SMLpf=buSP*hWH~d%uT!wh{I6fM?-J%3#9DLYRE=b8=YQe`kHBj z&0Zfn^2IP6WP(hxDr(#vLo`#IXMp&I<)p1Qz{%C|8AJdH`84%UhKn*PrNt|AoMa_N6Kb!-AAgc->9Pka8Av`iEZqa z5kKo5I+wD_A^TlrXh25$-cIgg)n6X8ud)wgO7{3_7z5T1H!g(t;jcgKD8B11CeI(<3C@-MlZr{5mVF|t5qhID!s`z1q^u|e0B87lL+?(ao^0_CrAGe!o0rJX@G@Z{Zs;DuyEj7N_SOwe02 z-!5+`WlJ^(WaRX|)8!tq^2Xjui7rx2h8ED^-`_~4j>*ehWQN9OOsdN z2`4UQLFtTJ;x10u_@tFM_TkcE9|wQ`enszl>O{s0xIC?I%ExO)SDW6?65;>D(lz}EiN;8r*aK}*3TLw&7=~gV}{|{+#Lg600KG0HR9`L9e155 ztDoUgZZ@)*`PhVwdL94X3866yn%mXYOX(T+`tRRP%o*+yyrtjLzvY}vPK>HTz=ws6AO&q)`VeaW(;SMd^;N0U z*uT3|Cpn^c@b(_=8aY&E=U7ZgT1BF9k;i877{IUSSd*73A*H zj1t(4m6`k1kKf2T3Ytx`2_zd|%R5;&JG}4g*1I|1J=ffy-pd@!u5vtAn3E^CtsYzpM{HcEMEj*w&8iiBReEi5fic5&ns(`OhGp8~ zFG=s&lJt1gusk(kB7tHnZ=SCmPu^MWHH$LR=2@x>Ml&t)e4ttK{2ad}i`MIsti-M! zeVc!z-}{MtFDk? zGW{j=@`0Vsx8K=X6NXaR$ z2fH$Y{a|v*DQ;6QY3{*wBK_gB-_ZpeoYh&r!Eu;H)XT@J4pmjz+O*T#UiHIU^tLm~ zpS;KgI;eAMR)d0D5?pH^mCtEEd9=#uxEfwm>#!w2U6mkL!NCdQ&5OG+=>eKN94Sud zOR09fM5CyHFMo-aWd_4hTyJM8@mmzXOlXIARYAS@K2W_o^i*b}yBzkQgf_i3E!Pc% z{3l6+Jdq3dB$yA(ED7YvwC|e$9%#dI9U5|!5s8D^cS`I!Ga1HK@V%)MXy)K4cB^cr zwRy?V6s%2o&V1KNZmT*YpOt~!u)wR@>;(e=^+IIgB|J&9YTDZeT4Tc`OBn%dN#^e2 zM!{H0knW(sRc8RW+Ix?)^z{9bEAfMReU{_$C}f_=YxlONTfBwz-h-u|cz{e&vWM{K zuaOj)8J5~fE@M`K_tge-KKfmu8JfVF*rZR??er5Eur5KSLl+u^7TC$miB)8mW z-aDz+a>R?MmK}@^hJW;K=w6IVyxohr(o?o2n4OSeRZOGOj1?%s+j`I9WMUEMJCpoDqCY{~!~lr$uG z+lY}I^2p+#+tb(6D~2Ajj6SaAPNca5ik6f`I2)R7w^n4b0Yj~4_@=RC8j)cCT3Z-*%yT~Gi*q_54} z_hEG*yitYfJRQy%d)nX>r447>IXKnl;bW>_ug*QZE6Kz-*sY@YG`v$gw0!9(Va_jg z^k8T4sO~N~N$e~xm5eev<`XZNo>1obxHrBm7oC$~zv1=awfNQbJ1f5zh_dwkjUUmw z9wi+(oA(mJF+sXJyx!Zpv(D(>!&ffP;U@`!Ec?}y*Qb~3I5^#-8-$k6XJ9$6T}(H^ z5azg&YnW}DPnWN(9=ly(>M7Z)K31>l=IxMRk$nxDSt#3b+QtpJyI&S}lO>vva}*QD zM`(G=tTx-<18=+wzYY@PF@GKz!=T)IQv7N`K~B+$GY`&?(|qjr>E!qoy<7Yjwap(l zc25j^uKX3*5MMd=nLOv>=yW~IL+f!L z{@Z5-`D)j&70zF`lmd7^(D0s~;SN*jmi4|ZbMzy_Aihq~mH(aA{()mJOPluOFx91c z(z!Z;HC?1@p=uNfq&GS9(%D1$WIxr#9h=p}^q10y&78N;kb54c3SA5D2{C-O-3k2~ z;GnWkmT2_rz}pm{^eahCwbg@E7cWYPON@6DPUhV;s3G1Ttk~>+UkxBldIzM)z*$cJ zMAsTdmTT}RawhL`s}{!s|a2XG~yZQ~ujjme|x1UjX0sO}*mBO#OaeGxbtvKue8 z!&BD7E4KJfMgZ(Cb6(o~$(p2(R?HQMHNQ_9l>$48K2nL-`q|qpSz8GSXRNA2z`0u? z$Y=9QC={ctc9tly3LvlOpqA6;$yO!27tl02{k@R`B%rL4Z%i9a^&?^G$*JU$m5>vn z-4>J7!%!Lu6CRM_X;Bh0#|!Zyt|!-+$>-euth}bhp)QG<#3o@Yr(bllzHMiEM1S!j zOndE3230xh2vrNK2TNim*LN(}ldDgL@o)TQ4dB{F!{rz)JzX@H1=?}f;Odjbdd~8$ zH#R&686~d>aqq42%q99JuI~nNvHP@2z9Ppl-S`^i-x!OoajOyMf`IOkBdWkZDjUWk z-8lysbv*Z)HK4ixSQvkCY^jdCQ1Az|9rqj!X(X%U&D+`D&Mc^~xj3m6mVA2l#fOD| zj_W{{9i@2hBVsyqnB5EcxP}{vGR7_j zgTdIw{GQ#P`@X;5>w16g??3RnUsqQy%zqpf|~Z2UNSYG~Kk?ptU`UHt=u z$0xNCvE31b6TsR{v@cGTLA*zF6&fV}pr{5(CAHu64X6r5j?m8HUQ%;Y--ANUY~TJ9 z_X`s&>VqQz07FofGqUdCW_GGaMg4zXtbUR{w!#CTGJuGnoD|XJE5d#>l@55ykDcK^ zqDOdwD}K!dy0c$uXFvN*Ear?m!0L7{tV0q$>DNfc#u#^g^^FW!>zAz^TICX6P<`a! zyYUfU-)a@g)~T)NKKJPJ6A>R*655M9GtIdS^c(8W^kd)F{!0rW+VSBm$(wC0RhfD+ z;xWh2)Qsm+C%f(gepty*xyO>1=)I$lX>_T=_m2GtWLdW7v%K^(BiZgOAzs8~qv26h z6&{TwMNU2FuFC1IvU_!%GHoi-fp|7nRIRwM++M8laYZ*y;6${?H*4-VCBs~4Vj~Z+ zal%Mo&?@OcS*Gt>h~^21vn)vUDhcmJ`+c82ie;S7v8+AMoarVkv9s>h*zscQ%_fhK zVN^{`jlb^6rn0RgqI4{sL!hVgyMn&AZ2ZN+{zM7Rqnm|YEi=6LDHOKxv3cv6g}qNrzi$ffUND#T7p<^K2NTAl zO#IbyX*A@47y_!I*!yNpyQ9}6<$nwT`9JB?y93BHwS}fOo6WvzFW%o7?M3qoty8& zTCmWoBN|=7Iyg}G?p@R-2;8_{Uhpkd=f99mT%7vg;yCHv?jAn%Hl0s>umN zof0UL)+xzWv6&HBj`v&r(EEjc&j8Q?UC9cD>^=Tcd+qiM--FiIf12n&i7F~sN<3Ds z5rt&ARX1eMfxun~{k8SRNthIg{>|9!d_l=GJnPV$Otvnux_gOz%{KX7rq6hX13cEQ z`8C`)A?2LEPxY>&wFINtjI+WXi^0mf(d#{Bdpqs&*?~;qj+=g6dIs6u*KgtC3g!E* ztC<@xL=hPV79!crvrwds?srUKT=QSQ0-q8is-@-8AAkqwVipSHt?L3i}3a*le7t*mXX z5ZO5boqvEY0903STWo`lh_Csh#pp3aYc>V7 zc!GN9{`5~4)cxNSI{y?AEVKc57g^j^=yeOw)^r1zVnT-JN(dn>ORfkV<7KaNtEuOj z;cj6u=E%4KE%y4(s_!sCUSwL{VxZQJII2$Y28C`u;&IU`7mPv}ZNRvukmDCMUyW{+ zPSpp|s;)kTjWz6AB1}a=|G3bjk7U}=R`~OoXaw*Ag_|JZ>rcB=*xO6db}-S&$*sVRe#t%YL$5TqhVK%#KO4rjtsq6wxF{ zZGh=1Z!HH|KWtFTJg$EQ4vTLPEG+QydCoBNWbetDiZf_GPt(}ED!J)xt)Ai=l|Wm-gHl^tSqZ9P zda7I%J(WA+2y`8B+PaB;%5H9M*frB8H~+Ncrz+Ed_xV2gHd3cYU4VHvdp6=J)W0Sf zTS4U(cqe;Ay#%9P5|T%rlp50gdg4J^XLF@koPi4~H?ov#oAsow6tq_X3L_ps3Rs*v zPjsBTU-)YCG%^2JHTh#RdCTogAP*EqmNDdFr*)GtQk)SGS^2jXvH|jpc`C@a-)C5r zX{U1s52iTH|1#q^m>%Tb}a}Mo|KkRx{e*lq&;ERI|K2$Nxw{ zN6#J9{#5Zb=4FPSdv_|h^-Tdp)cD==Zzc}PSJol?=#In$J%Z9p4-wbeKOM|^aVnsl zW^S|8o}*yjwUayPF2=F}(zCI=-Agbt323jF2*T?SuBw(tL?z)afE0v#DQ~TVHKrfR z&&zzi9T@u!&0!>`vfMHMPWoc{DYmaloy_QI`HkJhX;I4DsK<{jrJmLQxi1e$xK`)C`kjRx^xX&jn4pLUB~F>3ueyLvQo!ii0x(;&2U9Qy3qR{)fx zO;anyG;~gW!QH}KR;VDX1l%ij;<}(d-2XVubWCd>?KyvV#p8ECuVwJ(Sn2@1&$`~1 zKwxb?b)sW`{&mv%oYkj2C*<Zk^Ps_aJF@uqk(V=Ti@W~H@`XE~(XoHe|1skkOj}|?JNY&8l$Gg7V#~~2-9@2S zp%$n>H^NN{9w>@jivX(F7+gLb@AQqMo`vACb@E|(0B%HwdbXuVXjz-!5ofkUaYueV%eUyg1(@2~-szM-dp&_Y?u^ zL3upP+k+shd~!@Zu~N<*jR4b?J*Wrz76(^{>&?oScK9FtQoQ}wqo~J@)fLNqXqucu z`xP|mak6KXRvlTBhn@38%NEYCUPkw)-&_n_-r*UzX`bXSaWwI6 zsDKtW(N1^WiEdVuf^8#UCXMOxSd`v6oKK!c0Cm;k!ohIoao>G;iN56w4xkON@Y@z$ z_0hK`2>6;6YYPG)5+LdvI|6FZ0QTWzWdOgOU^pbO8c+rq2w0N@=^ui1msy=G=|52H z4!A_G6*=|GhiATlxS;pE%!p?mLiQrQ-wJlm>hOR3Bucir8nB*`An@x>-cLa=>07mg zlzL|DicG^K#MuHT+#9`fuB@9iEsXL5z?7SI@lWlY971y%C+2PHey8(#`f-lLDvT#( zGVsn*^Z&Efjn_^*_r2MXrkS5QZvky~`D`O29W(Ck^KpkQ)huR#6lv?UNHuen3rEp8 zyrq|C8KPQ%|K|l;I*3LYCiM^OZ71sXpyQjcP;+Lhq zj;>=4e&X7gY=b!aWqF-1nst!JDOk^X;a}1Dx>zy=O=cQ5vD_$K43uv4xSKGX?yL7* zJSw;@n8$%Rt$^hmlYxm>GhqRErJ{t+a?~z-atar6-fN~(Wd}Q(PoMO<;qmN)o9R}M zE$>YhM+SYXfM4Oh#J_!b=A$@|$t{jW=rWBc{jl-7%b{3|nAHL-0XOL1V=Gy9;SeHaTm@J*Vdt!P1 zL?Oy$uzWT^@e|1G5qI&7ME(e%5eY(DiN5^-nC5-&jaLvu5(;}`z@~YhZnt#)|FvAc zV7c&Kf}mAV4mkZW#t)3>i1R9;MVE_s=NbavD+{=Dn`dNL2{G;@S1Ec&doB~H<+D{= zKEr`#!=sl4C2?;63!3=q>EwFd+|>tPcMq_-H_z=zk2gNrJ~{LIaKaTap^Bpyki_?& z@~t>_n}1017?>k(A{cUk88a7{np!)dwUg0%s!yHo)6BcND%W|I#xGt=+XC%8Gg#s_~7FSDL34VEb`12WS0$Ybs~HiN9c z*5iEDp^k7C&>(!8A0{wW*UfY=+Byc~Hcvx7S9rJcrDo#gK^gD5rF2qqR?&9O*D(KS z`Mi|O*N>f6H~0Kv(b&Wr5cHdY%X~)nxok|Px-`&e_E!<=?J8ztw*z)uTNvkuQD(!6 z_=kK?mpM+*4jK)fnrJKEglf;$IrQxx%#MpSmJ7b`)U2JV-_m6Ne*HRpb^Z@%*dMNE zJ^sK6bY1ZfUay_YACpQGcQuMxLI3{k$VJeAKWwu{TJyy^__4ZVv`?H|vnobL4061n znXb<+E}-U|cO@4VL~@6VV1dqn>r^mJBN^w)VN

    31|3!uD3U)W*?s%x%%pDi~TpsS`K#1fcT7^eqhGCvWrN0`2A|H#=Bj3EvikAd z+xjncu@rWdzpB`+HVYH_D=}0!t>xy6&F`ZAeMLrhX6x@2KfhPG%n93nJm_u)6z6k- zeCta5Fq#wOZE3(}9U<;rhu19G3X+l7af_QhnYE;2bI~ci=wr(XnFy-+c!PKN0P^Os)Aw{pFI$UFO2v#nDAq;(!0u1PGje%f z`p*J=zr$`Od#%Ntpx8YspSRh!=d?q%L_b}4bKqlfFF4CIe{RuCA+eqUw=lf%&qCzy zAL{%-5K(gHZgTNW2P9?7$XR314Xu&o1ts@J^FgN}z)l+G_WX=H0=)j2fP*Ao$K4NN z4gK!1^Yefkt0%`1Ul1A2_`nG?EHK3rK)*j5CnPbcsC*=zxL%h`Jffv>G?qh^(o%(I z0rZJE;g{vtv8&LQ8>NHkTkr?7iqDTR>MHy}za-G?29m-k_Vq%E7=3CL7TVNd0$nfBC&=NoD~7aB^rmLmypn9wX-iVX#j5 z0Tt%kHib_v5{oL+fksN-O0ut2$Gm)mNRF-{c~;>0ZeR#5<&QeqD;`I|YG8T~%En(z zGYL>)OjAk54LvVV-G(?>&0f~n-ikX2=&M^E&m*(vQM0>Wn2s&nBqxR~1e!cO=mB(4 zb$XCZzzY#0a&Frx{X{LgqsS&H3tVCUEf~$vLBaMeN=##^5UAGEM6&dgwmvFc1Y!Wh zB?RO*Ww$u@dbGB*^dwDFDYteCOShKq@s>Ic~eB`13OskBe9>@UG@|_X*R2JT}v|=Y@&SM~P7<25D zAk_B*Y0crjckz;md%)+dwdUPfVWS6U z@?K5G`%JZ&KyRnJ>H%Tr&6i&VI-d8#dPg>_`Y{@xZkCPocpVa8Za{IgHj=AWEPFOH z&0z{xJQo{Ur+&XgEDoYMr@6kfkN*JOUXLsPfwL-LL$j_jg-3%XDa@~d6)CibJlf9D%&q-CZ0^ry!$PZGz>&A;P{birD73O zrAs+&0qDQhF3Sr^u^r>ee7NjiHI}8?EJk6Pr9T$Y~v=|kk<1>t3 zwla@&n0D2UvYKAh?MX~2eiu-c*#Y_@`E>j^zU>Bdv~V0JX6knMUqIIqakRHTyhE-N z_U>%y@S}f-0+qB!20TaoJ*;%TM?3~db?fIWP2~)$+0Dy*parmJ+wp|$(KR(;S!+TV z+lBn^FHvRSbAwY9R$3hy-M&{urhVUy{k*V0GnX%C(`0DEe7OOora4%*h>eK#WvS%Cih$wXe%XXL8W7%f6x^E;8+t}KsPfVbl zivBY)=IqYpC}#i!&Obz+&tsRV1=CeA+CU0mGn75Y0QU4~Cp~b)!|z@gNxcy92!O{y zAs^SL#qKXoXlSm|^~+4T+U&P4gB#O6B*!)0pAMiaX79L{d8=VK)Ld`lR-qHv;hVAU zOnuioaI0=*h}47Dh3!?*2uuFW@6vy!Sic_fjYRs*+yFhBM6U|a2n8BJiL&o+KClFb zhpJrs5iWc}<$^~HdslsUmBWnO>5m<@EAJ0;F8U0glk~QOGH0)YpU81ML{s#{*_UzZ zP+!QU;)gSD{8%y+I`>-|e+Z@(DV+n zn>wj~w`+Y28c^jjSXO z!4Uhv$V#!@ni|rE>-Z!CGC18lZEf>j@0)(0xI4L1V!kIL3)H@ysTTGE+KaJ_oIs9G zxgp{8HEj)*S}b)oB0RS3uam2GSO2uh0RF;?ZyZOuxW8&w<>g5^X8!;PIPFOj;F#6g z@+{pa;%T&0AJb95hLcDWwpDn$Y$S8W96?v)tDXrozh z|M;YkDiXRkeF{K;17t}^3^;BWaNP>%INOjxZwo!bfP9`L7s?}3sT4&9r>H$%pzm@# zSRZypWF1nD5aNTJ1NQEb&78SGwa~J%>$9s1uo%2-&~hC(Cb* z$=6Rhn5iLH0fM^<|RyJ5nQKfvZ2?&I_=on|SXbHZ{RMU&u(y`f#wO@mmnbk2MH+ z|K)36y)}sm|F~zD&yq3@CY8ee^X^N@uJcy$HDnADb=>9ACh(q(0^RN&&+77Jqho{u zUv~o2#;4LyQY;0B5(FHJ>|DC{$1ggn z1n*v{9oG8Qe-b>I`u>&hmel$1m&tbDo?Ive$bj)~F2}JlH=bZNdCw-?Zk=2T@*EDK zeLisp1JgYT5k(KkU688*$H9P@F`Nw|JEFq4mK!EOSE2WYZN4d%tq7(+=!seMS&J2V zVKw%k@U1XVvF0Q$bbi`nyZjaC1~V%V3puu$PYv?w=uK!NHy3#OdE^*b{GzHPU=6%}mBsjHU;smN96`Yl<#G?;; zNIxF%rd~~z>HW6)yvjU74iH}e>j@Of8^^+<;<5(QW2CQoUM?@ldh+7;(^ID}lds!0 zxy>ZV_wuGj3b+jmq~Y2>OP`~X2>qHRxZPOq2nO;zG4W~~$uaw>vaKEl zOs*WYwp8VPg)7d~zQ{L5#*TTXbV&CW0o1)8D~q}c>IX91n*w763ppCIW*alibP%-* zjUn=GJd|DK&%5#tJ)1c;qHm4g&434ZtUW2zoK+YnZvQ@aHKO2Ax{_XSqv12HK1~s3 z+*e&b_O|qN!-B(%N$c?%)=^S`&+}M;7~;;f>OMI3&2-1`lE6NbbjG)z6>nQ?Yf6CKG+!t#}YO>n{>g+)~BTt4|j8`#=v<%&A1B|K>E;B)(y0S zB}HzYjcIFTBVRu42QAkH9WI`Gkh{}rkG%Uy69N-yNM(GGyU1}BaNt5JjLNj1x6fcy zE!)KsKVfv(Cn2xji-3xR+QW@ZM@-!^oR9pr1j^{W;8n`4!)@6n@C$lC!!Dk8Cga`n zaTf;bzd$GPxGq%t6GzDJ-G$ebON?4${;iyD#UA;%8Dt|_d}UPALoAab^M|q6S(bje zzq8J*VD|cGqMFBFYqpbb7EiWX-?9^r-PuR(o1e_>N>=%anZK{u3yW0w_;NDOZ1y0` zt)cY=s;z`jht|o||0tKT|5hmuy4Hu~81jQu4Fkm*h-V#$8gQOvT~}jT{>q7rO^-jdYc(HMbGR2-EYfZoW`YAEex!uf2|F*Tu`_mP?Fw}xmcXJn#pcl6k+4h3$x%?iS)bGT`>Pt zOe$fB3gKk=^)ZO>=G~k{i!l6VKaD&R(@s2lSP-5dY8nKug zJq`X=P})fIHm+Qb`6oh18^(2>*X~KmMMHyX7X%bqha_T+l^eVkmKB0S8V5qiZLH+g zA9{yV5zQ&G65O-X6CyL^Jr@2mRR^QF(Yp^z8dGesv)Dy3ZQPs#MB$VjWl0J>^J8S@ z2)SQ~){8k5!wDrGgw!ZwGUk>}CFb%zg-kP_QN4}mU6G-YVlM4^D|90X2%9l)|NWgS zK6-Gg2M0l@ev&y&&`>u|H%PUs@b^@W&qwvQ`#u?j0P=%sL-7T-rY*Z_)R!IVai_v@ zb8)h6$PSs3?J_o>I3h;O>!$gxfCpi%i_qo+Nwe;-!(~GQIPvTCk!nJZTgbv!Z?jYn zvwZQWhUz&h{QSSP0J1s;9HP-SKb*-SA-=aXH)Na0D;4gwv~N&kIj!6bYN(wEtqT^e z;p{z_>VM`>+Ei{p4M4VnN(z<@=2nBC-Ds>aJZOKwOZOw8u;5@AVOU=0DIfA2)*wg? zn4a)V8|$?_Bx-O=5I8Jg`vc@1h*Ak=fSlzwg~R9>%u6)EMM;-=bG3L~X^qhYBFU>z zTgC-}>OJH`1`>TnixpAm#*lFa5}~OSS${v_ur>5jg_?5az(xps&$w9EJ}&rDckkxs ztyLI|i07vUPzd<1Fj}L<{*d>g=cnRZqhHVVa5&ZHLjodN1I25%x zaD|=tmap*|4ii=|z&ZiH&T9%8AnwmBr!u$nm(L!y=}qE7}PK^GCXm)4ut?Oo9=j5S^= z?_x-e9rWooH^>u<;*Qf;tePqjGj|w}!5NRuaD^q9j$cBqv=u+JLv`;E z6nnc)@P*JC69P`_pKAw5iE=p!PVU8kzr2Px5w+ZgBVHBCj8B3uU7A-lSk$D87ghI|Q); z&VR6v){y7+sgo?wKWAIay=;%IUaXfvREM0pYnJ3sT$9{dQJV0 zqc)?^H<`C1NPqZ56VaMd?Da z^JJtVp}AmVYO8nEPjhc;G&i*resMJVSCiq~PEdU0`l7&9Gn{RiOZm7=-6PzWXeqbl zm8~(^iHF$gIyoL&@V_@#IcLX+AZ6Y4l5UODko2yzGA6&mAKYdia=lsAyi(&U3|i~` zarOrhFHTiW(HvYJK3kJYge%=7;m*3?GL($ger~OXnn%o#mzfPw4Cze`52La|9I}w( z|9Unpqy+3kl5md200U5ufF^wi^vX_Hs6{ zE}XOWy66clM;CA%$%wlTL%5_-$o5jI&bvd2Wm>F9Vq zZ|Z{XCbD|LhS0JEWx1|q8fVDWR9|R~cyEw<(8ruEa}biF7t+~`1VwwL7843qc}DZ3 z_I=W_#5_&oJf}qO5@+Dsi#hnn=V{oZ2Seh8i}lLI7Y$~98pH#KvTX|9wAtJz`WzNQ z_huS={P?u%Im@BZSzWbz(}#cp_^4iJhMSvG(qC3dU3XR8j(Smp-U}i4YGNzvU16%wsA9>h2ZYm~) zcjnozYj)sKCeX|&)P`Wc4?Zj8ADy6U2mD*ki9^f#zejQd)n_3xCEijosZBMBp+yy>@lR_Ofrnr+^Yep zb0F&g%b;NnE?R6rWfz}{bns3 zCXeVDSl0y|%^=a%Db0ROskZTr<>0e!M}=ppYP2Kj6^aBmc&1aLeIR)tG0CY-R&U|7nlr(l8)@%W>G%7_O^I?QHGuqn)$x;<`APQwpKNS6UR*<{z^9kdIGE+7J{j!H7?2xfzar@khZ~8 zuudVqH1pz2t%`A(yt02jrLDy)AXjBu9M%10Ob928)dJx#Sj2tBCM7OESY)k;egBKg z*c^5QVyia`l6qS`Vc7*YH{hAJPZZWITFL(46RX}^QM)wtKDc~U_&r!rr59lShOJ-U}u-QrDe}Ynz>hPH> zXl?AuD_TCoJMddTn$DQJ#;cuIR}&EM3Fq4l0ui~tnl+&58z~3BLJ6xMsOpyHTtY+d zeW(3WK8GX;okkY%DlvI|rjD%e#L5|$cm*r;J9D;uq%YUx?b{AXu}6;RR;&t_5k5$G z+Sm0BbqmVQM~_>SRQyu5XrBm*pzfA}l!hwYdQMsRf@S)v!!9u{59>T8ZYZ1x;6_6k zzW1}(t!uq-6WhJ+RP8Z_!+d?%0^$#10k0T05%YTf5Bz!^*(;%s+d_Z$9Gl(qPFYFK z%GI?aJEw2gg`u!B6%Rv#T*v;{W$rJ4x3mSsqxjA0oyS(M^CD%D?oCMZXHW~Vij~m{ z^2}(5W2}1i^3dFf^k^h4M8+l1cf8sxb!0W>EV9q8?q*0eLlpQN10$@A+N1-k;S;lg zx4(m(wc*vewG!58FqY@C|NQ`97+rf5!-po0w;4PZ$W`kVC*i~*gpkHeBS5`N<#8iR zktze?!5@dc&HlvdpdZLc!nWoq;JWV~kajh_~dkAa+WND@Yj0PW@kEc=B z<_Lr~^IfM9`a@RJ|MkpL2x@vT_n}0Q!tZ+V zijn2(YW)^FQknpf{IrdqS(nS#oO3iP$7|oG7OXoNbJR)mfSk$wbBoiQHnIFF^;xtq zZMlPbsoMBr#GdMW)-ivAf#o7T$ToKp|Etcsa-sG5|N5qt485$Eoke^p@C% za0Y)(7lPKqB*0RIz)6AtbyM-u9a$oeuPB0Sv6HS!wst*iS0^--}*n{|LZFiVo> zj(e=UZUOzie2lgtelldZUp$(>WF#3Nd=kp7C~lrks6rhRs-@D zrruGC;0(4~pUjx@dw9n;n=`kOTv)OkaIj47Wnt3Kn#L8VzsfVHJX|rG3PVGt@#t{A>Kys=@$c>qTW>2t*$w-<7 zjUU9ag9%|-sgMB%n}Ocm2Hp%F8Yc`*KmnM~6SL9So9gulC;z#?c@%g<2o!h&6cs4Z zyRv%l5bTQrjUwG)ZG;PMe)a5hSE@giJJ}YLWph*si<&5Bl#DEhgvhBqc=aJy42Ea$ z_aJ&tdu(oXX@pmKua4YUj2WObZ;W=l0~Kuwu!sgsS8Bw0qau*9AK9Mi*#vd&`k(#U z^Jq-Aol4cjVXzFD=0CS8#1h&9k_yB?5v!kSD<0E1QdrR7wJ6EC1OTc+58PmG+98Qw zK9FsG)H!p+7ZE3vC{{f-K8_a_LinT~Ehb@>JcxJ_+_7tr^2nq# zlTAX-YqBgUL}n|5kXJ~vF^!D}^EDFoPg3fXEM{%0x=CIJB7M@R1;85lEA`{1 zo7ngg9icS0>Rsmv-NImlW2I=vdU0*?G={8ip{yLuAs_U6XIXVS71jjvXOlaxnTK%M zUb<#rJXcz*pWjNEo z{uO`G1k(#Ko)E6Ns>ViBZpsf2+}7D8&M_C)RiScd?L#Ed z6wul0s(kxlWYV5Z%9g-D^MWz5p^s=EqT7wDcWwN=h0DfJq%^V>_5rNJk6qBgP`U#W z3sJJyf`+_2KmXs(72HK!SiD4^aOfwVIL;d*Rb1d4EzY^5uvl-_rM#-T-7i#F-9NPG zEc-S%VMuj~QaGbkg%gN-}5s;O!yqEdQ&sRtXo=F7!?64OBo9s6n!<=#ySs(zWokc|(XYE`(QiL=G}u&xGn zU%yxe?eQ-Wel5MZZY)S`^H8I)>BRGEfc13rt=Xa}==~ObCh+Eu=l(<+TRoeSYx*2& zQG#kwfG-Kt`^Q48{;+EDaet57?gU#5AU*)?6nhO^`#1rP%G2m$0R89n&s$7Zi^owYo^z8|NW8prDrWZ0*8ntbC<}Q{t?xlF zFkCdSO~_fS*TNyj&Be$+Y|;UQnjdo+=Knu4Dx3Jimyfs)bQ+`){HR+Eu%IN~hW$U) zS`P7X&VA$aqS5@>{r|eDBcb*}V9qFc z_$s2fKhLGYWlY*7m0xqvu0p$BPxP!cE19=<>TvV#|oLV#ESZ48vC=$Db@j4m1z@?!r# zQ!@#7{$X#hD65{yc`KfPRr%qNwCkJ5ueSPC#@HEQs+0!8+$?~IbAUbbMXWY7%l-Qf zAj1E+1MRW-C2C@lT09cpn3{DTtVu$p*&&zMOhkW>NuL55@@MQnLCx%LjfOr0%E9}gFzj@@pMgR`+UnBh22>%lo z{_6?<^@RU=!v9#{zn<`4Pxv1T9K60U@GmXEf4$ZLn<}H z%kyh+_cf;XE!OE<0n2ygoS%mY+@aGNbGeY>st|DT+f6GCt$UHQm;O?5c9}ZiNNq|b zK7~^(%TcS+`1A$T?bjDd%fwzjdd94Brl4oE&=b0F&_tfJ$z3gY2E&?d`m4?M*BhrT zPx-E9rmc{v z>$b2U%$)EvluqS_rQyY1fp)$2hr+Q3^Rl^@a#lizK8+u_b-(Ax^RBp_QwwlmL~-F& z%c>W8GlD8ib5|x1%w#O__Rj)l#uP15o44w>%{{V_nXI{jawGanI;G?}q_?H7bhBZd z+nPO}f}P>S!44iC9u{xOZKT`URgSqVZ7CoSh$9?ot$vXC-@or$`Lh4*ipPCcRP1uS zbNqg5%1igKZu>|%DUlYI4=27QK5dymP8RB{*q<)6n3Gh!5Q1e4q>s~>UYi#x2jAq; zd_#`38DdmV=h zQzjjQdb(>qL#k^hpk*ngnm%#%(cCw}Chqm^PeD^+hdBmq}{#a71 z605Si--H#Tw|pQYzJPLvzIu|a1%IWvbEeD51cjeh<=tb;(22Cz%eCOwJd+9EbF9aU zH`Df;F+YC%Sjo(K^Y?3<9v1W-zw~;DnY6=<85VH2`E@~VBNfqqyj@;cTyD;!Jw25v zZ>&=9(wpEnr($D@UNkk03{4-M&eAWIvpRSGlbm}|Wnpy(K`)PJBCep8|G+*q`lV!3 z?*w}2qfD0Y>l;^%PW^i0hpNwPuq(Y#X6ZNe=uW~#nWK-_;@f_e9^LtbywR2U%2vIc z=PxFY?Dl6mc$f zyzXCtsAXua`0XRP*s?J_)Q0LER#_ix?F|U>#DLqd9PkE{ubl@6kRuuz8ngZb0|T+&mj4$^z&`xL^Xa9zEo6cm z_f9bgVfUT7;sd9R@MY;!3k$6R;h9!G_kE8)ZT=l0%27CZv7O%FIAzyr3eo?P-9QzF zEMP)0D7(Lb8MAIzc267+y}NcQY#4^3E3zwH-Kp1iy=TO@W4hJkqB2yfys=oNz??1| z%TUTydt4jM^?J-q59wPA-XGNNgTly}rL@uwah<1tp-2k4Eb=hIdhSEnfudUtmlO9NqsN_zd&AVS%KbYf8 z4!gf$aOGt;l72bsy({=Tgg;v2%WCj@biVQ;WfF=_N16;GT7(H+QQ6miiNoH|IWrr{ zE+>k5^sh&R%I5^K92Zt*hA+3#=IhKePuCjku#n&sln3l^4-ujbL-h_1 zpZCusRF>WQ`!(B%PrmDWop1T|wV#(u_>3|Yr@ogyN0l=DvS1;nm~t&?3o|L7-U%<* z+B*+MLZ}rVoG>1P5I(x#h|s?WysnJ-;?w2%)FZTRQPkC^Ppu|yc?h46MV-}vxnF6ZBb?a2 zOjMxXX)g>#9Ye$3XBI__tPE{2_#M<_|%qveegG7_zwZY_*X+M zsV}dtRqojxW8vN|_#Wi(Nab*2om)hhJx+a;`NKsSnTDg%tqUy+mx-6EXY40f`8~h( zl;?;&+<-LW_5Xe|?D2~2_=`52b;pgYf69FN6%O&aa1nBD{YKk9T=NaJKGzcQ$W`4> zwPpB>*wNURG}FhfslVde&PjdqnB^eE<`gn*yI*m=_l+2;uvVe|OJ-TPSo`eXcPBsp zMuFaYs_!ZLpo!b({gD0**KJevTdoMw)AL;BhBPyI)$(v+_U*o+VS+WdHK3{ld!&o-^hHi^LKiWmr9y%rF$Ff4uR3 z@vvmkE;x!V^2l&`VdTi=hQCf`#MKtRuhd3ASIE4jWFGghZCUjDQLdI?>~4k@8a}b! zt?hiNIlcEQR@*4#eANV^X4rb9i0zaLJgaP z4}n|7#>JIz*|G9{cdgM(pu2@QG1*&gBYEMYJOs{Fq?-^P(8%(nb?<@zIQ9N&=EFtJ z1n@%e1}L*1`ohH+b&A#prCuqXptZZ@V#lvwyfOn<Y+g(LDwiD%5e1n+yEzF)Zh%^SiX7AYsTE*Gp5TA6X@au2hxOh&CgS17|JRRv8; zx^>Cv2#dx~RCTEWrW`U;%Ecx+o;>t*KAtC2F1-6Ir^-=hh1EZ5E!;K}cGIWYo+uvo z3t{eSS!m@veaezW`N&>NJBGtLvcP~4TPS}fpX|gX{qnRJ!j;YM9PU=Qe^on2kc*=&#iDt_&DfE+6b-E^00u>Om>>0_Mo<3*L^t)6m0 zj8}C{O*i?VVCuy!0d4)KhCtlv0izFVbv`H$+6@!Y)-M;!#xB0kD>-=S%N33xf9$sL zZ1r$$dKPjc?P9-OqIz~Kz|K`B%%R;j-&$LbwoSdCa^Fd>EB;^XIPd`2k?-g;`e&@a zXzAa23^JkgG;_g`o69%z<(fLZy^j@LD!R0d5Eav{SW$er5rXN4ooZm|R#Dk;cvbSU zky#~U@!T|9E5D$?E4c?;{!^_(E+M#bs zPo1t4wOS{#d)XgYe1hC`zU**eJ0+?1SegD4#0eJHAb85f#p6e19$LSUd^zhFxpKeZ zHT~8H#lz!a0&PMD{1cbh#@GoWt(VPb`&xG{ol~)NbbJf`WK~wBd+*}^q@s7~+0VFL zn{NFi`tZPatcHh&{b>6H3_vMlGFeblB|-2=#hO>Rx)m636|X>X`xkhmp& zDb`g6(XgOI`h;Cvi*DYDmPTk>zvJfSSwqndVIXuK{MLZhTYtd!IY@IA6&G(Hns*UJ zB_$Q*<=+7CUq|!ikF9LvN(8@!H?!VJ5G*Px`o8GBBTJhnl$LugGK@O@p1@y>LkE!p z3EAM4!XX>N+`PPmq%8^Q;dd!nm$-w|MbGz!pm9oPE4w5|CcGb4wd$$SU0)XR+x+CdsQ{<7Tu1w4y;v- znA(}&5OwP41!GA%m!_!=V=HG`b-71}Fdt@o{=c@q1RBcy|KBdVkUdKjN|AkEOVpK0 zgtBKW+4ps>nauOd^ZC4& z*Ycj{$#371ZLtxE+9LGYLqFA2KT>x*V60ds z7JH?QboBs^46M7JH$a**u>1wPv%EuiZWmx6_c5KT!Ys}grJpwdTToJ3s)?FBqer&Z z6R~faV%=n7Vv;lDR9aoFI8I`PfZlv9ZEP|GH`?4x=H=xDds}D>sljZ!(8ghzE8g8* zS@Y=sBlx&D28e{0+1c5Yr4oK>TE}u7yWnjx*EoIxfmZsF&70K zT%RWHhKGjj?go5cm5bAP!#$J%VKy{=c;gt(|EGem}rht#_>>k1g zn+U1o66oKeTN3+Cqd7r2=9X*{#vr&)KHB%NVXJlH#W7jYR5x=HDQ;uqhd&wCV_xRP4cd{VdpjR+4SSGqj94bj2ic)tbMZ&f}%<4_0vZ=Sj$4q zT9j{O6g1Di7Pn@4dgYpPM+v_M?{2{`A5zhSr4)O`ncqKq)+KVVqrgiu#EqvdPEW%L7PA95h{F+o8= z^p9HTEWEyR5__ckO9su;RM*zZ+mAjbd~)w9KL$D3-x>cG6@M_t8fcpm%zo;my|{e{6M@#a7iWo`-YywX05k zSWl^{efaPpWtljtXy@#|H{VkNbuAKO(W_Fxo!@^?aCM?~X`RPIDS<{G8CWvyc4EcQq)w$dSzmmSvl-V37<*#%ojnAmj?Ol2QQY1^1Rn# zy+r_{cybnK?7qD`nrc5om1Qir>tvO`gH%0_^{APh@n+2HC5o9)A@xIV$hS}0SBZE3 zHL?O_4yaBwww{1snA^Ff9h71R5MX#@WZ7Fu{QP-nUq7}3p;A>=Rw^heS_6RQ6q?_7 z2`PcY;UMqqeSBUpCz+N978(obCDBD3UetlWFij~fFV{{Io*>o`-VZs!fo`%<78De) zUG~s?gL>7=-7_*W0{jX(;Y8-aL5Q!PpQ4&teJ-!u-NEvFdBGj*U{UzM(3GJa}xrBN>ze-IHP{bE=Sa}e#v zdzZXH#w);(&Fk6v6N-saKX&1_tPK{8t~vXNM(TLP3_coILQ1dnop;q#{vqNf!eMx z4-Zf9-at0O!{b@@?yGETc(!G6@Z!0bT-4OmsZnDI2?+t1(hevSS58SedNiG4CS);!5gte(Dz64bq=lJ+IbWF={Gsr+b zgX`DvlhX&2@XW*8pihzW<4K@6gFd0f@EJw=8 zP*9~FnEb~#+DJ%i?M!0w#|UMS_q;<(9eMm~o?8`=Vm0oO z7aRET^KxP$e46}D^6#!rKX$Jtn;WH3CNY-nxMKC*%#xkq=MC3OTC{o{x<6^+=c`$- z9Co1<-p9{3*{|Y;-qAn$w!ZR7V?vN^Xu9=}t3!C?Tg1WLTv6Kg`HPdTshmPP)1m5x z#xc-!-*~vmeuS23W@ZD~3I!54FMxYkBq+Q`E6>Mg!4k+9r{j$>*E+Tk5RU>Gz@ftI zNCv^~D5|K`-mfhahku3j4WTdS4Lmn4xSaM5kkYM`(=jnI&+R$^@aBlsk2xp36N4g+ z!D7p+s^sZvEw9IVusbxEWTByMk5!jRhw8U1wsDS4Pj^g894pInyqfk703CN6)eXu? zkz|@lSHDWdnXA`}3ufK@^+#^UK8uWuG$lWDxE<1_0}73s8#YOw>fcrP)UE1skr;7K z9FN%Z3^6vxXd< zp+E3=M!Hgi0LG05E&r)?O`4U&V5#uaZ=COF;@mIXo<6qD7-=iAf0+hFH61VE#`(!4 zOpT(vIwJ4L-KT_a$^B2y`LX!U_jPZb$>`a zw6)ZReLH|J0RSGmySiwx!Dwadt7sgwVQRH?QaO3~SJ~Nl;&k;}nb;C&xP}I*Z6Mp) zrK7e~(ADTHTLs>-h6aNaJ8^OGFV1oSBdDo1h<(V1REi3NyJN8Cq5B=7JxfbxvJZdd znVOl+g;Voc0qC(j7_zi;b4w7(@{fSCJyPF4!rTsW>5!*eWBtgK!ou?w!5f#OTV{>! z_F6u9)$1xhx3Ca(g+Jsub2H^C7V`TDG#p{>^%f@b^+7{}BdFWVQQz3v*?Ep{M~0K% zw)P#>_E(42s`~Bo(4h^~4`hJhNDK9QA%DH$zsQ$hKl>ZW{dm9_=m+vSiVWJ>f?g|Y zMkzHk@+MTl1|E$Qr7X?yM*0|45k`sfzZfs&>->6lC$260Cv8`e?ZWbd;m4RwPWI$pf3q;FqxgVb4Pw0>m<-l##}3gmIpVeI=C5muIt9$7Zc3Y z@n9)9vs1v(+MQ0P;(73D%cKs8Lt7bBez$zfOb`Rb_!?jxT_DIut6ouY1eEFtYbbl} z@5#c}z-J)yl7y|L-g6HyF*0(-=@fHjW&Lp`w!;=Axi?=Q113!p2IlNoHO@g%)6ZcG zq!F4dVCwmK+cNR=BKyeedDJr&MkPoTY;f5TdqMwPwQCtpQ6yttIx@q{<{D^o?!H|PYfYb zF{R8`yyhUQbur%km|V5Ac2QNNzQBH?UQIZ@Lf1pmg}Nc>m^G*M4-GOZM|NH5H?>+C zL!>?Z-ImH1H5@dhvJ(%xqee-C%9C>ww`mo^HbP0pJ^U}sv93)p6Lx9W(I+coH;GS2 zYnuoepJ>PWxjyDNa}F{PZ(QY7!0Cv1vnCNK7CkyC>9_=O0-1C#srw9SP&X^`EbS#n zBj;xq9cr%>M{HExG)>{<<#qS=Znak@m_ig;?tM#LdlHQk?*=+I5lTF8c6LpCCkD4> zXds{(Y#%&$z+R6$o!l8^pA`@dOUs_hP9tayA;01ON0RWJ;7LbmJhY`Fk#uZgV&T~p zzSl!e8(!)=UQp;-nABrmm45u#3Y!bW&>jGhNOrL2@K0pbr?2fhD@G}&A-T#o4CEbu zv$CHT|CUM_#z}cdHt1hp_G=bqj*l}r`&o^($(O;m(IBIYzc?!D-9~!-|qS4?Jtg1>yfNKP{gmq7@-# zU8}Ag3vaTLq#GfRVs`hli7iL_pe9WR@RM`C4@V8eKK%xNdHVX^tUSQcOH5 zXa1~awLLMMa`gx0{?@bL0s&l?^z$HPw&)Xim+?A1v58}2X-(v#8P;5*4O9i%8;P1W zQS&X-9*;;!_)hCx(!9;09@dE^RgPEX7eueomS%feQFyTE)zceG&&fBXhFj`eQ~IkW zl45nAXyt4n~pqn>6< zq=?X6YGq=>h&ydh7+bKI7I|&T?S#t$^7{hKv*x@u>WdnB6&^Me!?Mmcx#Q6sI@P=z zN+vDtk)9#k_uP1_=}U9XSc_e=3=2CQM-?yX=5N<7)qA8@HXB$bkhlH9`@Jle`B*jX z!p_bPR|f#otZ?`=!zz5;qTIIbdk^R|ns2@eL`gnisIr1Z`RX9kH^|Me_446p%%3^p z*XWhK=&DiDob>Q!Kp+4`9$)NRazK<sHp_I`y9^w3b@i z7M;U7D$pd}uq*0s&g;a;+;*ACGj$Qa{rX6{e7N;nFf2;l?oi-5j%Z#g-)!RD0W1UIW2lhV1AcPi8u-M37fx z%PcYb(YOx2pjc?d*~H!K@|v2iJhg4HBXPjO0^#WBIQ-@Cvrs}wRaM8E1huZiLp4bM z;=q>>AZsYB+RL{f(nm)%h$Wj|MS0W(B((FrXG8AD!-p}i%(4#Cz(%dcKLw#l@d0B~P z$jr{yMH4q5x6h{Hf5Jxnijly?k;cPJ_IH=oA23ZO zta>XIZ?$nMDkxZ~@MSM^qN&R!L)e|wRa7!$3oP|h>;|1=e{SzN$6YuKsk5-Rwhjjf zet3AO&k%9W@Hf@rR36*?)H?tMm>P-@hM53ywpI*F{etaV}c&=}ai+km$Nsg$uYt=4jk6i%CfCY`diVrc) zWB8S)K7AG`fh_+mInmL?jnv#x#wfbnx1WA({P6q5oMnI7%3Hl+AGrOs6vbYDV%VyE zaf+=@*MZ`PvYNYzOv%|ExAQIi5?J3yG?E)mCqD$n%dSQ&;R-lL>ZSL~8(HyyUVNlxHmwc#?hOqMjit$^>+7@}zo_%{Pq;nT+}x>(z&N0v zU1j${=~3wi=~UhYiGDQZRykmc@@g`R`Azy~UV#f2^&$_ICC`#$c%%GU2GjW1)_#+4 zdW!VW-im9^EJA{FHk(I}1FxE=AYMAL_FfW2>cR5FK-+_p=A4*uq3kmrxwpm0y+L_gaK#~DV??~RAJ zs)o#?noXG6DFdzSU}xF@WJzE=XvJyb;Q&C8^*FcSNbngg{AVcv!^28(&b!c+l0#a9 zoS~Q0vjletHZItx8L5jG*2B*-h3aOZtL_I&zX7qVva-EH{m=t~1aLDz!o%*+T?wcZ zjEs!H#P+`zA8>=nmO)tdI&x`L2$=FCcAbe0e)QGByTz;eg>^`nO25RYWTG&QUed(3 zi@zxs3wxrUFI1&idNWD7knK#As#4R99?INlpc)h?Qu<_lF1%H6?aHSR(}lY&1+p`> z`(^quV-o&m-p*EEs85ws1(|7_{Gc;5ij*a7jyTT2OE-l7Rj0Jp`Z{B{ZW_TMUs7dO zXi83KU!vOb+5%^_Clxz)kqY;!; zS2rjg5sG^7ez-wSFR2997vwz9i$FL!e&o{ka<4EkF}+Gpe>06uwmY&zfRgO(DP7b7 zf#UMb8*F}~*B@nx1RoJm49JEhd+)#cOsBlL?-sv$s4Xp zrAv|#_|*u{b1aWEY7ZU+K}^N_)y^kT{RgPRoCOndot%-0X&0(b)9kF^k-D1Pxe)JQ z)nIay4N$*f%4vyu{3%dCN~<;NZlcYxfgBQbzgURtOvFZimHN~3{QyF~CLo{p_Ua$O zlV|V5)DJfjK;Hzh#?m@8H2Y6c7=2lI{fN)T);7`$-3MvDyZ%VM2k>scI#C6LgY5kU z5qUO*ymZOWL&T%*$xLJed`VcOT^Bgyg$q)g^sdV~;fAIu8=&9bDjos)nT<4T+<_eS z8{8KC%_`^JNIa-m^R-WTIh4a6fBGMHCZV5=V6ms>S9;A9+bA-_uudO5R60?@(R8h0|2u{5 z5=*mIJN<70#}9x5AfJDMSf7qwd#j9 z`XlMVN6G_l!%O93xyA+H9S0y6C;9SLXzm%B5WO>b;;29xE0+c94=OBBLKwbj^%n_t zfAa5YIH3ql@^05_C!~6Bk;R#opNi`lw&pEFg*l$LBj9zi3}r$}VPx*ENYMU*E*(nml~T^(zz4bbQ)VMI9?}Tp@zF zu>o6xU6+fje70}))F0cj*?FC( zVS@*f-D7P+aSGoqUshh;g2TyK68A>9=&A0gKqn5Y>l>dw^ffDF2Lg9gAm=0w=aNRC zvrKJe?UYnkcN!3X8A!{kUwOs>13Mc>62%(k}6 zYP+2VWuTZ~_xrF%=*>ZUZ@NjiHLkH^2O zxBR!e_Y9R8ff;Fv&?NuBV0~I}u;rnFyU2Ck1m;H@?JEUw`AsO&PS3`qn24OOVpPZf zyz_efj_1ZbS40JFG2>$SPsK}K&Uv&o-;evdJ19@d%+=U(T3<}m5hci5o4Z{i8)UHb zlN1^=I3`r0#+=81Uc)=qf6q@p9Szd~$}2PHOSo@@#W5&LDM+bU{8~DdsBFz-Lz_bG zQ*t@7*{b(glv6cb{s>Q$nSeM&*Y#WzKP?8yNMvhZ{7yeIt|pvPz~|d`)Pr83ps3LV z|79chDM5WhD9Ha&YyZ}!in&L7p$)J?I71HrU2s~K0h0=aj6fg=>j4gg+s*^Sbf~%u zQDB7tHUaKIK-T5D zDiR||@qZ>ElmuTCL!*NVlvQ#+7!2saM5keX(}LGzY0E4Vlm@3LV>35TGIF35JHppP zO2|bIY3Bqi)yTEXAxn`h%&#(mB2cB@bI~CU)fRwe$ngPP$Q@3T}hF=IQquP zxmeMdyg|LO_E|ko?P}{ojacSTdh78LH>#&~NeVSG>t^()IynSQY5>AhI0w^wj@-9N=dIRML`a7Xt zcZ;!#hk2~d9WyAa04tcTgpCqW-_<*k4`sNa&kevWh5!^yL!v7vDb4Mov+Y&*ri~ad z9Os@!KuU#!Vs-c<;qWoIyK39-4NAc#EZ*-Ty!^v+>R`ow`W z4Xn!$SWM(*GqRu0&v7iiccGX)EyQLlYcl=AoocYjP*f?E3(NUo*@y?9`9i-zj!`XK z!CkA$34x|KW^_*Zyr^=#V$`nt7B{vy^=9vbKC$%UXKwIv(Ps!N#KH^=nNNrGE_@H z-}}+qc_Y{4tx(K#4ez$4$qh0)_k1Wi6Y`@E+-;1M>tJB0$84-z9A!gJ0A^^;WpM;} zz_<7H^_7$c1S~22-OHuyLpuz>0IF`X06F4q_(7%0(2aMpg zb%RQpH&A__5BfV`*}VhgHkZ53S|6MU#9c5qkA*`A$cExHH#hg86Di5rLx#>HJs8m0 zFB6wI_QtY%6z(Kq5mL|^z_17L1g2zDHOs&2e?XyO;(?NXm|lUIQab7n-{2X%XVgXc z)1TLZE(u zYceM?+6fO9XKYJI*Ka-Hn8~DYvg zla6w5H~D&?vrZMDH;u5tO)v?NouTX0p^fn2_=)%qQ%g(qNvuCLU4#6Wd>@KyTM@9MqJ z(XjZ*W=>8{Q~AcfXl2i!+H+@y9?ZpG_jERy)U9Iut6_5>gkmGNk?MwpLAz4Kwji%T z#{=`l-HvdZ+cNr3{0rfI!r zUB)<_hHNP0YQ>?Ns(f-j>0~?u52y9G^~idiF6PRo@7pX*URM33WZzbAWbRbOp6R_jx>a^l&$~lt#@~(E7`l`up9w(}j7o<+qa>p_P)1j^ zUB2qO${Bl+dH(>kB(hwuHEhJ7%1(HOG}I1o%fLgu(;qUu4gJr5`{#*)<>&AmFY}Sx zMiow-ZWXS-ItqpLlK$(@oeSGA0u>V}K_0_F!(VR{F?c{vkE&}x$>mbjG_t8#oB7RP;TxQ(#IO8cJh^{KUqn*!M zI90r4GUXLdlfQi_pLH8SNn^uzDZ1?@QaUO#XU#ZQ>84Mb?DT^Rd0shcb(VdvDRy(2 zxw<+jmx8$RmxgN$lZRP$EgF?t`O`E4>KH!ec&D8Si6>hkrT!|jjUCF{A@%y4Jitr{ zNOff7KT~zOZol@?^1YKix*YhWg{nGWrwe9)&Cgt1lM^gM%X(9&TLCHMKu>v`hrbYME3 zHK{idH}*>Vd^inveOw%C>R%>SJJaiCzd0BMAq)#GRdMcoZ~-PQ~J3^%Wlg zrSDhL;^N|@?+&MiA7pGjY{8Hi89{e&tS~EJCZMxJVSLRxD2RuJmm%S0FQV3eBVyUR z3*rdf1GZ&QuVY9^h|eLPYG7;CG_w1EuroF_wFKP+-Je|!tOYDg?;<*y|JqL2+M4sX zu6m;;tCpzUd@nm|{1Hjgh3Dq^L?<52@!q&0Crm}n9Qz|_xL1D$W*5nqI5R5!sdw?b z1lqnGFKpo9GVi8c?`+xgrAp3)GlOc8gL=IWIaU(eY0UJQGr~BjdRP|)tf)#;pKwVo zVAye0aBDth-un)kPUJj^!z_*|y4q0Z*=>N3=bR6?k{ctR=SvZ9a^oOTDIj6vZs$tWGAt!tA%|{#pA#7%Ebnj ze{Eja{O@p25?*HiG^Xjhypuzs{g}<2L|D{FruX1IO_7eGi-x+3c|wmwg8CqtO`hom z3yq2wy5}EvXztNH57OM{`kDE3kDuS)BV6t2E!y(2*i;G2q+5UHpS^XppfOPF;bXjT z+E4Z94C%LQ$irRqP;j`>XfT%w@rx`F35Y9SRPWy2UVg>lVoWMbpB1HTo&HBACc&rj z%s{jV$rCkaetjJ(Y;Z7+2}q82tIPlE=cOLOV#*DH$4L&8g}NselC>^vsZ7#$YgDMk zC#X)_jR(Q7W6^_KS3w>4@clb_t2z$z&&xh|3WrY-OBNuaz66>iAF{Q+(F!zXEx$B& z;Y;!3P)d4ooMg`h^9Z1>^?N_hWvK)-LpmthBlkD105Sptxd*+qGWF*_V@>|(sFgIL z^ID2UIoZ1Bb$7zI1=P=$P;F4;_daDm(GyN~Gs5#$nsI=1)wk-~MwhF`?ZCwBNZ{XE zpVdAUG3MM3Wf`160P z{<4nL=L?evu2U^GiSO+->+Z5jJ8Yl*rgI7*JDAAo&L-I_N!`qh@=kf2^Ss$z;l5gS zISSQnC7CmgmF7!eDE*%mXN2K$b2A&+Q?S=jf+ovL?{lX9%87O=jG(<$up8(oKenyV z(|YRh@f$@!E#G?YjL7<7r%t~GZ(0!D=MoY@n-l{@3(N&|9!FO rxu)+wPa~}N?FFp<|Grpri%98jv3;vnZ Date: Thu, 31 Mar 2022 14:16:11 +0200 Subject: [PATCH 154/258] [FLINK-26810][connectors/elasticsearch] Use local timezone for TIMESTAMP_WITH_LOCAL_TIMEZONE fields in dynamic index --- .../table/IndexGeneratorFactory.java | 10 +++-- .../table/IndexGeneratorFactory.java | 10 +++-- .../table/IndexGeneratorTest.java | 44 +++++++++++++++++++ .../table/IndexGeneratorFactoryTest.java | 35 ++++++++++++--- 4 files changed, 85 insertions(+), 14 deletions(-) diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java index ec2a00693658a..92886f40e0932 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorFactory.java @@ -33,7 +33,6 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.time.ZoneId; -import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Arrays; @@ -133,7 +132,8 @@ public String generate(RowData row) { final String dateTimeFormat = indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); DynamicFormatter formatFunction = - createFormatFunction(indexFieldType, indexFieldLogicalTypeRoot); + createFormatFunction( + indexFieldType, indexFieldLogicalTypeRoot, localTimeZoneId); return new AbstractTimeIndexGenerator(index, dateTimeFormat) { @Override @@ -163,7 +163,9 @@ public String generate(RowData row) { } private static DynamicFormatter createFormatFunction( - LogicalType indexFieldType, LogicalTypeRoot indexFieldLogicalTypeRoot) { + LogicalType indexFieldType, + LogicalTypeRoot indexFieldLogicalTypeRoot, + ZoneId localTimeZoneId) { switch (indexFieldLogicalTypeRoot) { case DATE: return (value, dateTimeFormatter) -> { @@ -186,7 +188,7 @@ private static DynamicFormatter createFormatFunction( case TIMESTAMP_WITH_LOCAL_TIME_ZONE: return (value, dateTimeFormatter) -> { TimestampData indexField = (TimestampData) value; - return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter); + return indexField.toInstant().atZone(localTimeZoneId).format(dateTimeFormatter); }; default: throw new TableException( diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java index 48f0107b7141b..8347a479e5a7a 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/main/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactory.java @@ -34,7 +34,6 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.time.ZoneId; -import java.time.ZoneOffset; import java.time.format.DateTimeFormatter; import java.util.ArrayList; import java.util.Arrays; @@ -130,7 +129,8 @@ public String generate(RowData row) { final String dateTimeFormat = indexHelper.extractDateFormat(index, indexFieldLogicalTypeRoot); DynamicFormatter formatFunction = - createFormatFunction(indexFieldType, indexFieldLogicalTypeRoot); + createFormatFunction( + indexFieldType, indexFieldLogicalTypeRoot, localTimeZoneId); return new AbstractTimeIndexGenerator(index, dateTimeFormat) { @Override @@ -160,7 +160,9 @@ public String generate(RowData row) { } private static DynamicFormatter createFormatFunction( - LogicalType indexFieldType, LogicalTypeRoot indexFieldLogicalTypeRoot) { + LogicalType indexFieldType, + LogicalTypeRoot indexFieldLogicalTypeRoot, + ZoneId localTimeZoneId) { switch (indexFieldLogicalTypeRoot) { case DATE: return (value, dateTimeFormatter) -> { @@ -183,7 +185,7 @@ private static DynamicFormatter createFormatFunction( case TIMESTAMP_WITH_LOCAL_TIME_ZONE: return (value, dateTimeFormatter) -> { TimestampData indexField = (TimestampData) value; - return indexField.toInstant().atZone(ZoneOffset.UTC).format(dateTimeFormatter); + return indexField.toInstant().atZone(localTimeZoneId).format(dateTimeFormatter); }; default: throw new TableException( diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java index 7840bfa81a85a..8760d0d4a4372 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/IndexGeneratorTest.java @@ -35,11 +35,14 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.time.temporal.UnsupportedTemporalTypeException; import java.util.Arrays; import java.util.List; +import static org.junit.jupiter.api.Assumptions.assumingThat; + /** Suite tests for {@link IndexGenerator}. */ public class IndexGeneratorTest { @@ -54,6 +57,7 @@ public class IndexGeneratorTest { "local_datetime", "local_date", "local_time", + "local_timestamp", "note", "status"); @@ -68,6 +72,7 @@ public class IndexGeneratorTest { DataTypes.TIMESTAMP().bridgedTo(LocalDateTime.class), DataTypes.DATE().bridgedTo(LocalDate.class), DataTypes.TIME().bridgedTo(LocalTime.class), + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(), DataTypes.STRING(), DataTypes.BOOLEAN()); @@ -86,6 +91,10 @@ public class IndexGeneratorTest { LocalDateTime.of(2020, 3, 18, 12, 12, 14, 1000)), (int) LocalDate.of(2020, 3, 18).toEpochDay(), (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L), + TimestampData.fromInstant( + LocalDateTime.of(2020, 3, 18, 3, 12, 14, 1000) + .atZone(ZoneId.of("Asia/Shanghai")) + .toInstant()), "test1", true), GenericRowData.of( @@ -101,9 +110,44 @@ public class IndexGeneratorTest { LocalDateTime.of(2020, 3, 19, 12, 22, 14, 1000)), (int) LocalDate.of(2020, 3, 19).toEpochDay(), (int) (LocalTime.of(12, 13, 14, 2000).toNanoOfDay() / 1_000_000L), + TimestampData.fromInstant( + LocalDateTime.of(2020, 3, 19, 20, 22, 14, 1000) + .atZone(ZoneId.of("America/Los_Angeles")) + .toInstant()), "test2", false)); + @Test + public void testDynamicIndexFromTimestampTzUTC() { + assumingThat( + ZoneId.systemDefault().equals(ZoneId.of("UTC")), + () -> { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{local_timestamp|yyyy_MM_dd_HH-ss}_index", + fieldNames, + dataTypes); + indexGenerator.open(); + Assertions.assertEquals( + "2020_03_17_19-14_index", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals( + "2020_03_20_03-14_index", indexGenerator.generate(rows.get(1))); + }); + } + + @Test + public void testDynamicIndexFromTimestampTzWithSpecificTimezone() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "{local_timestamp|yyyy_MM_dd_HH-ss}_index", + fieldNames, + dataTypes, + ZoneId.of("Europe/Berlin")); + indexGenerator.open(); + Assertions.assertEquals("2020_03_17_20-14_index", indexGenerator.generate(rows.get(0))); + Assertions.assertEquals("2020_03_20_04-14_index", indexGenerator.generate(rows.get(1))); + } + @Test public void testDynamicIndexFromTimestamp() { IndexGenerator indexGenerator = diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java index a5f7759512896..bbb5081b73389 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java +++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/streaming/connectors/elasticsearch/table/IndexGeneratorFactoryTest.java @@ -32,16 +32,19 @@ import org.junit.Test; import java.sql.Timestamp; -import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.time.temporal.UnsupportedTemporalTypeException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.Assume.assumeThat; + /** Tests for {@link IndexGeneratorFactory}. */ public class IndexGeneratorFactoryTest extends TestLogger { @@ -71,7 +74,10 @@ public void prepareData() { (int) LocalDate.parse("2020-03-18").toEpochDay(), (int) (LocalTime.parse("12:12:14").toNanoOfDay() / 1_000_000L), TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-18T12:12:14")), - TimestampData.fromInstant(Instant.parse("2020-03-18T12:12:14Z")), + TimestampData.fromInstant( + LocalDateTime.of(2020, 3, 18, 3, 12, 14, 1000) + .atZone(ZoneId.of("Asia/Shanghai")) + .toInstant()), true)); rows.add( GenericRowData.of( @@ -81,7 +87,10 @@ public void prepareData() { (int) LocalDate.parse("2020-03-19").toEpochDay(), (int) (LocalTime.parse("12:22:21").toNanoOfDay() / 1_000_000L), TimestampData.fromLocalDateTime(LocalDateTime.parse("2020-03-19T12:22:14")), - TimestampData.fromInstant(Instant.parse("2020-03-19T12:12:14Z")), + TimestampData.fromInstant( + LocalDateTime.of(2020, 3, 19, 20, 22, 14, 1000) + .atZone(ZoneId.of("America/Los_Angeles")) + .toInstant()), false)); } @@ -193,12 +202,26 @@ public void testDynamicIndexFromSystemTime() { } @Test - public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZone() { + public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZoneUTC() { + assumeThat(ZoneId.systemDefault(), is(ZoneId.of("UTC"))); + IndexGenerator indexGenerator = IndexGeneratorFactory.createIndexGenerator("my-index-{local_timestamp|}", schema); indexGenerator.open(); - Assert.assertEquals("my-index-2020_03_18_12_12_14Z", indexGenerator.generate(rows.get(0))); - Assert.assertEquals("my-index-2020_03_19_12_12_14Z", indexGenerator.generate(rows.get(1))); + Assert.assertEquals("my-index-2020_03_17_19_12_14Z", indexGenerator.generate(rows.get(0))); + Assert.assertEquals("my-index-2020_03_20_03_22_14Z", indexGenerator.generate(rows.get(1))); + } + + @Test + public void testDynamicIndexDefaultFormatTimestampWithLocalTimeZoneWithSpecificTimeZone() { + IndexGenerator indexGenerator = + IndexGeneratorFactory.createIndexGenerator( + "my-index-{local_timestamp|}", schema, ZoneId.of("Europe/Berlin")); + indexGenerator.open(); + Assert.assertEquals( + "my-index-2020_03_17_20_12_14+01", indexGenerator.generate(rows.get(0))); + Assert.assertEquals( + "my-index-2020_03_20_04_22_14+01", indexGenerator.generate(rows.get(1))); } @Test From a833196ce58e9ce7a42306bc4ae44c6ed66283d5 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 7 Apr 2022 15:29:02 +0800 Subject: [PATCH 155/258] [FLINK-27108][python] Fix the state cache clean up logic This closes #19386. --- flink-python/pyflink/fn_execution/state_impl.py | 3 +++ .../python/beam/BeamPythonFunctionRunner.java | 12 +++++++----- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/flink-python/pyflink/fn_execution/state_impl.py b/flink-python/pyflink/fn_execution/state_impl.py index 0b2d2d5db88b9..590c37d465ece 100644 --- a/flink-python/pyflink/fn_execution/state_impl.py +++ b/flink-python/pyflink/fn_execution/state_impl.py @@ -87,6 +87,9 @@ def __len__(self): def __iter__(self): return iter(self._cache.values()) + def __contains__(self, key): + return key in self._cache + class SynchronousKvRuntimeState(InternalKvState, ABC): """ diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java index da4e111cdfabd..8a73aed799940 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java @@ -216,15 +216,17 @@ public void open(PythonConfig config) throws Exception { PortablePipelineOptions portableOptions = PipelineOptionsFactory.as(PortablePipelineOptions.class); - if (jobOptions.containsKey(PythonOptions.STATE_CACHE_SIZE.key())) { + int stateCacheSize = + Integer.parseInt( + jobOptions.getOrDefault( + PythonOptions.STATE_CACHE_SIZE.key(), + PythonOptions.STATE_CACHE_SIZE.defaultValue().toString())); + if (stateCacheSize > 0) { portableOptions .as(ExperimentalOptions.class) .setExperiments( Collections.singletonList( - ExperimentalOptions.STATE_CACHE_SIZE - + "=" - + jobOptions.get( - PythonOptions.STATE_CACHE_SIZE.key()))); + ExperimentalOptions.STATE_CACHE_SIZE + "=" + stateCacheSize)); } Struct pipelineOptions = PipelineOptionsTranslation.toProto(portableOptions); From c0c8b53a9713d692fc9c1547b41731f5e892d178 Mon Sep 17 00:00:00 2001 From: zp <764028663@qq.com> Date: Thu, 7 Apr 2022 12:11:36 +0800 Subject: [PATCH 156/258] [FLINK-27088][Documentation] Fix the example of using StringDeserializer for deserializing Kafka message value as string --- docs/content.zh/docs/connectors/datastream/kafka.md | 2 +- docs/content/docs/connectors/datastream/kafka.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/connectors/datastream/kafka.md b/docs/content.zh/docs/connectors/datastream/kafka.md index caecbfe302551..39010fbf791c2 100644 --- a/docs/content.zh/docs/connectors/datastream/kafka.md +++ b/docs/content.zh/docs/connectors/datastream/kafka.md @@ -102,7 +102,7 @@ Kafka Source 提供了 3 种 Topic / Partition 的订阅方式: import org.apache.kafka.common.serialization.StringDeserializer; KafkaSource.builder() - .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringSerializer.class)); + .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)); ``` ### 起始消费位点 diff --git a/docs/content/docs/connectors/datastream/kafka.md b/docs/content/docs/connectors/datastream/kafka.md index 8a623fc589b6a..8a1fbede6a02d 100644 --- a/docs/content/docs/connectors/datastream/kafka.md +++ b/docs/content/docs/connectors/datastream/kafka.md @@ -103,7 +103,7 @@ Kafka message value as string: import org.apache.kafka.common.serialization.StringDeserializer; KafkaSource.builder() - .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringSerializer.class)); + .setDeserializer(KafkaRecordDeserializationSchema.valueOnly(StringDeserializer.class)); ``` ### Starting Offset From 0a9bd82a29643fa6c3e88b4151e9e7c28cfef71c Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 25 Mar 2022 14:19:08 +0800 Subject: [PATCH 157/258] [FLINK-26190][python] Remove getTableConfig from ExecNodeConfiguration This closes #19333. --- .../stream_execution_environment.py | 2 +- .../test_stream_execution_environment.py | 14 +- .../pyflink/fn_execution/beam/beam_boot.py | 4 +- flink-python/pyflink/fn_execution/coders.py | 6 +- .../pyflink/table/table_environment.py | 8 +- .../org/apache/flink/python/PythonConfig.java | 189 +++++++----------- .../flink/python/PythonFunctionRunner.java | 3 +- .../apache/flink/python/PythonOptions.java | 36 +++- .../python/env/PythonDependencyInfo.java | 52 +++-- .../flink/python/util/PythonConfigUtil.java | 92 ++------- .../python/util/PythonDependencyUtils.java | 97 ++++----- ...bstractEmbeddedPythonFunctionOperator.java | 15 +- ...bstractExternalPythonFunctionOperator.java | 5 +- .../AbstractPythonFunctionOperator.java | 35 +--- .../python/PythonCoProcessOperator.java | 1 - .../python/PythonKeyedCoProcessOperator.java | 1 - .../python/PythonKeyedProcessOperator.java | 1 - .../python/PythonProcessOperator.java | 1 - .../BeamDataStreamPythonFunctionRunner.java | 9 +- .../python/beam/BeamPythonFunctionRunner.java | 66 +++--- .../beam/SimpleStateRequestHandler.java | 18 +- .../AbstractStatelessFunctionOperator.java | 1 - ...AbstractPythonStreamAggregateOperator.java | 17 +- ...tArrowPythonAggregateFunctionOperator.java | 6 +- ...onOverWindowAggregateFunctionOperator.java | 6 +- .../AbstractPythonScalarFunctionOperator.java | 7 +- .../EmbeddedPythonScalarFunctionOperator.java | 7 +- .../ArrowPythonScalarFunctionOperator.java | 3 +- .../table/PythonTableFunctionOperator.java | 6 +- .../beam/BeamTablePythonFunctionRunner.java | 7 - .../apache/flink/python/PythonConfigTest.java | 146 -------------- .../python/env/PythonDependencyInfoTest.java | 28 ++- .../python/util/PythonConfigUtilTest.java | 12 -- .../util/PythonDependencyUtilsTest.java | 42 ++-- ...honStreamGroupWindowAggregateOperator.java | 1 - ...ythonStreamGroupAggregateOperatorTest.java | 2 - ...StreamGroupTableAggregateOperatorTest.java | 2 - ...honGroupAggregateFunctionOperatorTest.java | 2 - ...upWindowAggregateFunctionOperatorTest.java | 2 - ...erWindowAggregateFunctionOperatorTest.java | 2 - ...upWindowAggregateFunctionOperatorTest.java | 2 - ...ythonProcTimeBoundedRangeOperatorTest.java | 2 - ...PythonProcTimeBoundedRowsOperatorTest.java | 2 - ...PythonRowTimeBoundedRangeOperatorTest.java | 2 - ...wPythonRowTimeBoundedRowsOperatorTest.java | 2 - .../PythonScalarFunctionOperatorTest.java | 2 - ...ArrowPythonScalarFunctionOperatorTest.java | 2 - .../PythonTableFunctionOperatorTest.java | 2 - ...sThroughPythonAggregateFunctionRunner.java | 7 +- ...PassThroughPythonScalarFunctionRunner.java | 3 - .../PassThroughPythonTableFunctionRunner.java | 3 - ...ghStreamAggregatePythonFunctionRunner.java | 3 - ...upWindowAggregatePythonFunctionRunner.java | 4 - ...eamTableAggregatePythonFunctionRunner.java | 3 - .../plan/nodes/exec/ExecNodeConfig.java | 21 +- .../batch/BatchExecPythonGroupAggregate.java | 2 +- .../BatchExecPythonGroupWindowAggregate.java | 2 +- .../batch/BatchExecPythonOverAggregate.java | 2 +- .../exec/common/CommonExecPythonCalc.java | 2 +- .../common/CommonExecPythonCorrelate.java | 2 +- .../StreamExecPythonGroupAggregate.java | 2 +- .../StreamExecPythonGroupTableAggregate.java | 2 +- .../StreamExecPythonGroupWindowAggregate.java | 2 +- .../stream/StreamExecPythonOverAggregate.java | 2 +- .../nodes/exec/utils/CommonPythonUtil.java | 25 +-- 65 files changed, 374 insertions(+), 683 deletions(-) delete mode 100644 flink-python/src/test/java/org/apache/flink/python/PythonConfigTest.java diff --git a/flink-python/pyflink/datastream/stream_execution_environment.py b/flink-python/pyflink/datastream/stream_execution_environment.py index 545d2b3e0185b..d6469310cecec 100644 --- a/flink-python/pyflink/datastream/stream_execution_environment.py +++ b/flink-python/pyflink/datastream/stream_execution_environment.py @@ -964,7 +964,7 @@ def startup_loopback_server(): BeamFnLoopbackWorkerPoolServicer config = Configuration(j_configuration=j_configuration) config.set_string( - "PYFLINK_LOOPBACK_SERVER_ADDRESS", BeamFnLoopbackWorkerPoolServicer().start()) + "python.loopback-server.address", BeamFnLoopbackWorkerPoolServicer().start()) python_worker_execution_mode = os.environ.get('_python_worker_execution_mode') diff --git a/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py b/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py index dd5dc475ffa33..16b129f4505f1 100644 --- a/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py +++ b/flink-python/pyflink/datastream/tests/test_stream_execution_environment.py @@ -445,10 +445,10 @@ def plus_three(value): from test_dep2 import add_three return add_three(value) + env.add_python_file(python_file_path) t_env = StreamTableEnvironment.create( stream_execution_environment=env, environment_settings=EnvironmentSettings.in_streaming_mode()) - env.add_python_file(python_file_path) from pyflink.table.udf import udf from pyflink.table.expressions import col @@ -678,13 +678,15 @@ def add_from_file(i): # The parallelism of Sink: Test Sink should be 4 self.assertEqual(nodes[4]['parallelism'], 4) - env_config_with_dependencies = dict(get_gateway().jvm.org.apache.flink.python.util - .PythonConfigUtil.getEnvConfigWithDependencies( - env._j_stream_execution_environment).toMap()) + python_dependency_config = dict( + get_gateway().jvm.org.apache.flink.python.util.PythonDependencyUtils. + configurePythonDependencies( + env._j_stream_execution_environment.getCachedFiles(), + env._j_stream_execution_environment.getConfiguration()).toMap()) # Make sure that user specified files and archives are correctly added. - self.assertIsNotNone(env_config_with_dependencies['python.files']) - self.assertIsNotNone(env_config_with_dependencies['python.archives']) + self.assertIsNotNone(python_dependency_config['python.internal.files-key-map']) + self.assertIsNotNone(python_dependency_config['python.internal.archives-key-map']) def test_register_slot_sharing_group(self): slot_sharing_group_1 = SlotSharingGroup.builder('slot_sharing_group_1') \ diff --git a/flink-python/pyflink/fn_execution/beam/beam_boot.py b/flink-python/pyflink/fn_execution/beam/beam_boot.py index 411745c0c23e6..4908944864ce6 100644 --- a/flink-python/pyflink/fn_execution/beam/beam_boot.py +++ b/flink-python/pyflink/fn_execution/beam/beam_boot.py @@ -75,12 +75,12 @@ def check_not_empty(check_str, error_message): logging.info("Initializing Python harness: %s" % " ".join(sys.argv)) - if 'PYFLINK_LOOPBACK_SERVER_ADDRESS' in os.environ: + if 'PYTHON_LOOPBACK_SERVER_ADDRESS' in os.environ: logging.info("Starting up Python harness in loopback mode.") params = dict(os.environ) params.update({'SEMI_PERSISTENT_DIRECTORY': semi_persist_dir}) - with grpc.insecure_channel(os.environ['PYFLINK_LOOPBACK_SERVER_ADDRESS']) as channel: + with grpc.insecure_channel(os.environ['PYTHON_LOOPBACK_SERVER_ADDRESS']) as channel: client = BeamFnExternalWorkerPoolStub(channel=channel) request = StartWorkerRequest( worker_id=worker_id, diff --git a/flink-python/pyflink/fn_execution/coders.py b/flink-python/pyflink/fn_execution/coders.py index 2ddd2f30109f2..7e43f78b2b243 100644 --- a/flink-python/pyflink/fn_execution/coders.py +++ b/flink-python/pyflink/fn_execution/coders.py @@ -79,12 +79,12 @@ def _to_field_coder(cls, coder_info_descriptor_proto): field_names = [f.name for f in schema_proto.fields] return RowCoder(field_coders, field_names) elif coder_info_descriptor_proto.HasField('arrow_type'): - timezone = pytz.timezone(os.environ['table.exec.timezone']) + timezone = pytz.timezone(os.environ['TABLE_LOCAL_TIME_ZONE']) schema_proto = coder_info_descriptor_proto.arrow_type.schema row_type = cls._to_row_type(schema_proto) return ArrowCoder(cls._to_arrow_schema(row_type), row_type, timezone) elif coder_info_descriptor_proto.HasField('over_window_arrow_type'): - timezone = pytz.timezone(os.environ['table.exec.timezone']) + timezone = pytz.timezone(os.environ['TABLE_LOCAL_TIME_ZONE']) schema_proto = coder_info_descriptor_proto.over_window_arrow_type.schema row_type = cls._to_row_type(schema_proto) return OverWindowArrowCoder( @@ -633,7 +633,7 @@ def from_proto(field_type): if field_type_name == type_name.TIMESTAMP: return TimestampCoder(field_type.timestamp_info.precision) if field_type_name == type_name.LOCAL_ZONED_TIMESTAMP: - timezone = pytz.timezone(os.environ['table.exec.timezone']) + timezone = pytz.timezone(os.environ['TABLE_LOCAL_TIME_ZONE']) return LocalZonedTimestampCoder(field_type.local_zoned_timestamp_info.precision, timezone) elif field_type_name == type_name.BASIC_ARRAY: return GenericArrayCoder(from_proto(field_type.collection_element_type)) diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py index 07ed66c3c827e..2e38ad9006a4d 100644 --- a/flink-python/pyflink/table/table_environment.py +++ b/flink-python/pyflink/table/table_environment.py @@ -1625,14 +1625,10 @@ def _config_chaining_optimization(self): def _open(self): # start BeamFnLoopbackWorkerPoolServicer when executed in MiniCluster def startup_loopback_server(): - from pyflink.common import Configuration from pyflink.fn_execution.beam.beam_worker_pool_service import \ BeamFnLoopbackWorkerPoolServicer - - j_configuration = get_j_env_configuration(self._get_j_env()) - config = Configuration(j_configuration=j_configuration) - config.set_string( - "PYFLINK_LOOPBACK_SERVER_ADDRESS", BeamFnLoopbackWorkerPoolServicer().start()) + self.get_config().set("python.loopback-server.address", + BeamFnLoopbackWorkerPoolServicer().start()) python_worker_execution_mode = os.environ.get('_python_worker_execution_mode') diff --git a/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java b/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java index 5195f1f78fa13..e5c24f9beba8b 100644 --- a/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java +++ b/flink-python/src/main/java/org/apache/flink/python/PythonConfig.java @@ -19,152 +19,101 @@ package org.apache.flink.python; import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigUtils; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.python.util.PythonDependencyUtils; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.util.Preconditions; -import javax.annotation.Nullable; - -import java.io.Serializable; +import java.time.ZoneId; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; +import static org.apache.flink.python.PythonOptions.PYTHON_LOOPBACK_SERVER_ADDRESS; + /** Configurations for the Python job which are used at run time. */ @Internal -public class PythonConfig implements Serializable { - - private static final long serialVersionUID = 1L; - - /** Max number of elements to include in a bundle. */ - private final int maxBundleSize; - - /** Max duration of a bundle. */ - private final long maxBundleTimeMills; +public class PythonConfig implements ReadableConfig { - /** Max number of elements to include in an arrow batch. */ - private final int maxArrowBatchSize; + private static final List> PYTHON_CONFIG_OPTIONS; - /** - * The python files uploaded by pyflink.table.TableEnvironment#add_python_file() or command line - * option "-pyfs". The key is the file key in distribute cache and the value is the - * corresponding origin file name. - */ - private final Map pythonFilesInfo; - - /** - * The file key of the requirements file in distribute cache. It is specified by - * pyflink.table.TableEnvironment#set_python_requirements() or command line option "-pyreq". - */ - @Nullable private final String pythonRequirementsFileInfo; - - /** - * The file key of the requirements cached directory in distribute cache. It is specified by - * pyflink.table.TableEnvironment#set_python_requirements() or command line option "-pyreq". It - * is used to support installing python packages offline. - */ - @Nullable private final String pythonRequirementsCacheDirInfo; + static { + PYTHON_CONFIG_OPTIONS = + new ArrayList<>(ConfigUtils.getAllConfigOptions(PythonOptions.class)); + } /** - * The python archives uploaded by pyflink.table.TableEnvironment#add_python_archive() or - * command line option "-pyarch". The key is the file key of the archives in distribute cache - * and the value is the name of the directory to extract to. + * Configuration adopted from the outer layer, e.g. flink-conf.yaml, command line arguments, + * TableConfig, etc. */ - private final Map pythonArchivesInfo; + private final ReadableConfig configuration; /** - * The path of the python interpreter (e.g. /usr/local/bin/python) specified by - * pyflink.table.TableConfig#set_python_executable() or command line option "-pyexec". + * Configuration generated in the dependency management mechanisms. See {@link + * PythonDependencyUtils.PythonDependencyManager} for more details. */ - private final String pythonExec; - - /** Whether metric is enabled. */ - private final boolean metricEnabled; - - /** Whether to use managed memory for the Python worker. */ - private final boolean isUsingManagedMemory; - - /** The Configuration that contains execution configs and dependencies info. */ - private final Configuration config; - - /** Whether profile is enabled. */ - private final boolean profileEnabled; - - /** Execution Mode. */ - private final String executionMode; - - public PythonConfig(Configuration config) { - this.config = config; - maxBundleSize = config.get(PythonOptions.MAX_BUNDLE_SIZE); - maxBundleTimeMills = config.get(PythonOptions.MAX_BUNDLE_TIME_MILLS); - maxArrowBatchSize = config.get(PythonOptions.MAX_ARROW_BATCH_SIZE); - pythonFilesInfo = - config.getOptional(PythonDependencyUtils.PYTHON_FILES).orElse(new HashMap<>()); - pythonRequirementsFileInfo = - config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) - .orElse(new HashMap<>()) - .get(PythonDependencyUtils.FILE); - pythonRequirementsCacheDirInfo = - config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) - .orElse(new HashMap<>()) - .get(PythonDependencyUtils.CACHE); - pythonArchivesInfo = - config.getOptional(PythonDependencyUtils.PYTHON_ARCHIVES).orElse(new HashMap<>()); - pythonExec = config.get(PythonOptions.PYTHON_EXECUTABLE); - metricEnabled = config.getBoolean(PythonOptions.PYTHON_METRIC_ENABLED); - isUsingManagedMemory = config.getBoolean(PythonOptions.USE_MANAGED_MEMORY); - profileEnabled = config.getBoolean(PythonOptions.PYTHON_PROFILE_ENABLED); - executionMode = config.getString(PythonOptions.PYTHON_EXECUTION_MODE); - } - - public int getMaxBundleSize() { - return maxBundleSize; - } + private final ReadableConfig pythonDependencyConfiguration; - public long getMaxBundleTimeMills() { - return maxBundleTimeMills; + public PythonConfig( + ReadableConfig configuration, ReadableConfig pythonDependencyConfiguration) { + this.configuration = Preconditions.checkNotNull(configuration); + this.pythonDependencyConfiguration = + Preconditions.checkNotNull(pythonDependencyConfiguration); } - public int getMaxArrowBatchSize() { - return maxArrowBatchSize; + @Override + public T get(ConfigOption option) { + return pythonDependencyConfiguration + .getOptional(option) + .orElseGet(() -> configuration.get(option)); } - public Map getPythonFilesInfo() { - return pythonFilesInfo; + @Override + public Optional getOptional(ConfigOption option) { + final Optional value = pythonDependencyConfiguration.getOptional(option); + if (value.isPresent()) { + return value; + } + return configuration.getOptional(option); } - public Optional getPythonRequirementsFileInfo() { - return Optional.ofNullable(pythonRequirementsFileInfo); - } - - public Optional getPythonRequirementsCacheDirInfo() { - return Optional.ofNullable(pythonRequirementsCacheDirInfo); - } - - public Map getPythonArchivesInfo() { - return pythonArchivesInfo; - } - - public String getPythonExec() { - return pythonExec; - } - - public String getExecutionMode() { - return executionMode; - } + public Configuration toConfiguration() { + final Configuration config = new Configuration(); + PYTHON_CONFIG_OPTIONS.forEach( + option -> + getOptional((ConfigOption) option) + .ifPresent(v -> config.set((ConfigOption) option, v))); + + // prepare the job options + Map jobOptions = config.get(PythonOptions.PYTHON_JOB_OPTIONS); + if (jobOptions == null) { + jobOptions = new HashMap<>(); + config.set(PythonOptions.PYTHON_JOB_OPTIONS, jobOptions); + } + jobOptions.put("TABLE_LOCAL_TIME_ZONE", getLocalTimeZone(configuration).getId()); + if (config.contains(PYTHON_LOOPBACK_SERVER_ADDRESS)) { + jobOptions.put( + "PYTHON_LOOPBACK_SERVER_ADDRESS", config.get(PYTHON_LOOPBACK_SERVER_ADDRESS)); + } - public boolean isMetricEnabled() { - return metricEnabled; - } - - public boolean isProfileEnabled() { - return profileEnabled; - } - - public boolean isUsingManagedMemory() { - return isUsingManagedMemory; + return config; } - public Configuration getConfig() { - return config; + /** + * Returns the current session time zone id. It is used when converting to/from {@code TIMESTAMP + * WITH LOCAL TIME ZONE}. + * + * @see org.apache.flink.table.types.logical.LocalZonedTimestampType + */ + private static ZoneId getLocalTimeZone(ReadableConfig config) { + String zone = config.get(TableConfigOptions.LOCAL_TIME_ZONE); + return TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone) + ? ZoneId.systemDefault() + : ZoneId.of(zone); } } diff --git a/flink-python/src/main/java/org/apache/flink/python/PythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/python/PythonFunctionRunner.java index 710a04605555d..4a4174a768d74 100644 --- a/flink-python/src/main/java/org/apache/flink/python/PythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/python/PythonFunctionRunner.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; /** The base interface of runner which is responsible for the execution of Python functions. */ @Internal @@ -28,7 +29,7 @@ public interface PythonFunctionRunner { /** * Prepares the Python function runner, such as preparing the Python execution environment, etc. */ - void open(PythonConfig config) throws Exception; + void open(ReadableConfig config) throws Exception; /** Tear-down the Python function runner. */ void close() throws Exception; diff --git a/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java b/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java index 78cd3b5556ca9..47bfbd3d56709 100644 --- a/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java +++ b/flink-python/src/main/java/org/apache/flink/python/PythonOptions.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -20,11 +20,14 @@ import org.apache.flink.annotation.Experimental; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.annotation.docs.Documentation; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.configuration.description.Description; +import java.util.Map; + /** Configuration options for the Python API. */ @PublicEvolving public class PythonOptions { @@ -236,4 +239,35 @@ public class PythonOptions { + "The `thread` mode means that the Python user-defined functions will be executed in the same process of the Java operator. " + "Note that currently it still doesn't support to execute Python user-defined functions in `thread` mode in all places. " + "It will fall back to `process` mode in these cases."); + + // ------------------------------------------------------------------------------------------ + // config options used for internal purpose + // ------------------------------------------------------------------------------------------ + + @Documentation.ExcludeFromDocumentation( + "Internal use only. The options will be exported as environment variables which could be accessed in Python worker process.") + public static final ConfigOption> PYTHON_JOB_OPTIONS = + ConfigOptions.key("python.job-options").mapType().noDefaultValue(); + + @Documentation.ExcludeFromDocumentation( + "Internal use only. The distributed cache entries for 'python.files'.") + public static final ConfigOption> PYTHON_FILES_DISTRIBUTED_CACHE_INFO = + ConfigOptions.key("python.internal.files-key-map").mapType().noDefaultValue(); + + @Documentation.ExcludeFromDocumentation( + "Internal use only. The distributed cache entries for 'python.requirements'.") + public static final ConfigOption> + PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO = + ConfigOptions.key("python.internal.requirements-file-key") + .mapType() + .noDefaultValue(); + + @Documentation.ExcludeFromDocumentation( + "Internal use only. The distributed cache entries for 'python.archives'.") + public static final ConfigOption> PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO = + ConfigOptions.key("python.internal.archives-key-map").mapType().noDefaultValue(); + + @Documentation.ExcludeFromDocumentation("Internal use only. Used for local debug.") + public static final ConfigOption PYTHON_LOOPBACK_SERVER_ADDRESS = + ConfigOptions.key("python.loopback-server.address").stringType().noDefaultValue(); } diff --git a/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java b/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java index 4ef685203b537..80f570b271491 100644 --- a/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java +++ b/flink-python/src/main/java/org/apache/flink/python/env/PythonDependencyInfo.java @@ -20,8 +20,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.cache.DistributedCache; -import org.apache.flink.python.PythonConfig; -import org.apache.flink.python.PythonOptions; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.python.util.PythonDependencyUtils; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -33,6 +33,12 @@ import java.util.Objects; import java.util.Optional; +import static org.apache.flink.python.PythonOptions.PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO; +import static org.apache.flink.python.PythonOptions.PYTHON_EXECUTABLE; +import static org.apache.flink.python.PythonOptions.PYTHON_EXECUTION_MODE; +import static org.apache.flink.python.PythonOptions.PYTHON_FILES_DISTRIBUTED_CACHE_INFO; +import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO; + /** PythonDependencyInfo contains the information of third-party dependencies. */ @Internal public final class PythonDependencyInfo { @@ -85,7 +91,7 @@ public PythonDependencyInfo( requirementsCacheDir, archives, pythonExec, - PythonOptions.PYTHON_EXECUTION_MODE.defaultValue()); + PYTHON_EXECUTION_MODE.defaultValue()); } public PythonDependencyInfo( @@ -130,15 +136,18 @@ public String getExecutionMode() { /** * Creates PythonDependencyInfo from GlobalJobParameters and DistributedCache. * - * @param pythonConfig The python config. + * @param config The config. * @param distributedCache The DistributedCache object of current task. * @return The PythonDependencyInfo object that contains whole information of python dependency. */ public static PythonDependencyInfo create( - PythonConfig pythonConfig, DistributedCache distributedCache) { + ReadableConfig config, DistributedCache distributedCache) { Map pythonFiles = new LinkedHashMap<>(); - for (Map.Entry entry : pythonConfig.getPythonFilesInfo().entrySet()) { + for (Map.Entry entry : + config.getOptional(PYTHON_FILES_DISTRIBUTED_CACHE_INFO) + .orElse(new HashMap<>()) + .entrySet()) { File pythonFile = distributedCache.getFile(entry.getKey()); String filePath = pythonFile.getAbsolutePath(); pythonFiles.put(filePath, entry.getValue()); @@ -146,27 +155,34 @@ public static PythonDependencyInfo create( String requirementsFilePath = null; String requirementsCacheDir = null; - if (pythonConfig.getPythonRequirementsFileInfo().isPresent()) { - requirementsFilePath = - distributedCache - .getFile(pythonConfig.getPythonRequirementsFileInfo().get()) - .getAbsolutePath(); - if (pythonConfig.getPythonRequirementsCacheDirInfo().isPresent()) { + + String requirementsFileName = + config.getOptional(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) + .orElse(new HashMap<>()) + .get(PythonDependencyUtils.FILE); + if (requirementsFileName != null) { + requirementsFilePath = distributedCache.getFile(requirementsFileName).getAbsolutePath(); + String requirementsFileCacheDir = + config.getOptional(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) + .orElse(new HashMap<>()) + .get(PythonDependencyUtils.CACHE); + if (requirementsFileCacheDir != null) { requirementsCacheDir = - distributedCache - .getFile(pythonConfig.getPythonRequirementsCacheDirInfo().get()) - .getAbsolutePath(); + distributedCache.getFile(requirementsFileCacheDir).getAbsolutePath(); } } Map archives = new HashMap<>(); - for (Map.Entry entry : pythonConfig.getPythonArchivesInfo().entrySet()) { + for (Map.Entry entry : + config.getOptional(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) + .orElse(new HashMap<>()) + .entrySet()) { String archiveFilePath = distributedCache.getFile(entry.getKey()).getAbsolutePath(); String targetPath = entry.getValue(); archives.put(archiveFilePath, targetPath); } - String pythonExec = pythonConfig.getPythonExec(); + String pythonExec = config.get(PYTHON_EXECUTABLE); return new PythonDependencyInfo( pythonFiles, @@ -174,6 +190,6 @@ public static PythonDependencyInfo create( requirementsCacheDir, archives, pythonExec, - pythonConfig.getExecutionMode()); + config.get(PYTHON_EXECUTION_MODE)); } } diff --git a/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java b/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java index 18ddb5845c085..1da956fb04755 100644 --- a/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java +++ b/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java @@ -17,12 +17,11 @@ package org.apache.flink.python.util; -import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.dag.Transformation; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; +import org.apache.flink.python.PythonConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; @@ -34,8 +33,6 @@ import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner; -import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.TableException; import org.apache.flink.shaded.guava30.com.google.common.collect.Queues; import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; @@ -47,10 +44,7 @@ import java.util.Queue; import java.util.Set; -/** - * A Util class to get the {@link StreamExecutionEnvironment} configuration and merged configuration - * with environment settings. - */ +/** A Util class to handle the configurations of Python jobs. */ public class PythonConfigUtil { public static final String KEYED_STREAM_VALUE_OPERATOR_NAME = "_keyed_stream_values_operator"; @@ -59,19 +53,9 @@ public class PythonConfigUtil { "_partition_custom_map_operator"; /** - * A static method to get the {@link StreamExecutionEnvironment} configuration merged with - * python dependency management configurations. - */ - public static Configuration getEnvConfigWithDependencies(StreamExecutionEnvironment env) - throws InvocationTargetException, IllegalAccessException, NoSuchFieldException { - return PythonDependencyUtils.configurePythonDependencies( - env.getCachedFiles(), (Configuration) env.getConfiguration()); - } - - /** - * Get the private field {@code StreamExecutionEnvironment#configuration} by reflection - * recursively. Then access the field to get the configuration of the given - * StreamExecutionEnvironment. + * Get the private field {@link StreamExecutionEnvironment#configuration} by reflection + * recursively. It allows modification to the configuration compared with {@link + * StreamExecutionEnvironment#getConfiguration()}. */ public static Configuration getEnvironmentConfig(StreamExecutionEnvironment env) throws InvocationTargetException, IllegalAccessException, NoSuchFieldException { @@ -93,62 +77,33 @@ public static Configuration getEnvironmentConfig(StreamExecutionEnvironment env) return (Configuration) configurationField.get(env); } - @SuppressWarnings("unchecked") public static void configPythonOperator(StreamExecutionEnvironment env) - throws IllegalAccessException, InvocationTargetException, NoSuchFieldException { - Configuration mergedConfig = getEnvConfigWithDependencies(env); + throws IllegalAccessException, NoSuchFieldException { + final Configuration config = extractPythonConfiguration(env, env.getConfiguration()); - Field transformationsField = - StreamExecutionEnvironment.class.getDeclaredField("transformations"); - transformationsField.setAccessible(true); - List> transformations = - (List>) transformationsField.get(env); - for (Transformation transformation : transformations) { + for (Transformation transformation : env.getTransformations()) { alignTransformation(transformation); if (isPythonOperator(transformation)) { - // declare it is a Python operator + // declare the use case of managed memory transformation.declareManagedMemoryUseCaseAtSlotScope(ManagedMemoryUseCase.PYTHON); AbstractPythonFunctionOperator pythonFunctionOperator = getPythonOperator(transformation); if (pythonFunctionOperator != null) { - Configuration oldConfig = pythonFunctionOperator.getConfiguration(); - // update dependency related configurations for Python operators - pythonFunctionOperator.setConfiguration( - generateNewPythonConfig(oldConfig, mergedConfig)); + pythonFunctionOperator.getConfiguration().addAll(config); } } } } - public static Configuration getMergedConfig( - StreamExecutionEnvironment env, TableConfig tableConfig) { - Configuration config = new Configuration((Configuration) env.getConfiguration()); - PythonDependencyUtils.merge(config, tableConfig.getConfiguration()); - Configuration mergedConfig = + /** Extract the configurations which is used in the Python operators. */ + public static Configuration extractPythonConfiguration( + StreamExecutionEnvironment env, ReadableConfig config) { + final Configuration pythonDependencyConfig = PythonDependencyUtils.configurePythonDependencies(env.getCachedFiles(), config); - mergedConfig.setString("table.exec.timezone", tableConfig.getLocalTimeZone().getId()); - return mergedConfig; - } - - @SuppressWarnings("unchecked") - public static Configuration getMergedConfig(ExecutionEnvironment env, TableConfig tableConfig) { - try { - Field field = ExecutionEnvironment.class.getDeclaredField("cacheFile"); - field.setAccessible(true); - Configuration config = new Configuration(env.getConfiguration()); - PythonDependencyUtils.merge(config, tableConfig.getConfiguration()); - Configuration mergedConfig = - PythonDependencyUtils.configurePythonDependencies( - (List>) - field.get(env), - config); - mergedConfig.setString("table.exec.timezone", tableConfig.getLocalTimeZone().getId()); - return mergedConfig; - } catch (NoSuchFieldException | IllegalAccessException e) { - throw new TableException("Method getMergedConfig failed.", e); - } + final PythonConfig pythonConfig = new PythonConfig(config, pythonDependencyConfig); + return pythonConfig.toConfiguration(); } /** @@ -220,7 +175,7 @@ private static AbstractPythonFunctionOperator getPythonOperator( return null; } - public static boolean isPythonOperator(Transformation transform) { + private static boolean isPythonOperator(Transformation transform) { if (transform instanceof OneInputTransformation) { return isPythonOperator( ((OneInputTransformation) transform).getOperatorFactory()); @@ -266,17 +221,6 @@ private static boolean isPythonDataStreamOperator( } } - /** - * Generator a new {@link Configuration} with the combined config which is derived from - * oldConfig. - */ - private static Configuration generateNewPythonConfig( - Configuration oldConfig, Configuration newConfig) { - Configuration mergedConfig = newConfig.clone(); - mergedConfig.addAll(oldConfig); - return mergedConfig; - } - public static void setPartitionCustomOperatorNumPartitions( List> transformations) { // Update the numPartitions of PartitionCustomOperator after aligned all operators. diff --git a/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java b/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java index f90aa72f8c333..c93a6f4994901 100644 --- a/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java +++ b/flink-python/src/main/java/org/apache/flink/python/util/PythonDependencyUtils.java @@ -20,8 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.python.PythonOptions; @@ -39,7 +37,6 @@ import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import static org.apache.flink.client.cli.CliFrontendParser.PYARCHIVE_OPTION; @@ -47,8 +44,11 @@ import static org.apache.flink.client.cli.CliFrontendParser.PYEXEC_OPTION; import static org.apache.flink.client.cli.CliFrontendParser.PYFILES_OPTION; import static org.apache.flink.client.cli.CliFrontendParser.PYREQUIREMENTS_OPTION; +import static org.apache.flink.python.PythonOptions.PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO; import static org.apache.flink.python.PythonOptions.PYTHON_CLIENT_EXECUTABLE; import static org.apache.flink.python.PythonOptions.PYTHON_EXECUTABLE; +import static org.apache.flink.python.PythonOptions.PYTHON_FILES_DISTRIBUTED_CACHE_INFO; +import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO; /** * Utility class for Python dependency management. The dependencies will be registered at the @@ -63,15 +63,6 @@ public class PythonDependencyUtils { public static final String PARAM_DELIMITER = "#"; private static final String HASH_ALGORITHM = "SHA-256"; - // Internal Python Config Options. - - public static final ConfigOption> PYTHON_FILES = - ConfigOptions.key("python.internal.files-key-map").mapType().noDefaultValue(); - public static final ConfigOption> PYTHON_REQUIREMENTS_FILE = - ConfigOptions.key("python.internal.requirements-file-key").mapType().noDefaultValue(); - public static final ConfigOption> PYTHON_ARCHIVES = - ConfigOptions.key("python.internal.archives-key-map").mapType().noDefaultValue(); - /** * Adds python dependencies to registered cache file list according to given configuration and * returns a new configuration which contains the metadata of the registered python @@ -84,10 +75,12 @@ public class PythonDependencyUtils { */ public static Configuration configurePythonDependencies( List> cachedFiles, - Configuration config) { - PythonDependencyManager pythonDependencyManager = + ReadableConfig config) { + final PythonDependencyManager pythonDependencyManager = new PythonDependencyManager(cachedFiles, config); - return pythonDependencyManager.getConfigWithPythonDependencyOptions(); + final Configuration pythonDependencyConfig = new Configuration(); + pythonDependencyManager.applyToConfiguration(pythonDependencyConfig); + return pythonDependencyConfig; } public static Configuration parsePythonDependencyConfiguration(CommandLine commandLine) { @@ -162,14 +155,13 @@ private static class PythonDependencyManager { private static final String PYTHON_ARCHIVE_PREFIX = "python_archive"; private final List> cachedFiles; - private final Configuration internalConfig; + private final ReadableConfig config; private PythonDependencyManager( List> cachedFiles, - Configuration config) { + ReadableConfig config) { this.cachedFiles = cachedFiles; - this.internalConfig = new Configuration(config); - configure(config); + this.config = config; } /** @@ -179,14 +171,17 @@ private PythonDependencyManager( * * @param filePath The path of the Python dependency. */ - private void addPythonFile(String filePath) { + private void addPythonFile(Configuration pythonDependencyConfig, String filePath) { Preconditions.checkNotNull(filePath); String fileKey = generateUniqueFileKey(PYTHON_FILE_PREFIX, filePath); registerCachedFileIfNotExist(filePath, fileKey); - if (!internalConfig.contains(PYTHON_FILES)) { - internalConfig.set(PYTHON_FILES, new LinkedHashMap<>()); + if (!pythonDependencyConfig.contains(PYTHON_FILES_DISTRIBUTED_CACHE_INFO)) { + pythonDependencyConfig.set( + PYTHON_FILES_DISTRIBUTED_CACHE_INFO, new LinkedHashMap<>()); } - internalConfig.get(PYTHON_FILES).put(fileKey, new File(filePath).getName()); + pythonDependencyConfig + .get(PYTHON_FILES_DISTRIBUTED_CACHE_INFO) + .put(fileKey, new File(filePath).getName()); } /** @@ -196,8 +191,9 @@ private void addPythonFile(String filePath) { * * @param requirementsFilePath The path of the requirements file. */ - private void setPythonRequirements(String requirementsFilePath) { - setPythonRequirements(requirementsFilePath, null); + private void setPythonRequirements( + Configuration pythonDependencyConfig, String requirementsFilePath) { + setPythonRequirements(pythonDependencyConfig, requirementsFilePath, null); } /** @@ -210,26 +206,33 @@ private void setPythonRequirements(String requirementsFilePath) { * @param requirementsCachedDir The path of the requirements cached directory. */ private void setPythonRequirements( - String requirementsFilePath, @Nullable String requirementsCachedDir) { + Configuration pythonDependencyConfig, + String requirementsFilePath, + @Nullable String requirementsCachedDir) { Preconditions.checkNotNull(requirementsFilePath); - if (!internalConfig.contains(PYTHON_REQUIREMENTS_FILE)) { - internalConfig.set(PYTHON_REQUIREMENTS_FILE, new HashMap<>()); + if (!pythonDependencyConfig.contains(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO)) { + pythonDependencyConfig.set( + PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO, new HashMap<>()); } - internalConfig.get(PYTHON_REQUIREMENTS_FILE).clear(); + pythonDependencyConfig.get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO).clear(); removeCachedFilesByPrefix(PYTHON_REQUIREMENTS_FILE_PREFIX); removeCachedFilesByPrefix(PYTHON_REQUIREMENTS_CACHE_PREFIX); String fileKey = generateUniqueFileKey(PYTHON_REQUIREMENTS_FILE_PREFIX, requirementsFilePath); registerCachedFileIfNotExist(requirementsFilePath, fileKey); - internalConfig.get(PYTHON_REQUIREMENTS_FILE).put(FILE, fileKey); + pythonDependencyConfig + .get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) + .put(FILE, fileKey); if (requirementsCachedDir != null) { String cacheDirKey = generateUniqueFileKey( PYTHON_REQUIREMENTS_CACHE_PREFIX, requirementsCachedDir); registerCachedFileIfNotExist(requirementsCachedDir, cacheDirKey); - internalConfig.get(PYTHON_REQUIREMENTS_FILE).put(CACHE, cacheDirKey); + pythonDependencyConfig + .get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) + .put(CACHE, cacheDirKey); } } @@ -242,24 +245,27 @@ private void setPythonRequirements( * @param archivePath The path of the archive file. * @param targetDir The name of the target directory. */ - private void addPythonArchive(String archivePath, String targetDir) { + private void addPythonArchive( + Configuration pythonDependencyConfig, String archivePath, String targetDir) { Preconditions.checkNotNull(archivePath); - if (!internalConfig.contains(PYTHON_ARCHIVES)) { - internalConfig.set(PYTHON_ARCHIVES, new HashMap<>()); + if (!pythonDependencyConfig.contains(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO)) { + pythonDependencyConfig.set(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO, new HashMap<>()); } String fileKey = generateUniqueFileKey( PYTHON_ARCHIVE_PREFIX, archivePath + PARAM_DELIMITER + targetDir); registerCachedFileIfNotExist(archivePath, fileKey); - internalConfig.get(PYTHON_ARCHIVES).put(fileKey, targetDir); + pythonDependencyConfig + .get(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) + .put(fileKey, targetDir); } - private void configure(ReadableConfig config) { + private void applyToConfiguration(Configuration pythonDependencyConfig) { config.getOptional(PythonOptions.PYTHON_FILES) .ifPresent( pyFiles -> { for (String filePath : pyFiles.split(FILE_DELIMITER)) { - addPythonFile(filePath); + addPythonFile(pythonDependencyConfig, filePath); } }); @@ -270,9 +276,11 @@ private void configure(ReadableConfig config) { String[] requirementFileAndCache = pyRequirements.split(PARAM_DELIMITER, 2); setPythonRequirements( - requirementFileAndCache[0], requirementFileAndCache[1]); + pythonDependencyConfig, + requirementFileAndCache[0], + requirementFileAndCache[1]); } else { - setPythonRequirements(pyRequirements); + setPythonRequirements(pythonDependencyConfig, pyRequirements); } }); @@ -294,15 +302,16 @@ private void configure(ReadableConfig config) { archivePath = archive; targetDir = new File(archivePath).getName(); } - addPythonArchive(archivePath, targetDir); + addPythonArchive( + pythonDependencyConfig, archivePath, targetDir); } }); config.getOptional(PYTHON_EXECUTABLE) - .ifPresent(e -> internalConfig.set(PYTHON_EXECUTABLE, e)); + .ifPresent(e -> pythonDependencyConfig.set(PYTHON_EXECUTABLE, e)); config.getOptional(PYTHON_CLIENT_EXECUTABLE) - .ifPresent(e -> internalConfig.set(PYTHON_CLIENT_EXECUTABLE, e)); + .ifPresent(e -> pythonDependencyConfig.set(PYTHON_CLIENT_EXECUTABLE, e)); } private String generateUniqueFileKey(String prefix, String hashString) { @@ -333,9 +342,5 @@ private void removeCachedFilesByPrefix(String prefix) { .filter(t -> t.f0.matches("^" + prefix + "_[a-z0-9]{64}$")) .collect(Collectors.toSet())); } - - private Configuration getConfigWithPythonDependencyOptions() { - return internalConfig; - } } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractEmbeddedPythonFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractEmbeddedPythonFunctionOperator.java index be709e65e924b..6dc961832ee4d 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractEmbeddedPythonFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractEmbeddedPythonFunctionOperator.java @@ -23,7 +23,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.python.PythonConfig; import org.apache.flink.python.env.PythonDependencyInfo; import org.apache.flink.python.env.embedded.EmbeddedPythonEnvironment; import org.apache.flink.python.env.embedded.EmbeddedPythonEnvironmentManager; @@ -36,6 +35,7 @@ import java.util.Map; import java.util.concurrent.locks.ReentrantLock; +import static org.apache.flink.python.PythonOptions.PYTHON_EXECUTABLE; import static org.apache.flink.python.env.AbstractPythonEnvironmentManager.PYTHON_WORKING_DIR; /** @@ -48,12 +48,9 @@ public abstract class AbstractEmbeddedPythonFunctionOperator private static final long serialVersionUID = 1L; - private static ReentrantLock lock = new ReentrantLock(); + private static final ReentrantLock lock = new ReentrantLock(); - private static Map> workingDirectories = new HashMap<>(); - - /** The python config. */ - protected transient PythonConfig pythonConfig; + private static final Map> workingDirectories = new HashMap<>(); /** Every operator will hold the only python interpreter. */ protected transient PythonInterpreter interpreter; @@ -67,7 +64,6 @@ public AbstractEmbeddedPythonFunctionOperator(Configuration config) { @Override public void open() throws Exception { super.open(); - pythonConfig = new PythonConfig(config); pythonEnvironmentManager = createPythonEnvironmentManager(); pythonEnvironmentManager.open(); EmbeddedPythonEnvironment environment = @@ -105,7 +101,7 @@ public void open() throws Exception { } } - openPythonInterpreter(pythonConfig.getPythonExec(), env); + openPythonInterpreter(config.get(PYTHON_EXECUTABLE), env); } @Override @@ -142,8 +138,7 @@ public void close() throws Exception { @Override protected EmbeddedPythonEnvironmentManager createPythonEnvironmentManager() { PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create( - pythonConfig, getRuntimeContext().getDistributedCache()); + PythonDependencyInfo.create(config, getRuntimeContext().getDistributedCache()); return new EmbeddedPythonEnvironmentManager( dependencyInfo, getContainingTask().getEnvironment().getTaskManagerInfo().getTmpDirectories(), diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractExternalPythonFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractExternalPythonFunctionOperator.java index 702e42e06973b..d95ffafb5afff 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractExternalPythonFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractExternalPythonFunctionOperator.java @@ -53,7 +53,7 @@ public AbstractExternalPythonFunctionOperator(Configuration config) { public void open() throws Exception { super.open(); this.pythonFunctionRunner = createPythonFunctionRunner(); - this.pythonFunctionRunner.open(pythonConfig); + this.pythonFunctionRunner.open(config); this.flushThreadPool = Executors.newSingleThreadExecutor(); } @@ -120,8 +120,7 @@ protected void invokeFinishBundle() throws Exception { @Override protected ProcessPythonEnvironmentManager createPythonEnvironmentManager() { PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create( - pythonConfig, getRuntimeContext().getDistributedCache()); + PythonDependencyInfo.create(config, getRuntimeContext().getDistributedCache()); PythonEnv pythonEnv = getPythonEnv(); if (pythonEnv.getExecType() == PythonEnv.ExecType.PROCESS) { return new ProcessPythonEnvironmentManager( diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java index aa1dfe1792093..9286b786506d4 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/AbstractPythonFunctionOperator.java @@ -20,8 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.Configuration; -import org.apache.flink.python.PythonConfig; -import org.apache.flink.python.PythonOptions; import org.apache.flink.python.env.PythonEnvironmentManager; import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.runtime.state.KeyedStateBackend; @@ -36,10 +34,12 @@ import java.lang.reflect.Field; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.concurrent.ScheduledFuture; +import static org.apache.flink.python.PythonOptions.MAX_BUNDLE_SIZE; +import static org.apache.flink.python.PythonOptions.MAX_BUNDLE_TIME_MILLS; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; import static org.apache.flink.streaming.api.utils.ClassLeakCleaner.cleanUpLeakingClasses; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; @@ -49,7 +49,7 @@ public abstract class AbstractPythonFunctionOperator extends AbstractStream private static final long serialVersionUID = 1L; - protected Configuration config; + protected final Configuration config; /** Max number of elements to include in a bundle. */ protected transient int maxBundleSize; @@ -57,12 +57,6 @@ public abstract class AbstractPythonFunctionOperator extends AbstractStream /** Number of processed elements in the current bundle. */ protected transient int elementCount; - /** The python config. */ - protected transient PythonConfig pythonConfig; - - /** The options used to configure the Python worker process. */ - protected transient Map jobOptions; - /** Max duration of a bundle. */ private transient long maxBundleTimeMills; @@ -83,12 +77,10 @@ public AbstractPythonFunctionOperator(Configuration config) { @Override public void open() throws Exception { try { - this.pythonConfig = new PythonConfig(config); - this.jobOptions = config.toMap(); - this.maxBundleSize = pythonConfig.getMaxBundleSize(); + this.maxBundleSize = config.get(MAX_BUNDLE_SIZE); if (this.maxBundleSize <= 0) { - this.maxBundleSize = PythonOptions.MAX_BUNDLE_SIZE.defaultValue(); - LOG.error( + this.maxBundleSize = MAX_BUNDLE_SIZE.defaultValue(); + LOG.warn( "Invalid value for the maximum bundle size. Using default value of " + this.maxBundleSize + '.'); @@ -96,10 +88,10 @@ public void open() throws Exception { LOG.info("The maximum bundle size is configured to {}.", this.maxBundleSize); } - this.maxBundleTimeMills = pythonConfig.getMaxBundleTimeMills(); + this.maxBundleTimeMills = config.get(MAX_BUNDLE_TIME_MILLS); if (this.maxBundleTimeMills <= 0L) { - this.maxBundleTimeMills = PythonOptions.MAX_BUNDLE_TIME_MILLS.defaultValue(); - LOG.error( + this.maxBundleTimeMills = MAX_BUNDLE_TIME_MILLS.defaultValue(); + LOG.warn( "Invalid value for the maximum bundle time. Using default value of " + this.maxBundleTimeMills + '.'); @@ -256,11 +248,6 @@ public boolean isBundleFinished() { return elementCount == 0; } - /** Reset the {@link Configuration} if needed. */ - public void setConfiguration(Configuration config) { - this.config = config; - } - /** Returns the {@link Configuration}. */ public Configuration getConfiguration() { return config; @@ -289,7 +276,7 @@ private void checkInvokeFinishBundleByTime() throws Exception { } protected FlinkMetricContainer getFlinkMetricContainer() { - return this.pythonConfig.isMetricEnabled() + return this.config.get(PYTHON_METRIC_ENABLED) ? new FlinkMetricContainer(getRuntimeContext().getMetricGroup()) : null; } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java index 820860014c7a3..2b519aeb24e65 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java @@ -74,7 +74,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getRuntimeContext(), getInternalParameters(), inBatchExecutionMode(getKeyedStateBackend())), - jobOptions, getFlinkMetricContainer(), null, null, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java index ba1c94c1a3f22..924b86122de4b 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java @@ -113,7 +113,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getInternalParameters(), keyTypeInfo, inBatchExecutionMode(getKeyedStateBackend())), - jobOptions, getFlinkMetricContainer(), getKeyedStateBackend(), keyTypeSerializer, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java index 3d81c1076bac4..cef5e6ce01dc3 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java @@ -143,7 +143,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getInternalParameters(), keyTypeInfo, inBatchExecutionMode(getKeyedStateBackend())), - jobOptions, getFlinkMetricContainer(), getKeyedStateBackend(), keyTypeSerializer, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java index 69d4e74146b42..1bc820e58585a 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java @@ -71,7 +71,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getRuntimeContext(), getInternalParameters(), inBatchExecutionMode(getKeyedStateBackend())), - jobOptions, getFlinkMetricContainer(), null, null, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java index a76871f7acbd9..df3a651b8d14b 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamDataStreamPythonFunctionRunner.java @@ -37,7 +37,6 @@ import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Optional; import static org.apache.flink.python.Constants.INPUT_COLLECTION_ID; @@ -71,11 +70,10 @@ public BeamDataStreamPythonFunctionRunner( ProcessPythonEnvironmentManager environmentManager, String headOperatorFunctionUrn, List userDefinedDataStreamFunctions, - Map jobOptions, @Nullable FlinkMetricContainer flinkMetricContainer, - KeyedStateBackend stateBackend, - TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, + @Nullable KeyedStateBackend stateBackend, + @Nullable TypeSerializer keySerializer, + @Nullable TypeSerializer namespaceSerializer, @Nullable TimerRegistration timerRegistration, MemoryManager memoryManager, double managedMemoryFraction, @@ -85,7 +83,6 @@ public BeamDataStreamPythonFunctionRunner( super( taskName, environmentManager, - jobOptions, flinkMetricContainer, stateBackend, keySerializer, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java index 8a73aed799940..ad04d9b5299bb 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/BeamPythonFunctionRunner.java @@ -22,8 +22,8 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.python.PythonConfig; import org.apache.flink.python.PythonFunctionRunner; import org.apache.flink.python.PythonOptions; import org.apache.flink.python.env.PythonEnvironment; @@ -94,6 +94,7 @@ import static org.apache.flink.python.Constants.WINDOW_CODER_ID; import static org.apache.flink.python.Constants.WINDOW_STRATEGY; import static org.apache.flink.python.Constants.WRAPPER_TIMER_CODER_ID; +import static org.apache.flink.python.PythonOptions.USE_MANAGED_MEMORY; import static org.apache.flink.streaming.api.utils.ProtoUtils.createCoderProto; /** A {@link BeamPythonFunctionRunner} used to execute Python functions. */ @@ -112,13 +113,16 @@ public abstract class BeamPythonFunctionRunner implements PythonFunctionRunner { /** The Python process execution environment manager. */ private final ProcessPythonEnvironmentManager environmentManager; - /** The options used to configure the Python worker process. */ - private final Map jobOptions; - /** The flinkMetricContainer will be set to null if metric is configured to be turned off. */ - @Nullable private FlinkMetricContainer flinkMetricContainer; + @Nullable private final FlinkMetricContainer flinkMetricContainer; + + @Nullable private final KeyedStateBackend keyedStateBackend; + + @Nullable private final TypeSerializer keySerializer; + + @Nullable private final TypeSerializer namespaceSerializer; - @Nullable private TimerRegistration timerRegistration; + @Nullable private final TimerRegistration timerRegistration; private final MemoryManager memoryManager; @@ -145,7 +149,7 @@ public abstract class BeamPythonFunctionRunner implements PythonFunctionRunner { private transient StageBundleFactory stageBundleFactory; /** Handler for state requests. */ - private final StateRequestHandler stateRequestHandler; + private transient StateRequestHandler stateRequestHandler; /** Handler for bundle progress messages, both during bundle execution and on its completion. */ private transient BundleProgressHandler progressHandler; @@ -178,11 +182,10 @@ public abstract class BeamPythonFunctionRunner implements PythonFunctionRunner { public BeamPythonFunctionRunner( String taskName, ProcessPythonEnvironmentManager environmentManager, - Map jobOptions, @Nullable FlinkMetricContainer flinkMetricContainer, - @Nullable KeyedStateBackend keyedStateBackend, - @Nullable TypeSerializer keySerializer, - @Nullable TypeSerializer namespaceSerializer, + @Nullable KeyedStateBackend keyedStateBackend, + @Nullable TypeSerializer keySerializer, + @Nullable TypeSerializer namespaceSerializer, @Nullable TimerRegistration timerRegistration, MemoryManager memoryManager, double managedMemoryFraction, @@ -190,11 +193,10 @@ public BeamPythonFunctionRunner( FlinkFnApi.CoderInfoDescriptor outputCoderDescriptor) { this.taskName = Preconditions.checkNotNull(taskName); this.environmentManager = Preconditions.checkNotNull(environmentManager); - this.jobOptions = Preconditions.checkNotNull(jobOptions); this.flinkMetricContainer = flinkMetricContainer; - this.stateRequestHandler = - getStateRequestHandler( - keyedStateBackend, keySerializer, namespaceSerializer, jobOptions); + this.keyedStateBackend = keyedStateBackend; + this.keySerializer = keySerializer; + this.namespaceSerializer = namespaceSerializer; this.timerRegistration = timerRegistration; this.memoryManager = memoryManager; this.managedMemoryFraction = managedMemoryFraction; @@ -205,22 +207,22 @@ public BeamPythonFunctionRunner( // ------------------------------------------------------------------------ @Override - public void open(PythonConfig config) throws Exception { + public void open(ReadableConfig config) throws Exception { this.bundleStarted = false; this.resultBuffer = new LinkedBlockingQueue<>(); this.reusableResultTuple = new Tuple2<>(); + stateRequestHandler = + getStateRequestHandler( + keyedStateBackend, keySerializer, namespaceSerializer, config); + // The creation of stageBundleFactory depends on the initialized environment manager. environmentManager.open(); PortablePipelineOptions portableOptions = PipelineOptionsFactory.as(PortablePipelineOptions.class); - int stateCacheSize = - Integer.parseInt( - jobOptions.getOrDefault( - PythonOptions.STATE_CACHE_SIZE.key(), - PythonOptions.STATE_CACHE_SIZE.defaultValue().toString())); + int stateCacheSize = config.get(PythonOptions.STATE_CACHE_SIZE); if (stateCacheSize > 0) { portableOptions .as(ExperimentalOptions.class) @@ -231,7 +233,7 @@ public void open(PythonConfig config) throws Exception { Struct pipelineOptions = PipelineOptionsTranslation.toProto(portableOptions); - if (memoryManager != null && config.isUsingManagedMemory()) { + if (memoryManager != null && config.get(USE_MANAGED_MEMORY)) { Preconditions.checkArgument( managedMemoryFraction > 0 && managedMemoryFraction <= 1.0, String.format( @@ -243,7 +245,7 @@ public void open(PythonConfig config) throws Exception { (size) -> new PythonSharedResources( createJobBundleFactory(pipelineOptions), - createPythonExecutionEnvironment(size)); + createPythonExecutionEnvironment(config, size)); sharedResources = memoryManager.getSharedMemoryResourceForManagedMemory( @@ -262,7 +264,7 @@ public void open(PythonConfig config) throws Exception { jobBundleFactory = createJobBundleFactory(pipelineOptions); stageBundleFactory = createStageBundleFactory( - jobBundleFactory, createPythonExecutionEnvironment(-1)); + jobBundleFactory, createPythonExecutionEnvironment(config, -1)); } progressHandler = getProgressHandler(flinkMetricContainer); } @@ -393,13 +395,13 @@ private void finishBundle() { * Creates a specification which specifies the portability Python execution environment. It's * used by Beam's portability framework to creates the actual Python execution environment. */ - private RunnerApi.Environment createPythonExecutionEnvironment(long memoryLimitBytes) - throws Exception { + private RunnerApi.Environment createPythonExecutionEnvironment( + ReadableConfig config, long memoryLimitBytes) throws Exception { PythonEnvironment environment = environmentManager.createEnvironment(); if (environment instanceof ProcessPythonEnvironment) { ProcessPythonEnvironment processEnvironment = (ProcessPythonEnvironment) environment; Map env = processEnvironment.getEnv(); - env.putAll(jobOptions); + config.getOptional(PythonOptions.PYTHON_JOB_OPTIONS).ifPresent(env::putAll); env.put(PYTHON_WORKER_MEMORY_LIMIT, String.valueOf(memoryLimitBytes)); return Environments.createProcessEnvironment( "", "", processEnvironment.getCommand(), env); @@ -599,16 +601,16 @@ private TimerReceiverFactory createTimerReceiverFactory() { } private static StateRequestHandler getStateRequestHandler( - KeyedStateBackend keyedStateBackend, - TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - Map jobOptions) { + KeyedStateBackend keyedStateBackend, + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + ReadableConfig config) { if (keyedStateBackend == null) { return StateRequestHandler.unsupported(); } else { assert keySerializer != null; return new SimpleStateRequestHandler( - keyedStateBackend, keySerializer, namespaceSerializer, jobOptions); + keyedStateBackend, keySerializer, namespaceSerializer, config); } } } diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java index 6f5e7524b721b..ba42dc4d385e9 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java @@ -29,6 +29,7 @@ import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.runtime.RowSerializer; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; import org.apache.flink.core.memory.DataInputViewStreamWrapper; @@ -135,12 +136,12 @@ public class SimpleStateRequestHandler implements StateRequestHandler { private final BeamFnApi.ProcessBundleRequest.CacheToken cacheToken; SimpleStateRequestHandler( - KeyedStateBackend keyedStateBackend, - TypeSerializer keySerializer, - TypeSerializer namespaceSerializer, - Map config) { + KeyedStateBackend keyedStateBackend, + TypeSerializer keySerializer, + TypeSerializer namespaceSerializer, + ReadableConfig config) { this.keyedStateBackend = keyedStateBackend; - TypeSerializer frameworkKeySerializer = keyedStateBackend.getKeySerializer(); + TypeSerializer frameworkKeySerializer = keyedStateBackend.getKeySerializer(); if (!(frameworkKeySerializer instanceof AbstractRowDataSerializer || frameworkKeySerializer instanceof RowSerializer)) { throw new RuntimeException("Currently SimpleStateRequestHandler only support row key!"); @@ -157,12 +158,7 @@ public class SimpleStateRequestHandler implements StateRequestHandler { stateDescriptorCache = new HashMap<>(); mapStateIteratorCache = new HashMap<>(); mapStateIterateResponseBatchSize = - Integer.valueOf( - config.getOrDefault( - PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE.key(), - PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE - .defaultValue() - .toString())); + config.get(PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE); if (mapStateIterateResponseBatchSize <= 0) { throw new RuntimeException( String.format( diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java index b925bbe4426da..c5f9f66a42d6d 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/AbstractStatelessFunctionOperator.java @@ -109,7 +109,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws IOException { createPythonEnvironmentManager(), getFunctionUrn(), getUserDefinedFunctionsProto(), - jobOptions, getFlinkMetricContainer(), getContainingTask().getEnvironment().getMemoryManager(), getOperatorConfig() diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java index 9ec99126be8c5..816ea04f3a227 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/AbstractPythonStreamAggregateOperator.java @@ -47,6 +47,8 @@ import java.util.Arrays; import java.util.stream.Collectors; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; import static org.apache.flink.table.runtime.typeutils.PythonTypeUtils.toProtoType; @@ -156,7 +158,6 @@ public void open() throws Exception { PythonTypeUtils.toInternalSerializer(userDefinedFunctionOutputType); rowDataWrapper = new StreamRecordRowDataWrappingCollector(output); super.open(); - configJobOptions(); } @Override @@ -175,7 +176,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { createPythonEnvironmentManager(), getFunctionUrn(), getUserDefinedFunctionsProto(), - jobOptions, getFlinkMetricContainer(), getKeyedStateBackend(), getKeySerializer(), @@ -235,8 +235,8 @@ TypeSerializer getWindowSerializer() { protected FlinkFnApi.UserDefinedAggregateFunctions getUserDefinedFunctionsProto() { FlinkFnApi.UserDefinedAggregateFunctions.Builder builder = FlinkFnApi.UserDefinedAggregateFunctions.newBuilder(); - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); builder.addAllGrouping(Arrays.stream(grouping).boxed().collect(Collectors.toList())); builder.setGenerateUpdateBefore(generateUpdateBefore); builder.setIndexOfCountStar(indexOfCountStar); @@ -263,15 +263,6 @@ protected FlinkFnApi.UserDefinedAggregateFunctions getUserDefinedFunctionsProto( public abstract RowType createUserDefinedFunctionOutputType(); - private void configJobOptions() { - jobOptions.put( - PythonOptions.STATE_CACHE_SIZE.key(), - String.valueOf(config.get(PythonOptions.STATE_CACHE_SIZE))); - jobOptions.put( - PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE.key(), - String.valueOf(config.get(PythonOptions.MAP_STATE_ITERATE_RESPONSE_BATCH_SIZE))); - } - public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor(RowType runnerInputType) { return createRowTypeCoderInfoDescriptorProto( runnerInputType, FlinkFnApi.CoderInfoDescriptor.Mode.MULTIPLE, false); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java index a893bb8617f8b..cd7167f1da9f1 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/AbstractArrowPythonAggregateFunctionOperator.java @@ -36,6 +36,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; import static org.apache.flink.streaming.api.utils.ProtoUtils.createArrowTypeCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.utils.ProtoUtils.getUserDefinedFunctionProto; @@ -153,8 +155,8 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { for (PythonFunctionInfo pythonFunctionInfo : pandasAggFunctions) { builder.addUdfs(getUserDefinedFunctionProto(pythonFunctionInfo)); } - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); return builder.build(); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java index 2fd586131e3fa..1425a35fe9548 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperator.java @@ -37,6 +37,8 @@ import java.util.List; import java.util.ListIterator; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; import static org.apache.flink.streaming.api.utils.ProtoUtils.createOverWindowArrowTypeCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.utils.ProtoUtils.getUserDefinedFunctionProto; @@ -259,8 +261,8 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { functionBuilder.setWindowIndex(aggWindowIndex[i]); builder.addUdfs(functionBuilder); } - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); // add windows for (int i = 0; i < lowerBoundary.length; i++) { FlinkFnApi.OverWindow.Builder windowBuilder = FlinkFnApi.OverWindow.newBuilder(); diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java index e5220fb005529..ed887657e2147 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/AbstractPythonScalarFunctionOperator.java @@ -35,6 +35,9 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; + /** * Base class for all stream operators to execute Python {@link ScalarFunction}s. It executes the * Python {@link ScalarFunction}s in separate Python execution environment. @@ -121,8 +124,8 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { for (PythonFunctionInfo pythonFunctionInfo : scalarFunctions) { builder.addUdfs(ProtoUtils.getUserDefinedFunctionProto(pythonFunctionInfo)); } - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); return builder.build(); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/EmbeddedPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/EmbeddedPythonScalarFunctionOperator.java index 3c843cd19b154..e30180bcb98e4 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/EmbeddedPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/EmbeddedPythonScalarFunctionOperator.java @@ -44,6 +44,9 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; + /** The Python {@link ScalarFunction} operator in embedded Python environment. */ @Internal public class EmbeddedPythonScalarFunctionOperator @@ -236,8 +239,8 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { for (PythonFunctionInfo pythonFunctionInfo : scalarFunctions) { builder.addUdfs(ProtoUtils.getUserDefinedFunctionProto(pythonFunctionInfo)); } - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); return builder.build(); } } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java index 1db9dda484a69..37e9010f22753 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperator.java @@ -30,6 +30,7 @@ import org.apache.flink.table.runtime.operators.python.scalar.AbstractPythonScalarFunctionOperator; import org.apache.flink.table.types.logical.RowType; +import static org.apache.flink.python.PythonOptions.MAX_ARROW_BATCH_SIZE; import static org.apache.flink.streaming.api.utils.ProtoUtils.createArrowTypeCoderInfoDescriptorProto; /** Arrow Python {@link ScalarFunction} operator. */ @@ -67,7 +68,7 @@ public ArrowPythonScalarFunctionOperator( @Override public void open() throws Exception { super.open(); - maxArrowBatchSize = Math.min(pythonConfig.getMaxArrowBatchSize(), maxBundleSize); + maxArrowBatchSize = Math.min(config.get(MAX_ARROW_BATCH_SIZE), maxBundleSize); arrowSerializer = new ArrowSerializer(udfInputType, udfOutputType); arrowSerializer.open(bais, baos); currentBatchCount = 0; diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java index 43778cfeb5b9b..81862b993007b 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperator.java @@ -41,6 +41,8 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; @@ -157,8 +159,8 @@ public FlinkFnApi.UserDefinedFunctions getUserDefinedFunctionsProto() { FlinkFnApi.UserDefinedFunctions.Builder builder = FlinkFnApi.UserDefinedFunctions.newBuilder(); builder.addUdfs(ProtoUtils.getUserDefinedFunctionProto(tableFunction)); - builder.setMetricEnabled(pythonConfig.isMetricEnabled()); - builder.setProfileEnabled(pythonConfig.isProfileEnabled()); + builder.setMetricEnabled(config.get(PYTHON_METRIC_ENABLED)); + builder.setProfileEnabled(config.get(PYTHON_PROFILE_ENABLED)); return builder.build(); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java index d502a47c01651..ff605356d3465 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/runners/python/beam/BeamTablePythonFunctionRunner.java @@ -34,7 +34,6 @@ import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.Optional; import static org.apache.flink.python.Constants.INPUT_COLLECTION_ID; @@ -57,7 +56,6 @@ public BeamTablePythonFunctionRunner( ProcessPythonEnvironmentManager environmentManager, String functionUrn, GeneratedMessageV3 userDefinedFunctionProto, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, KeyedStateBackend keyedStateBackend, TypeSerializer keySerializer, @@ -69,7 +67,6 @@ public BeamTablePythonFunctionRunner( super( taskName, environmentManager, - jobOptions, flinkMetricContainer, keyedStateBackend, keySerializer, @@ -122,7 +119,6 @@ public static BeamTablePythonFunctionRunner stateless( ProcessPythonEnvironmentManager environmentManager, String functionUrn, GeneratedMessageV3 userDefinedFunctionProto, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, MemoryManager memoryManager, double managedMemoryFraction, @@ -133,7 +129,6 @@ public static BeamTablePythonFunctionRunner stateless( environmentManager, functionUrn, userDefinedFunctionProto, - jobOptions, flinkMetricContainer, null, null, @@ -149,7 +144,6 @@ public static BeamTablePythonFunctionRunner stateful( ProcessPythonEnvironmentManager environmentManager, String functionUrn, GeneratedMessageV3 userDefinedFunctionProto, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, KeyedStateBackend keyedStateBackend, TypeSerializer keySerializer, @@ -163,7 +157,6 @@ public static BeamTablePythonFunctionRunner stateful( environmentManager, functionUrn, userDefinedFunctionProto, - jobOptions, flinkMetricContainer, keyedStateBackend, keySerializer, diff --git a/flink-python/src/test/java/org/apache/flink/python/PythonConfigTest.java b/flink-python/src/test/java/org/apache/flink/python/PythonConfigTest.java deleted file mode 100644 index 871c862282a63..0000000000000 --- a/flink-python/src/test/java/org/apache/flink/python/PythonConfigTest.java +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.python; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.python.util.PythonDependencyUtils; - -import org.junit.Test; - -import java.util.HashMap; -import java.util.Map; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThat; - -/** Tests for {@link PythonConfig}. */ -public class PythonConfigTest { - - @Test - public void testDefaultConfigure() { - PythonConfig pythonConfig = new PythonConfig(new Configuration()); - assertThat( - pythonConfig.getMaxBundleSize(), - is(equalTo(PythonOptions.MAX_BUNDLE_SIZE.defaultValue()))); - assertThat( - pythonConfig.getMaxBundleTimeMills(), - is(equalTo(PythonOptions.MAX_BUNDLE_TIME_MILLS.defaultValue()))); - assertThat( - pythonConfig.getMaxArrowBatchSize(), - is(equalTo(PythonOptions.MAX_ARROW_BATCH_SIZE.defaultValue()))); - assertThat(pythonConfig.getPythonFilesInfo().isEmpty(), is(true)); - assertThat(pythonConfig.getPythonRequirementsFileInfo().isPresent(), is(false)); - assertThat(pythonConfig.getPythonRequirementsCacheDirInfo().isPresent(), is(false)); - assertThat(pythonConfig.getPythonArchivesInfo().isEmpty(), is(true)); - assertThat(pythonConfig.getPythonExec(), is("python")); - assertThat( - pythonConfig.isUsingManagedMemory(), - is(equalTo(PythonOptions.USE_MANAGED_MEMORY.defaultValue()))); - } - - @Test - public void testMaxBundleSize() { - Configuration config = new Configuration(); - config.set(PythonOptions.MAX_BUNDLE_SIZE, 10); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getMaxBundleSize(), is(equalTo(10))); - } - - @Test - public void testMaxBundleTimeMills() { - Configuration config = new Configuration(); - config.set(PythonOptions.MAX_BUNDLE_TIME_MILLS, 10L); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getMaxBundleTimeMills(), is(equalTo(10L))); - } - - @Test - public void testMaxArrowBatchSize() { - Configuration config = new Configuration(); - config.set(PythonOptions.MAX_ARROW_BATCH_SIZE, 10); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getMaxArrowBatchSize(), is(equalTo(10))); - } - - @Test - public void testPythonFilesInfo() { - Configuration config = new Configuration(); - Map pythonFiles = new HashMap<>(); - pythonFiles.put("python_file_{SHA256}", "file0.py"); - config.set(PythonDependencyUtils.PYTHON_FILES, pythonFiles); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getPythonFilesInfo(), is(equalTo(pythonFiles))); - } - - @Test - public void testPythonRequirementsFileInfo() { - Configuration config = new Configuration(); - Map pythonRequirementsFile = - config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) - .orElse(new HashMap<>()); - pythonRequirementsFile.put(PythonDependencyUtils.FILE, "python_requirements_file_{SHA256}"); - config.set(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE, pythonRequirementsFile); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat( - pythonConfig.getPythonRequirementsFileInfo().get(), - is(equalTo("python_requirements_file_{SHA256}"))); - } - - @Test - public void testPythonRequirementsCacheDirInfo() { - Configuration config = new Configuration(); - Map pythonRequirementsFile = - config.getOptional(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) - .orElse(new HashMap<>()); - pythonRequirementsFile.put( - PythonDependencyUtils.CACHE, "python_requirements_cache_{SHA256}"); - config.set(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE, pythonRequirementsFile); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat( - pythonConfig.getPythonRequirementsCacheDirInfo().get(), - is(equalTo("python_requirements_cache_{SHA256}"))); - } - - @Test - public void testPythonArchivesInfo() { - Configuration config = new Configuration(); - Map pythonArchives = new HashMap<>(); - pythonArchives.put("python_archive_{SHA256}", "file0.zip"); - config.set(PythonDependencyUtils.PYTHON_ARCHIVES, pythonArchives); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getPythonArchivesInfo(), is(equalTo(pythonArchives))); - } - - @Test - public void testPythonExec() { - Configuration config = new Configuration(); - config.set(PythonOptions.PYTHON_EXECUTABLE, "/usr/local/bin/python3"); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.getPythonExec(), is(equalTo("/usr/local/bin/python3"))); - } - - @Test - public void testManagedMemory() { - Configuration config = new Configuration(); - config.set(PythonOptions.USE_MANAGED_MEMORY, true); - PythonConfig pythonConfig = new PythonConfig(config); - assertThat(pythonConfig.isUsingManagedMemory(), is(equalTo(true))); - } -} diff --git a/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java b/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java index b80f56f0e07de..5478130b1b01f 100644 --- a/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java +++ b/flink-python/src/test/java/org/apache/flink/python/env/PythonDependencyInfoTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; -import org.apache.flink.python.PythonConfig; import org.apache.flink.python.PythonOptions; import org.apache.flink.python.util.PythonDependencyUtils; import org.apache.flink.util.OperatingSystem; @@ -35,6 +34,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; +import static org.apache.flink.python.PythonOptions.PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO; +import static org.apache.flink.python.PythonOptions.PYTHON_FILES_DISTRIBUTED_CACHE_INFO; +import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -75,9 +77,8 @@ public void testParsePythonFiles() { Map pythonFiles = new HashMap<>(); pythonFiles.put("python_file_{SHA256_0}", "test_file1.py"); pythonFiles.put("python_file_{SHA256_1}", "test_file2.py"); - config.set(PythonDependencyUtils.PYTHON_FILES, pythonFiles); - PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create(new PythonConfig(config), distributedCache); + config.set(PYTHON_FILES_DISTRIBUTED_CACHE_INFO, pythonFiles); + PythonDependencyInfo dependencyInfo = PythonDependencyInfo.create(config, distributedCache); Map expected = new HashMap<>(); expected.put("/distributed_cache/file0", "test_file1.py"); @@ -91,18 +92,17 @@ public void testParsePythonRequirements() throws IOException { Assume.assumeFalse(OperatingSystem.isWindows()); Configuration config = new Configuration(); - config.set(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE, new HashMap<>()); - config.get(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) + config.set(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO, new HashMap<>()); + config.get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) .put(PythonDependencyUtils.FILE, "python_requirements_file_{SHA256}"); - PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create(new PythonConfig(config), distributedCache); + PythonDependencyInfo dependencyInfo = PythonDependencyInfo.create(config, distributedCache); assertEquals("/distributed_cache/file2", dependencyInfo.getRequirementsFilePath().get()); assertFalse(dependencyInfo.getRequirementsCacheDir().isPresent()); - config.get(PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE) + config.get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) .put(PythonDependencyUtils.CACHE, "python_requirements_cache_{SHA256}"); - dependencyInfo = PythonDependencyInfo.create(new PythonConfig(config), distributedCache); + dependencyInfo = PythonDependencyInfo.create(config, distributedCache); assertEquals("/distributed_cache/file2", dependencyInfo.getRequirementsFilePath().get()); assertEquals("/distributed_cache/file3", dependencyInfo.getRequirementsCacheDir().get()); @@ -117,9 +117,8 @@ public void testParsePythonArchives() { Map pythonArchives = new HashMap<>(); pythonArchives.put("python_archive_{SHA256_0}", "py27.zip"); pythonArchives.put("python_archive_{SHA256_1}", "py37"); - config.set(PythonDependencyUtils.PYTHON_ARCHIVES, pythonArchives); - PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create(new PythonConfig(config), distributedCache); + config.set(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO, pythonArchives); + PythonDependencyInfo dependencyInfo = PythonDependencyInfo.create(config, distributedCache); Map expected = new HashMap<>(); expected.put("/distributed_cache/file4", "py27.zip"); @@ -131,8 +130,7 @@ public void testParsePythonArchives() { public void testParsePythonExec() { Configuration config = new Configuration(); config.set(PythonOptions.PYTHON_EXECUTABLE, "/usr/bin/python3"); - PythonDependencyInfo dependencyInfo = - PythonDependencyInfo.create(new PythonConfig(config), distributedCache); + PythonDependencyInfo dependencyInfo = PythonDependencyInfo.create(config, distributedCache); assertEquals("/usr/bin/python3", dependencyInfo.getPythonExec()); } diff --git a/flink-python/src/test/java/org/apache/flink/python/util/PythonConfigUtilTest.java b/flink-python/src/test/java/org/apache/flink/python/util/PythonConfigUtilTest.java index c6e997c620788..2718634b479e4 100644 --- a/flink-python/src/test/java/org/apache/flink/python/util/PythonConfigUtilTest.java +++ b/flink-python/src/test/java/org/apache/flink/python/util/PythonConfigUtilTest.java @@ -25,25 +25,13 @@ import org.junit.Test; -import java.lang.reflect.InvocationTargetException; import java.util.Collections; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; /** A test class to test PythonConfigUtil getting executionEnvironment correctly. */ public class PythonConfigUtilTest { - @Test - public void testGetEnvironmentConfig() - throws IllegalAccessException, NoSuchFieldException, InvocationTargetException { - StreamExecutionEnvironment executionEnvironment = - StreamExecutionEnvironment.getExecutionEnvironment(); - Configuration envConfig = - PythonConfigUtil.getEnvConfigWithDependencies(executionEnvironment); - assertNotNull(envConfig); - } - @Test public void testJobName() { String jobName = "MyTestJob"; diff --git a/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java b/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java index 46f370be76802..01238c08a8626 100644 --- a/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java +++ b/flink-python/src/test/java/org/apache/flink/python/util/PythonDependencyUtilsTest.java @@ -32,14 +32,14 @@ import java.util.Properties; import java.util.stream.Collectors; +import static org.apache.flink.python.PythonOptions.PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO; import static org.apache.flink.python.PythonOptions.PYTHON_CLIENT_EXECUTABLE; import static org.apache.flink.python.PythonOptions.PYTHON_EXECUTABLE; +import static org.apache.flink.python.PythonOptions.PYTHON_FILES_DISTRIBUTED_CACHE_INFO; import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS; +import static org.apache.flink.python.PythonOptions.PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO; import static org.apache.flink.python.util.PythonDependencyUtils.CACHE; import static org.apache.flink.python.util.PythonDependencyUtils.FILE; -import static org.apache.flink.python.util.PythonDependencyUtils.PYTHON_ARCHIVES; -import static org.apache.flink.python.util.PythonDependencyUtils.PYTHON_FILES; -import static org.apache.flink.python.util.PythonDependencyUtils.PYTHON_REQUIREMENTS_FILE; import static org.apache.flink.python.util.PythonDependencyUtils.configurePythonDependencies; import static org.apache.flink.python.util.PythonDependencyUtils.merge; import static org.junit.Assert.assertEquals; @@ -74,20 +74,20 @@ public void testPythonFiles() { "tmp_dir/test_dir"); verifyCachedFiles(expectedCachedFiles); - Configuration expectedConfiguration = new Configuration(config); - expectedConfiguration.set(PYTHON_FILES, new HashMap<>()); + Configuration expectedConfiguration = new Configuration(); + expectedConfiguration.set(PYTHON_FILES_DISTRIBUTED_CACHE_INFO, new HashMap<>()); expectedConfiguration - .get(PYTHON_FILES) + .get(PYTHON_FILES_DISTRIBUTED_CACHE_INFO) .put( "python_file_83bbdaee494ad7d9b334c02ec71dc86a0868f7f8e49d1249a37c517dc6ee15a7", "test_file1.py"); expectedConfiguration - .get(PYTHON_FILES) + .get(PYTHON_FILES_DISTRIBUTED_CACHE_INFO) .put( "python_file_e57a895cb1256500098be0874128680cd9f56000d48fcd393c48d6371bd2d947", "test_file2.py"); expectedConfiguration - .get(PYTHON_FILES) + .get(PYTHON_FILES_DISTRIBUTED_CACHE_INFO) .put( "python_file_e56bc55ff643576457b3d012b2bba888727c71cf05a958930f2263398c4e9798", "test_dir"); @@ -106,10 +106,10 @@ public void testPythonRequirements() { "tmp_dir/requirements.txt"); verifyCachedFiles(expectedCachedFiles); - Configuration expectedConfiguration = new Configuration(config); - expectedConfiguration.set(PYTHON_REQUIREMENTS_FILE, new HashMap<>()); + Configuration expectedConfiguration = new Configuration(); + expectedConfiguration.set(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO, new HashMap<>()); expectedConfiguration - .get(PYTHON_REQUIREMENTS_FILE) + .get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) .put( FILE, "python_requirements_file_69390ca43c69ada3819226fcfbb5b6d27e111132a9427e7f201edd82e9d65ff6"); @@ -127,15 +127,15 @@ public void testPythonRequirements() { "tmp_dir/cache"); verifyCachedFiles(expectedCachedFiles); - expectedConfiguration = new Configuration(config); - expectedConfiguration.set(PYTHON_REQUIREMENTS_FILE, new HashMap<>()); + expectedConfiguration = new Configuration(); + expectedConfiguration.set(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO, new HashMap<>()); expectedConfiguration - .get(PYTHON_REQUIREMENTS_FILE) + .get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) .put( FILE, "python_requirements_file_56fd0c530faaa7129dca8d314cf69cbfc7c1c5c952f5176a003253e2f418873e"); expectedConfiguration - .get(PYTHON_REQUIREMENTS_FILE) + .get(PYTHON_REQUIREMENTS_FILE_DISTRIBUTED_CACHE_INFO) .put( CACHE, "python_requirements_cache_2f563dd6731c2c7c5e1ef1ef8279f61e907dc3bfc698adb71b109e43ed93e143"); @@ -169,25 +169,25 @@ public void testPythonArchives() { "tmp_dir/py37.zip"); verifyCachedFiles(expectedCachedFiles); - Configuration expectedConfiguration = new Configuration(config); - expectedConfiguration.set(PYTHON_ARCHIVES, new HashMap<>()); + Configuration expectedConfiguration = new Configuration(); + expectedConfiguration.set(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO, new HashMap<>()); expectedConfiguration - .get(PYTHON_ARCHIVES) + .get(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) .put( "python_archive_4cc74e4003de886434723f351771df2a84f72531c52085acc0915e19d70df2ba", "file1.zip"); expectedConfiguration - .get(PYTHON_ARCHIVES) + .get(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) .put( "python_archive_5f3fca2a4165c7d9c94b00bfab956c15f14c41e9e03f6037c83eb61157fce09c", "py37.zip"); expectedConfiguration - .get(PYTHON_ARCHIVES) + .get(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) .put( "python_archive_f8a1c874251230f21094880d9dd878ffb5714454b69184d8ad268a6563269f0c", "py37.zip#venv2"); expectedConfiguration - .get(PYTHON_ARCHIVES) + .get(PYTHON_ARCHIVES_DISTRIBUTED_CACHE_INFO) .put( "python_archive_c7d970ce1c5794367974ce8ef536c2343bed8fcfe7c2422c51548e58007eee6a", "py37.zip#venv"); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java index a5bf90a43d148..8f328680e81a8 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PassThroughPythonStreamGroupWindowAggregateOperator.java @@ -201,7 +201,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { userDefinedFunctionOutputType, STREAM_GROUP_WINDOW_AGGREGATE_URN, getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), getKeyedStateBackend(), getKeySerializer(), diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperatorTest.java index d5472fc27a505..2e6e9f377d84f 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupAggregateOperatorTest.java @@ -40,7 +40,6 @@ import org.junit.Test; import java.io.IOException; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.function.Function; @@ -262,7 +261,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { outputType, STREAM_GROUP_AGGREGATE_URN, getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), getKeyedStateBackend(), getKeySerializer(), diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperatorTest.java index bdef22d144c1e..d4cffdb493598 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/PythonStreamGroupTableAggregateOperatorTest.java @@ -40,7 +40,6 @@ import org.junit.Test; import java.io.IOException; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.function.Function; @@ -275,7 +274,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { outputType, STREAM_GROUP_TABLE_AGGREGATE_URN, getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), getKeyedStateBackend(), getKeySerializer(), diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperatorTest.java index e7581199b21cf..52981006ba122 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupAggregateFunctionOperatorTest.java @@ -42,7 +42,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -238,7 +237,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java index a6ec6db0c7802..0812090f1d7c2 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonGroupWindowAggregateFunctionOperatorTest.java @@ -44,7 +44,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -407,7 +406,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperatorTest.java index 1fee4769471da..276bae1f6a415 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/batch/BatchArrowPythonOverWindowAggregateFunctionOperatorTest.java @@ -43,7 +43,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.List; import java.util.concurrent.ConcurrentLinkedQueue; @@ -308,7 +307,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), true); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperatorTest.java index e7622221adc0b..be4bdabdb4830 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonGroupWindowAggregateFunctionOperatorTest.java @@ -55,7 +55,6 @@ import java.time.Duration; import java.time.ZoneId; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -522,7 +521,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRangeOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRangeOperatorTest.java index a95157c1277c6..5d85019a7a2e2 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRangeOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRangeOperatorTest.java @@ -41,7 +41,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** Test for {@link StreamArrowPythonProcTimeBoundedRangeOperator}. */ @@ -168,7 +167,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRowsOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRowsOperatorTest.java index b65a0fb226f99..117c40d1b4e8f 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRowsOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonProcTimeBoundedRowsOperatorTest.java @@ -41,7 +41,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** Test for {@link StreamArrowPythonProcTimeBoundedRowsOperator}. */ @@ -169,7 +168,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRangeOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRangeOperatorTest.java index 849c6b0a8a15b..620de10884432 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRangeOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRangeOperatorTest.java @@ -45,7 +45,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; import static org.junit.Assert.assertEquals; @@ -324,7 +323,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRowsOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRowsOperatorTest.java index 1ee0e7f1d81cc..dafdf3b54349b 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRowsOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/aggregate/arrow/stream/StreamArrowPythonRowTimeBoundedRowsOperatorTest.java @@ -43,7 +43,6 @@ import org.junit.Test; import java.util.Arrays; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedQueue; /** @@ -294,7 +293,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer(), false); } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java index 17ae33cbbd518..25f53d94166b1 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTest.java @@ -41,7 +41,6 @@ import java.io.IOException; import java.util.Collection; -import java.util.HashMap; import static org.apache.flink.table.runtime.util.StreamRecordUtils.row; @@ -152,7 +151,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws IOException { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java index 953afb0e3be33..e42a9c12dab6f 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/arrow/ArrowPythonScalarFunctionOperatorTest.java @@ -42,7 +42,6 @@ import java.io.IOException; import java.util.Collection; -import java.util.HashMap; import static org.apache.flink.table.runtime.util.StreamRecordUtils.row; @@ -151,7 +150,6 @@ public PythonFunctionRunner createPythonFunctionRunner() throws IOException { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java index 840503c2737cc..8b2b0b0d1620e 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/table/PythonTableFunctionOperatorTest.java @@ -40,7 +40,6 @@ import org.apache.calcite.rel.core.JoinRelType; import java.util.Collection; -import java.util.HashMap; import static org.apache.flink.table.runtime.util.StreamRecordUtils.row; @@ -133,7 +132,6 @@ public PythonFunctionRunner createPythonFunctionRunner() { udfOutputType, getFunctionUrn(), getUserDefinedFunctionsProto(), - new HashMap<>(), PythonTestUtils.createMockFlinkMetricContainer()); } } diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java index fb16068280957..f19b5c60188f5 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonAggregateFunctionRunner.java @@ -19,11 +19,11 @@ package org.apache.flink.table.runtime.utils; import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.fnexecution.v1.FlinkFnApi; -import org.apache.flink.python.PythonConfig; import org.apache.flink.python.env.process.ProcessPythonEnvironmentManager; import org.apache.flink.python.metric.FlinkMetricContainer; import org.apache.flink.table.data.RowData; @@ -37,7 +37,6 @@ import java.util.ArrayList; import java.util.LinkedList; import java.util.List; -import java.util.Map; import static org.apache.flink.streaming.api.utils.ProtoUtils.createArrowTypeCoderInfoDescriptorProto; @@ -72,7 +71,6 @@ public PassThroughPythonAggregateFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, boolean isBatchOverWindow) { super( @@ -80,7 +78,6 @@ public PassThroughPythonAggregateFunctionRunner( environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, null, null, @@ -97,7 +94,7 @@ public PassThroughPythonAggregateFunctionRunner( } @Override - public void open(PythonConfig config) throws Exception { + public void open(ReadableConfig config) throws Exception { super.open(config); bais = new ByteArrayInputStreamWithPos(); baisWrapper = new DataInputViewStreamWrapper(bais); diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java index 4815507f6aa65..e0cd0fbde1c07 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonScalarFunctionRunner.java @@ -29,7 +29,6 @@ import java.util.LinkedList; import java.util.List; -import java.util.Map; import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; @@ -48,14 +47,12 @@ public PassThroughPythonScalarFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer) { super( taskName, environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, null, null, diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java index 94694222b498b..48af1207b6e35 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughPythonTableFunctionRunner.java @@ -29,7 +29,6 @@ import java.util.LinkedList; import java.util.List; -import java.util.Map; import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; @@ -50,14 +49,12 @@ public PassThroughPythonTableFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer) { super( taskName, environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, null, null, diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java index e2886005c5400..230f2e4b86261 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamAggregatePythonFunctionRunner.java @@ -31,7 +31,6 @@ import java.util.LinkedList; import java.util.List; -import java.util.Map; import java.util.function.Function; import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; @@ -54,7 +53,6 @@ public PassThroughStreamAggregatePythonFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedAggregateFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, KeyedStateBackend keyedStateBackend, TypeSerializer keySerializer, @@ -64,7 +62,6 @@ public PassThroughStreamAggregatePythonFunctionRunner( environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, keyedStateBackend, keySerializer, diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java index 7f977f2dbcf17..a76fc1c7fd845 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamGroupWindowAggregatePythonFunctionRunner.java @@ -30,8 +30,6 @@ import org.apache.beam.runners.fnexecution.control.JobBundleFactory; import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct; -import java.util.Map; - import static org.apache.flink.streaming.api.utils.ProtoUtils.createFlattenRowTypeCoderInfoDescriptorProto; /** @@ -50,7 +48,6 @@ public PassThroughStreamGroupWindowAggregatePythonFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedAggregateFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, KeyedStateBackend keyedStateBackend, TypeSerializer keySerializer, @@ -60,7 +57,6 @@ public PassThroughStreamGroupWindowAggregatePythonFunctionRunner( environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, keyedStateBackend, keySerializer, diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java index f9ed8acefab86..d519a1a220f1d 100644 --- a/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java +++ b/flink-python/src/test/java/org/apache/flink/table/runtime/utils/PassThroughStreamTableAggregatePythonFunctionRunner.java @@ -32,7 +32,6 @@ import java.util.Arrays; import java.util.LinkedList; import java.util.List; -import java.util.Map; import java.util.function.Function; import static org.apache.flink.streaming.api.utils.ProtoUtils.createRowTypeCoderInfoDescriptorProto; @@ -56,7 +55,6 @@ public PassThroughStreamTableAggregatePythonFunctionRunner( RowType outputType, String functionUrn, FlinkFnApi.UserDefinedAggregateFunctions userDefinedFunctions, - Map jobOptions, FlinkMetricContainer flinkMetricContainer, KeyedStateBackend keyedStateBackend, TypeSerializer keySerializer, @@ -66,7 +64,6 @@ public PassThroughStreamTableAggregatePythonFunctionRunner( environmentManager, functionUrn, userDefinedFunctions, - jobOptions, flinkMetricContainer, keyedStateBackend, keySerializer, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java index 22b0ef24cd428..b33396fbec0d9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java @@ -21,11 +21,9 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil; import java.util.Optional; @@ -38,10 +36,7 @@ @Internal public final class ExecNodeConfig implements ReadableConfig { - // See https://issues.apache.org/jira/browse/FLINK-26190 - // Used only by CommonPythonUtil#getMergedConfig(StreamExecutionEnvironment, TableConfig)} - // otherwise it can be changed to ReadableConfig. - private final TableConfig tableConfig; + private final ReadableConfig tableConfig; private final ReadableConfig nodeConfig; @@ -50,20 +45,6 @@ public final class ExecNodeConfig implements ReadableConfig { this.tableConfig = tableConfig; } - /** - * Return the {@link PlannerBase#getTableConfig()}. - * - * @return the {@link PlannerBase#getTableConfig()}. - * @deprecated This method is used only for {@link - * CommonPythonUtil#getMergedConfig(StreamExecutionEnvironment, TableConfig)}. It should be - * removed when this method is refactored to accept a {@link ReadableConfig} instead. - */ - // See https://issues.apache.org/jira/browse/FLINK-26190 - @Deprecated - public TableConfig getTableConfig() { - return tableConfig; - } - @Override public T get(ConfigOption option) { return nodeConfig.getOptional(option).orElseGet(() -> tableConfig.get(option)); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java index 7e637014f9600..dc13cac46db64 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java @@ -94,7 +94,7 @@ protected Transformation translateToPlanInternal( final RowType inputRowType = (RowType) inputEdge.getOutputType(); final RowType outputRowType = InternalTypeInfo.of(getOutputType()).toRowType(); Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputTransformation transform = createPythonOneInputTransformation( inputTransform, inputRowType, outputRowType, pythonConfig, config); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java index 8dc3f6ca36c7a..2590673126ac6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java @@ -114,7 +114,7 @@ protected Transformation translateToPlanInternal( final Tuple2 windowSizeAndSlideSize = WindowCodeGenerator.getWindowDef(window); final Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); int groupBufferLimitSize = pythonConfig.getInteger( ExecutionConfigOptions.TABLE_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java index ae887103b8341..4b347bc785362 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java @@ -153,7 +153,7 @@ protected Transformation translateToPlanInternal( } } Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputTransformation transform = createPythonOneInputTransformation( inputTransform, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java index c31e3cc0f0101..16cd36cade475 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java @@ -103,7 +103,7 @@ protected Transformation translateToPlanInternal( final Transformation inputTransform = (Transformation) inputEdge.translateToPlan(planner); final Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputTransformation ret = createPythonOneInputTransformation(inputTransform, config, pythonConfig); if (CommonPythonUtil.isPythonWorkerUsingManagedMemory(pythonConfig)) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java index e60f04aff80ba..a9aae7ad34315 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java @@ -90,7 +90,7 @@ protected Transformation translateToPlanInternal( final Transformation inputTransform = (Transformation) inputEdge.translateToPlan(planner); final Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputTransformation transform = createPythonOneInputTransformation(inputTransform, config, pythonConfig); if (CommonPythonUtil.isPythonWorkerUsingManagedMemory(pythonConfig)) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java index d55ac1b7a23ca..0d1cb1da282a5 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java @@ -157,7 +157,7 @@ protected Transformation translateToPlanInternal( PythonAggregateFunctionInfo[] pythonFunctionInfos = aggInfosAndDataViewSpecs.f0; DataViewSpec[][] dataViewSpecs = aggInfosAndDataViewSpecs.f1; Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); final OneInputStreamOperator operator = getPythonAggregateFunctionOperator( pythonConfig, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java index 2303f03345c6a..0b4cd55b824e7 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java @@ -130,7 +130,7 @@ protected Transformation translateToPlanInternal( PythonAggregateFunctionInfo[] pythonFunctionInfos = aggInfosAndDataViewSpecs.f0; DataViewSpec[][] dataViewSpecs = aggInfosAndDataViewSpecs.f1; Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputStreamOperator pythonOperator = getPythonTableAggregateFunctionOperator( pythonConfig, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java index 90905af0e82df..ee0605e3af8ec 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java @@ -233,7 +233,7 @@ protected Transformation translateToPlanInternal( WindowAssigner windowAssigner = windowAssignerAndTrigger.f0; Trigger trigger = windowAssignerAndTrigger.f1; Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); boolean isGeneralPythonUDAF = Arrays.stream(aggCalls) .anyMatch(x -> PythonUtil.isPythonAggregate(x, PythonFunctionKind.GENERAL)); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java index f24ba0a414f29..671fda6d848e4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java @@ -179,7 +179,7 @@ protected Transformation translateToPlanInternal( } long precedingOffset = -1 * (long) boundValue; Configuration pythonConfig = - CommonPythonUtil.getMergedConfig(planner.getExecEnv(), config.getTableConfig()); + CommonPythonUtil.extractPythonConfiguration(planner.getExecEnv(), config); OneInputTransformation transform = createPythonOneInputTransformation( inputTransform, diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java index 538bfdb37ef6e..d9ac38b3725bf 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java @@ -21,8 +21,8 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.dataview.DataView; import org.apache.flink.table.api.dataview.ListView; @@ -98,7 +98,7 @@ public class CommonPythonUtil { private CommonPythonUtil() {} - public static Class loadClass(String className) { + public static Class loadClass(String className) { try { return Class.forName(className, false, Thread.currentThread().getContextClassLoader()); } catch (ClassNotFoundException e) { @@ -107,21 +107,22 @@ public static Class loadClass(String className) { } } - @SuppressWarnings("unchecked") - public static Configuration getMergedConfig( - StreamExecutionEnvironment env, TableConfig tableConfig) { - Class clazz = loadClass(PYTHON_CONFIG_UTILS_CLASS); + public static Configuration extractPythonConfiguration( + StreamExecutionEnvironment env, ReadableConfig tableConfig) { + Class clazz = loadClass(PYTHON_CONFIG_UTILS_CLASS); try { StreamExecutionEnvironment realEnv = getRealEnvironment(env); Method method = clazz.getDeclaredMethod( - "getMergedConfig", StreamExecutionEnvironment.class, TableConfig.class); + "extractPythonConfiguration", + StreamExecutionEnvironment.class, + ReadableConfig.class); return (Configuration) method.invoke(null, realEnv, tableConfig); } catch (NoSuchFieldException | IllegalAccessException | NoSuchMethodException | InvocationTargetException e) { - throw new TableException("Method getMergedConfig accessed failed.", e); + throw new TableException("Method extractPythonConfiguration accessed failed.", e); } } @@ -149,7 +150,7 @@ public static PythonFunctionInfo createPythonFunctionInfo( @SuppressWarnings("unchecked") public static boolean isPythonWorkerUsingManagedMemory(Configuration config) { - Class clazz = loadClass(PYTHON_OPTIONS_CLASS); + Class clazz = loadClass(PYTHON_OPTIONS_CLASS); try { return config.getBoolean( (ConfigOption) (clazz.getField("USE_MANAGED_MEMORY").get(null))); @@ -160,7 +161,7 @@ public static boolean isPythonWorkerUsingManagedMemory(Configuration config) { @SuppressWarnings("unchecked") public static boolean isPythonWorkerInProcessMode(Configuration config) { - Class clazz = loadClass(PYTHON_OPTIONS_CLASS); + Class clazz = loadClass(PYTHON_OPTIONS_CLASS); try { return config.getString( (ConfigOption) @@ -401,12 +402,12 @@ private static byte[] convertLiteralToPython(RexLiteral o, SqlTypeName typeName) return (byte[]) pickleValue.invoke(null, value, type); } - @SuppressWarnings("unchecked") private static void loadPickleValue() { if (pickleValue == null) { synchronized (CommonPythonUtil.class) { if (pickleValue == null) { - Class clazz = loadClass("org.apache.flink.api.common.python.PythonBridgeUtils"); + Class clazz = + loadClass("org.apache.flink.api.common.python.PythonBridgeUtils"); try { pickleValue = clazz.getMethod("pickleValue", Object.class, byte.class); } catch (NoSuchMethodException e) { From a2eb826e1fc4ae6018cc3e3ccfbcc8a642b68f96 Mon Sep 17 00:00:00 2001 From: wangfeifan Date: Thu, 7 Apr 2022 12:44:07 +0800 Subject: [PATCH 158/258] [FLINK-27105][docs] fix ChangelogStorage.uploadQueueSize metric type --- docs/content.zh/docs/ops/metrics.md | 2 +- docs/content/docs/ops/metrics.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/ops/metrics.md b/docs/content.zh/docs/ops/metrics.md index 30dc10f5f83e3..072b2fd0084e7 100644 --- a/docs/content.zh/docs/ops/metrics.md +++ b/docs/content.zh/docs/ops/metrics.md @@ -1310,7 +1310,7 @@ Note that the metrics are only available via reporters. uploadQueueSize Current size of upload queue. Queue items can be packed together and form a single upload. - Meter + Gauge diff --git a/docs/content/docs/ops/metrics.md b/docs/content/docs/ops/metrics.md index a06f62a0a710b..3a574e3999588 100644 --- a/docs/content/docs/ops/metrics.md +++ b/docs/content/docs/ops/metrics.md @@ -1303,7 +1303,7 @@ Note that the metrics are only available via reporters. uploadQueueSize Current size of upload queue. Queue items can be packed together and form a single upload. - Meter + Gauge From e379f3e1969037dd9aec6abf771fe22c5627b168 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 7 Apr 2022 19:55:03 +0800 Subject: [PATCH 159/258] [hotfix][python] Remove nosiy warnings --- .../apache/flink/client/python/PythonDriverOptions.java | 6 +++--- .../org/apache/flink/client/python/PythonEnvUtils.java | 6 +++--- .../api/runners/python/beam/SimpleStateRequestHandler.java | 4 ++-- .../flink/table/runtime/typeutils/PythonTypeUtils.java | 7 ------- 4 files changed, 8 insertions(+), 15 deletions(-) diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptions.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptions.java index bec0889d05b83..792af77b4212d 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptions.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonDriverOptions.java @@ -29,11 +29,11 @@ /** Options for the {@link PythonDriver}. */ final class PythonDriverOptions { - @Nullable private String entryPointModule; + @Nullable private final String entryPointModule; - @Nullable private String entryPointScript; + @Nullable private final String entryPointScript; - @Nonnull private List programArgs; + @Nonnull private final List programArgs; @Nullable String getEntryPointModule() { diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonEnvUtils.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonEnvUtils.java index ca7d9907c8b06..261aa82dc7656 100644 --- a/flink-python/src/main/java/org/apache/flink/client/python/PythonEnvUtils.java +++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonEnvUtils.java @@ -484,9 +484,9 @@ public static void shutdownPythonProcess(Process pythonProcess, long timeoutMill /** The shutdown hook used to destroy the Python process. */ public static class PythonProcessShutdownHook extends Thread { - private Process process; - private GatewayServer gatewayServer; - private String tmpDir; + private final Process process; + private final GatewayServer gatewayServer; + private final String tmpDir; public PythonProcessShutdownHook( Process process, GatewayServer gatewayServer, String tmpDir) { diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java index ba42dc4d385e9..5eef9b7a427cb 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/runners/python/beam/SimpleStateRequestHandler.java @@ -105,8 +105,8 @@ public class SimpleStateRequestHandler implements StateRequestHandler { BeamFnApi.StateGetResponse.newBuilder() .setData(ByteString.copyFrom(new byte[] {NOT_EMPTY_FLAG})); - private final TypeSerializer keySerializer; - private final TypeSerializer namespaceSerializer; + private final TypeSerializer keySerializer; + private final TypeSerializer namespaceSerializer; private final TypeSerializer valueSerializer; private final KeyedStateBackend keyedStateBackend; diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java index 8d7a83246fc1e..29abc5e561e18 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/typeutils/PythonTypeUtils.java @@ -64,7 +64,6 @@ import java.math.BigDecimal; import java.math.RoundingMode; import java.sql.Time; -import java.util.TimeZone; /** * Utilities for converting Flink logical types, such as convert it to the related TypeSerializer or @@ -75,12 +74,6 @@ public final class PythonTypeUtils { private static final String EMPTY_STRING = ""; - /** The local time zone. */ - private static final TimeZone LOCAL_TZ = TimeZone.getDefault(); - - /** The number of milliseconds in a day. */ - private static final long MILLIS_PER_DAY = 86400000L; // = 24 * 60 * 60 * 1000 - public static FlinkFnApi.Schema.FieldType toProtoType(LogicalType logicalType) { return logicalType.accept(new PythonTypeUtils.LogicalTypeToProtoTypeConverter()); } From 63521977fca5e05128ad3720724f6be550b3098d Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Thu, 7 Apr 2022 20:56:21 +0800 Subject: [PATCH 160/258] [FLINK-27126][python] Respect the state cache size configuration for Python DataStream operators This closes #19394. --- .../fn_execution/beam/beam_operations.py | 6 +- .../fn_execution/flink_fn_execution_pb2.py | 109 +++++++++++------- .../pyflink/proto/flink-fn-execution.proto | 7 ++ .../python/PythonCoProcessOperator.java | 12 +- .../python/PythonKeyedCoProcessOperator.java | 12 +- .../python/PythonKeyedProcessOperator.java | 12 +- .../python/PythonProcessOperator.java | 12 +- .../flink/streaming/api/utils/ProtoUtils.java | 48 ++++++-- 8 files changed, 160 insertions(+), 58 deletions(-) diff --git a/flink-python/pyflink/fn_execution/beam/beam_operations.py b/flink-python/pyflink/fn_execution/beam/beam_operations.py index 17074b23c5b79..3777c26563880 100644 --- a/flink-python/pyflink/fn_execution/beam/beam_operations.py +++ b/flink-python/pyflink/fn_execution/beam/beam_operations.py @@ -185,9 +185,9 @@ def _create_user_defined_function_operation(factory, transform_proto, consumers, factory.state_handler, key_row_coder, None, - 1000, - 1000, - 1000) + serialized_fn.state_cache_size, + serialized_fn.map_state_read_cache_size, + serialized_fn.map_state_write_cache_size) return beam_operation_cls( transform_proto.unique_name, spec, diff --git a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py index c087b641bedcd..90b0bc5af53e8 100644 --- a/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py +++ b/flink-python/pyflink/fn_execution/flink_fn_execution_pb2.py @@ -36,7 +36,7 @@ name='flink-fn-execution.proto', package='org.apache.flink.fn_execution.v1', syntax='proto3', - serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"\xa8\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12\x14\n\x0cwindow_index\x18\x03 \x01(\x05\x12\x1a\n\x12takes_row_as_input\x18\x04 \x01(\x08\x12\x15\n\ris_pandas_udf\x18\x05 \x01(\x08\"\xcb\x01\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12=\n\x07windows\x18\x03 \x03(\x0b\x32,.org.apache.flink.fn_execution.v1.OverWindow\x12\x17\n\x0fprofile_enabled\x18\x04 \x01(\x08\"\xdd\x02\n\nOverWindow\x12L\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x37.org.apache.flink.fn_execution.v1.OverWindow.WindowType\x12\x16\n\x0elower_boundary\x18\x02 \x01(\x03\x12\x16\n\x0eupper_boundary\x18\x03 \x01(\x03\"\xd0\x01\n\nWindowType\x12\x13\n\x0fRANGE_UNBOUNDED\x10\x00\x12\x1d\n\x19RANGE_UNBOUNDED_PRECEDING\x10\x01\x12\x1d\n\x19RANGE_UNBOUNDED_FOLLOWING\x10\x02\x12\x11\n\rRANGE_SLIDING\x10\x03\x12\x11\n\rROW_UNBOUNDED\x10\x04\x12\x1b\n\x17ROW_UNBOUNDED_PRECEDING\x10\x05\x12\x1b\n\x17ROW_UNBOUNDED_FOLLOWING\x10\x06\x12\x0f\n\x0bROW_SLIDING\x10\x07\"\x8b\x06\n\x1cUserDefinedAggregateFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12Z\n\x05specs\x18\x03 \x03(\x0b\x32K.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec\x12\x12\n\nfilter_arg\x18\x04 \x01(\x05\x12\x10\n\x08\x64istinct\x18\x05 \x01(\x08\x12\x1a\n\x12takes_row_as_input\x18\x06 \x01(\x08\x1a\x82\x04\n\x0c\x44\x61taViewSpec\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x66ield_index\x18\x02 \x01(\x05\x12i\n\tlist_view\x18\x03 \x01(\x0b\x32T.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.ListViewH\x00\x12g\n\x08map_view\x18\x04 \x01(\x0b\x32S.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapViewH\x00\x1aT\n\x08ListView\x12H\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x97\x01\n\x07MapView\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeB\x0b\n\tdata_view\"\xac\x04\n\x0bGroupWindow\x12M\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x38.org.apache.flink.fn_execution.v1.GroupWindow.WindowType\x12\x16\n\x0eis_time_window\x18\x02 \x01(\x08\x12\x14\n\x0cwindow_slide\x18\x03 \x01(\x03\x12\x13\n\x0bwindow_size\x18\x04 \x01(\x03\x12\x12\n\nwindow_gap\x18\x05 \x01(\x03\x12\x13\n\x0bis_row_time\x18\x06 \x01(\x08\x12\x18\n\x10time_field_index\x18\x07 \x01(\x05\x12\x17\n\x0f\x61llowedLateness\x18\x08 \x01(\x03\x12U\n\x0fnamedProperties\x18\t \x03(\x0e\x32<.org.apache.flink.fn_execution.v1.GroupWindow.WindowProperty\x12\x16\n\x0eshift_timezone\x18\n \x01(\t\"[\n\nWindowType\x12\x19\n\x15TUMBLING_GROUP_WINDOW\x10\x00\x12\x18\n\x14SLIDING_GROUP_WINDOW\x10\x01\x12\x18\n\x14SESSION_GROUP_WINDOW\x10\x02\"c\n\x0eWindowProperty\x12\x10\n\x0cWINDOW_START\x10\x00\x12\x0e\n\nWINDOW_END\x10\x01\x12\x16\n\x12ROW_TIME_ATTRIBUTE\x10\x02\x12\x17\n\x13PROC_TIME_ATTRIBUTE\x10\x03\"\x96\x04\n\x1dUserDefinedAggregateFunctions\x12L\n\x04udfs\x18\x01 \x03(\x0b\x32>.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12\x10\n\x08grouping\x18\x03 \x03(\x05\x12\x1e\n\x16generate_update_before\x18\x04 \x01(\x08\x12\x44\n\x08key_type\x18\x05 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x1b\n\x13index_of_count_star\x18\x06 \x01(\x05\x12\x1e\n\x16state_cleaning_enabled\x18\x07 \x01(\x08\x12\x18\n\x10state_cache_size\x18\x08 \x01(\x05\x12!\n\x19map_state_read_cache_size\x18\t \x01(\x05\x12\"\n\x1amap_state_write_cache_size\x18\n \x01(\x05\x12\x1b\n\x13\x63ount_star_inserted\x18\x0b \x01(\x08\x12\x43\n\x0cgroup_window\x18\x0c \x01(\x0b\x32-.org.apache.flink.fn_execution.v1.GroupWindow\x12\x17\n\x0fprofile_enabled\x18\r \x01(\x08\"\xec\x0f\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapInfo\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x1d\n\x08TimeInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\"\n\rTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a,\n\x17LocalZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\'\n\x12ZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a/\n\x0b\x44\x65\x63imalInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x12\r\n\x05scale\x18\x02 \x01(\x05\x1a\x1c\n\nBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1f\n\rVarBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1a\n\x08\x43harInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1d\n\x0bVarCharInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\xb0\x08\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_info\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapInfoH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12L\n\x0c\x64\x65\x63imal_info\x18\x06 \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.DecimalInfoH\x00\x12\x46\n\ttime_info\x18\x07 \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.TimeInfoH\x00\x12P\n\x0etimestamp_info\x18\x08 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.TimestampInfoH\x00\x12\x66\n\x1alocal_zoned_timestamp_info\x18\t \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfoH\x00\x12[\n\x14zoned_timestamp_info\x18\n \x01(\x0b\x32;.org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfoH\x00\x12J\n\x0b\x62inary_info\x18\x0b \x01(\x0b\x32\x33.org.apache.flink.fn_execution.v1.Schema.BinaryInfoH\x00\x12Q\n\x0fvar_binary_info\x18\x0c \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.VarBinaryInfoH\x00\x12\x46\n\tchar_info\x18\r \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.CharInfoH\x00\x12M\n\rvar_char_info\x18\x0e \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.VarCharInfoH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\xa1\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\r\n\tTIMESTAMP\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x12\x19\n\x15LOCAL_ZONED_TIMESTAMP\x10\x13\x12\x13\n\x0fZONED_TIMESTAMP\x10\x14\"\xf7\x08\n\x08TypeInfo\x12\x46\n\ttype_name\x18\x01 \x01(\x0e\x32\x33.org.apache.flink.fn_execution.v1.TypeInfo.TypeName\x12M\n\x17\x63ollection_element_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfoH\x00\x12O\n\rrow_type_info\x18\x03 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfoH\x00\x12S\n\x0ftuple_type_info\x18\x04 \x01(\x0b\x32\x38.org.apache.flink.fn_execution.v1.TypeInfo.TupleTypeInfoH\x00\x12O\n\rmap_type_info\x18\x05 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfoH\x00\x1a\x8b\x01\n\x0bMapTypeInfo\x12<\n\x08key_type\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x12>\n\nvalue_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1a\xb8\x01\n\x0bRowTypeInfo\x12L\n\x06\x66ields\x18\x01 \x03(\x0b\x32<.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field\x1a[\n\x05\x46ield\x12\x12\n\nfield_name\x18\x01 \x01(\t\x12>\n\nfield_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1aP\n\rTupleTypeInfo\x12?\n\x0b\x66ield_types\x18\x01 \x03(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\"\xb4\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\n\n\x06STRING\x10\x01\x12\x08\n\x04\x42YTE\x10\x02\x12\x0b\n\x07\x42OOLEAN\x10\x03\x12\t\n\x05SHORT\x10\x04\x12\x07\n\x03INT\x10\x05\x12\x08\n\x04LONG\x10\x06\x12\t\n\x05\x46LOAT\x10\x07\x12\n\n\x06\x44OUBLE\x10\x08\x12\x08\n\x04\x43HAR\x10\t\x12\x0b\n\x07\x42IG_INT\x10\n\x12\x0b\n\x07\x42IG_DEC\x10\x0b\x12\x0c\n\x08SQL_DATE\x10\x0c\x12\x0c\n\x08SQL_TIME\x10\r\x12\x11\n\rSQL_TIMESTAMP\x10\x0e\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x0f\x12\x13\n\x0fPRIMITIVE_ARRAY\x10\x10\x12\t\n\x05TUPLE\x10\x11\x12\x08\n\x04LIST\x10\x12\x12\x07\n\x03MAP\x10\x13\x12\x11\n\rPICKLED_BYTES\x10\x14\x12\x10\n\x0cOBJECT_ARRAY\x10\x15\x12\x0b\n\x07INSTANT\x10\x16\x42\x0b\n\ttype_info\"\xe6\x06\n\x1dUserDefinedDataStreamFunction\x12\x63\n\rfunction_type\x18\x01 \x01(\x0e\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.FunctionType\x12g\n\x0fruntime_context\x18\x02 \x01(\x0b\x32N.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext\x12\x0f\n\x07payload\x18\x03 \x01(\x0c\x12\x16\n\x0emetric_enabled\x18\x04 \x01(\x08\x12\x41\n\rkey_type_info\x18\x05 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x12\x17\n\x0fprofile_enabled\x18\x06 \x01(\x08\x1a*\n\x0cJobParameter\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t\x1a\xd0\x02\n\x0eRuntimeContext\x12\x11\n\ttask_name\x18\x01 \x01(\t\x12\x1f\n\x17task_name_with_subtasks\x18\x02 \x01(\t\x12#\n\x1bnumber_of_parallel_subtasks\x18\x03 \x01(\x05\x12\'\n\x1fmax_number_of_parallel_subtasks\x18\x04 \x01(\x05\x12\x1d\n\x15index_of_this_subtask\x18\x05 \x01(\x05\x12\x16\n\x0e\x61ttempt_number\x18\x06 \x01(\x05\x12\x64\n\x0ejob_parameters\x18\x07 \x03(\x0b\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter\x12\x1f\n\x17in_batch_execution_mode\x18\x08 \x01(\x08\"s\n\x0c\x46unctionType\x12\x0b\n\x07PROCESS\x10\x00\x12\x0e\n\nCO_PROCESS\x10\x01\x12\x11\n\rKEYED_PROCESS\x10\x02\x12\x14\n\x10KEYED_CO_PROCESS\x10\x03\x12\n\n\x06WINDOW\x10\x04\x12\x11\n\rREVISE_OUTPUT\x10\x64\"\xe4\x0e\n\x0fStateDescriptor\x12\x12\n\nstate_name\x18\x01 \x01(\t\x12Z\n\x10state_ttl_config\x18\x02 \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig\x1a\xe0\r\n\x0eStateTTLConfig\x12`\n\x0bupdate_type\x18\x01 \x01(\x0e\x32K.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.UpdateType\x12j\n\x10state_visibility\x18\x02 \x01(\x0e\x32P.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.StateVisibility\x12w\n\x17ttl_time_characteristic\x18\x03 \x01(\x0e\x32V.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.TtlTimeCharacteristic\x12\x0b\n\x03ttl\x18\x04 \x01(\x03\x12n\n\x12\x63leanup_strategies\x18\x05 \x01(\x0b\x32R.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies\x1a\xca\x08\n\x11\x43leanupStrategies\x12 \n\x18is_cleanup_in_background\x18\x01 \x01(\x08\x12y\n\nstrategies\x18\x02 \x03(\x0b\x32\x65.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.MapStrategiesEntry\x1aX\n\x1aIncrementalCleanupStrategy\x12\x14\n\x0c\x63leanup_size\x18\x01 \x01(\x05\x12$\n\x1crun_cleanup_for_every_record\x18\x02 \x01(\x08\x1aK\n#RocksdbCompactFilterCleanupStrategy\x12$\n\x1cquery_time_after_num_entries\x18\x01 \x01(\x03\x1a\xe0\x04\n\x12MapStrategiesEntry\x12o\n\x08strategy\x18\x01 \x01(\x0e\x32].org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.Strategies\x12\x81\x01\n\x0e\x65mpty_strategy\x18\x02 \x01(\x0e\x32g.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.EmptyCleanupStrategyH\x00\x12\x95\x01\n\x1cincremental_cleanup_strategy\x18\x03 \x01(\x0b\x32m.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.IncrementalCleanupStrategyH\x00\x12\xa9\x01\n\'rocksdb_compact_filter_cleanup_strategy\x18\x04 \x01(\x0b\x32v.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.RocksdbCompactFilterCleanupStrategyH\x00\x42\x11\n\x0f\x43leanupStrategy\"b\n\nStrategies\x12\x1c\n\x18\x46ULL_STATE_SCAN_SNAPSHOT\x10\x00\x12\x17\n\x13INCREMENTAL_CLEANUP\x10\x01\x12\x1d\n\x19ROCKSDB_COMPACTION_FILTER\x10\x02\"*\n\x14\x45mptyCleanupStrategy\x12\x12\n\x0e\x45MPTY_STRATEGY\x10\x00\"D\n\nUpdateType\x12\x0c\n\x08\x44isabled\x10\x00\x12\x14\n\x10OnCreateAndWrite\x10\x01\x12\x12\n\x0eOnReadAndWrite\x10\x02\"J\n\x0fStateVisibility\x12\x1f\n\x1bReturnExpiredIfNotCleanedUp\x10\x00\x12\x16\n\x12NeverReturnExpired\x10\x01\"+\n\x15TtlTimeCharacteristic\x12\x12\n\x0eProcessingTime\x10\x00\"\xf1\x07\n\x13\x43oderInfoDescriptor\x12`\n\x10\x66latten_row_type\x18\x01 \x01(\x0b\x32\x44.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.FlattenRowTypeH\x00\x12Q\n\x08row_type\x18\x02 \x01(\x0b\x32=.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RowTypeH\x00\x12U\n\narrow_type\x18\x03 \x01(\x0b\x32?.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.ArrowTypeH\x00\x12k\n\x16over_window_arrow_type\x18\x04 \x01(\x0b\x32I.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.OverWindowArrowTypeH\x00\x12Q\n\x08raw_type\x18\x05 \x01(\x0b\x32=.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RawTypeH\x00\x12H\n\x04mode\x18\x06 \x01(\x0e\x32:.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.Mode\x12\"\n\x1aseparated_with_end_message\x18\x07 \x01(\x08\x1aJ\n\x0e\x46lattenRowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1a\x43\n\x07RowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1a\x45\n\tArrowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1aO\n\x13OverWindowArrowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1aH\n\x07RawType\x12=\n\ttype_info\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\" \n\x04Mode\x12\n\n\x06SINGLE\x10\x00\x12\x0c\n\x08MULTIPLE\x10\x01\x42\x0b\n\tdata_typeB-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') + serialized_pb=_b('\n\x18\x66link-fn-execution.proto\x12 org.apache.flink.fn_execution.v1\"\x86\x01\n\x05Input\x12\x44\n\x03udf\x18\x01 \x01(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunctionH\x00\x12\x15\n\x0binputOffset\x18\x02 \x01(\x05H\x00\x12\x17\n\rinputConstant\x18\x03 \x01(\x0cH\x00\x42\x07\n\x05input\"\xa8\x01\n\x13UserDefinedFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12\x14\n\x0cwindow_index\x18\x03 \x01(\x05\x12\x1a\n\x12takes_row_as_input\x18\x04 \x01(\x08\x12\x15\n\ris_pandas_udf\x18\x05 \x01(\x08\"\xcb\x01\n\x14UserDefinedFunctions\x12\x43\n\x04udfs\x18\x01 \x03(\x0b\x32\x35.org.apache.flink.fn_execution.v1.UserDefinedFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12=\n\x07windows\x18\x03 \x03(\x0b\x32,.org.apache.flink.fn_execution.v1.OverWindow\x12\x17\n\x0fprofile_enabled\x18\x04 \x01(\x08\"\xdd\x02\n\nOverWindow\x12L\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x37.org.apache.flink.fn_execution.v1.OverWindow.WindowType\x12\x16\n\x0elower_boundary\x18\x02 \x01(\x03\x12\x16\n\x0eupper_boundary\x18\x03 \x01(\x03\"\xd0\x01\n\nWindowType\x12\x13\n\x0fRANGE_UNBOUNDED\x10\x00\x12\x1d\n\x19RANGE_UNBOUNDED_PRECEDING\x10\x01\x12\x1d\n\x19RANGE_UNBOUNDED_FOLLOWING\x10\x02\x12\x11\n\rRANGE_SLIDING\x10\x03\x12\x11\n\rROW_UNBOUNDED\x10\x04\x12\x1b\n\x17ROW_UNBOUNDED_PRECEDING\x10\x05\x12\x1b\n\x17ROW_UNBOUNDED_FOLLOWING\x10\x06\x12\x0f\n\x0bROW_SLIDING\x10\x07\"\x8b\x06\n\x1cUserDefinedAggregateFunction\x12\x0f\n\x07payload\x18\x01 \x01(\x0c\x12\x37\n\x06inputs\x18\x02 \x03(\x0b\x32\'.org.apache.flink.fn_execution.v1.Input\x12Z\n\x05specs\x18\x03 \x03(\x0b\x32K.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec\x12\x12\n\nfilter_arg\x18\x04 \x01(\x05\x12\x10\n\x08\x64istinct\x18\x05 \x01(\x08\x12\x1a\n\x12takes_row_as_input\x18\x06 \x01(\x08\x1a\x82\x04\n\x0c\x44\x61taViewSpec\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x66ield_index\x18\x02 \x01(\x05\x12i\n\tlist_view\x18\x03 \x01(\x0b\x32T.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.ListViewH\x00\x12g\n\x08map_view\x18\x04 \x01(\x0b\x32S.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction.DataViewSpec.MapViewH\x00\x1aT\n\x08ListView\x12H\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x97\x01\n\x07MapView\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeB\x0b\n\tdata_view\"\xac\x04\n\x0bGroupWindow\x12M\n\x0bwindow_type\x18\x01 \x01(\x0e\x32\x38.org.apache.flink.fn_execution.v1.GroupWindow.WindowType\x12\x16\n\x0eis_time_window\x18\x02 \x01(\x08\x12\x14\n\x0cwindow_slide\x18\x03 \x01(\x03\x12\x13\n\x0bwindow_size\x18\x04 \x01(\x03\x12\x12\n\nwindow_gap\x18\x05 \x01(\x03\x12\x13\n\x0bis_row_time\x18\x06 \x01(\x08\x12\x18\n\x10time_field_index\x18\x07 \x01(\x05\x12\x17\n\x0f\x61llowedLateness\x18\x08 \x01(\x03\x12U\n\x0fnamedProperties\x18\t \x03(\x0e\x32<.org.apache.flink.fn_execution.v1.GroupWindow.WindowProperty\x12\x16\n\x0eshift_timezone\x18\n \x01(\t\"[\n\nWindowType\x12\x19\n\x15TUMBLING_GROUP_WINDOW\x10\x00\x12\x18\n\x14SLIDING_GROUP_WINDOW\x10\x01\x12\x18\n\x14SESSION_GROUP_WINDOW\x10\x02\"c\n\x0eWindowProperty\x12\x10\n\x0cWINDOW_START\x10\x00\x12\x0e\n\nWINDOW_END\x10\x01\x12\x16\n\x12ROW_TIME_ATTRIBUTE\x10\x02\x12\x17\n\x13PROC_TIME_ATTRIBUTE\x10\x03\"\x96\x04\n\x1dUserDefinedAggregateFunctions\x12L\n\x04udfs\x18\x01 \x03(\x0b\x32>.org.apache.flink.fn_execution.v1.UserDefinedAggregateFunction\x12\x16\n\x0emetric_enabled\x18\x02 \x01(\x08\x12\x10\n\x08grouping\x18\x03 \x03(\x05\x12\x1e\n\x16generate_update_before\x18\x04 \x01(\x08\x12\x44\n\x08key_type\x18\x05 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x1b\n\x13index_of_count_star\x18\x06 \x01(\x05\x12\x1e\n\x16state_cleaning_enabled\x18\x07 \x01(\x08\x12\x18\n\x10state_cache_size\x18\x08 \x01(\x05\x12!\n\x19map_state_read_cache_size\x18\t \x01(\x05\x12\"\n\x1amap_state_write_cache_size\x18\n \x01(\x05\x12\x1b\n\x13\x63ount_star_inserted\x18\x0b \x01(\x08\x12\x43\n\x0cgroup_window\x18\x0c \x01(\x0b\x32-.org.apache.flink.fn_execution.v1.GroupWindow\x12\x17\n\x0fprofile_enabled\x18\r \x01(\x08\"\xec\x0f\n\x06Schema\x12>\n\x06\x66ields\x18\x01 \x03(\x0b\x32..org.apache.flink.fn_execution.v1.Schema.Field\x1a\x97\x01\n\x07MapInfo\x12\x44\n\x08key_type\x18\x01 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x12\x46\n\nvalue_type\x18\x02 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\x1a\x1d\n\x08TimeInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\"\n\rTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a,\n\x17LocalZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a\'\n\x12ZonedTimestampInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x1a/\n\x0b\x44\x65\x63imalInfo\x12\x11\n\tprecision\x18\x01 \x01(\x05\x12\r\n\x05scale\x18\x02 \x01(\x05\x1a\x1c\n\nBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1f\n\rVarBinaryInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1a\n\x08\x43harInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\x1d\n\x0bVarCharInfo\x12\x0e\n\x06length\x18\x01 \x01(\x05\x1a\xb0\x08\n\tFieldType\x12\x44\n\ttype_name\x18\x01 \x01(\x0e\x32\x31.org.apache.flink.fn_execution.v1.Schema.TypeName\x12\x10\n\x08nullable\x18\x02 \x01(\x08\x12U\n\x17\x63ollection_element_type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldTypeH\x00\x12\x44\n\x08map_info\x18\x04 \x01(\x0b\x32\x30.org.apache.flink.fn_execution.v1.Schema.MapInfoH\x00\x12>\n\nrow_schema\x18\x05 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.SchemaH\x00\x12L\n\x0c\x64\x65\x63imal_info\x18\x06 \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.DecimalInfoH\x00\x12\x46\n\ttime_info\x18\x07 \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.TimeInfoH\x00\x12P\n\x0etimestamp_info\x18\x08 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.TimestampInfoH\x00\x12\x66\n\x1alocal_zoned_timestamp_info\x18\t \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.Schema.LocalZonedTimestampInfoH\x00\x12[\n\x14zoned_timestamp_info\x18\n \x01(\x0b\x32;.org.apache.flink.fn_execution.v1.Schema.ZonedTimestampInfoH\x00\x12J\n\x0b\x62inary_info\x18\x0b \x01(\x0b\x32\x33.org.apache.flink.fn_execution.v1.Schema.BinaryInfoH\x00\x12Q\n\x0fvar_binary_info\x18\x0c \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.Schema.VarBinaryInfoH\x00\x12\x46\n\tchar_info\x18\r \x01(\x0b\x32\x31.org.apache.flink.fn_execution.v1.Schema.CharInfoH\x00\x12M\n\rvar_char_info\x18\x0e \x01(\x0b\x32\x34.org.apache.flink.fn_execution.v1.Schema.VarCharInfoH\x00\x42\x0b\n\ttype_info\x1al\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\x13\n\x0b\x64\x65scription\x18\x02 \x01(\t\x12@\n\x04type\x18\x03 \x01(\x0b\x32\x32.org.apache.flink.fn_execution.v1.Schema.FieldType\"\xa1\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\x0b\n\x07TINYINT\x10\x01\x12\x0c\n\x08SMALLINT\x10\x02\x12\x07\n\x03INT\x10\x03\x12\n\n\x06\x42IGINT\x10\x04\x12\x0b\n\x07\x44\x45\x43IMAL\x10\x05\x12\t\n\x05\x46LOAT\x10\x06\x12\n\n\x06\x44OUBLE\x10\x07\x12\x08\n\x04\x44\x41TE\x10\x08\x12\x08\n\x04TIME\x10\t\x12\r\n\tTIMESTAMP\x10\n\x12\x0b\n\x07\x42OOLEAN\x10\x0b\x12\n\n\x06\x42INARY\x10\x0c\x12\r\n\tVARBINARY\x10\r\x12\x08\n\x04\x43HAR\x10\x0e\x12\x0b\n\x07VARCHAR\x10\x0f\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x10\x12\x07\n\x03MAP\x10\x11\x12\x0c\n\x08MULTISET\x10\x12\x12\x19\n\x15LOCAL_ZONED_TIMESTAMP\x10\x13\x12\x13\n\x0fZONED_TIMESTAMP\x10\x14\"\xf7\x08\n\x08TypeInfo\x12\x46\n\ttype_name\x18\x01 \x01(\x0e\x32\x33.org.apache.flink.fn_execution.v1.TypeInfo.TypeName\x12M\n\x17\x63ollection_element_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfoH\x00\x12O\n\rrow_type_info\x18\x03 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfoH\x00\x12S\n\x0ftuple_type_info\x18\x04 \x01(\x0b\x32\x38.org.apache.flink.fn_execution.v1.TypeInfo.TupleTypeInfoH\x00\x12O\n\rmap_type_info\x18\x05 \x01(\x0b\x32\x36.org.apache.flink.fn_execution.v1.TypeInfo.MapTypeInfoH\x00\x1a\x8b\x01\n\x0bMapTypeInfo\x12<\n\x08key_type\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x12>\n\nvalue_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1a\xb8\x01\n\x0bRowTypeInfo\x12L\n\x06\x66ields\x18\x01 \x03(\x0b\x32<.org.apache.flink.fn_execution.v1.TypeInfo.RowTypeInfo.Field\x1a[\n\x05\x46ield\x12\x12\n\nfield_name\x18\x01 \x01(\t\x12>\n\nfield_type\x18\x02 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x1aP\n\rTupleTypeInfo\x12?\n\x0b\x66ield_types\x18\x01 \x03(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\"\xb4\x02\n\x08TypeName\x12\x07\n\x03ROW\x10\x00\x12\n\n\x06STRING\x10\x01\x12\x08\n\x04\x42YTE\x10\x02\x12\x0b\n\x07\x42OOLEAN\x10\x03\x12\t\n\x05SHORT\x10\x04\x12\x07\n\x03INT\x10\x05\x12\x08\n\x04LONG\x10\x06\x12\t\n\x05\x46LOAT\x10\x07\x12\n\n\x06\x44OUBLE\x10\x08\x12\x08\n\x04\x43HAR\x10\t\x12\x0b\n\x07\x42IG_INT\x10\n\x12\x0b\n\x07\x42IG_DEC\x10\x0b\x12\x0c\n\x08SQL_DATE\x10\x0c\x12\x0c\n\x08SQL_TIME\x10\r\x12\x11\n\rSQL_TIMESTAMP\x10\x0e\x12\x0f\n\x0b\x42\x41SIC_ARRAY\x10\x0f\x12\x13\n\x0fPRIMITIVE_ARRAY\x10\x10\x12\t\n\x05TUPLE\x10\x11\x12\x08\n\x04LIST\x10\x12\x12\x07\n\x03MAP\x10\x13\x12\x11\n\rPICKLED_BYTES\x10\x14\x12\x10\n\x0cOBJECT_ARRAY\x10\x15\x12\x0b\n\x07INSTANT\x10\x16\x42\x0b\n\ttype_info\"\xc7\x07\n\x1dUserDefinedDataStreamFunction\x12\x63\n\rfunction_type\x18\x01 \x01(\x0e\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.FunctionType\x12g\n\x0fruntime_context\x18\x02 \x01(\x0b\x32N.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.RuntimeContext\x12\x0f\n\x07payload\x18\x03 \x01(\x0c\x12\x16\n\x0emetric_enabled\x18\x04 \x01(\x08\x12\x41\n\rkey_type_info\x18\x05 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\x12\x17\n\x0fprofile_enabled\x18\x06 \x01(\x08\x12\x18\n\x10state_cache_size\x18\x07 \x01(\x05\x12!\n\x19map_state_read_cache_size\x18\x08 \x01(\x05\x12\"\n\x1amap_state_write_cache_size\x18\t \x01(\x05\x1a*\n\x0cJobParameter\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t\x1a\xd0\x02\n\x0eRuntimeContext\x12\x11\n\ttask_name\x18\x01 \x01(\t\x12\x1f\n\x17task_name_with_subtasks\x18\x02 \x01(\t\x12#\n\x1bnumber_of_parallel_subtasks\x18\x03 \x01(\x05\x12\'\n\x1fmax_number_of_parallel_subtasks\x18\x04 \x01(\x05\x12\x1d\n\x15index_of_this_subtask\x18\x05 \x01(\x05\x12\x16\n\x0e\x61ttempt_number\x18\x06 \x01(\x05\x12\x64\n\x0ejob_parameters\x18\x07 \x03(\x0b\x32L.org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.JobParameter\x12\x1f\n\x17in_batch_execution_mode\x18\x08 \x01(\x08\"s\n\x0c\x46unctionType\x12\x0b\n\x07PROCESS\x10\x00\x12\x0e\n\nCO_PROCESS\x10\x01\x12\x11\n\rKEYED_PROCESS\x10\x02\x12\x14\n\x10KEYED_CO_PROCESS\x10\x03\x12\n\n\x06WINDOW\x10\x04\x12\x11\n\rREVISE_OUTPUT\x10\x64\"\xe4\x0e\n\x0fStateDescriptor\x12\x12\n\nstate_name\x18\x01 \x01(\t\x12Z\n\x10state_ttl_config\x18\x02 \x01(\x0b\x32@.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig\x1a\xe0\r\n\x0eStateTTLConfig\x12`\n\x0bupdate_type\x18\x01 \x01(\x0e\x32K.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.UpdateType\x12j\n\x10state_visibility\x18\x02 \x01(\x0e\x32P.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.StateVisibility\x12w\n\x17ttl_time_characteristic\x18\x03 \x01(\x0e\x32V.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.TtlTimeCharacteristic\x12\x0b\n\x03ttl\x18\x04 \x01(\x03\x12n\n\x12\x63leanup_strategies\x18\x05 \x01(\x0b\x32R.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies\x1a\xca\x08\n\x11\x43leanupStrategies\x12 \n\x18is_cleanup_in_background\x18\x01 \x01(\x08\x12y\n\nstrategies\x18\x02 \x03(\x0b\x32\x65.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.MapStrategiesEntry\x1aX\n\x1aIncrementalCleanupStrategy\x12\x14\n\x0c\x63leanup_size\x18\x01 \x01(\x05\x12$\n\x1crun_cleanup_for_every_record\x18\x02 \x01(\x08\x1aK\n#RocksdbCompactFilterCleanupStrategy\x12$\n\x1cquery_time_after_num_entries\x18\x01 \x01(\x03\x1a\xe0\x04\n\x12MapStrategiesEntry\x12o\n\x08strategy\x18\x01 \x01(\x0e\x32].org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.Strategies\x12\x81\x01\n\x0e\x65mpty_strategy\x18\x02 \x01(\x0e\x32g.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.EmptyCleanupStrategyH\x00\x12\x95\x01\n\x1cincremental_cleanup_strategy\x18\x03 \x01(\x0b\x32m.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.IncrementalCleanupStrategyH\x00\x12\xa9\x01\n\'rocksdb_compact_filter_cleanup_strategy\x18\x04 \x01(\x0b\x32v.org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.RocksdbCompactFilterCleanupStrategyH\x00\x42\x11\n\x0f\x43leanupStrategy\"b\n\nStrategies\x12\x1c\n\x18\x46ULL_STATE_SCAN_SNAPSHOT\x10\x00\x12\x17\n\x13INCREMENTAL_CLEANUP\x10\x01\x12\x1d\n\x19ROCKSDB_COMPACTION_FILTER\x10\x02\"*\n\x14\x45mptyCleanupStrategy\x12\x12\n\x0e\x45MPTY_STRATEGY\x10\x00\"D\n\nUpdateType\x12\x0c\n\x08\x44isabled\x10\x00\x12\x14\n\x10OnCreateAndWrite\x10\x01\x12\x12\n\x0eOnReadAndWrite\x10\x02\"J\n\x0fStateVisibility\x12\x1f\n\x1bReturnExpiredIfNotCleanedUp\x10\x00\x12\x16\n\x12NeverReturnExpired\x10\x01\"+\n\x15TtlTimeCharacteristic\x12\x12\n\x0eProcessingTime\x10\x00\"\xf1\x07\n\x13\x43oderInfoDescriptor\x12`\n\x10\x66latten_row_type\x18\x01 \x01(\x0b\x32\x44.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.FlattenRowTypeH\x00\x12Q\n\x08row_type\x18\x02 \x01(\x0b\x32=.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RowTypeH\x00\x12U\n\narrow_type\x18\x03 \x01(\x0b\x32?.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.ArrowTypeH\x00\x12k\n\x16over_window_arrow_type\x18\x04 \x01(\x0b\x32I.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.OverWindowArrowTypeH\x00\x12Q\n\x08raw_type\x18\x05 \x01(\x0b\x32=.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.RawTypeH\x00\x12H\n\x04mode\x18\x06 \x01(\x0e\x32:.org.apache.flink.fn_execution.v1.CoderInfoDescriptor.Mode\x12\"\n\x1aseparated_with_end_message\x18\x07 \x01(\x08\x1aJ\n\x0e\x46lattenRowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1a\x43\n\x07RowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1a\x45\n\tArrowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1aO\n\x13OverWindowArrowType\x12\x38\n\x06schema\x18\x01 \x01(\x0b\x32(.org.apache.flink.fn_execution.v1.Schema\x1aH\n\x07RawType\x12=\n\ttype_info\x18\x01 \x01(\x0b\x32*.org.apache.flink.fn_execution.v1.TypeInfo\" \n\x04Mode\x12\n\n\x06SINGLE\x10\x00\x12\x0c\n\x08MULTIPLE\x10\x01\x42\x0b\n\tdata_typeB-\n\x1forg.apache.flink.fnexecution.v1B\nFlinkFnApib\x06proto3') ) @@ -380,8 +380,8 @@ ], containing_type=None, options=None, - serialized_start=6739, - serialized_end=6854, + serialized_start=6836, + serialized_end=6951, ) _sym_db.RegisterEnumDescriptor(_USERDEFINEDDATASTREAMFUNCTION_FUNCTIONTYPE) @@ -406,8 +406,8 @@ ], containing_type=None, options=None, - serialized_start=8416, - serialized_end=8514, + serialized_start=8513, + serialized_end=8611, ) _sym_db.RegisterEnumDescriptor(_STATEDESCRIPTOR_STATETTLCONFIG_CLEANUPSTRATEGIES_STRATEGIES) @@ -424,8 +424,8 @@ ], containing_type=None, options=None, - serialized_start=8516, - serialized_end=8558, + serialized_start=8613, + serialized_end=8655, ) _sym_db.RegisterEnumDescriptor(_STATEDESCRIPTOR_STATETTLCONFIG_CLEANUPSTRATEGIES_EMPTYCLEANUPSTRATEGY) @@ -450,8 +450,8 @@ ], containing_type=None, options=None, - serialized_start=8560, - serialized_end=8628, + serialized_start=8657, + serialized_end=8725, ) _sym_db.RegisterEnumDescriptor(_STATEDESCRIPTOR_STATETTLCONFIG_UPDATETYPE) @@ -472,8 +472,8 @@ ], containing_type=None, options=None, - serialized_start=8630, - serialized_end=8704, + serialized_start=8727, + serialized_end=8801, ) _sym_db.RegisterEnumDescriptor(_STATEDESCRIPTOR_STATETTLCONFIG_STATEVISIBILITY) @@ -490,8 +490,8 @@ ], containing_type=None, options=None, - serialized_start=8706, - serialized_end=8749, + serialized_start=8803, + serialized_end=8846, ) _sym_db.RegisterEnumDescriptor(_STATEDESCRIPTOR_STATETTLCONFIG_TTLTIMECHARACTERISTIC) @@ -512,8 +512,8 @@ ], containing_type=None, options=None, - serialized_start=9716, - serialized_end=9748, + serialized_start=9813, + serialized_end=9845, ) _sym_db.RegisterEnumDescriptor(_CODERINFODESCRIPTOR_MODE) @@ -1865,8 +1865,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6356, - serialized_end=6398, + serialized_start=6453, + serialized_end=6495, ) _USERDEFINEDDATASTREAMFUNCTION_RUNTIMECONTEXT = _descriptor.Descriptor( @@ -1944,8 +1944,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6401, - serialized_end=6737, + serialized_start=6498, + serialized_end=6834, ) _USERDEFINEDDATASTREAMFUNCTION = _descriptor.Descriptor( @@ -1997,6 +1997,27 @@ message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='state_cache_size', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.state_cache_size', index=6, + number=7, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='map_state_read_cache_size', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.map_state_read_cache_size', index=7, + number=8, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='map_state_write_cache_size', full_name='org.apache.flink.fn_execution.v1.UserDefinedDataStreamFunction.map_state_write_cache_size', index=8, + number=9, type=5, cpp_type=1, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + options=None, file=DESCRIPTOR), ], extensions=[ ], @@ -2011,7 +2032,7 @@ oneofs=[ ], serialized_start=5984, - serialized_end=6854, + serialized_end=6951, ) @@ -2048,8 +2069,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=7638, - serialized_end=7726, + serialized_start=7735, + serialized_end=7823, ) _STATEDESCRIPTOR_STATETTLCONFIG_CLEANUPSTRATEGIES_ROCKSDBCOMPACTFILTERCLEANUPSTRATEGY = _descriptor.Descriptor( @@ -2078,8 +2099,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=7728, - serialized_end=7803, + serialized_start=7825, + serialized_end=7900, ) _STATEDESCRIPTOR_STATETTLCONFIG_CLEANUPSTRATEGIES_MAPSTRATEGIESENTRY = _descriptor.Descriptor( @@ -2132,8 +2153,8 @@ name='CleanupStrategy', full_name='org.apache.flink.fn_execution.v1.StateDescriptor.StateTTLConfig.CleanupStrategies.MapStrategiesEntry.CleanupStrategy', index=0, containing_type=None, fields=[]), ], - serialized_start=7806, - serialized_end=8414, + serialized_start=7903, + serialized_end=8511, ) _STATEDESCRIPTOR_STATETTLCONFIG_CLEANUPSTRATEGIES = _descriptor.Descriptor( @@ -2171,8 +2192,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=7460, - serialized_end=8558, + serialized_start=7557, + serialized_end=8655, ) _STATEDESCRIPTOR_STATETTLCONFIG = _descriptor.Descriptor( @@ -2232,8 +2253,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6989, - serialized_end=8749, + serialized_start=7086, + serialized_end=8846, ) _STATEDESCRIPTOR = _descriptor.Descriptor( @@ -2269,8 +2290,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=6857, - serialized_end=8749, + serialized_start=6954, + serialized_end=8846, ) @@ -2300,8 +2321,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=9345, - serialized_end=9419, + serialized_start=9442, + serialized_end=9516, ) _CODERINFODESCRIPTOR_ROWTYPE = _descriptor.Descriptor( @@ -2330,8 +2351,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=9421, - serialized_end=9488, + serialized_start=9518, + serialized_end=9585, ) _CODERINFODESCRIPTOR_ARROWTYPE = _descriptor.Descriptor( @@ -2360,8 +2381,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=9490, - serialized_end=9559, + serialized_start=9587, + serialized_end=9656, ) _CODERINFODESCRIPTOR_OVERWINDOWARROWTYPE = _descriptor.Descriptor( @@ -2390,8 +2411,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=9561, - serialized_end=9640, + serialized_start=9658, + serialized_end=9737, ) _CODERINFODESCRIPTOR_RAWTYPE = _descriptor.Descriptor( @@ -2420,8 +2441,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=9642, - serialized_end=9714, + serialized_start=9739, + serialized_end=9811, ) _CODERINFODESCRIPTOR = _descriptor.Descriptor( @@ -2496,8 +2517,8 @@ name='data_type', full_name='org.apache.flink.fn_execution.v1.CoderInfoDescriptor.data_type', index=0, containing_type=None, fields=[]), ], - serialized_start=8752, - serialized_end=9761, + serialized_start=8849, + serialized_end=9858, ) _INPUT.fields_by_name['udf'].message_type = _USERDEFINEDFUNCTION diff --git a/flink-python/pyflink/proto/flink-fn-execution.proto b/flink-python/pyflink/proto/flink-fn-execution.proto index 95ee2f5c0377d..7ea69a55f8b93 100644 --- a/flink-python/pyflink/proto/flink-fn-execution.proto +++ b/flink-python/pyflink/proto/flink-fn-execution.proto @@ -371,6 +371,13 @@ message UserDefinedDataStreamFunction { bool metric_enabled = 4; TypeInfo key_type_info = 5; bool profile_enabled = 6; + + // The state cache size. + int32 state_cache_size = 7; + // The map state read cache size. + int32 map_state_read_cache_size = 8; + // The map_state_write_cache_size. + int32 map_state_write_cache_size = 9; } // A representation of State diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java index 2b519aeb24e65..3a13f6725dda2 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonCoProcessOperator.java @@ -30,6 +30,11 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import static org.apache.flink.python.Constants.STATELESS_FUNCTION_URN; +import static org.apache.flink.python.PythonOptions.MAP_STATE_READ_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.MAP_STATE_WRITE_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; +import static org.apache.flink.python.PythonOptions.STATE_CACHE_SIZE; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; /** @@ -73,7 +78,12 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getPythonFunctionInfo(), getRuntimeContext(), getInternalParameters(), - inBatchExecutionMode(getKeyedStateBackend())), + inBatchExecutionMode(getKeyedStateBackend()), + config.get(PYTHON_METRIC_ENABLED), + config.get(PYTHON_PROFILE_ENABLED), + config.get(STATE_CACHE_SIZE), + config.get(MAP_STATE_READ_CACHE_SIZE), + config.get(MAP_STATE_WRITE_CACHE_SIZE)), getFlinkMetricContainer(), null, null, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java index 924b86122de4b..26ccaedf9ad48 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedCoProcessOperator.java @@ -41,6 +41,11 @@ import org.apache.flink.types.Row; import static org.apache.flink.python.Constants.STATEFUL_FUNCTION_URN; +import static org.apache.flink.python.PythonOptions.MAP_STATE_READ_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.MAP_STATE_WRITE_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; +import static org.apache.flink.python.PythonOptions.STATE_CACHE_SIZE; import static org.apache.flink.streaming.api.operators.python.timer.TimerUtils.createTimerDataCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.operators.python.timer.TimerUtils.createTimerDataTypeInfo; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; @@ -112,7 +117,12 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getRuntimeContext(), getInternalParameters(), keyTypeInfo, - inBatchExecutionMode(getKeyedStateBackend())), + inBatchExecutionMode(getKeyedStateBackend()), + config.get(PYTHON_METRIC_ENABLED), + config.get(PYTHON_PROFILE_ENABLED), + config.get(STATE_CACHE_SIZE), + config.get(MAP_STATE_READ_CACHE_SIZE), + config.get(MAP_STATE_WRITE_CACHE_SIZE)), getFlinkMetricContainer(), getKeyedStateBackend(), keyTypeSerializer, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java index cef5e6ce01dc3..2896ed3be4731 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonKeyedProcessOperator.java @@ -41,6 +41,11 @@ import org.apache.flink.types.Row; import static org.apache.flink.python.Constants.STATEFUL_FUNCTION_URN; +import static org.apache.flink.python.PythonOptions.MAP_STATE_READ_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.MAP_STATE_WRITE_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; +import static org.apache.flink.python.PythonOptions.STATE_CACHE_SIZE; import static org.apache.flink.streaming.api.operators.python.timer.TimerUtils.createTimerDataCoderInfoDescriptorProto; import static org.apache.flink.streaming.api.operators.python.timer.TimerUtils.createTimerDataTypeInfo; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; @@ -142,7 +147,12 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getRuntimeContext(), getInternalParameters(), keyTypeInfo, - inBatchExecutionMode(getKeyedStateBackend())), + inBatchExecutionMode(getKeyedStateBackend()), + config.get(PYTHON_METRIC_ENABLED), + config.get(PYTHON_PROFILE_ENABLED), + config.get(STATE_CACHE_SIZE), + config.get(MAP_STATE_READ_CACHE_SIZE), + config.get(MAP_STATE_WRITE_CACHE_SIZE)), getFlinkMetricContainer(), getKeyedStateBackend(), keyTypeSerializer, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java index 1bc820e58585a..48e11fab123fa 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/operators/python/PythonProcessOperator.java @@ -31,6 +31,11 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import static org.apache.flink.python.Constants.STATELESS_FUNCTION_URN; +import static org.apache.flink.python.PythonOptions.MAP_STATE_READ_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.MAP_STATE_WRITE_CACHE_SIZE; +import static org.apache.flink.python.PythonOptions.PYTHON_METRIC_ENABLED; +import static org.apache.flink.python.PythonOptions.PYTHON_PROFILE_ENABLED; +import static org.apache.flink.python.PythonOptions.STATE_CACHE_SIZE; import static org.apache.flink.streaming.api.utils.PythonOperatorUtils.inBatchExecutionMode; /** @@ -70,7 +75,12 @@ public PythonFunctionRunner createPythonFunctionRunner() throws Exception { getPythonFunctionInfo(), getRuntimeContext(), getInternalParameters(), - inBatchExecutionMode(getKeyedStateBackend())), + inBatchExecutionMode(getKeyedStateBackend()), + config.get(PYTHON_METRIC_ENABLED), + config.get(PYTHON_PROFILE_ENABLED), + config.get(STATE_CACHE_SIZE), + config.get(MAP_STATE_READ_CACHE_SIZE), + config.get(MAP_STATE_WRITE_CACHE_SIZE)), getFlinkMetricContainer(), null, null, diff --git a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java index 6676d84235ba2..e5e482a6aa196 100644 --- a/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java +++ b/flink-python/src/main/java/org/apache/flink/streaming/api/utils/ProtoUtils.java @@ -132,7 +132,12 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre DataStreamPythonFunctionInfo dataStreamPythonFunctionInfo, RuntimeContext runtimeContext, Map internalParameters, - boolean inBatchExecutionMode) { + boolean inBatchExecutionMode, + boolean isMetricEnabled, + boolean isProfileEnabled, + int stateCacheSize, + int mapStateReadCacheSize, + int mapStateWriteCacheSize) { FlinkFnApi.UserDefinedDataStreamFunction.Builder builder = FlinkFnApi.UserDefinedDataStreamFunction.newBuilder(); builder.setFunctionType( @@ -175,7 +180,11 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre dataStreamPythonFunctionInfo .getPythonFunction() .getSerializedPythonFunction())); - builder.setMetricEnabled(true); + builder.setMetricEnabled(isMetricEnabled); + builder.setProfileEnabled(isProfileEnabled); + builder.setStateCacheSize(stateCacheSize); + builder.setMapStateReadCacheSize(mapStateReadCacheSize); + builder.setMapStateWriteCacheSize(mapStateWriteCacheSize); return builder.build(); } @@ -192,7 +201,12 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre DataStreamPythonFunctionInfo dataStreamPythonFunctionInfo, RuntimeContext runtimeContext, Map internalParameters, - boolean inBatchExecutionMode) { + boolean inBatchExecutionMode, + boolean isMetricEnabled, + boolean isProfileEnabled, + int stateCacheSize, + int mapStateReadCacheSize, + int mapStateWriteCacheSize) { List results = new ArrayList<>(); Object[] inputs = dataStreamPythonFunctionInfo.getInputs(); @@ -203,7 +217,12 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre (DataStreamPythonFunctionInfo) inputs[0], runtimeContext, internalParameters, - inBatchExecutionMode)); + inBatchExecutionMode, + isMetricEnabled, + isProfileEnabled, + stateCacheSize, + mapStateReadCacheSize, + mapStateWriteCacheSize)); } results.add( @@ -211,7 +230,12 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre dataStreamPythonFunctionInfo, runtimeContext, internalParameters, - inBatchExecutionMode)); + inBatchExecutionMode, + isMetricEnabled, + isProfileEnabled, + stateCacheSize, + mapStateReadCacheSize, + mapStateWriteCacheSize)); return results; } @@ -221,13 +245,23 @@ public static FlinkFnApi.UserDefinedDataStreamFunction createUserDefinedDataStre RuntimeContext runtimeContext, Map internalParameters, TypeInformation keyTypeInfo, - boolean inBatchExecutionMode) { + boolean inBatchExecutionMode, + boolean isMetricEnabled, + boolean isProfileEnabled, + int stateCacheSize, + int mapStateReadCacheSize, + int mapStateWriteCacheSize) { List results = createUserDefinedDataStreamFunctionProtos( dataStreamPythonFunctionInfo, runtimeContext, internalParameters, - inBatchExecutionMode); + inBatchExecutionMode, + isMetricEnabled, + isProfileEnabled, + stateCacheSize, + mapStateReadCacheSize, + mapStateWriteCacheSize); // set the key typeinfo for the head operator FlinkFnApi.TypeInfo builtKeyTypeInfo = From 7ce4be0995740c3ae4a455ba992c48d10d48bd4c Mon Sep 17 00:00:00 2001 From: yuxia Luo Date: Thu, 18 Nov 2021 20:17:01 +0800 Subject: [PATCH 161/258] [FLINK-24940][docs] Correct usage about how to create Hive catalog via Flink SQL CLI. This closes #17829 --- .../connectors/table/hive/hive_catalog.md | 34 ++++++++----------- .../connectors/table/hive/hive_catalog.md | 34 ++++++++----------- 2 files changed, 30 insertions(+), 38 deletions(-) diff --git a/docs/content.zh/docs/connectors/table/hive/hive_catalog.md b/docs/content.zh/docs/connectors/table/hive/hive_catalog.md index 6cc9e3041b240..dc2e461fd7f55 100644 --- a/docs/content.zh/docs/connectors/table/hive/hive_catalog.md +++ b/docs/content.zh/docs/connectors/table/hive/hive_catalog.md @@ -137,28 +137,23 @@ Time taken: 0.028 seconds, Fetched: 0 row(s) ``` -#### step 2: configure Flink cluster and SQL CLI - -Add all Hive dependencies to `/lib` dir in Flink distribution, and modify SQL CLI's yaml config file `sql-cli-defaults.yaml` as following: - -```yaml - -execution: - type: streaming - ... - current-catalog: myhive # set the HiveCatalog as the current catalog of the session - current-database: mydatabase - -catalogs: - - name: myhive - type: hive - hive-conf-dir: /opt/hive-conf # contains hive-site.xml +#### step 2: start SQL Client, and create a Hive catalog with Flink SQL DDL + +Add all Hive dependencies to `/lib` dir in Flink distribution, and create a Hive catalog in Flink SQL CLI as following: + +```bash + +Flink SQL> CREATE CATALOG myhive WITH ( + 'type' = 'hive', + 'hive-conf-dir' = '/opt/hive-conf' +); + ``` #### step 3: set up a Kafka cluster -Bootstrap a local Kafka 2.3.0 cluster with a topic named "test", and produce some simple data to the topic as tuple of name and age. +Bootstrap a local Kafka cluster with a topic named "test", and produce some simple data to the topic as tuple of name and age. ```bash @@ -180,11 +175,12 @@ john,21 ``` -#### step 4: start SQL Client, and create a Kafka table with Flink SQL DDL +#### step 4: create a Kafka table with Flink SQL DDL -Start Flink SQL Client, create a simple Kafka 2.3.0 table via DDL, and verify its schema. +Create a simple Kafka table with Flink SQL DDL, and verify its schema. ```bash +Flink SQL> USE CATALOG myhive; Flink SQL> CREATE TABLE mykafka (name String, age Int) WITH ( 'connector.type' = 'kafka', diff --git a/docs/content/docs/connectors/table/hive/hive_catalog.md b/docs/content/docs/connectors/table/hive/hive_catalog.md index 90c1aebf0576f..932e18fcc0d36 100644 --- a/docs/content/docs/connectors/table/hive/hive_catalog.md +++ b/docs/content/docs/connectors/table/hive/hive_catalog.md @@ -137,28 +137,23 @@ Time taken: 0.028 seconds, Fetched: 0 row(s) ``` -#### step 2: configure Flink cluster and SQL CLI - -Add all Hive dependencies to `/lib` dir in Flink distribution, and modify SQL CLI's yaml config file `sql-cli-defaults.yaml` as following: - -```yaml - -execution: - type: streaming - ... - current-catalog: myhive # set the HiveCatalog as the current catalog of the session - current-database: mydatabase - -catalogs: - - name: myhive - type: hive - hive-conf-dir: /opt/hive-conf # contains hive-site.xml +#### step 2: start SQL Client, and create a Hive catalog with Flink SQL DDL + +Add all Hive dependencies to `/lib` dir in Flink distribution, and create a Hive catalog in Flink SQL CLI as following: + +```bash + +Flink SQL> CREATE CATALOG myhive WITH ( + 'type' = 'hive', + 'hive-conf-dir' = '/opt/hive-conf' +); + ``` #### step 3: set up a Kafka cluster -Bootstrap a local Kafka 2.3.0 cluster with a topic named "test", and produce some simple data to the topic as tuple of name and age. +Bootstrap a local Kafka cluster with a topic named "test", and produce some simple data to the topic as tuple of name and age. ```bash @@ -180,11 +175,12 @@ john,21 ``` -#### step 4: start SQL Client, and create a Kafka table with Flink SQL DDL +#### step 4: create a Kafka table with Flink SQL DDL -Start Flink SQL Client, create a simple Kafka 2.3.0 table via DDL, and verify its schema. +Create a simple Kafka table with Flink SQL DDL, and verify its schema. ```bash +Flink SQL> USE CATALOG myhive; Flink SQL> CREATE TABLE mykafka (name String, age Int) WITH ( 'connector.type' = 'kafka', From 17b8d1fc8adca7a5760fa5074850c89df35b3275 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 8 Apr 2022 15:09:28 +0800 Subject: [PATCH 162/258] [hotfix] Fix the compiling issue --- .../planner/plan/nodes/exec/ExecNodeConfig.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java index b33396fbec0d9..ebfe752141b60 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java @@ -36,7 +36,7 @@ @Internal public final class ExecNodeConfig implements ReadableConfig { - private final ReadableConfig tableConfig; + private final TableConfig tableConfig; private final ReadableConfig nodeConfig; @@ -45,6 +45,16 @@ public final class ExecNodeConfig implements ReadableConfig { this.tableConfig = tableConfig; } + /** + * Return the {@link PlannerBase#getTableConfig()}. + * + * @return the {@link PlannerBase#getTableConfig()}. + */ + @Deprecated + public TableConfig getTableConfig() { + return tableConfig; + } + @Override public T get(ConfigOption option) { return nodeConfig.getOptional(option).orElseGet(() -> tableConfig.get(option)); From c8e37d9d850f76093fecf68fbfb81065bfc6f914 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Wed, 6 Apr 2022 17:13:35 +0300 Subject: [PATCH 163/258] [FLINK-27089][table-planner] Fix bug with TRY_CAST in batch mode When in batch mode, the `getMonotonicity()` method of a function is called, to determine possible optimisations. For `TRY_CAST` the implementation needs to call `getOperandType(1)` to get the target (also the function's return) type of the cast. This fails as for `CAST` and `TRY_CAST` at this point we have only one operand. `CAST` solves this in Calcite code, more specifically in `RexCallBinding#create()` where in case the `kind` of the function is `CAST`, a special `RexCastCallBinding` instance is created which stores the return (target) type and returns it when `getOperandType(1)` is called. For `TRY_CAST` we don't have access to the stack to do something similar, and we cannot set the kind of `TRY_CAST` to `CAST` (currently, it's `OTHER_FUNCTION`, as this will allow the calcite stack to apply rules and optimisations to the `TRY_CAST` call and at some point convert it to a regular `CAST` call, thus breaking the functionality of `TRY_CAST` (return null instead of failing). As a solution to the problem, we simply don't implement the `getMonotonicity()` method for `TRY_CAST`, lossing possible optmisations. This closes #19379. --- .../planner/functions/sql/SqlTryCastFunction.java | 8 +------- .../planner/expressions/ScalarOperatorsTest.scala | 13 +++++++++++++ .../planner/runtime/batch/sql/CalcITCase.scala | 8 ++++++++ 3 files changed, 22 insertions(+), 7 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlTryCastFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlTryCastFunction.java index fffb5fe5b0618..5c307e606f8c1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlTryCastFunction.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlTryCastFunction.java @@ -33,8 +33,6 @@ import org.apache.calcite.sql.fun.SqlCastFunction; import org.apache.calcite.sql.fun.SqlStdOperatorTable; -import static org.apache.flink.table.functions.BuiltInFunctionDefinition.DEFAULT_VERSION; - /** * This class implements the {@code TRY_CAST} built-in, essentially delegating all the method * invocations, whenever is possible, to Calcite's {@link SqlCastFunction}. @@ -49,16 +47,12 @@ public class SqlTryCastFunction extends BuiltInSqlFunction { SqlTryCastFunction() { super( "TRY_CAST", - DEFAULT_VERSION, SqlKind.OTHER_FUNCTION, null, SqlStdOperatorTable.CAST .getOperandTypeInference(), // From Calcite's SqlCastFunction null, - SqlFunctionCategory.SYSTEM, - true, - false, - SqlStdOperatorTable.CAST::getMonotonicity); + SqlFunctionCategory.SYSTEM); } @Override diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala index 459ded7797876..899a80cfabcc7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarOperatorsTest.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.expressions +import org.apache.flink.table.api.{DataTypes, LiteralStringExpression, UnresolvedFieldExpression} import org.apache.flink.table.planner.expressions.utils.ScalarOperatorsTestBase import org.junit.Test @@ -239,4 +240,16 @@ class ScalarOperatorsTest extends ScalarOperatorsTestBase { "FALSE") testSqlApi("uuid() = cast(f22 as timestamp_ltz)", "NULL") } + + @Test + def testTryCast(): Unit = { + testAllApis( + "non-numeric".tryCast(DataTypes.BIGINT()), + "TRY_CAST ('non-numeric' AS BIGINT)", + "NULL") + testAllApis( + 'f10.tryCast(DataTypes.BIGINT()), + "TRY_CAST (f10 AS BIGINT)", + "NULL") + } } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala index db904a19f425f..45976cef02b20 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/CalcITCase.scala @@ -1858,4 +1858,12 @@ class CalcITCase extends BatchTestBase { LocalTime.of(16, 50, 1, 123000000) ))) } + + @Test + def testTryCast(): Unit = { + checkResult("SELECT TRY_CAST('invalid' AS INT)", Seq(row(null))) + checkResult( + "SELECT TRY_CAST(g AS DOUBLE) FROM testTable", + Seq(row(null), row(null), row(null))) + } } From e4f0b539643f302f09fcaeac65bbdbe138a4fc40 Mon Sep 17 00:00:00 2001 From: slinkydeveloper Date: Mon, 14 Mar 2022 17:12:28 +0100 Subject: [PATCH 164/258] [FLINK-26467][table] Compile RowDataToStringConverter lazily This closes #19087. --- .../casting/RowDataToStringConverterImpl.java | 53 ++++++++++--------- 1 file changed, 27 insertions(+), 26 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java index 3698e7f553a12..fcbd429ebdfac 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java @@ -25,22 +25,23 @@ import org.apache.flink.table.data.utils.CastExecutor; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.table.utils.DateTimeUtils; import org.apache.flink.table.utils.print.PrintStyle; import org.apache.flink.table.utils.print.RowDataToStringConverter; import java.time.ZoneId; import java.util.List; -import java.util.Objects; import java.util.function.Function; -import static org.apache.flink.table.api.DataTypes.STRING; - /** {@link RowData} to {@link String} converter using {@link CastRule}. */ @Internal public final class RowDataToStringConverterImpl implements RowDataToStringConverter { - private final Function[] columnConverters; + private final DataType dataType; + private final CastRule.Context castRuleContext; + + private Function[] columnConverters; @VisibleForTesting public RowDataToStringConverterImpl(DataType dataType) { @@ -51,9 +52,14 @@ public RowDataToStringConverterImpl(DataType dataType) { false); } - @SuppressWarnings("unchecked") public RowDataToStringConverterImpl( DataType dataType, ZoneId zoneId, ClassLoader classLoader, boolean legacyBehaviour) { + this.dataType = dataType; + this.castRuleContext = CastRule.Context.create(legacyBehaviour, zoneId, classLoader); + } + + @SuppressWarnings("unchecked") + private void init() { List rowDataTypes = DataType.getFieldDataTypes(dataType); this.columnConverters = new Function[rowDataTypes.size()]; @@ -64,34 +70,29 @@ public RowDataToStringConverterImpl( CastExecutor castExecutor = (CastExecutor) CastRuleProvider.create( - CastRule.Context.create(legacyBehaviour, zoneId, classLoader), - fieldType, - STRING().getLogicalType()); + castRuleContext, fieldType, VarCharType.STRING_TYPE); if (castExecutor == null) { - // Fallback in case no casting rule is defined, for example for MULTISET and - // STRUCTURED - // Links to https://issues.apache.org/jira/browse/FLINK-24403 - this.columnConverters[index] = - row -> { - if (row.isNullAt(index)) { - return PrintStyle.NULL_VALUE; - } - return Objects.toString(getter.getFieldOrNull(row)); - }; - } else { - this.columnConverters[index] = - row -> { - if (row.isNullAt(index)) { - return PrintStyle.NULL_VALUE; - } - return castExecutor.cast(getter.getFieldOrNull(row)).toString(); - }; + throw new IllegalStateException( + "Cannot create a cast executor for converting " + + fieldType + + " to string. This is a bug, please open an issue."); } + this.columnConverters[index] = + row -> { + if (row.isNullAt(index)) { + return PrintStyle.NULL_VALUE; + } + return castExecutor.cast(getter.getFieldOrNull(row)).toString(); + }; } } @Override public String[] convert(RowData rowData) { + if (this.columnConverters == null) { + init(); + } + String[] result = new String[rowData.getArity()]; for (int i = 0; i < result.length; i++) { result[i] = this.columnConverters[i].apply(rowData); From c825d6b82a3b9c42fd88868f3334048e3a495db7 Mon Sep 17 00:00:00 2001 From: Timo Walther Date: Mon, 7 Mar 2022 12:52:24 +0100 Subject: [PATCH 165/258] [hotfix][table-common] Use correct naming convention for TRY_CAST --- .../flink/table/functions/BuiltInFunctionDefinitions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java index e947376891772..725d823a1bf90 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/functions/BuiltInFunctionDefinitions.java @@ -1720,7 +1720,7 @@ ANY, and(logical(LogicalTypeRoot.BOOLEAN), LITERAL) public static final BuiltInFunctionDefinition TRY_CAST = BuiltInFunctionDefinition.newBuilder() - .name("tryCast") + .name("TRY_CAST") .kind(SCALAR) .inputTypeStrategy(SpecificInputTypeStrategies.CAST) .outputTypeStrategy(forceNullable(TypeStrategies.argument(1))) From f35195737ce3228dbc7288c0786ff717f38e9d17 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas <5058131+matriv@users.noreply.github.com> Date: Fri, 8 Apr 2022 10:01:10 +0300 Subject: [PATCH 166/258] [FLINK-27111][table][docs] Update docs regarding TableEnvironment configuration Following the work on [FLINK-16835] update the docs accordingly to mention the use of `EnvironmentSettings` and that `TableEnvironement` related configuration can also be set in `flink-conf.yaml`. This closes #19401 (cherry picked from commit 7b7d96bf241ef9c97acb59c30fed70ed7b23f251) --- docs/content/docs/dev/table/config.md | 75 ++++++++++++++++++++------- 1 file changed, 55 insertions(+), 20 deletions(-) diff --git a/docs/content/docs/dev/table/config.md b/docs/content/docs/dev/table/config.md index ccdcc8777e8e3..6730703f1a996 100644 --- a/docs/content/docs/dev/table/config.md +++ b/docs/content/docs/dev/table/config.md @@ -35,7 +35,12 @@ that the required state size is capped (see [streaming concepts]({{< ref "docs/d ### Overview -In every table environment, the `TableConfig` offers options for configuring the current session. +When instantiating a `TableEnvironment`, `EnviromentSettings` can be used to pass the desired +configuration for the current session, by passing a `Configuration` object to the +`EnviromentSettings`. + +Additionally, in every table environment, the `TableConfig` offers options for configuring the +current session. For common or important configuration options, the `TableConfig` provides getters and setters methods with detailed inline documentation. @@ -51,40 +56,63 @@ table environment. {{< tab "Java" >}} ```java // instantiate table environment -TableEnvironment tEnv = ...; - -// access flink configuration -TableConfig configuration = tEnv.getConfig(); +Configuration configuration = new Configuration(); // set low-level key-value options -configuration.set("table.exec.mini-batch.enabled", "true"); -configuration.set("table.exec.mini-batch.allow-latency", "5 s"); -configuration.set("table.exec.mini-batch.size", "5000"); +configuration.setString("table.exec.mini-batch.enabled", "true"); +configuration.setString("table.exec.mini-batch.allow-latency", "5 s"); +configuration.setString("table.exec.mini-batch.size", "5000"); +EnvironmentSettings settings = EnvironmentSettings.newInstance() + .inStreamingMode().withConfiguration(configuration).build(); +TableEnvironment tEnv = TableEnvironment.create(settings); + +// access flink configuration after table environment instantiation +TableConfig tableConfig = tEnv.getConfig(); +// set low-level key-value options +tableConfig.set("table.exec.mini-batch.enabled", "true"); +tableConfig.set("table.exec.mini-batch.allow-latency", "5 s"); +tableConfig.set("table.exec.mini-batch.size", "5000"); ``` {{< /tab >}} {{< tab "Scala" >}} ```scala // instantiate table environment -val tEnv: TableEnvironment = ... - -// access flink configuration -val configuration = tEnv.getConfig() +val configuration = new Configuration; // set low-level key-value options -configuration.set("table.exec.mini-batch.enabled", "true") -configuration.set("table.exec.mini-batch.allow-latency", "5 s") -configuration.set("table.exec.mini-batch.size", "5000") +configuration.setString("table.exec.mini-batch.enabled", "true") +configuration.setString("table.exec.mini-batch.allow-latency", "5 s") +configuration.setString("table.exec.mini-batch.size", "5000") +val settings = EnvironmentSettings.newInstance + .inStreamingMode.withConfiguration(configuration).build +val tEnv: TableEnvironment = TableEnvironment.create(settings) + +// access flink configuration after table environment instantiation +val tableConfig = tEnv.getConfig() +// set low-level key-value options +tableConfig.set("table.exec.mini-batch.enabled", "true") +tableConfig.set("table.exec.mini-batch.allow-latency", "5 s") +tableConfig.set("table.exec.mini-batch.size", "5000") ``` {{< /tab >}} {{< tab "Python" >}} ```python # instantiate table environment -t_env = ... - -# access flink configuration -configuration = t_env.get_config() -# set low-level key-value options +configuration = Configuration() configuration.set("table.exec.mini-batch.enabled", "true") configuration.set("table.exec.mini-batch.allow-latency", "5 s") configuration.set("table.exec.mini-batch.size", "5000") +settings = EnvironmentSettings.new_instance() \ +... .in_streaming_mode() \ +... .with_configuration(configuration) \ +... .build() + +t_env = TableEnvironment.create(settings) + +# access flink configuration after table environment instantiation +table_config = t_env.get_config() +# set low-level key-value options +table_config.set("table.exec.mini-batch.enabled", "true") +table_config.set("table.exec.mini-batch.allow-latency", "5 s") +table_config.set("table.exec.mini-batch.size", "5000") ``` {{< /tab >}} {{< tab "SQL CLI" >}} @@ -96,6 +124,13 @@ Flink SQL> SET 'table.exec.mini-batch.size' = '5000'; {{< /tab >}} {{< /tabs >}} +{{< hint info >}} +**Note:** All of the following configuration options can also be set globally in +`conf/flink-conf.yaml` (see [configuration]({{< ref "docs/deployment/config" >}}) and can be later +on overridden in the application, through `EnvironmentSettings`, before instantiating +the `TableEnvironment`, or through the `TableConfig` of the `TableEnvironment`. +{{< /hint >}} + ### Execution Options The following options can be used to tune the performance of the query execution. From 6a352bd60420aad40c4b5e116a2a085bfda8b628 Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Fri, 8 Apr 2022 22:45:10 +0800 Subject: [PATCH 167/258] [hotfix][python] Cleanup the code to pass cachedFiles into extractPythonConfiguration --- .../org/apache/flink/python/util/PythonConfigUtil.java | 10 +++++++--- .../plan/nodes/exec/utils/CommonPythonUtil.java | 6 ++---- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java b/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java index 1da956fb04755..3c6085b725475 100644 --- a/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java +++ b/flink-python/src/main/java/org/apache/flink/python/util/PythonConfigUtil.java @@ -17,7 +17,9 @@ package org.apache.flink.python.util; +import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; @@ -79,7 +81,8 @@ public static Configuration getEnvironmentConfig(StreamExecutionEnvironment env) public static void configPythonOperator(StreamExecutionEnvironment env) throws IllegalAccessException, NoSuchFieldException { - final Configuration config = extractPythonConfiguration(env, env.getConfiguration()); + final Configuration config = + extractPythonConfiguration(env.getCachedFiles(), env.getConfiguration()); for (Transformation transformation : env.getTransformations()) { alignTransformation(transformation); @@ -99,9 +102,10 @@ public static void configPythonOperator(StreamExecutionEnvironment env) /** Extract the configurations which is used in the Python operators. */ public static Configuration extractPythonConfiguration( - StreamExecutionEnvironment env, ReadableConfig config) { + List> cachedFiles, + ReadableConfig config) { final Configuration pythonDependencyConfig = - PythonDependencyUtils.configurePythonDependencies(env.getCachedFiles(), config); + PythonDependencyUtils.configurePythonDependencies(cachedFiles, config); final PythonConfig pythonConfig = new PythonConfig(config, pythonDependencyConfig); return pythonConfig.toConfiguration(); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java index d9ac38b3725bf..a949ad2afa6c3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/utils/CommonPythonUtil.java @@ -114,10 +114,8 @@ public static Configuration extractPythonConfiguration( StreamExecutionEnvironment realEnv = getRealEnvironment(env); Method method = clazz.getDeclaredMethod( - "extractPythonConfiguration", - StreamExecutionEnvironment.class, - ReadableConfig.class); - return (Configuration) method.invoke(null, realEnv, tableConfig); + "extractPythonConfiguration", List.class, ReadableConfig.class); + return (Configuration) method.invoke(null, realEnv.getCachedFiles(), tableConfig); } catch (NoSuchFieldException | IllegalAccessException | NoSuchMethodException From 920e81b6478be1bb605d46bac800d4af675f8e79 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Thu, 7 Apr 2022 17:37:29 +0200 Subject: [PATCH 168/258] [FLINK-27118][yarn] TM ignores localhost BIND_HOST This closes #19395. --- flink-dist/src/main/resources/flink-conf.yaml | 4 ++++ .../flink/yarn/YarnClusterDescriptor.java | 22 +++++++++++++++++++ .../yarn/entrypoint/YarnEntrypointUtils.java | 4 ---- 3 files changed, 26 insertions(+), 4 deletions(-) diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml index e1ce1286c35bc..a8c0b7bb1a712 100644 --- a/flink-dist/src/main/resources/flink-conf.yaml +++ b/flink-dist/src/main/resources/flink-conf.yaml @@ -38,6 +38,8 @@ jobmanager.rpc.port: 6123 # The host interface the JobManager will bind to. My default, this is localhost, and will prevent # the JobManager from communicating outside the machine/container it is running on. +# On YARN this setting will be ignored if it is set to 'localhost', defaulting to 0.0.0.0. +# On Kubernetes this setting will be ignored, defaulting to 0.0.0.0. # # To enable this, set the bind-host address to one that has access to an outside facing network # interface, such as 0.0.0.0. @@ -53,6 +55,8 @@ jobmanager.memory.process.size: 1600m # The host interface the TaskManager will bind to. By default, this is localhost, and will prevent # the TaskManager from communicating outside the machine/container it is running on. +# On YARN this setting will be ignored if it is set to 'localhost', defaulting to 0.0.0.0. +# On Kubernetes this setting will be ignored, defaulting to 0.0.0.0. # # To enable this, set the bind-host address to one that has access to an outside facing network # interface, such as 0.0.0.0. diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java index e90bf65c5ebfc..92e94a12826a2 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -42,6 +42,7 @@ import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.SecurityOptions; +import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.core.plugin.PluginConfig; import org.apache.flink.core.plugin.PluginUtils; import org.apache.flink.runtime.clusterframework.BootstrapTools; @@ -1014,6 +1015,13 @@ private ApplicationReport startAppMaster( File tmpConfigurationFile = null; try { tmpConfigurationFile = File.createTempFile(appId + "-flink-conf.yaml", null); + + // remove localhost bind hosts as they render production clusters unusable + removeLocalhostBindHostSetting(configuration, JobManagerOptions.BIND_HOST); + removeLocalhostBindHostSetting(configuration, TaskManagerOptions.BIND_HOST); + // this setting is unconditionally overridden anyway, so we remove it for clarity + configuration.removeConfig(TaskManagerOptions.HOST); + BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); String flinkConfigKey = "flink-conf.yaml"; @@ -1282,6 +1290,20 @@ private ApplicationReport startAppMaster( return report; } + private void removeLocalhostBindHostSetting( + Configuration configuration, ConfigOption option) { + configuration + .getOptional(option) + .filter(bindHost -> bindHost.equals("localhost")) + .ifPresent( + bindHost -> { + LOG.info( + "Removing 'localhost' {} setting from effective configuration; using '0.0.0.0' instead.", + option); + configuration.removeConfig(option); + }); + } + /** * Returns the configured remote target home directory if set, otherwise returns the default * home directory. diff --git a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java index 986fd3785814c..2321355a390bc 100644 --- a/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java +++ b/flink-yarn/src/main/java/org/apache/flink/yarn/entrypoint/YarnEntrypointUtils.java @@ -25,7 +25,6 @@ import org.apache.flink.configuration.ResourceManagerOptions; import org.apache.flink.configuration.RestOptions; import org.apache.flink.configuration.SecurityOptions; -import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.configuration.WebOptions; import org.apache.flink.runtime.clusterframework.BootstrapTools; import org.apache.flink.util.Preconditions; @@ -65,9 +64,6 @@ public static Configuration loadConfiguration( ApplicationConstants.Environment.NM_HOST.key()); configuration.setString(JobManagerOptions.ADDRESS, hostname); - configuration.removeConfig(JobManagerOptions.BIND_HOST); - configuration.removeConfig(TaskManagerOptions.BIND_HOST); - configuration.removeConfig(TaskManagerOptions.HOST); configuration.setString(RestOptions.ADDRESS, hostname); configuration.setString(RestOptions.BIND_ADDRESS, hostname); From 5afff68c89c5e51a55f135109089c22d073b507f Mon Sep 17 00:00:00 2001 From: Roman Khachatryan Date: Fri, 1 Apr 2022 16:08:32 +0200 Subject: [PATCH 169/258] [FLINK-26985][runtime] Don't discard shared state of restored checkpoints Currently, in LEGACY restore mode, shared state of incremental checkpoints can be discarded regardless of whether they were created by this job or not. This invalidates the checkpoint from which the job was restored. The bug was introduced in FLINK-24611. Before that, reference count was maintained for each shared state entry; "initial" checkpoints did not decrement this count, preventing their shared state from being discarded. This change makes SharedStateRegistry to: 1. Remember the max checkpiont ID encountered during recovery 2. Associate each shared state entry with a checkpoint ID that created it 3. Only discard the entry if its createdByCheckpointID > highestRetainCheckpointID (1) is called from: - CheckpointCoordinator.restoreSavepoint - to cover initial restore from a checkpoint - SharedStateFactory, when building checkpoint store - to cover the failover case (see DefaultExecutionGraphFactory.createAndRestoreExecutionGraph) Adjusting only the CheckpointCoordinator path isn't sufficient: - job recovers from an existing checkpoints, adds it to the store - a new checkpoint is created - with the default restore settings - a failure happens, job recovers from a newer checkpoint - when a newer checkpoint is subsumed, its (inherited) shared state might be deleted --- .../KubernetesCheckpointRecoveryFactory.java | 7 +- .../kubernetes/utils/KubernetesUtils.java | 7 +- .../checkpoint/CheckpointCoordinator.java | 3 +- .../checkpoint/CheckpointRecoveryFactory.java | 5 +- .../checkpoint/CompletedCheckpoint.java | 7 +- .../EmbeddedCompletedCheckpointStore.java | 13 +- .../PerJobCheckpointRecoveryFactory.java | 12 +- .../StandaloneCheckpointRecoveryFactory.java | 9 +- .../StandaloneCompletedCheckpointStore.java | 17 +- .../ZooKeeperCheckpointRecoveryFactory.java | 7 +- .../CheckpointResourcesCleanupRunner.java | 7 +- ...beddedHaServicesWithLeadershipControl.java | 9 +- .../flink/runtime/jobgraph/RestoreMode.java | 2 + .../jobgraph/SavepointConfigOptions.java | 2 +- .../runtime/scheduler/SchedulerUtils.java | 14 +- .../runtime/state/SharedStateRegistry.java | 20 +- .../state/SharedStateRegistryFactory.java | 8 +- .../state/SharedStateRegistryImpl.java | 28 +- .../flink/runtime/util/ZooKeeperUtils.java | 8 +- .../CheckpointCoordinatorFailureTest.java | 3 +- .../CheckpointCoordinatorRestoringTest.java | 6 +- .../checkpoint/CheckpointCoordinatorTest.java | 266 ++++++++++-------- .../checkpoint/CompletedCheckpointTest.java | 5 +- .../DefaultCompletedCheckpointStoreTest.java | 5 +- .../PerJobCheckpointRecoveryTest.java | 13 +- .../TestingCheckpointRecoveryFactory.java | 4 +- ...oKeeperCompletedCheckpointStoreITCase.java | 4 +- ...ZooKeeperCompletedCheckpointStoreTest.java | 4 +- .../dispatcher/DispatcherCleanupITCase.java | 15 +- .../CheckpointResourcesCleanupRunnerTest.java | 4 +- .../runtime/scheduler/SchedulerUtilsTest.java | 13 +- .../runtime/testutils/CommonTestUtils.java | 34 ++- .../ResumeCheckpointManuallyITCase.java | 181 ++++++------ .../state/ChangelogCompatibilityITCase.java | 3 +- .../test/state/ChangelogRescalingITCase.java | 3 +- .../org/apache/flink/test/util/TestUtils.java | 9 + 36 files changed, 471 insertions(+), 286 deletions(-) diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java index ea78ecbbd2829..7150034bbb8a5 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java @@ -25,6 +25,7 @@ import org.apache.flink.runtime.checkpoint.CheckpointIDCounter; import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistryFactory; import javax.annotation.Nullable; @@ -81,7 +82,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobID, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception { final String configMapName = getConfigMapNameFunction.apply(jobID); KubernetesUtils.createConfigMapIfItDoesNotExist(kubeClient, configMapName, clusterId); @@ -94,7 +96,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( lockIdentity, maxNumberOfCheckpointsToRetain, sharedStateRegistryFactory, - ioExecutor); + ioExecutor, + restoreMode); } @Override diff --git a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java index e02d3a824e2f7..f8afbafde0aaa 100644 --- a/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java +++ b/flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java @@ -37,6 +37,7 @@ import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStoreUtils; import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmanager.DefaultJobGraphStore; import org.apache.flink.runtime.jobmanager.JobGraphStore; import org.apache.flink.runtime.jobmanager.NoOpJobGraphStoreWatcher; @@ -296,6 +297,7 @@ public static KubernetesStateHandleStore createJobGraphStateHandleStor * @param lockIdentity lock identity to check the leadership * @param maxNumberOfCheckpointsToRetain max number of checkpoints to retain on state store * handle + * @param restoreMode the mode in which the job is restoring * @return a {@link DefaultCompletedCheckpointStore} with {@link KubernetesStateHandleStore}. * @throws Exception when create the storage helper failed */ @@ -307,7 +309,8 @@ public static CompletedCheckpointStore createCompletedCheckpointStore( @Nullable String lockIdentity, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception { final RetrievableStateStorageHelper stateStorage = @@ -331,7 +334,7 @@ public static CompletedCheckpointStore createCompletedCheckpointStore( stateHandleStore, KubernetesCheckpointStoreUtil.INSTANCE, checkpoints, - sharedStateRegistryFactory.create(ioExecutor, checkpoints), + sharedStateRegistryFactory.create(ioExecutor, checkpoints, restoreMode), executor); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 2efc034ca90f8..72a6b7032deba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -1780,7 +1780,8 @@ public boolean restoreSavepoint( // register shared state - even before adding the checkpoint to the store // because the latter might trigger subsumption so the ref counts must be up-to-date savepoint.registerSharedStatesAfterRestored( - completedCheckpointStore.getSharedStateRegistry()); + completedCheckpointStore.getSharedStateRegistry(), + restoreSettings.getRestoreMode()); completedCheckpointStore.addCheckpointAndSubsumeOldestOne( savepoint, checkpointsCleaner, this::scheduleTriggerRequest); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java index ab0b5ef8506ed..64c68caa8a9d0 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointRecoveryFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryFactory; @@ -37,13 +38,15 @@ public interface CheckpointRecoveryFactory { * @param sharedStateRegistryFactory Simple factory to produce {@link SharedStateRegistry} * objects. * @param ioExecutor Executor used to run (async) deletes. + * @param restoreMode the restore mode with which the job is restoring. * @return {@link CompletedCheckpointStore} instance for the job */ CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception; /** diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java index 41c577452bb61..f0270ab661a7c 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CompletedCheckpoint.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.CompletedCheckpointStorageLocation; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.StateUtil; @@ -207,11 +208,13 @@ public long getStateSize() { * checkpoint is added into the store. * * @param sharedStateRegistry The registry where shared states are registered + * @param restoreMode the mode in which this checkpoint was restored from */ - public void registerSharedStatesAfterRestored(SharedStateRegistry sharedStateRegistry) { + public void registerSharedStatesAfterRestored( + SharedStateRegistry sharedStateRegistry, RestoreMode restoreMode) { // in claim mode we should not register any shared handles if (!props.isUnclaimed()) { - sharedStateRegistry.registerAll(operatorStates.values(), checkpointID); + sharedStateRegistry.registerAllAfterRestored(this, restoreMode); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/EmbeddedCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/EmbeddedCompletedCheckpointStore.java index 1e5e47c8855df..744083ca6afd9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/EmbeddedCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/EmbeddedCompletedCheckpointStore.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.util.Preconditions; import org.apache.flink.util.concurrent.Executors; @@ -53,16 +54,22 @@ public EmbeddedCompletedCheckpointStore() { @VisibleForTesting public EmbeddedCompletedCheckpointStore(int maxRetainedCheckpoints) { - this(maxRetainedCheckpoints, Collections.emptyList()); + this( + maxRetainedCheckpoints, + Collections.emptyList(), + /* Using the default restore mode in tests to detect any breaking changes early. */ + RestoreMode.DEFAULT); } public EmbeddedCompletedCheckpointStore( - int maxRetainedCheckpoints, Collection initialCheckpoints) { + int maxRetainedCheckpoints, + Collection initialCheckpoints, + RestoreMode restoreMode) { this( maxRetainedCheckpoints, initialCheckpoints, SharedStateRegistry.DEFAULT_FACTORY.create( - Executors.directExecutor(), initialCheckpoints)); + Executors.directExecutor(), initialCheckpoints, restoreMode)); } public EmbeddedCompletedCheckpointStore( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java index bc18c453969d1..7a70f5624a5b5 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryFactory.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistryFactory; import javax.annotation.Nullable; @@ -42,7 +43,7 @@ public class PerJobCheckpointRecoveryFactory public static CheckpointRecoveryFactory withoutCheckpointStoreRecovery(IntFunction storeFn) { return new PerJobCheckpointRecoveryFactory<>( - (maxCheckpoints, previous, sharedStateRegistry, ioExecutor) -> { + (maxCheckpoints, previous, sharedStateRegistry, ioExecutor, restoreMode) -> { if (previous != null) { throw new UnsupportedOperationException( "Checkpoint store recovery is not supported."); @@ -75,7 +76,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) { + Executor ioExecutor, + RestoreMode restoreMode) { return store.compute( jobId, (key, previous) -> @@ -83,7 +85,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( maxNumberOfCheckpointsToRetain, previous, sharedStateRegistryFactory, - ioExecutor)); + ioExecutor, + restoreMode)); } @Override @@ -98,6 +101,7 @@ StoreType recoverCheckpointStore( int maxNumberOfCheckpointsToRetain, @Nullable StoreType previousStore, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor); + Executor ioExecutor, + RestoreMode restoreMode); } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java index 95f9da72406ff..abcb704ad7c0a 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCheckpointRecoveryFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.state.SharedStateRegistryFactory; @@ -32,11 +33,15 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception { return new StandaloneCompletedCheckpointStore( - maxNumberOfCheckpointsToRetain, sharedStateRegistryFactory, ioExecutor); + maxNumberOfCheckpointsToRetain, + sharedStateRegistryFactory, + ioExecutor, + restoreMode); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java index 87a6486a911f6..6c89dcc712788 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/StandaloneCompletedCheckpointStore.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryFactory; @@ -56,32 +57,38 @@ public StandaloneCompletedCheckpointStore(int maxNumberOfCheckpointsToRetain) { this( maxNumberOfCheckpointsToRetain, SharedStateRegistry.DEFAULT_FACTORY, - Executors.directExecutor()); + Executors.directExecutor(), + /* Using the default restore mode in tests to detect any breaking changes early. */ + RestoreMode.DEFAULT); } /** * Creates {@link StandaloneCompletedCheckpointStore}. * + * @param restoreMode * @param maxNumberOfCheckpointsToRetain The maximum number of checkpoints to retain (at least * 1). Adding more checkpoints than this results in older checkpoints being discarded. */ public StandaloneCompletedCheckpointStore( int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) { + Executor ioExecutor, + RestoreMode restoreMode) { this( maxNumberOfCheckpointsToRetain, sharedStateRegistryFactory, new ArrayDeque<>(maxNumberOfCheckpointsToRetain + 1), - ioExecutor); + ioExecutor, + restoreMode); } private StandaloneCompletedCheckpointStore( int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, ArrayDeque checkpoints, - Executor ioExecutor) { - super(sharedStateRegistryFactory.create(ioExecutor, checkpoints)); + Executor ioExecutor, + RestoreMode restoreMode) { + super(sharedStateRegistryFactory.create(ioExecutor, checkpoints, restoreMode)); checkArgument(maxNumberOfCheckpointsToRetain >= 1, "Must retain at least one checkpoint."); this.maxNumberOfCheckpointsToRetain = maxNumberOfCheckpointsToRetain; this.checkpoints = checkpoints; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java index 052b3405235ae..c522296cf89e9 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/ZooKeeperCheckpointRecoveryFactory.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.state.SharedStateRegistryFactory; import org.apache.flink.runtime.util.ZooKeeperUtils; @@ -51,7 +52,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception { return ZooKeeperUtils.createCompletedCheckpoints( @@ -61,7 +63,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( maxNumberOfCheckpointsToRetain, sharedStateRegistryFactory, ioExecutor, - executor); + executor, + restoreMode); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java index c97f8644a1f6f..e657c8d740012 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunner.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.dispatcher.JobCancellationFailedException; import org.apache.flink.runtime.dispatcher.UnavailableDispatcherOperationException; import org.apache.flink.runtime.executiongraph.ArchivedExecutionGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmaster.JobManagerRunner; import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; import org.apache.flink.runtime.jobmaster.JobMaster; @@ -145,7 +146,11 @@ private CompletedCheckpointStore createCompletedCheckpointStore() throws Excepti DefaultCompletedCheckpointStoreUtils.getMaximumNumberOfRetainedCheckpoints( jobManagerConfiguration, LOG), sharedStateRegistryFactory, - cleanupExecutor); + cleanupExecutor, + // Using RestoreMode.CLAIM to be able to discard shared state, if any. + // Note that it also means that the original shared state might be discarded as well + // because the initial checkpoint might be subsumed. + RestoreMode.CLAIM); } private CheckpointIDCounter createCheckpointIDCounter() throws Exception { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesWithLeadershipControl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesWithLeadershipControl.java index ce59d0d99e926..b9859ba0f2718 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesWithLeadershipControl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/highavailability/nonha/embedded/EmbeddedHaServicesWithLeadershipControl.java @@ -40,13 +40,18 @@ public EmbeddedHaServicesWithLeadershipControl(Executor executor) { this( executor, new PerJobCheckpointRecoveryFactory( - (maxCheckpoints, previous, stateRegistryFactory, ioExecutor) -> { + (maxCheckpoints, + previous, + stateRegistryFactory, + ioExecutor, + restoreMode) -> { List checkpoints = previous != null ? previous.getAllCheckpoints() : Collections.emptyList(); SharedStateRegistry stateRegistry = - stateRegistryFactory.create(ioExecutor, checkpoints); + stateRegistryFactory.create( + ioExecutor, checkpoints, restoreMode); if (previous != null) { if (!previous.getShutdownStatus().isPresent()) { throw new IllegalStateException( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/RestoreMode.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/RestoreMode.java index 10a4f2ac60d1e..da6c325265a0d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/RestoreMode.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/RestoreMode.java @@ -53,4 +53,6 @@ public enum RestoreMode implements DescribedEnum { public InlineElement getDescription() { return text(description); } + + public static final RestoreMode DEFAULT = NO_CLAIM; } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java index e8a9dd86d1c70..a38e05c5f3f5e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/jobgraph/SavepointConfigOptions.java @@ -52,7 +52,7 @@ public class SavepointConfigOptions { public static final ConfigOption RESTORE_MODE = key("execution.savepoint-restore-mode") .enumType(RestoreMode.class) - .defaultValue(RestoreMode.NO_CLAIM) + .defaultValue(RestoreMode.DEFAULT) .withDescription( "Describes the mode how Flink should restore from the given" + " savepoint or retained checkpoint."); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java index 6b14801d8ebe5..87f2e56c6ba15 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java @@ -30,6 +30,7 @@ import org.apache.flink.runtime.client.JobExecutionException; import org.apache.flink.runtime.executiongraph.DefaultExecutionGraphBuilder; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.slf4j.Logger; @@ -55,7 +56,12 @@ public static CompletedCheckpointStore createCompletedCheckpointStoreIfCheckpoin if (DefaultExecutionGraphBuilder.isCheckpointingEnabled(jobGraph)) { try { return createCompletedCheckpointStore( - configuration, checkpointRecoveryFactory, ioExecutor, log, jobId); + configuration, + checkpointRecoveryFactory, + ioExecutor, + log, + jobId, + jobGraph.getSavepointRestoreSettings().getRestoreMode()); } catch (Exception e) { throw new JobExecutionException( jobId, @@ -73,14 +79,16 @@ static CompletedCheckpointStore createCompletedCheckpointStore( CheckpointRecoveryFactory recoveryFactory, Executor ioExecutor, Logger log, - JobID jobId) + JobID jobId, + RestoreMode restoreMode) throws Exception { return recoveryFactory.createRecoveredCompletedCheckpointStore( jobId, DefaultCompletedCheckpointStoreUtils.getMaximumNumberOfRetainedCheckpoints( jobManagerConfig, log), SharedStateRegistry.DEFAULT_FACTORY, - ioExecutor); + ioExecutor, + restoreMode); } public static CheckpointIDCounter createCheckpointIDCounterIfCheckpointingIsEnabled( diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java index 7172bec4c243c..b816f09e767ae 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistry.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.state; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.jobgraph.RestoreMode; /** * This registry manages state that is shared across (incremental) checkpoints, and is responsible @@ -32,11 +33,11 @@ public interface SharedStateRegistry extends AutoCloseable { /** A singleton object for the default implementation of a {@link SharedStateRegistryFactory} */ SharedStateRegistryFactory DEFAULT_FACTORY = - (deleteExecutor, checkpoints) -> { + (deleteExecutor, checkpoints, restoreMode) -> { SharedStateRegistry sharedStateRegistry = new SharedStateRegistryImpl(deleteExecutor); for (CompletedCheckpoint checkpoint : checkpoints) { - checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry); + checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry, restoreMode); } return sharedStateRegistry; }; @@ -66,10 +67,25 @@ StreamStateHandle registerReference( /** * Register given shared states in the registry. * + *

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

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

    This should hold for both cases: when recovering from that initial checkpoint; and from + * any subsequent checkpoint derived from it. + */ + void registerAllAfterRestored(CompletedCheckpoint checkpoint, RestoreMode mode); + void checkpointCompleted(long checkpointId); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java index bbdd2fd095944..bc8118cce4286 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryFactory.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.state; import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.jobgraph.RestoreMode; import java.util.Collection; import java.util.concurrent.Executor; @@ -29,10 +30,13 @@ public interface SharedStateRegistryFactory { /** * Factory method for {@link SharedStateRegistry}. * - * @param checkpoints whose shared state will be registered. * @param deleteExecutor executor used to run (async) deletes. + * @param checkpoints whose shared state will be registered. + * @param restoreMode the mode in which the given checkpoints were restored * @return a SharedStateRegistry object */ SharedStateRegistry create( - Executor deleteExecutor, Collection checkpoints); + Executor deleteExecutor, + Collection checkpoints, + RestoreMode restoreMode); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java index b87d864608646..4dce645527722 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/SharedStateRegistryImpl.java @@ -19,6 +19,8 @@ package org.apache.flink.runtime.state; import org.apache.flink.annotation.Internal; +import org.apache.flink.runtime.checkpoint.CompletedCheckpoint; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.util.concurrent.Executors; import org.slf4j.Logger; @@ -51,6 +53,9 @@ public class SharedStateRegistryImpl implements SharedStateRegistry { /** Executor for async state deletion */ private final Executor asyncDisposalExecutor; + /** Checkpoint ID below which no state is discarded, inclusive. */ + private long highestNotClaimedCheckpointID = -1L; + /** Default uses direct executor to delete unreferenced state */ public SharedStateRegistryImpl() { this(Executors.directExecutor()); @@ -147,7 +152,9 @@ public void unregisterUnusedState(long lowestCheckpointID) { while (it.hasNext()) { SharedStateEntry entry = it.next(); if (entry.lastUsedCheckpointID < lowestCheckpointID) { - subsumed.add(entry.stateHandle); + if (entry.createdByCheckpointID > highestNotClaimedCheckpointID) { + subsumed.add(entry.stateHandle); + } it.remove(); } } @@ -174,6 +181,20 @@ public void registerAll( } } + @Override + public void registerAllAfterRestored(CompletedCheckpoint checkpoint, RestoreMode mode) { + registerAll(checkpoint.getOperatorStates().values(), checkpoint.getCheckpointID()); + // In NO_CLAIM and LEGACY restore modes, shared state of the initial checkpoints must be + // preserved. This is achieved by advancing highestRetainCheckpointID here, and then + // checking entry.createdByCheckpointID against it on checkpoint subsumption. + // In CLAIM restore mode, the shared state of the initial checkpoints must be + // discarded as soon as it becomes unused - so highestRetainCheckpointID is not updated. + if (mode != RestoreMode.CLAIM) { + highestNotClaimedCheckpointID = + Math.max(highestNotClaimedCheckpointID, checkpoint.getCheckpointID()); + } + } + @Override public void checkpointCompleted(long checkpointId) { for (SharedStateEntry entry : registeredStates.values()) { @@ -251,6 +272,8 @@ private static final class SharedStateEntry { /** The shared state handle */ StreamStateHandle stateHandle; + private final long createdByCheckpointID; + private long lastUsedCheckpointID; /** Whether this entry is included into a confirmed checkpoint. */ @@ -258,6 +281,7 @@ private static final class SharedStateEntry { SharedStateEntry(StreamStateHandle value, long checkpointID) { this.stateHandle = value; + this.createdByCheckpointID = checkpointID; this.lastUsedCheckpointID = checkpointID; } @@ -266,6 +290,8 @@ public String toString() { return "SharedStateEntry{" + "stateHandle=" + stateHandle + + ", createdByCheckpointID=" + + createdByCheckpointID + ", lastUsedCheckpointID=" + lastUsedCheckpointID + '}'; diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java index c62b0c3bde066..2de67d006d27e 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/util/ZooKeeperUtils.java @@ -34,6 +34,7 @@ import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils; import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmanager.DefaultJobGraphStore; import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; import org.apache.flink.runtime.jobmanager.JobGraphStore; @@ -569,6 +570,7 @@ public static JobGraphStore createJobGraphs( * @param configuration {@link Configuration} object * @param maxNumberOfCheckpointsToRetain The maximum number of checkpoints to retain * @param executor to run ZooKeeper callbacks + * @param restoreMode the mode in which the job is being restored * @return {@link DefaultCompletedCheckpointStore} instance * @throws Exception if the completed checkpoint store cannot be created */ @@ -578,7 +580,8 @@ public static CompletedCheckpointStore createCompletedCheckpoints( int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, Executor ioExecutor, - Executor executor) + Executor executor, + RestoreMode restoreMode) throws Exception { checkNotNull(configuration, "Configuration"); @@ -597,7 +600,8 @@ public static CompletedCheckpointStore createCompletedCheckpoints( completedCheckpointStateHandleStore, ZooKeeperCheckpointStoreUtil.INSTANCE, completedCheckpoints, - sharedStateRegistryFactory.create(ioExecutor, completedCheckpoints), + sharedStateRegistryFactory.create( + ioExecutor, completedCheckpoints, restoreMode), executor); LOG.info( "Initialized {} in '{}' with {}.", diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java index 36745153959c2..f242f92e38742 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; import org.apache.flink.runtime.persistence.PossibleInconsistentStateException; import org.apache.flink.runtime.state.InputChannelStateHandle; @@ -280,7 +281,7 @@ private static final class FailingCompletedCheckpointStore public FailingCompletedCheckpointStore(Exception addCheckpointFailure) { super( SharedStateRegistry.DEFAULT_FACTORY.create( - Executors.directExecutor(), emptyList())); + Executors.directExecutor(), emptyList(), RestoreMode.DEFAULT)); this.addCheckpointFailure = addCheckpointFailure; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java index 3b839739663fc..3e9fb12d51a71 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorRestoringTest.java @@ -28,6 +28,7 @@ import org.apache.flink.runtime.executiongraph.ExecutionJobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration.CheckpointCoordinatorConfigurationBuilder; import org.apache.flink.runtime.messages.checkpoint.AcknowledgeCheckpoint; @@ -234,7 +235,7 @@ private void testRestoreLatestCheckpointedState( final ExecutionGraph executionGraph = createExecutionGraph(vertices); final EmbeddedCompletedCheckpointStore store = new EmbeddedCompletedCheckpointStore( - completedCheckpoints.size(), completedCheckpoints); + completedCheckpoints.size(), completedCheckpoints, RestoreMode.DEFAULT); // set up the coordinator and validate the initial state final CheckpointCoordinator coordinator = @@ -778,7 +779,8 @@ public void testStateRecoveryWithTopologyChange(TestScaleType scaleType) throws // set up the coordinator and validate the initial state SharedStateRegistry sharedStateRegistry = - SharedStateRegistry.DEFAULT_FACTORY.create(Executors.directExecutor(), emptyList()); + SharedStateRegistry.DEFAULT_FACTORY.create( + Executors.directExecutor(), emptyList(), RestoreMode.DEFAULT); CheckpointCoordinator coord = new CheckpointCoordinatorBuilder() .setExecutionGraph(newGraph) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 30af82fbe7d59..9d3a87ddd848f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -38,6 +38,7 @@ import org.apache.flink.runtime.executiongraph.utils.SimpleAckingTaskManagerGateway; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobmaster.LogicalSlot; import org.apache.flink.runtime.jobmaster.TestingLogicalSlotBuilder; @@ -2867,163 +2868,178 @@ public void testCheckpointStatsTrackerRestoreCallback() throws Exception { @Test public void testSharedStateRegistrationOnRestore() throws Exception { - JobVertexID jobVertexID1 = new JobVertexID(); + for (RestoreMode restoreMode : RestoreMode.values()) { + JobVertexID jobVertexID1 = new JobVertexID(); - int parallelism1 = 2; - int maxParallelism1 = 4; + int parallelism1 = 2; + int maxParallelism1 = 4; - ExecutionGraph graph = - new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() - .addJobVertex(jobVertexID1, parallelism1, maxParallelism1) - .build(); + ExecutionGraph graph = + new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() + .addJobVertex(jobVertexID1, parallelism1, maxParallelism1) + .build(); - ExecutionJobVertex jobVertex1 = graph.getJobVertex(jobVertexID1); + ExecutionJobVertex jobVertex1 = graph.getJobVertex(jobVertexID1); - List checkpoints = Collections.emptyList(); - SharedStateRegistry firstInstance = - SharedStateRegistry.DEFAULT_FACTORY.create( - org.apache.flink.util.concurrent.Executors.directExecutor(), checkpoints); - final EmbeddedCompletedCheckpointStore store = - new EmbeddedCompletedCheckpointStore(10, checkpoints, firstInstance); + List checkpoints = Collections.emptyList(); + SharedStateRegistry firstInstance = + SharedStateRegistry.DEFAULT_FACTORY.create( + org.apache.flink.util.concurrent.Executors.directExecutor(), + checkpoints, + restoreMode); + final EmbeddedCompletedCheckpointStore store = + new EmbeddedCompletedCheckpointStore(10, checkpoints, firstInstance); - // set up the coordinator and validate the initial state - final CheckpointCoordinatorBuilder coordinatorBuilder = - new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) - .setTimer(manuallyTriggeredScheduledExecutor); - final CheckpointCoordinator coordinator = - coordinatorBuilder.setCompletedCheckpointStore(store).build(); + // set up the coordinator and validate the initial state + final CheckpointCoordinatorBuilder coordinatorBuilder = + new CheckpointCoordinatorBuilder() + .setExecutionGraph(graph) + .setTimer(manuallyTriggeredScheduledExecutor); + final CheckpointCoordinator coordinator = + coordinatorBuilder.setCompletedCheckpointStore(store).build(); - final int numCheckpoints = 3; + final int numCheckpoints = 3; - List keyGroupPartitions1 = - StateAssignmentOperation.createKeyGroupPartitions(maxParallelism1, parallelism1); + List keyGroupPartitions1 = + StateAssignmentOperation.createKeyGroupPartitions( + maxParallelism1, parallelism1); - for (int i = 0; i < numCheckpoints; ++i) { - performIncrementalCheckpoint( - graph.getJobID(), coordinator, jobVertex1, keyGroupPartitions1, i); - } + for (int i = 0; i < numCheckpoints; ++i) { + performIncrementalCheckpoint( + graph.getJobID(), coordinator, jobVertex1, keyGroupPartitions1, i); + } - List completedCheckpoints = coordinator.getSuccessfulCheckpoints(); - assertEquals(numCheckpoints, completedCheckpoints.size()); + List completedCheckpoints = coordinator.getSuccessfulCheckpoints(); + assertEquals(numCheckpoints, completedCheckpoints.size()); - int sharedHandleCount = 0; + int sharedHandleCount = 0; - List> sharedHandlesByCheckpoint = - new ArrayList<>(numCheckpoints); + List> sharedHandlesByCheckpoint = + new ArrayList<>(numCheckpoints); - for (int i = 0; i < numCheckpoints; ++i) { - sharedHandlesByCheckpoint.add(new HashMap<>(2)); - } + for (int i = 0; i < numCheckpoints; ++i) { + sharedHandlesByCheckpoint.add(new HashMap<>(2)); + } - int cp = 0; - for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { - for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { - for (OperatorSubtaskState subtaskState : taskState.getStates()) { - for (KeyedStateHandle keyedStateHandle : subtaskState.getManagedKeyedState()) { - // test we are once registered with the current registry - verify(keyedStateHandle, times(1)) - .registerSharedStates( - firstInstance, completedCheckpoint.getCheckpointID()); - IncrementalRemoteKeyedStateHandle incrementalKeyedStateHandle = - (IncrementalRemoteKeyedStateHandle) keyedStateHandle; - - sharedHandlesByCheckpoint - .get(cp) - .putAll(incrementalKeyedStateHandle.getSharedState()); - - for (StreamStateHandle streamStateHandle : - incrementalKeyedStateHandle.getSharedState().values()) { - assertTrue( - !(streamStateHandle instanceof PlaceholderStreamStateHandle)); - verify(streamStateHandle, never()).discardState(); - ++sharedHandleCount; - } + int cp = 0; + for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { + for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { + for (OperatorSubtaskState subtaskState : taskState.getStates()) { + for (KeyedStateHandle keyedStateHandle : + subtaskState.getManagedKeyedState()) { + // test we are once registered with the current registry + verify(keyedStateHandle, times(1)) + .registerSharedStates( + firstInstance, completedCheckpoint.getCheckpointID()); + IncrementalRemoteKeyedStateHandle incrementalKeyedStateHandle = + (IncrementalRemoteKeyedStateHandle) keyedStateHandle; + + sharedHandlesByCheckpoint + .get(cp) + .putAll(incrementalKeyedStateHandle.getSharedState()); + + for (StreamStateHandle streamStateHandle : + incrementalKeyedStateHandle.getSharedState().values()) { + assertTrue( + !(streamStateHandle + instanceof PlaceholderStreamStateHandle)); + verify(streamStateHandle, never()).discardState(); + ++sharedHandleCount; + } + + for (StreamStateHandle streamStateHandle : + incrementalKeyedStateHandle.getPrivateState().values()) { + verify(streamStateHandle, never()).discardState(); + } - for (StreamStateHandle streamStateHandle : - incrementalKeyedStateHandle.getPrivateState().values()) { - verify(streamStateHandle, never()).discardState(); + verify(incrementalKeyedStateHandle.getMetaStateHandle(), never()) + .discardState(); } - verify(incrementalKeyedStateHandle.getMetaStateHandle(), never()) - .discardState(); + verify(subtaskState, never()).discardState(); } - - verify(subtaskState, never()).discardState(); } + ++cp; } - ++cp; - } - // 2 (parallelism) x (1 (CP0) + 2 (CP1) + 2 (CP2)) = 10 - assertEquals(10, sharedHandleCount); + // 2 (parallelism) x (1 (CP0) + 2 (CP1) + 2 (CP2)) = 10 + assertEquals(10, sharedHandleCount); - // discard CP0 - store.removeOldestCheckpoint(); + // discard CP0 + store.removeOldestCheckpoint(); - // we expect no shared state was discarded because the state of CP0 is still referenced by - // CP1 - for (Map cpList : sharedHandlesByCheckpoint) { - for (StreamStateHandle streamStateHandle : cpList.values()) { - verify(streamStateHandle, never()).discardState(); + // we expect no shared state was discarded because the state of CP0 is still referenced + // by + // CP1 + for (Map cpList : sharedHandlesByCheckpoint) { + for (StreamStateHandle streamStateHandle : cpList.values()) { + verify(streamStateHandle, never()).discardState(); + } } - } - // shutdown the store - store.shutdown(JobStatus.SUSPENDED, new CheckpointsCleaner()); - - // restore the store - Set tasks = new HashSet<>(); - tasks.add(jobVertex1); - - assertEquals(JobStatus.SUSPENDED, store.getShutdownStatus().orElse(null)); - SharedStateRegistry secondInstance = - SharedStateRegistry.DEFAULT_FACTORY.create( - org.apache.flink.util.concurrent.Executors.directExecutor(), - store.getAllCheckpoints()); - final EmbeddedCompletedCheckpointStore secondStore = - new EmbeddedCompletedCheckpointStore(10, store.getAllCheckpoints(), secondInstance); - final CheckpointCoordinator secondCoordinator = - coordinatorBuilder.setCompletedCheckpointStore(secondStore).build(); - assertTrue(secondCoordinator.restoreLatestCheckpointedStateToAll(tasks, false)); - - // validate that all shared states are registered again after the recovery. - cp = 0; - for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { - for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { - for (OperatorSubtaskState subtaskState : taskState.getStates()) { - for (KeyedStateHandle keyedStateHandle : subtaskState.getManagedKeyedState()) { - VerificationMode verificationMode; - // test we are once registered with the new registry - if (cp > 0) { - verificationMode = times(1); - } else { - verificationMode = never(); - } + // shutdown the store + store.shutdown(JobStatus.SUSPENDED, new CheckpointsCleaner()); + + // restore the store + Set tasks = new HashSet<>(); + tasks.add(jobVertex1); + + assertEquals(JobStatus.SUSPENDED, store.getShutdownStatus().orElse(null)); + SharedStateRegistry secondInstance = + SharedStateRegistry.DEFAULT_FACTORY.create( + org.apache.flink.util.concurrent.Executors.directExecutor(), + store.getAllCheckpoints(), + restoreMode); + final EmbeddedCompletedCheckpointStore secondStore = + new EmbeddedCompletedCheckpointStore( + 10, store.getAllCheckpoints(), secondInstance); + final CheckpointCoordinator secondCoordinator = + coordinatorBuilder.setCompletedCheckpointStore(secondStore).build(); + assertTrue(secondCoordinator.restoreLatestCheckpointedStateToAll(tasks, false)); + + // validate that all shared states are registered again after the recovery. + cp = 0; + for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) { + for (OperatorState taskState : completedCheckpoint.getOperatorStates().values()) { + for (OperatorSubtaskState subtaskState : taskState.getStates()) { + for (KeyedStateHandle keyedStateHandle : + subtaskState.getManagedKeyedState()) { + VerificationMode verificationMode; + // test we are once registered with the new registry + if (cp > 0) { + verificationMode = times(1); + } else { + verificationMode = never(); + } - // check that all are registered with the new registry - verify(keyedStateHandle, verificationMode) - .registerSharedStates( - secondInstance, completedCheckpoint.getCheckpointID()); + // check that all are registered with the new registry + verify(keyedStateHandle, verificationMode) + .registerSharedStates( + secondInstance, completedCheckpoint.getCheckpointID()); + } } } + ++cp; } - ++cp; - } - // discard CP1 - secondStore.removeOldestCheckpoint(); + // discard CP1 + secondStore.removeOldestCheckpoint(); - // we expect that all shared state from CP0 is no longer referenced and discarded. CP2 is - // still live and also - // references the state from CP1, so we expect they are not discarded. - verifyDiscard(sharedHandlesByCheckpoint, cpId -> cpId == 0 ? times(1) : never()); + // we expect that all shared state from CP0 is no longer referenced and discarded. CP2 + // is + // still live and also + // references the state from CP1, so we expect they are not discarded. + verifyDiscard( + sharedHandlesByCheckpoint, + cpId -> restoreMode == RestoreMode.CLAIM && cpId == 0 ? times(1) : never()); - // discard CP2 - secondStore.removeOldestCheckpoint(); + // discard CP2 + secondStore.removeOldestCheckpoint(); - // still expect shared state not to be discarded because it may be used in later checkpoints - verifyDiscard(sharedHandlesByCheckpoint, cpId -> cpId == 1 ? never() : atLeast(0)); + // still expect shared state not to be discarded because it may be used in later + // checkpoints + verifyDiscard(sharedHandlesByCheckpoint, cpId -> cpId == 1 ? never() : atLeast(0)); + } } @Test diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java index b6461a9c98462..a0b67cd0dfa29 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CompletedCheckpointTest.java @@ -23,6 +23,7 @@ import org.apache.flink.core.testutils.CommonTestUtils; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryImpl; import org.apache.flink.runtime.state.testutils.EmptyStreamStateHandle; @@ -235,7 +236,7 @@ public void testRegisterStatesAtRegistry() { null); SharedStateRegistry sharedStateRegistry = new SharedStateRegistryImpl(); - checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry); + checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry, RestoreMode.DEFAULT); verify(state, times(1)).registerSharedStates(sharedStateRegistry, 0L); } @@ -267,7 +268,7 @@ public void testCleanUpOnSubsume() throws Exception { null); SharedStateRegistry sharedStateRegistry = new SharedStateRegistryImpl(); - checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry); + checkpoint.registerSharedStatesAfterRestored(sharedStateRegistry, RestoreMode.DEFAULT); verify(state, times(1)).registerSharedStates(sharedStateRegistry, 0L); // Subsume diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java index 064e26a9b9fb6..22cd5dd021cd4 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/DefaultCompletedCheckpointStoreTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.core.testutils.FlinkMatchers; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.persistence.StateHandleStore; import org.apache.flink.runtime.persistence.TestingRetrievableStateStorageHelper; import org.apache.flink.runtime.persistence.TestingStateHandleStore; @@ -397,7 +398,9 @@ public long nameToCheckpointID(String name) { DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( stateHandleStore, checkpointStoreUtil), SharedStateRegistry.DEFAULT_FACTORY.create( - org.apache.flink.util.concurrent.Executors.directExecutor(), emptyList()), + org.apache.flink.util.concurrent.Executors.directExecutor(), + emptyList(), + RestoreMode.DEFAULT), executorService); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java index 58a0d954318dd..7203edfc7099a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PerJobCheckpointRecoveryTest.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.util.TestLogger; import org.apache.flink.util.concurrent.Executors; @@ -49,7 +50,8 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { firstJobId, 1, SharedStateRegistry.DEFAULT_FACTORY, - Executors.directExecutor())); + Executors.directExecutor(), + RestoreMode.DEFAULT)); assertThrows( UnsupportedOperationException.class, () -> @@ -57,7 +59,8 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { firstJobId, 1, SharedStateRegistry.DEFAULT_FACTORY, - Executors.directExecutor())); + Executors.directExecutor(), + RestoreMode.DEFAULT)); final JobID secondJobId = new JobID(); assertSame( @@ -66,7 +69,8 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { secondJobId, 1, SharedStateRegistry.DEFAULT_FACTORY, - Executors.directExecutor())); + Executors.directExecutor(), + RestoreMode.DEFAULT)); assertThrows( UnsupportedOperationException.class, () -> @@ -74,6 +78,7 @@ public void testFactoryWithoutCheckpointStoreRecovery() throws Exception { secondJobId, 1, SharedStateRegistry.DEFAULT_FACTORY, - Executors.directExecutor())); + Executors.directExecutor(), + RestoreMode.DEFAULT)); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java index e164543d0ab81..687196dff827c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/TestingCheckpointRecoveryFactory.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.checkpoint; import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.SharedStateRegistryFactory; import java.util.concurrent.Executor; @@ -39,7 +40,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) { + Executor ioExecutor, + RestoreMode restoreMode) { return store; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java index 240955927bbce..5daad82414dab 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.time.Deadline; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.RetrievableStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; import org.apache.flink.runtime.state.SharedStateRegistryImpl; @@ -92,7 +93,8 @@ protected CompletedCheckpointStore createRecoveredCompletedCheckpointStore( checkpointStoreUtil, DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints( checkpointsInZooKeeper, checkpointStoreUtil), - SharedStateRegistry.DEFAULT_FACTORY.create(Executors.directExecutor(), emptyList()), + SharedStateRegistry.DEFAULT_FACTORY.create( + Executors.directExecutor(), emptyList(), RestoreMode.DEFAULT), executor); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java index 804d285ae28dd..cd8561157cb70 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreTest.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.operators.testutils.ExpectedTestException; import org.apache.flink.runtime.rest.util.NoOpFatalErrorHandler; import org.apache.flink.runtime.state.RetrievableStateHandle; @@ -196,7 +197,8 @@ private CompletedCheckpointStore createZooKeeperCheckpointStore(CuratorFramework checkpointsInZooKeeper, zooKeeperCheckpointStoreUtil, Collections.emptyList(), - SharedStateRegistry.DEFAULT_FACTORY.create(Executors.directExecutor(), emptyList()), + SharedStateRegistry.DEFAULT_FACTORY.create( + Executors.directExecutor(), emptyList(), RestoreMode.DEFAULT), Executors.directExecutor()); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java index 4b4ce27433325..e7b9c57db97d8 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java @@ -35,6 +35,7 @@ import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobGraphBuilder; import org.apache.flink.runtime.jobgraph.JobVertex; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; import org.apache.flink.runtime.jobmanager.JobGraphStore; @@ -87,7 +88,11 @@ public void setUp() throws Exception { super.setUp(); haServices.setCheckpointRecoveryFactory( new PerJobCheckpointRecoveryFactory( - (maxCheckpoints, previous, sharedStateRegistryFactory, ioExecutor) -> { + (maxCheckpoints, + previous, + sharedStateRegistryFactory, + ioExecutor, + restoreMode) -> { if (previous != null) { // First job cleanup still succeeded for the // CompletedCheckpointStore because the JobGraph cleanup happens @@ -98,13 +103,17 @@ public void setUp() throws Exception { maxCheckpoints, previous.getAllCheckpoints(), sharedStateRegistryFactory.create( - ioExecutor, previous.getAllCheckpoints())); + ioExecutor, + previous.getAllCheckpoints(), + restoreMode)); } return new EmbeddedCompletedCheckpointStore( maxCheckpoints, Collections.emptyList(), sharedStateRegistryFactory.create( - ioExecutor, Collections.emptyList())); + ioExecutor, + Collections.emptyList(), + RestoreMode.DEFAULT)); })); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java index 8a226fed8aa7e..cedcfc3d43510 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/cleanup/CheckpointResourcesCleanupRunnerTest.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.dispatcher.UnavailableDispatcherOperationException; import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobmaster.JobManagerRunnerResult; import org.apache.flink.runtime.jobmaster.JobResult; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; @@ -622,7 +623,8 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) + Executor ioExecutor, + RestoreMode restoreMode) throws Exception { creationLatch.await(); return completedCheckpointStore; diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java index dfc3b40da07f6..9e21dd42e0813 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerUtilsTest.java @@ -32,6 +32,7 @@ import org.apache.flink.runtime.checkpoint.StandaloneCheckpointIDCounter; import org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory; import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyedStateHandle; @@ -78,7 +79,8 @@ public void testSettingMaxNumberOfCheckpointsToRetain() throws Exception { new StandaloneCheckpointRecoveryFactory(), Executors.directExecutor(), log, - new JobID()); + new JobID(), + RestoreMode.CLAIM); assertEquals( maxNumberOfCheckpointsToRetain, @@ -104,7 +106,8 @@ public void testSharedStateRegistration() throws Exception { recoveryFactory, Executors.directExecutor(), log, - new JobID()); + new JobID(), + RestoreMode.CLAIM); SharedStateRegistry sharedStateRegistry = checkpointStore.getSharedStateRegistry(); @@ -122,12 +125,14 @@ public CompletedCheckpointStore createRecoveredCompletedCheckpointStore( JobID jobId, int maxNumberOfCheckpointsToRetain, SharedStateRegistryFactory sharedStateRegistryFactory, - Executor ioExecutor) { + Executor ioExecutor, + RestoreMode restoreMode) { List checkpoints = singletonList(checkpoint); return new EmbeddedCompletedCheckpointStore( maxNumberOfCheckpointsToRetain, checkpoints, - sharedStateRegistryFactory.create(ioExecutor, checkpoints)); + sharedStateRegistryFactory.create( + ioExecutor, checkpoints, RestoreMode.DEFAULT)); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java index 678eb2bac1f7b..4392a4ba67bcc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java @@ -57,6 +57,7 @@ import java.util.stream.Stream; import static java.lang.String.format; +import static org.apache.flink.util.Preconditions.checkState; import static org.junit.jupiter.api.Assertions.fail; /** This class contains auxiliary methods for unit tests. */ @@ -363,18 +364,31 @@ public static void waitForSubtasksToFinish( Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES))); } - /** Wait for at least one successful checkpoint. */ - public static void waitForCheckpoint(JobID jobID, MiniCluster miniCluster, Deadline timeout) + /** Wait for (at least) the given number of successful checkpoints. */ + public static void waitForCheckpoint( + JobID jobID, MiniCluster miniCluster, Deadline timeout, int numCheckpoints) throws Exception, FlinkJobNotFoundException { waitUntilCondition( - () -> - Optional.ofNullable( - miniCluster - .getExecutionGraph(jobID) - .get() - .getCheckpointStatsSnapshot()) - .filter(st -> st.getCounts().getNumberOfCompletedCheckpoints() > 0) - .isPresent(), + () -> { + AccessExecutionGraph graph = miniCluster.getExecutionGraph(jobID).get(); + if (Optional.ofNullable(graph.getCheckpointStatsSnapshot()) + .filter( + st -> + st.getCounts().getNumberOfCompletedCheckpoints() + >= numCheckpoints) + .isPresent()) { + return true; + } else if (graph.getState().isGloballyTerminalState()) { + checkState( + graph.getFailureInfo() != null, + "Job terminated before taking required %s checkpoints: %s", + numCheckpoints, + graph.getState()); + throw graph.getFailureInfo().getException(); + } else { + return false; + } + }, timeout); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java index eeda1b08b3237..03dfa55cf22f8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java @@ -18,25 +18,26 @@ package org.apache.flink.test.checkpointing; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.eventtime.AscendingTimestampsWatermarks; import org.apache.flink.api.common.eventtime.TimestampAssigner; import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; import org.apache.flink.api.common.eventtime.WatermarkGenerator; import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory; -import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.state.StateBackend; import org.apache.flink.runtime.state.filesystem.FsStateBackend; +import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.environment.CheckpointConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -51,18 +52,18 @@ import org.junit.ClassRule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Optional; +import java.time.Duration; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.stream.Stream; +import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForCheckpoint; +import static org.apache.flink.test.util.TestUtils.waitUntilJobCanceled; import static org.junit.Assert.assertNotNull; /** @@ -74,26 +75,42 @@ *

    This tests considers full and incremental checkpoints and was introduced to guard against * problems like FLINK-6964. */ +@RunWith(Parameterized.class) public class ResumeCheckpointManuallyITCase extends TestLogger { private static final int PARALLELISM = 2; private static final int NUM_TASK_MANAGERS = 2; private static final int SLOTS_PER_TASK_MANAGER = 2; + @Parameterized.Parameter public RestoreMode restoreMode; + + @Parameterized.Parameters(name = "RestoreMode = {0}") + public static Object[] parameters() { + return RestoreMode.values(); + } + @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); @Test public void testExternalizedIncrementalRocksDBCheckpointsStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); testExternalizedCheckpoints( - checkpointDir, null, createRocksDBStateBackend(checkpointDir, true), false); + checkpointDir, + null, + createRocksDBStateBackend(checkpointDir, true), + false, + restoreMode); } @Test public void testExternalizedFullRocksDBCheckpointsStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); testExternalizedCheckpoints( - checkpointDir, null, createRocksDBStateBackend(checkpointDir, false), false); + checkpointDir, + null, + createRocksDBStateBackend(checkpointDir, false), + false, + restoreMode); } @Test @@ -101,7 +118,11 @@ public void testExternalizedIncrementalRocksDBCheckpointsWithLocalRecoveryStanda throws Exception { final File checkpointDir = temporaryFolder.newFolder(); testExternalizedCheckpoints( - checkpointDir, null, createRocksDBStateBackend(checkpointDir, true), true); + checkpointDir, + null, + createRocksDBStateBackend(checkpointDir, true), + true, + restoreMode); } @Test @@ -109,20 +130,25 @@ public void testExternalizedFullRocksDBCheckpointsWithLocalRecoveryStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); testExternalizedCheckpoints( - checkpointDir, null, createRocksDBStateBackend(checkpointDir, false), true); + checkpointDir, + null, + createRocksDBStateBackend(checkpointDir, false), + true, + restoreMode); } @Test public void testExternalizedFSCheckpointsStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); testExternalizedCheckpoints( - checkpointDir, null, createFsStateBackend(checkpointDir), false); + checkpointDir, null, createFsStateBackend(checkpointDir), false, restoreMode); } @Test public void testExternalizedFSCheckpointsWithLocalRecoveryStandalone() throws Exception { final File checkpointDir = temporaryFolder.newFolder(); - testExternalizedCheckpoints(checkpointDir, null, createFsStateBackend(checkpointDir), true); + testExternalizedCheckpoints( + checkpointDir, null, createFsStateBackend(checkpointDir), true, restoreMode); } @Test @@ -135,7 +161,8 @@ public void testExternalizedIncrementalRocksDBCheckpointsZookeeper() throws Exce checkpointDir, zkServer.getConnectString(), createRocksDBStateBackend(checkpointDir, true), - false); + false, + restoreMode); } finally { zkServer.stop(); } @@ -151,7 +178,8 @@ public void testExternalizedFullRocksDBCheckpointsZookeeper() throws Exception { checkpointDir, zkServer.getConnectString(), createRocksDBStateBackend(checkpointDir, false), - false); + false, + restoreMode); } finally { zkServer.stop(); } @@ -168,7 +196,8 @@ public void testExternalizedIncrementalRocksDBCheckpointsWithLocalRecoveryZookee checkpointDir, zkServer.getConnectString(), createRocksDBStateBackend(checkpointDir, true), - true); + true, + restoreMode); } finally { zkServer.stop(); } @@ -185,7 +214,8 @@ public void testExternalizedFullRocksDBCheckpointsWithLocalRecoveryZookeeper() checkpointDir, zkServer.getConnectString(), createRocksDBStateBackend(checkpointDir, false), - true); + true, + restoreMode); } finally { zkServer.stop(); } @@ -201,7 +231,8 @@ public void testExternalizedFSCheckpointsZookeeper() throws Exception { checkpointDir, zkServer.getConnectString(), createFsStateBackend(checkpointDir), - false); + false, + restoreMode); } finally { zkServer.stop(); } @@ -217,7 +248,8 @@ public void testExternalizedFSCheckpointsWithLocalRecoveryZookeeper() throws Exc checkpointDir, zkServer.getConnectString(), createFsStateBackend(checkpointDir), - true); + true, + restoreMode); } finally { zkServer.stop(); } @@ -233,8 +265,12 @@ private RocksDBStateBackend createRocksDBStateBackend( return new RocksDBStateBackend(checkpointDir.toURI().toString(), incrementalCheckpointing); } - private void testExternalizedCheckpoints( - File checkpointDir, String zooKeeperQuorum, StateBackend backend, boolean localRecovery) + private static void testExternalizedCheckpoints( + File checkpointDir, + String zooKeeperQuorum, + StateBackend backend, + boolean localRecovery, + RestoreMode restoreMode) throws Exception { final Configuration config = new Configuration(); @@ -270,22 +306,28 @@ private void testExternalizedCheckpoints( cluster.before(); - ClusterClient client = cluster.getClusterClient(); - try { // main test sequence: start job -> eCP -> restore job -> eCP -> restore job String firstExternalCheckpoint = - runJobAndGetExternalizedCheckpoint(backend, checkpointDir, null, client); + runJobAndGetExternalizedCheckpoint(backend, null, cluster, restoreMode); assertNotNull(firstExternalCheckpoint); String secondExternalCheckpoint = runJobAndGetExternalizedCheckpoint( - backend, checkpointDir, firstExternalCheckpoint, client); + backend, firstExternalCheckpoint, cluster, restoreMode); assertNotNull(secondExternalCheckpoint); String thirdExternalCheckpoint = runJobAndGetExternalizedCheckpoint( - backend, checkpointDir, secondExternalCheckpoint, client); + backend, + // in CLAIM mode, the previous run is only guaranteed to preserve the + // latest checkpoint; in NO_CLAIM/LEGACY, even the initial checkpoints + // must remain valid + restoreMode == RestoreMode.CLAIM + ? secondExternalCheckpoint + : firstExternalCheckpoint, + cluster, + restoreMode); assertNotNull(thirdExternalCheckpoint); } finally { cluster.after(); @@ -294,77 +336,35 @@ private void testExternalizedCheckpoints( private static String runJobAndGetExternalizedCheckpoint( StateBackend backend, - File checkpointDir, @Nullable String externalCheckpoint, - ClusterClient client) + MiniClusterWithClientResource cluster, + RestoreMode restoreMode) throws Exception { - JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint); + JobGraph initialJobGraph = getJobGraph(backend, externalCheckpoint, restoreMode); NotifyingInfiniteTupleSource.countDownLatch = new CountDownLatch(PARALLELISM); - - client.submitJob(initialJobGraph).get(); + cluster.getClusterClient().submitJob(initialJobGraph).get(); // wait until all sources have been started NotifyingInfiniteTupleSource.countDownLatch.await(); - waitUntilExternalizedCheckpointCreated(checkpointDir, initialJobGraph.getJobID()); - client.cancel(initialJobGraph.getJobID()).get(); - waitUntilCanceled(initialJobGraph.getJobID(), client); - - return getExternalizedCheckpointCheckpointPath(checkpointDir, initialJobGraph.getJobID()); - } - - private static String getExternalizedCheckpointCheckpointPath(File checkpointDir, JobID jobId) - throws IOException { - Optional checkpoint = findExternalizedCheckpoint(checkpointDir, jobId); - if (!checkpoint.isPresent()) { - throw new AssertionError("No complete checkpoint could be found."); - } else { - return checkpoint.get().toString(); - } - } - - private static void waitUntilExternalizedCheckpointCreated(File checkpointDir, JobID jobId) - throws InterruptedException, IOException { - while (true) { - Thread.sleep(50); - Optional externalizedCheckpoint = - findExternalizedCheckpoint(checkpointDir, jobId); - if (externalizedCheckpoint.isPresent()) { - break; - } - } - } - - private static Optional findExternalizedCheckpoint(File checkpointDir, JobID jobId) - throws IOException { - try (Stream checkpoints = - Files.list(checkpointDir.toPath().resolve(jobId.toString()))) { - return checkpoints - .filter(path -> path.getFileName().toString().startsWith("chk-")) - .filter( - path -> { - try (Stream checkpointFiles = Files.list(path)) { - return checkpointFiles.anyMatch( - child -> - child.getFileName() - .toString() - .contains("meta")); - } catch (IOException ignored) { - return false; - } - }) - .findAny(); - } - } - - private static void waitUntilCanceled(JobID jobId, ClusterClient client) - throws ExecutionException, InterruptedException { - while (client.getJobStatus(jobId).get() != JobStatus.CANCELED) { - Thread.sleep(50); - } + // complete at least two checkpoints so that the initial checkpoint can be subsumed + waitForCheckpoint( + initialJobGraph.getJobID(), + cluster.getMiniCluster(), + Deadline.fromNow(Duration.ofMinutes(5)), + 2); + cluster.getClusterClient().cancel(initialJobGraph.getJobID()).get(); + waitUntilJobCanceled(initialJobGraph.getJobID(), cluster.getClusterClient()); + return CommonTestUtils.getLatestCompletedCheckpointPath( + initialJobGraph.getJobID(), cluster.getMiniCluster()) + .orElseThrow( + () -> { + throw new IllegalStateException("Checkpoint not generated"); + }); } - private static JobGraph getJobGraph(StateBackend backend, @Nullable String externalCheckpoint) { + private static JobGraph getJobGraph( + StateBackend backend, @Nullable String externalCheckpoint, RestoreMode restoreMode) { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.enableCheckpointing(500); @@ -373,6 +373,7 @@ private static JobGraph getJobGraph(StateBackend backend, @Nullable String exter env.getCheckpointConfig() .setExternalizedCheckpointCleanup( CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION); + env.setRestartStrategy(RestartStrategies.noRestart()); env.addSource(new NotifyingInfiniteTupleSource(10_000)) .assignTimestampsAndWatermarks(IngestionTimeWatermarkStrategy.create()) @@ -388,7 +389,7 @@ private static JobGraph getJobGraph(StateBackend backend, @Nullable String exter // recover from previous iteration? if (externalCheckpoint != null) { jobGraph.setSavepointRestoreSettings( - SavepointRestoreSettings.forPath(externalCheckpoint)); + SavepointRestoreSettings.forPath(externalCheckpoint, false, restoreMode)); } return jobGraph; diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java index ef4052c76a980..d4791ac2e63d9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java @@ -148,7 +148,8 @@ private String tryCheckpointAndStop(JobGraph jobGraph) throws Exception { waitForCheckpoint( jobGraph.getJobID(), miniClusterResource.getMiniCluster(), - Deadline.fromNow(Duration.ofMinutes(5))); + Deadline.fromNow(Duration.ofMinutes(5)), + 1); client.cancel(jobGraph.getJobID()).get(); // obtain the latest checkpoint *after* cancellation - that one won't be subsumed return CommonTestUtils.getLatestCompletedCheckpointPath( diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java index b3f70047537da..c56b233d2e2c5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java @@ -327,7 +327,8 @@ public boolean request() { } private String checkpointAndCancel(JobID jobID) throws Exception { - waitForCheckpoint(jobID, cluster.getMiniCluster(), Deadline.fromNow(Duration.ofMinutes(5))); + waitForCheckpoint( + jobID, cluster.getMiniCluster(), Deadline.fromNow(Duration.ofMinutes(5)), 1); cluster.getClusterClient().cancel(jobID).get(); checkStatus(jobID); return CommonTestUtils.getLatestCompletedCheckpointPath(jobID, cluster.getMiniCluster()) diff --git a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java index 033f1780a29cf..af58c57aad0cc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java +++ b/flink-tests/src/test/java/org/apache/flink/test/util/TestUtils.java @@ -19,6 +19,7 @@ package org.apache.flink.test.util; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; import org.apache.flink.client.ClientUtils; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.core.execution.JobClient; @@ -43,6 +44,7 @@ import java.nio.file.attribute.BasicFileAttributes; import java.util.Comparator; import java.util.Optional; +import java.util.concurrent.ExecutionException; import static org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess.CHECKPOINT_DIR_PREFIX; import static org.apache.flink.runtime.state.filesystem.AbstractFsCheckpointStorageAccess.METADATA_FILE_NAME; @@ -165,4 +167,11 @@ private static boolean hasMetadata(Path file) { return false; // should never happen } } + + public static void waitUntilJobCanceled(JobID jobId, ClusterClient client) + throws ExecutionException, InterruptedException { + while (client.getJobStatus(jobId).get() != JobStatus.CANCELED) { + Thread.sleep(50); + } + } } From c956e822ccbab326f3b7638eda6dd0ee848715ed Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Mon, 11 Apr 2022 19:34:06 +0800 Subject: [PATCH 170/258] [hotfix][python][docs] Sync the documentation of a few classes with the correspoinding Java classes --- .../pyflink/table/environment_settings.py | 4 +- flink-python/pyflink/table/statement_set.py | 14 ++--- flink-python/pyflink/table/table.py | 60 +++++++++++++------ flink-python/pyflink/table/table_config.py | 28 +++++++++ 4 files changed, 77 insertions(+), 29 deletions(-) diff --git a/flink-python/pyflink/table/environment_settings.py b/flink-python/pyflink/table/environment_settings.py index 1fbf09076817d..bc615824961b9 100644 --- a/flink-python/pyflink/table/environment_settings.py +++ b/flink-python/pyflink/table/environment_settings.py @@ -39,13 +39,13 @@ class EnvironmentSettings(object): ... .with_built_in_database_name("my_database") \\ ... .build() - :func:`EnvironmentSettings.in_streaming_mode` or :func:`EnvironmentSettings.in_batch_mode` + :func:`~EnvironmentSettings.in_streaming_mode` or :func:`~EnvironmentSettings.in_batch_mode` might be convenient as shortcuts. """ class Builder(object): """ - A builder for :class:`EnvironmentSettings`. + A builder for :class:`~EnvironmentSettings`. """ def __init__(self): diff --git a/flink-python/pyflink/table/statement_set.py b/flink-python/pyflink/table/statement_set.py index c87fbea375c6a..4c3bb7a4625d7 100644 --- a/flink-python/pyflink/table/statement_set.py +++ b/flink-python/pyflink/table/statement_set.py @@ -27,14 +27,10 @@ class StatementSet(object): """ - A StatementSet accepts DML statements or Tables, - the planner can optimize all added statements and Tables together - and then submit as one job. + A :class:`~StatementSet` accepts pipelines defined by DML statements or :class:`~Table` objects. + The planner can optimize all added statements together and then submit them as one job. - .. note:: - - The added statements and Tables will be cleared - when calling the `execute` method. + The added statements will be cleared when calling the :func:`~StatementSet.execute` method. .. versionadded:: 1.11.0 """ @@ -93,9 +89,9 @@ def add_insert(self, >>> stmt_set = table_env.create_statement_set() >>> source_table = table_env.from_path("SourceTable") - >>> sink_descriptor = TableDescriptor.for_connector("blackhole") + >>> sink_descriptor = TableDescriptor.for_connector("blackhole") \\ ... .schema(Schema.new_builder() - ... .build()) + ... .build()) \\ ... .build() >>> stmt_set.add_insert(sink_descriptor, source_table) diff --git a/flink-python/pyflink/table/table.py b/flink-python/pyflink/table/table.py index b7d8ae60673f5..200cafa23424d 100644 --- a/flink-python/pyflink/table/table.py +++ b/flink-python/pyflink/table/table.py @@ -40,32 +40,56 @@ class Table(object): - """ - A :class:`~pyflink.table.Table` is the core component of the Table API. + A :class:`~pyflink.table.Table` object is the core abstraction of the Table API. Similar to how the DataStream API has DataStream, the Table API is built around :class:`~pyflink.table.Table`. - Use the methods of :class:`~pyflink.table.Table` to transform data. + A :class:`~pyflink.table.Table` object describes a pipeline of data transformations. It does not + contain the data itself in any way. Instead, it describes how to read data from a table source, + and how to eventually write data to a table sink. The declared pipeline can be + printed, optimized, and eventually executed in a cluster. The pipeline can work with bounded or + unbounded streams which enables both streaming and batch scenarios. + + By the definition above, a :class:`~pyflink.table.Table` object can actually be considered as + a view in SQL terms. + + The initial :class:`~pyflink.table.Table` object is constructed by a + :class:`~pyflink.table.TableEnvironment`. For example, + :func:`~pyflink.table.TableEnvironment.from_path` obtains a table from a catalog. + Every :class:`~pyflink.table.Table` object has a schema that is available through + :func:`~pyflink.table.Table.get_schema`. A :class:`~pyflink.table.Table` object is + always associated with its original table environment during programming. + + Every transformation (i.e. :func:`~pyflink.table.Table.select`} or + :func:`~pyflink.table.Table.filter` on a :class:`~pyflink.table.Table` object leads to a new + :class:`~pyflink.table.Table` object. + + Use :func:`~pyflink.table.Table.execute` to execute the pipeline and retrieve the transformed + data locally during development. Otherwise, use :func:`~pyflink.table.Table.execute_insert` to + write the data into a table sink. + + Many methods of this class take one or more :class:`~pyflink.table.Expression` as parameters. + For fluent definition of expressions and easier readability, we recommend to add a star import: + + Example: + :: + + >>> from pyflink.table.expressions import * + + Check the documentation for more programming language specific APIs. + + The following example shows how to work with a :class:`~pyflink.table.Table` object. Example: :: - >>> env = StreamExecutionEnvironment.get_execution_environment() - >>> env.set_parallelism(1) - >>> t_env = StreamTableEnvironment.create(env) - >>> ... - >>> t_env.register_table_source("source", ...) - >>> t = t_env.from_path("source") - >>> t.select(...) - >>> ... - >>> t_env.register_table_sink("result", ...) - >>> t.execute_insert("result") - - Operations such as :func:`~pyflink.table.Table.join`, :func:`~pyflink.table.Table.select`, - :func:`~pyflink.table.Table.where` and :func:`~pyflink.table.Table.group_by` - take arguments in an expression string. Please refer to the documentation for - the expression syntax. + >>> from pyflink.table import TableEnvironment + >>> from pyflink.table.expressions import * + >>> env_settings = EnvironmentSettings.in_streaming_mode() + >>> t_env = TableEnvironment.create(env_settings) + >>> table = t_env.from_path("my_table").select(col("colA").trim(), col("colB") + 12) + >>> table.execute().print() """ def __init__(self, j_table, t_env): diff --git a/flink-python/pyflink/table/table_config.py b/flink-python/pyflink/table/table_config.py index 6d0899ee9afaf..5fa9896a56635 100644 --- a/flink-python/pyflink/table/table_config.py +++ b/flink-python/pyflink/table/table_config.py @@ -35,12 +35,40 @@ class TableConfig(object): Configuration for the current :class:`TableEnvironment` session to adjust Table & SQL API programs. + This class is a pure API class that abstracts configuration from various sources. Currently, + configuration can be set in any of the following layers (in the given order): + + - flink-conf.yaml + - CLI parameters + - :class:`~pyflink.datastream.StreamExecutionEnvironment` when bridging to DataStream API + - :func:`~EnvironmentSettings.Builder.with_configuration` + - :func:`~TableConfig.set` + + The latter two represent the application-specific part of the configuration. They initialize + and directly modify :func:`~TableConfig.get_configuration`. Other layers represent the + configuration of the execution context and are immutable. + + The getter :func:`~TableConfig.get` gives read-only access to the full configuration. However, + application-specific configuration has precedence. Configuration of outer layers is used for + defaults and fallbacks. The setter :func:`~TableConfig.set` will only affect + application-specific configuration. + For common or important configuration options, this class provides getters and setters methods with detailed inline documentation. For more advanced configuration, users can directly access the underlying key-value map via :func:`~pyflink.table.TableConfig.get_configuration`. + Example: + :: + + >>> table_config = t_env.get_config() + >>> config = Configuration() + >>> config.set_string("parallelism.default", "128") \\ + ... .set_string("pipeline.auto-watermark-interval", "800ms") \\ + ... .set_string("execution.checkpointing.interval", "30s") + >>> table_config.add_configuration(config) + .. note:: Because options are read at different point in time when performing operations, it is From 241c0971061c9bdb6491f9c6f0d2bce296e270a3 Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Wed, 16 Mar 2022 11:48:01 +0800 Subject: [PATCH 171/258] support sql end with ";" in for Hive dialect --- .../delegation/hive/parse/HiveASTParser.g | 4 +-- .../src/test/resources/sql/set.q | 28 ++++++++++++++++++- 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g index a9fedc6b5041d..367d4b21cca2d 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g @@ -732,8 +732,8 @@ catch (RecognitionException e) { // starting rule statement - : explainStatement EOF - | execStatement EOF + : explainStatement SEMICOLON? EOF + | execStatement SEMICOLON? EOF ; explainStatement diff --git a/flink-table/flink-sql-client/src/test/resources/sql/set.q b/flink-table/flink-sql-client/src/test/resources/sql/set.q index 5d214ef175921..5768953d40105 100644 --- a/flink-table/flink-sql-client/src/test/resources/sql/set.q +++ b/flink-table/flink-sql-client/src/test/resources/sql/set.q @@ -16,10 +16,25 @@ # limitations under the License. # test set a configuration +SET 'sql-client.execution.result-mode' = 'tableau'; +[INFO] Session property has been set. +!info + SET 'table.sql-dialect' = 'hive'; [INFO] Session property has been set. !info +create catalog hivecatalog with ( + 'type' = 'hive-test', + 'hive-version' = '2.3.4' +); +[INFO] Execute statement succeed. +!info + +use catalog hivecatalog; +[INFO] Execute statement succeed. +!info + # test create a hive table to verify the configuration works CREATE TABLE hive_table ( product_id STRING, @@ -28,7 +43,7 @@ CREATE TABLE hive_table ( pv_count BIGINT, like_count BIGINT, comment_count BIGINT, - update_time TIMESTAMP(3), + update_time TIMESTAMP, update_user STRING ) PARTITIONED BY (pt_year STRING, pt_month STRING, pt_day STRING) TBLPROPERTIES ( 'streaming-source.enable' = 'true' @@ -36,6 +51,16 @@ CREATE TABLE hive_table ( [INFO] Execute statement succeed. !info +# test "ctas" only supported in Hive Dialect +CREATE TABLE foo as select 1; ++-------------------------+ +| hivecatalog.default.foo | ++-------------------------+ +| -1 | ++-------------------------+ +1 row in set +!ok + # list the configured configuration set; 'execution.attached' = 'true' @@ -47,6 +72,7 @@ set; 'pipeline.classpaths' = '' 'pipeline.jars' = '' 'rest.port' = '$VAR_REST_PORT' +'sql-client.execution.result-mode' = 'tableau' 'table.exec.legacy-cast-behaviour' = 'DISABLED' 'table.sql-dialect' = 'hive' !ok From f97463d14331339820c3a9ba4b9c1641a9fd6c90 Mon Sep 17 00:00:00 2001 From: Jing Date: Tue, 12 Apr 2022 09:38:24 +0800 Subject: [PATCH 172/258] Revert "support sql end with ";" in for Hive dialect" This reverts commit 241c0971 Signed-off-by: Jing --- .../delegation/hive/parse/HiveASTParser.g | 4 +-- .../src/test/resources/sql/set.q | 28 +------------------ 2 files changed, 3 insertions(+), 29 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g index 367d4b21cca2d..a9fedc6b5041d 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g @@ -732,8 +732,8 @@ catch (RecognitionException e) { // starting rule statement - : explainStatement SEMICOLON? EOF - | execStatement SEMICOLON? EOF + : explainStatement EOF + | execStatement EOF ; explainStatement diff --git a/flink-table/flink-sql-client/src/test/resources/sql/set.q b/flink-table/flink-sql-client/src/test/resources/sql/set.q index 5768953d40105..5d214ef175921 100644 --- a/flink-table/flink-sql-client/src/test/resources/sql/set.q +++ b/flink-table/flink-sql-client/src/test/resources/sql/set.q @@ -16,25 +16,10 @@ # limitations under the License. # test set a configuration -SET 'sql-client.execution.result-mode' = 'tableau'; -[INFO] Session property has been set. -!info - SET 'table.sql-dialect' = 'hive'; [INFO] Session property has been set. !info -create catalog hivecatalog with ( - 'type' = 'hive-test', - 'hive-version' = '2.3.4' -); -[INFO] Execute statement succeed. -!info - -use catalog hivecatalog; -[INFO] Execute statement succeed. -!info - # test create a hive table to verify the configuration works CREATE TABLE hive_table ( product_id STRING, @@ -43,7 +28,7 @@ CREATE TABLE hive_table ( pv_count BIGINT, like_count BIGINT, comment_count BIGINT, - update_time TIMESTAMP, + update_time TIMESTAMP(3), update_user STRING ) PARTITIONED BY (pt_year STRING, pt_month STRING, pt_day STRING) TBLPROPERTIES ( 'streaming-source.enable' = 'true' @@ -51,16 +36,6 @@ CREATE TABLE hive_table ( [INFO] Execute statement succeed. !info -# test "ctas" only supported in Hive Dialect -CREATE TABLE foo as select 1; -+-------------------------+ -| hivecatalog.default.foo | -+-------------------------+ -| -1 | -+-------------------------+ -1 row in set -!ok - # list the configured configuration set; 'execution.attached' = 'true' @@ -72,7 +47,6 @@ set; 'pipeline.classpaths' = '' 'pipeline.jars' = '' 'rest.port' = '$VAR_REST_PORT' -'sql-client.execution.result-mode' = 'tableau' 'table.exec.legacy-cast-behaviour' = 'DISABLED' 'table.sql-dialect' = 'hive' !ok From 85e70e2222976375a5e33987da67166c0ffb4c92 Mon Sep 17 00:00:00 2001 From: luoyuxia Date: Wed, 16 Mar 2022 11:48:01 +0800 Subject: [PATCH 173/258] [FLINK-26681][hive] Support sql end with ";" for Hive dialect This closes #19109 --- .../delegation/hive/parse/HiveASTParser.g | 4 +-- .../src/test/resources/sql/set.q | 28 ++++++++++++++++++- 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g index a9fedc6b5041d..367d4b21cca2d 100644 --- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g +++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/planner/delegation/hive/parse/HiveASTParser.g @@ -732,8 +732,8 @@ catch (RecognitionException e) { // starting rule statement - : explainStatement EOF - | execStatement EOF + : explainStatement SEMICOLON? EOF + | execStatement SEMICOLON? EOF ; explainStatement diff --git a/flink-table/flink-sql-client/src/test/resources/sql/set.q b/flink-table/flink-sql-client/src/test/resources/sql/set.q index 5d214ef175921..5768953d40105 100644 --- a/flink-table/flink-sql-client/src/test/resources/sql/set.q +++ b/flink-table/flink-sql-client/src/test/resources/sql/set.q @@ -16,10 +16,25 @@ # limitations under the License. # test set a configuration +SET 'sql-client.execution.result-mode' = 'tableau'; +[INFO] Session property has been set. +!info + SET 'table.sql-dialect' = 'hive'; [INFO] Session property has been set. !info +create catalog hivecatalog with ( + 'type' = 'hive-test', + 'hive-version' = '2.3.4' +); +[INFO] Execute statement succeed. +!info + +use catalog hivecatalog; +[INFO] Execute statement succeed. +!info + # test create a hive table to verify the configuration works CREATE TABLE hive_table ( product_id STRING, @@ -28,7 +43,7 @@ CREATE TABLE hive_table ( pv_count BIGINT, like_count BIGINT, comment_count BIGINT, - update_time TIMESTAMP(3), + update_time TIMESTAMP, update_user STRING ) PARTITIONED BY (pt_year STRING, pt_month STRING, pt_day STRING) TBLPROPERTIES ( 'streaming-source.enable' = 'true' @@ -36,6 +51,16 @@ CREATE TABLE hive_table ( [INFO] Execute statement succeed. !info +# test "ctas" only supported in Hive Dialect +CREATE TABLE foo as select 1; ++-------------------------+ +| hivecatalog.default.foo | ++-------------------------+ +| -1 | ++-------------------------+ +1 row in set +!ok + # list the configured configuration set; 'execution.attached' = 'true' @@ -47,6 +72,7 @@ set; 'pipeline.classpaths' = '' 'pipeline.jars' = '' 'rest.port' = '$VAR_REST_PORT' +'sql-client.execution.result-mode' = 'tableau' 'table.exec.legacy-cast-behaviour' = 'DISABLED' 'table.sql-dialect' = 'hive' !ok From dbfe5da2a2bc4564a1cf715a918299d107758b6c Mon Sep 17 00:00:00 2001 From: tsreaper Date: Wed, 9 Mar 2022 16:25:58 +0800 Subject: [PATCH 174/258] [FLINK-24586][table-planner] JSON_VALUE should return STRING instead of VARCHAR(2000) This closes #19014. --- .../functions/sql/FlinkSqlOperatorTable.java | 2 +- .../sql/SqlJsonValueFunctionWrapper.java | 83 +++++++++++++++++++ .../functions/JsonFunctionsITCase.java | 9 +- 3 files changed, 88 insertions(+), 6 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlJsonValueFunctionWrapper.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java index cd9499610582b..2b18ffee5e3a3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/FlinkSqlOperatorTable.java @@ -1140,7 +1140,7 @@ public List getAuxiliaryFunctions() { // JSON FUNCTIONS public static final SqlFunction JSON_EXISTS = SqlStdOperatorTable.JSON_EXISTS; - public static final SqlFunction JSON_VALUE = SqlStdOperatorTable.JSON_VALUE; + public static final SqlFunction JSON_VALUE = new SqlJsonValueFunctionWrapper("JSON_VALUE"); public static final SqlFunction JSON_QUERY = new SqlJsonQueryFunctionWrapper(); public static final SqlFunction JSON_OBJECT = new SqlJsonObjectFunctionWrapper(); public static final SqlAggFunction JSON_OBJECTAGG_NULL_ON_NULL = diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlJsonValueFunctionWrapper.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlJsonValueFunctionWrapper.java new file mode 100644 index 0000000000000..b28ef4786e47f --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/sql/SqlJsonValueFunctionWrapper.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.functions.sql; + +import org.apache.calcite.rel.type.RelDataType; +import org.apache.calcite.sql.SqlJsonValueReturning; +import org.apache.calcite.sql.SqlOperatorBinding; +import org.apache.calcite.sql.fun.SqlJsonValueFunction; +import org.apache.calcite.sql.type.ReturnTypes; +import org.apache.calcite.sql.type.SqlReturnTypeInference; +import org.apache.calcite.sql.type.SqlTypeTransforms; + +import java.util.Optional; + +import static org.apache.flink.table.planner.plan.type.FlinkReturnTypes.VARCHAR_FORCE_NULLABLE; + +/** + * This class is a wrapper class for the {@link SqlJsonValueFunction} but using the {@code + * VARCHAR_FORCE_NULLABLE} return type inference by default. It also supports specifying return type + * with the RETURNING keyword just like the original {@link SqlJsonValueFunction}. + */ +class SqlJsonValueFunctionWrapper extends SqlJsonValueFunction { + + private final SqlReturnTypeInference returnTypeInference; + + SqlJsonValueFunctionWrapper(String name) { + super(name); + this.returnTypeInference = + ReturnTypes.cascade( + SqlJsonValueFunctionWrapper::explicitTypeSpec, + SqlTypeTransforms.FORCE_NULLABLE) + .orElse(VARCHAR_FORCE_NULLABLE); + } + + @Override + public RelDataType inferReturnType(SqlOperatorBinding opBinding) { + RelDataType returnType = returnTypeInference.inferReturnType(opBinding); + if (returnType == null) { + throw new IllegalArgumentException( + "Cannot infer return type for " + + opBinding.getOperator() + + "; operand types: " + + opBinding.collectOperandTypes()); + } + return returnType; + } + + @Override + public SqlReturnTypeInference getReturnTypeInference() { + return returnTypeInference; + } + + /** + * Copied and modified from the original {@link SqlJsonValueFunction}. + * + *

    Changes: Instead of returning {@link Optional} this method returns null directly. + */ + private static RelDataType explicitTypeSpec(SqlOperatorBinding opBinding) { + if (opBinding.getOperandCount() > 2 + && opBinding.isOperandLiteral(2, false) + && opBinding.getOperandLiteralValue(2, Object.class) + instanceof SqlJsonValueReturning) { + return opBinding.getOperandType(3); + } + return null; + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java index e01c0f74df8f8..9fb64e7e58bcb 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/JsonFunctionsITCase.java @@ -57,7 +57,6 @@ import static org.apache.flink.table.api.DataTypes.TIMESTAMP; import static org.apache.flink.table.api.DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE; import static org.apache.flink.table.api.DataTypes.VARBINARY; -import static org.apache.flink.table.api.DataTypes.VARCHAR; import static org.apache.flink.table.api.Expressions.$; import static org.apache.flink.table.api.Expressions.call; import static org.apache.flink.table.api.Expressions.jsonArray; @@ -183,7 +182,7 @@ private static TestSpec jsonValueSpec() throws Exception { "JSON_VALUE(CAST(NULL AS STRING), 'lax $')", null, STRING(), - VARCHAR(2000)), + STRING()), // RETURNING + Supported Data Types resultSpec( @@ -191,7 +190,7 @@ private static TestSpec jsonValueSpec() throws Exception { "JSON_VALUE(f0, '$.type')", "account", STRING(), - VARCHAR(2000)), + STRING()), resultSpec( $("f0").jsonValue("$.activated", BOOLEAN()), "JSON_VALUE(f0, '$.activated' RETURNING BOOLEAN)", @@ -220,7 +219,7 @@ private static TestSpec jsonValueSpec() throws Exception { "JSON_VALUE(f0, 'lax $.invalid' NULL ON EMPTY ERROR ON ERROR)", null, STRING(), - VARCHAR(2000)), + STRING()), resultSpec( $("f0").jsonValue( "lax $.invalid", @@ -243,7 +242,7 @@ private static TestSpec jsonValueSpec() throws Exception { "JSON_VALUE(f0, 'strict $.invalid' ERROR ON EMPTY NULL ON ERROR)", null, STRING(), - VARCHAR(2000)), + STRING()), resultSpec( $("f0").jsonValue( "strict $.invalid", From 55fa1cf56dd8633aa0a04fc19e2c86d9ae2c2617 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 12 Apr 2022 13:57:15 +0200 Subject: [PATCH 175/258] [FLINK-26977] Remove no longer valid tests in JobMasterStopWithSavepointITCase Tests in JobMasterStopWithSavepointITCase are no longer valid. In particular the way number of restarts is calculated is wrong. Moreover cases that were supposed to be tested in the class are already covered in SavepointITCase. --- .../JobMasterStopWithSavepointITCase.java | 410 ------------------ 1 file changed, 410 deletions(-) delete mode 100644 flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java deleted file mode 100644 index b763cad5295de..0000000000000 --- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java +++ /dev/null @@ -1,410 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.jobmaster; - -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.client.program.MiniClusterClient; -import org.apache.flink.core.execution.SavepointFormatType; -import org.apache.flink.core.testutils.OneShotLatch; -import org.apache.flink.runtime.checkpoint.CheckpointException; -import org.apache.flink.runtime.checkpoint.CheckpointFailureReason; -import org.apache.flink.runtime.checkpoint.CheckpointMetaData; -import org.apache.flink.runtime.checkpoint.CheckpointOptions; -import org.apache.flink.runtime.checkpoint.CheckpointRetentionPolicy; -import org.apache.flink.runtime.checkpoint.SavepointType; -import org.apache.flink.runtime.checkpoint.SnapshotType; -import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.jobgraph.JobGraph; -import org.apache.flink.runtime.jobgraph.JobGraphBuilder; -import org.apache.flink.runtime.jobgraph.JobVertex; -import org.apache.flink.runtime.jobgraph.tasks.CheckpointCoordinatorConfiguration; -import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings; -import org.apache.flink.runtime.jobgraph.tasks.TaskInvokable; -import org.apache.flink.runtime.testutils.CommonTestUtils; -import org.apache.flink.streaming.runtime.tasks.StreamTask; -import org.apache.flink.streaming.runtime.tasks.StreamTaskTest.NoOpStreamTask; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; -import org.apache.flink.test.util.AbstractTestBase; -import org.apache.flink.util.ExceptionUtils; - -import org.junit.Assume; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.time.Duration; -import java.util.Collections; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.either; -import static org.hamcrest.Matchers.equalTo; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -/** - * ITCases testing the stop with savepoint functionality. This includes checking both SUSPEND and - * TERMINATE. - */ -public class JobMasterStopWithSavepointITCase extends AbstractTestBase { - - @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private static final long CHECKPOINT_INTERVAL = 10; - private static final int PARALLELISM = 2; - - private static OneShotLatch finishingLatch; - - private static CountDownLatch invokeLatch; - - private static CountDownLatch numberOfRestarts; - private static final AtomicLong syncSavepointId = new AtomicLong(); - private static volatile CountDownLatch checkpointsToWaitFor; - - private Path savepointDirectory; - private MiniClusterClient clusterClient; - - private JobGraph jobGraph; - - @Test - public void throwingExceptionOnCallbackWithNoRestartsShouldFailTheSuspend() throws Exception { - throwingExceptionOnCallbackWithoutRestartsHelper(false); - } - - @Test - public void throwingExceptionOnCallbackWithNoRestartsShouldFailTheTerminate() throws Exception { - throwingExceptionOnCallbackWithoutRestartsHelper(true); - } - - private void throwingExceptionOnCallbackWithoutRestartsHelper(final boolean terminate) - throws Exception { - setUpJobGraph(ExceptionOnCallbackStreamTask.class, RestartStrategies.noRestart()); - - assertThat(getJobStatus(), equalTo(JobStatus.RUNNING)); - - try { - stopWithSavepoint(terminate).get(); - fail(); - } catch (Exception e) { - } - - // verifying that we actually received a synchronous checkpoint - assertTrue(syncSavepointId.get() > 0); - assertThat( - getJobStatus(), either(equalTo(JobStatus.FAILED)).or(equalTo(JobStatus.FAILING))); - } - - @Test - public void throwingExceptionOnCallbackWithRestartsShouldSimplyRestartInSuspend() - throws Exception { - throwingExceptionOnCallbackWithRestartsHelper(false); - } - - @Test - public void throwingExceptionOnCallbackWithRestartsShouldSimplyRestartInTerminate() - throws Exception { - throwingExceptionOnCallbackWithRestartsHelper(true); - } - - private void throwingExceptionOnCallbackWithRestartsHelper(final boolean terminate) - throws Exception { - final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(15)); - final int numberOfCheckpointsToExpect = 10; - - numberOfRestarts = new CountDownLatch(2); - checkpointsToWaitFor = new CountDownLatch(numberOfCheckpointsToExpect); - - setUpJobGraph( - ExceptionOnCallbackStreamTask.class, - RestartStrategies.fixedDelayRestart(15, Time.milliseconds(10))); - assertThat(getJobStatus(), equalTo(JobStatus.RUNNING)); - try { - stopWithSavepoint(terminate).get(50, TimeUnit.MILLISECONDS); - fail(); - } catch (Exception e) { - // expected - } - - // wait until we restart at least 2 times and until we see at least 10 checkpoints. - assertTrue(numberOfRestarts.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); - assertTrue( - checkpointsToWaitFor.await(deadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS)); - - // verifying that we actually received a synchronous checkpoint - assertTrue(syncSavepointId.get() > 0); - - assertThat(getJobStatus(), equalTo(JobStatus.RUNNING)); - - // make sure that we saw the synchronous savepoint and - // that after that we saw more checkpoints due to restarts. - final long syncSavepoint = syncSavepointId.get(); - assertTrue(syncSavepoint > 0 && syncSavepoint < numberOfCheckpointsToExpect); - - clusterClient.cancel(jobGraph.getJobID()).get(); - assertThat( - getJobStatus(), - either(equalTo(JobStatus.CANCELLING)).or(equalTo(JobStatus.CANCELED))); - } - - @Test - public void testRestartCheckpointCoordinatorIfStopWithSavepointFails() throws Exception { - setUpJobGraph(CheckpointCountingTask.class, RestartStrategies.noRestart()); - - try { - Files.setPosixFilePermissions(savepointDirectory, Collections.emptySet()); - } catch (IOException e) { - Assume.assumeNoException(e); - } - - try { - stopWithSavepoint(true).get(); - fail(); - } catch (Exception e) { - Optional checkpointExceptionOptional = - ExceptionUtils.findThrowable(e, CheckpointException.class); - if (!checkpointExceptionOptional.isPresent()) { - throw e; - } - String exceptionMessage = checkpointExceptionOptional.get().getMessage(); - assertTrue( - "Stop with savepoint failed because of another cause " + exceptionMessage, - exceptionMessage.contains(CheckpointFailureReason.IO_EXCEPTION.message())); - } - - final JobStatus jobStatus = - clusterClient.getJobStatus(jobGraph.getJobID()).get(60, TimeUnit.SECONDS); - assertThat(jobStatus, equalTo(JobStatus.RUNNING)); - // assert that checkpoints are continued to be triggered - checkpointsToWaitFor = new CountDownLatch(1); - assertTrue(checkpointsToWaitFor.await(60L, TimeUnit.SECONDS)); - } - - private CompletableFuture stopWithSavepoint(boolean terminate) { - return MINI_CLUSTER_RESOURCE - .getMiniCluster() - .stopWithSavepoint( - jobGraph.getJobID(), - savepointDirectory.toAbsolutePath().toString(), - terminate, - SavepointFormatType.CANONICAL); - } - - private JobStatus getJobStatus() throws InterruptedException, ExecutionException { - return clusterClient.getJobStatus(jobGraph.getJobID()).get(); - } - - private void setUpJobGraph( - final Class invokable, - final RestartStrategies.RestartStrategyConfiguration restartStrategy) - throws Exception { - - finishingLatch = new OneShotLatch(); - - invokeLatch = new CountDownLatch(PARALLELISM); - - numberOfRestarts = new CountDownLatch(2); - checkpointsToWaitFor = new CountDownLatch(10); - - syncSavepointId.set(-1); - - savepointDirectory = temporaryFolder.newFolder().toPath(); - - Assume.assumeTrue( - "ClusterClient is not an instance of MiniClusterClient", - MINI_CLUSTER_RESOURCE.getClusterClient() instanceof MiniClusterClient); - - clusterClient = (MiniClusterClient) MINI_CLUSTER_RESOURCE.getClusterClient(); - - final ExecutionConfig config = new ExecutionConfig(); - config.setRestartStrategy(restartStrategy); - - final JobVertex vertex = new JobVertex("testVertex"); - vertex.setInvokableClass(invokable); - vertex.setParallelism(PARALLELISM); - - final JobCheckpointingSettings jobCheckpointingSettings = - new JobCheckpointingSettings( - new CheckpointCoordinatorConfiguration( - CHECKPOINT_INTERVAL, - 60_000, - 10, - 1, - CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION, - true, - false, - 0, - 0), - null); - - jobGraph = - JobGraphBuilder.newStreamingJobGraphBuilder() - .setExecutionConfig(config) - .addJobVertex(vertex) - .setJobCheckpointingSettings(jobCheckpointingSettings) - .build(); - - clusterClient.submitJob(jobGraph).get(); - assertTrue(invokeLatch.await(60, TimeUnit.SECONDS)); - waitForJob(); - } - - private void waitForJob() throws Exception { - Deadline deadline = Deadline.fromNow(Duration.ofMinutes(5)); - JobID jobID = jobGraph.getJobID(); - CommonTestUtils.waitForAllTaskRunning( - () -> - MINI_CLUSTER_RESOURCE - .getMiniCluster() - .getExecutionGraph(jobID) - .get(60, TimeUnit.SECONDS), - deadline, - false); - } - - /** - * A {@link StreamTask} which throws an exception in the {@code notifyCheckpointComplete()} for - * subtask 0. - */ - public static class ExceptionOnCallbackStreamTask extends CheckpointCountingTask { - - private long synchronousSavepointId = Long.MIN_VALUE; - - public ExceptionOnCallbackStreamTask(final Environment environment) throws Exception { - super(environment); - } - - @Override - protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { - final long taskIndex = getEnvironment().getTaskInfo().getIndexOfThisSubtask(); - if (taskIndex == 0) { - numberOfRestarts.countDown(); - } - super.processInput(controller); - } - - @Override - public CompletableFuture triggerCheckpointAsync( - CheckpointMetaData checkpointMetaData, CheckpointOptions checkpointOptions) { - final long checkpointId = checkpointMetaData.getCheckpointId(); - final SnapshotType checkpointType = checkpointOptions.getCheckpointType(); - - if (checkpointType.isSavepoint() && ((SavepointType) checkpointType).isSynchronous()) { - synchronousSavepointId = checkpointId; - syncSavepointId.compareAndSet(-1, synchronousSavepointId); - } - - return super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions); - } - - @Override - public Future notifyCheckpointCompleteAsync(long checkpointId) { - final long taskIndex = getEnvironment().getTaskInfo().getIndexOfThisSubtask(); - if (checkpointId == synchronousSavepointId && taskIndex == 0) { - throw new RuntimeException("Expected Exception"); - } - - return super.notifyCheckpointCompleteAsync(checkpointId); - } - - @Override - public Future notifyCheckpointAbortAsync( - long checkpointId, long latestCompletedCheckpointId) { - return CompletableFuture.completedFuture(null); - } - } - - /** A {@link StreamTask} that simply waits to be terminated normally. */ - public static class NoOpBlockingStreamTask extends NoOpStreamTask { - - private transient MailboxDefaultAction.Suspension suspension; - - public NoOpBlockingStreamTask(final Environment environment) throws Exception { - super(environment); - } - - @Override - protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { - invokeLatch.countDown(); - if (suspension == null) { - suspension = controller.suspendDefaultAction(); - } else { - controller.suspendDefaultAction(); - mailboxProcessor.suspend(); - } - } - } - - /** - * A {@link StreamTask} that simply calls {@link CountDownLatch#countDown()} when invoking - * {@link #triggerCheckpointAsync}. - */ - public static class CheckpointCountingTask extends NoOpStreamTask { - - private transient MailboxDefaultAction.Suspension suspension; - - public CheckpointCountingTask(final Environment environment) throws Exception { - super(environment); - } - - @Override - protected void processInput(MailboxDefaultAction.Controller controller) throws Exception { - invokeLatch.countDown(); - if (suspension == null) { - suspension = controller.suspendDefaultAction(); - } else { - controller.suspendDefaultAction(); - mailboxProcessor.suspend(); - } - } - - @Override - protected void cancelTask() throws Exception { - super.cancelTask(); - if (suspension != null) { - suspension.resume(); - } - } - - @Override - public CompletableFuture triggerCheckpointAsync( - final CheckpointMetaData checkpointMetaData, - final CheckpointOptions checkpointOptions) { - final long taskIndex = getEnvironment().getTaskInfo().getIndexOfThisSubtask(); - if (taskIndex == 0) { - checkpointsToWaitFor.countDown(); - } - - return super.triggerCheckpointAsync(checkpointMetaData, checkpointOptions); - } - } -} From 1fe11253572a1b1d1224de638e313666da346f62 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Tue, 12 Apr 2022 17:17:40 +0200 Subject: [PATCH 176/258] [FLINK-26977] Unify SavepointITCase#testStopWithSavepointFailingAfterSnapshotCreation for both schedulers This closes #19439 --- .../test/checkpointing/SavepointITCase.java | 46 +++---------------- 1 file changed, 6 insertions(+), 40 deletions(-) diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 92ddcdf757c53..4be614c93a2ea 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -1018,22 +1018,12 @@ public void testStopWithSavepointFailingAfterSnapshotCreation() throws Exception // 2. job failover triggered by SchedulerBase.stopWithSavepoint 0, (jobId, actualException) -> { - if (ClusterOptions.isAdaptiveSchedulerEnabled(new Configuration())) { - return actualException - .getMessage() - .contains("Stop with savepoint operation could not be completed"); - } else { - Optional actualFlinkException = - findThrowable( - actualException, StopWithSavepointStoppingException.class); - return actualFlinkException - .map( - e -> - e.getMessage() - .startsWith( - "A savepoint has been created at:")) - .orElse(false); - } + Optional actualFlinkException = + findThrowable( + actualException, StopWithSavepointStoppingException.class); + return actualFlinkException + .map(e -> e.getMessage().startsWith("A savepoint has been created at:")) + .orElse(false); }, false); } @@ -1088,30 +1078,6 @@ public void testStopWithSavepointWithDrainGlobalFailoverIfSavepointAborted() thr } } - private static BiFunction - assertAfterSnapshotCreationFailure() { - return (jobId, actualException) -> { - if (ClusterOptions.isAdaptiveSchedulerEnabled(new Configuration())) { - return actualException - .getMessage() - .contains("Stop with savepoint operation could not be completed"); - } else { - Optional actualFlinkException = - findThrowable(actualException, FlinkException.class); - if (!actualFlinkException.isPresent()) { - return false; - } - return actualFlinkException - .get() - .getMessage() - .contains( - String.format( - "A global fail-over is triggered to recover the job %s.", - jobId)); - } - }; - } - private static BiFunction assertInSnapshotCreationFailure() { return (ignored, actualException) -> { From 489827520b1a53db04a94346c98327d0d42301c5 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 29 Mar 2022 09:44:49 +0200 Subject: [PATCH 177/258] [FLINK-26616][tests] Remove deadlines from CommonTestUtils --- .../ApplicationDispatcherBootstrapITCase.java | 35 ++------ .../connectors/rabbitmq/RMQSourceITCase.java | 13 +-- .../flink/FlinkContainerTestEnvironment.java | 5 +- .../common/HAJobRunOnMinioS3StoreITCase.java | 6 +- .../KubernetesResourceManagerDriverTest.java | 9 +- .../KubernetesCheckpointIDCounterTest.java | 4 +- ...vailabilityRecoverFromSavepointITCase.java | 4 - .../KubernetesHighAvailabilityTestBase.java | 2 +- ...netesLeaderElectionAndRetrievalITCase.java | 6 +- .../KubernetesLeaderElectionDriverTest.java | 10 +-- .../KubernetesLeaderRetrievalDriverTest.java | 7 +- ...ipleComponentLeaderElectionDriverTest.java | 2 +- .../KubernetesStateHandleStoreITCase.java | 5 +- .../KubernetesStateHandleStoreTest.java | 4 +- .../KubernetesLeaderElectorITCase.java | 10 +-- .../TestingLeaderCallbackHandler.java | 37 ++------ .../runtime/blob/BlobServerCleanupTest.java | 6 +- .../runtime/blob/PermanentBlobCacheTest.java | 7 +- .../runtime/blob/TransientBlobCacheTest.java | 6 +- ...oKeeperCompletedCheckpointStoreITCase.java | 8 +- .../dispatcher/AbstractDispatcherTest.java | 5 +- .../dispatcher/DispatcherCleanupITCase.java | 19 +--- .../runtime/dispatcher/DispatcherTest.java | 3 - .../dispatcher/JobDispatcherITCase.java | 16 +--- .../ZooKeeperDefaultDispatcherRunnerTest.java | 8 +- .../ZooKeeperJobGraphStoreWatcherTest.java | 11 +-- .../runtime/jobmaster/JobMasterTest.java | 14 +-- .../slotpool/SlotPoolInteractionsTest.java | 10 +-- .../DefaultLeaderElectionServiceTest.java | 18 +--- .../LeaderChangeClusterComponentsTest.java | 10 +-- .../StandaloneLeaderElectionTest.java | 4 +- .../leaderelection/TestingLeaderBase.java | 40 +++------ .../leaderelection/TestingRetrievalBase.java | 42 +++------ .../ZooKeeperLeaderElectionTest.java | 37 +++----- ...ipleComponentLeaderElectionDriverTest.java | 6 +- .../DefaultLeaderRetrievalServiceTest.java | 18 +--- .../SettableLeaderRetrievalServiceTest.java | 7 +- ...LeaderRetrievalConnectionHandlingTest.java | 4 +- .../AdaptiveSchedulerClusterITCase.java | 7 +- .../AdaptiveSchedulerSimpleITCase.java | 3 - .../taskexecutor/TaskExecutorITCase.java | 7 +- .../taskexecutor/TaskExecutorTest.java | 2 - .../runtime/testutils/CommonTestUtils.java | 88 ++++--------------- .../testutils/MiniClusterResource.java | 27 ++---- .../tasks/StreamTaskFinalCheckpointsTest.java | 5 +- .../flink/table/client/cli/CliClientTest.java | 5 +- .../client/cli/CliTableauResultViewTest.java | 10 +-- .../MiniClusterTestEnvironment.java | 5 +- .../testsuites/SinkTestSuiteBase.java | 29 ++---- .../testsuites/SourceTestSuiteBase.java | 43 ++------- .../utils/ConnectorTestConstants.java | 1 - .../checkpointing/ManualCheckpointITCase.java | 13 +-- .../ResumeCheckpointManuallyITCase.java | 8 +- .../test/checkpointing/SavepointITCase.java | 3 +- ...lignedCheckpointFailureHandlingITCase.java | 3 +- .../recovery/ClusterEntrypointITCase.java | 8 +- ...ManagerHAProcessFailureRecoveryITCase.java | 2 +- .../test/recovery/LocalRecoveryITCase.java | 12 +-- .../recovery/TaskManagerRunnerITCase.java | 7 +- .../DefaultSchedulerLocalRecoveryITCase.java | 6 +- .../ZooKeeperLeaderElectionITCase.java | 23 ++--- .../scheduling/AdaptiveSchedulerITCase.java | 18 +--- .../test/scheduling/ReactiveModeITCase.java | 9 +- .../state/ChangelogCompatibilityITCase.java | 8 +- .../test/state/ChangelogRescalingITCase.java | 4 +- .../yarn/YARNHighAvailabilityITCase.java | 20 ++--- .../YARNSessionCapacitySchedulerITCase.java | 14 ++- .../flink/yarn/YARNSessionFIFOITCase.java | 17 +--- 68 files changed, 206 insertions(+), 659 deletions(-) diff --git a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java index c20a959db4894..47036fcf3528e 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java +++ b/flink-clients/src/test/java/org/apache/flink/client/deployment/application/ApplicationDispatcherBootstrapITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.cli.ClientOptions; import org.apache.flink.client.deployment.application.executors.EmbeddedExecutor; import org.apache.flink.client.program.PackagedProgram; @@ -74,8 +73,6 @@ @ExtendWith(TestLoggerExtension.class) public class ApplicationDispatcherBootstrapITCase { - private static final Duration TIMEOUT = Duration.ofMinutes(10); - private static Supplier createApplicationModeDispatcherResourceManagerComponentFactorySupplier( Configuration configuration, PackagedProgram program) { @@ -97,7 +94,6 @@ public class ApplicationDispatcherBootstrapITCase { @Test public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exception { final String blockId = UUID.randomUUID().toString(); - final Deadline deadline = Deadline.fromNow(TIMEOUT); final Configuration configuration = new Configuration(); configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); configuration.set(DeploymentOptions.TARGET, EmbeddedExecutor.NAME); @@ -121,11 +117,7 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc cluster.start(); // wait until job is running - awaitJobStatus( - cluster, - ApplicationDispatcherBootstrap.ZERO_JOB_ID, - JobStatus.RUNNING, - deadline); + awaitJobStatus(cluster, ApplicationDispatcherBootstrap.ZERO_JOB_ID, JobStatus.RUNNING); // make sure the operator is actually running BlockingJob.awaitRunning(blockId); @@ -140,11 +132,7 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc haServices.grantDispatcherLeadership(); // job is suspended, wait until it's running - awaitJobStatus( - cluster, - ApplicationDispatcherBootstrap.ZERO_JOB_ID, - JobStatus.RUNNING, - deadline); + awaitJobStatus(cluster, ApplicationDispatcherBootstrap.ZERO_JOB_ID, JobStatus.RUNNING); // unblock processing so the job can finish BlockingJob.unblock(blockId); @@ -157,7 +145,7 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc .isEqualTo(ApplicationStatus.SUCCEEDED); // the cluster should shut down automatically once the application completes - awaitClusterStopped(cluster, deadline); + awaitClusterStopped(cluster); } finally { BlockingJob.cleanUp(blockId); } @@ -165,7 +153,6 @@ public void testDispatcherRecoversAfterLosingAndRegainingLeadership() throws Exc @Test public void testDirtyJobResultRecoveryInApplicationMode() throws Exception { - final Deadline deadline = Deadline.fromNow(TIMEOUT); final Configuration configuration = new Configuration(); configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); configuration.set(DeploymentOptions.TARGET, EmbeddedExecutor.NAME); @@ -203,7 +190,7 @@ public JobResultStore getJobResultStore() { cluster.start(); // the cluster should shut down automatically once the application completes - awaitClusterStopped(cluster, deadline); + awaitClusterStopped(cluster); } FlinkAssertions.assertThatChainOfCauses(ErrorHandlingSubmissionJob.getSubmissionException()) @@ -223,7 +210,6 @@ public JobResultStore getJobResultStore() { @Test public void testSubmitFailedJobOnApplicationError() throws Exception { - final Deadline deadline = Deadline.fromNow(TIMEOUT); final JobID jobId = new JobID(); final Configuration configuration = new Configuration(); configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); @@ -251,7 +237,7 @@ public void testSubmitFailedJobOnApplicationError() throws Exception { cluster.start(); // wait until the failed job has been submitted - awaitJobStatus(cluster, jobId, JobStatus.FAILED, deadline); + awaitJobStatus(cluster, jobId, JobStatus.FAILED); final ArchivedExecutionGraph graph = cluster.getArchivedExecutionGraph(jobId).get(); @@ -272,13 +258,11 @@ public void testSubmitFailedJobOnApplicationError() throws Exception { } } - private static void awaitClusterStopped(MiniCluster cluster, Deadline deadline) - throws Exception { - CommonTestUtils.waitUntilCondition(() -> !cluster.isRunning(), deadline); + private static void awaitClusterStopped(MiniCluster cluster) throws Exception { + CommonTestUtils.waitUntilCondition(() -> !cluster.isRunning()); } - private static void awaitJobStatus( - MiniCluster cluster, JobID jobId, JobStatus status, Deadline deadline) + private static void awaitJobStatus(MiniCluster cluster, JobID jobId, JobStatus status) throws Exception { CommonTestUtils.waitUntilCondition( () -> { @@ -292,7 +276,6 @@ private static void awaitJobStatus( } throw e; } - }, - deadline); + }); } } diff --git a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceITCase.java b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceITCase.java index cc932d3c1884c..402d659194af8 100644 --- a/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceITCase.java +++ b/flink-connectors/flink-connector-rabbitmq/src/test/java/org/apache/flink/streaming/connectors/rabbitmq/RMQSourceITCase.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.serialization.SimpleStringSchema; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.execution.ExecutionState; @@ -54,7 +53,6 @@ import org.testcontainers.utility.DockerImageName; import java.io.IOException; -import java.time.Duration; import java.util.List; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; @@ -132,7 +130,6 @@ public void testStopWithSavepoint() throws Exception { info -> info.getExecutionState() == ExecutionState.RUNNING), - Deadline.fromNow(Duration.ofSeconds(10)), 5L); clusterClient @@ -156,10 +153,7 @@ public void testMessageDelivery() throws Exception { source.addSink(CountingSink.getInstance()); final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); JobID jobId = clusterClient.submitJob(jobGraph).get(); - CommonTestUtils.waitUntilCondition( - () -> CountingSink.getCount() == msgs.size(), - Deadline.fromNow(Duration.ofSeconds(30)), - 5L); + CommonTestUtils.waitUntilCondition(() -> CountingSink.getCount() == msgs.size(), 5L); clusterClient.cancel(jobId); } @@ -190,10 +184,7 @@ protected void acknowledgeSessionIDs(List sessionIds) { source.addSink(CountingSink.getInstance()); final JobGraph jobGraph = env.getStreamGraph().getJobGraph(); JobID jobId = clusterClient.submitJob(jobGraph).get(); - CommonTestUtils.waitUntilCondition( - () -> CountingSink.getCount() == msgs.size(), - Deadline.fromNow(Duration.ofSeconds(60)), - 5L); + CommonTestUtils.waitUntilCondition(() -> CountingSink.getCount() == msgs.size(), 5L); clusterClient.cancel(jobId); } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkContainerTestEnvironment.java b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkContainerTestEnvironment.java index 1fbc8ae131b66..9b78dec617687 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkContainerTestEnvironment.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkContainerTestEnvironment.java @@ -18,7 +18,6 @@ package org.apache.flink.tests.util.flink; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.testframe.environment.ClusterControllable; import org.apache.flink.connector.testframe.environment.TestEnvironment; @@ -35,7 +34,6 @@ import org.slf4j.LoggerFactory; import java.net.URL; -import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -149,8 +147,7 @@ public void triggerTaskManagerFailover(JobClient jobClient, Runnable afterFailAc flinkContainers .getRestClusterClient() .getJobDetails(jobClient.getJobID()) - .get(), - Deadline.fromNow(Duration.ofMinutes(5))); + .get()); afterFailAction.run(); }); } diff --git a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAJobRunOnMinioS3StoreITCase.java b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAJobRunOnMinioS3StoreITCase.java index bd34590d0eede..f53a801aa6fc2 100644 --- a/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAJobRunOnMinioS3StoreITCase.java +++ b/flink-filesystems/flink-s3-fs-base/src/test/java/org/apache/flink/fs/s3/common/HAJobRunOnMinioS3StoreITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.fs.s3.common; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.testutils.AllCallbackWrapper; @@ -39,7 +38,6 @@ import org.junit.jupiter.api.Order; import org.junit.jupiter.api.extension.RegisterExtension; -import java.time.Duration; import java.util.List; import static org.apache.flink.shaded.guava30.com.google.common.base.Predicates.not; @@ -130,9 +128,7 @@ protected void runAfterJobTermination() throws Exception { FileSystemJobResultStore ::hasValidDirtyJobResultStoreEntryExtension); }, - Deadline.fromNow(Duration.ofSeconds(60)), - 2000L, - "Wait for the JobResult being written to the JobResultStore."); + 2000L); final S3ObjectSummary objRef = Iterables.getOnlyElement(getObjectsFromJobResultStore()); assertThat(objRef.getKey()) diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriverTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriverTest.java index 9063f549edf62..6d09f903ca04c 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriverTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/KubernetesResourceManagerDriverTest.java @@ -18,7 +18,6 @@ package org.apache.flink.kubernetes; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.TaskManagerOptions; import org.apache.flink.kubernetes.configuration.KubernetesConfigOptions; import org.apache.flink.kubernetes.configuration.KubernetesResourceManagerDriverConfiguration; @@ -40,7 +39,6 @@ import io.fabric8.kubernetes.api.model.ResourceRequirements; import org.junit.Test; -import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -211,12 +209,7 @@ public void testNewWatchCreationWhenKubernetesTooOldResourceVersionException() new KubernetesTooOldResourceVersionException( new Exception("too old resource version"))); // Verify the old watch is closed and a new one is created - CommonTestUtils.waitUntilCondition( - () -> getPodsWatches().size() == 2, - Deadline.fromNow(Duration.ofSeconds(TIMEOUT_SEC)), - String.format( - "New watch is not created in %s seconds.", - TIMEOUT_SEC)); + CommonTestUtils.waitUntilCondition(() -> getPodsWatches().size() == 2); assertThat(getPodsWatches().get(0).isClosed(), is(true)); assertThat(getPodsWatches().get(1).isClosed(), is(false)); }); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java index 30b8875bd1ca8..f599e5c30d82b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointIDCounterTest.java @@ -203,7 +203,7 @@ public void testGetAndIncrementWithNoLeadership() throws Exception { // lost leadership getLeaderCallback().notLeader(); - electionEventHandler.waitForRevokeLeader(TIMEOUT); + electionEventHandler.waitForRevokeLeader(); getLeaderConfigMap() .getAnnotations() .remove(KubernetesLeaderElector.LEADER_ANNOTATION_KEY); @@ -284,7 +284,7 @@ public void testSetWithNoLeadershipShouldNotBeIssued() throws Exception { // lost leadership getLeaderCallback().notLeader(); - electionEventHandler.waitForRevokeLeader(TIMEOUT); + electionEventHandler.waitForRevokeLeader(); getLeaderConfigMap() .getAnnotations() .remove(KubernetesLeaderElector.LEADER_ANNOTATION_KEY); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java index 4559d175cf280..baa0603df529a 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java @@ -25,7 +25,6 @@ import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.client.program.ClusterClient; @@ -59,7 +58,6 @@ import org.junit.rules.TemporaryFolder; import java.io.IOException; -import java.time.Duration; import java.util.Random; import java.util.concurrent.TimeUnit; @@ -110,7 +108,6 @@ public void testRecoverFromSavepoint() throws Exception { // Wait until all tasks running and getting a successful savepoint CommonTestUtils.waitUntilCondition( () -> triggerSavepoint(clusterClient, jobGraph.getJobID(), savepointPath) != null, - Deadline.fromNow(TestingUtils.infiniteDuration()), 1000); // Trigger savepoint 2 @@ -121,7 +118,6 @@ public void testRecoverFromSavepoint() throws Exception { clusterClient.cancel(jobGraph.getJobID()); CommonTestUtils.waitUntilCondition( () -> clusterClient.getJobStatus(jobGraph.getJobID()).get() == JobStatus.CANCELED, - Deadline.fromNow(Duration.ofMillis(TIMEOUT)), 1000); // Start a new job with savepoint 2 diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java index 543a589a6eb3c..eddefe81a927b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityTestBase.java @@ -129,7 +129,7 @@ KubernetesConfigMap getLeaderConfigMap() { // Use the leader callback to manually grant leadership void leaderCallbackGrantLeadership() throws Exception { kubernetesTestFixture.leaderCallbackGrantLeadership(); - electionEventHandler.waitForLeader(TIMEOUT); + electionEventHandler.waitForLeader(); } FlinkKubeClient.WatchCallbackHandler diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java index 6717a3a1c619e..694e8a83c0074 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionAndRetrievalITCase.java @@ -56,8 +56,6 @@ public class KubernetesLeaderElectionAndRetrievalITCase extends TestLogger { "akka.tcp://flink@172.20.1.21:6123/user/rpc/dispatcher"; @ClassRule public static KubernetesResource kubernetesResource = new KubernetesResource(); - private static final long TIMEOUT = 120L * 1000L; - @Test public void testLeaderElectionAndRetrieval() throws Exception { final String configMapName = LEADER_CONFIGMAP_NAME + System.currentTimeMillis(); @@ -101,14 +99,14 @@ public void testLeaderElectionAndRetrieval() throws Exception { KubernetesUtils::getLeaderInformationFromConfigMap, retrievalEventHandler::handleError); - electionEventHandler.waitForLeader(TIMEOUT); + electionEventHandler.waitForLeader(); // Check the new leader is confirmed final LeaderInformation confirmedLeaderInformation = electionEventHandler.getConfirmedLeaderInformation(); assertThat(confirmedLeaderInformation.getLeaderAddress(), is(LEADER_ADDRESS)); // Check the leader retrieval driver should be notified the leader address - retrievalEventHandler.waitForNewLeader(TIMEOUT); + retrievalEventHandler.waitForNewLeader(); assertThat( retrievalEventHandler.getLeaderSessionID(), is(confirmedLeaderInformation.getLeaderSessionID())); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java index fa5cc17150844..6072051ea58e0 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderElectionDriverTest.java @@ -70,7 +70,7 @@ public void testNotLeader() throws Exception { // Revoke leadership getLeaderCallback().notLeader(); - electionEventHandler.waitForRevokeLeader(TIMEOUT); + electionEventHandler.waitForRevokeLeader(); assertThat(electionEventHandler.isLeader(), is(false)); assertThat( electionEventHandler.getConfirmedLeaderInformation(), @@ -94,7 +94,7 @@ public void testHasLeadershipWhenConfigMapNotExist() throws Exception { runTest( () -> { leaderElectionDriver.hasLeadership(); - electionEventHandler.waitForError(TIMEOUT); + electionEventHandler.waitForError(); final String errorMsg = "ConfigMap " + LEADER_CONFIGMAP_NAME + " does not exist."; assertThat(electionEventHandler.getError(), is(notNullValue())); @@ -137,7 +137,7 @@ public void testWriteLeaderInformationWhenConfigMapNotExist() throws Exception { () -> { leaderElectionDriver.writeLeaderInformation( LeaderInformation.known(UUID.randomUUID(), LEADER_ADDRESS)); - electionEventHandler.waitForError(TIMEOUT); + electionEventHandler.waitForError(); final String errorMsg = "Could not write leader information since ConfigMap " @@ -205,7 +205,7 @@ public void testLeaderConfigMapDeletedExternally() throws Exception { callbackHandler.onDeleted( Collections.singletonList(getLeaderConfigMap())); - electionEventHandler.waitForError(TIMEOUT); + electionEventHandler.waitForError(); final String errorMsg = "ConfigMap " + LEADER_CONFIGMAP_NAME + " is deleted externally"; assertThat(electionEventHandler.getError(), is(notNullValue())); @@ -230,7 +230,7 @@ public void testErrorForwarding() throws Exception { callbackHandler.onError( Collections.singletonList(getLeaderConfigMap())); - electionEventHandler.waitForError(TIMEOUT); + electionEventHandler.waitForError(); final String errorMsg = "Error while watching the ConfigMap " + LEADER_CONFIGMAP_NAME; assertThat(electionEventHandler.getError(), is(notNullValue())); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverTest.java index acd261307f35b..3193f9f15494c 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesLeaderRetrievalDriverTest.java @@ -48,7 +48,7 @@ public void testErrorForwarding() throws Exception { Collections.singletonList(getLeaderConfigMap())); final String errMsg = "Error while watching the ConfigMap " + LEADER_CONFIGMAP_NAME; - retrievalEventHandler.waitForError(TIMEOUT); + retrievalEventHandler.waitForError(); assertThat( retrievalEventHandler.getError(), FlinkMatchers.containsMessage(errMsg)); @@ -76,8 +76,7 @@ public void testKubernetesLeaderRetrievalOnModified() throws Exception { callbackHandler.onModified( Collections.singletonList(getLeaderConfigMap())); - assertThat( - retrievalEventHandler.waitForNewLeader(TIMEOUT), is(newLeader)); + assertThat(retrievalEventHandler.waitForNewLeader(), is(newLeader)); }); } }; @@ -98,7 +97,7 @@ public void testKubernetesLeaderRetrievalOnModifiedWithEmpty() throws Exception getLeaderConfigMap().getData().clear(); callbackHandler.onModified( Collections.singletonList(getLeaderConfigMap())); - retrievalEventHandler.waitForEmptyLeaderInformation(TIMEOUT); + retrievalEventHandler.waitForEmptyLeaderInformation(); assertThat(retrievalEventHandler.getAddress(), is(nullValue())); }); } diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java index c4c0a180cd33f..37d64771e8df3 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesMultipleComponentLeaderElectionDriverTest.java @@ -118,7 +118,7 @@ public void testPublishLeaderInformation() throws Exception { notifyLeaderRetrievalWatchOnModifiedConfigMap(); - leaderRetrievalListener.waitForNewLeader(10_000L); + leaderRetrievalListener.waitForNewLeader(); assertThat(leaderRetrievalListener.getLeader()) .isEqualTo(leaderInformation); }); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreITCase.java index f4467c47f7d93..5c78168953e4b 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreITCase.java @@ -51,8 +51,6 @@ public class KubernetesStateHandleStoreITCase extends TestLogger { private final FlinkKubeClientFactory kubeClientFactory = new FlinkKubeClientFactory(); - private static final long TIMEOUT = 120L * 1000L; - private static final String KEY = "state-handle-test"; @Test @@ -94,8 +92,7 @@ public void testMultipleKubernetesStateHandleStores() throws Exception { } // Wait for the leader - final String lockIdentity = - TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(TIMEOUT); + final String lockIdentity = TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(); Long expectedState = null; for (int i = 0; i < leaderNum; i++) { diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java index 51294f40b810b..c2b46c8ed808e 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesStateHandleStoreTest.java @@ -445,7 +445,7 @@ public void testReplaceWithNoLeadershipAndDiscardState() throws Exception { store.addAndLock(key, state); // Lost leadership getLeaderCallback().notLeader(); - electionEventHandler.waitForRevokeLeader(TIMEOUT); + electionEventHandler.waitForRevokeLeader(); getLeaderConfigMap() .getAnnotations() .remove(KubernetesLeaderElector.LEADER_ANNOTATION_KEY); @@ -851,7 +851,7 @@ public void testRemoveFailedShouldNotDiscardState() throws Exception { store.addAndLock(key, state); // Lost leadership getLeaderCallback().notLeader(); - electionEventHandler.waitForRevokeLeader(TIMEOUT); + electionEventHandler.waitForRevokeLeader(); getLeaderConfigMap() .getAnnotations() .remove(KubernetesLeaderElector.LEADER_ANNOTATION_KEY); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java index 940d8dcfe0235..bce3069422626 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/KubernetesLeaderElectorITCase.java @@ -42,8 +42,6 @@ public class KubernetesLeaderElectorITCase extends TestLogger { @ClassRule public static KubernetesResource kubernetesResource = new KubernetesResource(); - private static final long TIMEOUT = 120L * 1000L; - private final FlinkKubeClientFactory kubeClientFactory = new FlinkKubeClientFactory(); private static final String LEADER_CONFIGMAP_NAME_PREFIX = "leader-test-cluster"; @@ -81,17 +79,17 @@ public void testMultipleKubernetesLeaderElectors() throws Exception { // Wait for the first leader final String firstLockIdentity = - TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(TIMEOUT); + TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(); for (int i = 0; i < leaderNum; i++) { if (leaderCallbackHandlers[i].getLockIdentity().equals(firstLockIdentity)) { // Check the callback isLeader is called. - leaderCallbackHandlers[i].waitForNewLeader(TIMEOUT); + leaderCallbackHandlers[i].waitForNewLeader(); assertThat(leaderCallbackHandlers[i].hasLeadership(), is(true)); // Current leader died leaderElectors[i].stop(); // Check the callback notLeader is called. - leaderCallbackHandlers[i].waitForRevokeLeader(TIMEOUT); + leaderCallbackHandlers[i].waitForRevokeLeader(); assertThat(leaderCallbackHandlers[i].hasLeadership(), is(false)); } else { assertThat(leaderCallbackHandlers[i].hasLeadership(), is(false)); @@ -100,7 +98,7 @@ public void testMultipleKubernetesLeaderElectors() throws Exception { // Another leader should be elected successfully and update the lock identity final String anotherLockIdentity = - TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(TIMEOUT); + TestingLeaderCallbackHandler.waitUntilNewLeaderAppears(); assertThat(anotherLockIdentity, is(not(firstLockIdentity))); } finally { // Cleanup the resources diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/TestingLeaderCallbackHandler.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/TestingLeaderCallbackHandler.java index edc1e70e483fb..bdfd86ff534d2 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/TestingLeaderCallbackHandler.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/kubeclient/resources/TestingLeaderCallbackHandler.java @@ -18,14 +18,10 @@ package org.apache.flink.kubernetes.kubeclient.resources; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.testutils.CommonTestUtils; -import java.time.Duration; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; /** Testing implementation for {@link KubernetesLeaderElector.LeaderCallbackHandler}. */ public class TestingLeaderCallbackHandler extends KubernetesLeaderElector.LeaderCallbackHandler { @@ -63,38 +59,23 @@ public boolean hasLeadership() { return isLeader; } - public static String waitUntilNewLeaderAppears(long timeout) throws Exception { - final AtomicReference leaderRef = new AtomicReference<>(); - CommonTestUtils.waitUntilCondition( - () -> { - final String lockIdentity = sharedQueue.poll(timeout, TimeUnit.MILLISECONDS); - leaderRef.set(lockIdentity); - return lockIdentity != null; - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - "No leader is elected with " + timeout + "ms"); - return leaderRef.get(); + public static String waitUntilNewLeaderAppears() throws Exception { + return sharedQueue.take(); } - public void waitForNewLeader(long timeout) throws Exception { - final String errorMsg = - "No leader with " + lockIdentity + " is elected within " + timeout + "ms"; - poll(leaderQueue, timeout, errorMsg); + public void waitForNewLeader() throws Exception { + poll(leaderQueue); } - public void waitForRevokeLeader(long timeout) throws Exception { - final String errorMsg = - "No leader with " + lockIdentity + " is revoke within " + timeout + "ms"; - poll(revokeQueue, timeout, errorMsg); + public void waitForRevokeLeader() throws Exception { + poll(revokeQueue); } - private void poll(BlockingQueue queue, long timeout, String errorMsg) throws Exception { + private void poll(BlockingQueue queue) throws Exception { CommonTestUtils.waitUntilCondition( () -> { - final String lockIdentity = queue.poll(timeout, TimeUnit.MILLISECONDS); + final String lockIdentity = queue.take(); return this.lockIdentity.equals(lockIdentity); - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + }); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java index 15cc61e52de87..982df0c94c06f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/BlobServerCleanupTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.blob; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; @@ -380,10 +379,7 @@ private void runBlobServerExpiresRecoveredTransientBlob(@Nullable JobID jobId) try (final BlobServer blobServer = createTestInstance(temporaryFolder.getAbsolutePath(), cleanupInterval)) { - CommonTestUtils.waitUntilCondition( - () -> !blob.exists(), - Deadline.fromNow(Duration.ofSeconds(cleanupInterval * 5L)), - "The transient blob has not been cleaned up automatically."); + CommonTestUtils.waitUntilCondition(() -> !blob.exists()); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/PermanentBlobCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/PermanentBlobCacheTest.java index d26156888b4a3..99599af225d56 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/PermanentBlobCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/PermanentBlobCacheTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.blob; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -35,7 +34,6 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Duration; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -107,10 +105,7 @@ public void permanentBlobCacheTimesOutRecoveredBlobs(@TempDir Path storageDirect try (final PermanentBlobCache permanentBlobCache = new PermanentBlobCache( configuration, storageDirectory.toFile(), new VoidBlobStore(), null)) { - CommonTestUtils.waitUntilCondition( - () -> !blobFile.exists(), - Deadline.fromNow(Duration.ofSeconds(cleanupInterval * 5L)), - "The permanent blob file was not cleaned up automatically."); + CommonTestUtils.waitUntilCondition(() -> !blobFile.exists()); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/TransientBlobCacheTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/TransientBlobCacheTest.java index db5d6407db9cc..d29b31b6dc628 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/blob/TransientBlobCacheTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/blob/TransientBlobCacheTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.blob; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.BlobServerOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -33,7 +32,6 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; -import java.time.Duration; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -92,9 +90,7 @@ public void transientBlobCacheTimesOutRecoveredBlobs(@TempDir Path storageDirect try (final TransientBlobCache transientBlobCache = new TransientBlobCache(configuration, storageDirectory.toFile(), null)) { - CommonTestUtils.waitUntilCondition( - () -> !blobFile.exists(), - Deadline.fromNow(Duration.ofSeconds(cleanupInterval * 5L))); + CommonTestUtils.waitUntilCondition(() -> !blobFile.exists()); } } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java index 5daad82414dab..d3d705d51f719 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.jobgraph.RestoreMode; import org.apache.flink.runtime.state.RetrievableStateHandle; import org.apache.flink.runtime.state.SharedStateRegistry; @@ -42,7 +41,6 @@ import org.junit.Test; import java.io.Serializable; -import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; @@ -385,8 +383,7 @@ public void testChekpointingPausesAndResumeWhenTooManyCheckpoints() throws Excep CommonTestUtils.waitUntilCondition( () -> checkpointsCleaner.getNumberOfCheckpointsToClean() - == nbCheckpointsSubmittedForCleaning, - Deadline.fromNow(Duration.ofSeconds(3))); + == nbCheckpointsSubmittedForCleaning); assertEquals( nbCheckpointsSubmittedForCleaning, checkpointsCleaner.getNumberOfCheckpointsToClean()); @@ -403,8 +400,7 @@ public void testChekpointingPausesAndResumeWhenTooManyCheckpoints() throws Excep CommonTestUtils.waitUntilCondition( () -> checkpointsCleaner.getNumberOfCheckpointsToClean() - < nbCheckpointsSubmittedForCleaning, - Deadline.fromNow(Duration.ofSeconds(3))); + < nbCheckpointsSubmittedForCleaning); // some checkpoints were cleaned assertTrue( checkpointsCleaner.getNumberOfCheckpointsToClean() diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java index 22d32d509146b..31f034e94ed62 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java @@ -19,7 +19,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.blob.BlobServer; @@ -36,7 +35,6 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.util.TestingFatalErrorHandlerResource; import org.apache.flink.util.TestLogger; -import org.apache.flink.util.TimeUtils; import org.junit.After; import org.junit.AfterClass; @@ -69,8 +67,7 @@ public static void teardownClass() throws Exception { static void awaitStatus(DispatcherGateway dispatcherGateway, JobID jobId, JobStatus status) throws Exception { CommonTestUtils.waitUntilCondition( - () -> status.equals(dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get()), - Deadline.fromNow(TimeUtils.toDuration(TIMEOUT))); + () -> status.equals(dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get())); } @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java index e7b9c57db97d8..d305011a7e929 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java @@ -19,7 +19,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.CleanupOptions; import org.apache.flink.core.testutils.FlinkMatchers; import org.apache.flink.core.testutils.OneShotLatch; @@ -51,7 +50,6 @@ import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.TestingJobGraphStore; import org.apache.flink.runtime.testutils.TestingJobResultStore; -import org.apache.flink.util.TimeUtils; import org.apache.flink.util.concurrent.FutureUtils; import org.hamcrest.CoreMatchers; @@ -61,7 +59,6 @@ import org.junit.Before; import org.junit.Test; -import java.time.Duration; import java.util.Collections; import java.util.List; import java.util.UUID; @@ -201,9 +198,7 @@ public void testCleanupThroughRetries() throws Exception { IsEmptyCollection.empty()); CommonTestUtils.waitUntilCondition( - () -> haServices.getJobResultStore().hasJobResultEntry(jobId), - Deadline.fromNow(Duration.ofMinutes(5)), - "The JobResultStore should have this job marked as clean."); + () -> haServices.getJobResultStore().hasJobResultEntry(jobId)); } @Test @@ -233,10 +228,7 @@ public void testCleanupNotCancellable() throws Exception { toTerminate.add(dispatcher); - CommonTestUtils.waitUntilCondition( - () -> jobManagerRunnerEntry.get() != null, - Deadline.fromNow(Duration.ofSeconds(10)), - "JobManagerRunner wasn't loaded in time."); + CommonTestUtils.waitUntilCondition(() -> jobManagerRunnerEntry.get() != null); assertThat( "The JobResultStore should have this job still marked as dirty.", @@ -255,9 +247,7 @@ public void testCleanupNotCancellable() throws Exception { jobManagerRunnerCleanupFuture.complete(null); CommonTestUtils.waitUntilCondition( - () -> haServices.getJobResultStore().hasCleanJobResultEntry(jobId), - Deadline.fromNow(Duration.ofSeconds(60)), - "The JobResultStore should have this job marked as clean now."); + () -> haServices.getJobResultStore().hasCleanJobResultEntry(jobId)); } @Test @@ -349,8 +339,7 @@ public void testCleanupAfterLeadershipChange() throws Exception { leaderElectionService.isLeader(UUID.randomUUID()); CommonTestUtils.waitUntilCondition( - () -> haServices.getJobResultStore().getDirtyResults().isEmpty(), - Deadline.fromNow(TimeUtils.toDuration(TIMEOUT))); + () -> haServices.getJobResultStore().getDirtyResults().isEmpty()); assertThat( "The JobGraph is not stored in the JobGraphStore.", diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index a3d1666953847..e68c52750de81 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.operators.ResourceSpec; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.SavepointFormatType; @@ -87,7 +86,6 @@ import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; -import org.apache.flink.util.TimeUtils; import org.apache.flink.util.concurrent.FutureUtils; import org.assertj.core.api.Assertions; @@ -854,7 +852,6 @@ public void testJobStatusIsShownDuringTermination() throws Exception { .getState(); return status == JobStatus.SUSPENDED; }, - Deadline.fromNow(TimeUtils.toDuration(TIMEOUT)), 5L); } finally { // Unblock the termination of the second job diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java index 9be03d377b07e..f6fbcdde6bc1a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/JobDispatcherITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.runtime.clusterframework.ApplicationStatus; @@ -59,7 +58,6 @@ import java.io.ObjectOutputStream; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Duration; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; @@ -73,8 +71,6 @@ @ExtendWith(TestLoggerExtension.class) public class JobDispatcherITCase { - private static final Duration TIMEOUT = Duration.ofMinutes(10); - private Supplier createJobModeDispatcherResourceManagerComponentFactorySupplier( Configuration configuration) { @@ -95,7 +91,6 @@ public class JobDispatcherITCase { @Test public void testRecoverFromCheckpointAfterLosingAndRegainingLeadership(@TempDir Path tmpPath) throws Exception { - final Deadline deadline = Deadline.fromNow(TIMEOUT); final Configuration configuration = new Configuration(); configuration.set(HighAvailabilityOptions.HA_MODE, HighAvailabilityMode.ZOOKEEPER.name()); final TestingMiniClusterConfiguration clusterConfiguration = @@ -134,7 +129,7 @@ public void testRecoverFromCheckpointAfterLosingAndRegainingLeadership(@TempDir haServices.grantDispatcherLeadership(); // job is suspended, wait until it's running - awaitJobStatus(cluster, jobID, JobStatus.RUNNING, deadline); + awaitJobStatus(cluster, jobID, JobStatus.RUNNING); CommonTestUtils.waitUntilCondition( () -> @@ -142,8 +137,7 @@ public void testRecoverFromCheckpointAfterLosingAndRegainingLeadership(@TempDir .get() .getCheckpointStatsSnapshot() .getLatestRestoredCheckpoint() - != null, - deadline); + != null); } } @@ -174,8 +168,7 @@ private JobID generateAndPersistJobGraph( return jobGraph.getJobID(); } - private static void awaitJobStatus( - MiniCluster cluster, JobID jobId, JobStatus status, Deadline deadline) + private static void awaitJobStatus(MiniCluster cluster, JobID jobId, JobStatus status) throws Exception { CommonTestUtils.waitUntilCondition( () -> { @@ -189,8 +182,7 @@ private static void awaitJobStatus( } throw e; } - }, - deadline); + }); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java index 1617a3cc629bb..dde393d4fa782 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/runner/ZooKeeperDefaultDispatcherRunnerTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.dispatcher.runner; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; @@ -71,7 +70,6 @@ import java.io.File; import java.io.IOException; -import java.time.Duration; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ForkJoinPool; @@ -88,8 +86,6 @@ public class ZooKeeperDefaultDispatcherRunnerTest extends TestLogger { private static final Time TESTING_TIMEOUT = Time.seconds(10L); - private static final Duration VERIFICATION_TIMEOUT = Duration.ofSeconds(10L); - @ClassRule public static ZooKeeperResource zooKeeperResource = new ZooKeeperResource(); @ClassRule public static TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -225,9 +221,7 @@ public JobResultStore createJobResultStore() { final JobGraphStore submittedJobGraphStore = createZooKeeperJobGraphStore(client); CommonTestUtils.waitUntilCondition( - () -> submittedJobGraphStore.getJobIds().isEmpty(), - Deadline.fromNow(VERIFICATION_TIMEOUT), - 20L); + () -> submittedJobGraphStore.getJobIds().isEmpty(), 20L); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphStoreWatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphStoreWatcherTest.java index fd19143e52369..7a8210c416345 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphStoreWatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmanager/ZooKeeperJobGraphStoreWatcherTest.java @@ -19,7 +19,6 @@ package org.apache.flink.runtime.jobmanager; import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.HighAvailabilityOptions; import org.apache.flink.runtime.highavailability.zookeeper.CuratorFrameworkWithUnhandledErrorListener; @@ -41,8 +40,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.time.Duration; - import static org.hamcrest.Matchers.contains; import static org.junit.Assert.assertThat; @@ -53,8 +50,6 @@ public class ZooKeeperJobGraphStoreWatcherTest extends TestLogger { @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); - private static final Duration TIMEOUT = Duration.ofMillis(30 * 1000); - private Configuration configuration; private TestingJobGraphListener testingJobGraphListener; @@ -87,16 +82,14 @@ public void testJobGraphAddedAndRemovedShouldNotifyGraphStoreListener() throws E stateHandleStore.addAndLock("/" + jobID, jobGraph); CommonTestUtils.waitUntilCondition( - () -> testingJobGraphListener.getAddedJobGraphs().size() > 0, - Deadline.fromNow(TIMEOUT)); + () -> testingJobGraphListener.getAddedJobGraphs().size() > 0); assertThat(testingJobGraphListener.getAddedJobGraphs(), contains(jobID)); stateHandleStore.releaseAndTryRemove("/" + jobID); CommonTestUtils.waitUntilCondition( - () -> testingJobGraphListener.getRemovedJobGraphs().size() > 0, - Deadline.fromNow(TIMEOUT)); + () -> testingJobGraphListener.getRemovedJobGraphs().size() > 0); assertThat(testingJobGraphListener.getRemovedJobGraphs(), contains(jobID)); jobGraphStoreWatcher.stop(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java index f2c0864b8ff05..f61e6690f480a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.ClosureCleaner; import org.apache.flink.api.java.tuple.Tuple3; @@ -115,7 +114,6 @@ import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLogger; -import org.apache.flink.util.TimeUtils; import org.apache.flink.util.concurrent.FutureUtils; import org.hamcrest.Matchers; @@ -778,7 +776,6 @@ public void testResourceManagerBecomesUnreachableTriggersDisconnect() throws Exc jobMasterGateway.heartbeatFromResourceManager(rmResourceId); return disconnectedJobManagerFuture.isDone(); }, - Deadline.fromNow(TimeUtils.toDuration(testingTimeout)), 50L); // heartbeat timeout should trigger disconnect JobManager from ResourceManager @@ -1177,8 +1174,6 @@ private Supplier getInputSplitSupplier( private void waitUntilAllExecutionsAreScheduledOrDeployed( final JobMasterGateway jobMasterGateway) throws Exception { - final Duration duration = Duration.ofMillis(testingTimeout.toMilliseconds()); - final Deadline deadline = Deadline.fromNow(duration); CommonTestUtils.waitUntilCondition( () -> { @@ -1190,8 +1185,7 @@ private void waitUntilAllExecutionsAreScheduledOrDeployed( execution.getState() == ExecutionState.SCHEDULED || execution.getState() == ExecutionState.DEPLOYING); - }, - deadline); + }); } private static AccessExecution getFirstExecution( @@ -1946,8 +1940,7 @@ public void testJobMasterAcceptsSlotsWhenJobIsRestarting() throws Exception { CommonTestUtils.waitUntilCondition( () -> jobMasterGateway.requestJobStatus(testingTimeout).get() - == JobStatus.RUNNING, - Deadline.fromNow(TimeUtils.toDuration(testingTimeout))); + == JobStatus.RUNNING); jobMasterGateway.disconnectTaskManager( unresolvedTaskManagerLocation.getResourceID(), @@ -1956,8 +1949,7 @@ public void testJobMasterAcceptsSlotsWhenJobIsRestarting() throws Exception { CommonTestUtils.waitUntilCondition( () -> jobMasterGateway.requestJobStatus(testingTimeout).get() - == JobStatus.RESTARTING, - Deadline.fromNow(TimeUtils.toDuration(testingTimeout))); + == JobStatus.RESTARTING); assertThat( registerSlotsAtJobMaster( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java index b98029cdbc974..0406e9b34bd3a 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/slotpool/SlotPoolInteractionsTest.java @@ -18,13 +18,11 @@ package org.apache.flink.runtime.jobmaster.slotpool; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.runtime.clusterframework.types.ResourceProfile; import org.apache.flink.runtime.executiongraph.TestingComponentMainThreadExecutor; import org.apache.flink.runtime.jobmaster.SlotRequestId; import org.apache.flink.runtime.testutils.CommonTestUtils; -import org.apache.flink.testutils.TestingUtils; import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.TestLogger; @@ -96,9 +94,7 @@ public void testCancelSlotAllocationWithoutResourceManager() throws Exception { assertTrue(ExceptionUtils.stripExecutionException(e) instanceof TimeoutException); } - CommonTestUtils.waitUntilCondition( - () -> pool.getNumPendingRequests() == 0, - Deadline.fromNow(TestingUtils.TESTING_DURATION)); + CommonTestUtils.waitUntilCondition(() -> pool.getNumPendingRequests() == 0); } } @@ -123,9 +119,7 @@ public void testSlotAllocationTimeout() throws Exception { assertTrue(ExceptionUtils.stripExecutionException(e) instanceof TimeoutException); } - CommonTestUtils.waitUntilCondition( - () -> pool.getNumPendingRequests() == 0, - Deadline.fromNow(TestingUtils.TESTING_DURATION)); + CommonTestUtils.waitUntilCondition(() -> pool.getNumPendingRequests() == 0); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java index 23c689a33d401..17eb19f1b4311 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/DefaultLeaderElectionServiceTest.java @@ -27,19 +27,16 @@ import org.junit.Test; import java.util.UUID; -import java.util.concurrent.TimeoutException; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; /** Tests for {@link DefaultLeaderElectionService}. */ public class DefaultLeaderElectionServiceTest extends TestLogger { private static final String TEST_URL = "akka//user/jobmanager"; - private static final long timeout = 50L; @Test public void testOnGrantAndRevokeLeadership() throws Exception { @@ -50,7 +47,7 @@ public void testOnGrantAndRevokeLeadership() throws Exception { // grant leadership testingLeaderElectionDriver.isLeader(); - testingContender.waitForLeader(timeout); + testingContender.waitForLeader(); assertThat(testingContender.getDescription(), is(TEST_URL)); assertThat( testingContender.getLeaderSessionID(), @@ -65,7 +62,7 @@ public void testOnGrantAndRevokeLeadership() throws Exception { // revoke leadership testingLeaderElectionDriver.notLeader(); - testingContender.waitForRevokeLeader(timeout); + testingContender.waitForRevokeLeader(); assertThat(testingContender.getLeaderSessionID(), is(nullValue())); assertThat(leaderElectionService.getLeaderSessionID(), is(nullValue())); // External storage should be cleared @@ -238,7 +235,7 @@ public void testErrorForwarding() throws Exception { testingLeaderElectionDriver.onFatalError(testException); - testingContender.waitForError(timeout); + testingContender.waitForError(); assertThat(testingContender.getError(), is(notNullValue())); assertThat( testingContender.getError(), @@ -258,15 +255,6 @@ public void testErrorIsIgnoredAfterBeingStop() throws Exception { leaderElectionService.stop(); testingLeaderElectionDriver.onFatalError(testException); - - try { - testingContender.waitForError(timeout); - fail( - "We expect to have a timeout here because there's no error should be passed to contender."); - } catch (TimeoutException ex) { - // noop - } - assertThat(testingContender.getError(), is(nullValue())); }); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java index e0fcb72c67f84..f25b13069452f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/LeaderChangeClusterComponentsTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.clusterframework.ApplicationStatus; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.highavailability.nonha.embedded.EmbeddedHaServicesWithLeadershipControl; @@ -155,8 +154,7 @@ public void testReelectionOfJobMaster() throws Exception { @Test public void testTaskExecutorsReconnectToClusterWithLeadershipChange() throws Exception { - final Deadline deadline = Deadline.fromNow(TESTING_TIMEOUT); - waitUntilTaskExecutorsHaveConnected(NUM_TMS, deadline); + waitUntilTaskExecutorsHaveConnected(NUM_TMS); highAvailabilityServices.revokeResourceManagerLeadership().get(); highAvailabilityServices.grantResourceManagerLeadership(); @@ -168,16 +166,14 @@ public void testTaskExecutorsReconnectToClusterWithLeadershipChange() throws Exc .getLeaderSessionId(), is(notNullValue())); - waitUntilTaskExecutorsHaveConnected(NUM_TMS, deadline); + waitUntilTaskExecutorsHaveConnected(NUM_TMS); } - private void waitUntilTaskExecutorsHaveConnected(int numTaskExecutors, Deadline deadline) - throws Exception { + private void waitUntilTaskExecutorsHaveConnected(int numTaskExecutors) throws Exception { CommonTestUtils.waitUntilCondition( () -> miniCluster.requestClusterOverview().get().getNumTaskManagersConnected() == numTaskExecutors, - deadline, 10L); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java index 82f3893494f4f..ae715a3ee8397 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/StandaloneLeaderElectionTest.java @@ -46,13 +46,13 @@ public void testStandaloneLeaderElectionRetrieval() throws Exception { leaderElectionService.start(contender); leaderRetrievalService.start(testingListener); - contender.waitForLeader(1000l); + contender.waitForLeader(); assertTrue(contender.isLeader()); assertEquals( HighAvailabilityServices.DEFAULT_LEADER_ID, contender.getLeaderSessionID()); - testingListener.waitForNewLeader(1000l); + testingListener.waitForNewLeader(); assertEquals(TEST_URL, testingListener.getAddress()); assertEquals( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java index a18019f52ea6c..f11cd58572202 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingLeaderBase.java @@ -18,16 +18,13 @@ package org.apache.flink.runtime.leaderelection; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.util.ExceptionUtils; import javax.annotation.Nullable; -import java.time.Duration; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; /** * Base class which provides some convenience functions for testing purposes of {@link @@ -41,47 +38,32 @@ public class TestingLeaderBase { private boolean isLeader = false; private Throwable error; - public void waitForLeader(long timeout) throws Exception { + public void waitForLeader() throws Exception { throwExceptionIfNotNull(); - final String errorMsg = "Contender was not elected as the leader within " + timeout + "ms"; CommonTestUtils.waitUntilCondition( () -> { - final LeaderInformation leader = - leaderEventQueue.poll(timeout, TimeUnit.MILLISECONDS); - return leader != null && !leader.isEmpty(); - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + final LeaderInformation leader = leaderEventQueue.take(); + return !leader.isEmpty(); + }); isLeader = true; } - public void waitForRevokeLeader(long timeout) throws Exception { + public void waitForRevokeLeader() throws Exception { throwExceptionIfNotNull(); - final String errorMsg = "Contender was not revoked within " + timeout + "ms"; CommonTestUtils.waitUntilCondition( () -> { - final LeaderInformation leader = - leaderEventQueue.poll(timeout, TimeUnit.MILLISECONDS); - return leader != null && leader.isEmpty(); - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + final LeaderInformation leader = leaderEventQueue.take(); + return leader.isEmpty(); + }); isLeader = false; } - public void waitForError(long timeout) throws Exception { - final String errorMsg = "Contender did not see an exception with " + timeout + "ms"; - CommonTestUtils.waitUntilCondition( - () -> { - error = errorQueue.poll(timeout, TimeUnit.MILLISECONDS); - return error != null; - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + public void waitForError() throws Exception { + error = errorQueue.take(); } public void handleError(Throwable ex) { @@ -95,7 +77,7 @@ public void handleError(Throwable ex) { */ @Nullable public Throwable getError() { - return this.error; + return error == null ? errorQueue.poll() : error; } public boolean isLeader() { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java index 449eba9162939..958557ad2fb1e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/TestingRetrievalBase.java @@ -18,18 +18,15 @@ package org.apache.flink.runtime.leaderelection; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.leaderretrieval.LeaderRetrievalListener; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.util.ExceptionUtils; import javax.annotation.Nullable; -import java.time.Duration; import java.util.UUID; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.TimeUnit; /** * Base class which provides some convenience functions for testing purposes of {@link @@ -45,51 +42,34 @@ public class TestingRetrievalBase { private String oldAddress; private Throwable error; - public String waitForNewLeader(long timeout) throws Exception { + public String waitForNewLeader() throws Exception { throwExceptionIfNotNull(); - final String errorMsg = - "Listener was not notified about a new leader within " + timeout + "ms"; CommonTestUtils.waitUntilCondition( () -> { - leader = leaderEventQueue.poll(timeout, TimeUnit.MILLISECONDS); - return leader != null - && !leader.isEmpty() - && !leader.getLeaderAddress().equals(oldAddress); - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + leader = leaderEventQueue.take(); + return !leader.isEmpty() && !leader.getLeaderAddress().equals(oldAddress); + }); oldAddress = leader.getLeaderAddress(); return leader.getLeaderAddress(); } - public void waitForEmptyLeaderInformation(long timeout) throws Exception { + public void waitForEmptyLeaderInformation() throws Exception { throwExceptionIfNotNull(); - final String errorMsg = - "Listener was not notified about an empty leader within " + timeout + "ms"; CommonTestUtils.waitUntilCondition( () -> { - leader = leaderEventQueue.poll(timeout, TimeUnit.MILLISECONDS); - return leader != null && leader.isEmpty(); - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + leader = leaderEventQueue.take(); + return leader.isEmpty(); + }); oldAddress = null; } - public void waitForError(long timeout) throws Exception { - final String errorMsg = "Listener did not see an exception with " + timeout + "ms"; - CommonTestUtils.waitUntilCondition( - () -> { - error = errorQueue.poll(timeout, TimeUnit.MILLISECONDS); - return error != null; - }, - Deadline.fromNow(Duration.ofMillis(timeout)), - errorMsg); + public void waitForError() throws Exception { + error = errorQueue.take(); } public void handleError(Throwable ex) { @@ -123,7 +103,7 @@ public int getLeaderEventQueueSize() { */ @Nullable public Throwable getError() { - return this.error; + return error == null ? errorQueue.poll() : error; } private void throwExceptionIfNotNull() throws Exception { diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java index f6ae64ba89f4e..c101c3dc49212 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperLeaderElectionTest.java @@ -69,7 +69,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -164,12 +163,12 @@ public void testZooKeeperLeaderElectionRetrieval() throws Exception { .createLeaderRetrievalDriver( retrievalEventHandler, retrievalEventHandler::handleError); - electionEventHandler.waitForLeader(timeout); + electionEventHandler.waitForLeader(); final LeaderInformation confirmedLeaderInformation = electionEventHandler.getConfirmedLeaderInformation(); assertThat(confirmedLeaderInformation.getLeaderAddress(), is(LEADER_ADDRESS)); - retrievalEventHandler.waitForNewLeader(timeout); + retrievalEventHandler.waitForNewLeader(); assertThat( retrievalEventHandler.getLeaderSessionID(), @@ -233,7 +232,7 @@ public void testZooKeeperReelection() throws Exception { while (deadline.hasTimeLeft() && numberSeenLeaders < num) { LOG.debug("Wait for new leader #{}.", numberSeenLeaders); - String address = listener.waitForNewLeader(deadline.timeLeft().toMillis()); + String address = listener.waitForNewLeader(); Matcher m = regex.matcher(address); @@ -320,7 +319,7 @@ public void testZooKeeperReelectionWithReplacement() throws Exception { Pattern regex = Pattern.compile(pattern); for (int i = 0; i < numTries; i++) { - listener.waitForNewLeader(timeout); + listener.waitForNewLeader(); String address = listener.getAddress(); @@ -388,7 +387,7 @@ public void testLeaderShouldBeCorrectedWhenOverwritten() throws Exception { createAndInitLeaderElectionDriver( curatorFrameworkWrapper.asCuratorFramework(), electionEventHandler); - electionEventHandler.waitForLeader(timeout); + electionEventHandler.waitForLeader(); final LeaderInformation confirmedLeaderInformation = electionEventHandler.getConfirmedLeaderInformation(); assertThat(confirmedLeaderInformation.getLeaderAddress(), is(LEADER_ADDRESS)); @@ -436,8 +435,8 @@ public void testLeaderShouldBeCorrectedWhenOverwritten() throws Exception { .createLeaderRetrievalDriver( retrievalEventHandler, retrievalEventHandler::handleError); - if (retrievalEventHandler.waitForNewLeader(timeout).equals(faultyContenderUrl)) { - retrievalEventHandler.waitForNewLeader(timeout); + if (retrievalEventHandler.waitForNewLeader().equals(faultyContenderUrl)) { + retrievalEventHandler.waitForNewLeader(); } assertThat( @@ -491,7 +490,7 @@ public void testExceptionForwarding() throws Exception { leaderElectionDriver = createAndInitLeaderElectionDriver(client, electionEventHandler); - electionEventHandler.waitForError(timeout); + electionEventHandler.waitForError(); assertNotNull(electionEventHandler.getError()); assertThat( @@ -556,9 +555,9 @@ public void testEphemeralZooKeeperNodes() throws Exception { cache.getListenable().addListener(existsListener); cache.start(); - electionEventHandler.waitForLeader(timeout); + electionEventHandler.waitForLeader(); - retrievalEventHandler.waitForNewLeader(timeout); + retrievalEventHandler.waitForNewLeader(); Future existsFuture = existsListener.nodeExists(); @@ -576,15 +575,7 @@ public void testEphemeralZooKeeperNodes() throws Exception { // make sure that the leader node has been deleted deletedFuture.get(timeout, TimeUnit.MILLISECONDS); - try { - retrievalEventHandler.waitForNewLeader(1000L); - - fail( - "TimeoutException was expected because there is no leader registered and " - + "thus there shouldn't be any leader information in ZooKeeper."); - } catch (TimeoutException e) { - // that was expected - } + retrievalEventHandler.waitForEmptyLeaderInformation(); } finally { electionEventHandler.close(); if (leaderRetrievalDriver != null) { @@ -616,14 +607,14 @@ public void testNotLeaderShouldNotCleanUpTheLeaderInformation() throws Exception createAndInitLeaderElectionDriver( curatorFrameworkWrapper.asCuratorFramework(), electionEventHandler); - electionEventHandler.waitForLeader(timeout); + electionEventHandler.waitForLeader(); final LeaderInformation confirmedLeaderInformation = electionEventHandler.getConfirmedLeaderInformation(); assertThat(confirmedLeaderInformation.getLeaderAddress(), is(LEADER_ADDRESS)); // Leader is revoked leaderElectionDriver.notLeader(); - electionEventHandler.waitForRevokeLeader(timeout); + electionEventHandler.waitForRevokeLeader(); assertThat( electionEventHandler.getConfirmedLeaderInformation(), is(LeaderInformation.empty())); @@ -634,7 +625,7 @@ public void testNotLeaderShouldNotCleanUpTheLeaderInformation() throws Exception .createLeaderRetrievalDriver( retrievalEventHandler, retrievalEventHandler::handleError); - retrievalEventHandler.waitForNewLeader(timeout); + retrievalEventHandler.waitForNewLeader(); assertThat( retrievalEventHandler.getLeaderSessionID(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java index cc0a62cd05046..80325506733de 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderelection/ZooKeeperMultipleComponentLeaderElectionDriverTest.java @@ -112,7 +112,7 @@ public void testPublishLeaderInformation() throws Exception { leaderElectionDriver.publishLeaderInformation( componentId, leaderInformation); - leaderRetrievalListener.waitForNewLeader(10_000L); + leaderRetrievalListener.waitForNewLeader(); assertThat(leaderRetrievalListener.getLeader()) .isEqualTo(leaderInformation); @@ -146,12 +146,12 @@ public void testPublishEmptyLeaderInformation() throws Exception { componentId, LeaderInformation.known(UUID.randomUUID(), "foobar")); - leaderRetrievalListener.waitForNewLeader(10_000L); + leaderRetrievalListener.waitForNewLeader(); leaderElectionDriver.publishLeaderInformation( componentId, LeaderInformation.empty()); - leaderRetrievalListener.waitForEmptyLeaderInformation(10_000L); + leaderRetrievalListener.waitForEmptyLeaderInformation(); assertThat(leaderRetrievalListener.getLeader()) .isEqualTo(LeaderInformation.empty()); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java index e2e83aa985b13..0e751e4db29e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/DefaultLeaderRetrievalServiceTest.java @@ -28,19 +28,16 @@ import org.junit.Test; import java.util.UUID; -import java.util.concurrent.TimeoutException; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; -import static org.junit.Assert.fail; /** Tests for {@link DefaultLeaderElectionService}. */ public class DefaultLeaderRetrievalServiceTest extends TestLogger { private static final String TEST_URL = "akka//user/jobmanager"; - private static final long timeout = 50L; @Test public void testNotifyLeaderAddress() throws Exception { @@ -51,7 +48,7 @@ public void testNotifyLeaderAddress() throws Exception { final LeaderInformation newLeader = LeaderInformation.known(UUID.randomUUID(), TEST_URL); testingLeaderRetrievalDriver.onUpdate(newLeader); - testingListener.waitForNewLeader(timeout); + testingListener.waitForNewLeader(); assertThat( testingListener.getLeaderSessionID(), is(newLeader.getLeaderSessionID())); @@ -71,10 +68,10 @@ public void testNotifyLeaderAddressEmpty() throws Exception { final LeaderInformation newLeader = LeaderInformation.known(UUID.randomUUID(), TEST_URL); testingLeaderRetrievalDriver.onUpdate(newLeader); - testingListener.waitForNewLeader(timeout); + testingListener.waitForNewLeader(); testingLeaderRetrievalDriver.onUpdate(LeaderInformation.empty()); - testingListener.waitForEmptyLeaderInformation(timeout); + testingListener.waitForEmptyLeaderInformation(); assertThat(testingListener.getLeaderSessionID(), is(nullValue())); assertThat(testingListener.getAddress(), is(nullValue())); }); @@ -92,7 +89,7 @@ public void testErrorForwarding() throws Exception { testingLeaderRetrievalDriver.onFatalError(testException); - testingListener.waitForError(timeout); + testingListener.waitForError(); assertThat( testingListener.getError(), FlinkMatchers.containsCause(testException)); @@ -112,13 +109,6 @@ public void testErrorIsIgnoredAfterBeingStop() throws Exception { leaderRetrievalService.stop(); testingLeaderRetrievalDriver.onFatalError(testException); - try { - testingListener.waitForError(timeout); - fail( - "We expect to have a timeout here because there's no error should be passed to listener."); - } catch (TimeoutException ex) { - // noop - } assertThat(testingListener.getError(), is(nullValue())); }); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/SettableLeaderRetrievalServiceTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/SettableLeaderRetrievalServiceTest.java index 9ca315324f8f1..c2e329ef12b85 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/SettableLeaderRetrievalServiceTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/SettableLeaderRetrievalServiceTest.java @@ -25,8 +25,6 @@ import org.junit.Before; import org.junit.Test; -import java.time.Duration; - import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; @@ -34,7 +32,6 @@ public class SettableLeaderRetrievalServiceTest extends TestLogger { private SettableLeaderRetrievalService settableLeaderRetrievalService; - private static final Duration TIMEOUT = Duration.ofHours(1); @Before public void setUp() { @@ -50,7 +47,7 @@ public void testNotifyListenerLater() throws Exception { final TestingListener listener = new TestingListener(); settableLeaderRetrievalService.start(listener); - listener.waitForNewLeader(TIMEOUT.toMillis()); + listener.waitForNewLeader(); assertThat(listener.getAddress(), equalTo(localhost)); assertThat( listener.getLeaderSessionID(), equalTo(HighAvailabilityServices.DEFAULT_LEADER_ID)); @@ -65,7 +62,7 @@ public void testNotifyListenerImmediately() throws Exception { settableLeaderRetrievalService.notifyListener( localhost, HighAvailabilityServices.DEFAULT_LEADER_ID); - listener.waitForNewLeader(TIMEOUT.toMillis()); + listener.waitForNewLeader(); assertThat(listener.getAddress(), equalTo(localhost)); assertThat( listener.getLeaderSessionID(), equalTo(HighAvailabilityServices.DEFAULT_LEADER_ID)); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalConnectionHandlingTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalConnectionHandlingTest.java index f705103fb94ba..e8cda90a0635d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalConnectionHandlingTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/leaderretrieval/ZooKeeperLeaderRetrievalConnectionHandlingTest.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.leaderretrieval; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.core.testutils.EachCallbackWrapper; import org.apache.flink.runtime.leaderelection.LeaderInformation; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -279,8 +278,7 @@ public void testNewLeaderAfterReconnectTriggersListenerNotification() throws Exc && afterConnectionReconnect .getLeaderAddress() .equals(newLeaderAddress); - }, - Deadline.fromNow(Duration.ofSeconds(30L))); + }); }); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java index 6973722ccc2a3..adc8a2ee6899e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.configuration.WebOptions; @@ -188,8 +187,7 @@ public void testCheckpointStatsPersistedAcrossRescale() throws Exception { .getCounts() .getNumberOfCompletedCheckpoints() > 0) - .get(), - Deadline.fromNow(Duration.ofHours(1))); + .get()); miniCluster.terminateTaskManager(0); @@ -281,7 +279,6 @@ private void waitUntilParallelismForVertexReached( } return executionJobVertex.getParallelism() == targetParallelism; - }, - Deadline.fromNow(Duration.ofMinutes(5))); + }); } } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerSimpleITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerSimpleITCase.java index 992d2c47ab4a5..b4b66ed29e1e5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerSimpleITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerSimpleITCase.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.execution.Environment; @@ -46,7 +45,6 @@ import java.io.IOException; import java.time.Duration; -import java.time.temporal.ChronoUnit; import static org.junit.Assert.assertTrue; @@ -129,7 +127,6 @@ public void testJobCancellationWhileRestartingSucceeds() throws Exception { // wait until we are in RESTARTING state CommonTestUtils.waitUntilCondition( () -> miniCluster.getJobStatus(jobGraph.getJobID()).get() == JobStatus.RESTARTING, - Deadline.fromNow(Duration.of(timeInRestartingState, ChronoUnit.MILLIS)), 5); // now cancel while in RESTARTING state diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java index e980a51215b23..cbcc99c955650 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.execution.ExecutionState; import org.apache.flink.runtime.executiongraph.AccessExecution; @@ -46,7 +45,6 @@ import org.junit.Test; import java.io.IOException; -import java.time.Duration; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.function.Predicate; @@ -58,7 +56,6 @@ /** Integration tests for the {@link TaskExecutor}. */ public class TaskExecutorITCase extends TestLogger { - private static final Duration TESTING_TIMEOUT = Duration.ofMinutes(2L); private static final int NUM_TMS = 2; private static final int SLOTS_PER_TM = 2; private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; @@ -137,9 +134,7 @@ private CompletableFuture submitJobAndWaitUntilRunning(JobGraph jobGr assertThat(jobResultFuture.isDone(), is(false)); CommonTestUtils.waitUntilCondition( - jobIsRunning(() -> miniCluster.getExecutionGraph(jobGraph.getJobID())), - Deadline.fromNow(TESTING_TIMEOUT), - 50L); + jobIsRunning(() -> miniCluster.getExecutionGraph(jobGraph.getJobID())), 50L); return jobResultFuture; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java index 2963e7f8e97a0..a2852c2a9738e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/taskexecutor/TaskExecutorTest.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.resources.CPUResource; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.configuration.ConfigConstants; @@ -489,7 +488,6 @@ public void testResourceManagerBecomesUnreachableTriggersDisconnect() throws Exc taskExecutorGateway.heartbeatFromResourceManager(rmResourceId); return taskExecutorDisconnectFuture.isDone(); }, - Deadline.fromNow(TimeUtils.toDuration(timeout)), 50L)); } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java index 4392a4ba67bcc..d36100be22eee 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/CommonTestUtils.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.core.execution.JobClient; import org.apache.flink.runtime.checkpoint.CompletedCheckpointStats; import org.apache.flink.runtime.execution.ExecutionState; @@ -43,8 +42,6 @@ import java.io.StringWriter; import java.lang.management.ManagementFactory; import java.lang.management.RuntimeMXBean; -import java.time.Duration; -import java.time.temporal.ChronoUnit; import java.util.Arrays; import java.util.Collection; import java.util.List; @@ -136,40 +133,16 @@ public static void printLog4jDebugConfig(File file) throws IOException { } } - public static void waitUntilCondition( - SupplierWithException condition, Deadline timeout) - throws Exception { - waitUntilCondition(condition, timeout, RETRY_INTERVAL); - } - - public static void waitUntilCondition( - SupplierWithException condition, - Deadline timeout, - long retryIntervalMillis) - throws Exception { - waitUntilCondition( - condition, timeout, retryIntervalMillis, "Condition was not met in given timeout."); - } - - public static void waitUntilCondition( - SupplierWithException condition, Deadline timeout, String errorMsg) + public static void waitUntilCondition(SupplierWithException condition) throws Exception { - waitUntilCondition(condition, timeout, RETRY_INTERVAL, errorMsg); + waitUntilCondition(condition, RETRY_INTERVAL); } public static void waitUntilCondition( - SupplierWithException condition, - Deadline timeout, - long retryIntervalMillis, - String errorMsg) + SupplierWithException condition, long retryIntervalMillis) throws Exception { - while (timeout.hasTimeLeft() && !condition.get()) { - final long timeLeft = Math.max(0, timeout.timeLeft().toMillis()); - Thread.sleep(Math.min(retryIntervalMillis, timeLeft)); - } - - if (!timeout.hasTimeLeft()) { - throw new TimeoutException(errorMsg); + while (!condition.get()) { + Thread.sleep(retryIntervalMillis); } } @@ -187,17 +160,6 @@ public static void waitForAllTaskRunning( SupplierWithException executionGraphSupplier, boolean allowFinished) throws Exception { - waitForAllTaskRunning( - executionGraphSupplier, - Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES)), - allowFinished); - } - - public static void waitForAllTaskRunning( - SupplierWithException executionGraphSupplier, - Deadline timeout, - boolean allowFinished) - throws Exception { Predicate subtaskPredicate = task -> { switch (task.getExecutionState()) { @@ -230,13 +192,11 @@ public static void waitForAllTaskRunning( jobVertex -> Arrays.stream(jobVertex.getTaskVertices()) .allMatch(subtaskPredicate)); - }, - timeout); + }); } public static void waitForAllTaskRunning( - SupplierWithException jobDetailsSupplier, Deadline timeout) - throws Exception { + SupplierWithException jobDetailsSupplier) throws Exception { waitUntilCondition( () -> { final JobDetailsInfo jobDetailsInfo = jobDetailsSupplier.get(); @@ -254,39 +214,27 @@ public static void waitForAllTaskRunning( } } return true; - }, - timeout, - "Some tasks are not running until timeout"); + }); } public static void waitForNoTaskRunning( - SupplierWithException jobDetailsSupplier, Deadline timeout) - throws Exception { + SupplierWithException jobDetailsSupplier) throws Exception { waitUntilCondition( () -> { final Map state = jobDetailsSupplier.get().getJobVerticesPerState(); final Integer numRunningTasks = state.get(ExecutionState.RUNNING); return numRunningTasks == null || numRunningTasks.equals(0); - }, - timeout, - "Some tasks are still running until timeout"); + }); } public static void waitUntilJobManagerIsInitialized( SupplierWithException jobStatusSupplier) throws Exception { - waitUntilJobManagerIsInitialized( - jobStatusSupplier, Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES))); + waitUntilCondition(() -> jobStatusSupplier.get() != JobStatus.INITIALIZING, 20L); } - public static void waitUntilJobManagerIsInitialized( - SupplierWithException jobStatusSupplier, Deadline timeout) + public static void waitForJobStatus(JobClient client, List expectedStatus) throws Exception { - waitUntilCondition(() -> jobStatusSupplier.get() != JobStatus.INITIALIZING, timeout, 20L); - } - - public static void waitForJobStatus( - JobClient client, List expectedStatus, Deadline deadline) throws Exception { waitUntilCondition( () -> { final JobStatus currentStatus = client.getJobStatus().get(); @@ -316,8 +264,7 @@ public static void waitForJobStatus( // Continue waiting for expected status return false; - }, - deadline); + }); } public static void terminateJob(JobClient client) throws Exception { @@ -360,13 +307,11 @@ public static void waitForSubtasksToFinish( return allSubtasks ? vertexStream.allMatch(subtaskPredicate) : vertexStream.anyMatch(subtaskPredicate); - }, - Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES))); + }); } /** Wait for (at least) the given number of successful checkpoints. */ - public static void waitForCheckpoint( - JobID jobID, MiniCluster miniCluster, Deadline timeout, int numCheckpoints) + public static void waitForCheckpoint(JobID jobID, MiniCluster miniCluster, int numCheckpoints) throws Exception, FlinkJobNotFoundException { waitUntilCondition( () -> { @@ -388,8 +333,7 @@ public static void waitForCheckpoint( } else { return false; } - }, - timeout); + }); } /** diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java index f1971e802edb7..2e24b91d407cf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/MiniClusterResource.java @@ -18,7 +18,6 @@ package org.apache.flink.runtime.testutils; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.configuration.HeartbeatManagerOptions; @@ -43,7 +42,6 @@ import org.slf4j.LoggerFactory; import java.net.URI; -import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -128,39 +126,25 @@ public void cancelAllJobs() { private void cancelAllJobs(boolean waitUntilSlotsAreFreed) { try { - final long shutdownTimeout = - miniClusterResourceConfiguration.getShutdownTimeout().toMilliseconds(); - final Deadline jobCancellationDeadline = - Deadline.fromNow(Duration.ofMillis(shutdownTimeout)); final List> jobCancellationFutures = - miniCluster.listJobs() - .get( - jobCancellationDeadline.timeLeft().toMillis(), - TimeUnit.MILLISECONDS) - .stream() + miniCluster.listJobs().get().stream() .filter(status -> !status.getJobState().isGloballyTerminalState()) .map(status -> miniCluster.cancelJob(status.getJobId())) .collect(Collectors.toList()); - FutureUtils.waitForAll(jobCancellationFutures) - .get(jobCancellationDeadline.timeLeft().toMillis(), TimeUnit.MILLISECONDS); + FutureUtils.waitForAll(jobCancellationFutures).get(); CommonTestUtils.waitUntilCondition( () -> { final long unfinishedJobs = - miniCluster.listJobs() - .get( - jobCancellationDeadline.timeLeft().toMillis(), - TimeUnit.MILLISECONDS) - .stream() + miniCluster.listJobs().get().stream() .filter( status -> !status.getJobState() .isGloballyTerminalState()) .count(); return unfinishedJobs == 0; - }, - jobCancellationDeadline); + }); if (waitUntilSlotsAreFreed) { CommonTestUtils.waitUntilCondition( @@ -169,8 +153,7 @@ private void cancelAllJobs(boolean waitUntilSlotsAreFreed) { miniCluster.getResourceOverview().get(); return resourceOverview.getNumberRegisteredSlots() == resourceOverview.getNumberFreeSlots(); - }, - jobCancellationDeadline); + }); } } catch (Exception e) { log.warn("Exception while shutting down remaining jobs.", e); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskFinalCheckpointsTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskFinalCheckpointsTest.java index c85218d673d0e..08ab5d99e6025 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskFinalCheckpointsTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskFinalCheckpointsTest.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.core.execution.SavepointFormatType; @@ -62,7 +61,6 @@ import javax.annotation.Nullable; -import java.time.Duration; import java.util.Collections; import java.util.concurrent.CompletableFuture; import java.util.concurrent.Future; @@ -886,8 +884,7 @@ public void testWaitingForUnalignedChannelStatesIfFinishedOnRestore() throws Exc CommonTestUtils.waitUntilCondition( () -> checkpointResponder.getAcknowledgeLatch().isTriggered() - || checkpointResponder.getDeclinedLatch().isTriggered(), - Deadline.fromNow(Duration.ofSeconds(10))); + || checkpointResponder.getDeclinedLatch().isTriggered()); assertEquals( Collections.singletonList(2L), diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java index 1aae9f3d077fa..f6fe6b6a567ea 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliClientTest.java @@ -18,7 +18,6 @@ package org.apache.flink.table.client.cli; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.cli.DefaultCLI; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; @@ -64,7 +63,6 @@ import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -327,8 +325,7 @@ public void testCancelExecutionInteractiveMode() throws Exception { client.getTerminal().raise(Terminal.Signal.INT); CommonTestUtils.waitUntilCondition( - () -> outputStream.toString().contains("'key' = 'value'"), - Deadline.fromNow(Duration.ofMillis(10000))); + () -> outputStream.toString().contains("'key' = 'value'")); } } diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java index c0a5b1f095830..02b6287763476 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java @@ -19,7 +19,6 @@ package org.apache.flink.table.client.cli; import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.table.api.DataTypes; @@ -45,7 +44,6 @@ import java.io.ByteArrayOutputStream; import java.math.BigDecimal; import java.sql.Timestamp; -import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.concurrent.ExecutorService; @@ -243,9 +241,7 @@ public void testCancelBatchResult() throws Exception { // wait until we trying to get batch result CommonTestUtils.waitUntilCondition( - () -> mockExecutor.getNumRetrieveResultChancesCalls() > 1, - Deadline.now().plus(Duration.ofSeconds(5)), - 50L); + () -> mockExecutor.getNumRetrieveResultChancesCalls() > 1, 50L); // send signal to cancel terminal.raise(Terminal.Signal.INT); @@ -442,9 +438,7 @@ public void testCancelStreamingResult() throws Exception { // wait until we processed first result CommonTestUtils.waitUntilCondition( - () -> mockExecutor.getNumRetrieveResultChancesCalls() > 1, - Deadline.now().plus(Duration.ofSeconds(5)), - 50L); + () -> mockExecutor.getNumRetrieveResultChancesCalls() > 1, 50L); // send signal to cancel terminal.raise(Terminal.Signal.INT); diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/environment/MiniClusterTestEnvironment.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/environment/MiniClusterTestEnvironment.java index 0ecbe7f085c6c..e68560ada54b0 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/environment/MiniClusterTestEnvironment.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/environment/MiniClusterTestEnvironment.java @@ -19,7 +19,6 @@ package org.apache.flink.connector.testframe.environment; import org.apache.flink.annotation.Experimental; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; @@ -38,7 +37,6 @@ import java.net.URI; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Duration; import java.util.Collections; import java.util.Optional; import java.util.concurrent.ExecutionException; @@ -132,8 +130,7 @@ public void triggerTaskManagerFailover(JobClient jobClient, Runnable afterFailAc throws Exception { terminateTaskManager(); CommonTestUtils.waitForNoTaskRunning( - () -> miniCluster.getRestClusterClient().getJobDetails(jobClient.getJobID()).get(), - Deadline.fromNow(Duration.ofMinutes(5))); + () -> miniCluster.getRestClusterClient().getJobDetails(jobClient.getJobID()).get()); afterFailAction.run(); startTaskManager(); } diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SinkTestSuiteBase.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SinkTestSuiteBase.java index e958e70f71f23..df30164097fae 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SinkTestSuiteBase.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SinkTestSuiteBase.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.connector.sink2.Sink; import org.apache.flink.api.connector.source.Boundedness; @@ -80,7 +79,6 @@ import java.util.stream.Collectors; import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; -import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_JOB_STATUS_CHANGE_TIMEOUT; import static org.apache.flink.connector.testframe.utils.MetricQuerier.getJobDetails; import static org.apache.flink.runtime.testutils.CommonTestUtils.terminateJob; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; @@ -152,10 +150,7 @@ public void testBasicSink( .name("sinkInSinkTest"); final JobClient jobClient = execEnv.executeAsync("DataStream Sink Test"); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.FINISHED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); // Check test result checkResultWithSemantic( @@ -281,8 +276,7 @@ private void restartFromSavepoint( getJobDetails( new RestClient(new Configuration(), executorService), testEnv.getRestEndpoint(), - jobClient.getJobID()), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + jobClient.getJobID())); waitExpectedSizeData(iterator, numBeforeSuccess); @@ -291,10 +285,7 @@ private void restartFromSavepoint( .stopWithSavepoint( true, testEnv.getCheckpointUri(), SavepointFormatType.CANONICAL) .get(30, TimeUnit.SECONDS); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.FINISHED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); } catch (Exception e) { executorService.shutdown(); killJob(jobClient); @@ -392,8 +383,7 @@ public void testMetrics( getJobDetails( new RestClient(new Configuration(), executorService), testEnv.getRestEndpoint(), - jobClient.getJobID()), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + jobClient.getJobID())); waitUntilCondition( () -> { @@ -411,8 +401,7 @@ public void testMetrics( // skip failed assert try return false; } - }, - Deadline.fromNow(DEFAULT_COLLECT_DATA_TIMEOUT)); + }); } finally { // Clean up executorService.shutdown(); @@ -521,8 +510,7 @@ private void checkResultWithSemantic( } catch (Throwable t) { return false; } - }, - Deadline.fromNow(DEFAULT_COLLECT_DATA_TIMEOUT)); + }); } /** Compare the metrics. */ @@ -562,10 +550,7 @@ private TestingSinkSettings getTestingSinkSettings(CheckpointingMode checkpointi private void killJob(JobClient jobClient) throws Exception { terminateJob(jobClient); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.CANCELED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.CANCELED)); } private DataStreamSink tryCreateSink( diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SourceTestSuiteBase.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SourceTestSuiteBase.java index a90d05bc135a0..b345b4114e53e 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SourceTestSuiteBase.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/testsuites/SourceTestSuiteBase.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; @@ -77,7 +76,6 @@ import java.util.concurrent.TimeUnit; import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; -import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_JOB_STATUS_CHANGE_TIMEOUT; import static org.apache.flink.connector.testframe.utils.MetricQuerier.getJobDetails; import static org.apache.flink.runtime.testutils.CommonTestUtils.terminateJob; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; @@ -159,10 +157,7 @@ public void testSourceSingleSplit( } // Step 5: Clean up - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.FINISHED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); } /** @@ -346,10 +341,7 @@ private void restartFromSavepoint( .stopWithSavepoint( true, testEnv.getCheckpointUri(), SavepointFormatType.CANONICAL) .get(30, TimeUnit.SECONDS); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.FINISHED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); // Step 5: Generate new test data final List> newTestRecordCollections = new ArrayList<>(); @@ -380,10 +372,7 @@ private void restartFromSavepoint( final JobClient restartJobClient = restartEnv.executeAsync("Restart Test"); - waitForJobStatus( - restartJobClient, - Collections.singletonList(JobStatus.RUNNING), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(restartJobClient, Collections.singletonList(JobStatus.RUNNING)); try { iterator.setJobClient(restartJobClient); @@ -458,8 +447,7 @@ public void testSourceMetrics( getJobDetails( new RestClient(new Configuration(), executorService), testEnv.getRestEndpoint(), - jobClient.getJobID()), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + jobClient.getJobID())); waitUntilCondition( () -> { @@ -475,8 +463,7 @@ public void testSourceMetrics( // skip failed assert try return false; } - }, - Deadline.fromNow(DEFAULT_COLLECT_DATA_TIMEOUT)); + }); } finally { // Clean up executorService.shutdown(); @@ -539,10 +526,7 @@ public void testIdleReader( } // Step 5: Clean up - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.FINISHED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.FINISHED)); } /** @@ -614,10 +598,7 @@ public void testTaskManagerFailure( controller.triggerTaskManagerFailover(jobClient, () -> {}); LOG.info("Waiting for job recovering from failure"); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.RUNNING), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.RUNNING)); // Step 6: Write test data again to external system List testRecordsAfterFailure = @@ -639,10 +620,7 @@ public void testTaskManagerFailure( // Step 8: Clean up terminateJob(jobClient); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.CANCELED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.CANCELED)); iterator.close(); } @@ -790,10 +768,7 @@ private boolean checkSourceMetrics( private void killJob(JobClient jobClient) throws Exception { terminateJob(jobClient); - waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.CANCELED), - Deadline.fromNow(DEFAULT_JOB_STATUS_CHANGE_TIMEOUT)); + waitForJobStatus(jobClient, Collections.singletonList(JobStatus.CANCELED)); } /** Builder class for constructing {@link CollectResultIterator} of collect sink. */ diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/utils/ConnectorTestConstants.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/utils/ConnectorTestConstants.java index f9a83195c4166..88c46c2851796 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/utils/ConnectorTestConstants.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/utils/ConnectorTestConstants.java @@ -26,6 +26,5 @@ public class ConnectorTestConstants { public static final long SLOT_REQUEST_TIMEOUT_MS = 10_000L; public static final long HEARTBEAT_TIMEOUT_MS = 5_000L; public static final long HEARTBEAT_INTERVAL_MS = 1000L; - public static final Duration DEFAULT_JOB_STATUS_CHANGE_TIMEOUT = Duration.ofSeconds(30L); public static final Duration DEFAULT_COLLECT_DATA_TIMEOUT = Duration.ofSeconds(120L); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java index dc9c6647e828e..6e6250e47fbee 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.connector.source.mocks.MockSource; import org.apache.flink.configuration.Configuration; @@ -46,7 +45,6 @@ import org.junit.runners.Parameterized; import java.io.IOException; -import java.time.Duration; import java.util.Collections; import java.util.Optional; import java.util.concurrent.ExecutionException; @@ -91,10 +89,7 @@ public void testTriggeringWhenPeriodicDisabled() throws Exception { final JobID jobID = jobClient.getJobID(); final MiniCluster miniCluster = MINI_CLUSTER_RESOURCE.getMiniCluster(); - CommonTestUtils.waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.RUNNING), - Deadline.fromNow(Duration.ofSeconds(30))); + CommonTestUtils.waitForJobStatus(jobClient, Collections.singletonList(JobStatus.RUNNING)); CommonTestUtils.waitForAllTaskRunning(miniCluster, jobID, false); // wait for the checkpoint to be taken @@ -126,14 +121,10 @@ public void testTriggeringWhenPeriodicEnabled() throws Exception { final JobID jobID = jobClient.getJobID(); final MiniCluster miniCluster = MINI_CLUSTER_RESOURCE.getMiniCluster(); - CommonTestUtils.waitForJobStatus( - jobClient, - Collections.singletonList(JobStatus.RUNNING), - Deadline.fromNow(Duration.ofSeconds(30))); + CommonTestUtils.waitForJobStatus(jobClient, Collections.singletonList(JobStatus.RUNNING)); CommonTestUtils.waitForAllTaskRunning(miniCluster, jobID, false); CommonTestUtils.waitUntilCondition( () -> queryCompletedCheckpoints(miniCluster, jobID) > 0L, - Deadline.fromNow(Duration.ofSeconds(30)), checkpointingInterval / 2); final long numberOfPeriodicCheckpoints = queryCompletedCheckpoints(miniCluster, jobID); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java index 03dfa55cf22f8..0ea14a8d36aa4 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ResumeCheckpointManuallyITCase.java @@ -25,7 +25,6 @@ import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory; import org.apache.flink.configuration.CheckpointingOptions; @@ -59,7 +58,6 @@ import java.io.File; import java.io.IOException; -import java.time.Duration; import java.util.concurrent.CountDownLatch; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForCheckpoint; @@ -348,11 +346,7 @@ private static String runJobAndGetExternalizedCheckpoint( NotifyingInfiniteTupleSource.countDownLatch.await(); // complete at least two checkpoints so that the initial checkpoint can be subsumed - waitForCheckpoint( - initialJobGraph.getJobID(), - cluster.getMiniCluster(), - Deadline.fromNow(Duration.ofMinutes(5)), - 2); + waitForCheckpoint(initialJobGraph.getJobID(), cluster.getMiniCluster(), 2); cluster.getClusterClient().cancel(initialJobGraph.getJobID()).get(); waitUntilJobCanceled(initialJobGraph.getJobID(), cluster.getClusterClient()); return CommonTestUtils.getLatestCompletedCheckpointPath( diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 4be614c93a2ea..b41febb04ee09 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -1196,8 +1196,7 @@ public static void waitUntilAllTasksAreRunning( detailsInfo -> allVerticesRunning( detailsInfo.getJobVerticesPerState())) - .get(), - Deadline.fromNow(Duration.ofSeconds(10))); + .get()); } private static boolean allVerticesRunning(Map states) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointFailureHandlingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointFailureHandlingITCase.java index e834ff9229cf9..dac1536e434e1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointFailureHandlingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointFailureHandlingITCase.java @@ -63,7 +63,6 @@ import static java.util.Collections.singletonList; import static org.apache.flink.api.common.JobStatus.RUNNING; -import static org.apache.flink.api.common.time.Deadline.fromNow; import static org.apache.flink.core.fs.Path.fromLocalFile; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForAllTaskRunning; import static org.apache.flink.runtime.testutils.CommonTestUtils.waitForJobStatus; @@ -105,7 +104,7 @@ public void testCheckpointSuccessAfterFailure() throws Exception { JobID jobID = jobClient.getJobID(); MiniCluster miniCluster = miniClusterResource.getMiniCluster(); - waitForJobStatus(jobClient, singletonList(RUNNING), fromNow(Duration.ofSeconds(30))); + waitForJobStatus(jobClient, singletonList(RUNNING)); waitForAllTaskRunning(miniCluster, jobID, false); triggerFailingCheckpoint(jobID, TestException.class, miniCluster); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ClusterEntrypointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ClusterEntrypointITCase.java index c1b6ccccaa6e9..186f531736f3f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ClusterEntrypointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ClusterEntrypointITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.test.recovery; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; @@ -38,7 +37,6 @@ import java.io.File; import java.nio.file.Files; import java.nio.file.Path; -import java.time.Duration; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -73,8 +71,7 @@ public void testDeterministicWorkingDirectoryIsNotDeletedInCaseOfProcessFailure( boolean success = false; try { - CommonTestUtils.waitUntilCondition( - workingDirectory::exists, Deadline.fromNow(Duration.ofMinutes(1L))); + CommonTestUtils.waitUntilCondition(workingDirectory::exists); jobManagerProcess.getProcess().destroy(); @@ -112,8 +109,7 @@ public void testNondeterministicWorkingDirectoryIsDeletedInCaseOfProcessFailure( try (Stream files = Files.list(workingDirBase.toPath())) { return files.findAny().isPresent(); } - }, - Deadline.fromNow(Duration.ofMinutes(1L))); + }); final File workingDirectory = Iterables.getOnlyElement( diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java index 00c977f14ef1a..c41001f674220 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/JobManagerHAProcessFailureRecoveryITCase.java @@ -309,7 +309,7 @@ public void testDispatcherProcessFailure() throws Exception { leaderRetrievalService.start(leaderListener); // Initial submission - leaderListener.waitForNewLeader(deadline.timeLeft().toMillis()); + leaderListener.waitForNewLeader(); String leaderAddress = leaderListener.getAddress(); UUID leaderId = leaderListener.getLeaderSessionID(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/LocalRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/LocalRecoveryITCase.java index 108cf2f9085f4..1639cda95b92a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/LocalRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/LocalRecoveryITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; @@ -105,10 +104,7 @@ public void testRecoverLocallyFromProcessCrashWithWorkingDirectory() throws Exce final long waitingTimeInSeconds = 45L; waitUntilCheckpointCompleted( - configuration, - clusterEntrypoint.getRestPort(), - jobClient.getJobID(), - Deadline.fromNow(Duration.ofSeconds(waitingTimeInSeconds))); + configuration, clusterEntrypoint.getRestPort(), jobClient.getJobID()); restartTaskManagerProcesses(taskManagerProcesses, parallelism - 1); @@ -219,8 +215,7 @@ public void terminate() { } private void waitUntilCheckpointCompleted( - Configuration configuration, int restPort, JobID jobId, Deadline deadline) - throws Exception { + Configuration configuration, int restPort, JobID jobId) throws Exception { final RestClient restClient = new RestClient(configuration, Executors.directExecutor()); final JobMessageParameters messageParameters = new JobMessageParameters(); messageParameters.jobPathParameter.resolve(jobId); @@ -237,8 +232,7 @@ private void waitUntilCheckpointCompleted( EmptyRequestBody.getInstance()) .join(); return checkpointingStatistics.getCounts().getNumberCompletedCheckpoints() > 0; - }, - deadline); + }); } private JobClient submitJob( diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerRunnerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerRunnerITCase.java index cd7824dd85391..64b2e7c1b2a75 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerRunnerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerRunnerITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.test.recovery; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.AkkaOptions; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; @@ -76,8 +75,7 @@ public void testDeterministicWorkingDirIsNotDeletedInCaseOfProcessFailure() thro boolean success = false; try { - CommonTestUtils.waitUntilCondition( - workingDirectory::exists, Deadline.fromNow(Duration.ofMinutes(1L))); + CommonTestUtils.waitUntilCondition(workingDirectory::exists); taskManagerProcess.getProcess().destroy(); @@ -115,8 +113,7 @@ public void testNondeterministicWorkingDirIsDeletedInCaseOfProcessFailure() thro try (Stream files = Files.list(workingDirBase.toPath())) { return files.findAny().isPresent(); } - }, - Deadline.fromNow(Duration.ofMinutes(1L))); + }); final File workingDirectory = Iterables.getOnlyElement( diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/DefaultSchedulerLocalRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/DefaultSchedulerLocalRecoveryITCase.java index 2f16fe75a5882..0bcccecaaa80c 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/DefaultSchedulerLocalRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/DefaultSchedulerLocalRecoveryITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.program.MiniClusterClient; import org.apache.flink.configuration.CheckpointingOptions; import org.apache.flink.configuration.Configuration; @@ -47,7 +46,6 @@ import org.junit.experimental.categories.Category; import java.io.IOException; -import java.time.Duration; import java.util.Arrays; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -160,9 +158,7 @@ private ArchivedExecutionGraph executeSchedulingTest( private void waitUntilAllVerticesRunning(JobID jobId, MiniCluster miniCluster) throws Exception { CommonTestUtils.waitForAllTaskRunning( - () -> miniCluster.getExecutionGraph(jobId).get(TIMEOUT, TimeUnit.SECONDS), - Deadline.fromNow(Duration.ofMillis(TIMEOUT)), - false); + () -> miniCluster.getExecutionGraph(jobId).get(TIMEOUT, TimeUnit.SECONDS), false); } private JobGraph createJobGraph(int parallelism) throws IOException { diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java index 8aab37a88e251..2ebd53564e419 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/leaderelection/ZooKeeperLeaderElectionITCase.java @@ -21,7 +21,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; @@ -63,8 +62,6 @@ /** Test the election of a new JobManager leader. */ public class ZooKeeperLeaderElectionITCase extends TestLogger { - private static final Duration TEST_TIMEOUT = Duration.ofMinutes(5L); - private static final Time RPC_TIMEOUT = Time.minutes(1L); private static TestingServer zkServer; @@ -111,8 +108,6 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { .setNumSlotsPerTaskManager(numSlotsPerTM) .build(); - final Deadline timeout = Deadline.fromNow(TEST_TIMEOUT); - try (TestingMiniCluster miniCluster = TestingMiniCluster.newBuilder(miniClusterConfiguration).build(); final CuratorFrameworkWithUnhandledErrorListener curatorFramework = @@ -146,20 +141,19 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { for (int i = 0; i < numDispatchers - 1; i++) { final DispatcherGateway leaderDispatcherGateway = - getNextLeadingDispatcherGateway( - miniCluster, previousLeaderAddress, timeout); + getNextLeadingDispatcherGateway(miniCluster, previousLeaderAddress); // Make sure resource manager has also changed leadership. resourceManagerLeaderFutures[i].get(); previousLeaderAddress = leaderDispatcherGateway.getAddress(); - awaitRunningStatus(leaderDispatcherGateway, jobGraph, timeout); + awaitRunningStatus(leaderDispatcherGateway, jobGraph); leaderDispatcherGateway.shutDownCluster(); } final DispatcherGateway leaderDispatcherGateway = - getNextLeadingDispatcherGateway(miniCluster, previousLeaderAddress, timeout); + getNextLeadingDispatcherGateway(miniCluster, previousLeaderAddress); // Make sure resource manager has also changed leadership. resourceManagerLeaderFutures[numDispatchers - 1].get(); - awaitRunningStatus(leaderDispatcherGateway, jobGraph, timeout); + awaitRunningStatus(leaderDispatcherGateway, jobGraph); CompletableFuture jobResultFuture = leaderDispatcherGateway.requestJobResult(jobGraph.getJobID(), RPC_TIMEOUT); BlockingOperator.unblock(); @@ -170,21 +164,17 @@ public void testJobExecutionOnClusterWithLeaderChange() throws Exception { } } - private static void awaitRunningStatus( - DispatcherGateway dispatcherGateway, JobGraph jobGraph, Deadline timeout) + private static void awaitRunningStatus(DispatcherGateway dispatcherGateway, JobGraph jobGraph) throws Exception { CommonTestUtils.waitUntilCondition( () -> dispatcherGateway.requestJobStatus(jobGraph.getJobID(), RPC_TIMEOUT).get() == JobStatus.RUNNING, - timeout, 50L); } private DispatcherGateway getNextLeadingDispatcherGateway( - TestingMiniCluster miniCluster, - @Nullable String previousLeaderAddress, - Deadline timeout) + TestingMiniCluster miniCluster, @Nullable String previousLeaderAddress) throws Exception { CommonTestUtils.waitUntilCondition( () -> @@ -193,7 +183,6 @@ private DispatcherGateway getNextLeadingDispatcherGateway( .get() .getAddress() .equals(previousLeaderAddress), - timeout, 20L); return miniCluster.getDispatcherGatewayFuture().get(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java index 96cb91e0f602e..a4124dfe08c34 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/scheduling/AdaptiveSchedulerITCase.java @@ -23,7 +23,6 @@ import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.ClusterOptions; import org.apache.flink.configuration.Configuration; @@ -61,8 +60,6 @@ import javax.annotation.Nullable; import java.io.File; -import java.time.Duration; -import java.time.temporal.ChronoUnit; import java.util.Arrays; import java.util.Collections; import java.util.concurrent.CompletableFuture; @@ -183,9 +180,7 @@ public void testStopWithSavepointFailOnCheckpoint() throws Exception { assertThat(e, containsCause(FlinkException.class)); } // expect job to run again (maybe restart) - CommonTestUtils.waitUntilCondition( - () -> client.getJobStatus().get() == JobStatus.RUNNING, - Deadline.fromNow(Duration.of(1, ChronoUnit.MINUTES))); + CommonTestUtils.waitUntilCondition(() -> client.getJobStatus().get() == JobStatus.RUNNING); } @Test @@ -250,9 +245,7 @@ public void testStopWithSavepointFailOnFirstSavepointSucceedOnSecond() throws Ex // ensure failed savepoint files have been removed from the directory. // We execute this in a retry loop with a timeout, because the savepoint deletion happens // asynchronously and is not bound to the job lifecycle. See FLINK-22493 for more details. - CommonTestUtils.waitUntilCondition( - () -> isDirectoryEmpty(savepointDirectory), - Deadline.fromNow(Duration.ofSeconds(10))); + CommonTestUtils.waitUntilCondition(() -> isDirectoryEmpty(savepointDirectory)); // trigger second savepoint final String savepoint = @@ -266,7 +259,6 @@ public void testStopWithSavepointFailOnFirstSavepointSucceedOnSecond() throws Ex @Test public void testExceptionHistoryIsRetrievableFromTheRestAPI() throws Exception { - final Deadline deadline = Deadline.fromNow(Duration.ofMinutes(1)); final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(PARALLELISM); env.enableCheckpointing(20L, CheckpointingMode.EXACTLY_ONCE); @@ -288,11 +280,9 @@ public void testExceptionHistoryIsRetrievableFromTheRestAPI() throws Exception { exceptionsFuture.get(); return jobExceptionsInfoWithHistory.getExceptionHistory().getEntries().size() > 0; - }, - deadline); + }); jobClient.cancel().get(); - CommonTestUtils.waitForJobStatus( - jobClient, Collections.singletonList(JobStatus.CANCELED), deadline); + CommonTestUtils.waitForJobStatus(jobClient, Collections.singletonList(JobStatus.CANCELED)); } private boolean isDirectoryEmpty(File directory) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java b/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java index 9859a87a7bf83..ee09c293b8ac5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.JobManagerOptions; @@ -40,7 +39,6 @@ import org.junit.Rule; import org.junit.Test; -import java.time.Duration; import java.util.concurrent.ExecutionException; /** Tests for Reactive Mode (FLIP-159). */ @@ -146,9 +144,7 @@ private int getNumberOfConnectedTaskManagers() throws ExecutionException, Interr private void startAdditionalTaskManager() throws Exception { miniClusterResource.getMiniCluster().startTaskManager(); - CommonTestUtils.waitUntilCondition( - () -> getNumberOfConnectedTaskManagers() == 2, - Deadline.fromNow(Duration.ofMillis(10_000L))); + CommonTestUtils.waitUntilCondition(() -> getNumberOfConnectedTaskManagers() == 2); } private static class DummySource implements SourceFunction { @@ -213,7 +209,6 @@ public static void waitUntilParallelismForVertexReached( } } return false; - }, - Deadline.fromNow(Duration.ofSeconds(10))); + }); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java index d4791ac2e63d9..aafa09668789b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogCompatibilityITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.test.state; import org.apache.flink.api.common.functions.ReduceFunction; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.changelog.fs.FsStateChangelogStorageFactory; import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; @@ -39,7 +38,6 @@ import org.junit.runners.Parameterized; import java.io.File; -import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.NoSuchElementException; @@ -145,11 +143,7 @@ private String tryCheckpointAndStop(JobGraph jobGraph) throws Exception { ClusterClient client = miniClusterResource.getClusterClient(); submit(jobGraph, client); if (testCase.restoreSource == RestoreSource.CHECKPOINT) { - waitForCheckpoint( - jobGraph.getJobID(), - miniClusterResource.getMiniCluster(), - Deadline.fromNow(Duration.ofMinutes(5)), - 1); + waitForCheckpoint(jobGraph.getJobID(), miniClusterResource.getMiniCluster(), 1); client.cancel(jobGraph.getJobID()).get(); // obtain the latest checkpoint *after* cancellation - that one won't be subsumed return CommonTestUtils.getLatestCompletedCheckpointPath( diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java index c56b233d2e2c5..ed7b3d9e28923 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/ChangelogRescalingITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; @@ -327,8 +326,7 @@ public boolean request() { } private String checkpointAndCancel(JobID jobID) throws Exception { - waitForCheckpoint( - jobID, cluster.getMiniCluster(), Deadline.fromNow(Duration.ofMinutes(5)), 1); + waitForCheckpoint(jobID, cluster.getMiniCluster(), 1); cluster.getClusterClient().cancel(jobID).get(); checkStatus(jobID); return CommonTestUtils.getLatestCompletedCheckpointPath(jobID, cluster.getMiniCluster()) diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java index 00882d079ed3a..308c51c80b7a9 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNHighAvailabilityITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.client.deployment.ClusterDeploymentException; import org.apache.flink.client.deployment.ClusterSpecification; import org.apache.flink.client.program.ClusterClient; @@ -76,7 +75,6 @@ import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.time.Duration; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -107,7 +105,6 @@ public class YARNHighAvailabilityITCase extends YarnTestBase { @ClassRule public static final TemporaryFolder FOLDER = new TemporaryFolder(); private static final String LOG_DIR = "flink-yarn-tests-ha"; - private static final Duration TIMEOUT = Duration.ofSeconds(200L); private static TestingServer zkServer; private static String storageDir; @@ -262,8 +259,7 @@ private void waitForApplicationAttempt(final ApplicationId applicationId, final yarnClient.getApplicationReport(applicationId); return applicationReport.getCurrentApplicationAttemptId().getAttemptId() >= attemptId; - }, - Deadline.fromNow(TIMEOUT)); + }); log.info("Attempt {} id detected.", attemptId); } @@ -319,8 +315,7 @@ private void killApplicationAndWait(final ApplicationId id) throws Exception { EnumSet.of( YarnApplicationState.KILLED, YarnApplicationState.FINISHED)) - .isEmpty(), - Deadline.fromNow(TIMEOUT)); + .isEmpty()); } private void waitForJobTermination( @@ -329,7 +324,7 @@ private void waitForJobTermination( log.info("Sending stop job signal"); stopJobSignal.signal(); final CompletableFuture jobResult = restClusterClient.requestJobResult(jobId); - jobResult.get(TIMEOUT.toMillis(), TimeUnit.MILLISECONDS); + jobResult.get(200, TimeUnit.SECONDS); } @Nonnull @@ -389,8 +384,7 @@ private void killApplicationMaster(final String processName) throws Exception { () -> { final Set curPids = getApplicationMasterPids(processName); return origPids.stream().noneMatch(curPids::contains); - }, - Deadline.fromNow(TIMEOUT)); + }); } private Set getApplicationMasterPids(final String processName) @@ -417,8 +411,7 @@ private static void waitUntilJobIsRunning( && jobDetails.getJobVertexInfos().stream() .map(toExecutionState()) .allMatch(isRunning()); - }, - Deadline.fromNow(TIMEOUT)); + }); } private static Function @@ -436,8 +429,7 @@ private static void waitUntilJobIsRestarted( final int expectedFullRestarts) throws Exception { CommonTestUtils.waitUntilCondition( - () -> getJobFullRestarts(restClusterClient, jobId) >= expectedFullRestarts, - Deadline.fromNow(TIMEOUT)); + () -> getJobFullRestarts(restClusterClient, jobId) >= expectedFullRestarts); } private static int getJobFullRestarts( diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java index ce71d3b8ae5c1..2089b5cbaa41d 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionCapacitySchedulerITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.yarn; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.common.time.Time; import org.apache.flink.client.cli.CliFrontend; import org.apache.flink.configuration.Configuration; @@ -66,7 +65,6 @@ import java.io.IOException; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.time.Duration; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; @@ -333,7 +331,7 @@ public void perJobYarnClusterOffHeap() throws Exception { // // Assert the number of TaskManager slots are set // - waitForTaskManagerRegistration(host, port, Duration.ofMillis(30_000)); + waitForTaskManagerRegistration(host, port); assertNumberOfSlotsPerTask(host, port, 3); final Map flinkConfig = getFlinkConfig(host, port); @@ -388,18 +386,16 @@ private void submitJob(final String jobFileName) throws IOException, Interrupted jobRunner.join(); } - private static void waitForTaskManagerRegistration( - final String host, final int port, final Duration waitDuration) throws Exception { - CommonTestUtils.waitUntilCondition( - () -> getNumberOfTaskManagers(host, port) > 0, Deadline.fromNow(waitDuration)); + private static void waitForTaskManagerRegistration(final String host, final int port) + throws Exception { + CommonTestUtils.waitUntilCondition(() -> getNumberOfTaskManagers(host, port) > 0); } private static void assertNumberOfSlotsPerTask( final String host, final int port, final int slotsNumber) throws Exception { try { CommonTestUtils.waitUntilCondition( - () -> getNumberOfSlotsPerTaskManager(host, port) == slotsNumber, - Deadline.fromNow(Duration.ofSeconds(30))); + () -> getNumberOfSlotsPerTaskManager(host, port) == slotsNumber); } catch (final TimeoutException e) { final int currentNumberOfSlots = getNumberOfSlotsPerTaskManager(host, port); fail( diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java index e080ae79a96b5..8356576b4bc42 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/YARNSessionFIFOITCase.java @@ -18,7 +18,6 @@ package org.apache.flink.yarn; -import org.apache.flink.api.common.time.Deadline; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.runtime.testutils.CommonTestUtils; @@ -45,7 +44,6 @@ import org.slf4j.event.Level; import java.io.File; -import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.EnumSet; @@ -150,19 +148,13 @@ ApplicationId runDetachedModeTest(Map securityProperties) throws jobRunner.join(); - final Duration timeout = Duration.ofMinutes(1); final long testConditionIntervalInMillis = 500; // in "new" mode we can only wait after the job is submitted, because TMs // are spun up lazily // wait until two containers are running LOG.info("Waiting until two containers are running"); CommonTestUtils.waitUntilCondition( - () -> getRunningContainers() >= 2, - Deadline.fromNow(timeout), - testConditionIntervalInMillis, - "We didn't reach the state of two containers running (instead: " - + getRunningContainers() - + ")"); + () -> getRunningContainers() >= 2, testConditionIntervalInMillis); LOG.info("Waiting until the job reaches FINISHED state"); final ApplicationId applicationId = getOnlyApplicationReport().getApplicationId(); @@ -172,11 +164,7 @@ ApplicationId runDetachedModeTest(Map securityProperties) throws new String[] {"switched from state RUNNING to FINISHED"}, applicationId, "jobmanager.log"), - Deadline.fromNow(timeout), - testConditionIntervalInMillis, - "The deployed job didn't finish on time reaching the timeout of " - + timeout - + " seconds. The application will be cancelled forcefully."); + testConditionIntervalInMillis); // kill application "externally". try { @@ -202,7 +190,6 @@ ApplicationId runDetachedModeTest(Map securityProperties) throws YarnApplicationState.KILLED, YarnApplicationState.FINISHED)) .isEmpty(), - Deadline.fromNow(timeout), testConditionIntervalInMillis); } catch (Throwable t) { LOG.warn("Killing failed", t); From 1556adf753d11a9e3548cb7e6b73160a0721bdcb Mon Sep 17 00:00:00 2001 From: godfreyhe Date: Mon, 21 Mar 2022 15:40:26 +0800 Subject: [PATCH 178/258] [FLINK-26756][table-planner] Fix the deserialization error for match recognize This closes #19179 (cherry picked from commit f8d2bdb19eb954ef384a78b5e21991a4327c23db) --- .../exec/serde/RexNodeJsonDeserializer.java | 46 +++++++++++++++++++ .../exec/serde/RexNodeJsonSerializer.java | 14 ++++-- .../table/planner/utils/JsonPlanTestBase.java | 6 ++- .../testMatch.out | 4 +- 4 files changed, 63 insertions(+), 7 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java index 8e52495ea3fb3..cbb7df0568031 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonDeserializer.java @@ -46,8 +46,10 @@ import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.SqlKind; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.SqlSyntax; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParserPos; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.sql.validate.SqlNameMatchers; @@ -91,6 +93,7 @@ import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_OPERANDS; import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_RANGES; import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_SARG; +import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_SQL_KIND; import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_SYMBOL; import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_SYNTAX; import static org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer.FIELD_NAME_SYSTEM_NAME; @@ -340,6 +343,9 @@ static SqlOperator deserializeSqlOperator(JsonNode jsonNode, SerdeContext serdeC } else if (jsonNode.has(FIELD_NAME_SYSTEM_NAME)) { return deserializeSystemFunction( jsonNode.required(FIELD_NAME_SYSTEM_NAME).asText(), syntax, serdeContext); + } else if (jsonNode.has(FIELD_NAME_SQL_KIND)) { + return deserializeInternalFunction( + syntax, SqlKind.valueOf(jsonNode.get(FIELD_NAME_SQL_KIND).asText())); } else { throw new TableException("Invalid function call."); } @@ -375,6 +381,13 @@ private static SqlOperator deserializeInternalFunction( if (latestOperator.isPresent()) { return latestOperator.get(); } + + Optional sqlStdOperator = + lookupOptionalSqlStdOperator(publicName, syntax, null); + if (sqlStdOperator.isPresent()) { + return sqlStdOperator.get(); + } + throw new TableException( String.format( "Could not resolve internal system function '%s'. " @@ -382,6 +395,19 @@ private static SqlOperator deserializeInternalFunction( internalName)); } + private static SqlOperator deserializeInternalFunction(SqlSyntax syntax, SqlKind sqlKind) { + final Optional stdOperator = lookupOptionalSqlStdOperator("", syntax, sqlKind); + if (stdOperator.isPresent()) { + return stdOperator.get(); + } + + throw new TableException( + String.format( + "Could not resolve internal system function '%s'. " + + "This is a bug, please file an issue.", + sqlKind.name())); + } + private static SqlOperator deserializeFunctionClass( JsonNode jsonNode, SerdeContext serdeContext) { final String className = jsonNode.required(FIELD_NAME_CLASS).asText(); @@ -506,6 +532,26 @@ private static Optional lookupOptionalSqlOperator( } } + private static Optional lookupOptionalSqlStdOperator( + String operatorName, SqlSyntax syntax, @Nullable SqlKind sqlKind) { + List foundOperators = new ArrayList<>(); + // try to find operator from std operator table. + SqlStdOperatorTable.instance() + .lookupOperatorOverloads( + new SqlIdentifier(operatorName, new SqlParserPos(0, 0)), + null, // category + syntax, + foundOperators, + SqlNameMatchers.liberal()); + if (foundOperators.size() == 1) { + return Optional.of(foundOperators.get(0)); + } + // in case different operator has the same kind, check with both name and kind. + return foundOperators.stream() + .filter(o -> sqlKind != null && o.getKind() == sqlKind) + .findFirst(); + } + private static TableException missingSystemFunction(String systemName) { return new TableException( String.format( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerializer.java index 7e38317e729d3..5e788723cd3ca 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerializer.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/RexNodeJsonSerializer.java @@ -116,6 +116,7 @@ final class RexNodeJsonSerializer extends StdSerializer { static final String FIELD_NAME_SYSTEM_NAME = "systemName"; static final String FIELD_NAME_CATALOG_NAME = "catalogName"; static final String FIELD_NAME_SYNTAX = "syntax"; + static final String FIELD_NAME_SQL_KIND = "sqlKind"; static final String FIELD_NAME_CLASS = "class"; RexNodeJsonSerializer() { @@ -393,10 +394,15 @@ static void serializeSqlOperator( || operator instanceof AggSqlFunction) { throw legacyException(operator.toString()); } else { - // We assume that all regular SqlOperators are internal. Only the function definitions - // stack is exposed to the user and can thus be external. - gen.writeStringField( - FIELD_NAME_INTERNAL_NAME, BuiltInSqlOperator.toQualifiedName(operator)); + if (operator.getName().isEmpty()) { + gen.writeStringField(FIELD_NAME_SQL_KIND, operator.getKind().name()); + } else { + // We assume that all regular SqlOperators are internal. Only the function + // definitions + // stack is exposed to the user and can thus be external. + gen.writeStringField( + FIELD_NAME_INTERNAL_NAME, BuiltInSqlOperator.toQualifiedName(operator)); + } } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java index f175fccba8982..7415ee80caa46 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/utils/JsonPlanTestBase.java @@ -22,6 +22,7 @@ import org.apache.flink.streaming.api.transformations.UnionTransformation; import org.apache.flink.table.api.CompiledPlan; import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.PlanReference; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.internal.CompiledPlanUtils; @@ -73,7 +74,10 @@ public void after() { protected TableResult compileSqlAndExecutePlan(String sql) { CompiledPlan compiledPlan = tableEnv.compilePlanSql(sql); checkTransformationUids(compiledPlan); - return compiledPlan.execute(); + // try to execute the string json plan to validate to ser/de result + String jsonPlan = compiledPlan.asJsonString(); + CompiledPlan newCompiledPlan = tableEnv.loadPlan(PlanReference.fromJsonString(jsonPlan)); + return newCompiledPlan.execute(); } protected void checkTransformationUids(CompiledPlan compiledPlan) { diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out index c7bbd666aba9a..62fc80c1cbc50 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/MatchRecognizeJsonPlanTest_jsonplan/testMatch.out @@ -130,11 +130,11 @@ "pattern" : { "kind" : "CALL", "syntax" : "BINARY", - "internalName" : "$$1", + "sqlKind" : "PATTERN_CONCAT", "operands" : [ { "kind" : "CALL", "syntax" : "BINARY", - "internalName" : "$$1", + "sqlKind" : "PATTERN_CONCAT", "operands" : [ { "kind" : "LITERAL", "value" : "A\"", From ab5d7cc262138f88cf550d5db0793486597a3100 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Wed, 13 Apr 2022 16:53:39 +0200 Subject: [PATCH 179/258] [hotfix][docs] Add missing slash --- docs/layouts/shortcodes/ref_static.html | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/layouts/shortcodes/ref_static.html b/docs/layouts/shortcodes/ref_static.html index 797d25c2f1e58..82c726939c659 100644 --- a/docs/layouts/shortcodes/ref_static.html +++ b/docs/layouts/shortcodes/ref_static.html @@ -1,3 +1,4 @@ + {{/* Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE file @@ -18,4 +19,4 @@ */}}{{/* Shortcode for embedding a link to a static resource. */}} -{{ .Site.BaseURL }}{{ .Get 0 }} +{{ .Site.BaseURL }}/{{ .Get 0 }} From bb5080574aafb287646e89bdc35baca899580f5d Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Mon, 6 Sep 2021 15:04:29 +0800 Subject: [PATCH 180/258] [hotfix][python][examples] Add a few examples about the basic operations of Python Table API & DataStream API --- .../examples/datastream/basic_operations.py | 76 +++ .../examples/table/basic_operations.py | 445 ++++++++++++++++++ flink-python/tox.ini | 2 +- 3 files changed, 522 insertions(+), 1 deletion(-) create mode 100644 flink-python/pyflink/examples/datastream/basic_operations.py create mode 100644 flink-python/pyflink/examples/table/basic_operations.py diff --git a/flink-python/pyflink/examples/datastream/basic_operations.py b/flink-python/pyflink/examples/datastream/basic_operations.py new file mode 100644 index 0000000000000..44f0fb864f51d --- /dev/null +++ b/flink-python/pyflink/examples/datastream/basic_operations.py @@ -0,0 +1,76 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import json +import logging +import sys + +from pyflink.common import Types +from pyflink.datastream import StreamExecutionEnvironment + + +def show(ds, env): + ds.print() + env.execute() + + +def basic_operations(): + env = StreamExecutionEnvironment.get_execution_environment() + env.set_parallelism(1) + + # define the source + ds = env.from_collection( + collection=[ + (1, '{"name": "Flink", "tel": 123, "addr": {"country": "Germany", "city": "Berlin"}}'), + (2, '{"name": "hello", "tel": 135, "addr": {"country": "China", "city": "Shanghai"}}'), + (3, '{"name": "world", "tel": 124, "addr": {"country": "USA", "city": "NewYork"}}'), + (4, '{"name": "PyFlink", "tel": 32, "addr": {"country": "China", "city": "Hangzhou"}}') + ], + type_info=Types.ROW_NAMED(["id", "info"], [Types.INT(), Types.STRING()]) + ) + + # map + def update_tel(data): + # parse the json + json_data = json.loads(data.info) + json_data['tel'] += 1 + return data.id, json.dumps(json_data) + + show(ds.map(update_tel), env) + # (1, '{"name": "Flink", "tel": 124, "addr": {"country": "Germany", "city": "Berlin"}}') + # (2, '{"name": "hello", "tel": 136, "addr": {"country": "China", "city": "Shanghai"}}') + # (3, '{"name": "world", "tel": 125, "addr": {"country": "USA", "city": "NewYork"}}') + # (4, '{"name": "PyFlink", "tel": 33, "addr": {"country": "China", "city": "Hangzhou"}}') + + # filter + show(ds.filter(lambda data: data.id == 1).map(update_tel), env) + # (1, '{"name": "Flink", "tel": 124, "addr": {"country": "Germany", "city": "Berlin"}}') + + # key by + show(ds.map(lambda data: (json.loads(data.info)['addr']['country'], + json.loads(data.info)['tel'])) + .key_by(lambda data: data[0]).reduce(lambda a, b: (a[0], a[1] + b[1])), env) + # ('Germany', 123) + # ('China', 135) + # ('USA', 124) + # ('China', 167) + + +if __name__ == '__main__': + logging.basicConfig(stream=sys.stdout, level=logging.INFO, format="%(message)s") + + basic_operations() diff --git a/flink-python/pyflink/examples/table/basic_operations.py b/flink-python/pyflink/examples/table/basic_operations.py new file mode 100644 index 0000000000000..6b9454b757db9 --- /dev/null +++ b/flink-python/pyflink/examples/table/basic_operations.py @@ -0,0 +1,445 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import json +import logging +import sys + +from pyflink.common import Row +from pyflink.table import (DataTypes, TableEnvironment, EnvironmentSettings) +from pyflink.table.expressions import * +from pyflink.table.udf import udtf, udf, udaf, AggregateFunction, TableAggregateFunction, udtaf + + +def basic_operations(): + t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) + + # define the source + table = t_env.from_elements( + elements=[ + (1, '{"name": "Flink", "tel": 123, "addr": {"country": "Germany", "city": "Berlin"}}'), + (2, '{"name": "hello", "tel": 135, "addr": {"country": "China", "city": "Shanghai"}}'), + (3, '{"name": "world", "tel": 124, "addr": {"country": "USA", "city": "NewYork"}}'), + (4, '{"name": "PyFlink", "tel": 32, "addr": {"country": "China", "city": "Hangzhou"}}') + ], + schema=['id', 'data']) + + right_table = t_env.from_elements(elements=[(1, 18), (2, 30), (3, 25), (4, 10)], + schema=['id', 'age']) + + table = table.add_columns( + col('data').json_value('$.name', DataTypes.STRING()).alias('name'), + col('data').json_value('$.tel', DataTypes.STRING()).alias('tel'), + col('data').json_value('$.addr.country', DataTypes.STRING()).alias('country')) \ + .drop_columns(col('data')) + table.execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | + # | +I | 2 | hello | 135 | China | + # | +I | 3 | world | 124 | USA | + # | +I | 4 | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + + # limit the number of outputs + table.limit(3).execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | + # | +I | 2 | hello | 135 | China | + # | +I | 3 | world | 124 | USA | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + + # filter + table.filter(col('id') != 3).execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | + # | +I | 2 | hello | 135 | China | + # | +I | 4 | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + + # aggregation + table.group_by(col('country')) \ + .select(col('country'), col('id').count, col('tel').cast(DataTypes.BIGINT()).max) \ + .execute().print() + # +----+--------------------------------+----------------------+----------------------+ + # | op | country | EXPR$0 | EXPR$1 | + # +----+--------------------------------+----------------------+----------------------+ + # | +I | Germany | 1 | 123 | + # | +I | USA | 1 | 124 | + # | +I | China | 1 | 135 | + # | -U | China | 1 | 135 | + # | +U | China | 2 | 135 | + # +----+--------------------------------+----------------------+----------------------+ + + # distinct + table.select(col('country')).distinct() \ + .execute().print() + # +----+--------------------------------+ + # | op | country | + # +----+--------------------------------+ + # | +I | Germany | + # | +I | China | + # | +I | USA | + # +----+--------------------------------+ + + # join + # Note that it still doesn't support duplicate column names between the joined tables + table.join(right_table.rename_columns(col('id').alias('r_id')), col('id') == col('r_id')) \ + .execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+----------------------+----------------------+ + # | op | id | name | tel | country | r_id | age | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+----------------------+----------------------+ + # | +I | 4 | PyFlink | 32 | China | 4 | 10 | + # | +I | 1 | Flink | 123 | Germany | 1 | 18 | + # | +I | 2 | hello | 135 | China | 2 | 30 | + # | +I | 3 | world | 124 | USA | 3 | 25 | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+----------------------+----------------------+ + + # join lateral + @udtf(result_types=[DataTypes.STRING()]) + def split(r: Row): + for s in r.name.split("i"): + yield s + + table.join_lateral(split.alias('a')) \ + .execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | tel | country | a | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | Fl | + # | +I | 1 | Flink | 123 | Germany | nk | + # | +I | 2 | hello | 135 | China | hello | + # | +I | 3 | world | 124 | USA | world | + # | +I | 4 | PyFlink | 32 | China | PyFl | + # | +I | 4 | PyFlink | 32 | China | nk | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + + # show schema + table.print_schema() + # ( + # `id` BIGINT, + # `name` STRING, + # `tel` STRING, + # `country` STRING + # ) + + # show execute plan + print(table.join_lateral(split.alias('a')).explain()) + # == Abstract Syntax Tree == + # LogicalCorrelate(correlation=[$cor1], joinType=[inner], requiredColumns=[{}]) + # :- LogicalProject(id=[$0], name=[JSON_VALUE($1, _UTF-16LE'$.name', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR))], tel=[JSON_VALUE($1, _UTF-16LE'$.tel', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR))], country=[JSON_VALUE($1, _UTF-16LE'$.addr.country', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR))]) + # : +- LogicalTableScan(table=[[default_catalog, default_database, Unregistered_TableSource_249535355, source: [PythonInputFormatTableSource(id, data)]]]) + # +- LogicalTableFunctionScan(invocation=[*org.apache.flink.table.functions.python.PythonTableFunction$1f0568d1f39bef59b4c969a5d620ba46*($0, $1, $2, $3)], rowType=[RecordType(VARCHAR(2147483647) a)], elementType=[class [Ljava.lang.Object;]) + # + # == Optimized Physical Plan == + # PythonCorrelate(invocation=[*org.apache.flink.table.functions.python.PythonTableFunction$1f0568d1f39bef59b4c969a5d620ba46*($0, $1, $2, $3)], correlate=[table(split(id,name,tel,country))], select=[id,name,tel,country,a], rowType=[RecordType(BIGINT id, VARCHAR(2147483647) name, VARCHAR(2147483647) tel, VARCHAR(2147483647) country, VARCHAR(2147483647) a)], joinType=[INNER]) + # +- Calc(select=[id, JSON_VALUE(data, _UTF-16LE'$.name', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR)) AS name, JSON_VALUE(data, _UTF-16LE'$.tel', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR)) AS tel, JSON_VALUE(data, _UTF-16LE'$.addr.country', FLAG(NULL), FLAG(ON EMPTY), FLAG(NULL), FLAG(ON ERROR)) AS country]) + # +- LegacyTableSourceScan(table=[[default_catalog, default_database, Unregistered_TableSource_249535355, source: [PythonInputFormatTableSource(id, data)]]], fields=[id, data]) + # + # == Optimized Execution Plan == + # PythonCorrelate(invocation=[*org.apache.flink.table.functions.python.PythonTableFunction$1f0568d1f39bef59b4c969a5d620ba46*($0, $1, $2, $3)], correlate=[table(split(id,name,tel,country))], select=[id,name,tel,country,a], rowType=[RecordType(BIGINT id, VARCHAR(2147483647) name, VARCHAR(2147483647) tel, VARCHAR(2147483647) country, VARCHAR(2147483647) a)], joinType=[INNER]) + # +- Calc(select=[id, JSON_VALUE(data, '$.name', NULL, ON EMPTY, NULL, ON ERROR) AS name, JSON_VALUE(data, '$.tel', NULL, ON EMPTY, NULL, ON ERROR) AS tel, JSON_VALUE(data, '$.addr.country', NULL, ON EMPTY, NULL, ON ERROR) AS country]) + # +- LegacyTableSourceScan(table=[[default_catalog, default_database, Unregistered_TableSource_249535355, source: [PythonInputFormatTableSource(id, data)]]], fields=[id, data]) + + +def sql_operations(): + t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) + + # define the source + table = t_env.from_elements( + elements=[ + (1, '{"name": "Flink", "tel": 123, "addr": {"country": "Germany", "city": "Berlin"}}'), + (2, '{"name": "hello", "tel": 135, "addr": {"country": "China", "city": "Shanghai"}}'), + (3, '{"name": "world", "tel": 124, "addr": {"country": "USA", "city": "NewYork"}}'), + (4, '{"name": "PyFlink", "tel": 32, "addr": {"country": "China", "city": "Hangzhou"}}') + ], + schema=['id', 'data']) + + t_env.sql_query("SELECT * FROM %s" % table) \ + .execute().print() + # +----+----------------------+--------------------------------+ + # | op | id | data | + # +----+----------------------+--------------------------------+ + # | +I | 1 | {"name": "Flink", "tel": 12... | + # | +I | 2 | {"name": "hello", "tel": 13... | + # | +I | 3 | {"name": "world", "tel": 12... | + # | +I | 4 | {"name": "PyFlink", "tel": ... | + # +----+----------------------+--------------------------------+ + + # execute sql statement + @udtf(result_types=[DataTypes.STRING(), DataTypes.INT(), DataTypes.STRING()]) + def parse_data(data: str): + json_data = json.loads(data) + yield json_data['name'], json_data['tel'], json_data['addr']['country'] + + t_env.create_temporary_function('parse_data', parse_data) + t_env.execute_sql( + """ + SELECT * + FROM %s, LATERAL TABLE(parse_data(`data`)) t(name, tel, country) + """ % table + ).print() + # +----+----------------------+--------------------------------+--------------------------------+-------------+--------------------------------+ + # | op | id | data | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+-------------+--------------------------------+ + # | +I | 1 | {"name": "Flink", "tel": 12... | Flink | 123 | Germany | + # | +I | 2 | {"name": "hello", "tel": 13... | hello | 135 | China | + # | +I | 3 | {"name": "world", "tel": 12... | world | 124 | USA | + # | +I | 4 | {"name": "PyFlink", "tel": ... | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+-------------+--------------------------------+ + + # explain sql plan + print(t_env.explain_sql( + """ + SELECT * + FROM %s, LATERAL TABLE(parse_data(`data`)) t(name, tel, country) + """ % table + )) + # == Abstract Syntax Tree == + # LogicalProject(id=[$0], data=[$1], name=[$2], tel=[$3], country=[$4]) + # +- LogicalCorrelate(correlation=[$cor1], joinType=[inner], requiredColumns=[{1}]) + # :- LogicalTableScan(table=[[default_catalog, default_database, Unregistered_TableSource_734856049, source: [PythonInputFormatTableSource(id, data)]]]) + # +- LogicalTableFunctionScan(invocation=[parse_data($cor1.data)], rowType=[RecordType:peek_no_expand(VARCHAR(2147483647) f0, INTEGER f1, VARCHAR(2147483647) f2)]) + # + # == Optimized Physical Plan == + # PythonCorrelate(invocation=[parse_data($1)], correlate=[table(parse_data(data))], select=[id,data,f0,f1,f2], rowType=[RecordType(BIGINT id, VARCHAR(2147483647) data, VARCHAR(2147483647) f0, INTEGER f1, VARCHAR(2147483647) f2)], joinType=[INNER]) + # +- LegacyTableSourceScan(table=[[default_catalog, default_database, Unregistered_TableSource_734856049, source: [PythonInputFormatTableSource(id, data)]]], fields=[id, data]) + # + # == Optimized Execution Plan == + # PythonCorrelate(invocation=[parse_data($1)], correlate=[table(parse_data(data))], select=[id,data,f0,f1,f2], rowType=[RecordType(BIGINT id, VARCHAR(2147483647) data, VARCHAR(2147483647) f0, INTEGER f1, VARCHAR(2147483647) f2)], joinType=[INNER]) + # +- LegacyTableSourceScan(table=[[default_catalog, default_database, Unregistered_TableSource_734856049, source: [PythonInputFormatTableSource(id, data)]]], fields=[id, data]) + + +def column_operations(): + t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) + + # define the source + table = t_env.from_elements( + elements=[ + (1, '{"name": "Flink", "tel": 123, "addr": {"country": "Germany", "city": "Berlin"}}'), + (2, '{"name": "hello", "tel": 135, "addr": {"country": "China", "city": "Shanghai"}}'), + (3, '{"name": "world", "tel": 124, "addr": {"country": "USA", "city": "NewYork"}}'), + (4, '{"name": "PyFlink", "tel": 32, "addr": {"country": "China", "city": "Hangzhou"}}') + ], + schema=['id', 'data']) + + # add columns + table = table.add_columns( + col('data').json_value('$.name', DataTypes.STRING()).alias('name'), + col('data').json_value('$.tel', DataTypes.STRING()).alias('tel'), + col('data').json_value('$.addr.country', DataTypes.STRING()).alias('country')) + + table.execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | data | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | {"name": "Flink", "tel": 12... | Flink | 123 | Germany | + # | +I | 2 | {"name": "hello", "tel": 13... | hello | 135 | China | + # | +I | 3 | {"name": "world", "tel": 12... | world | 124 | USA | + # | +I | 4 | {"name": "PyFlink", "tel": ... | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + + # drop columns + table = table.drop_columns(col('data')) + table.execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | tel | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | + # | +I | 2 | hello | 135 | China | + # | +I | 3 | world | 124 | USA | + # | +I | 4 | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + + # rename columns + table = table.rename_columns(col('tel').alias('telephone')) + table.execute().print() + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | telephone | country | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1 | Flink | 123 | Germany | + # | +I | 2 | hello | 135 | China | + # | +I | 3 | world | 124 | USA | + # | +I | 4 | PyFlink | 32 | China | + # +----+----------------------+--------------------------------+--------------------------------+--------------------------------+ + + # replace columns + table = table.add_or_replace_columns( + concat(col('id').cast(DataTypes.STRING()), '_', col('name')).alias('id')) + table.execute().print() + # +----+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | op | id | name | telephone | country | + # +----+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + # | +I | 1_Flink | Flink | 123 | Germany | + # | +I | 2_hello | hello | 135 | China | + # | +I | 3_world | world | 124 | USA | + # | +I | 4_PyFlink | PyFlink | 32 | China | + # +----+--------------------------------+--------------------------------+--------------------------------+--------------------------------+ + + +def row_operations(): + t_env = TableEnvironment.create(EnvironmentSettings.in_streaming_mode()) + + # define the source + table = t_env.from_elements( + elements=[ + (1, '{"name": "Flink", "tel": 123, "addr": {"country": "Germany", "city": "Berlin"}}'), + (2, '{"name": "hello", "tel": 135, "addr": {"country": "China", "city": "Shanghai"}}'), + (3, '{"name": "world", "tel": 124, "addr": {"country": "China", "city": "NewYork"}}'), + (4, '{"name": "PyFlink", "tel": 32, "addr": {"country": "China", "city": "Hangzhou"}}') + ], + schema=['id', 'data']) + + # map operation + @udf(result_type=DataTypes.ROW([DataTypes.FIELD("id", DataTypes.BIGINT()), + DataTypes.FIELD("country", DataTypes.STRING())])) + def extract_country(input_row: Row): + data = json.loads(input_row.data) + return Row(input_row.id, data['addr']['country']) + + table.map(extract_country) \ + .execute().print() + # +----+----------------------+--------------------------------+ + # | op | _c0 | _c1 | + # +----+----------------------+--------------------------------+ + # | +I | 1 | Germany | + # | +I | 2 | China | + # | +I | 3 | China | + # | +I | 4 | China | + # +----+----------------------+--------------------------------+ + + # flat_map operation + @udtf(result_types=[DataTypes.BIGINT(), DataTypes.STRING()]) + def extract_city(input_row: Row): + data = json.loads(input_row.data) + yield input_row.id, data['addr']['city'] + + table.flat_map(extract_city) \ + .execute().print() + # +----+----------------------+--------------------------------+ + # | op | f0 | f1 | + # +----+----------------------+--------------------------------+ + # | +I | 1 | Berlin | + # | +I | 2 | Shanghai | + # | +I | 3 | NewYork | + # | +I | 4 | Hangzhou | + # +----+----------------------+--------------------------------+ + + # aggregate operation + class CountAndSumAggregateFunction(AggregateFunction): + + def get_value(self, accumulator): + return Row(accumulator[0], accumulator[1]) + + def create_accumulator(self): + return Row(0, 0) + + def accumulate(self, accumulator, input_row): + accumulator[0] += 1 + accumulator[1] += int(input_row.tel) + + def retract(self, accumulator, input_row): + accumulator[0] -= 1 + accumulator[1] -= int(input_row.tel) + + def merge(self, accumulator, accumulators): + for other_acc in accumulators: + accumulator[0] += other_acc[0] + accumulator[1] += other_acc[1] + + def get_accumulator_type(self): + return DataTypes.ROW( + [DataTypes.FIELD("cnt", DataTypes.BIGINT()), + DataTypes.FIELD("sum", DataTypes.BIGINT())]) + + def get_result_type(self): + return DataTypes.ROW( + [DataTypes.FIELD("cnt", DataTypes.BIGINT()), + DataTypes.FIELD("sum", DataTypes.BIGINT())]) + + count_sum = udaf(CountAndSumAggregateFunction()) + table.add_columns( + col('data').json_value('$.name', DataTypes.STRING()).alias('name'), + col('data').json_value('$.tel', DataTypes.STRING()).alias('tel'), + col('data').json_value('$.addr.country', DataTypes.STRING()).alias('country')) \ + .group_by(col('country')) \ + .aggregate(count_sum.alias("cnt", "sum")) \ + .select(col('country'), col('cnt'), col('sum')) \ + .execute().print() + # +----+--------------------------------+----------------------+----------------------+ + # | op | country | cnt | sum | + # +----+--------------------------------+----------------------+----------------------+ + # | +I | China | 3 | 291 | + # | +I | Germany | 1 | 123 | + # +----+--------------------------------+----------------------+----------------------+ + + # flat_aggregate operation + class Top2(TableAggregateFunction): + + def emit_value(self, accumulator): + for v in accumulator: + if v: + yield Row(v) + + def create_accumulator(self): + return [None, None] + + def accumulate(self, accumulator, input_row): + tel = int(input_row.tel) + if accumulator[0] is None or tel > accumulator[0]: + accumulator[1] = accumulator[0] + accumulator[0] = tel + elif accumulator[1] is None or tel > accumulator[1]: + accumulator[1] = tel + + def get_accumulator_type(self): + return DataTypes.ARRAY(DataTypes.BIGINT()) + + def get_result_type(self): + return DataTypes.ROW( + [DataTypes.FIELD("tel", DataTypes.BIGINT())]) + + top2 = udtaf(Top2()) + table.add_columns( + col('data').json_value('$.name', DataTypes.STRING()).alias('name'), + col('data').json_value('$.tel', DataTypes.STRING()).alias('tel'), + col('data').json_value('$.addr.country', DataTypes.STRING()).alias('country')) \ + .group_by(col('country')) \ + .flat_aggregate(top2) \ + .select(col('country'), col('tel')) \ + .execute().print() + # +----+--------------------------------+----------------------+ + # | op | country | tel | + # +----+--------------------------------+----------------------+ + # | +I | China | 135 | + # | +I | China | 124 | + # | +I | Germany | 123 | + # +----+--------------------------------+----------------------+ + + +if __name__ == '__main__': + logging.basicConfig(stream=sys.stdout, level=logging.INFO, format="%(message)s") + + basic_operations() + sql_operations() + column_operations() + row_operations() diff --git a/flink-python/tox.ini b/flink-python/tox.ini index 7a6a934b0f78c..44149024bbe08 100644 --- a/flink-python/tox.ini +++ b/flink-python/tox.ini @@ -48,7 +48,7 @@ install_command = {toxinidir}/dev/install_command.sh {opts} {packages} # up to 100 characters in length, not 79. ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 max-line-length=100 -exclude=.tox/*,dev/*,lib/*,target/*,build/*,dist/*,pyflink/shell.py,.eggs/*,pyflink/fn_execution/tests/process_mode_test_data.py,pyflink/fn_execution/*_pb2.py +exclude=.tox/*,dev/*,lib/*,target/*,build/*,dist/*,pyflink/shell.py,.eggs/*,pyflink/fn_execution/tests/process_mode_test_data.py,pyflink/fn_execution/*_pb2.py,pyflink/examples/table/basic_operations.py [mypy] files=pyflink/common/*.py,pyflink/table/*.py,pyflink/datastream/*.py,pyflink/metrics/*.py From b070a7bb424da40ce0839a1151b110701de48a2c Mon Sep 17 00:00:00 2001 From: Dian Fu Date: Wed, 13 Apr 2022 19:00:36 +0800 Subject: [PATCH 181/258] [FLINK-27223][python] Fix the state access problem when python.state.cache-size is set to 0 This closes #19457. --- flink-python/pyflink/fn_execution/state_impl.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/flink-python/pyflink/fn_execution/state_impl.py b/flink-python/pyflink/fn_execution/state_impl.py index 590c37d465ece..38668dd4e83c1 100644 --- a/flink-python/pyflink/fn_execution/state_impl.py +++ b/flink-python/pyflink/fn_execution/state_impl.py @@ -149,7 +149,8 @@ def get_internal_state(self): return self._internal_state def _maybe_clear_write_cache(self): - if self._cache_type == SynchronousKvRuntimeState.CacheType.DISABLE_CACHE: + if self._cache_type == SynchronousKvRuntimeState.CacheType.DISABLE_CACHE or \ + self._remote_state_backend._state_cache_size <= 0: self._internal_state.commit() self._internal_state._cleared = False self._internal_state._added_elements = [] From bafaeec66bd770e32227c179fe6dd1c76241f610 Mon Sep 17 00:00:00 2001 From: Matthias Pohl Date: Mon, 4 Apr 2022 14:59:16 +0200 Subject: [PATCH 182/258] [FLINK-27050][runtime] Removes default RpcSystem instance Having a default RpcSystem in TestingDispatcher.Builder caused threads being spawned without cleanup. Instead, we should rely on the RpcSystem instance provided by the test. --- .../dispatcher/AbstractDispatcherTest.java | 1 - .../dispatcher/DispatcherCleanupITCase.java | 8 ++++---- .../DispatcherResourceCleanupTest.java | 6 ++++-- .../runtime/dispatcher/DispatcherTest.java | 20 ++++++++++--------- .../runtime/dispatcher/TestingDispatcher.java | 9 +-------- 5 files changed, 20 insertions(+), 24 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java index 31f034e94ed62..77cb28f169f64 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/AbstractDispatcherTest.java @@ -103,7 +103,6 @@ public void setUp() throws Exception { protected TestingDispatcher.Builder createTestingDispatcherBuilder() { return TestingDispatcher.builder() - .setRpcService(rpcService) .setConfiguration(configuration) .setHeartbeatServices(heartbeatServices) .setHighAvailabilityServices(haServices) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java index d305011a7e929..2383eae943c80 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherCleanupITCase.java @@ -177,7 +177,7 @@ public void testCleanupThroughRetries() throws Exception { haServices, UnregisteredMetricGroups .createUnregisteredJobManagerMetricGroup())) - .build(); + .build(rpcService); dispatcher.start(); toTerminate.add(dispatcher); @@ -223,7 +223,7 @@ public void testCleanupNotCancellable() throws Exception { final Dispatcher dispatcher = createTestingDispatcherBuilder() .setJobManagerRunnerRegistry(jobManagerRunnerRegistry) - .build(); + .build(rpcService); dispatcher.start(); toTerminate.add(dispatcher); @@ -290,7 +290,7 @@ public void testCleanupAfterLeadershipChange() throws Exception { configuration.set( CleanupOptions.CLEANUP_STRATEGY, CleanupOptions.NONE_PARAM_VALUES.iterator().next()); - final Dispatcher dispatcher = createTestingDispatcherBuilder().build(); + final Dispatcher dispatcher = createTestingDispatcherBuilder().build(rpcService); dispatcher.start(); toTerminate.add(dispatcher); @@ -332,7 +332,7 @@ public void testCleanupAfterLeadershipChange() throws Exception { final Dispatcher secondDispatcher = createTestingDispatcherBuilder() .setRecoveredDirtyJobs(haServices.getJobResultStore().getDirtyResults()) - .build(); + .build(rpcService); secondDispatcher.start(); toTerminate.add(secondDispatcher); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java index 73cf51a7bf6b8..c017e72f555a5 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherResourceCleanupTest.java @@ -165,7 +165,10 @@ private void startDispatcher( TestingDispatcher.Builder dispatcherBuilder, JobManagerRunnerFactory jobManagerRunnerFactory) throws Exception { - dispatcher = dispatcherBuilder.setJobManagerRunnerFactory(jobManagerRunnerFactory).build(); + dispatcher = + dispatcherBuilder + .setJobManagerRunnerFactory(jobManagerRunnerFactory) + .build(rpcService); dispatcher.start(); @@ -176,7 +179,6 @@ private TestingDispatcher.Builder createTestingDispatcherBuilder() { final JobManagerRunnerRegistry jobManagerRunnerRegistry = new DefaultJobManagerRunnerRegistry(2); return TestingDispatcher.builder() - .setRpcService(rpcService) .setBlobServer(blobServer) .setJobManagerRunnerRegistry(jobManagerRunnerRegistry) .setFatalErrorHandler(testingFatalErrorHandlerResource.getFatalErrorHandler()) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index e68c52750de81..e806583c8cc20 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -169,7 +169,7 @@ private TestingDispatcher createAndStartDispatcher( .setJobManagerRunnerFactory(jobManagerRunnerFactory) .setJobGraphWriter(haServices.getJobGraphStore()) .setJobResultStore(haServices.getJobResultStore()) - .build(); + .build(rpcService); dispatcher.start(); return dispatcher; } @@ -251,7 +251,7 @@ public void testDuplicateJobSubmissionWithRunningJobId() throws Exception { new ExpectedJobIdJobManagerRunnerFactory( jobId, createdJobManagerRunnerLatch)) .setRecoveredJobs(Collections.singleton(jobGraph)) - .build(); + .build(rpcService); dispatcher.start(); final DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); @@ -483,7 +483,7 @@ public void testNoHistoryServerArchiveCreatedForSuspendedJob() throws Exception archiveAttemptFuture.complete(null); return CompletableFuture.completedFuture(null); }) - .build(); + .build(rpcService); dispatcher.start(); jobMasterLeaderElectionService.isLeader(UUID.randomUUID()); DispatcherGateway dispatcherGateway = dispatcher.getSelfGateway(DispatcherGateway.class); @@ -678,7 +678,7 @@ public void testFatalErrorIfRecoveredJobsCannotBeStarted() throws Exception { createTestingDispatcherBuilder() .setJobManagerRunnerFactory(jobManagerRunnerFactory) .setRecoveredJobs(Collections.singleton(JobGraphTestUtils.emptyJobGraph())) - .build(); + .build(rpcService); dispatcher.start(); @@ -723,7 +723,7 @@ public void testThatDirtilyFinishedJobsNotBeingRetriggered() throws Exception { createTestingDispatcherBuilder() .setRecoveredJobs(Collections.singleton(jobGraph)) .setRecoveredDirtyJobs(Collections.singleton(jobResult)) - .build(); + .build(rpcService); } @Test @@ -752,7 +752,7 @@ public void testJobCleanupWithoutRecoveredJobGraph() throws Exception { dispatcherBootstrapLatch.trigger(); return new NoOpDispatcherBootstrap(); }) - .build(); + .build(rpcService); dispatcher.start(); @@ -779,7 +779,9 @@ public void testPersistedJobGraphWhenDispatcherIsShutDown() throws Exception { haServices.setJobGraphStore(submittedJobGraphStore); dispatcher = - createTestingDispatcherBuilder().setJobGraphWriter(submittedJobGraphStore).build(); + createTestingDispatcherBuilder() + .setJobGraphWriter(submittedJobGraphStore) + .build(rpcService); dispatcher.start(); @@ -899,7 +901,7 @@ public void testOnRemovedJobGraphDoesNotCleanUpHAFiles() throws Exception { createTestingDispatcherBuilder() .setRecoveredJobs(Collections.singleton(jobGraph)) .setJobGraphWriter(testingJobGraphStore) - .build(); + .build(rpcService); dispatcher.start(); final CompletableFuture processFuture = @@ -1068,7 +1070,7 @@ public void testOnlyRecoveredJobsAreRetainedInTheBlobServer() throws Exception { dispatcher = createTestingDispatcherBuilder() .setRecoveredJobs(Collections.singleton(new JobGraph(jobId1, "foobar"))) - .build(); + .build(rpcService); Assertions.assertThat(blobServer.getFile(jobId1, blobKey1)).hasBinaryContent(fileContent); Assertions.assertThatThrownBy(() -> blobServer.getFile(jobId2, blobKey2)) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java index 444c77e8a553f..9c18622d136cc 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/TestingDispatcher.java @@ -41,7 +41,6 @@ import org.apache.flink.runtime.resourcemanager.utils.TestingResourceManagerGateway; import org.apache.flink.runtime.rpc.FatalErrorHandler; import org.apache.flink.runtime.rpc.RpcService; -import org.apache.flink.runtime.rpc.TestingRpcService; import org.apache.flink.runtime.scheduler.ExecutionGraphInfo; import org.apache.flink.runtime.util.TestingFatalErrorHandler; import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever; @@ -177,7 +176,6 @@ public static TestingDispatcher.Builder builder() { } public static class Builder { - private RpcService rpcService = new TestingRpcService(); private DispatcherId fencingToken = DispatcherId.generate(); private Collection recoveredJobs = Collections.emptyList(); @Nullable private Collection recoveredDirtyJobs = null; @@ -217,11 +215,6 @@ public static class Builder { new DefaultJobManagerRunnerRegistry(1); @Nullable private ResourceCleanerFactory resourceCleanerFactory; - public Builder setRpcService(RpcService rpcService) { - this.rpcService = rpcService; - return this; - } - public Builder setFencingToken(DispatcherId fencingToken) { this.fencingToken = fencingToken; return this; @@ -354,7 +347,7 @@ private ResourceCleanerFactory createDefaultResourceCleanerFactory() { jobManagerMetricGroup); } - public TestingDispatcher build() throws Exception { + public TestingDispatcher build(RpcService rpcService) throws Exception { return new TestingDispatcher( rpcService, fencingToken, From 9fc89a05f128ab645b73687f240fb14b57790fc6 Mon Sep 17 00:00:00 2001 From: Gen Luo Date: Wed, 6 Apr 2022 16:38:39 +0800 Subject: [PATCH 183/258] [FLINK-26394][checkpoint] Cancel the checkpoint completable future when checkpoint is aborting. --- .../checkpoint/CheckpointCoordinator.java | 17 +++++-- .../runtime/checkpoint/PendingCheckpoint.java | 7 ++- .../checkpoint/CheckpointCoordinatorTest.java | 44 +++++++++++++++++++ .../checkpoint/PendingCheckpointTest.java | 3 +- 4 files changed, 65 insertions(+), 6 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 72a6b7032deba..2d47d79f06351 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -536,6 +536,8 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) { boolean initializeBaseLocations = !baseLocationsForCheckpointInitialized; baseLocationsForCheckpointInitialized = true; + CompletableFuture masterTriggerCompletionPromise = new CompletableFuture<>(); + final CompletableFuture pendingCheckpointCompletableFuture = checkpointPlanFuture .thenApplyAsync( @@ -560,7 +562,8 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) { checkpointInfo.f0, request.isPeriodic, checkpointInfo.f1, - request.getOnCompletionFuture()), + request.getOnCompletionFuture(), + masterTriggerCompletionPromise), timer); final CompletableFuture coordinatorCheckpointsComplete = @@ -615,8 +618,12 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) { }, timer); + FutureUtils.forward( + CompletableFuture.allOf(masterStatesComplete, coordinatorCheckpointsComplete), + masterTriggerCompletionPromise); + FutureUtils.assertNoException( - CompletableFuture.allOf(masterStatesComplete, coordinatorCheckpointsComplete) + masterTriggerCompletionPromise .handleAsync( (ignored, throwable) -> { final PendingCheckpoint checkpoint = @@ -778,7 +785,8 @@ private PendingCheckpoint createPendingCheckpoint( CheckpointPlan checkpointPlan, boolean isPeriodic, long checkpointID, - CompletableFuture onCompletionPromise) { + CompletableFuture onCompletionPromise, + CompletableFuture masterTriggerCompletionPromise) { synchronized (lock) { try { @@ -803,7 +811,8 @@ private PendingCheckpoint createPendingCheckpoint( masterHooks.keySet(), props, onCompletionPromise, - pendingCheckpointStats); + pendingCheckpointStats, + masterTriggerCompletionPromise); synchronized (lock) { pendingCheckpoints.put(checkpointID, checkpoint); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index b4bd8eacc5ebc..8ca6e22020e8b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -112,6 +112,8 @@ public enum TaskAcknowledgeResult { @Nullable private final PendingCheckpointStats pendingCheckpointStats; + private final CompletableFuture masterTriggerCompletionPromise; + /** Target storage location to persist the checkpoint metadata to. */ @Nullable private CheckpointStorageLocation targetLocation; @@ -136,7 +138,8 @@ public PendingCheckpoint( Collection masterStateIdentifiers, CheckpointProperties props, CompletableFuture onCompletionPromise, - @Nullable PendingCheckpointStats pendingCheckpointStats) { + @Nullable PendingCheckpointStats pendingCheckpointStats, + CompletableFuture masterTriggerCompletionPromise) { checkArgument( checkpointPlan.getTasksToWaitFor().size() > 0, "Checkpoint needs at least one vertex that commits the checkpoint"); @@ -166,6 +169,7 @@ public PendingCheckpoint( this.acknowledgedTasks = new HashSet<>(checkpointPlan.getTasksToWaitFor().size()); this.onCompletionPromise = checkNotNull(onCompletionPromise); this.pendingCheckpointStats = pendingCheckpointStats; + this.masterTriggerCompletionPromise = checkNotNull(masterTriggerCompletionPromise); } // -------------------------------------------------------------------------------------------- @@ -544,6 +548,7 @@ public void abort( try { failureCause = new CheckpointException(reason, cause); onCompletionPromise.completeExceptionally(failureCause); + masterTriggerCompletionPromise.completeExceptionally(failureCause); assertAbortSubsumedForced(reason); } finally { dispose(true, checkpointsCleaner, postCleanup, executor); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 9d3a87ddd848f..234765b298fc0 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -3658,6 +3658,50 @@ public void testNotifyCheckpointAbortionInOperatorCoordinator() throws Exception } } + @Test + public void testTimeoutWhileCheckpointOperatorCoordinatorNotFinishing() throws Exception { + JobVertexID jobVertexID = new JobVertexID(); + ExecutionGraph graph = + new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() + .addJobVertex(jobVertexID) + .build(); + + CheckpointCoordinatorTestingUtils.MockOperatorCoordinatorCheckpointContext context = + new CheckpointCoordinatorTestingUtils + .MockOperatorCheckpointCoordinatorContextBuilder() + .setOperatorID(new OperatorID()) + .setOnCallingCheckpointCoordinator( + (ignored, future) -> { + // Never complete + }) + .build(); + + ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor(); + CheckpointCoordinator checkpointCoordinator = + new CheckpointCoordinatorBuilder() + .setExecutionGraph(graph) + .setCheckpointCoordinatorConfiguration( + CheckpointCoordinatorConfiguration.builder() + .setCheckpointTimeout(10) + .build()) + .setTimer(manuallyTriggeredScheduledExecutor) + .setCoordinatorsToCheckpoint(Collections.singleton(context)) + .build(); + try { + checkpointCoordinator.triggerCheckpoint(false); + manuallyTriggeredScheduledExecutor.triggerAll(); + Assert.assertTrue(checkpointCoordinator.isTriggering()); + + manuallyTriggeredScheduledExecutor.triggerNonPeriodicScheduledTasks(); + manuallyTriggeredScheduledExecutor.triggerAll(); + + Assert.assertFalse(checkpointCoordinator.isTriggering()); + } finally { + checkpointCoordinator.shutdown(); + executorService.shutdownNow(); + } + } + @Test public void testReportLatestCompletedCheckpointIdWithAbort() throws Exception { JobVertexID jobVertexID = new JobVertexID(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index 392bd0ef67c5e..fc8d57f9f936d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -644,7 +644,8 @@ private PendingCheckpoint createPendingCheckpoint( masterStateIdentifiers, props, new CompletableFuture<>(), - null); + null, + new CompletableFuture<>()); pendingCheckpoint.setCheckpointTargetLocation(location); return pendingCheckpoint; } From 0c718666476ae469fb825e8fdf362470d7af2488 Mon Sep 17 00:00:00 2001 From: "Jiangjie (Becket) Qin" Date: Thu, 14 Apr 2022 17:56:59 +0800 Subject: [PATCH 184/258] Revert "[FLINK-26394][checkpoint] Cancel the checkpoint completable future when checkpoint is aborting." This reverts commit 9fc89a05f128ab645b73687f240fb14b57790fc6 due to a FLINK-27148. --- .../checkpoint/CheckpointCoordinator.java | 17 ++----- .../runtime/checkpoint/PendingCheckpoint.java | 7 +-- .../checkpoint/CheckpointCoordinatorTest.java | 44 ------------------- .../checkpoint/PendingCheckpointTest.java | 3 +- 4 files changed, 6 insertions(+), 65 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java index 2d47d79f06351..72a6b7032deba 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java @@ -536,8 +536,6 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) { boolean initializeBaseLocations = !baseLocationsForCheckpointInitialized; baseLocationsForCheckpointInitialized = true; - CompletableFuture masterTriggerCompletionPromise = new CompletableFuture<>(); - final CompletableFuture pendingCheckpointCompletableFuture = checkpointPlanFuture .thenApplyAsync( @@ -562,8 +560,7 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) { checkpointInfo.f0, request.isPeriodic, checkpointInfo.f1, - request.getOnCompletionFuture(), - masterTriggerCompletionPromise), + request.getOnCompletionFuture()), timer); final CompletableFuture coordinatorCheckpointsComplete = @@ -618,12 +615,8 @@ private void startTriggeringCheckpoint(CheckpointTriggerRequest request) { }, timer); - FutureUtils.forward( - CompletableFuture.allOf(masterStatesComplete, coordinatorCheckpointsComplete), - masterTriggerCompletionPromise); - FutureUtils.assertNoException( - masterTriggerCompletionPromise + CompletableFuture.allOf(masterStatesComplete, coordinatorCheckpointsComplete) .handleAsync( (ignored, throwable) -> { final PendingCheckpoint checkpoint = @@ -785,8 +778,7 @@ private PendingCheckpoint createPendingCheckpoint( CheckpointPlan checkpointPlan, boolean isPeriodic, long checkpointID, - CompletableFuture onCompletionPromise, - CompletableFuture masterTriggerCompletionPromise) { + CompletableFuture onCompletionPromise) { synchronized (lock) { try { @@ -811,8 +803,7 @@ private PendingCheckpoint createPendingCheckpoint( masterHooks.keySet(), props, onCompletionPromise, - pendingCheckpointStats, - masterTriggerCompletionPromise); + pendingCheckpointStats); synchronized (lock) { pendingCheckpoints.put(checkpointID, checkpoint); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java index 8ca6e22020e8b..b4bd8eacc5ebc 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/PendingCheckpoint.java @@ -112,8 +112,6 @@ public enum TaskAcknowledgeResult { @Nullable private final PendingCheckpointStats pendingCheckpointStats; - private final CompletableFuture masterTriggerCompletionPromise; - /** Target storage location to persist the checkpoint metadata to. */ @Nullable private CheckpointStorageLocation targetLocation; @@ -138,8 +136,7 @@ public PendingCheckpoint( Collection masterStateIdentifiers, CheckpointProperties props, CompletableFuture onCompletionPromise, - @Nullable PendingCheckpointStats pendingCheckpointStats, - CompletableFuture masterTriggerCompletionPromise) { + @Nullable PendingCheckpointStats pendingCheckpointStats) { checkArgument( checkpointPlan.getTasksToWaitFor().size() > 0, "Checkpoint needs at least one vertex that commits the checkpoint"); @@ -169,7 +166,6 @@ public PendingCheckpoint( this.acknowledgedTasks = new HashSet<>(checkpointPlan.getTasksToWaitFor().size()); this.onCompletionPromise = checkNotNull(onCompletionPromise); this.pendingCheckpointStats = pendingCheckpointStats; - this.masterTriggerCompletionPromise = checkNotNull(masterTriggerCompletionPromise); } // -------------------------------------------------------------------------------------------- @@ -548,7 +544,6 @@ public void abort( try { failureCause = new CheckpointException(reason, cause); onCompletionPromise.completeExceptionally(failureCause); - masterTriggerCompletionPromise.completeExceptionally(failureCause); assertAbortSubsumedForced(reason); } finally { dispose(true, checkpointsCleaner, postCleanup, executor); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java index 234765b298fc0..9d3a87ddd848f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorTest.java @@ -3658,50 +3658,6 @@ public void testNotifyCheckpointAbortionInOperatorCoordinator() throws Exception } } - @Test - public void testTimeoutWhileCheckpointOperatorCoordinatorNotFinishing() throws Exception { - JobVertexID jobVertexID = new JobVertexID(); - ExecutionGraph graph = - new CheckpointCoordinatorTestingUtils.CheckpointExecutionGraphBuilder() - .addJobVertex(jobVertexID) - .build(); - - CheckpointCoordinatorTestingUtils.MockOperatorCoordinatorCheckpointContext context = - new CheckpointCoordinatorTestingUtils - .MockOperatorCheckpointCoordinatorContextBuilder() - .setOperatorID(new OperatorID()) - .setOnCallingCheckpointCoordinator( - (ignored, future) -> { - // Never complete - }) - .build(); - - ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor(); - CheckpointCoordinator checkpointCoordinator = - new CheckpointCoordinatorBuilder() - .setExecutionGraph(graph) - .setCheckpointCoordinatorConfiguration( - CheckpointCoordinatorConfiguration.builder() - .setCheckpointTimeout(10) - .build()) - .setTimer(manuallyTriggeredScheduledExecutor) - .setCoordinatorsToCheckpoint(Collections.singleton(context)) - .build(); - try { - checkpointCoordinator.triggerCheckpoint(false); - manuallyTriggeredScheduledExecutor.triggerAll(); - Assert.assertTrue(checkpointCoordinator.isTriggering()); - - manuallyTriggeredScheduledExecutor.triggerNonPeriodicScheduledTasks(); - manuallyTriggeredScheduledExecutor.triggerAll(); - - Assert.assertFalse(checkpointCoordinator.isTriggering()); - } finally { - checkpointCoordinator.shutdown(); - executorService.shutdownNow(); - } - } - @Test public void testReportLatestCompletedCheckpointIdWithAbort() throws Exception { JobVertexID jobVertexID = new JobVertexID(); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java index fc8d57f9f936d..392bd0ef67c5e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/PendingCheckpointTest.java @@ -644,8 +644,7 @@ private PendingCheckpoint createPendingCheckpoint( masterStateIdentifiers, props, new CompletableFuture<>(), - null, - new CompletableFuture<>()); + null); pendingCheckpoint.setCheckpointTargetLocation(location); return pendingCheckpoint; } From 5f4178bb782dafb4520b1459e51e417d641a03a0 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 11 Apr 2022 17:33:51 +0200 Subject: [PATCH 185/258] [hotfix][tests] Allow retrieval of termination future for running jobs --- .../apache/flink/runtime/dispatcher/Dispatcher.java | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 5150244134964..6d8897e3a659b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -1215,15 +1215,10 @@ private CompletableFuture waitForTerminatingJob( getMainThreadExecutor()); } + @VisibleForTesting CompletableFuture getJobTerminationFuture(JobID jobId) { - if (jobManagerRunnerRegistry.isRegistered(jobId)) { - return FutureUtils.completedExceptionally( - new DispatcherException( - String.format("Job with job id %s is still running.", jobId))); - } else { - return jobManagerRunnerTerminationFutures.getOrDefault( - jobId, CompletableFuture.completedFuture(null)); - } + return jobManagerRunnerTerminationFutures.getOrDefault( + jobId, CompletableFuture.completedFuture(null)); } private void registerDispatcherMetrics(MetricGroup jobManagerMetricGroup) { From df874ad72513771b3a3f2473dbcd60d672d668fa Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Mon, 11 Apr 2022 12:00:41 +0200 Subject: [PATCH 186/258] [hotfix][tests] Wait for JobManagerRunner termination Need to make sure the job manager runner is complete, because the test runner does not implement required methods to query job details. --- .../flink/runtime/dispatcher/DispatcherTest.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java index e806583c8cc20..b2476562b104f 100755 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/dispatcher/DispatcherTest.java @@ -420,7 +420,7 @@ public void testCancellationOfCanceledTerminalDoesNotThrowException() throws Exc .build()))); // wait for job to finish - dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); // sanity check assertThat( dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get(), is(JobStatus.CANCELED)); @@ -454,7 +454,7 @@ public void testCancellationOfNonCanceledTerminalJobFailsWithAppropriateExceptio .build()))); // wait for job to finish - dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); // sanity check assertThat( dispatcherGateway.requestJobStatus(jobId, TIMEOUT).get(), is(JobStatus.FINISHED)); @@ -541,7 +541,7 @@ public void testJobManagerRunnerInitializationFailureFailsJob() throws Exception testFailure)); // wait till job has failed - dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); // get failure cause ArchivedExecutionGraph execGraph = @@ -949,7 +949,7 @@ public void testRequestMultipleJobDetails_returnsSuspendedJobs() throws Exceptio jobMasterLeaderElectionService.isLeader(UUID.randomUUID()); dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); assertOnlyContainsSingleJobWithState( JobStatus.SUSPENDED, dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get()); @@ -995,6 +995,8 @@ public void testRequestMultipleJobDetails_returnsFinishedOverSuspendedJob() thro // run second job, which completes with FINISHED dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); + assertOnlyContainsSingleJobWithState( JobStatus.FINISHED, dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get()); } @@ -1010,7 +1012,7 @@ public void testRequestMultipleJobDetails_isSerializable() throws Exception { jobMasterLeaderElectionService.isLeader(UUID.randomUUID()); dispatcherGateway.submitJob(jobGraph, TIMEOUT).get(); - dispatcherGateway.requestJobResult(jobId, TIMEOUT).get(); + dispatcher.getJobTerminationFuture(jobId, TIMEOUT).get(); final MultipleJobsDetails multipleJobsDetails = dispatcherGateway.requestMultipleJobDetails(TIMEOUT).get(); From 2d5bebb91df23a3c4dd60506984f1b0cfb539db5 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 8 Apr 2022 12:28:13 +0200 Subject: [PATCH 187/258] [FLINK-27140][coordination] Write job result in ioExecutor --- .../flink/runtime/dispatcher/Dispatcher.java | 89 +++++++++++-------- .../runtime/dispatcher/MiniDispatcher.java | 43 +++++---- 2 files changed, 79 insertions(+), 53 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java index 6d8897e3a659b..e6c9e74b443bd 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/Dispatcher.java @@ -612,10 +612,12 @@ private void runJob(JobManagerRunner jobManagerRunner, ExecutionType executionTy return handleJobManagerRunnerResult( jobManagerRunnerResult, executionType); } else { - return jobManagerRunnerFailed(jobId, throwable); + return CompletableFuture.completedFuture( + jobManagerRunnerFailed(jobId, throwable)); } }, - getMainThreadExecutor()); + getMainThreadExecutor()) + .thenCompose(Function.identity()); final CompletableFuture jobTerminationFuture = cleanupJobStateFuture.thenCompose( @@ -644,13 +646,14 @@ private Void logCleanupErrorWarning(JobID jobId, Throwable cleanupError) { return null; } - private CleanupJobState handleJobManagerRunnerResult( + private CompletableFuture handleJobManagerRunnerResult( JobManagerRunnerResult jobManagerRunnerResult, ExecutionType executionType) { if (jobManagerRunnerResult.isInitializationFailure() && executionType == ExecutionType.RECOVERY) { - return jobManagerRunnerFailed( - jobManagerRunnerResult.getExecutionGraphInfo().getJobId(), - jobManagerRunnerResult.getInitializationFailure()); + return CompletableFuture.completedFuture( + jobManagerRunnerFailed( + jobManagerRunnerResult.getExecutionGraphInfo().getJobId(), + jobManagerRunnerResult.getInitializationFailure())); } return jobReachedTerminalState(jobManagerRunnerResult.getExecutionGraphInfo()); } @@ -985,7 +988,7 @@ private CompletableFuture removeJob(JobID jobId, CleanupJobState cleanupJo case GLOBAL: return globalResourceCleaner .cleanupAsync(jobId) - .thenRun(() -> markJobAsClean(jobId)); + .thenRunAsync(() -> markJobAsClean(jobId), ioExecutor); default: throw new IllegalStateException("Invalid cleanup state: " + cleanupJobState); } @@ -1030,7 +1033,8 @@ protected void onFatalError(Throwable throwable) { fatalErrorHandler.onFatalError(throwable); } - protected CleanupJobState jobReachedTerminalState(ExecutionGraphInfo executionGraphInfo) { + protected CompletableFuture jobReachedTerminalState( + ExecutionGraphInfo executionGraphInfo) { final ArchivedExecutionGraph archivedExecutionGraph = executionGraphInfo.getArchivedExecutionGraph(); final JobStatus terminalJobStatus = archivedExecutionGraph.getState(); @@ -1062,35 +1066,50 @@ protected CleanupJobState jobReachedTerminalState(ExecutionGraphInfo executionGr archiveExecutionGraph(executionGraphInfo); - if (terminalJobStatus.isGloballyTerminalState()) { - final JobID jobId = executionGraphInfo.getJobId(); - try { - if (jobResultStore.hasCleanJobResultEntry(jobId)) { - log.warn( - "Job {} is already marked as clean but clean up was triggered again.", - jobId); - } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) { - jobResultStore.createDirtyResult( - new JobResultEntry( - JobResult.createFrom( - executionGraphInfo.getArchivedExecutionGraph()))); - log.info( - "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.", - jobId); - } - } catch (IOException e) { - fatalErrorHandler.onFatalError( - new FlinkException( - String.format( - "The job %s couldn't be marked as pre-cleanup finished in JobResultStore.", - jobId), - e)); - } + if (!terminalJobStatus.isGloballyTerminalState()) { + return CompletableFuture.completedFuture(CleanupJobState.LOCAL); } - return terminalJobStatus.isGloballyTerminalState() - ? CleanupJobState.GLOBAL - : CleanupJobState.LOCAL; + final CompletableFuture writeFuture = new CompletableFuture<>(); + final JobID jobId = executionGraphInfo.getJobId(); + + ioExecutor.execute( + () -> { + try { + if (jobResultStore.hasCleanJobResultEntry(jobId)) { + log.warn( + "Job {} is already marked as clean but clean up was triggered again.", + jobId); + } else if (!jobResultStore.hasDirtyJobResultEntry(jobId)) { + jobResultStore.createDirtyResult( + new JobResultEntry( + JobResult.createFrom( + executionGraphInfo + .getArchivedExecutionGraph()))); + log.info( + "Job {} has been registered for cleanup in the JobResultStore after reaching a terminal state.", + jobId); + } + } catch (IOException e) { + writeFuture.completeExceptionally(e); + return; + } + writeFuture.complete(null); + }); + + return writeFuture.handleAsync( + (ignored, error) -> { + if (error != null) { + fatalErrorHandler.onFatalError( + new FlinkException( + String.format( + "The job %s couldn't be marked as pre-cleanup finished in JobResultStore.", + executionGraphInfo.getJobId()), + error)); + } + return CleanupJobState.GLOBAL; + }, + getMainThreadExecutor()); } private void archiveExecutionGraph(ExecutionGraphInfo executionGraphInfo) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java index 76afe73bf9d18..85a08332e6653 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/MiniDispatcher.java @@ -129,26 +129,33 @@ public CompletableFuture cancelJob(JobID jobId, Time timeout) { } @Override - protected CleanupJobState jobReachedTerminalState(ExecutionGraphInfo executionGraphInfo) { + protected CompletableFuture jobReachedTerminalState( + ExecutionGraphInfo executionGraphInfo) { final ArchivedExecutionGraph archivedExecutionGraph = executionGraphInfo.getArchivedExecutionGraph(); - final CleanupJobState cleanupHAState = super.jobReachedTerminalState(executionGraphInfo); - - JobStatus jobStatus = - Objects.requireNonNull( - archivedExecutionGraph.getState(), "JobStatus should not be null here."); - if (jobStatus.isGloballyTerminalState() - && (jobCancelled || executionMode == ClusterEntrypoint.ExecutionMode.DETACHED)) { - // shut down if job is cancelled or we don't have to wait for the execution result - // retrieval - log.info( - "Shutting down cluster with state {}, jobCancelled: {}, executionMode: {}", - jobStatus, - jobCancelled, - executionMode); - shutDownFuture.complete(ApplicationStatus.fromJobStatus(jobStatus)); - } + final CompletableFuture cleanupHAState = + super.jobReachedTerminalState(executionGraphInfo); + + return cleanupHAState.thenApply( + cleanupJobState -> { + JobStatus jobStatus = + Objects.requireNonNull( + archivedExecutionGraph.getState(), + "JobStatus should not be null here."); + if (jobStatus.isGloballyTerminalState() + && (jobCancelled + || executionMode == ClusterEntrypoint.ExecutionMode.DETACHED)) { + // shut down if job is cancelled or we don't have to wait for the execution + // result retrieval + log.info( + "Shutting down cluster with state {}, jobCancelled: {}, executionMode: {}", + jobStatus, + jobCancelled, + executionMode); + shutDownFuture.complete(ApplicationStatus.fromJobStatus(jobStatus)); + } - return cleanupHAState; + return cleanupJobState; + }); } } From 9d996ebd2aed6cb0b575dbddb674f55492106231 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Thu, 14 Apr 2022 11:37:32 +0800 Subject: [PATCH 188/258] [FLINK-27231][licence] Fix the SQL Pulsar licence issue --- .../src/main/resources/META-INF/NOTICE | 6 +++++- .../main/resources/META-INF/licences/LICENSE.bouncycastle | 7 +++++++ 2 files changed, 12 insertions(+), 1 deletion(-) create mode 100644 flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/licences/LICENSE.bouncycastle diff --git a/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE index 56ad187e06b3c..db82a498c9e87 100644 --- a/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE @@ -10,8 +10,12 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.pulsar:pulsar-client-admin-api:2.9.1 - org.apache.pulsar:pulsar-client-all:2.9.1 - org.apache.pulsar:pulsar-client-api:2.9.1 +- org.slf4j:jul-to-slf4j:1.7.25 + +This project bundles the following dependencies under the Bouncy Castle license. +See bundled license files for details. + - org.bouncycastle:bcpkix-jdk15on:1.69 - org.bouncycastle:bcprov-ext-jdk15on:1.69 - org.bouncycastle:bcprov-jdk15on:1.69 - org.bouncycastle:bcutil-jdk15on:1.69 -- org.slf4j:jul-to-slf4j:1.7.25 diff --git a/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/licences/LICENSE.bouncycastle b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/licences/LICENSE.bouncycastle new file mode 100644 index 0000000000000..e904785dcafae --- /dev/null +++ b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/licences/LICENSE.bouncycastle @@ -0,0 +1,7 @@ +Copyright (c) 2000 - 2021 The Legion of the Bouncy Castle Inc. (https://www.bouncycastle.org) + +Permission is hereby granted, free of charge, to any person obtaining a copy of this software and associated documentation files (the "Software"), to deal in the Software without restriction, including without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file From 1ebfe8558fe82679d25836f4cd5979a4e6917358 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Thu, 14 Apr 2022 11:39:39 +0800 Subject: [PATCH 189/258] [FLINK-27230][licence] Remove the unused licence entries from Kinesis connector --- .../flink-connector-kinesis/src/main/resources/META-INF/NOTICE | 2 -- 1 file changed, 2 deletions(-) diff --git a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE index 00e24d6d2f510..375ffaf9aa84c 100644 --- a/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-connector-kinesis/src/main/resources/META-INF/NOTICE @@ -54,8 +54,6 @@ This project bundles the following dependencies under the Apache Software Licens - io.netty:netty-transport-native-unix-common:4.1.70.Final - com.typesafe.netty:netty-reactive-streams-http:2.0.5 - com.typesafe.netty:netty-reactive-streams:2.0.5 -- commons-logging:commons-logging:1.1.3 -- com.fasterxml.jackson.core:jackson-core:2.13.2 This project bundles the following dependencies under the BSD license. See bundled license files for details. From 15d409b158bc293520437e0e898b0132098d82e8 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Thu, 14 Apr 2022 11:42:02 +0800 Subject: [PATCH 190/258] [FLINK-27233][licence] Remove the unused licence entries from Elasticsearch7 connector --- .../src/main/resources/META-INF/NOTICE | 5 ----- 1 file changed, 5 deletions(-) diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE index 718480db4e420..ed85ea436c25a 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-elasticsearch7/src/main/resources/META-INF/NOTICE @@ -8,8 +8,6 @@ This project bundles the following dependencies under the Apache Software Licens - com.carrotsearch:hppc:0.8.1 - com.fasterxml.jackson.core:jackson-core:2.13.2 -- com.fasterxml.jackson.core:jackson-databind:2.13.2.2 -- com.fasterxml.jackson.core:jackson-annotations:2.13.2 - com.fasterxml.jackson.dataformat:jackson-dataformat-cbor:2.13.2 - com.fasterxml.jackson.dataformat:jackson-dataformat-smile:2.13.2 - com.fasterxml.jackson.dataformat:jackson-dataformat-yaml:2.13.2 @@ -31,7 +29,6 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.lucene:lucene-queries:8.7.0 - org.apache.lucene:lucene-queryparser:8.7.0 - org.apache.lucene:lucene-sandbox:8.7.0 -- org.apache.lucene:lucene-spatial:8.7.0 - org.apache.lucene:lucene-spatial-extras:8.7.0 - org.apache.lucene:lucene-spatial3d:8.7.0 - org.apache.lucene:lucene-suggest:8.7.0 @@ -41,7 +38,6 @@ This project bundles the following dependencies under the Apache Software Licens - org.elasticsearch:elasticsearch-geo:7.10.2 - org.elasticsearch:elasticsearch-secure-sm:7.10.2 - org.elasticsearch:elasticsearch-x-content:7.10.2 -- org.elasticsearch:elasticsearch-plugin-classloader:7.10.2 - org.elasticsearch.client:elasticsearch-rest-high-level-client:7.10.2 - org.elasticsearch.client:elasticsearch-rest-client:7.10.2 - org.elasticsearch.plugin:aggs-matrix-stats-client:7.10.2 @@ -49,4 +45,3 @@ This project bundles the following dependencies under the Apache Software Licens - org.elasticsearch.plugin:mapper-extras-client:7.10.2 - org.elasticsearch.plugin:parent-join-client:7.10.2 - org.elasticsearch.plugin:rank-eval-client:7.10.2 -- org.lz4:lz4-java:1.8.0 From 4727a20922bbab81faf4247a2a89e820ba122c60 Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Fri, 15 Apr 2022 09:16:56 +0200 Subject: [PATCH 191/258] [FLINK-27222][coordination] Decouple last (al)location from execution history --- .../runtime/executiongraph/Execution.java | 4 ++ .../executiongraph/ExecutionVertex.java | 50 +++++++------------ .../runtime/scheduler/DefaultScheduler.java | 2 +- .../executiongraph/ExecutionVertexTest.java | 14 ++++-- 4 files changed, 32 insertions(+), 38 deletions(-) diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java index 352a184d62f42..487d4e3da77e6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java @@ -288,6 +288,10 @@ public boolean tryAssignResource(final LogicalSlot logicalSlot) { && !taskManagerLocationFuture.isDone()) { taskManagerLocationFuture.complete(logicalSlot.getTaskManagerLocation()); assignedAllocationID = logicalSlot.getAllocationId(); + getVertex() + .setLatestPriorSlotAllocation( + assignedResource.getTaskManagerLocation(), + logicalSlot.getAllocationId()); return true; } else { // free assigned resource and return false diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java index fef96e27f10ed..4bf20f3faabcf 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionVertex.java @@ -36,6 +36,7 @@ import org.apache.flink.runtime.scheduler.strategy.ExecutionVertexID; import org.apache.flink.runtime.taskmanager.TaskManagerLocation; import org.apache.flink.runtime.util.EvictingBoundedList; +import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; @@ -47,7 +48,6 @@ import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; -import java.util.function.Function; import static org.apache.flink.runtime.execution.ExecutionState.FINISHED; import static org.apache.flink.util.Preconditions.checkArgument; @@ -84,6 +84,11 @@ public class ExecutionVertex private final ArrayList inputSplits; + /** This field holds the allocation id of the last successful assignment. */ + @Nullable private TaskManagerLocation lastAssignedLocation; + + @Nullable private AllocationID lastAssignedAllocationID; + // -------------------------------------------------------------------------------------------- /** @@ -280,44 +285,23 @@ public ArchivedExecution getPriorExecutionAttempt(int attemptNumber) { } } - /** - * Gets the latest property from a prior execution that is not null. - * - * @param extractor defining the property to extract - * @param type of the property - * @return Optional containing the latest property if it exists; otherwise {@code - * Optional.empty()}. - */ - private Optional getLatestPriorProperty(Function extractor) { - int index = priorExecutions.size() - 1; - - while (index >= 0 && !priorExecutions.isDroppedIndex(index)) { - final ArchivedExecution archivedExecution = priorExecutions.get(index); - - final T extractedValue = extractor.apply(archivedExecution); - - if (extractedValue != null) { - return Optional.of(extractedValue); - } - - index -= 1; - } - - return Optional.empty(); + void setLatestPriorSlotAllocation( + TaskManagerLocation taskManagerLocation, AllocationID lastAssignedAllocationID) { + this.lastAssignedLocation = Preconditions.checkNotNull(taskManagerLocation); + this.lastAssignedAllocationID = Preconditions.checkNotNull(lastAssignedAllocationID); } /** - * Gets the location where the latest completed/canceled/failed execution of the vertex's task - * happened. + * Gets the location that an execution of this vertex was assigned to. * - * @return The latest prior execution location, or null, if there is none, yet. + * @return The last execution location, or null, if there is none, yet. */ - public Optional findLatestPriorLocation() { - return getLatestPriorProperty(ArchivedExecution::getAssignedResourceLocation); + public Optional findLastLocation() { + return Optional.ofNullable(lastAssignedLocation); } - public Optional findLatestPriorAllocation() { - return getLatestPriorProperty(ArchivedExecution::getAssignedAllocationID); + public Optional findLastAllocation() { + return Optional.ofNullable(lastAssignedAllocationID); } EvictingBoundedList getCopyOfPriorExecutionsList() { @@ -353,7 +337,7 @@ public Optional getPreferredLocationBasedOnState() { // only restore to same execution if it has state if (currentExecution.getTaskRestore() != null && currentExecution.getTaskRestore().getTaskStateSnapshot().hasState()) { - return findLatestPriorLocation(); + return findLastLocation(); } return Optional.empty(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java index 3af4b61ece238..e9e369c3d1915 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultScheduler.java @@ -732,7 +732,7 @@ public ResourceProfile getResourceProfile(final ExecutionVertexID executionVerte @Override public Optional findPriorAllocationId( final ExecutionVertexID executionVertexId) { - return getExecutionVertex(executionVertexId).findLatestPriorAllocation(); + return getExecutionVertex(executionVertexId).findLastAllocation(); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java index f39460d318fb8..26dc6bab119e6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexTest.java @@ -18,6 +18,8 @@ package org.apache.flink.runtime.executiongraph; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; import org.apache.flink.runtime.clusterframework.types.AllocationID; import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutorServiceAdapter; import org.apache.flink.runtime.io.network.partition.ResultPartitionID; @@ -106,9 +108,13 @@ public void testFindLatestAllocationIgnoresFailedAttempts() throws Exception { final JobGraph jobGraph = JobGraphTestUtils.streamingJobGraph(source); final TestingPhysicalSlotProvider withLimitedAmountOfPhysicalSlots = TestingPhysicalSlotProvider.createWithLimitedAmountOfPhysicalSlots(1); + final Configuration configuration = new Configuration(); + // make sure that retrieving the last (al)location is independent from the history size + configuration.set(JobManagerOptions.MAX_ATTEMPTS_HISTORY_SIZE, 1); final SchedulerBase scheduler = SchedulerTestingUtils.newSchedulerBuilder( jobGraph, ComponentMainThreadExecutorServiceAdapter.forMainThread()) + .setJobMasterConfiguration(configuration) .setExecutionSlotAllocatorFactory( SchedulerTestingUtils.newSlotSharingExecutionSlotAllocatorFactory( withLimitedAmountOfPhysicalSlots)) @@ -130,15 +136,15 @@ public void testFindLatestAllocationIgnoresFailedAttempts() throws Exception { cancelExecution(firstExecution); sourceExecutionVertex.resetForNewExecution(); - assertThat(sourceExecutionVertex.findLatestPriorAllocation()).hasValue(allocationId); - assertThat(sourceExecutionVertex.findLatestPriorLocation()).hasValue(taskManagerLocation); + assertThat(sourceExecutionVertex.findLastAllocation()).hasValue(allocationId); + assertThat(sourceExecutionVertex.findLastLocation()).hasValue(taskManagerLocation); final Execution secondExecution = sourceExecutionVertex.getCurrentExecutionAttempt(); cancelExecution(secondExecution); sourceExecutionVertex.resetForNewExecution(); - assertThat(sourceExecutionVertex.findLatestPriorAllocation()).hasValue(allocationId); - assertThat(sourceExecutionVertex.findLatestPriorLocation()).hasValue(taskManagerLocation); + assertThat(sourceExecutionVertex.findLastAllocation()).hasValue(allocationId); + assertThat(sourceExecutionVertex.findLastLocation()).hasValue(taskManagerLocation); } private void cancelExecution(Execution execution) { From c1ff5840127ce851b18c361a9afa7e4bb0ec26c1 Mon Sep 17 00:00:00 2001 From: snailHumming Date: Fri, 8 Apr 2022 15:07:48 +0800 Subject: [PATCH 192/258] [FLINK-25716][docs-zh] Translate "Streaming Concepts" page of "Application Development > Table API & SQL" to Chinese. This closes #19401 Co-authored-by: Roc Marshal --- .../docs/dev/table/concepts/overview.md | 116 ++++++++---------- 1 file changed, 50 insertions(+), 66 deletions(-) diff --git a/docs/content.zh/docs/dev/table/concepts/overview.md b/docs/content.zh/docs/dev/table/concepts/overview.md index 53ef0dc3a7e3d..ed6561c8638a8 100644 --- a/docs/content.zh/docs/dev/table/concepts/overview.md +++ b/docs/content.zh/docs/dev/table/concepts/overview.md @@ -34,106 +34,90 @@ Flink 的 [Table API]({{< ref "docs/dev/table/tableApi" >}}) 和 [SQL]({{< ref " 下面这些页面包含了概念、实际的限制,以及流式数据处理中的一些特定的配置。 -State Management + + +状态管理 ---------------- +流模式下运行的表程序利用了 Flink 作为有状态流处理器的所有能力。 -Table programs that run in streaming mode leverage all capabilities of Flink as a stateful stream -processor. +事实上,一个表程序(Table program)可以配置一个 [state backend]({{< ref "docs/ops/state/state_backends" >}}) +和多个不同的 [checkpoint 选项]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) +以处理对不同状态大小和容错需求。这可以对正在运行的 Table API & SQL 管道(pipeline)生成 savepoint,并在这之后用其恢复应用程序的状态。 -In particular, a table program can be configured with a [state backend]({{< ref "docs/ops/state/state_backends" >}}) -and various [checkpointing options]({{< ref "docs/dev/datastream/fault-tolerance/checkpointing" >}}) -for handling different requirements regarding state size and fault tolerance. It is possible to take -a savepoint of a running Table API & SQL pipeline and to restore the application's state at a later -point in time. + -### State Usage +### 状态使用 -Due to the declarative nature of Table API & SQL programs, it is not always obvious where and how much -state is used within a pipeline. The planner decides whether state is necessary to compute a correct -result. A pipeline is optimized to claim as little state as possible given the current set of optimizer -rules. +由于 Table API & SQL 程序是声明式的,管道内的状态会在哪以及如何被使用并不明确。 Planner 会确认是否需要状态来得到正确的计算结果, +管道会被现有优化规则集优化成尽可能少地使用状态。 {{< hint info >}} -Conceptually, source tables are never kept entirely in state. An implementer deals with logical tables -(i.e. [dynamic tables]({{< ref "docs/dev/table/concepts/dynamic_tables" >}})). Their state requirements -depend on the used operations. +从概念上讲, 源表从来不会在状态中被完全保存。 实现者处理的是逻辑表(即[动态表]({{< ref "docs/dev/table/concepts/dynamic_tables" >}}))。 +它们的状态取决于用到的操作。 {{< /hint >}} -Queries such as `SELECT ... FROM ... WHERE` which only consist of field projections or filters are usually -stateless pipelines. However, operations such as joins, aggregations, or deduplications require keeping -intermediate results in a fault-tolerant storage for which Flink's state abstractions are used. +形如 `SELECT ... FROM ... WHERE` 这种只包含字段映射或过滤器的查询的查询语句通常是无状态的管道。 然而诸如 join、 +聚合或去重操作需要在 Flink 抽象的容错存储内保存中间结果。 {{< hint info >}} -Please refer to the individual operator documentation for more details about how much state is required -and how to limit a potentially ever-growing state size. +请参考独立的算子文档来获取更多关于状态需求量和限制潜在增长状态大小的信息。 {{< /hint >}} -For example, a regular SQL join of two tables requires the operator to keep both input tables in state -entirely. For correct SQL semantics, the runtime needs to assume that a matching could occur at any -point in time from both sides. Flink provides [optimized window and interval joins]({{< ref "docs/dev/table/sql/queries/joins" >}}) -that aim to keep the state size small by exploiting the concept of [watermarks]({{< ref "docs/dev/table/concepts/time_attributes" >}}). +例如对两个表进行 join 操作的普通 SQL 需要算子保存两个表的全部输入。基于正确的 SQL 语义,运行时假设两表会在任意时间点进行匹配。 +Flink 提供了 [优化窗口和时段 Join 聚合]({{< ref "docs/dev/table/sql/queries/joins" >}}) +以利用 [watermarks]({{< ref "docs/dev/table/concepts/time_attributes" >}}) 概念来让保持较小的状态规模。 -Another example is the following query that computes the number of clicks per session. +另一个计算每个会话的点击次数的查询语句的例子如下 ```sql SELECT sessionId, COUNT(*) FROM clicks GROUP BY sessionId; ``` -The `sessionId` attribute is used as a grouping key and the continuous query maintains a count -for each `sessionId` it observes. The `sessionId` attribute is evolving over time and `sessionId` -values are only active until the session ends, i.e., for a limited period of time. However, the -continuous query cannot know about this property of `sessionId` and expects that every `sessionId` -value can occur at any point of time. It maintains a count for each observed `sessionId` value. -Consequently, the total state size of the query is continuously growing as more and more `sessionId` -values are observed. +`sessionId` 是用于分组的键,连续查询(Continuous Query)维护了每个观察到的 `sessionId` 次数。 `sessionId` 属性随着时间逐步演变, +且 `sessionId` 的值只活跃到会话结束(即在有限的时间周期内)。然而连续查询无法得知sessionId的这个性质, +并且预期每个 `sessionId` 值会在任何时间点上出现。这维护了每个可见的 `sessionId` 值。因此总状态量会随着 `sessionId` 的发现不断地增长。 + + + +#### 空闲状态维持时间 -#### Idle State Retention Time +*空间状态位置时间*参数 [`table.exec.state.ttl`]({{< ref "docs/dev/table/config" >}}#table-exec-state-ttl) +定义了状态的键在被更新后要保持多长时间才被移除。在之前的查询例子中,`sessionId` 的数目会在配置的时间内未更新时立刻被移除。 -The *Idle State Retention Time* parameter [`table.exec.state.ttl`]({{< ref "docs/dev/table/config" >}}#table-exec-state-ttl) -defines for how long the state of a key is retained without being updated before it is removed. -For the previous example query, the count of a`sessionId` would be removed as soon as it has not -been updated for the configured period of time. +通过移除状态的键,连续查询会完全忘记它曾经见过这个键。如果一个状态带有曾被移除状态的键被处理了,这条记录将被认为是 +对应键的第一条记录。上述例子中意味着 `sessionId` 会再次从 `0` 开始计数。 -By removing the state of a key, the continuous query completely forgets that it has seen this key -before. If a record with a key, whose state has been removed before, is processed, the record will -be treated as if it was the first record with the respective key. For the example above this means -that the count of a `sessionId` would start again at `0`. + -### Stateful Upgrades and Evolution +### 状态化更新与演化 -Table programs that are executed in streaming mode are intended as *standing queries* which means they -are defined once and are continuously evaluated as static end-to-end pipelines. +表程序在流模式下执行将被视为*标准查询*,这意味着它们被定义一次后将被一直视为静态的端到端 (end-to-end) 管道 -In case of stateful pipelines, any change to both the query or Flink's planner might lead to a completely -different execution plan. This makes stateful upgrades and the evolution of table programs challenging -at the moment. The community is working on improving those shortcomings. +对于这种状态化的管道,对查询和Flink的Planner的改动都有可能导致完全不同的执行计划。这让表程序的状态化的升级和演化在目前而言 +仍具有挑战,社区正致力于改进这一缺点。 -For example, by adding a filter predicate, the optimizer might decide to reorder joins or change the -schema of an intermediate operator. This prevents restoring from a savepoint due to either changed -topology or different column layout within the state of an operator. +例如为了添加过滤谓词,优化器可能决定重排 join 或改变内部算子的 schema。 这会阻碍从 savepoint 的恢复,因为其被改变的拓扑和 +算子状态的列布局差异。 -The query implementer must ensure that the optimized plans before and after the change are compatible. -Use the `EXPLAIN` command in SQL or `table.explain()` in Table API to [get insights]({{< ref "docs/dev/table/common" >}}#explaining-a-table). +查询实现者需要确保改变在优化计划前后是兼容的,在 SQL 中使用 `EXPLAIN` 或在 Table API 中使用 `table.explain()` +可[获取详情]({{< ref "docs/dev/table/common" >}}#explaining-a-table)。 -Since new optimizer rules are continuously added, and operators become more efficient and specialized, -also the upgrade to a newer Flink version could lead to incompatible plans. +由于新的优化器规则正不断地被添加,算子变得更加高效和专用,升级到更新的Flink版本可能造成不兼容的计划。 {{< hint warning >}} -Currently, the framework cannot guarantee that state can be mapped from a savepoint to a new table -operator topology. +当前框架无法保证状态可以从 savepoint 映射到新的算子拓扑上。 -In other words: Savepoints are only supported if both the query and the Flink version remain constant. +换言之: Savepoint 只在查询语句和版本保持恒定的情况下被支持。 {{< /hint >}} -Since the community rejects contributions that modify the optimized plan and the operator topology -in a patch version (e.g. from `1.13.1` to `1.13.2`), it should be safe to upgrade a Table API & SQL -pipeline to a newer bug fix release. However, major-minor upgrades from (e.g. from `1.12` to `1.13`) -are not supported. +由于社区拒绝在版本补丁(如 `1.13.1` 至 `1.13.2`)上对优化计划和算子拓扑进行修改的贡献,对一个 Table API & SQL 管道 +升级到新的 bug fix 发行版应当是安全的。然而主次(major-minor)版本的更新(如 `1.12` 至 `1.13`)不被支持。 + +由于这两个缺点(即修改查询语句和修改Flink版本),我们推荐实现调查升级后的表程序是否可以在切换到实时数据前,被历史数据"暖机" +(即被初始化)。Flink社区正致力于 [混合源]({{< ref "docs/connectors/datastream/hybridsource" >}}) 来让切换变得尽可能方便。 + -For both shortcomings (i.e. modified query and modified Flink version), we recommend to investigate -whether the state of an updated table program can be "warmed up" (i.e. initialized) with historical -data again before switching to real-time data. The Flink community is working on a [hybrid source]({{< ref "docs/connectors/datastream/hybridsource" >}}) -to make this switching as convenient as possible. + 接下来? ----------------- From 1556c3c75535cbb653e4bd8251cb4f0b3ca2e8f0 Mon Sep 17 00:00:00 2001 From: godfreyhe Date: Mon, 18 Apr 2022 10:27:57 +0800 Subject: [PATCH 193/258] [FLINK-27272][table-planner] Fix the incorrect plan for query with local sort is incorrect if adaptive batch scheduler is enabled (cherry picked from commit 456ceb299d0601dee283f718f5d3d0a9d108196e) This closes #19497 --- .../ForwardHashExchangeProcessor.java | 50 +-- .../physical/batch/BatchPhysicalRank.scala | 10 +- .../batch/sql/ForwardHashExchangeTest.java | 101 +++++- .../batch/sql/ForwardHashExchangeTest.xml | 301 ++++++++++++++---- 4 files changed, 379 insertions(+), 83 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java index d8238605ec084..9e03ac9da9687 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java @@ -87,24 +87,25 @@ protected void visitNode(ExecNode node) { inputProperty.getRequiredDistribution(); ExecEdge edge = node.getInputEdges().get(i); - if (requiredDistribution.getType() == DistributionType.SINGLETON) { - if (!hasExchangeInput(edge) && isInputSortedNode(node)) { - // if operation chaining is disabled, this could mark sure the - // sort node and its output can also be connected by - // ForwardPartitioner + if (requiredDistribution.getType() != DistributionType.HASH) { + boolean visitChild = + requiredDistribution.getType() + == DistributionType.SINGLETON; + if (!hasExchangeInput(edge) + && hasSortInputForInputSortedNode(node)) { ExecEdge newEdge = addExchangeAndReconnectEdge( - tableConfig, edge, inputProperty, true); + tableConfig, + edge, + inputProperty, + true, + visitChild); newEdges.set(i, newEdge); changed = true; } continue; } - if (requiredDistribution.getType() != DistributionType.HASH) { - continue; - } - if (!hasExchangeInput(edge)) { ExecEdge newEdge; if (isInputSortedNode(node)) { @@ -117,7 +118,8 @@ protected void visitNode(ExecNode node) { tableConfig, sort.getInputEdges().get(0), inputProperty, - false); + false, + true); sort.setInputEdges( Collections.singletonList(newEdgeOfSort)); } @@ -127,13 +129,13 @@ protected void visitNode(ExecNode node) { // ForwardPartitioner newEdge = addExchangeAndReconnectEdge( - tableConfig, edge, inputProperty, true); + tableConfig, edge, inputProperty, true, true); } else { // add Exchange with keep_input_as_is distribution as the input // of the node newEdge = addExchangeAndReconnectEdge( - tableConfig, edge, inputProperty, false); + tableConfig, edge, inputProperty, false, true); updateOriginalEdgeInMultipleInput( node, i, (BatchExecExchange) newEdge.getSource()); } @@ -145,7 +147,7 @@ protected void visitNode(ExecNode node) { // node and its output can also be connected by ForwardPartitioner ExecEdge newEdge = addExchangeAndReconnectEdge( - tableConfig, edge, inputProperty, true); + tableConfig, edge, inputProperty, true, true); newEdges.set(i, newEdge); changed = true; } @@ -164,21 +166,27 @@ private ExecEdge addExchangeAndReconnectEdge( ReadableConfig tableConfig, ExecEdge edge, InputProperty inputProperty, - boolean strict) { + boolean strict, + boolean visitChild) { ExecNode target = edge.getTarget(); ExecNode source = edge.getSource(); if (source instanceof CommonExecExchange) { return edge; } // only Calc, Correlate and Sort can propagate sort property and distribution property - if (source instanceof BatchExecCalc - || source instanceof BatchExecPythonCalc - || source instanceof BatchExecSort - || source instanceof BatchExecCorrelate - || source instanceof BatchExecPythonCorrelate) { + if (visitChild + && (source instanceof BatchExecCalc + || source instanceof BatchExecPythonCalc + || source instanceof BatchExecSort + || source instanceof BatchExecCorrelate + || source instanceof BatchExecPythonCorrelate)) { ExecEdge newEdge = addExchangeAndReconnectEdge( - tableConfig, source.getInputEdges().get(0), inputProperty, strict); + tableConfig, + source.getInputEdges().get(0), + inputProperty, + strict, + true); source.setInputEdges(Collections.singletonList(newEdge)); } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala index 1d3c5720148b9..18626e783dcac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala @@ -229,10 +229,14 @@ class BatchPhysicalRank( } override def translateToExecNode(): ExecNode[_] = { - val requiredDistribution = if (partitionKey.length() == 0) { - InputProperty.SINGLETON_DISTRIBUTION + val requiredDistribution = if (isGlobal) { + if (partitionKey.length() == 0) { + InputProperty.SINGLETON_DISTRIBUTION + } else { + InputProperty.hashDistribution(partitionKey.toArray) + } } else { - InputProperty.hashDistribution(partitionKey.toArray) + InputProperty.UNKNOWN_DISTRIBUTION } new BatchExecRank( unwrapTableConfig(this), diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java index fd68753bb42e1..b5edb3632f969 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.java @@ -36,6 +36,12 @@ public void before() { util = batchTestUtil(TableConfig.getDefault()); util.getStreamEnv().getConfig().setDynamicGraph(true); + util.tableEnv() + .getConfig() + .getConfiguration() + .set( + OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED, + false); util.tableEnv() .executeSql( "CREATE TABLE T (\n" @@ -71,6 +77,20 @@ public void before() { + ")"); } + @Test + public void testRankWithHashShuffle() { + util.verifyExecPlan( + "SELECT * FROM (SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM T) WHERE rk <= 10"); + } + + @Test + public void testSortAggregateWithHashShuffle() { + util.tableEnv() + .getConfig() + .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); + util.verifyExecPlan(" SELECT a, SUM(b) AS b FROM T GROUP BY a"); + } + @Test public void testOverAggOnHashAggWithHashShuffle() { util.tableEnv() @@ -130,12 +150,45 @@ public void testHashAggOnHashJoinWithHashShuffle() { } @Test - public void testSortAggOnSortMergeJoinWithHashShuffle() { + public void testOnePhaseSortAggOnSortMergeJoinWithHashShuffle() { + util.tableEnv() + .getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, + "HashJoin,NestedLoopJoin,HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE"); + util.verifyExecPlan( + "WITH r AS (SELECT * FROM T1, T2 WHERE a1 = a2 AND c1 LIKE 'He%')\n" + + "SELECT sum(b1) FROM r group by a1"); + } + + @Test + public void testTwoPhaseSortAggOnSortMergeJoinWithHashShuffle() { + util.tableEnv() + .getConfig() + .set( + ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, + "HashJoin,NestedLoopJoin,HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); + util.verifyExecPlan( + "WITH r AS (SELECT * FROM T1, T2 WHERE a1 = a2 AND c1 LIKE 'He%')\n" + + "SELECT sum(b1) FROM r group by a1"); + } + + @Test + public void testAutoPhaseSortAggOnSortMergeJoinWithHashShuffle() { util.tableEnv() .getConfig() .set( ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "AUTO"); util.verifyExecPlan( "WITH r AS (SELECT * FROM T1, T2 WHERE a1 = a2 AND c1 LIKE 'He%')\n" + "SELECT sum(b1) FROM r group by a1"); @@ -196,10 +249,13 @@ public void testRankOnHashAggWithGlobalShuffle() { } @Test - public void testRankOnSortAggWithHashShuffle() { + public void testRankOnOnePhaseSortAggWithHashShuffle() { util.tableEnv() .getConfig() - .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg"); + .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE"); util.verifyExecPlan( "SELECT * FROM (\n" + " SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM (\n" @@ -209,10 +265,45 @@ public void testRankOnSortAggWithHashShuffle() { } @Test - public void testRankOnSortAggWithGlobalShuffle() { + public void testRankOnTwoPhaseSortAggWithHashShuffle() { util.tableEnv() .getConfig() - .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg"); + .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); + util.verifyExecPlan( + "SELECT * FROM (\n" + + " SELECT a, b, RANK() OVER(PARTITION BY a ORDER BY b) rk FROM (\n" + + " SELECT a, SUM(b) AS b FROM T GROUP BY a\n" + + " )\n" + + " ) WHERE rk <= 10"); + } + + @Test + public void testRankOnOnePhaseSortAggWithGlobalShuffle() { + util.tableEnv() + .getConfig() + .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE"); + util.verifyExecPlan( + "SELECT * FROM (\n" + + " SELECT b, RANK() OVER(ORDER BY b) rk FROM (\n" + + " SELECT SUM(b) AS b FROM T\n" + + " )\n" + + " ) WHERE rk <= 10"); + } + + @Test + public void testRankOnTwoPhaseSortAggWithGlobalShuffle() { + util.tableEnv() + .getConfig() + .set(ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg"); + util.tableEnv() + .getConfig() + .set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE"); util.verifyExecPlan( "SELECT * FROM (\n" + " SELECT b, RANK() OVER(ORDER BY b) rk FROM (\n" diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.xml index 960e1165e4863..898426eb39efd 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/ForwardHashExchangeTest.xml @@ -16,6 +16,38 @@ See the License for the specific language governing permissions and limitations under the License. --> + + + + + + + + + + + + + + + + + + + + + + @@ -229,7 +293,8 @@ OverAggregate(orderBy=[b ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED +- Exchange(distribution=[forward]) +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS b]) +- Exchange(distribution=[single]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[], aggregates=[grouping=[], aggFunctions=[LongSumAggFunction(b)]]]], fields=[sum$0]) + +- LocalHashAggregate(select=[Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[]]], fields=[b]) ]]> @@ -260,7 +325,8 @@ Calc(select=[sum_b, (CASE((w0$o0 > 0), w0$o1, null:BIGINT) / w0$o0) AS avg_b, w1 +- Exchange(distribution=[keep_input_as_is[hash[c]]]) +- HashAggregate(isMerge=[true], groupBy=[c], select=[c, Final_SUM(sum$0) AS sum_b]) +- Exchange(distribution=[hash[c]]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[c, b], metadata=[], aggregates=[grouping=[c], aggFunctions=[LongSumAggFunction(b)]]]], fields=[c, sum$0]) + +- LocalHashAggregate(groupBy=[c], select=[c, Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[c, b], metadata=[]]], fields=[c, b]) ]]> @@ -284,7 +350,8 @@ OverAggregate(orderBy=[b ASC], window#0=[RANK(*) AS w0$o0 RANG BETWEEN UNBOUNDED +- Exchange(distribution=[forward]) +- SortAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS b]) +- Exchange(distribution=[single]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[], aggregates=[grouping=[], aggFunctions=[LongSumAggFunction(b)]]]], fields=[sum$0]) + +- LocalSortAggregate(select=[Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[]]], fields=[b]) ]]> @@ -315,7 +382,10 @@ Calc(select=[sum_b, (CASE((w0$o0 > 0), w0$o1, null:BIGINT) / w0$o0) AS avg_b, w1 +- Exchange(distribution=[forward]) +- Sort(orderBy=[c ASC]) +- Exchange(distribution=[hash[c]]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[c, b], metadata=[], aggregates=[grouping=[c], aggFunctions=[LongSumAggFunction(b)]]]], fields=[c, sum$0]) + +- LocalSortAggregate(groupBy=[c], select=[c, Partial_SUM(b) AS sum$0]) + +- Exchange(distribution=[forward]) + +- Sort(orderBy=[c ASC]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[c, b], metadata=[]]], fields=[c, b]) ]]> @@ -345,7 +415,8 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[], order +- Exchange(distribution=[forward]) +- HashAggregate(isMerge=[true], select=[Final_SUM(sum$0) AS b]) +- Exchange(distribution=[single]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[], aggregates=[grouping=[], aggFunctions=[LongSumAggFunction(b)]]]], fields=[sum$0]) + +- LocalHashAggregate(select=[Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[b], metadata=[]]], fields=[b]) ]]> @@ -375,11 +446,12 @@ Rank(rankType=[RANK], rankRange=[rankStart=1, rankEnd=10], partitionBy=[a], orde +- Exchange(distribution=[keep_input_as_is[hash[a]]]) +- HashAggregate(isMerge=[true], groupBy=[a], select=[a, Final_SUM(sum$0) AS b]) +- Exchange(distribution=[hash[a]]) - +- TableSourceScan(table=[[default_catalog, default_database, T, project=[a, b], metadata=[], aggregates=[grouping=[a], aggFunctions=[LongSumAggFunction(b)]]]], fields=[a, sum$0]) + +- LocalHashAggregate(groupBy=[a], select=[a, Partial_SUM(b) AS sum$0]) + +- TableSourceScan(table=[[default_catalog, default_database, T, project=[a, b], metadata=[]]], fields=[a, b]) ]]> - + - + - + - + + + + + + + + + + + + + + + + + + + + + + + @@ -509,7 +631,31 @@ SortAggregate(isMerge=[false], select=[SUM(b1) AS EXPR$0]) ]]> - + + + + + + + + + + + + @@ -528,16 +674,63 @@ LogicalProject(EXPR$0=[$1]) + + + + + + + + + + + From 0965060253821ec1167ff3b8760a11a3579d838c Mon Sep 17 00:00:00 2001 From: wangfeifan Date: Sun, 27 Mar 2022 23:23:45 +0800 Subject: [PATCH 194/258] [FLINK-25867][docs-zh] translate ChangelogBackend documentation to chinese --- .../docs/ops/state/state_backends.md | 130 +++++++++--------- 1 file changed, 66 insertions(+), 64 deletions(-) diff --git a/docs/content.zh/docs/ops/state/state_backends.md b/docs/content.zh/docs/ops/state/state_backends.md index f5776dbdaea3c..06d4276b0f049 100644 --- a/docs/content.zh/docs/ops/state/state_backends.md +++ b/docs/content.zh/docs/ops/state/state_backends.md @@ -306,77 +306,75 @@ public class MyOptionsFactory implements ConfigurableRocksDBOptionsFactory { {{< top >}} -## Enabling Changelog + -{{< hint warning >}} This feature is in experimental status. {{< /hint >}} +## 开启 Changelog -{{< hint warning >}} Enabling Changelog may have a negative performance impact on your application (see below). {{< /hint >}} +{{< hint warning >}} 该功能处于实验状态。 {{< /hint >}} -### Introduction +{{< hint warning >}} 开启 Changelog 可能会给您的应用带来性能损失。(见下文) {{< /hint >}} -Changelog is a feature that aims to decrease checkpointing time and, therefore, end-to-end latency in exactly-once mode. + -Most commonly, checkpoint duration is affected by: +### 介绍 -1. Barrier travel time and alignment, addressed by - [Unaligned checkpoints]({{< ref "docs/ops/state/checkpointing_under_backpressure#unaligned-checkpoints" >}}) - and [Buffer debloating]({{< ref "docs/ops/state/checkpointing_under_backpressure#buffer-debloating" >}}) -2. Snapshot creation time (so-called synchronous phase), addressed by asynchronous snapshots (mentioned [above]({{< - ref "#the-embeddedrocksdbstatebackend">}})) -4. Snapshot upload time (asynchronous phase) +Changelog 是一项旨在减少 checkpointing 时间的功能,因此也可以减少 exactly-once 模式下的端到端延迟。 -Upload time can be decreased by [incremental checkpoints]({{< ref "#incremental-checkpoints" >}}). -However, most incremental state backends perform some form of compaction periodically, which results in re-uploading the -old state in addition to the new changes. In large deployments, the probability of at least one task uploading lots of -data tends to be very high in every checkpoint. +一般情况下 checkpoint 的持续时间受如下因素影响: -With Changelog enabled, Flink uploads state changes continuously and forms a changelog. On checkpoint, only the relevant -part of this changelog needs to be uploaded. The configured state backend is snapshotted in the -background periodically. Upon successful upload, the changelog is truncated. +1. Barrier 到达和对齐时间,可以通过 [Unaligned checkpoints]({{< ref "docs/ops/state/checkpointing_under_backpressure#unaligned-checkpoints" >}}) 和 [Buffer debloating]({{< ref "docs/ops/state/checkpointing_under_backpressure#buffer-debloating" >}}) 解决。 -As a result, asynchronous phase duration is reduced, as well as synchronous phase - because no data needs to be flushed -to disk. In particular, long-tail latency is improved. +2. 快照制作时间(所谓同步阶段), 可以通过异步快照解决(如[上文]({{< + ref "#the-embeddedrocksdbstatebackend">}})所述)。 -However, resource usage is higher: +3. 快照上传时间(异步阶段)。 -- more files are created on DFS -- more files can be left undeleted DFS (this will be addressed in the future versions in FLINK-25511 and FLINK-25512) -- more IO bandwidth is used to upload state changes -- more CPU used to serialize state changes -- more memory used by Task Managers to buffer state changes +可以用[增量 checkpoints]({{< ref "#incremental-checkpoints" >}}) 来减少上传时间。但是,大多数支持增量checkpoint的状态后端会定期执行合并类型的操作,这会导致除了新的变更之外还要重新上传旧状态。在大规模部署中,每次 checkpoint 中至少有一个 task 上传大量数据的可能性往往非常高。 -Recovery time is another thing to consider. Depending on the `state.backend.changelog.periodic-materialize.interval` -setting, the changelog can become lengthy and replaying it may take more time. However, recovery time combined with -checkpoint duration will likely still be lower than in non-changelog setups, providing lower end-to-end latency even in -failover case. However, it's also possible that the effective recovery time will increase, depending on the actual ratio -of the aforementioned times. +开启 Changelog 功能之后,Flink 会不断上传状态变更并形成 changelog。创建 checkpoint 时,只有 changelog 中的相关部分需要上传。而配置的状态后端则会定期在后台进行快照,快照成功上传后,相关的changelog 将会被截断。 -For more details, see [FLIP-158](https://cwiki.apache.org/confluence/display/FLINK/FLIP-158%3A+Generalized+incremental+checkpoints). +基于此,异步阶段的持续时间减少(另外因为不需要将数据刷新到磁盘,同步阶段持续时间也减少了),特别是长尾延迟得到了改善。 -### Installation +但是,资源使用会变得更高: -Changelog JARs are included into the standard Flink distribution. +- 将会在 DFS 上创建更多文件 +- 将可能在 DFS 上残留更多文件(这将在 FLINK-25511 和 FLINK-25512 之后的新版本中被解决) +- 将使用更多的 IO 带宽用来上传状态变更 +- 将使用更多 CPU 资源来序列化状态变更 +- Task Managers 将会使用更多内存来缓存状态变更 -Make sure to [add]({{< ref "docs/deployment/filesystems/overview" >}}) the necessary filesystem plugins. +另一项需要考虑的事情是恢复时间。取决于 `state.backend.changelog.periodic-materialize.interval` 的设置,changelog 可能会变得冗长,因此重放会花费更多时间。即使这样,恢复时间加上 checkpoint 持续时间仍然可能低于不开启 changelog 功能的时间,从而在故障恢复的情况下也能提供更低的端到端延迟。当然,取决于上述时间的实际比例,有效恢复时间也有可能会增加。 -### Configuration +有关更多详细信息,请参阅 [FLIP-158](https://cwiki.apache.org/confluence/display/FLINK/FLIP-158%3A+Generalized+incremental+checkpoints)。 -Here is an example configuration in YAML: + + +### 安装 + +标准的 Flink 发行版包含 Changelog 所需要的 JAR包。 + +请确保[添加]({{< ref "docs/deployment/filesystems/overview" >}})所需的文件系统插件。 + + + +### 配置 + +这是 YAML 中的示例配置: ```yaml state.backend.changelog.enabled: true -state.backend.changelog.storage: filesystem # currently, only filesystem and memory (for tests) are supported -dstl.dfs.base-path: s3:// # similar to state.checkpoints.dir +state.backend.changelog.storage: filesystem # 当前只支持 filesystem 和 memory(仅供测试用) +dstl.dfs.base-path: s3:// # 类似于 state.checkpoints.dir ``` -Please keep the following defaults (see [limitations](#limitations)): +请将如下配置保持默认值 (参见[限制](#limitations)): ```yaml execution.checkpointing.max-concurrent-checkpoints: 1 state.backend.local-recovery: false ``` -Please refer to the [configuration section]({{< ref "docs/deployment/config#state-changelog-options" >}}) for other options. +有关其他配置选项,请参阅[配置]({{< ref "docs/deployment/config#state-changelog-options" >}})部分。 -Changelog can also be enabled or disabled per job programmatically: +也可以通过编程方式为每个作业开启或关闭 Changelog: {{< tabs >}} {{< tab "Java" >}} ```java @@ -398,36 +396,40 @@ env.enable_changelog_statebackend(true) {{< /tab >}} {{< /tabs >}} -### Monitoring + + +### 监控 + +[此处]({{< ref "docs/ops/metrics#changelog" >}})列出了可用的指标。 + +如果 task 因写状态变更而被反压,他将在 UI 中被显示为忙碌(红色)。 -Available metrics are listed [here]({{< ref "docs/ops/metrics#changelog" >}}). + -If a task is backpressured by writing state changes, it will be shown as busy (red) in the UI. +### 升级现有作业 -### Upgrading existing jobs +**开启 Changelog** -**Enabling Changelog** +仅支持从标准格式的 savepoint 恢复: +- 给定一个没有开启 Changelog 的作业 +- 创建一个 [savepoint]({{< ref "docs/ops/state/savepoints#resuming-from-savepoints" >}}) (默认为标准格式) +- 更改配置(开启 Changelog) +- 从创建的 snapshot 恢复 -Resuming only from savepoints in canonical format is supported: -- given an existing non-changelog job -- take a [savepoint]({{< ref "docs/ops/state/savepoints#resuming-from-savepoints" >}}) (canonical format is the default) -- alter configuration (enable Changelog) -- resume from the taken snapshot +**关闭 Changelog** -**Disabling Changelog** +仅支持从 [savepoints]({{< ref "docs/ops/state/savepoints#resuming-from-savepoints" >}}) 恢复。从 [checkpoints]({{< ref "docs/ops/state/checkpoints#resuming-from-a-retained-checkpoint" >}}) 恢复计划在未来版本中支持。 -Resuming only from [savepoints]({{< ref "docs/ops/state/savepoints#resuming-from-savepoints" >}}) -is supported. Resuming from [checkpoints]({{< ref "docs/ops/state/checkpoints#resuming-from-a-retained-checkpoint" >}}) -is planned in the future versions. +当前不支持**状态迁移**(包括改变 TTL)。 -**State migration** (including changing TTL) is currently not supported + -### Limitations - - At most one concurrent checkpoint - - Local recovery not supported - - As of Flink 1.15, only `filesystem` changelog implementation is available - - State migration (including changing TTL) is currently not supported -- [NO_CLAIM]({{< ref "docs/deployment/config#execution-savepoint-restore-mode" >}}) mode not supported +### 限制 +- 最多同时创建一个 checkpoint +- 本地恢复暂不支持 +- 到 Flink 1.15 为止, 只有 `filesystem` changelog 实现可用 +- 尚不支持状态迁移(包括修改 TTL) +- 尚不支持 [NO_CLAIM]({{< ref "docs/deployment/config#execution-savepoint-restore-mode" >}}) 模式 {{< top >}} From 2606b25ede5f822a408be03e4f1c918708760631 Mon Sep 17 00:00:00 2001 From: wangfeifan Date: Mon, 28 Mar 2022 14:51:42 +0800 Subject: [PATCH 195/258] [hotfix][docs] fix anchor mistask in changelog monitoring --- docs/content.zh/docs/ops/state/state_backends.md | 2 +- docs/content/docs/ops/state/state_backends.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content.zh/docs/ops/state/state_backends.md b/docs/content.zh/docs/ops/state/state_backends.md index 06d4276b0f049..492517e5ddb76 100644 --- a/docs/content.zh/docs/ops/state/state_backends.md +++ b/docs/content.zh/docs/ops/state/state_backends.md @@ -400,7 +400,7 @@ env.enable_changelog_statebackend(true) ### 监控 -[此处]({{< ref "docs/ops/metrics#changelog" >}})列出了可用的指标。 +[此处]({{< ref "docs/ops/metrics#state-changelog" >}})列出了可用的指标。 如果 task 因写状态变更而被反压,他将在 UI 中被显示为忙碌(红色)。 diff --git a/docs/content/docs/ops/state/state_backends.md b/docs/content/docs/ops/state/state_backends.md index 065c7a444ac71..3465de8a35f04 100644 --- a/docs/content/docs/ops/state/state_backends.md +++ b/docs/content/docs/ops/state/state_backends.md @@ -421,7 +421,7 @@ env.enable_changelog_statebackend(true) ### Monitoring -Available metrics are listed [here]({{< ref "docs/ops/metrics#changelog" >}}). +Available metrics are listed [here]({{< ref "docs/ops/metrics#state-changelog" >}}). If a task is backpressured by writing state changes, it will be shown as busy (red) in the UI. From 40108d0763c3bef2741933f7ea9e5fe71bce77b0 Mon Sep 17 00:00:00 2001 From: wangfeifan Date: Mon, 11 Apr 2022 11:04:39 +0800 Subject: [PATCH 196/258] [hotfix][docs-zh] fix missing link tag in State Backends document --- .../docs/ops/state/state_backends.md | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/docs/content.zh/docs/ops/state/state_backends.md b/docs/content.zh/docs/ops/state/state_backends.md index 492517e5ddb76..485960b6a2620 100644 --- a/docs/content.zh/docs/ops/state/state_backends.md +++ b/docs/content.zh/docs/ops/state/state_backends.md @@ -38,6 +38,8 @@ under the License. 在启动 CheckPoint 机制时,状态会随着 CheckPoint 而持久化,以防止数据丢失、保障恢复时的一致性。 状态内部的存储格式、状态在 CheckPoint 时如何持久化以及持久化在哪里均取决于选择的 **State Backend**。 + + # 可用的 State Backends Flink 内置了以下这些开箱即用的 state backends : @@ -48,6 +50,8 @@ Flink 内置了以下这些开箱即用的 state backends : 如果不设置,默认使用 HashMapStateBackend。 + + ### HashMapStateBackend 在 *HashMapStateBackend* 内部,数据以 Java 对象的形式存储在堆中。 Key/value 形式的状态和窗口算子会持有一个 hash table,其中存储着状态值、触发器。 @@ -61,6 +65,8 @@ HashMapStateBackend 的适用场景: 与 EmbeddedRocksDBStateBackend 不同的是,由于 HashMapStateBackend 将数据以对象形式存储在堆中,因此重用这些对象数据是不安全的。 + + ### EmbeddedRocksDBStateBackend EmbeddedRocksDBStateBackend 将正在运行中的状态数据保存在 [RocksDB](http://rocksdb.org) 数据库中,RocksDB 数据库默认将数据存储在 TaskManager 的数据目录。 @@ -91,6 +97,8 @@ EmbeddedRocksDBStateBackend 是目前唯一支持增量 CheckPoint 的 State Bac 每个 slot 中的 RocksDB instance 的内存大小是有限制的,详情请见 [这里]({{< ref "docs/ops/state/large_state_tuning" >}})。 + + # 选择合适的 State Backend 在选择 `HashMapStateBackend` 和 `RocksDB` 的时候,其实就是在性能与可扩展性之间权衡。`HashMapStateBackend` 是非常快的,因为每个状态的读取和算子对于 objects 的更新都是在 Java 的 heap 上;但是状态的大小受限于集群中可用的内存。 @@ -102,11 +110,15 @@ EmbeddedRocksDBStateBackend 是目前唯一支持增量 CheckPoint 的 State Bac 从 1.13 版本开始,所有的 state backends 都会生成一种普适的格式。因此,如果想切换 state backend 的话,那么最好先升级你的 Flink 版本,在新版本中生成 savepoint,在这之后你才可以使用一个不同的 state backend 来读取并恢复它。 {{< /hint >}} + + ## 设置 State Backend 如果没有明确指定,将使用 jobmanager 做为默认的 state backend。你能在 **flink-conf.yaml** 中为所有 Job 设置其他默认的 State Backend。 每一个 Job 的 state backend 配置会覆盖默认的 state backend 配置,如下所示: + + ### 设置每个 Job 的 State Backend `StreamExecutionEnvironment` 可以对每个 Job 的 State Backend 进行设置,如下所示: @@ -141,6 +153,7 @@ env.setStateBackend(new HashMapStateBackend()) **注意:** 由于 RocksDB 是 Flink 默认分发包的一部分,所以如果你没在代码中使用 RocksDB,则不需要添加此依赖。而且可以在 `flink-conf.yaml` 文件中通过 `state.backend` 配置 State Backend,以及更多的 [checkpointing]({{< ref "docs/deployment/config" >}}#checkpointing) 和 [RocksDB 特定的]({{< ref "docs/deployment/config" >}}#rocksdb-state-backend) 参数。 {{< /hint >}} + ### 设置默认的(全局的) State Backend @@ -166,10 +179,14 @@ state.backend: filesystem state.checkpoints.dir: hdfs://namenode:40010/flink/checkpoints ``` + + # RocksDB State Backend 进阶 *该小节描述 RocksDB state backend 的更多细节* + + ### 增量快照 RocksDB 支持*增量快照*。不同于产生一个包含所有数据的全量备份,增量快照中只包含自上一次快照完成之后被修改的记录,因此可以显著减少快照完成的耗时。 @@ -184,6 +201,8 @@ RocksDB 支持*增量快照*。不同于产生一个包含所有数据的全量 需要注意的是,一旦启用了增量快照,网页上展示的 `Checkpointed Data Size` 只代表增量上传的数据量,而不是一次快照的完整数据量。 + + ### 内存管理 Flink 致力于控制整个进程的内存消耗,以确保 Flink 任务管理器(TaskManager)有良好的内存使用,从而既不会在容器(Docker/Kubernetes, Yarn等)环境中由于内存超用被杀掉,也不会因为内存利用率过低导致不必要的数据落盘或是缓存命中率下降,致使性能下降。 @@ -213,6 +232,8 @@ Flink还提供了两个参数来控制*写路径*(MemTable)和*读路径*( 或者可以复用上述 cache/write-buffer-manager 机制,但将内存大小设置为与 Flink 的托管内存大小无关的固定大小(通过 `state.backend.rocksdb.memory.fixed-per-slot` 选项)。 注意在这两种情况下,用户都需要确保在 JVM 之外有足够的内存可供 RocksDB 使用。 + + ### 计时器(内存 vs. RocksDB) 计时器(Timer)用于安排稍后的操作(基于事件时间或处理时间),例如触发窗口或回调 `ProcessFunction`。 @@ -223,6 +244,8 @@ Flink还提供了两个参数来控制*写路径*(MemTable)和*读路径*( 注意 *在 RocksDB state backend 中使用基于堆的计时器的组合当前不支持计时器状态的异步快照。其他状态(如 keyed state)可以被异步快照。* + + ### 开启 RocksDB 原生监控指标 您可以选择使用 Flink 的监控指标系统来汇报 RocksDB 的原生指标,并且可以选择性的指定特定指标进行汇报。 @@ -433,6 +456,8 @@ env.enable_changelog_statebackend(true) {{< top >}} + + ## 自旧版本迁移 从 **Flink 1.13** 版本开始,社区改进了 state backend 的公开类,进而帮助用户更好理解本地状态存储和 checkpoint 存储的区分。 From 98c3817596a145305e4d4433552155f709d6a444 Mon Sep 17 00:00:00 2001 From: wangfeifan Date: Tue, 12 Apr 2022 11:01:37 +0800 Subject: [PATCH 197/258] [hotfix][docs-zh] fix title level in State Backends document --- docs/content.zh/docs/ops/state/state_backends.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/content.zh/docs/ops/state/state_backends.md b/docs/content.zh/docs/ops/state/state_backends.md index 485960b6a2620..6c8c4593686b0 100644 --- a/docs/content.zh/docs/ops/state/state_backends.md +++ b/docs/content.zh/docs/ops/state/state_backends.md @@ -40,7 +40,7 @@ under the License. -# 可用的 State Backends +## 可用的 State Backends Flink 内置了以下这些开箱即用的 state backends : @@ -99,7 +99,7 @@ EmbeddedRocksDBStateBackend 是目前唯一支持增量 CheckPoint 的 State Bac -# 选择合适的 State Backend +## 选择合适的 State Backend 在选择 `HashMapStateBackend` 和 `RocksDB` 的时候,其实就是在性能与可扩展性之间权衡。`HashMapStateBackend` 是非常快的,因为每个状态的读取和算子对于 objects 的更新都是在 Java 的 heap 上;但是状态的大小受限于集群中可用的内存。 另一方面,`RocksDB` 可以根据可用的 disk 空间扩展,并且只有它支持增量 snapshot。 @@ -181,7 +181,7 @@ state.checkpoints.dir: hdfs://namenode:40010/flink/checkpoints -# RocksDB State Backend 进阶 +## RocksDB State Backend 进阶 *该小节描述 RocksDB state backend 的更多细节* @@ -255,7 +255,7 @@ Flink还提供了两个参数来控制*写路径*(MemTable)和*读路径*( **注意:** 启用 RocksDB 的原生指标可能会对应用程序的性能产生负面影响。 {{< /hint >}} -### 列族(ColumnFamily)级别的预定义选项 +#### 列族(ColumnFamily)级别的预定义选项 注意 在引入 [RocksDB 使用托管内存](#memory-management) 功能后,此机制应限于在*专家调优*或*故障处理*中使用。 @@ -272,7 +272,7 @@ Flink还提供了两个参数来控制*写路径*(MemTable)和*读路径*( RocksDB State Backend 会将 [这里定义]({{< ref "docs/deployment/config" >}}#advanced-rocksdb-state-backends-options) 的所有配置项全部加载。 因此您可以简单的通过关闭 RocksDB 使用托管内存的功能并将需要的设置选项加入配置文件来配置底层的列族选项。 -### 通过 RocksDBOptionsFactory 配置 RocksDB 选项 +#### 通过 RocksDBOptionsFactory 配置 RocksDB 选项 注意 在引入 [RocksDB 使用托管内存](#memory-management) 功能后,此机制应限于在*专家调优*或*故障处理*中使用。 From b62a39e14db6dab5823704fa348c5b2ffa74ab4d Mon Sep 17 00:00:00 2001 From: Chesnay Schepler Date: Tue, 19 Apr 2022 11:19:47 +0200 Subject: [PATCH 198/258] [FLINK-27229][cassandra][build] Remove test netty dependency --- flink-connectors/flink-connector-cassandra/pom.xml | 7 ------- 1 file changed, 7 deletions(-) diff --git a/flink-connectors/flink-connector-cassandra/pom.xml b/flink-connectors/flink-connector-cassandra/pom.xml index b03ed1b4e4c05..3f4a8e6078a96 100644 --- a/flink-connectors/flink-connector-cassandra/pom.xml +++ b/flink-connectors/flink-connector-cassandra/pom.xml @@ -205,13 +205,6 @@ under the License. cassandra test - - - io.netty - netty-all - 4.1.46.Final - test - From 5175087052f66e3d66d02685787d1e5def9a50e9 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Mon, 18 Apr 2022 12:59:40 +0300 Subject: [PATCH 199/258] [FLINK-27212][table-planner] Use UTF-8 encoding when casting between binary/character string Revert behaviour of binary to string casts and vice versa, to not use hex enconding/decoding, but simple UTF-8 bytes transformation from a byte[] to a string and vice versa. (cherry picked from commit 4cdafffa6cee1c428e23dae616c98cc57ee3e20b) --- .../jdbc/catalog/PostgresCatalogITCase.java | 2 +- .../casting/BinaryToStringCastRule.java | 64 +++++---- .../casting/StringToBinaryCastRule.java | 12 +- .../planner/functions/CastFunctionITCase.java | 35 ++--- .../functions/CastFunctionMiscITCase.java | 6 +- .../functions/casting/CastRulesTest.java | 121 +++++++++++------- .../expressions/ScalarFunctionsTest.scala | 4 +- 7 files changed, 131 insertions(+), 113 deletions(-) diff --git a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java index 020b7da021582..d85d8678b1ae4 100644 --- a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java +++ b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java @@ -126,7 +126,7 @@ public void testGroupByInsert() throws Exception { .execute() .collect()); assertEquals( - "[+I[1, [65], 3, 4, 5.5, 6.6, 7.70000, 8.8, true, a, B, C , d, 2016-06-22T19:10:25, 2015-01-01, 00:51:03, 500.000000000000000000]]", + "[+I[1, [52, 49], 3, 4, 5.5, 6.6, 7.70000, 8.8, true, a, B, C , d, 2016-06-22T19:10:25, 2015-01-01, 00:51:03, 500.000000000000000000]]", results.toString()); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java index b4182f0bf431a..d48047809b6d6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java @@ -21,7 +21,6 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeFamily; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; -import org.apache.flink.table.utils.EncodingUtils; import java.nio.charset.StandardCharsets; @@ -29,7 +28,6 @@ import static org.apache.flink.table.planner.codegen.calls.BuiltInMethods.BINARY_STRING_DATA_FROM_STRING; import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.accessStaticField; import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.constructorCall; -import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.staticCall; /** * {@link LogicalTypeFamily#BINARY_STRING} to {@link LogicalTypeFamily#CHARACTER_STRING} cast rule. @@ -50,25 +48,26 @@ private BinaryToStringCastRule() { isNull$0 = _myInputIsNull; if (!isNull$0) { - java.lang.String hexString$0; - hexString$0 = org.apache.flink.table.utils.EncodingUtils.hex(_myInput); - java.lang.String resultString$152; - resultString$152 = hexString$0.toString(); - if (hexString$0.length() > 3) { - resultString$152 = hexString$0.substring(0, java.lang.Math.min(hexString$0.length(), 3)); + java.lang.String resultString$435; + resultString$435 = new java.lang.String(_myInput, java.nio.charset.StandardCharsets.UTF_8); + java.lang.String resultPadOrTrim$538; + resultPadOrTrim$538 = resultString$435.toString(); + if (resultString$435.length() > 12) { + resultPadOrTrim$538 = resultString$435.substring(0, java.lang.Math.min(resultString$435.length(), 12)); } else { - if (resultString$1.length() < 12) { - int padLength$3; - padLength$3 = 12 - resultString$152.length(); - java.lang.StringBuilder sbPadding$4; - sbPadding$4 = new java.lang.StringBuilder(); - for (int i$5 = 0; i$5 < padLength$3; i$5++) { - sbPadding$4.append(" "); + if (resultPadOrTrim$538.length() < 12) { + int padLength$539; + padLength$539 = 12 - resultPadOrTrim$538.length(); + java.lang.StringBuilder sbPadding$540; + sbPadding$540 = new java.lang.StringBuilder(); + for (int i$541 = 0; i$541 < padLength$539; i$541++) { + sbPadding$540.append(" "); } - resultString$152 = resultString$152 + sbPadding$4.toString(); + resultPadOrTrim$538 = resultPadOrTrim$538 + sbPadding$540.toString(); } } - result$1 = org.apache.flink.table.data.binary.BinaryStringData.fromString(resultString$152); + resultString$435 = resultPadOrTrim$538; + result$1 = org.apache.flink.table.data.binary.BinaryStringData.fromString(resultString$435); isNull$0 = result$1 == null; } else { result$1 = org.apache.flink.table.data.binary.BinaryStringData.EMPTY_UTF8; @@ -84,28 +83,27 @@ protected String generateCodeBlockInternal( LogicalType inputLogicalType, LogicalType targetLogicalType) { final String resultStringTerm = newName("resultString"); - CastRuleUtils.CodeWriter writer = new CastRuleUtils.CodeWriter(); - if (context.legacyBehaviour()) { - writer.declStmt(String.class, resultStringTerm) - .assignStmt( - resultStringTerm, - constructorCall( - String.class, - inputTerm, - accessStaticField(StandardCharsets.class, "UTF_8"))); - } else { - final int length = LogicalTypeChecks.getLength(targetLogicalType); + final CastRuleUtils.CodeWriter writer = new CastRuleUtils.CodeWriter(); - final String hexStringTerm = newName("hexString"); - writer.declStmt(String.class, hexStringTerm) - .assignStmt(hexStringTerm, staticCall(EncodingUtils.class, "hex", inputTerm)); + writer.declStmt(String.class, resultStringTerm) + .assignStmt( + resultStringTerm, + constructorCall( + String.class, + inputTerm, + accessStaticField(StandardCharsets.class, "UTF_8"))); + + if (!context.legacyBehaviour()) { + final String resultPadOrTrim = newName("resultPadOrTrim"); + final int length = LogicalTypeChecks.getLength(targetLogicalType); CharVarCharTrimPadCastRule.padAndTrimStringIfNeeded( writer, targetLogicalType, context.legacyBehaviour(), length, - resultStringTerm, - hexStringTerm); + resultPadOrTrim, + resultStringTerm); + writer.assignStmt(resultStringTerm, resultPadOrTrim); } return writer // Assign the result value diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/StringToBinaryCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/StringToBinaryCastRule.java index ad95a4192be2d..0bfc2982a24be 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/StringToBinaryCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/StringToBinaryCastRule.java @@ -22,14 +22,12 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeFamily; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; -import org.apache.flink.table.utils.EncodingUtils; import static org.apache.flink.table.codesplit.CodeSplitUtil.newName; import static org.apache.flink.table.planner.functions.casting.BinaryToBinaryCastRule.couldPad; import static org.apache.flink.table.planner.functions.casting.BinaryToBinaryCastRule.trimOrPadByteArray; import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.arrayLength; import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.methodCall; -import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.staticCall; /** * {@link LogicalTypeFamily#CHARACTER_STRING} to {@link LogicalTypeFamily#BINARY_STRING} cast rule. @@ -60,8 +58,7 @@ private StringToBinaryCastRule() { // new behavior isNull$0 = _myInputIsNull; if (!isNull$0) { - java.lang.String hexStringTerm$10 = _myInput.toString(); - byte[] byteArrayTerm$0 = org.apache.flink.table.utils.EncodingUtils.decodeHex(hexStringTerm$10); + byte[] byteArrayTerm$0 = _myInput.toBytes(); if (byteArrayTerm$0.length <= 2) { // If could pad result$1 = java.util.Arrays.copyOf(byteArrayTerm$0, 2); @@ -90,14 +87,9 @@ protected String generateCodeBlockInternal( } else { final int targetLength = LogicalTypeChecks.getLength(targetLogicalType); final String byteArrayTerm = newName("byteArrayTerm"); - final String hexStringTerm = newName("hexStringTerm"); return new CastRuleUtils.CodeWriter() - .declStmt(String.class, hexStringTerm, methodCall(inputTerm, "toString")) - .declStmt( - byte[].class, - byteArrayTerm, - staticCall(EncodingUtils.class, "decodeHex", hexStringTerm)) + .declStmt(byte[].class, byteArrayTerm, methodCall(inputTerm, "toBytes")) .ifStmt( arrayLength(byteArrayTerm) + " <= " + targetLength, thenWriter -> { diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java index baa7d366419e8..86629098f035e 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionITCase.java @@ -188,11 +188,11 @@ public static List allTypesBasic() { .build(), CastTestSpecBuilder.testCastTo(BINARY(2)) .fromCase(BINARY(5), null, null) - .fromCase(CHAR(4), "666F", new byte[] {102, 111}) - .fromCase(VARCHAR(8), "666f", new byte[] {102, 111}) - .fromCase(STRING(), "AAbbcCdD", new byte[] {-86, -69}) - .fromCase(VARCHAR(4), "FC", new byte[] {-4, 0}) - .fromCase(STRING(), "df", new byte[] {-33, 0}) + .fromCase(CHAR(4), "666F", new byte[] {54, 54}) + .fromCase(VARCHAR(8), "666f", new byte[] {54, 54}) + .fromCase(STRING(), "a", new byte[] {97, 0}) + .fromCase(VARCHAR(4), "FC", new byte[] {70, 67}) + .fromCase(STRING(), "foobar", new byte[] {102, 111}) // Not supported - no fix .failValidation(BOOLEAN(), true) // @@ -226,9 +226,9 @@ public static List allTypesBasic() { .build(), CastTestSpecBuilder.testCastTo(VARBINARY(4)) .fromCase(VARBINARY(5), null, null) - .fromCase(CHAR(4), "666F", new byte[] {102, 111}) - .fromCase(VARCHAR(8), "666f", new byte[] {102, 111}) - .fromCase(STRING(), "AAbbCcDdEe", new byte[] {-86, -69, -52, -35}) + .fromCase(CHAR(4), "foo", new byte[] {102, 111, 111, 32}) + .fromCase(VARCHAR(8), "foobar", new byte[] {102, 111, 111, 98}) + .fromCase(STRING(), "AAbbCcDdEe", new byte[] {65, 65, 98, 98}) // Not supported - no fix .failValidation(BOOLEAN(), true) // @@ -260,9 +260,12 @@ public static List allTypesBasic() { .build(), CastTestSpecBuilder.testCastTo(BYTES()) .fromCase(BYTES(), null, null) - .fromCase(CHAR(4), "666f", new byte[] {102, 111}) - .fromCase(VARCHAR(8), "666F", new byte[] {102, 111}) - .fromCase(STRING(), "aaBBCcDdEe", new byte[] {-86, -69, -52, -35, -18}) + .fromCase(CHAR(4), "foo", new byte[] {102, 111, 111, 32}) + .fromCase(VARCHAR(8), "foobar", new byte[] {102, 111, 111, 98, 97, 114}) + .fromCase( + STRING(), + "Apache Flink", + new byte[] {65, 112, 97, 99, 104, 101, 32, 70, 108, 105, 110, 107}) // Not supported - no fix .failValidation(BOOLEAN(), true) // @@ -992,11 +995,11 @@ public static List toStringCasts() { .fromCase(STRING(), "Apache Flink", "Apache Flink") .fromCase(STRING(), null, null) .fromCase(BOOLEAN(), true, "TRUE") - .fromCase(BINARY(2), DEFAULT_BINARY, "0001") - .fromCase(BINARY(3), DEFAULT_BINARY, "000100") - .fromCase(VARBINARY(3), DEFAULT_VARBINARY, "000102") - .fromCase(VARBINARY(5), DEFAULT_VARBINARY, "000102") - .fromCase(BYTES(), DEFAULT_BYTES, "0001020304") + .fromCase(BINARY(2), DEFAULT_BINARY, "\u0000\u0001") + .fromCase(BINARY(3), DEFAULT_BINARY, "\u0000\u0001\u0000") + .fromCase(VARBINARY(3), DEFAULT_VARBINARY, "\u0000\u0001\u0002") + .fromCase(VARBINARY(5), DEFAULT_VARBINARY, "\u0000\u0001\u0002") + .fromCase(BYTES(), DEFAULT_BYTES, "\u0000\u0001\u0002\u0003\u0004") .fromCase(DECIMAL(4, 3), 9.87, "9.870") .fromCase(DECIMAL(10, 5), 1, "1.00000") .fromCase( diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java index 7d312468fe6a2..473dd7dc49885 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/CastFunctionMiscITCase.java @@ -206,14 +206,14 @@ public static List testData() { .onFieldsWithData("foo") .testSqlResult( "CAST(CAST(x'68656C6C6F20636F6465' AS BINARY(10)) AS VARCHAR)", - "68656c6c6f20636f6465", + "hello code", STRING().notNull()), TestSpec.forFunction( BuiltInFunctionDefinitions.CAST, "test the x'....' binary syntax") .onFieldsWithData("foo") .testSqlResult( - "CAST(CAST(x'68656C6C6F2063617374' AS BINARY(10)) AS VARCHAR)", - "68656c6c6f2063617374", + "CAST(CAST(x'68656C6C6F20636F6465' AS BINARY(5)) AS VARCHAR)", + "hello", STRING().notNull()), TestSpec.forFunction(BuiltInFunctionDefinitions.CAST, "cast STRUCTURED to STRING") .onFieldsWithData(123456, "Flink") diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java index 6fdc98455818d..bb6663f5c5d69 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java @@ -651,19 +651,20 @@ Stream testCases() { .fromCase(BOOLEAN(), false, fromString("FALSE")) .fromCaseLegacy(BOOLEAN(), true, fromString("true")) .fromCaseLegacy(BOOLEAN(), false, fromString("false")) - .fromCase(BINARY(2), new byte[] {0, 1}, fromString("0001")) + .fromCase(BINARY(2), new byte[] {0, 1}, fromString("\u0000\u0001")) .fromCaseLegacy(BINARY(2), new byte[] {0, 1}, fromString("\u0000\u0001")) - .fromCase(VARBINARY(3), new byte[] {0, 1, 2}, fromString("000102")) + .fromCase( + VARBINARY(3), + new byte[] {0, 1, 2}, + fromString("\u0000\u0001\u0002")) .fromCaseLegacy( VARBINARY(3), new byte[] {0, 1, 2}, fromString("\u0000\u0001\u0002")) - .fromCase(VARBINARY(5), new byte[] {0, -1, -2}, fromString("00fffe")) .fromCaseLegacy(VARBINARY(5), new byte[] {102, 111, 111}, fromString("foo")) - .fromCase( - BYTES(), - new byte[] {-123, 43, -4, 125, 5}, - fromString("852bfc7d05")) + .fromCaseLegacy(VARBINARY(5), new byte[] {102, 111, 111}, fromString("foo")) + .fromCaseLegacy( + BYTES(), new byte[] {70, 108, 105, 110, 107}, fromString("Flink")) .fromCaseLegacy( BYTES(), new byte[] {70, 108, 105, 110, 107}, fromString("Flink")) .fromCase(BOOLEAN(), true, StringData.fromString("TRUE")) @@ -838,11 +839,11 @@ Stream testCases() { .fromCaseLegacy(BOOLEAN(), true, fromString("true")) .fromCase(BOOLEAN(), false, fromString("FALSE ")) .fromCaseLegacy(BOOLEAN(), false, fromString("false")) - .fromCase(BINARY(1), new byte[] {-12}, fromString("f4 ")) + .fromCase(BINARY(1), new byte[] {102}, fromString("f ")) .fromCaseLegacy(BINARY(1), new byte[] {102}, fromString("f")) - .fromCase(VARBINARY(1), new byte[] {23}, fromString("17 ")) + .fromCase(VARBINARY(1), new byte[] {33}, fromString("\u0021 ")) .fromCaseLegacy(VARBINARY(1), new byte[] {33}, fromString("\u0021")) - .fromCase(BYTES(), new byte[] {32}, fromString("20 ")) + .fromCase(BYTES(), new byte[] {32}, fromString(" ")) .fromCaseLegacy(BYTES(), new byte[] {32}, fromString(" ")) .fromCase(TINYINT(), (byte) -125, fromString("-125 ")) .fromCaseLegacy(TINYINT(), (byte) -125, fromString("-125")) @@ -863,20 +864,20 @@ Stream testCases() { CastTestSpecBuilder.testCastTo(CHAR(12)) .fromCase( BINARY(4), - new byte[] {-12, 32, 46, -72}, - fromString("f4202eb8 ")) + new byte[] {1, 11, 111, 2}, + fromString("\u0001\u000B\u006F\u0002 ")) .fromCaseLegacy( BINARY(4), new byte[] {1, 11, 111, 2}, fromString("\u0001\u000B\u006F\u0002")) - .fromCase(VARBINARY(4), new byte[] {1, 11, 22}, fromString("010b16 ")) + .fromCase( + VARBINARY(4), + new byte[] {1, 11, 22}, + fromString("\u0001\u000B\u0016 ")) .fromCaseLegacy( VARBINARY(4), new byte[] {1, 11, 22}, fromString("\u0001\u000B\u0016")) - .fromCase(BYTES(), new byte[] {1, 11}, fromString("010b ")) - .fromCaseLegacy( - BYTES(), new byte[] {1, 11, 111}, fromString("\u0001\u000B\u006F")) .fromCase( ARRAY(INT()), new GenericArrayData(new int[] {-1, 2, 3}), @@ -963,17 +964,28 @@ Stream testCases() { .fromCaseLegacy(BOOLEAN(), true, fromString("true")) .fromCase(BOOLEAN(), false, fromString("FAL")) .fromCaseLegacy(BOOLEAN(), false, fromString("false")) - .fromCase(BINARY(5), new byte[] {0, 1, 2, 3, 4}, fromString("000")) + .fromCase(BINARY(2), new byte[] {0, 1}, fromString("\u0000\u0001")) + .fromCaseLegacy(BINARY(1), new byte[] {0, 1}, fromString("\u0000\u0001")) + .fromCase( + BINARY(5), + new byte[] {0, 1, 2, 3, 4}, + fromString("\u0000\u0001\u0002")) .fromCaseLegacy( BINARY(5), new byte[] {0, 1, 2, 3, 4}, fromString("\u0000\u0001\u0002\u0003\u0004")) - .fromCase(VARBINARY(5), new byte[] {0, 1, 2, 3, 4}, fromString("000")) + .fromCase( + VARBINARY(5), + new byte[] {0, 1, 2, 3, 4}, + fromString("\u0000\u0001\u0002")) .fromCaseLegacy( VARBINARY(5), new byte[] {0, 1, 2, 3, 4}, fromString("\u0000\u0001\u0002\u0003\u0004")) - .fromCase(BYTES(), new byte[] {0, 1, 2, 3, 4}, fromString("000")) + .fromCase( + BYTES(), + new byte[] {0, 1, 2, 3, 4}, + fromString("\u0000\u0001\u0002")) .fromCaseLegacy( BYTES(), new byte[] {0, 1, 2, 3, 4}, @@ -1127,30 +1139,27 @@ Stream testCases() { .fromCase(DOUBLE(), 0.0d, false) .fromCase(DOUBLE(), -0.12345678d, true), CastTestSpecBuilder.testCastTo(BINARY(4)) - .fromCase(CHAR(4), fromString("66"), new byte[] {102, 0, 0, 0}) + .fromCase(CHAR(4), fromString("66"), new byte[] {54, 54, 0, 0}) + .fromCaseLegacy(CHAR(4), fromString("66"), new byte[] {54, 54}) + .fromCase(CHAR(3), fromString("foo"), new byte[] {102, 111, 111, 0}) .fromCaseLegacy(CHAR(3), fromString("foo"), new byte[] {102, 111, 111}) - .fromCase(CHAR(10), fromString("66A2"), new byte[] {102, -94, 0, 0}) + .fromCase(CHAR(10), fromString("66A2"), new byte[] {54, 54, 65, 50}) + .fromCaseLegacy(CHAR(10), fromString("66A2"), new byte[] {54, 54, 65, 50}) + .fromCase(CHAR(1), fromString("f"), new byte[] {102, 0, 0, 0}) .fromCaseLegacy(CHAR(1), fromString("f"), new byte[] {102}) - .fromCase(CHAR(16), fromString("12f4aBc7"), new byte[] {18, -12, -85, -57}) - .fromCaseLegacy(CHAR(3), fromString("f"), new byte[] {102}) - .fromCase(VARCHAR(8), fromString("bACd"), new byte[] {-70, -51, 0, 0}) + .fromCase(CHAR(16), fromString("12f4aBc7"), new byte[] {49, 50, 102, 52}) + .fromCase(CHAR(3), fromString("A f "), new byte[] {65, 32, 102, 32}) + .fromCase(VARCHAR(8), fromString("bAC"), new byte[] {98, 65, 67, 0}) + .fromCase(VARCHAR(5), fromString("Flink"), new byte[] {70, 108, 105, 110}) .fromCaseLegacy( VARCHAR(5), fromString("Flink"), new byte[] {70, 108, 105, 110, 107}) - .fromCase( - STRING(), - fromString("12f4ABc71232"), - new byte[] {18, -12, -85, -57}) + .fromCase(STRING(), fromString("Apache"), new byte[] {65, 112, 97, 99}) .fromCaseLegacy( STRING(), fromString("Apache"), new byte[] {65, 112, 97, 99, 104, 101}) - .fromCase(STRING(), fromString("12F4ab"), new byte[] {18, -12, -85, 0}) - .fromCaseLegacy(STRING(), fromString("bar"), new byte[] {98, 97, 114}) - .fail(STRING(), fromString("123"), TableException.class) - .fail(STRING(), fromString("12P9"), TableException.class) - .fail(STRING(), fromString("12 A9"), TableException.class) .fromCase(BINARY(2), new byte[] {1, 2}, new byte[] {1, 2, 0, 0}) .fromCaseLegacy(BINARY(2), new byte[] {1, 2}, new byte[] {1, 2}) .fromCase(VARBINARY(3), new byte[] {1, 2, 3}, new byte[] {1, 2, 3, 0}) @@ -1158,24 +1167,27 @@ Stream testCases() { .fromCase(BYTES(), new byte[] {1, 2, 3}, new byte[] {1, 2, 3, 0}) .fromCaseLegacy(BYTES(), new byte[] {1, 2, 3}, new byte[] {1, 2, 3}), CastTestSpecBuilder.testCastTo(VARBINARY(4)) - .fromCase(CHAR(4), fromString("c9"), new byte[] {-55}) + .fromCase(CHAR(4), fromString("c9"), new byte[] {99, 57}) + .fromCaseLegacy(CHAR(4), fromString("c9"), new byte[] {99, 57}) + .fromCase(CHAR(3), fromString("foo"), new byte[] {102, 111, 111}) .fromCaseLegacy(CHAR(3), fromString("foo"), new byte[] {102, 111, 111}) - .fromCase(VARCHAR(8), fromString("7de2"), new byte[] {125, -30}) + .fromCase(VARCHAR(8), fromString("7de2"), new byte[] {55, 100, 101, 50}) + .fromCaseLegacy( + VARCHAR(8), fromString("7de2"), new byte[] {55, 100, 101, 50}) + .fromCase(VARCHAR(5), fromString("Flink"), new byte[] {70, 108, 105, 110}) .fromCaseLegacy( VARCHAR(5), fromString("Flink"), new byte[] {70, 108, 105, 110, 107}) - .fromCase( + .fromCase(STRING(), fromString("12F4a bC7"), new byte[] {49, 50, 70, 52}) + .fromCaseLegacy( STRING(), - fromString("12F4abC71232"), - new byte[] {18, -12, -85, -57}) + fromString("12F4a bC7"), + new byte[] {49, 50, 70, 52, 97, 32, 98, 67, 55}) .fromCaseLegacy( STRING(), fromString("Apache"), new byte[] {65, 112, 97, 99, 104, 101}) - .fail(STRING(), fromString("123"), TableException.class) - .fail(STRING(), fromString("12P9"), TableException.class) - .fail(STRING(), fromString("12 A9"), TableException.class) // We assume that the input length is respected, therefore, no trimming is // applied .fromCase(BINARY(2), new byte[] {1, 2, 3, 4, 5}, new byte[] {1, 2, 3, 4, 5}) @@ -1190,24 +1202,37 @@ Stream testCases() { new byte[] {1, 2, 3, 4, 5}, new byte[] {1, 2, 3, 4, 5}), CastTestSpecBuilder.testCastTo(BYTES()) - .fromCase(CHAR(4), fromString("9C"), new byte[] {-100}) + .fromCase(CHAR(4), fromString("9C"), new byte[] {57, 67}) + .fromCaseLegacy(CHAR(4), fromString("9C"), new byte[] {57, 67}) + .fromCase(CHAR(3), fromString("foo"), new byte[] {102, 111, 111}) .fromCaseLegacy(CHAR(3), fromString("foo"), new byte[] {102, 111, 111}) - .fromCase(VARCHAR(8), fromString("3ee3"), new byte[] {62, -29}) + .fromCase(VARCHAR(8), fromString("3ee3"), new byte[] {51, 101, 101, 51}) + .fromCaseLegacy( + VARCHAR(8), fromString("3ee3"), new byte[] {51, 101, 101, 51}) + .fromCase( + VARCHAR(5), + fromString("Flink"), + new byte[] {70, 108, 105, 110, 107}) .fromCaseLegacy( VARCHAR(5), fromString("Flink"), new byte[] {70, 108, 105, 110, 107}) .fromCase( STRING(), - fromString("AAbbCcDdff"), - new byte[] {-86, -69, -52, -35, -1}) + fromString("AAbb Cc Dd"), + new byte[] {65, 65, 98, 98, 32, 67, 99, 32, 68, 100}) .fromCaseLegacy( + STRING(), + fromString("AAbb Cc Dd"), + new byte[] {65, 65, 98, 98, 32, 67, 99, 32, 68, 100}) + .fromCase( STRING(), fromString("Apache"), new byte[] {65, 112, 97, 99, 104, 101}) - .fail(STRING(), fromString("123"), TableException.class) - .fail(STRING(), fromString("12P9"), TableException.class) - .fail(STRING(), fromString("12 A9"), TableException.class), + .fromCaseLegacy( + STRING(), + fromString("Apache"), + new byte[] {65, 112, 97, 99, 104, 101}), CastTestSpecBuilder.testCastTo(DECIMAL(5, 3)) .fail(CHAR(3), fromString("foo"), TableException.class) .fail(VARCHAR(5), fromString("Flink"), TableException.class) diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala index ccdb37b7c93e4..490551ad7572e 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/expressions/ScalarFunctionsTest.scala @@ -4190,7 +4190,7 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { // the answer BINARY will cast to STRING in ExpressionTestBase.scala testSqlApi( "IF(f7 < 5, f53, f54)", - "68656c6c6f20776f726c64") // hello world + "hello world") // hello world // test DATE, DATE testSqlApi( @@ -4398,6 +4398,6 @@ class ScalarFunctionsTest extends ScalarTypesTestBase { testSqlApi(s"IFNULL(CAST(INTERVAL '2' DAY AS VARCHAR(20)), $str2)", "+2 00:00:00.000") testSqlApi( s"IFNULL(CAST(f53 AS VARCHAR(100)), $str2)", - "68656c6c6f20776f726c64") + "hello world") } } From 5f280f2eca4ad80ee981c44f114b549af5f5da71 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 19 Apr 2022 12:10:38 +0300 Subject: [PATCH 200/258] [FLINK-27212][table-planner] Change printing of binary columns to use `x'ab3234f0'` format Add an `isPrinting()` method to the `CastRule.Context` set to true used by `RowDataToStringConverterImpl` which defines a different casting behaviour for `BinaryToStringCastRule` when printing binary columns, so that we output columns of binary type as (for example): `x'ab03f98e'`, which can easily be copy pasted as a binary literal to another SQL query. (cherry picked from commit 75007f6ab601c285a2dbfc4f1c41952269979967) This closes #19516. --- .../client/cli/CliTableauResultViewTest.java | 107 ++++++++++-------- .../AbstractCodeGeneratorCastRule.java | 5 + ...stractExpressionCodeGeneratorCastRule.java | 5 + .../casting/BinaryToStringCastRule.java | 25 ++-- .../planner/functions/casting/CastRule.java | 16 ++- .../functions/casting/CastRuleUtils.java | 4 + .../casting/CodeGeneratorCastRule.java | 6 + .../casting/RowDataToStringConverterImpl.java | 2 +- .../codegen/calls/ScalarOperatorGens.scala | 3 + .../functions/casting/CastRulesTest.java | 26 ++++- 10 files changed, 137 insertions(+), 62 deletions(-) diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java index 02b6287763476..4deb8da86f86c 100644 --- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java +++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliTableauResultViewTest.java @@ -80,7 +80,8 @@ public void setUp() { Column.physical("varchar", DataTypes.STRING()), Column.physical("decimal(10, 5)", DataTypes.DECIMAL(10, 5)), Column.physical( - "timestamp", DataTypes.TIMESTAMP(6).bridgedTo(Timestamp.class))); + "timestamp", DataTypes.TIMESTAMP(6).bridgedTo(Timestamp.class)), + Column.physical("binary", DataTypes.BYTES())); rowDataToStringConverter = new RowDataToStringConverterImpl(schema.toPhysicalRowDataType()); List rows = @@ -92,7 +93,8 @@ public void setUp() { 2L, "abc", BigDecimal.valueOf(1.23), - Timestamp.valueOf("2020-03-01 18:39:14")), + Timestamp.valueOf("2020-03-01 18:39:14"), + new byte[] {50, 51, 52, -123, 54, 93, 115, 126}), Row.ofKind( RowKind.UPDATE_BEFORE, false, @@ -100,7 +102,8 @@ public void setUp() { 0L, "", BigDecimal.valueOf(1), - Timestamp.valueOf("2020-03-01 18:39:14.1")), + Timestamp.valueOf("2020-03-01 18:39:14.1"), + new byte[] {100, -98, 32, 121, -125}), Row.ofKind( RowKind.UPDATE_AFTER, true, @@ -108,7 +111,8 @@ public void setUp() { null, "abcdefg", BigDecimal.valueOf(12345), - Timestamp.valueOf("2020-03-01 18:39:14.12")), + Timestamp.valueOf("2020-03-01 18:39:14.12"), + new byte[] {-110, -23, 1, 2}), Row.ofKind( RowKind.DELETE, false, @@ -116,7 +120,8 @@ public void setUp() { Long.MAX_VALUE, null, BigDecimal.valueOf(12345.06789), - Timestamp.valueOf("2020-03-01 18:39:14.123")), + Timestamp.valueOf("2020-03-01 18:39:14.123"), + new byte[] {50, 51, 52, -123, 54, 93, 115, 126}), Row.ofKind( RowKind.INSERT, true, @@ -124,7 +129,8 @@ public void setUp() { Long.MIN_VALUE, "abcdefg111", null, - Timestamp.valueOf("2020-03-01 18:39:14.123456")), + Timestamp.valueOf("2020-03-01 18:39:14.123456"), + new byte[] {110, 23, -1, -2}), Row.ofKind( RowKind.DELETE, null, @@ -132,6 +138,7 @@ public void setUp() { -1L, "abcdefghijklmnopqrstuvwxyz", BigDecimal.valueOf(-12345.06789), + null, null), Row.ofKind( RowKind.INSERT, @@ -140,7 +147,8 @@ public void setUp() { -1L, "这是一段中文", BigDecimal.valueOf(-12345.06789), - Timestamp.valueOf("2020-03-04 18:39:14")), + Timestamp.valueOf("2020-03-04 18:39:14"), + new byte[] {-3, -2, -1, 0, 1, 2, 3}), Row.ofKind( RowKind.DELETE, null, @@ -148,7 +156,8 @@ public void setUp() { -1L, "これは日本語をテストするための文です", BigDecimal.valueOf(-12345.06789), - Timestamp.valueOf("2020-03-04 18:39:14"))); + Timestamp.valueOf("2020-03-04 18:39:14"), + new byte[] {-3, -2, -1, 0, 1, 2, 3})); final DataStructureConverter dataStructureConverter = DataStructureConverters.getConverter(schema.toPhysicalRowDataType()); @@ -187,29 +196,29 @@ public void testBatchResult() { view.displayResults(); view.close(); Assert.assertEquals( - "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+---------------------+" + System.lineSeparator() - + "| boolean | int | bigint | varchar | decimal(10, 5) | timestamp |" + + "| boolean | int | bigint | varchar | decimal(10, 5) | timestamp | binary |" + System.lineSeparator() - + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+---------------------+" + System.lineSeparator() - + "| | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 |" + + "| | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 | x'32333485365d737e' |" + System.lineSeparator() - + "| FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 |" + + "| FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 | x'649e207983' |" + System.lineSeparator() - + "| TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 |" + + "| TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 | x'92e90102' |" + System.lineSeparator() - + "| FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 |" + + "| FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 | x'32333485365d737e' |" + System.lineSeparator() - + "| TRUE | 100 | -9223372036854775808 | abcdefg111 | | 2020-03-01 18:39:14.123456 |" + + "| TRUE | 100 | -9223372036854775808 | abcdefg111 | | 2020-03-01 18:39:14.123456 | x'6e17fffe' |" + System.lineSeparator() - + "| | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | |" + + "| | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | | |" + System.lineSeparator() - + "| | -1 | -1 | 这是一段中文 | -12345.06789 | 2020-03-04 18:39:14.000000 |" + + "| | -1 | -1 | 这是一段中文 | -12345.06789 | 2020-03-04 18:39:14.000000 | x'fdfeff00010203' |" + System.lineSeparator() - + "| | -1 | -1 | これは日本語をテストするた... | -12345.06789 | 2020-03-04 18:39:14.000000 |" + + "| | -1 | -1 | これは日本語をテストするた... | -12345.06789 | 2020-03-04 18:39:14.000000 | x'fdfeff00010203' |" + System.lineSeparator() - + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+---------------------+" + System.lineSeparator() + "8 rows in set" + System.lineSeparator(), @@ -350,29 +359,29 @@ public void testStreamingResult() { // source file // by vim or just cat the file to check the regular result. Assert.assertEquals( - "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |" + + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp | binary |" + System.lineSeparator() - + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 |" + + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 | x'32333485365d737e' |" + System.lineSeparator() - + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 |" + + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 | x'649e207983' |" + System.lineSeparator() - + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 |" + + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 | x'92e90102' |" + System.lineSeparator() - + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 |" + + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 | x'32333485365d737e' |" + System.lineSeparator() - + "| +I | TRUE | 100 | -9223372036854775808 | abcdefg111 | | 2020-03-01 18:39:14.123456 |" + + "| +I | TRUE | 100 | -9223372036854775808 | abcdefg111 | | 2020-03-01 18:39:14.123456 | x'6e17fffe' |" + System.lineSeparator() - + "| -D | | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | |" + + "| -D | | -1 | -1 | abcdefghijklmnopqrstuvwxyz | -12345.06789 | | |" + System.lineSeparator() - + "| +I | | -1 | -1 | 这是一段中文 | -12345.06789 | 2020-03-04 18:39:14.000000 |" + + "| +I | | -1 | -1 | 这是一段中文 | -12345.06789 | 2020-03-04 18:39:14.000000 | x'fdfeff00010203' |" + System.lineSeparator() - + "| -D | | -1 | -1 | これは日本語をテストするた... | -12345.06789 | 2020-03-04 18:39:14.000000 |" + + "| -D | | -1 | -1 | これは日本語をテストするた... | -12345.06789 | 2020-03-04 18:39:14.000000 | x'fdfeff00010203' |" + System.lineSeparator() - + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() + "Received a total of 8 rows" + System.lineSeparator(), @@ -400,11 +409,11 @@ public void testEmptyStreamingResult() { view.close(); Assert.assertEquals( - "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |" + + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp | binary |" + System.lineSeparator() - + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() + "Received a total of 0 row" + System.lineSeparator(), @@ -446,19 +455,19 @@ public void testCancelStreamingResult() throws Exception { view.close(); Assert.assertEquals( - "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |" + + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp | binary |" + System.lineSeparator() - + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 |" + + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 | x'32333485365d737e' |" + System.lineSeparator() - + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 |" + + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 | x'649e207983' |" + System.lineSeparator() - + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 |" + + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 | x'92e90102' |" + System.lineSeparator() - + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 |" + + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 | x'32333485365d737e' |" + System.lineSeparator() + "Query terminated, received a total of 4 rows" + System.lineSeparator(), @@ -498,19 +507,19 @@ public void testFailedStreamingResult() { view.close(); Assert.assertEquals( - "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp |" + + "| op | boolean | int | bigint | varchar | decimal(10, 5) | timestamp | binary |" + System.lineSeparator() - + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+" + + "+----+---------+-------------+----------------------+--------------------------------+----------------+----------------------------+--------------------------------+" + System.lineSeparator() - + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 |" + + "| +I | | 1 | 2 | abc | 1.23000 | 2020-03-01 18:39:14.000000 | x'32333485365d737e' |" + System.lineSeparator() - + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 |" + + "| -U | FALSE | | 0 | | 1.00000 | 2020-03-01 18:39:14.100000 | x'649e207983' |" + System.lineSeparator() - + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 |" + + "| +U | TRUE | 2147483647 | | abcdefg | 12345.00000 | 2020-03-01 18:39:14.120000 | x'92e90102' |" + System.lineSeparator() - + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 |" + + "| -D | FALSE | -2147483648 | 9223372036854775807 | | 12345.06789 | 2020-03-01 18:39:14.123000 | x'32333485365d737e' |" + System.lineSeparator(), terminalOutput.toString()); assertThat(mockExecutor.getNumCancelCalls(), is(1)); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCodeGeneratorCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCodeGeneratorCastRule.java index 7ef1a2852a361..90c5f226dd8e9 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCodeGeneratorCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractCodeGeneratorCastRule.java @@ -186,6 +186,11 @@ private CastExecutorCodeGeneratorContext(CastRule.Context castRuleCtx) { this.castRuleCtx = castRuleCtx; } + @Override + public boolean isPrinting() { + return castRuleCtx.isPrinting(); + } + @Override public boolean legacyBehaviour() { return castRuleCtx.legacyBehaviour(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractExpressionCodeGeneratorCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractExpressionCodeGeneratorCastRule.java index c28d6463803ee..4b05d0bbf0832 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractExpressionCodeGeneratorCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/AbstractExpressionCodeGeneratorCastRule.java @@ -103,6 +103,11 @@ public CastExecutor create( private static CodeGeneratorCastRule.Context createCodeGeneratorCastRuleContext( CastRule.Context ctx) { return new CodeGeneratorCastRule.Context() { + @Override + public boolean isPrinting() { + return ctx.isPrinting(); + } + @Override public boolean legacyBehaviour() { return ctx.legacyBehaviour(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java index d48047809b6d6..ccb3073ce40f3 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/BinaryToStringCastRule.java @@ -21,6 +21,7 @@ import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.LogicalTypeFamily; import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; +import org.apache.flink.table.utils.EncodingUtils; import java.nio.charset.StandardCharsets; @@ -28,6 +29,7 @@ import static org.apache.flink.table.planner.codegen.calls.BuiltInMethods.BINARY_STRING_DATA_FROM_STRING; import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.accessStaticField; import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.constructorCall; +import static org.apache.flink.table.planner.functions.casting.CastRuleUtils.staticCall; /** * {@link LogicalTypeFamily#BINARY_STRING} to {@link LogicalTypeFamily#CHARACTER_STRING} cast rule. @@ -85,15 +87,22 @@ protected String generateCodeBlockInternal( final String resultStringTerm = newName("resultString"); final CastRuleUtils.CodeWriter writer = new CastRuleUtils.CodeWriter(); - writer.declStmt(String.class, resultStringTerm) - .assignStmt( - resultStringTerm, - constructorCall( - String.class, - inputTerm, - accessStaticField(StandardCharsets.class, "UTF_8"))); + writer.declStmt(String.class, resultStringTerm); + if (context.isPrinting()) { + writer.assignStmt(resultStringTerm, "\"x'\"") + .assignPlusStmt( + resultStringTerm, staticCall(EncodingUtils.class, "hex", inputTerm)) + .assignPlusStmt(resultStringTerm, "\"'\""); + } else { + writer.assignStmt( + resultStringTerm, + constructorCall( + String.class, + inputTerm, + accessStaticField(StandardCharsets.class, "UTF_8"))); + } - if (!context.legacyBehaviour()) { + if (!context.legacyBehaviour() && !context.isPrinting()) { final String resultPadOrTrim = newName("resultPadOrTrim"); final int length = LogicalTypeChecks.getLength(targetLogicalType); CharVarCharTrimPadCastRule.padAndTrimStringIfNeeded( diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java index 57cf29ed2bc67..6b6e036d763fb 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRule.java @@ -34,7 +34,7 @@ @Internal public interface CastRule { - /** @see CastRulePredicate for more details about a cast rule predicate definition */ + /** @see CastRulePredicate for more details about a cast rule predicate definition. */ CastRulePredicate getPredicateDefinition(); /** @@ -50,6 +50,9 @@ CastExecutor create( /** Casting context. */ interface Context { + + boolean isPrinting(); + @Deprecated boolean legacyBehaviour(); @@ -58,8 +61,17 @@ interface Context { ClassLoader getClassLoader(); /** Create a casting context. */ - static Context create(boolean legacyBehaviour, ZoneId zoneId, ClassLoader classLoader) { + static Context create( + boolean isPrinting, + boolean legacyBehaviour, + ZoneId zoneId, + ClassLoader classLoader) { return new Context() { + @Override + public boolean isPrinting() { + return isPrinting; + } + @Override public boolean legacyBehaviour() { return legacyBehaviour; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java index bb1a8dae0f6ef..8468a587f8dea 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CastRuleUtils.java @@ -212,6 +212,10 @@ public CodeWriter assignStmt(String varName, String value) { return stmt(varName + " = " + value); } + public CodeWriter assignPlusStmt(String varName, String value) { + return stmt(varName + " += " + value); + } + public CodeWriter assignArrayStmt(String varName, String index, String value) { return stmt(varName + "[" + index + "] = " + value); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CodeGeneratorCastRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CodeGeneratorCastRule.java index 4dba7141a9280..3f6be48b6b3d0 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CodeGeneratorCastRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/CodeGeneratorCastRule.java @@ -44,6 +44,12 @@ CastCodeBlock generateCodeBlock( /** Context for code generation. */ interface Context { + /** + * @return whether it's in printing mode or not. Printing is used by {@link + * RowDataToStringConverterImpl} when printing table row results. + */ + boolean isPrinting(); + /** @return where the legacy behaviour should be followed or not. */ @Deprecated boolean legacyBehaviour(); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java index fcbd429ebdfac..f9a2c2f3e3379 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/functions/casting/RowDataToStringConverterImpl.java @@ -55,7 +55,7 @@ public RowDataToStringConverterImpl(DataType dataType) { public RowDataToStringConverterImpl( DataType dataType, ZoneId zoneId, ClassLoader classLoader, boolean legacyBehaviour) { this.dataType = dataType; - this.castRuleContext = CastRule.Context.create(legacyBehaviour, zoneId, classLoader); + this.castRuleContext = CastRule.Context.create(true, legacyBehaviour, zoneId, classLoader); } @SuppressWarnings("unchecked") diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala index 6d920b4787078..543cc92d22036 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/calls/ScalarOperatorGens.scala @@ -1829,6 +1829,7 @@ object ScalarOperatorGens { def toCodegenCastContext(ctx: CodeGeneratorContext): CodeGeneratorCastRule.Context = { new CodeGeneratorCastRule.Context { + override def isPrinting(): Boolean = false override def legacyBehaviour(): Boolean = isLegacyCastBehaviourEnabled(ctx) override def getSessionTimeZoneTerm: String = ctx.addReusableSessionTimeZone() override def declareVariable(ty: String, variablePrefix: String): String = @@ -1844,6 +1845,8 @@ object ScalarOperatorGens { def toCastContext(ctx: CodeGeneratorContext): CastRule.Context = { new CastRule.Context { + override def isPrinting(): Boolean = false + override def legacyBehaviour(): Boolean = isLegacyCastBehaviourEnabled(ctx) override def getSessionZoneId: ZoneId = ctx.tableConfig.getLocalTimeZone diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java index bb6663f5c5d69..f676f88ff8f39 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/casting/CastRulesTest.java @@ -107,9 +107,11 @@ class CastRulesTest { private static final ZoneId CET = ZoneId.of("CET"); private static final CastRule.Context CET_CONTEXT = - CastRule.Context.create(false, CET, Thread.currentThread().getContextClassLoader()); + CastRule.Context.create( + false, false, CET, Thread.currentThread().getContextClassLoader()); private static final CastRule.Context CET_CONTEXT_LEGACY = - CastRule.Context.create(true, CET, Thread.currentThread().getContextClassLoader()); + CastRule.Context.create( + false, true, CET, Thread.currentThread().getContextClassLoader()); private static final byte DEFAULT_POSITIVE_TINY_INT = (byte) 5; private static final byte DEFAULT_NEGATIVE_TINY_INT = (byte) -5; @@ -667,6 +669,10 @@ Stream testCases() { BYTES(), new byte[] {70, 108, 105, 110, 107}, fromString("Flink")) .fromCaseLegacy( BYTES(), new byte[] {70, 108, 105, 110, 107}, fromString("Flink")) + .fromCasePrinting( + BYTES(), + new byte[] {70, 108, 105, 110, 107}, + fromString("x'466c696e6b'")) .fromCase(BOOLEAN(), true, StringData.fromString("TRUE")) .fromCase(BOOLEAN(), false, StringData.fromString("FALSE")) .fromCase( @@ -1573,6 +1579,20 @@ private CastTestSpecBuilder fromCase(DataType srcDataType, Object src, Object ta return fromCase( srcDataType, CastRule.Context.create( + false, + false, + DateTimeUtils.UTC_ZONE.toZoneId(), + Thread.currentThread().getContextClassLoader()), + src, + target); + } + + private CastTestSpecBuilder fromCasePrinting( + DataType srcDataType, Object src, Object target) { + return fromCase( + srcDataType, + CastRule.Context.create( + true, false, DateTimeUtils.UTC_ZONE.toZoneId(), Thread.currentThread().getContextClassLoader()), @@ -1585,6 +1605,7 @@ private CastTestSpecBuilder fromCaseLegacy( return fromCase( srcDataType, CastRule.Context.create( + false, true, DateTimeUtils.UTC_ZONE.toZoneId(), Thread.currentThread().getContextClassLoader()), @@ -1614,6 +1635,7 @@ private CastTestSpecBuilder fail( return fail( dataType, CastRule.Context.create( + false, false, DateTimeUtils.UTC_ZONE.toZoneId(), Thread.currentThread().getContextClassLoader()), From 5e9ccf1cdd4328885ac50fad4d32fc93a62290fd Mon Sep 17 00:00:00 2001 From: Shengkai <1059623455@qq.com> Date: Fri, 15 Apr 2022 16:58:57 +0800 Subject: [PATCH 201/258] [FLINK-27263][table] Rename the metadata column to the user specified name in DDL This closes #19490. --- .../table/catalog/DefaultSchemaResolver.java | 28 ++++++++++ .../table/catalog/SchemaResolutionTest.java | 20 +++++++ .../abilities/SupportsWritingMetadata.java | 16 +++--- .../abilities/SupportsReadingMetadata.java | 11 ++-- .../planner/connectors/DynamicSinkUtils.java | 47 ++++++++++------- .../connectors/DynamicSourceUtils.java | 47 +++++++++-------- .../PushProjectIntoTableSourceScanRule.java | 30 ++++++++--- ...ushProjectIntoTableSourceScanRuleTest.java | 14 +++-- .../file/table/FileSystemTableSourceTest.xml | 4 +- .../plan/batch/sql/TableSourceTest.xml | 4 +- .../testWritingMetadata.out | 2 +- .../testReadingMetadata.out | 10 ++-- ...PushProjectIntoTableSourceScanRuleTest.xml | 17 ++++++ ...shWatermarkIntoTableSourceScanRuleTest.xml | 4 +- ...ushLocalAggIntoTableSourceScanRuleTest.xml | 2 +- .../plan/stream/sql/SourceWatermarkTest.xml | 4 +- .../planner/plan/stream/sql/TableScanTest.xml | 12 ++--- .../planner/plan/stream/sql/TableSinkTest.xml | 40 +++++++------- .../plan/stream/sql/TableSourceTest.xml | 4 +- .../table/api/TableEnvironmentTest.scala | 1 - .../plan/stream/sql/TableScanTest.scala | 52 +++++++++++++++++++ .../stream/sql/TableSourceITCase.scala | 17 ++++++ 22 files changed, 282 insertions(+), 104 deletions(-) diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java index 0b68aeff79b52..f2f011e22c297 100644 --- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java +++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/DefaultSchemaResolver.java @@ -43,6 +43,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -95,6 +96,7 @@ public ResolvedSchema resolve(Schema schema) { private List resolveColumns(List unresolvedColumns) { validateDuplicateColumns(unresolvedColumns); + validateDuplicateMetadataKeys(unresolvedColumns); final Column[] resolvedColumns = new Column[unresolvedColumns.size()]; // process source columns first before computed columns @@ -175,6 +177,32 @@ private void validateDuplicateColumns(List columns) { } } + private void validateDuplicateMetadataKeys(List columns) { + Map metadataKeyToColumnNames = new HashMap<>(); + for (Schema.UnresolvedColumn column : columns) { + if (!(column instanceof UnresolvedMetadataColumn)) { + continue; + } + + UnresolvedMetadataColumn metadataColumn = (UnresolvedMetadataColumn) column; + String metadataKey = + metadataColumn.getMetadataKey() == null + ? metadataColumn.getName() + : metadataColumn.getMetadataKey(); + if (metadataKeyToColumnNames.containsKey(metadataKey)) { + throw new ValidationException( + String.format( + "The column `%s` and `%s` in the table are both from the same metadata key '%s'. " + + "Please specify one of the columns as the metadata column and use the " + + "computed column syntax to specify the others.", + metadataKeyToColumnNames.get(metadataKey), + metadataColumn.getName(), + metadataKey)); + } + metadataKeyToColumnNames.put(metadataKey, metadataColumn.getName()); + } + } + private List resolveWatermarkSpecs( List unresolvedWatermarkSpecs, List inputColumns) { if (unresolvedWatermarkSpecs.size() == 0) { diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java index 7bf66e96b0855..2e50f73ae8108 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/catalog/SchemaResolutionTest.java @@ -221,6 +221,26 @@ public void testSchemaResolutionErrors() { Schema.newBuilder().columnByExpression("invalid", callSql("INVALID")).build(), "Invalid expression for computed column 'invalid'."); + // metadata columns + + testError( + Schema.newBuilder() + .columnByMetadata("metadata", DataTypes.INT()) + .columnByMetadata("from_metadata", DataTypes.BIGINT(), "metadata", false) + .build(), + "The column `metadata` and `from_metadata` in the table are both from the same metadata key 'metadata'. " + + "Please specify one of the columns as the metadata column and use the " + + "computed column syntax to specify the others."); + + testError( + Schema.newBuilder() + .columnByMetadata("from_metadata", DataTypes.BIGINT(), "metadata", false) + .columnByMetadata("from_metadata2", DataTypes.STRING(), "metadata", true) + .build(), + "The column `from_metadata` and `from_metadata2` in the table are both from the same metadata key 'metadata'. " + + "Please specify one of the columns as the metadata column and use the " + + "computed column syntax to specify the others."); + // time attributes and watermarks testError( diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java index 858bc05ef6c5c..d789da1d842f6 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/sink/abilities/SupportsWritingMetadata.java @@ -68,13 +68,17 @@ * casting from INT will be performed by the planner in a preceding operation: * *

    {@code
    - * // for t1 and t2
    - * ROW < i INT, s STRING, d DOUBLE >                                              // physical input
    - * ROW < i INT, s STRING, d DOUBLE, $metadata$timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE > // final input
    + * // physical input
    + * ROW < i INT, s STRING, d DOUBLE >
      *
    - * // for t3
    - * ROW < i INT, s STRING, d DOUBLE >                                              // physical input
    - * ROW < i INT, s STRING, d DOUBLE >                                              // final input
    + * // final input (i.e. consumed type) for t1
    + * ROW < i INT, s STRING, d DOUBLE, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE >
    + *
    + * // final input (i.e. consumed type) for t2
    + * ROW < i INT, s STRING, d DOUBLE, myTimestamp TIMESTAMP(3) WITH LOCAL TIME ZONE >
    + *
    + * // final input (i.e. consumed type) for t3
    + * ROW < i INT, s STRING, d DOUBLE >
      * }
    */ @PublicEvolving diff --git a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java index ac462b668a992..60b6932b61769 100644 --- a/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java +++ b/flink-table/flink-table-common/src/main/java/org/apache/flink/table/connector/source/abilities/SupportsReadingMetadata.java @@ -76,9 +76,14 @@ * casting to INT will be performed by the planner in a subsequent operation: * *
    {@code
    - * // for t1 and t2
    - * ROW < i INT, s STRING, d DOUBLE >                                              // physical output
    - * ROW < i INT, s STRING, d DOUBLE, $metadata$timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE > // final output
    + * // physical output
    + * ROW < i INT, s STRING, d DOUBLE >
    + *
    + * // final output (i.e. produced type) for t1
    + * ROW < i INT, s STRING, d DOUBLE, timestamp TIMESTAMP(3) WITH LOCAL TIME ZONE >
    + *
    + * // final output (i.e. produced type) for t2
    + * ROW < i INT, s STRING, d DOUBLE, myTimestamp TIMESTAMP(3) WITH LOCAL TIME ZONE >
      * }
    */ @PublicEvolving diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java index 7a906ffa9bd26..b810e71c61551 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSinkUtils.java @@ -66,9 +66,9 @@ import java.time.ZoneId; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -84,9 +84,6 @@ @Internal public final class DynamicSinkUtils { - // Ensures that physical and metadata columns don't collide. - private static final String METADATA_COLUMN_PREFIX = "$metadata$"; - /** Converts an {@link TableResult#collect()} sink to a {@link RelNode}. */ public static RelNode convertCollectToRel( FlinkRelBuilder relBuilder, @@ -330,7 +327,8 @@ private static void pushMetadataProjection( Function.identity())); final List metadataColumns = - createRequiredMetadataKeys(schema, sink).stream() + createRequiredMetadataColumns(schema, sink).stream() + .map(col -> col.getMetadataKey().orElse(col.getName())) .map(keyToMetadataColumn::get) .collect(Collectors.toList()); @@ -405,28 +403,31 @@ private static void prepareDynamicSink( } /** - * Returns a list of required metadata keys. Ordered by the iteration order of {@link + * Returns a list of required metadata columns. Ordered by the iteration order of {@link * SupportsWritingMetadata#listWritableMetadata()}. * *

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

    This method assumes that source and schema have been validated via {@link * #prepareDynamicSource(String, ResolvedCatalogTable, DynamicTableSource, boolean, * ReadableConfig)}. */ - public static List createRequiredMetadataKeys( + public static List createRequiredMetadataColumns( ResolvedSchema schema, DynamicTableSource source) { final List metadataColumns = extractMetadataColumns(schema); - final Set requiredMetadataKeys = - metadataColumns.stream() - .map(c -> c.getMetadataKey().orElse(c.getName())) - .collect(Collectors.toSet()); + Map metadataKeysToMetadataColumns = new HashMap<>(); + + for (MetadataColumn column : metadataColumns) { + String metadataKey = column.getMetadataKey().orElse(column.getName()); + // After resolving, every metadata column has the unique metadata key. + metadataKeysToMetadataColumns.put(metadataKey, column); + } final Map metadataMap = extractMetadataMap(source); + // reorder the column return metadataMap.keySet().stream() - .filter(requiredMetadataKeys::contains) + .filter(metadataKeysToMetadataColumns::containsKey) + .map(metadataKeysToMetadataColumns::get) .collect(Collectors.toList()); } @@ -206,12 +208,16 @@ public static RowType createProducedType(ResolvedSchema schema, DynamicTableSour ((RowType) schema.toPhysicalRowDataType().getLogicalType()).getFields().stream(); final Stream metadataFields = - createRequiredMetadataKeys(schema, source).stream() + createRequiredMetadataColumns(schema, source).stream() .map( k -> new RowField( - METADATA_COLUMN_PREFIX + k, - metadataMap.get(k).getLogicalType())); + // Use the alias to ensure that physical and + // metadata columns don't collide + k.getName(), + metadataMap + .get(k.getMetadataKey().orElse(k.getName())) + .getLogicalType())); final List rowFields = Stream.concat(physicalFields, metadataFields).collect(Collectors.toList()); @@ -317,14 +323,9 @@ private static void pushMetadataProjection(FlinkRelBuilder relBuilder, ResolvedS c.getDataType().getLogicalType()); if (c instanceof MetadataColumn) { final MetadataColumn metadataColumn = (MetadataColumn) c; - final String metadataKey = - metadataColumn - .getMetadataKey() - .orElse(metadataColumn.getName()); + String columnName = metadataColumn.getName(); return rexBuilder.makeAbstractCast( - relDataType, - relBuilder.field( - METADATA_COLUMN_PREFIX + metadataKey)); + relDataType, relBuilder.field(columnName)); } else { return relBuilder.field(c.getName()); } @@ -444,7 +445,9 @@ private static void validateAndApplyMetadata( }); metadataSource.applyReadableMetadata( - createRequiredMetadataKeys(schema, source), + createRequiredMetadataColumns(schema, source).stream() + .map(column -> column.getMetadataKey().orElse(column.getName())) + .collect(Collectors.toList()), TypeConversions.fromLogicalToDataType(createProducedType(schema, source))); } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java index 58af37cbcbf50..17e87a4bd4581 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java @@ -60,9 +60,8 @@ import java.util.stream.IntStream; import java.util.stream.Stream; -import static org.apache.flink.table.planner.connectors.DynamicSourceUtils.METADATA_COLUMN_PREFIX; import static org.apache.flink.table.planner.connectors.DynamicSourceUtils.createProducedType; -import static org.apache.flink.table.planner.connectors.DynamicSourceUtils.createRequiredMetadataKeys; +import static org.apache.flink.table.planner.connectors.DynamicSourceUtils.createRequiredMetadataColumns; import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapContext; import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTypeFactory; @@ -250,12 +249,16 @@ private RowType performPushDown( final List projectedMetadataColumns; if (supportsMetadata(source.tableSource())) { final List declaredMetadataKeys = - createRequiredMetadataKeys( - source.contextResolvedTable().getResolvedSchema(), - source.tableSource()); + createRequiredMetadataColumns( + source.contextResolvedTable().getResolvedSchema(), + source.tableSource()) + .stream() + .map(col -> col.getMetadataKey().orElse(col.getName())) + .collect(Collectors.toList()); numPhysicalColumns = producedType.getFieldCount() - declaredMetadataKeys.size(); + // the projected metadata column name projectedMetadataColumns = IntStream.range(0, declaredMetadataKeys.size()) .mapToObj(i -> producedType.getFieldNames().get(numPhysicalColumns + i)) @@ -306,10 +309,23 @@ private RowType performPushDown( (RowType) Projection.of(projectedFields).project(producedType); if (supportsMetadata(source.tableSource())) { + // Use the projected column name to get the metadata key final List projectedMetadataKeys = projectedMetadataColumns.stream() - .map(NestedColumn::name) - .map(k -> k.substring(METADATA_COLUMN_PREFIX.length())) + .map( + nestedColumn -> + source.contextResolvedTable() + .getResolvedSchema() + .getColumn(nestedColumn.name()) + .orElseThrow( + () -> + new TableException( + String.format( + "Can not find the column %s in the origin schema.", + nestedColumn + .name())))) + .map(Column.MetadataColumn.class::cast) + .map(col -> col.getMetadataKey().orElse(col.getName())) .collect(Collectors.toList()); abilitySpecs.add(new ReadingMetadataSpec(projectedMetadataKeys, newProducedType)); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java index 3f38bae8ab52a..8f3a4196d7143 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java @@ -118,7 +118,8 @@ public void setup() { + " id int,\n" + " deepNested row, nested2 row>,\n" + " metadata_1 int metadata,\n" - + " metadata_2 string metadata\n" + + " metadata_2 string metadata,\n" + + " metadata_3 as cast(metadata_1 as bigint)\n" + ") WITH (" + " 'connector' = 'values'," + " 'nested-projection-supported' = 'true'," @@ -207,6 +208,13 @@ public void testComplicatedNestedProject() { util().verifyRelPlan(sqlQuery); } + @Test + public void testProjectWithDuplicateMetadataKey() { + String sqlQuery = "SELECT id, metadata_3, metadata_1 FROM MetadataTable"; + + util().verifyRelPlan(sqlQuery); + } + @Test public void testNestProjectWithMetadata() { String sqlQuery = @@ -349,7 +357,7 @@ public void testProjectionIncludingOnlyMetadata() { equalTo(Collections.emptyList())); assertThat( DataType.getFieldNames(appliedMetadataDataType.get()), - equalTo(Collections.singletonList("$metadata$m2"))); + equalTo(Collections.singletonList("metadata"))); } @Test @@ -375,7 +383,7 @@ public void testProjectionWithMetadataAndPhysicalFields() { equalTo(Collections.singletonList("f1"))); assertThat( DataType.getFieldNames(appliedMetadataDataType.get()), - equalTo(Arrays.asList("f1", "$metadata$m2"))); + equalTo(Arrays.asList("f1", "metadata"))); } // --------------------------------------------------------------------------------------------- diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml index 479eb8ad78795..41096c54733cf 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/connector/file/table/FileSystemTableSourceTest.xml @@ -50,8 +50,8 @@ LogicalSink(table=[default_catalog.default_database.MySink], fields=[a, b, filem diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml index 037458a3857f5..4c1344574125b 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.xml @@ -113,8 +113,8 @@ LogicalProject(id=[$0], nested1=[$1.nested1], results=[+(+($1.nested1.value, $1. diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out index 53107af6c8869..60411d8a3d952 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSinkJsonPlanTest_jsonplan/testWritingMetadata.out @@ -69,7 +69,7 @@ "abilities" : [ { "type" : "WritingMetadata", "metadataKeys" : [ "m" ], - "consumedType" : "ROW<`a` BIGINT, `b` INT, `$metadata$m` VARCHAR(2147483647)> NOT NULL" + "consumedType" : "ROW<`a` BIGINT, `b` INT, `m` VARCHAR(2147483647)> NOT NULL" } ] }, "inputChangelogMode" : [ "INSERT" ], diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out index 81bc950fda9fe..a8014cf440021 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/TableSourceJsonPlanTest_jsonplan/testReadingMetadata.out @@ -40,11 +40,11 @@ }, { "type" : "ReadingMetadata", "metadataKeys" : [ "m" ], - "producedType" : "ROW<`a` BIGINT, `b` INT, `$metadata$m` VARCHAR(2147483647)> NOT NULL" + "producedType" : "ROW<`a` BIGINT, `b` INT, `m` VARCHAR(2147483647)> NOT NULL" } ] }, - "outputType" : "ROW<`a` BIGINT, `b` INT, `$metadata$m` VARCHAR(2147483647)>", - "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, b], metadata=[m]]], fields=[a, b, $metadata$m])", + "outputType" : "ROW<`a` BIGINT, `b` INT, `m` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable2, project=[a, b], metadata=[m]]], fields=[a, b, m])", "inputProperties" : [ ] }, { "id" : 2, @@ -88,8 +88,8 @@ "damBehavior" : "PIPELINED", "priority" : 0 } ], - "outputType" : "ROW<`a` BIGINT, `b` INT, `$metadata$m` VARCHAR(2147483647)>", - "description" : "Sink(table=[default_catalog.default_database.sink], fields=[a, b, $metadata$m])" + "outputType" : "ROW<`a` BIGINT, `b` INT, `m` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.sink], fields=[a, b, m])" } ], "edges" : [ { "source" : 1, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml index 2115b4b73f2e4..9c99e35123b64 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.xml @@ -312,6 +312,23 @@ LogicalProject(id=[$0], EXPR$1=[ITEM($5, _UTF-16LE'e')]) + + + + + + + + + + + diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml index b0874d69e861c..ae25b30e84677 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleTest.xml @@ -111,8 +111,8 @@ LogicalProject(a=[$0], b=[$1], c=[$2], metadata=[$3], computed=[$4]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml index abaf03610ccc8..b8de2fc902e7c 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.xml @@ -332,7 +332,7 @@ LogicalProject(EXPR$0=[$2], EXPR$1=[$3], name=[$0], type=[$1]) Calc(select=[EXPR$0, EXPR$1, name, type]) +- HashAggregate(isMerge=[true], groupBy=[name, type], select=[name, type, Final_SUM(sum$0) AS EXPR$0, Final_MAX(max$1) AS EXPR$1]) +- Exchange(distribution=[hash[name, type]]) - +- TableSourceScan(table=[[default_catalog, default_database, inventory_meta, filter=[=(id, 123:BIGINT)], project=[name, type, amount], metadata=[metadata_1], aggregates=[grouping=[name,type], aggFunctions=[LongSumAggFunction(amount),LongMaxAggFunction($metadata$metadata_1)]]]], fields=[name, type, sum$0, max$1]) + +- TableSourceScan(table=[[default_catalog, default_database, inventory_meta, filter=[=(id, 123:BIGINT)], project=[name, type, amount], metadata=[metadata_1], aggregates=[grouping=[name,type], aggFunctions=[LongSumAggFunction(amount),LongMaxAggFunction(metadata_1)]]]], fields=[name, type, sum$0, max$1]) ]]> diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml index 88d4eba4d3909..b6fb8db11bd87 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/SourceWatermarkTest.xml @@ -144,7 +144,7 @@ LogicalProject(a=[$0], b=[$1]) @@ -182,7 +182,7 @@ LogicalProject(a=[$0], b=[$1]) diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml index 7c252f7ae3116..62c3334a959cc 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.xml @@ -192,8 +192,8 @@ LogicalProject(a=[$0], other_metadata=[CAST($4):INTEGER], b=[$1], c=[$2], metada @@ -209,8 +209,8 @@ LogicalProject(b=[$1], other_metadata=[CAST($4):INTEGER]) @@ -226,8 +226,8 @@ LogicalProject(timestamp=[$0], metadata_timestamp=[$2], other=[$1], computed_oth diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml index 41f48c1cc4e87..2b8b78770c8c2 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.xml @@ -255,6 +255,22 @@ Sink(table=[default_catalog.default_database.sink], fields=[id, city_name]) }]]> + + + + + + + + @@ -504,8 +520,8 @@ LogicalSink(table=[default_catalog.default_database.MetadataTable], fields=[meta @@ -763,20 +779,4 @@ Sink(table=[default_catalog.default_database.SinkJoinChangeLog], fields=[person, ]]> - - - - - - - - diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml index 648a40ad94e62..f8009ff219748 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/TableSourceTest.xml @@ -79,8 +79,8 @@ LogicalProject(id=[$0], nested1=[$1.nested1], results=[+(+($1.nested1.value, $1. diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala index 9f2e80372316e..e1459435b14b0 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala @@ -42,7 +42,6 @@ import org.apache.flink.types.Row import org.apache.calcite.plan.RelOptUtil import org.apache.calcite.sql.SqlExplainLevel import org.assertj.core.api.Assertions.{assertThat, assertThatThrownBy} - import org.junit.Assert.{assertEquals, assertFalse, assertTrue, fail} import org.junit.rules.ExpectedException import org.junit.{Rule, Test} diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala index c380c7ac9c50f..f4a4c72033f96 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala @@ -19,11 +19,15 @@ package org.apache.flink.table.planner.plan.stream.sql import org.apache.flink.api.scala._ +import org.apache.flink.core.testutils.FlinkAssertions +import org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches import org.apache.flink.table.api._ import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.planner.expressions.utils.Func0 import org.apache.flink.table.planner.factories.TestValuesTableFactory.MockedLookupTableSource import org.apache.flink.table.planner.utils.TableTestBase + +import org.assertj.core.api.Assertions.assertThatThrownBy import org.junit.Test class TableScanTest extends TableTestBase { @@ -177,6 +181,54 @@ class TableScanTest extends TableTestBase { util.verifyExecPlan("SELECT * FROM src WHERE a > 1") } + @Test + def testDDLWithMultipleColumnsFromSameMetadataKey(): Unit = { + assertThatThrownBy(() => + util.tableEnv.executeSql( + """ + |CREATE TABLE source ( + | a INT METADATA, + | b INT METADATA FROM 'a' + |) WITH ( + | 'connector' = 'COLLECTION' + |) + |""".stripMargin)).satisfies( + FlinkAssertions.anyCauseMatches( + classOf[ValidationException], + "The column `a` and `b` in the table are both from the same metadata key 'a'. " + + "Please specify one of the columns as the metadata column and use the computed column" + + " syntax to specify the others.")) + } + + @Test + def testDDLWithMultipleColumnsFromSameMetadataKey2(): Unit = { + util.tableEnv.executeSql( + """ + |CREATE TABLE source ( + | a INT METADATA + |) WITH ( + | 'connector' = 'COLLECTION' + |) + |""".stripMargin) + assertThatThrownBy(() => + util.tableEnv.executeSql( + """ + |CREATE TABLE like_source ( + | b INT METADATA FROM 'a' + |) + |WITH ( + | 'connector' = 'COLLECTION' + |) LIKE source ( + | INCLUDING METADATA + |) + |""".stripMargin + )).satisfies(anyCauseMatches( + "The column `a` and `b` in the table are both from the same metadata key 'a'. " + + "Please specify one of the columns as the metadata column and use the computed column" + + " syntax to specify the others." + )) + } + @Test def testKeywordsWithWatermarkComputedColumn(): Unit = { // Create table with field as atom expression. diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala index 7ff10b2ccf81c..465db9cbe83c7 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TableSourceITCase.scala @@ -76,6 +76,7 @@ class TableSourceITCase extends StreamingTestBase { |CREATE TABLE MetadataTable ( | `a` INT, | `other_metadata` INT METADATA FROM 'metadata_3', + | `other_metadata2` AS CAST(`other_metadata` AS BIGINT), | `b` BIGINT, | `metadata_1` INT METADATA, | `computed` AS `metadata_1` * 2, @@ -313,6 +314,22 @@ class TableSourceITCase extends StreamingTestBase { assertEquals(expected.sorted, sink.getAppendResults.sorted) } + @Test + def testDuplicateMetadataFromSameKey(): Unit = { + val result = tEnv.sqlQuery( + "SELECT other_metadata, other_metadata2, metadata_2 FROM MetadataTable") + .toAppendStream[Row] + val sink = new TestingAppendSink + result.addSink(sink) + env.execute() + + val expected = Seq( + "1,1,Hallo", + "1,1,Hallo Welt wie", + "2,2,Hallo Welt") + assertEquals(expected.sorted, sink.getAppendResults.sorted) + } + @Test def testNestedProjectionWithMetadataAccess(): Unit = { val query = From 229c5f053731f1af127971075ae8b052e6e48a83 Mon Sep 17 00:00:00 2001 From: David N Perkins Date: Thu, 14 Apr 2022 15:35:12 -0400 Subject: [PATCH 202/258] [FLINK-25694][Filesystem][S3] Upgrade Presto to resolve GSON/Alluxio Vulnerability. This closes #19479 Signed-off-by: David N Perkins --- flink-filesystems/flink-s3-fs-presto/pom.xml | 2 +- .../src/main/resources/META-INF/NOTICE | 11 ++++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/flink-filesystems/flink-s3-fs-presto/pom.xml b/flink-filesystems/flink-s3-fs-presto/pom.xml index 9a876559f65e7..618798c41c98f 100644 --- a/flink-filesystems/flink-s3-fs-presto/pom.xml +++ b/flink-filesystems/flink-s3-fs-presto/pom.xml @@ -33,7 +33,7 @@ under the License. jar - 0.257 + 0.272 diff --git a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE index 037a9f881e3ad..ebcbf9a40f326 100644 --- a/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE +++ b/flink-filesystems/flink-s3-fs-presto/src/main/resources/META-INF/NOTICE @@ -17,10 +17,10 @@ This project bundles the following dependencies under the Apache Software Licens - com.amazonaws:aws-java-sdk-s3:1.11.951 - com.amazonaws:aws-java-sdk-sts:1.11.951 - com.amazonaws:jmespath-java:1.11.951 -- com.facebook.presto:presto-common:0.257 -- com.facebook.presto:presto-hive:0.257 -- com.facebook.presto:presto-hive-common:0.257 -- com.facebook.presto:presto-hive-metastore:0.257 +- com.facebook.presto:presto-common:0.272 +- com.facebook.presto:presto-hive:0.272 +- com.facebook.presto:presto-hive-common:0.272 +- com.facebook.presto:presto-hive-metastore:0.272 - com.facebook.presto.hadoop:hadoop-apache2:2.7.4-9 - com.fasterxml.jackson.core:jackson-annotations:2.13.2 - com.fasterxml.jackson.core:jackson-core:2.13.2 @@ -35,7 +35,7 @@ This project bundles the following dependencies under the Apache Software Licens - io.airlift:units:1.3 - io.airlift:slice:0.38 - joda-time:joda-time:2.5 -- org.alluxio:alluxio-shaded-client:2.5.0-3 +- org.alluxio:alluxio-shaded-client:2.7.3 - org.apache.commons:commons-configuration2:2.1.1 - org.apache.commons:commons-lang3:3.3.2 - org.apache.commons:commons-text:1.4 @@ -46,6 +46,7 @@ This project bundles the following dependencies under the Apache Software Licens - org.apache.htrace:htrace-core4:4.1.0-incubating - org.apache.httpcomponents:httpclient:4.5.13 - org.apache.httpcomponents:httpcore:4.4.14 +- org.apache.hudi:hudi-presto-bundle:0.10.1 - org.weakref:jmxutils:1.19 - software.amazon.ion:ion-java:1.0.2 From bacce4e25ea8293005f189591f1a7393f47110f2 Mon Sep 17 00:00:00 2001 From: Dawid Wysakowicz Date: Wed, 20 Apr 2022 11:55:12 +0200 Subject: [PATCH 203/258] [FLINK-27319] Duplicated '-t' option for savepoint format and deployment target --- .../java/org/apache/flink/client/cli/CliFrontendParser.java | 2 +- .../org/apache/flink/client/cli/CliFrontendSavepointTest.java | 2 +- .../flink/client/cli/CliFrontendStopWithSavepointTest.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java index 8cc4ad4c61b0c..05ac0757c1b9c 100644 --- a/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java +++ b/flink-clients/src/main/java/org/apache/flink/client/cli/CliFrontendParser.java @@ -149,7 +149,7 @@ public class CliFrontendParser { static final Option SAVEPOINT_FORMAT_OPTION = new Option( - "t", + "type", "type", true, "Describes the binary format in which a savepoint should be taken. Supported" diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java index da4590efc2d55..2b4c6a79a1179 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendSavepointTest.java @@ -184,7 +184,7 @@ public void testTriggerSavepointCustomTarget() throws Exception { @Test public void testTriggerSavepointCustomFormatShortOption() throws Exception { - testTriggerSavepointCustomFormat("-t", SavepointFormatType.NATIVE); + testTriggerSavepointCustomFormat("-type", SavepointFormatType.NATIVE); } @Test diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java index da4d40727d161..89716ca19f983 100644 --- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java +++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendStopWithSavepointTest.java @@ -121,7 +121,7 @@ public void testStopWithExplicitSavepointDir() throws Exception { @Test public void testStopWithExplicitSavepointTypeShortOption() throws Exception { - testStopWithExplicitSavepointType("-t", SavepointFormatType.NATIVE); + testStopWithExplicitSavepointType("-type", SavepointFormatType.NATIVE); } @Test From 3a4c11371e6f2aacd641d86c1d5b4fd86435f802 Mon Sep 17 00:00:00 2001 From: Yun Gao Date: Thu, 21 Apr 2022 01:50:32 +0800 Subject: [PATCH 204/258] Commit for release 1.15.0 --- flink-annotations/pom.xml | 2 +- .../flink-architecture-tests-base/pom.xml | 2 +- .../flink-architecture-tests-production/pom.xml | 2 +- .../flink-architecture-tests-test/pom.xml | 2 +- flink-architecture-tests/pom.xml | 2 +- flink-clients/pom.xml | 2 +- flink-connectors/flink-connector-aws-base/pom.xml | 2 +- flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml | 2 +- flink-connectors/flink-connector-aws-kinesis-streams/pom.xml | 2 +- flink-connectors/flink-connector-base/pom.xml | 2 +- flink-connectors/flink-connector-cassandra/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch-base/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-connector-elasticsearch7/pom.xml | 2 +- flink-connectors/flink-connector-files/pom.xml | 2 +- flink-connectors/flink-connector-gcp-pubsub/pom.xml | 2 +- flink-connectors/flink-connector-hbase-1.4/pom.xml | 2 +- flink-connectors/flink-connector-hbase-2.2/pom.xml | 2 +- flink-connectors/flink-connector-hbase-base/pom.xml | 2 +- flink-connectors/flink-connector-hive/pom.xml | 2 +- flink-connectors/flink-connector-jdbc/pom.xml | 2 +- flink-connectors/flink-connector-kafka/pom.xml | 2 +- flink-connectors/flink-connector-kinesis/pom.xml | 2 +- flink-connectors/flink-connector-nifi/pom.xml | 2 +- flink-connectors/flink-connector-pulsar/pom.xml | 2 +- flink-connectors/flink-connector-rabbitmq/pom.xml | 2 +- flink-connectors/flink-file-sink-common/pom.xml | 2 +- flink-connectors/flink-hadoop-compatibility/pom.xml | 2 +- flink-connectors/flink-hcatalog/pom.xml | 2 +- .../flink-sql-connector-aws-kinesis-firehose/pom.xml | 2 +- .../flink-sql-connector-aws-kinesis-streams/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch6/pom.xml | 2 +- flink-connectors/flink-sql-connector-elasticsearch7/pom.xml | 2 +- flink-connectors/flink-sql-connector-hbase-1.4/pom.xml | 2 +- flink-connectors/flink-sql-connector-hbase-2.2/pom.xml | 2 +- flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml | 2 +- flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml | 2 +- flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml | 2 +- flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml | 2 +- flink-connectors/flink-sql-connector-kafka/pom.xml | 2 +- flink-connectors/flink-sql-connector-kinesis/pom.xml | 2 +- flink-connectors/flink-sql-connector-pulsar/pom.xml | 2 +- flink-connectors/flink-sql-connector-rabbitmq/pom.xml | 2 +- flink-connectors/pom.xml | 2 +- flink-container/pom.xml | 2 +- flink-contrib/flink-connector-wikiedits/pom.xml | 2 +- flink-contrib/pom.xml | 2 +- flink-core/pom.xml | 2 +- flink-dist-scala/pom.xml | 2 +- flink-dist/pom.xml | 2 +- flink-docs/pom.xml | 2 +- flink-dstl/flink-dstl-dfs/pom.xml | 2 +- flink-dstl/pom.xml | 2 +- flink-end-to-end-tests/flink-batch-sql-test/pom.xml | 2 +- flink-end-to-end-tests/flink-cli-test/pom.xml | 2 +- .../flink-confluent-schema-registry/pom.xml | 2 +- .../flink-connector-gcp-pubsub-emulator-tests/pom.xml | 2 +- flink-end-to-end-tests/flink-dataset-allround-test/pom.xml | 2 +- .../flink-dataset-fine-grained-recovery-test/pom.xml | 2 +- flink-end-to-end-tests/flink-datastream-allround-test/pom.xml | 2 +- .../flink-distributed-cache-via-blob-test/pom.xml | 2 +- flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml | 2 +- flink-end-to-end-tests/flink-elasticsearch7-test/pom.xml | 2 +- .../flink-end-to-end-tests-aws-kinesis-firehose/pom.xml | 2 +- .../flink-end-to-end-tests-aws-kinesis-streams/pom.xml | 2 +- .../flink-end-to-end-tests-common-kafka/pom.xml | 2 +- flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml | 2 +- flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml | 2 +- flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml | 2 +- flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml | 2 +- flink-end-to-end-tests/flink-file-sink-test/pom.xml | 2 +- .../flink-glue-schema-registry-avro-test/pom.xml | 2 +- .../flink-glue-schema-registry-json-test/pom.xml | 2 +- .../flink-heavy-deployment-stress-test/pom.xml | 2 +- .../flink-high-parallelism-iterations-test/pom.xml | 2 +- .../flink-local-recovery-and-allocation-test/pom.xml | 2 +- .../flink-metrics-availability-test/pom.xml | 2 +- .../flink-metrics-reporter-prometheus-test/pom.xml | 2 +- .../flink-netty-shuffle-memory-control-test/pom.xml | 2 +- .../flink-parent-child-classloading-test-lib-package/pom.xml | 2 +- .../flink-parent-child-classloading-test-program/pom.xml | 2 +- .../flink-plugins-test/another-dummy-fs/pom.xml | 2 +- flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml | 2 +- flink-end-to-end-tests/flink-plugins-test/pom.xml | 2 +- flink-end-to-end-tests/flink-python-test/pom.xml | 2 +- flink-end-to-end-tests/flink-queryable-state-test/pom.xml | 2 +- flink-end-to-end-tests/flink-quickstart-test/pom.xml | 2 +- .../flink-rocksdb-state-memory-control-test/pom.xml | 2 +- flink-end-to-end-tests/flink-sql-client-test/pom.xml | 2 +- flink-end-to-end-tests/flink-state-evolution-test/pom.xml | 2 +- flink-end-to-end-tests/flink-stream-sql-test/pom.xml | 2 +- flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml | 2 +- .../flink-stream-stateful-job-upgrade-test/pom.xml | 2 +- .../flink-streaming-kafka-test-base/pom.xml | 2 +- flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml | 2 +- flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml | 2 +- flink-end-to-end-tests/flink-tpcds-test/pom.xml | 2 +- flink-end-to-end-tests/flink-tpch-test/pom.xml | 2 +- flink-end-to-end-tests/pom.xml | 2 +- flink-examples/flink-examples-batch/pom.xml | 2 +- .../flink-examples-streaming-gcp-pubsub/pom.xml | 2 +- .../flink-examples-streaming-state-machine/pom.xml | 2 +- flink-examples/flink-examples-build-helper/pom.xml | 2 +- flink-examples/flink-examples-streaming/pom.xml | 2 +- flink-examples/flink-examples-table/pom.xml | 2 +- flink-examples/pom.xml | 2 +- flink-external-resources/flink-external-resource-gpu/pom.xml | 2 +- flink-external-resources/pom.xml | 2 +- flink-filesystems/flink-azure-fs-hadoop/pom.xml | 2 +- flink-filesystems/flink-fs-hadoop-shaded/pom.xml | 2 +- flink-filesystems/flink-gs-fs-hadoop/pom.xml | 2 +- flink-filesystems/flink-hadoop-fs/pom.xml | 2 +- flink-filesystems/flink-oss-fs-hadoop/pom.xml | 2 +- flink-filesystems/flink-s3-fs-base/pom.xml | 2 +- flink-filesystems/flink-s3-fs-hadoop/pom.xml | 2 +- flink-filesystems/flink-s3-fs-presto/pom.xml | 2 +- flink-filesystems/pom.xml | 2 +- flink-formats/flink-avro-confluent-registry/pom.xml | 2 +- flink-formats/flink-avro-glue-schema-registry/pom.xml | 2 +- flink-formats/flink-avro/pom.xml | 2 +- flink-formats/flink-compress/pom.xml | 2 +- flink-formats/flink-csv/pom.xml | 2 +- flink-formats/flink-format-common/pom.xml | 2 +- flink-formats/flink-hadoop-bulk/pom.xml | 2 +- flink-formats/flink-json-glue-schema-registry/pom.xml | 2 +- flink-formats/flink-json/pom.xml | 2 +- flink-formats/flink-orc-nohive/pom.xml | 2 +- flink-formats/flink-orc/pom.xml | 2 +- flink-formats/flink-parquet/pom.xml | 2 +- flink-formats/flink-sequence-file/pom.xml | 2 +- flink-formats/flink-sql-avro-confluent-registry/pom.xml | 2 +- flink-formats/flink-sql-avro/pom.xml | 2 +- flink-formats/flink-sql-orc/pom.xml | 2 +- flink-formats/flink-sql-parquet/pom.xml | 2 +- flink-formats/pom.xml | 2 +- flink-fs-tests/pom.xml | 2 +- flink-java/pom.xml | 2 +- flink-kubernetes/pom.xml | 2 +- flink-libraries/flink-cep-scala/pom.xml | 2 +- flink-libraries/flink-cep/pom.xml | 2 +- flink-libraries/flink-gelly-examples/pom.xml | 2 +- flink-libraries/flink-gelly-scala/pom.xml | 2 +- flink-libraries/flink-gelly/pom.xml | 2 +- flink-libraries/flink-state-processing-api/pom.xml | 2 +- flink-libraries/pom.xml | 2 +- flink-metrics/flink-metrics-core/pom.xml | 2 +- flink-metrics/flink-metrics-datadog/pom.xml | 2 +- flink-metrics/flink-metrics-dropwizard/pom.xml | 2 +- flink-metrics/flink-metrics-graphite/pom.xml | 2 +- flink-metrics/flink-metrics-influxdb/pom.xml | 2 +- flink-metrics/flink-metrics-jmx/pom.xml | 2 +- flink-metrics/flink-metrics-prometheus/pom.xml | 2 +- flink-metrics/flink-metrics-slf4j/pom.xml | 2 +- flink-metrics/flink-metrics-statsd/pom.xml | 2 +- flink-metrics/pom.xml | 2 +- flink-optimizer/pom.xml | 2 +- flink-python/pom.xml | 2 +- flink-python/pyflink/version.py | 2 +- .../flink-queryable-state-client-java/pom.xml | 2 +- flink-queryable-state/flink-queryable-state-runtime/pom.xml | 2 +- flink-queryable-state/pom.xml | 2 +- flink-quickstart/flink-quickstart-java/pom.xml | 2 +- flink-quickstart/flink-quickstart-scala/pom.xml | 2 +- flink-quickstart/pom.xml | 2 +- flink-rpc/flink-rpc-akka-loader/pom.xml | 2 +- flink-rpc/flink-rpc-akka/pom.xml | 2 +- flink-rpc/flink-rpc-core/pom.xml | 2 +- flink-rpc/pom.xml | 2 +- flink-runtime-web/pom.xml | 2 +- flink-runtime/pom.xml | 2 +- flink-scala/pom.xml | 2 +- flink-state-backends/flink-statebackend-changelog/pom.xml | 2 +- .../flink-statebackend-heap-spillable/pom.xml | 2 +- flink-state-backends/flink-statebackend-rocksdb/pom.xml | 2 +- flink-state-backends/pom.xml | 2 +- flink-streaming-java/pom.xml | 2 +- flink-streaming-scala/pom.xml | 2 +- flink-table/flink-sql-client/pom.xml | 2 +- flink-table/flink-sql-parser-hive/pom.xml | 2 +- flink-table/flink-sql-parser/pom.xml | 2 +- flink-table/flink-table-api-bridge-base/pom.xml | 2 +- flink-table/flink-table-api-java-bridge/pom.xml | 2 +- flink-table/flink-table-api-java-uber/pom.xml | 2 +- flink-table/flink-table-api-java/pom.xml | 2 +- flink-table/flink-table-api-scala-bridge/pom.xml | 2 +- flink-table/flink-table-api-scala/pom.xml | 2 +- flink-table/flink-table-code-splitter/pom.xml | 2 +- flink-table/flink-table-common/pom.xml | 2 +- flink-table/flink-table-planner-loader-bundle/pom.xml | 2 +- flink-table/flink-table-planner-loader/pom.xml | 2 +- flink-table/flink-table-planner/pom.xml | 2 +- flink-table/flink-table-runtime/pom.xml | 2 +- flink-table/flink-table-test-utils/pom.xml | 2 +- flink-table/pom.xml | 2 +- flink-test-utils-parent/flink-connector-test-utils/pom.xml | 2 +- flink-test-utils-parent/flink-test-utils-junit/pom.xml | 2 +- flink-test-utils-parent/flink-test-utils/pom.xml | 2 +- flink-test-utils-parent/pom.xml | 2 +- flink-tests/pom.xml | 2 +- flink-walkthroughs/flink-walkthrough-common/pom.xml | 2 +- flink-walkthroughs/flink-walkthrough-datastream-java/pom.xml | 2 +- flink-walkthroughs/flink-walkthrough-datastream-scala/pom.xml | 2 +- flink-walkthroughs/pom.xml | 2 +- flink-yarn-tests/pom.xml | 2 +- flink-yarn/pom.xml | 2 +- pom.xml | 2 +- tools/ci/java-ci-tools/pom.xml | 4 ++-- 207 files changed, 208 insertions(+), 208 deletions(-) diff --git a/flink-annotations/pom.xml b/flink-annotations/pom.xml index a2260ba99e534..b07f8069c5a28 100644 --- a/flink-annotations/pom.xml +++ b/flink-annotations/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-architecture-tests/flink-architecture-tests-base/pom.xml b/flink-architecture-tests/flink-architecture-tests-base/pom.xml index 53f61c8ec85d5..69aa346256e65 100644 --- a/flink-architecture-tests/flink-architecture-tests-base/pom.xml +++ b/flink-architecture-tests/flink-architecture-tests-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-architecture-tests - 1.15-SNAPSHOT + 1.15.0 ../pom.xml diff --git a/flink-architecture-tests/flink-architecture-tests-production/pom.xml b/flink-architecture-tests/flink-architecture-tests-production/pom.xml index 4752e9b6f872e..5b5964bbc8461 100644 --- a/flink-architecture-tests/flink-architecture-tests-production/pom.xml +++ b/flink-architecture-tests/flink-architecture-tests-production/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-architecture-tests - 1.15-SNAPSHOT + 1.15.0 ../pom.xml diff --git a/flink-architecture-tests/flink-architecture-tests-test/pom.xml b/flink-architecture-tests/flink-architecture-tests-test/pom.xml index a5e41eff5fe5b..ebc4a54363e5c 100644 --- a/flink-architecture-tests/flink-architecture-tests-test/pom.xml +++ b/flink-architecture-tests/flink-architecture-tests-test/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-architecture-tests - 1.15-SNAPSHOT + 1.15.0 ../pom.xml diff --git a/flink-architecture-tests/pom.xml b/flink-architecture-tests/pom.xml index 788b158493cfb..8218024729665 100644 --- a/flink-architecture-tests/pom.xml +++ b/flink-architecture-tests/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 ../pom.xml diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml index 72844be81429e..2410a6d0daece 100644 --- a/flink-clients/pom.xml +++ b/flink-clients/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-aws-base/pom.xml b/flink-connectors/flink-connector-aws-base/pom.xml index 431cbaca5e6d9..c99ba15fad934 100644 --- a/flink-connectors/flink-connector-aws-base/pom.xml +++ b/flink-connectors/flink-connector-aws-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml b/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml index 74639990bf8d1..d6b58a02aa055 100644 --- a/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml +++ b/flink-connectors/flink-connector-aws-kinesis-firehose/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml b/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml index 7c864efd5e5ab..7b502535a73b0 100644 --- a/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml +++ b/flink-connectors/flink-connector-aws-kinesis-streams/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-base/pom.xml b/flink-connectors/flink-connector-base/pom.xml index 4f8ff0eabd1a6..f9e3646cd19ce 100644 --- a/flink-connectors/flink-connector-base/pom.xml +++ b/flink-connectors/flink-connector-base/pom.xml @@ -25,7 +25,7 @@ org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-cassandra/pom.xml b/flink-connectors/flink-connector-cassandra/pom.xml index 3f4a8e6078a96..e44a12c0d0f3e 100644 --- a/flink-connectors/flink-connector-cassandra/pom.xml +++ b/flink-connectors/flink-connector-cassandra/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-elasticsearch-base/pom.xml b/flink-connectors/flink-connector-elasticsearch-base/pom.xml index cd2c6c5973e6b..744a8decc4a2e 100644 --- a/flink-connectors/flink-connector-elasticsearch-base/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch-base/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-elasticsearch6/pom.xml b/flink-connectors/flink-connector-elasticsearch6/pom.xml index 6a5b5e8657871..df955cb72552f 100644 --- a/flink-connectors/flink-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-elasticsearch7/pom.xml b/flink-connectors/flink-connector-elasticsearch7/pom.xml index 1c1a1951b3249..d5d751edcbdfe 100644 --- a/flink-connectors/flink-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-files/pom.xml b/flink-connectors/flink-connector-files/pom.xml index 2a0ed9d4ffcd9..ec887187430d4 100644 --- a/flink-connectors/flink-connector-files/pom.xml +++ b/flink-connectors/flink-connector-files/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-gcp-pubsub/pom.xml b/flink-connectors/flink-connector-gcp-pubsub/pom.xml index f5409ec7eeda5..eb321155423c5 100644 --- a/flink-connectors/flink-connector-gcp-pubsub/pom.xml +++ b/flink-connectors/flink-connector-gcp-pubsub/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-hbase-1.4/pom.xml b/flink-connectors/flink-connector-hbase-1.4/pom.xml index 37ed9fee59752..4465ef287204c 100644 --- a/flink-connectors/flink-connector-hbase-1.4/pom.xml +++ b/flink-connectors/flink-connector-hbase-1.4/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-hbase-2.2/pom.xml b/flink-connectors/flink-connector-hbase-2.2/pom.xml index 0da174f1b5102..39a5fb5f2d45a 100644 --- a/flink-connectors/flink-connector-hbase-2.2/pom.xml +++ b/flink-connectors/flink-connector-hbase-2.2/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-hbase-base/pom.xml b/flink-connectors/flink-connector-hbase-base/pom.xml index 1de021f8573c6..338352b39d6f7 100644 --- a/flink-connectors/flink-connector-hbase-base/pom.xml +++ b/flink-connectors/flink-connector-hbase-base/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml index 28db2b0cc31cc..a9cdbb33ab648 100644 --- a/flink-connectors/flink-connector-hive/pom.xml +++ b/flink-connectors/flink-connector-hive/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-jdbc/pom.xml b/flink-connectors/flink-connector-jdbc/pom.xml index aa3aa895c3f40..7a4116205fa2f 100644 --- a/flink-connectors/flink-connector-jdbc/pom.xml +++ b/flink-connectors/flink-connector-jdbc/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-kafka/pom.xml b/flink-connectors/flink-connector-kafka/pom.xml index 3cacec0b2ea27..3799ebff9f8c7 100644 --- a/flink-connectors/flink-connector-kafka/pom.xml +++ b/flink-connectors/flink-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-kinesis/pom.xml b/flink-connectors/flink-connector-kinesis/pom.xml index e9cb2a5441e5d..8206b4f2acbdc 100644 --- a/flink-connectors/flink-connector-kinesis/pom.xml +++ b/flink-connectors/flink-connector-kinesis/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-nifi/pom.xml b/flink-connectors/flink-connector-nifi/pom.xml index 56b2a686ebba3..2873fb25eb9a6 100644 --- a/flink-connectors/flink-connector-nifi/pom.xml +++ b/flink-connectors/flink-connector-nifi/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index 655c57fe18a81..7529e533c82df 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-connector-rabbitmq/pom.xml b/flink-connectors/flink-connector-rabbitmq/pom.xml index 1861c6e87cf90..a0c630934f722 100644 --- a/flink-connectors/flink-connector-rabbitmq/pom.xml +++ b/flink-connectors/flink-connector-rabbitmq/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-file-sink-common/pom.xml b/flink-connectors/flink-file-sink-common/pom.xml index 081e566ea6a67..e657308a2c963 100644 --- a/flink-connectors/flink-file-sink-common/pom.xml +++ b/flink-connectors/flink-file-sink-common/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-hadoop-compatibility/pom.xml b/flink-connectors/flink-hadoop-compatibility/pom.xml index 54c1b966faf9f..0ec0b56568fdf 100644 --- a/flink-connectors/flink-hadoop-compatibility/pom.xml +++ b/flink-connectors/flink-hadoop-compatibility/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-hcatalog/pom.xml b/flink-connectors/flink-hcatalog/pom.xml index 4351c85ed4a77..93b6b430bed3d 100644 --- a/flink-connectors/flink-hcatalog/pom.xml +++ b/flink-connectors/flink-hcatalog/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml b/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml index 8b4a2df0d4db2..7dbe7ecfdd520 100644 --- a/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml +++ b/flink-connectors/flink-sql-connector-aws-kinesis-firehose/pom.xml @@ -23,7 +23,7 @@ flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml b/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml index 0804a3a48f0be..13842b151fa17 100644 --- a/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml +++ b/flink-connectors/flink-sql-connector-aws-kinesis-streams/pom.xml @@ -23,7 +23,7 @@ flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml index 6aaaf4acba7be..7c17fa1bf3250 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml index 489e9ea385595..41cb27ba68393 100644 --- a/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml +++ b/flink-connectors/flink-sql-connector-elasticsearch7/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml b/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml index 78c1e5f32bbaa..98a05cbe23600 100644 --- a/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml +++ b/flink-connectors/flink-sql-connector-hbase-1.4/pom.xml @@ -23,7 +23,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml b/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml index 2b2873d4b0936..addc06aec314a 100644 --- a/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml +++ b/flink-connectors/flink-sql-connector-hbase-2.2/pom.xml @@ -23,7 +23,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml b/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml index fa5438ce05acc..23ec6063eb7db 100644 --- a/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-1.2.2/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml b/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml index cbcbc7a9551bb..c41328f45feb6 100644 --- a/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-2.2.0/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml b/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml index 9a4d9b6e1a9f0..81dad17c7b14d 100644 --- a/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-2.3.6/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml b/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml index 907e747d626e2..c65aa9573e060 100644 --- a/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml +++ b/flink-connectors/flink-sql-connector-hive-3.1.2/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-connectors - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-kafka/pom.xml b/flink-connectors/flink-sql-connector-kafka/pom.xml index ad63e6c1f6680..a331e6c52091d 100644 --- a/flink-connectors/flink-sql-connector-kafka/pom.xml +++ b/flink-connectors/flink-sql-connector-kafka/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-kinesis/pom.xml b/flink-connectors/flink-sql-connector-kinesis/pom.xml index 1eb1694577f3b..03c0c439c6951 100644 --- a/flink-connectors/flink-sql-connector-kinesis/pom.xml +++ b/flink-connectors/flink-sql-connector-kinesis/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-pulsar/pom.xml b/flink-connectors/flink-sql-connector-pulsar/pom.xml index 8011f82b4988b..7324bc33c2e5a 100644 --- a/flink-connectors/flink-sql-connector-pulsar/pom.xml +++ b/flink-connectors/flink-sql-connector-pulsar/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/flink-sql-connector-rabbitmq/pom.xml b/flink-connectors/flink-sql-connector-rabbitmq/pom.xml index c521fafb323c0..503ad90c06f5f 100644 --- a/flink-connectors/flink-sql-connector-rabbitmq/pom.xml +++ b/flink-connectors/flink-sql-connector-rabbitmq/pom.xml @@ -26,7 +26,7 @@ under the License. flink-connectors org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-connectors/pom.xml b/flink-connectors/pom.xml index a1229f38cc1b1..6d9aca55608ee 100644 --- a/flink-connectors/pom.xml +++ b/flink-connectors/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-container/pom.xml b/flink-container/pom.xml index 6c1e1c9e491ea..4f3b534925ff5 100644 --- a/flink-container/pom.xml +++ b/flink-container/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-contrib/flink-connector-wikiedits/pom.xml b/flink-contrib/flink-connector-wikiedits/pom.xml index efba541cb0506..288497de8297e 100644 --- a/flink-contrib/flink-connector-wikiedits/pom.xml +++ b/flink-contrib/flink-connector-wikiedits/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-contrib - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-contrib/pom.xml b/flink-contrib/pom.xml index 6f8c92557aaaf..ace198c4e1f2d 100644 --- a/flink-contrib/pom.xml +++ b/flink-contrib/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-core/pom.xml b/flink-core/pom.xml index 8d255adb479a8..78fd16c57d0a0 100644 --- a/flink-core/pom.xml +++ b/flink-core/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-dist-scala/pom.xml b/flink-dist-scala/pom.xml index b571e1f94aba9..b86f2b68983af 100644 --- a/flink-dist-scala/pom.xml +++ b/flink-dist-scala/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml index 22bbb0967d5cd..57e02b6a7683a 100644 --- a/flink-dist/pom.xml +++ b/flink-dist/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-docs/pom.xml b/flink-docs/pom.xml index c9450f3560c8b..b16043140710e 100644 --- a/flink-docs/pom.xml +++ b/flink-docs/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-dstl/flink-dstl-dfs/pom.xml b/flink-dstl/flink-dstl-dfs/pom.xml index 6f059e1a0263b..083d6ceb90b1a 100644 --- a/flink-dstl/flink-dstl-dfs/pom.xml +++ b/flink-dstl/flink-dstl-dfs/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-dstl - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-dstl/pom.xml b/flink-dstl/pom.xml index dd0ff26fbd21d..52fbb16382bc3 100644 --- a/flink-dstl/pom.xml +++ b/flink-dstl/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-batch-sql-test/pom.xml b/flink-end-to-end-tests/flink-batch-sql-test/pom.xml index f38bd2ac8b852..5ed81b3d0c5ff 100644 --- a/flink-end-to-end-tests/flink-batch-sql-test/pom.xml +++ b/flink-end-to-end-tests/flink-batch-sql-test/pom.xml @@ -24,7 +24,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-cli-test/pom.xml b/flink-end-to-end-tests/flink-cli-test/pom.xml index 6d046e66e3c6a..33cfe71ffe362 100644 --- a/flink-end-to-end-tests/flink-cli-test/pom.xml +++ b/flink-end-to-end-tests/flink-cli-test/pom.xml @@ -24,7 +24,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml b/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml index 55fbbf2bb45e4..eed32bfb5a361 100644 --- a/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml +++ b/flink-end-to-end-tests/flink-confluent-schema-registry/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml index 3368a307cf241..6c1303956615e 100644 --- a/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml +++ b/flink-end-to-end-tests/flink-connector-gcp-pubsub-emulator-tests/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml b/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml index 44f5e7372d7e7..e47e3361e3c05 100644 --- a/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml +++ b/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml b/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml index 1e33f3084e471..5e1aa4a2671f7 100644 --- a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml +++ b/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/pom.xml b/flink-end-to-end-tests/flink-datastream-allround-test/pom.xml index 4b400806be58d..537b9b9437d62 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/pom.xml +++ b/flink-end-to-end-tests/flink-datastream-allround-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml b/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml index 21bdec7ca363b..57a370fe4457f 100644 --- a/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml +++ b/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml @@ -24,7 +24,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml b/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml index ee1aecaf70a37..4d42aea40ef8a 100644 --- a/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml +++ b/flink-end-to-end-tests/flink-elasticsearch6-test/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-elasticsearch7-test/pom.xml b/flink-end-to-end-tests/flink-elasticsearch7-test/pom.xml index f210e3b0b7374..f7d09566e7e99 100644 --- a/flink-end-to-end-tests/flink-elasticsearch7-test/pom.xml +++ b/flink-end-to-end-tests/flink-elasticsearch7-test/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml index 31cc067fe431e..d1a54d53ccbe1 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-firehose/pom.xml @@ -23,7 +23,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml index 423afd529b88f..a7b1d9594f0cc 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-aws-kinesis-streams/pom.xml @@ -23,7 +23,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml index fce2debda9029..6296e64556529 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml index b2e6ee57e9d1a..90dc526ec60ec 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-common/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml index 6b76dddfffbd5..5bc2de72b4cf6 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-hbase/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml index 424b106f4d3e9..f76011f4c6b9f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml index 0a514c8165e3e..beed64a76581e 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml @@ -21,7 +21,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-file-sink-test/pom.xml b/flink-end-to-end-tests/flink-file-sink-test/pom.xml index 740d57e9bdb88..0a06535215ca9 100644 --- a/flink-end-to-end-tests/flink-file-sink-test/pom.xml +++ b/flink-end-to-end-tests/flink-file-sink-test/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-glue-schema-registry-avro-test/pom.xml b/flink-end-to-end-tests/flink-glue-schema-registry-avro-test/pom.xml index 96316403fdcea..c747c12d7509e 100644 --- a/flink-end-to-end-tests/flink-glue-schema-registry-avro-test/pom.xml +++ b/flink-end-to-end-tests/flink-glue-schema-registry-avro-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-glue-schema-registry-json-test/pom.xml b/flink-end-to-end-tests/flink-glue-schema-registry-json-test/pom.xml index 19121084be082..1633406271e37 100644 --- a/flink-end-to-end-tests/flink-glue-schema-registry-json-test/pom.xml +++ b/flink-end-to-end-tests/flink-glue-schema-registry-json-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml index a96caf2c66cf6..77c2c7c46800d 100644 --- a/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml +++ b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-high-parallelism-iterations-test/pom.xml b/flink-end-to-end-tests/flink-high-parallelism-iterations-test/pom.xml index f49e254609c1c..fbc2caefd23ed 100644 --- a/flink-end-to-end-tests/flink-high-parallelism-iterations-test/pom.xml +++ b/flink-end-to-end-tests/flink-high-parallelism-iterations-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/pom.xml b/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/pom.xml index 75bc71ec21168..592ed4a90a073 100644 --- a/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/pom.xml +++ b/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml b/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml index e292694be3aba..a1819998053de 100644 --- a/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml +++ b/flink-end-to-end-tests/flink-metrics-availability-test/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml index a28039b5a0468..f292ff3702366 100644 --- a/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml +++ b/flink-end-to-end-tests/flink-metrics-reporter-prometheus-test/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml b/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml index 43e4737888259..752bc2dae7f0e 100644 --- a/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml +++ b/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-parent-child-classloading-test-lib-package/pom.xml b/flink-end-to-end-tests/flink-parent-child-classloading-test-lib-package/pom.xml index 2627dacd5a4c8..7198b00d29a10 100644 --- a/flink-end-to-end-tests/flink-parent-child-classloading-test-lib-package/pom.xml +++ b/flink-end-to-end-tests/flink-parent-child-classloading-test-lib-package/pom.xml @@ -29,7 +29,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-parent-child-classloading-test-program/pom.xml b/flink-end-to-end-tests/flink-parent-child-classloading-test-program/pom.xml index fc7f2f5bacce1..c045f4284e0d3 100644 --- a/flink-end-to-end-tests/flink-parent-child-classloading-test-program/pom.xml +++ b/flink-end-to-end-tests/flink-parent-child-classloading-test-program/pom.xml @@ -29,7 +29,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml index ecdc27ea3e287..a8bbce94e615c 100644 --- a/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml +++ b/flink-end-to-end-tests/flink-plugins-test/another-dummy-fs/pom.xml @@ -23,7 +23,7 @@ under the License. flink-plugins-test org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml index b7661dd6f67d9..93e34f2fa7d9b 100644 --- a/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml +++ b/flink-end-to-end-tests/flink-plugins-test/dummy-fs/pom.xml @@ -23,7 +23,7 @@ under the License. flink-plugins-test org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-plugins-test/pom.xml b/flink-end-to-end-tests/flink-plugins-test/pom.xml index fc7c6a80a0e89..abc2a433ed392 100644 --- a/flink-end-to-end-tests/flink-plugins-test/pom.xml +++ b/flink-end-to-end-tests/flink-plugins-test/pom.xml @@ -25,7 +25,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 pom diff --git a/flink-end-to-end-tests/flink-python-test/pom.xml b/flink-end-to-end-tests/flink-python-test/pom.xml index 432ea5eb15979..67a883dc54221 100644 --- a/flink-end-to-end-tests/flink-python-test/pom.xml +++ b/flink-end-to-end-tests/flink-python-test/pom.xml @@ -23,7 +23,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-queryable-state-test/pom.xml b/flink-end-to-end-tests/flink-queryable-state-test/pom.xml index 8eac23f213f5d..26c2988d43cf4 100644 --- a/flink-end-to-end-tests/flink-queryable-state-test/pom.xml +++ b/flink-end-to-end-tests/flink-queryable-state-test/pom.xml @@ -23,7 +23,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-quickstart-test/pom.xml b/flink-end-to-end-tests/flink-quickstart-test/pom.xml index 634c1851fffc7..2c91ff5f1dc25 100644 --- a/flink-end-to-end-tests/flink-quickstart-test/pom.xml +++ b/flink-end-to-end-tests/flink-quickstart-test/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/pom.xml b/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/pom.xml index b96df95c0f082..6425db3197ccd 100644 --- a/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/pom.xml +++ b/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-sql-client-test/pom.xml b/flink-end-to-end-tests/flink-sql-client-test/pom.xml index 8a6233c0108cb..77e4745d423e0 100644 --- a/flink-end-to-end-tests/flink-sql-client-test/pom.xml +++ b/flink-end-to-end-tests/flink-sql-client-test/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-state-evolution-test/pom.xml b/flink-end-to-end-tests/flink-state-evolution-test/pom.xml index 0e575997a2091..8e8f482cd6ca8 100644 --- a/flink-end-to-end-tests/flink-state-evolution-test/pom.xml +++ b/flink-end-to-end-tests/flink-state-evolution-test/pom.xml @@ -22,7 +22,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-stream-sql-test/pom.xml b/flink-end-to-end-tests/flink-stream-sql-test/pom.xml index a34ac2298cbbb..cf4612154c66f 100644 --- a/flink-end-to-end-tests/flink-stream-sql-test/pom.xml +++ b/flink-end-to-end-tests/flink-stream-sql-test/pom.xml @@ -24,7 +24,7 @@ flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml b/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml index 516867f0b054d..09ab468d82ca5 100644 --- a/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml +++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-end-to-end-tests - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/pom.xml b/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/pom.xml index 3eda420e72e58..c587270692b42 100644 --- a/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/pom.xml +++ b/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml index 799450d0ff147..0701c02a2743b 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka-test-base/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml index d94999415770e..1b0731c90edc1 100644 --- a/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kafka-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml b/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml index fafa40790462e..e3e4754d9c9f3 100644 --- a/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml +++ b/flink-end-to-end-tests/flink-streaming-kinesis-test/pom.xml @@ -23,7 +23,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-tpcds-test/pom.xml b/flink-end-to-end-tests/flink-tpcds-test/pom.xml index dbcfb2ecb0f3a..773e3694ae6e6 100644 --- a/flink-end-to-end-tests/flink-tpcds-test/pom.xml +++ b/flink-end-to-end-tests/flink-tpcds-test/pom.xml @@ -21,7 +21,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/flink-tpch-test/pom.xml b/flink-end-to-end-tests/flink-tpch-test/pom.xml index b534bded19f5a..a747749155966 100644 --- a/flink-end-to-end-tests/flink-tpch-test/pom.xml +++ b/flink-end-to-end-tests/flink-tpch-test/pom.xml @@ -21,7 +21,7 @@ under the License. flink-end-to-end-tests org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml index 6a28b30d7240f..52cad76871dc4 100644 --- a/flink-end-to-end-tests/pom.xml +++ b/flink-end-to-end-tests/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-batch/pom.xml b/flink-examples/flink-examples-batch/pom.xml index 6ead88d94d278..cdc3c8c5955f7 100644 --- a/flink-examples/flink-examples-batch/pom.xml +++ b/flink-examples/flink-examples-batch/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-examples - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml index cf5824c62e94d..cb4e8846b5537 100644 --- a/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml +++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-gcp-pubsub/pom.xml @@ -24,7 +24,7 @@ under the License. flink-examples-build-helper org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml index bbf565be029bb..5c49245e4c9d5 100644 --- a/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml +++ b/flink-examples/flink-examples-build-helper/flink-examples-streaming-state-machine/pom.xml @@ -24,7 +24,7 @@ under the License. flink-examples-build-helper org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-build-helper/pom.xml b/flink-examples/flink-examples-build-helper/pom.xml index 5ade20d32bb6a..cb73913447d00 100644 --- a/flink-examples/flink-examples-build-helper/pom.xml +++ b/flink-examples/flink-examples-build-helper/pom.xml @@ -24,7 +24,7 @@ under the License. flink-examples org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-streaming/pom.xml b/flink-examples/flink-examples-streaming/pom.xml index 744463d422f7c..24ac544ca317d 100644 --- a/flink-examples/flink-examples-streaming/pom.xml +++ b/flink-examples/flink-examples-streaming/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-examples - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/flink-examples-table/pom.xml b/flink-examples/flink-examples-table/pom.xml index 1c09ec52ad1b4..d874aff840405 100644 --- a/flink-examples/flink-examples-table/pom.xml +++ b/flink-examples/flink-examples-table/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-examples - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-examples/pom.xml b/flink-examples/pom.xml index 8c89630c107fc..ab51ec0a87d39 100644 --- a/flink-examples/pom.xml +++ b/flink-examples/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-external-resources/flink-external-resource-gpu/pom.xml b/flink-external-resources/flink-external-resource-gpu/pom.xml index 9416ec12e76f5..e045b7227e485 100644 --- a/flink-external-resources/flink-external-resource-gpu/pom.xml +++ b/flink-external-resources/flink-external-resource-gpu/pom.xml @@ -25,7 +25,7 @@ under the License. flink-external-resources org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-external-resources/pom.xml b/flink-external-resources/pom.xml index 4a98561c9847b..99520497dc8f3 100644 --- a/flink-external-resources/pom.xml +++ b/flink-external-resources/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-azure-fs-hadoop/pom.xml b/flink-filesystems/flink-azure-fs-hadoop/pom.xml index edcd5d2aabd3d..78ab0898d3b80 100644 --- a/flink-filesystems/flink-azure-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-azure-fs-hadoop/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-fs-hadoop-shaded/pom.xml b/flink-filesystems/flink-fs-hadoop-shaded/pom.xml index 8f8a70c98817a..6d36614adb3c5 100644 --- a/flink-filesystems/flink-fs-hadoop-shaded/pom.xml +++ b/flink-filesystems/flink-fs-hadoop-shaded/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-gs-fs-hadoop/pom.xml b/flink-filesystems/flink-gs-fs-hadoop/pom.xml index ce00f7a9864a2..e443d1532e89d 100644 --- a/flink-filesystems/flink-gs-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-gs-fs-hadoop/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-hadoop-fs/pom.xml b/flink-filesystems/flink-hadoop-fs/pom.xml index c9a73dd6376c0..64cab658112f3 100644 --- a/flink-filesystems/flink-hadoop-fs/pom.xml +++ b/flink-filesystems/flink-hadoop-fs/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-oss-fs-hadoop/pom.xml b/flink-filesystems/flink-oss-fs-hadoop/pom.xml index fba791dd2ee04..bbf95f95e92c0 100644 --- a/flink-filesystems/flink-oss-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-oss-fs-hadoop/pom.xml @@ -21,7 +21,7 @@ under the License. flink-filesystems org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-filesystems/flink-s3-fs-base/pom.xml b/flink-filesystems/flink-s3-fs-base/pom.xml index 9ccf84ed08542..d3f3941f4b040 100644 --- a/flink-filesystems/flink-s3-fs-base/pom.xml +++ b/flink-filesystems/flink-s3-fs-base/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-s3-fs-hadoop/pom.xml b/flink-filesystems/flink-s3-fs-hadoop/pom.xml index 6c3799d8fadde..0dd5e3a566e0c 100644 --- a/flink-filesystems/flink-s3-fs-hadoop/pom.xml +++ b/flink-filesystems/flink-s3-fs-hadoop/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/flink-s3-fs-presto/pom.xml b/flink-filesystems/flink-s3-fs-presto/pom.xml index 618798c41c98f..cdcefeb7cf29e 100644 --- a/flink-filesystems/flink-s3-fs-presto/pom.xml +++ b/flink-filesystems/flink-s3-fs-presto/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-filesystems - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-filesystems/pom.xml b/flink-filesystems/pom.xml index fdd61ab9c43ba..37ecbe0aa5ffb 100644 --- a/flink-filesystems/pom.xml +++ b/flink-filesystems/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-avro-confluent-registry/pom.xml b/flink-formats/flink-avro-confluent-registry/pom.xml index 4b791287a44c1..d564d2a29c2c5 100644 --- a/flink-formats/flink-avro-confluent-registry/pom.xml +++ b/flink-formats/flink-avro-confluent-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-formats/flink-avro-glue-schema-registry/pom.xml b/flink-formats/flink-avro-glue-schema-registry/pom.xml index 41e59016cf1e3..5ba84a7cc0e6f 100644 --- a/flink-formats/flink-avro-glue-schema-registry/pom.xml +++ b/flink-formats/flink-avro-glue-schema-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-formats/flink-avro/pom.xml b/flink-formats/flink-avro/pom.xml index 1f60cee550ce2..c8ea0bb090328 100644 --- a/flink-formats/flink-avro/pom.xml +++ b/flink-formats/flink-avro/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-compress/pom.xml b/flink-formats/flink-compress/pom.xml index dae4a8b387fca..e1c1f0f80c5a7 100644 --- a/flink-formats/flink-compress/pom.xml +++ b/flink-formats/flink-compress/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-csv/pom.xml b/flink-formats/flink-csv/pom.xml index d2fbd4743384a..b46613fa44079 100644 --- a/flink-formats/flink-csv/pom.xml +++ b/flink-formats/flink-csv/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-format-common/pom.xml b/flink-formats/flink-format-common/pom.xml index ca31700a9e988..d6a31a490f471 100644 --- a/flink-formats/flink-format-common/pom.xml +++ b/flink-formats/flink-format-common/pom.xml @@ -22,7 +22,7 @@ under the License. flink-formats org.apache.flink - 1.15-SNAPSHOT + 1.15.0 4.0.0 diff --git a/flink-formats/flink-hadoop-bulk/pom.xml b/flink-formats/flink-hadoop-bulk/pom.xml index 60abe050c21aa..cea562ddf5a11 100644 --- a/flink-formats/flink-hadoop-bulk/pom.xml +++ b/flink-formats/flink-hadoop-bulk/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-json-glue-schema-registry/pom.xml b/flink-formats/flink-json-glue-schema-registry/pom.xml index 3982ae53d2dbf..c0755cf391179 100644 --- a/flink-formats/flink-json-glue-schema-registry/pom.xml +++ b/flink-formats/flink-json-glue-schema-registry/pom.xml @@ -23,7 +23,7 @@ under the License. flink-formats org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-formats/flink-json/pom.xml b/flink-formats/flink-json/pom.xml index a04294917a1d8..c6c5cd11850a1 100644 --- a/flink-formats/flink-json/pom.xml +++ b/flink-formats/flink-json/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-orc-nohive/pom.xml b/flink-formats/flink-orc-nohive/pom.xml index a619d610ddf8e..34431207ea5d2 100644 --- a/flink-formats/flink-orc-nohive/pom.xml +++ b/flink-formats/flink-orc-nohive/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-orc/pom.xml b/flink-formats/flink-orc/pom.xml index fad8c272dba3f..bbaf5ff01c406 100644 --- a/flink-formats/flink-orc/pom.xml +++ b/flink-formats/flink-orc/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-parquet/pom.xml b/flink-formats/flink-parquet/pom.xml index f5e340a9ac008..47662b1df23d2 100644 --- a/flink-formats/flink-parquet/pom.xml +++ b/flink-formats/flink-parquet/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-sequence-file/pom.xml b/flink-formats/flink-sequence-file/pom.xml index b4391c500be79..ba7363db1b778 100644 --- a/flink-formats/flink-sequence-file/pom.xml +++ b/flink-formats/flink-sequence-file/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-sql-avro-confluent-registry/pom.xml b/flink-formats/flink-sql-avro-confluent-registry/pom.xml index b00aa61a8a527..c14ae92c11b6c 100644 --- a/flink-formats/flink-sql-avro-confluent-registry/pom.xml +++ b/flink-formats/flink-sql-avro-confluent-registry/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-sql-avro/pom.xml b/flink-formats/flink-sql-avro/pom.xml index fb2cdabe01fbd..e808b45468db8 100644 --- a/flink-formats/flink-sql-avro/pom.xml +++ b/flink-formats/flink-sql-avro/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-sql-orc/pom.xml b/flink-formats/flink-sql-orc/pom.xml index dd26c3a5c9f5a..449b53bffe490 100644 --- a/flink-formats/flink-sql-orc/pom.xml +++ b/flink-formats/flink-sql-orc/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/flink-sql-parquet/pom.xml b/flink-formats/flink-sql-parquet/pom.xml index f857017d4bcc6..1b6bd8a1005c4 100644 --- a/flink-formats/flink-sql-parquet/pom.xml +++ b/flink-formats/flink-sql-parquet/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-formats - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-formats/pom.xml b/flink-formats/pom.xml index 44272566db40a..2e6be0f48015c 100644 --- a/flink-formats/pom.xml +++ b/flink-formats/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-fs-tests/pom.xml b/flink-fs-tests/pom.xml index 4ba5dc774b689..bee60d5d9d3fd 100644 --- a/flink-fs-tests/pom.xml +++ b/flink-fs-tests/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-java/pom.xml b/flink-java/pom.xml index ff789cc8593f6..9af518436334b 100644 --- a/flink-java/pom.xml +++ b/flink-java/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-kubernetes/pom.xml b/flink-kubernetes/pom.xml index 9173ebad36ff5..3f16f9724fd45 100644 --- a/flink-kubernetes/pom.xml +++ b/flink-kubernetes/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/flink-cep-scala/pom.xml b/flink-libraries/flink-cep-scala/pom.xml index ac0dae0f6d933..1258dd8c9a925 100644 --- a/flink-libraries/flink-cep-scala/pom.xml +++ b/flink-libraries/flink-cep-scala/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/flink-cep/pom.xml b/flink-libraries/flink-cep/pom.xml index 5dcdaab52afac..9613479149359 100644 --- a/flink-libraries/flink-cep/pom.xml +++ b/flink-libraries/flink-cep/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/flink-gelly-examples/pom.xml b/flink-libraries/flink-gelly-examples/pom.xml index d53855f70935e..193bef3135698 100644 --- a/flink-libraries/flink-gelly-examples/pom.xml +++ b/flink-libraries/flink-gelly-examples/pom.xml @@ -23,7 +23,7 @@ org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/flink-gelly-scala/pom.xml b/flink-libraries/flink-gelly-scala/pom.xml index b99752fd686c0..9506fd3ef826f 100644 --- a/flink-libraries/flink-gelly-scala/pom.xml +++ b/flink-libraries/flink-gelly-scala/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. 4.0.0 diff --git a/flink-libraries/flink-gelly/pom.xml b/flink-libraries/flink-gelly/pom.xml index c1d8248db3594..f71ad5968a697 100644 --- a/flink-libraries/flink-gelly/pom.xml +++ b/flink-libraries/flink-gelly/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/flink-state-processing-api/pom.xml b/flink-libraries/flink-state-processing-api/pom.xml index 3d13cb304c4de..c131b1622c6f8 100644 --- a/flink-libraries/flink-state-processing-api/pom.xml +++ b/flink-libraries/flink-state-processing-api/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-libraries - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-libraries/pom.xml b/flink-libraries/pom.xml index e746ab834a050..58c93b30db2a7 100644 --- a/flink-libraries/pom.xml +++ b/flink-libraries/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-core/pom.xml b/flink-metrics/flink-metrics-core/pom.xml index 8895ef3629092..ba2cbf0382db5 100644 --- a/flink-metrics/flink-metrics-core/pom.xml +++ b/flink-metrics/flink-metrics-core/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-datadog/pom.xml b/flink-metrics/flink-metrics-datadog/pom.xml index 0707ff0527a2f..ea0d877cc79ea 100644 --- a/flink-metrics/flink-metrics-datadog/pom.xml +++ b/flink-metrics/flink-metrics-datadog/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-dropwizard/pom.xml b/flink-metrics/flink-metrics-dropwizard/pom.xml index 4009c866704ce..85a5033f743c2 100644 --- a/flink-metrics/flink-metrics-dropwizard/pom.xml +++ b/flink-metrics/flink-metrics-dropwizard/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-graphite/pom.xml b/flink-metrics/flink-metrics-graphite/pom.xml index 509b1b7efff8f..f6b4cad50b866 100644 --- a/flink-metrics/flink-metrics-graphite/pom.xml +++ b/flink-metrics/flink-metrics-graphite/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-influxdb/pom.xml b/flink-metrics/flink-metrics-influxdb/pom.xml index 27d5a6e9d2e89..790e3b2a773de 100644 --- a/flink-metrics/flink-metrics-influxdb/pom.xml +++ b/flink-metrics/flink-metrics-influxdb/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-jmx/pom.xml b/flink-metrics/flink-metrics-jmx/pom.xml index af846b70e53c1..1734e440869d6 100644 --- a/flink-metrics/flink-metrics-jmx/pom.xml +++ b/flink-metrics/flink-metrics-jmx/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-prometheus/pom.xml b/flink-metrics/flink-metrics-prometheus/pom.xml index 7f2b12dabe597..d14bd3adf2b1a 100644 --- a/flink-metrics/flink-metrics-prometheus/pom.xml +++ b/flink-metrics/flink-metrics-prometheus/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-slf4j/pom.xml b/flink-metrics/flink-metrics-slf4j/pom.xml index 53654fdc83c43..0731dae652e75 100644 --- a/flink-metrics/flink-metrics-slf4j/pom.xml +++ b/flink-metrics/flink-metrics-slf4j/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/flink-metrics-statsd/pom.xml b/flink-metrics/flink-metrics-statsd/pom.xml index 399e22cdbf2ed..d67499f01a2d6 100644 --- a/flink-metrics/flink-metrics-statsd/pom.xml +++ b/flink-metrics/flink-metrics-statsd/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-metrics - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-metrics/pom.xml b/flink-metrics/pom.xml index 9d5e69c94a442..3469289c6a671 100644 --- a/flink-metrics/pom.xml +++ b/flink-metrics/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-optimizer/pom.xml b/flink-optimizer/pom.xml index 18cf197c5c1f5..6b2f7150d2951 100644 --- a/flink-optimizer/pom.xml +++ b/flink-optimizer/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-python/pom.xml b/flink-python/pom.xml index 43b00cc6e1dd3..77e2a87afff94 100644 --- a/flink-python/pom.xml +++ b/flink-python/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-python/pyflink/version.py b/flink-python/pyflink/version.py index c29e6a8a03be9..054b900b4ca4e 100644 --- a/flink-python/pyflink/version.py +++ b/flink-python/pyflink/version.py @@ -20,4 +20,4 @@ The pyflink version will be consistent with the flink version and follow the PEP440. .. seealso:: https://www.python.org/dev/peps/pep-0440 """ -__version__ = "1.15.dev0" +__version__ = "1.15.0" diff --git a/flink-queryable-state/flink-queryable-state-client-java/pom.xml b/flink-queryable-state/flink-queryable-state-client-java/pom.xml index 8566c24172311..be384d304e933 100644 --- a/flink-queryable-state/flink-queryable-state-client-java/pom.xml +++ b/flink-queryable-state/flink-queryable-state-client-java/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-queryable-state - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-queryable-state/flink-queryable-state-runtime/pom.xml b/flink-queryable-state/flink-queryable-state-runtime/pom.xml index 75de79c11c6bb..8cc2eb9cbb3d4 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/pom.xml +++ b/flink-queryable-state/flink-queryable-state-runtime/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-queryable-state - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-queryable-state/pom.xml b/flink-queryable-state/pom.xml index f0f85230da932..9ba1e593dceee 100644 --- a/flink-queryable-state/pom.xml +++ b/flink-queryable-state/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-quickstart/flink-quickstart-java/pom.xml b/flink-quickstart/flink-quickstart-java/pom.xml index c532434215a47..0dbc7dd8012fd 100644 --- a/flink-quickstart/flink-quickstart-java/pom.xml +++ b/flink-quickstart/flink-quickstart-java/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-quickstart - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-quickstart/flink-quickstart-scala/pom.xml b/flink-quickstart/flink-quickstart-scala/pom.xml index a0feb5e51c7fa..573a729e35537 100644 --- a/flink-quickstart/flink-quickstart-scala/pom.xml +++ b/flink-quickstart/flink-quickstart-scala/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-quickstart - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-quickstart/pom.xml b/flink-quickstart/pom.xml index c6010915ab182..ef0fac1251293 100644 --- a/flink-quickstart/pom.xml +++ b/flink-quickstart/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-rpc/flink-rpc-akka-loader/pom.xml b/flink-rpc/flink-rpc-akka-loader/pom.xml index 1b0dedaf1af07..e4fd77f82b2ba 100644 --- a/flink-rpc/flink-rpc-akka-loader/pom.xml +++ b/flink-rpc/flink-rpc-akka-loader/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-rpc - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-rpc/flink-rpc-akka/pom.xml b/flink-rpc/flink-rpc-akka/pom.xml index 5d028ae5c58ef..36fc16bd3de47 100644 --- a/flink-rpc/flink-rpc-akka/pom.xml +++ b/flink-rpc/flink-rpc-akka/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-rpc - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-rpc/flink-rpc-core/pom.xml b/flink-rpc/flink-rpc-core/pom.xml index c781c7e162f48..08c8ab9b3eaba 100644 --- a/flink-rpc/flink-rpc-core/pom.xml +++ b/flink-rpc/flink-rpc-core/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-rpc - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-rpc/pom.xml b/flink-rpc/pom.xml index 0eed91f9f2c1f..d9ee427f6767f 100644 --- a/flink-rpc/pom.xml +++ b/flink-rpc/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-runtime-web/pom.xml b/flink-runtime-web/pom.xml index f6eabd5bf3b51..4420d5fa2cdb0 100644 --- a/flink-runtime-web/pom.xml +++ b/flink-runtime-web/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-runtime/pom.xml b/flink-runtime/pom.xml index b728d8d92d0eb..9f84e6fe65d68 100644 --- a/flink-runtime/pom.xml +++ b/flink-runtime/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-scala/pom.xml b/flink-scala/pom.xml index 3f8b5a1e63aeb..d333c50ab7a32 100644 --- a/flink-scala/pom.xml +++ b/flink-scala/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-state-backends/flink-statebackend-changelog/pom.xml b/flink-state-backends/flink-statebackend-changelog/pom.xml index eba8d12572883..c4796a77b992a 100644 --- a/flink-state-backends/flink-statebackend-changelog/pom.xml +++ b/flink-state-backends/flink-statebackend-changelog/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-state-backends - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-state-backends/flink-statebackend-heap-spillable/pom.xml b/flink-state-backends/flink-statebackend-heap-spillable/pom.xml index 3a2e7248ba28d..7669fd37cf4da 100644 --- a/flink-state-backends/flink-statebackend-heap-spillable/pom.xml +++ b/flink-state-backends/flink-statebackend-heap-spillable/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-state-backends - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-state-backends/flink-statebackend-rocksdb/pom.xml b/flink-state-backends/flink-statebackend-rocksdb/pom.xml index 07f8ba451d9cd..855ff764d8878 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/pom.xml +++ b/flink-state-backends/flink-statebackend-rocksdb/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-state-backends - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-state-backends/pom.xml b/flink-state-backends/pom.xml index 805bd6252d72b..42a4adf25ae31 100644 --- a/flink-state-backends/pom.xml +++ b/flink-state-backends/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-streaming-java/pom.xml b/flink-streaming-java/pom.xml index bc87e1d0e9d21..3dd3f10025845 100644 --- a/flink-streaming-java/pom.xml +++ b/flink-streaming-java/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-streaming-scala/pom.xml b/flink-streaming-scala/pom.xml index bbee28d315c53..5b959c1e1d7b7 100644 --- a/flink-streaming-scala/pom.xml +++ b/flink-streaming-scala/pom.xml @@ -24,7 +24,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-sql-client/pom.xml b/flink-table/flink-sql-client/pom.xml index fbd61f6bb1665..b998f69d51c4a 100644 --- a/flink-table/flink-sql-client/pom.xml +++ b/flink-table/flink-sql-client/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-sql-parser-hive/pom.xml b/flink-table/flink-sql-parser-hive/pom.xml index d5459052c923d..8aa0d31ab5702 100644 --- a/flink-table/flink-sql-parser-hive/pom.xml +++ b/flink-table/flink-sql-parser-hive/pom.xml @@ -24,7 +24,7 @@ under the License. flink-table org.apache.flink - 1.15-SNAPSHOT + 1.15.0 flink-sql-parser-hive diff --git a/flink-table/flink-sql-parser/pom.xml b/flink-table/flink-sql-parser/pom.xml index 216408ae1414b..82587b87ba8b8 100644 --- a/flink-table/flink-sql-parser/pom.xml +++ b/flink-table/flink-sql-parser/pom.xml @@ -24,7 +24,7 @@ under the License. flink-table org.apache.flink - 1.15-SNAPSHOT + 1.15.0 flink-sql-parser diff --git a/flink-table/flink-table-api-bridge-base/pom.xml b/flink-table/flink-table-api-bridge-base/pom.xml index 31628f0fb5b75..cca23fe6a52b0 100644 --- a/flink-table/flink-table-api-bridge-base/pom.xml +++ b/flink-table/flink-table-api-bridge-base/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-java-bridge/pom.xml b/flink-table/flink-table-api-java-bridge/pom.xml index c2246fe10124a..094bf2f48b8f6 100644 --- a/flink-table/flink-table-api-java-bridge/pom.xml +++ b/flink-table/flink-table-api-java-bridge/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-java-uber/pom.xml b/flink-table/flink-table-api-java-uber/pom.xml index e0f68eb3db000..35e67d9517ac1 100644 --- a/flink-table/flink-table-api-java-uber/pom.xml +++ b/flink-table/flink-table-api-java-uber/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-java/pom.xml b/flink-table/flink-table-api-java/pom.xml index ba16fbf01215c..8a492592bb05a 100644 --- a/flink-table/flink-table-api-java/pom.xml +++ b/flink-table/flink-table-api-java/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-scala-bridge/pom.xml b/flink-table/flink-table-api-scala-bridge/pom.xml index 9dd3d220721d5..539acc460dcc4 100644 --- a/flink-table/flink-table-api-scala-bridge/pom.xml +++ b/flink-table/flink-table-api-scala-bridge/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-api-scala/pom.xml b/flink-table/flink-table-api-scala/pom.xml index 84b79e716ceb6..281bd9ae5c79d 100644 --- a/flink-table/flink-table-api-scala/pom.xml +++ b/flink-table/flink-table-api-scala/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-code-splitter/pom.xml b/flink-table/flink-table-code-splitter/pom.xml index 2be8a7e7111bb..76daebe6014a0 100644 --- a/flink-table/flink-table-code-splitter/pom.xml +++ b/flink-table/flink-table-code-splitter/pom.xml @@ -24,7 +24,7 @@ under the License. flink-table org.apache.flink - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-common/pom.xml b/flink-table/flink-table-common/pom.xml index 514efd26168a7..a8a211fd82370 100644 --- a/flink-table/flink-table-common/pom.xml +++ b/flink-table/flink-table-common/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-planner-loader-bundle/pom.xml b/flink-table/flink-table-planner-loader-bundle/pom.xml index 5df0d1284fbc0..4c9f772909666 100644 --- a/flink-table/flink-table-planner-loader-bundle/pom.xml +++ b/flink-table/flink-table-planner-loader-bundle/pom.xml @@ -26,7 +26,7 @@ org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-planner-loader/pom.xml b/flink-table/flink-table-planner-loader/pom.xml index 8a1a7746777e7..19bdf7e36d9d8 100644 --- a/flink-table/flink-table-planner-loader/pom.xml +++ b/flink-table/flink-table-planner-loader/pom.xml @@ -26,7 +26,7 @@ org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-planner/pom.xml b/flink-table/flink-table-planner/pom.xml index 469a7a3d591b2..9f4cfeef174ac 100644 --- a/flink-table/flink-table-planner/pom.xml +++ b/flink-table/flink-table-planner/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-runtime/pom.xml b/flink-table/flink-table-runtime/pom.xml index a846bf9857dca..9681d923bb2d8 100644 --- a/flink-table/flink-table-runtime/pom.xml +++ b/flink-table/flink-table-runtime/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/flink-table-test-utils/pom.xml b/flink-table/flink-table-test-utils/pom.xml index 28f14ad4a68fe..537bd696d0b3b 100644 --- a/flink-table/flink-table-test-utils/pom.xml +++ b/flink-table/flink-table-test-utils/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-table - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-table/pom.xml b/flink-table/pom.xml index b3d66fd5e422e..9c66bdb5c5f58 100644 --- a/flink-table/pom.xml +++ b/flink-table/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-test-utils-parent/flink-connector-test-utils/pom.xml b/flink-test-utils-parent/flink-connector-test-utils/pom.xml index 13aeea784b55b..eb989a86311db 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/pom.xml +++ b/flink-test-utils-parent/flink-connector-test-utils/pom.xml @@ -25,7 +25,7 @@ org.apache.flink flink-test-utils-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-test-utils-parent/flink-test-utils-junit/pom.xml b/flink-test-utils-parent/flink-test-utils-junit/pom.xml index 85cab96d721e2..0a50f3fe20760 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/pom.xml +++ b/flink-test-utils-parent/flink-test-utils-junit/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-test-utils-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml index 06b1872f5e973..945ce88755f8c 100644 --- a/flink-test-utils-parent/flink-test-utils/pom.xml +++ b/flink-test-utils-parent/flink-test-utils/pom.xml @@ -25,7 +25,7 @@ under the License. org.apache.flink flink-test-utils-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-test-utils-parent/pom.xml b/flink-test-utils-parent/pom.xml index 812dd6bcb8a45..ef5712dc5dbc0 100644 --- a/flink-test-utils-parent/pom.xml +++ b/flink-test-utils-parent/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-tests/pom.xml b/flink-tests/pom.xml index 1acd328ea7e19..aa25f3edcb87f 100644 --- a/flink-tests/pom.xml +++ b/flink-tests/pom.xml @@ -26,7 +26,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-walkthroughs/flink-walkthrough-common/pom.xml b/flink-walkthroughs/flink-walkthrough-common/pom.xml index 15ed33e98eeb1..d2b7e68599aa3 100644 --- a/flink-walkthroughs/flink-walkthrough-common/pom.xml +++ b/flink-walkthroughs/flink-walkthrough-common/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-walkthroughs - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-walkthroughs/flink-walkthrough-datastream-java/pom.xml b/flink-walkthroughs/flink-walkthrough-datastream-java/pom.xml index d5a29a2251485..36e58ed4a9b22 100644 --- a/flink-walkthroughs/flink-walkthrough-datastream-java/pom.xml +++ b/flink-walkthroughs/flink-walkthrough-datastream-java/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-walkthroughs - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-walkthroughs/flink-walkthrough-datastream-scala/pom.xml b/flink-walkthroughs/flink-walkthrough-datastream-scala/pom.xml index fcd6c9460ac79..72b80c73465b8 100644 --- a/flink-walkthroughs/flink-walkthrough-datastream-scala/pom.xml +++ b/flink-walkthroughs/flink-walkthrough-datastream-scala/pom.xml @@ -27,7 +27,7 @@ under the License. org.apache.flink flink-walkthroughs - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-walkthroughs/pom.xml b/flink-walkthroughs/pom.xml index bc88b1de2bdd8..9ca2c705be64b 100644 --- a/flink-walkthroughs/pom.xml +++ b/flink-walkthroughs/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-yarn-tests/pom.xml b/flink-yarn-tests/pom.xml index 0bb1ac219b6a5..6dd8be9b5ef0c 100644 --- a/flink-yarn-tests/pom.xml +++ b/flink-yarn-tests/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/flink-yarn/pom.xml b/flink-yarn/pom.xml index 55ee45245ebda..20ab003d1590d 100644 --- a/flink-yarn/pom.xml +++ b/flink-yarn/pom.xml @@ -23,7 +23,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 .. diff --git a/pom.xml b/pom.xml index 711c631e097ee..1c79a73b08ccd 100644 --- a/pom.xml +++ b/pom.xml @@ -28,7 +28,7 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 Flink : pom diff --git a/tools/ci/java-ci-tools/pom.xml b/tools/ci/java-ci-tools/pom.xml index a749bb48ede33..a82f032113106 100644 --- a/tools/ci/java-ci-tools/pom.xml +++ b/tools/ci/java-ci-tools/pom.xml @@ -25,12 +25,12 @@ under the License. org.apache.flink flink-parent - 1.15-SNAPSHOT + 1.15.0 ../../.. java-ci-tools - 1.15-SNAPSHOT + 1.15.0 Flink : Tools : CI : Java From e5d2bc7ce0f1017814915c495c9f164aa0472941 Mon Sep 17 00:00:00 2001 From: "affe (Yufei Zhang)" Date: Thu, 12 May 2022 09:48:49 +0800 Subject: [PATCH 205/258] 89: setup flink PR checkt pipeline (#90) --- .github/workflows/daily-end-to-end-test.yml | 25 +++++++++++++++++++++ .github/workflows/run-pr-check.yml | 20 +++++++++++++++++ 2 files changed, 45 insertions(+) create mode 100644 .github/workflows/daily-end-to-end-test.yml create mode 100644 .github/workflows/run-pr-check.yml diff --git a/.github/workflows/daily-end-to-end-test.yml b/.github/workflows/daily-end-to-end-test.yml new file mode 100644 index 0000000000000..1d4ad642427e8 --- /dev/null +++ b/.github/workflows/daily-end-to-end-test.yml @@ -0,0 +1,25 @@ +name: Daily End to End Test + +on: + schedule: + - cron: 0 1 * * * + workflow_dispatch: +jobs: + sync: + name: Run End to End test for pulsar + runs-on: ubuntu-latest + strategy: + matrix: + branch: + - develop + - relesae-sn-1.14 + - release-sn-1.15 + steps: + - name: Checkout the code + uses: actions/checkout@v2 + with: + ref: ${{ matrix.branch }} + - name: Run e2e Test + run: | + ./mvnw clean install -DskipTests + ./mvnw -pl 'flink-end-to-end-tests/flink-end-to-end-tests-pulsar' test diff --git a/.github/workflows/run-pr-check.yml b/.github/workflows/run-pr-check.yml new file mode 100644 index 0000000000000..3c765bb17a8db --- /dev/null +++ b/.github/workflows/run-pr-check.yml @@ -0,0 +1,20 @@ +name: Run Test for Pull Request + +on: + pull_request: + branches: [ develop, release-sn-** ] + +jobs: + test: + runs-on: ubuntu-latest + steps: + - name: Checkout the code + uses: actions/checkout@v2 + with: + ref: ${{ github.event.pull_request.head.sha }} + + - name: Run Unit Test and Install + run: | + ./mvnw -pl 'flink-connectors/flink-connector-pulsar,flink-connectors/flink-sql-connector-pulsar' \ + clean install + From f7d1135f7b0cd9fd6a69fedcd32144cf132b7fae Mon Sep 17 00:00:00 2001 From: "affe (Yufei Zhang)" Date: Mon, 23 May 2022 22:30:19 +0800 Subject: [PATCH 206/258] change the e2e test maven to use a fixed maven version (#95) --- .github/workflows/daily-end-to-end-test.yml | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/.github/workflows/daily-end-to-end-test.yml b/.github/workflows/daily-end-to-end-test.yml index 1d4ad642427e8..be05b300c6eaa 100644 --- a/.github/workflows/daily-end-to-end-test.yml +++ b/.github/workflows/daily-end-to-end-test.yml @@ -19,7 +19,11 @@ jobs: uses: actions/checkout@v2 with: ref: ${{ matrix.branch }} + - name: Set up Maven + uses: apache/pulsar-test-infra/setup-maven@master + with: + maven-version: 3.6.2 - name: Run e2e Test run: | - ./mvnw clean install -DskipTests - ./mvnw -pl 'flink-end-to-end-tests/flink-end-to-end-tests-pulsar' test + mvn clean install -DskipTests + mvn -pl 'flink-end-to-end-tests/flink-end-to-end-tests-pulsar' test From 57adf06b768a57a1583156ca1098a558a793ba66 Mon Sep 17 00:00:00 2001 From: "affe (Yufei Zhang)" Date: Tue, 24 May 2022 10:03:24 +0800 Subject: [PATCH 207/258] fix a typo in daily e2e test (#96) --- .github/workflows/daily-end-to-end-test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/daily-end-to-end-test.yml b/.github/workflows/daily-end-to-end-test.yml index be05b300c6eaa..4e8d492a4b992 100644 --- a/.github/workflows/daily-end-to-end-test.yml +++ b/.github/workflows/daily-end-to-end-test.yml @@ -12,7 +12,7 @@ jobs: matrix: branch: - develop - - relesae-sn-1.14 + - release-sn-1.14 - release-sn-1.15 steps: - name: Checkout the code From b297ceefa8d4af7384e59996f680785c840e0650 Mon Sep 17 00:00:00 2001 From: "affe (Yufei Zhang)" Date: Tue, 7 Jun 2022 15:02:09 +0800 Subject: [PATCH 208/258] Introduce SQL Connector (#92) * [FLINK-25686][Connector/Pulsar]: Add schema evolution support for pulsar source connector * [FLINK-25686] fix according to review * [SQL Connector]: add source and sink SQL connector * [FLINK-25686] fix a test and run PR --- .github/workflows/run-pr-check.yml | 7 +- .../docs/connectors/datastream/pulsar.md | 13 + .../docs/connectors/datastream/pulsar.md | 15 + .../pulsar_source_configuration.html | 6 + .../48c7dd05-c840-4ac4-a3ba-919e07191450 | 0 .../a2ce237e-b050-4ba0-8748-d83637a207a8 | 0 .../archunit-violations/stored.rules | 4 + .../flink-connector-pulsar/pom.xml | 85 +++- .../schema/factories/AvroSchemaFactory.java | 13 + .../schema/factories/JSONSchemaFactory.java | 13 + .../factories/KeyValueSchemaFactory.java | 7 +- .../pulsar/sink/writer/PulsarWriter.java | 17 +- .../sink/writer/message/PulsarMessage.java | 8 + .../writer/message/PulsarMessageBuilder.java | 11 + .../connector/pulsar/source/PulsarSource.java | 2 +- .../pulsar/source/PulsarSourceBuilder.java | 26 ++ .../pulsar/source/PulsarSourceOptions.java | 11 + .../source/config/SourceConfiguration.java | 13 + .../cursor/start/MessageIdStartCursor.java | 19 + .../cursor/start/TimestampStartCursor.java | 21 +- .../subscriber/impl/BasePulsarSubscriber.java | 4 +- .../PulsarDeserializationSchema.java | 11 +- .../PulsarDeserializationSchemaWrapper.java | 3 +- .../deserializer/PulsarSchemaWrapper.java | 35 +- .../PulsarTypeInformationWrapper.java | 2 +- .../PulsarOrderedPartitionSplitReader.java | 6 +- .../split/PulsarPartitionSplitReaderBase.java | 35 +- .../PulsarUnorderedPartitionSplitReader.java | 8 +- .../pulsar/table/PulsarTableFactory.java | 282 ++++++++++++ .../pulsar/table/PulsarTableOptionUtils.java | 272 +++++++++++ .../pulsar/table/PulsarTableOptions.java | 216 +++++++++ .../table/PulsarTableValidationUtils.java | 158 +++++++ .../sink/PulsarTableSerializationSchema.java | 110 +++++ ...PulsarTableSerializationSchemaFactory.java | 153 +++++++ .../pulsar/table/sink/PulsarTableSink.java | 171 +++++++ .../table/sink/PulsarWritableMetadata.java | 140 ++++++ .../table/source/PulsarReadableMetadata.java | 145 ++++++ .../table/source/PulsarRowDataConverter.java | 118 +++++ .../PulsarTableDeserializationSchema.java | 109 +++++ ...lsarTableDeserializationSchemaFactory.java | 204 +++++++++ .../table/source/PulsarTableSource.java | 213 +++++++++ .../org.apache.flink.table.factories.Factory | 16 + .../TestCodeArchitectureTest.java | 1 - .../factories/JSONSchemaFactoryTest.java | 4 +- .../pulsar/source/PulsarSourceITCase.java | 6 +- .../PulsarDeserializationSchemaTest.java | 253 ++++++++++- .../PulsarPartitionSplitReaderTestBase.java | 3 +- .../pulsar/table/PulsarTableFactoryTest.java | 427 ++++++++++++++++++ .../pulsar/table/PulsarTableITCase.java | 295 ++++++++++++ .../table/PulsarTableOptionUtilsTest.java | 376 +++++++++++++++ .../pulsar/table/PulsarTableOptionsTest.java | 420 +++++++++++++++++ .../pulsar/table/PulsarTableTestBase.java | 75 +++ .../table/PulsarTableValidationUtilsTest.java | 193 ++++++++ .../testutils/MockPulsarAuthentication.java | 84 ++++ .../table/testutils/MockTopicRouter.java | 37 ++ .../table/testutils/PulsarTableTestUtils.java | 52 +++ 56 files changed, 4875 insertions(+), 53 deletions(-) create mode 100644 flink-connectors/flink-connector-pulsar/archunit-violations/48c7dd05-c840-4ac4-a3ba-919e07191450 create mode 100644 flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 create mode 100644 flink-connectors/flink-connector-pulsar/archunit-violations/stored.rules create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarRowDataConverter.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchemaFactory.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java diff --git a/.github/workflows/run-pr-check.yml b/.github/workflows/run-pr-check.yml index 3c765bb17a8db..54c7ef67b2a11 100644 --- a/.github/workflows/run-pr-check.yml +++ b/.github/workflows/run-pr-check.yml @@ -12,9 +12,12 @@ jobs: uses: actions/checkout@v2 with: ref: ${{ github.event.pull_request.head.sha }} - + - name: Set up Maven + uses: apache/pulsar-test-infra/setup-maven@master + with: + maven-version: 3.6.2 - name: Run Unit Test and Install run: | - ./mvnw -pl 'flink-connectors/flink-connector-pulsar,flink-connectors/flink-sql-connector-pulsar' \ + mvn -ntp -pl 'flink-connectors/flink-connector-pulsar,flink-connectors/flink-sql-connector-pulsar' \ clean install diff --git a/docs/content.zh/docs/connectors/datastream/pulsar.md b/docs/content.zh/docs/connectors/datastream/pulsar.md index 8d44be88c71f9..68bf281510a43 100644 --- a/docs/content.zh/docs/connectors/datastream/pulsar.md +++ b/docs/content.zh/docs/connectors/datastream/pulsar.md @@ -155,6 +155,19 @@ Pulsar Source 提供了两种订阅 Topic 或 Topic 分区的方式。 PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig); ``` +如果使用 KeyValue 或者 Struct 类型的Schema, 那么 pulsar `Schema` 讲不会含有类型类信息, 但 `PulsarSchemaTypeInformation` 需要通过传入类型类信息来构造。因此我们提供的API支持用户传入类型类信息。 + +例子如下: + +```java + // Primitive 类型: 不需要提供类型信息 + PulsarDeserializationSchema.pulsarSchema(Schema.INT32); + // Struct 类型 (JSON, Protobuf, Avro, 等等.) + PulsarDeserializationSchema.pulsarSchema(Schema.AVRO(SomeClass), SomeClass.class); + // KeyValue 类型 + PulsarDeserializationSchema.pulsarSchema(Schema.KeyValue(Schema.INT32, Schema.AVRO(SomeClass)), Integer.class, SomeClass.class); +``` + Pulsar 的 `Message` 包含了很多 [额外的属性](https://pulsar.apache.org/docs/zh-CN/concepts-messaging/#%E6%B6%88%E6%81%AF)。例如,消息的 key、消息发送时间、消息生产时间、用户在消息上自定义的键值对属性等。可以使用 `Message` 接口来获取这些属性。 如果用户需要基于这些额外的属性来解析一条消息,可以实现 `PulsarDeserializationSchema` 接口。并一定要确保 `PulsarDeserializationSchema.getProducedType()` 方法返回的 `TypeInformation` 是正确的结果。Flink 使用 `TypeInformation` 将解析出来的结果序列化传递到下游算子。 diff --git a/docs/content/docs/connectors/datastream/pulsar.md b/docs/content/docs/connectors/datastream/pulsar.md index 0953753ee33fb..a8c7057cdddf2 100644 --- a/docs/content/docs/connectors/datastream/pulsar.md +++ b/docs/content/docs/connectors/datastream/pulsar.md @@ -176,6 +176,21 @@ you can use the predefined `PulsarDeserializationSchema`. Pulsar connector provi ```java PulsarDeserializationSchema.flinkTypeInfo(TypeInformation, ExecutionConfig); ``` +If using KeyValue type or Struct types, the pulsar `Schema` does not contain type class info which is +needed by `PulsarSchemaTypeInformation`. So the two APIs provides 2 parameter to pass the type info. + +A example would be: + +```java + // Primitive types: do not need to provide type class info + PulsarDeserializationSchema.pulsarSchema(Schema.INT32); + + // Struct types (JSON, Protobuf, Avro, etc.) + PulsarDeserializationSchema.pulsarSchema(Schema.AVRO(SomeClass), SomeClass.class); + + // KeyValue type + PulsarDeserializationSchema.pulsarSchema(Schema.KeyValue(Schema.INT32, Schema.AVRO(SomeClass)), Integer.class, SomeClass.class); +``` Pulsar `Message` contains some [extra properties](https://pulsar.apache.org/docs/en/concepts-messaging/#messages), such as message key, message publish time, message time, and application-defined key/value pairs etc. diff --git a/docs/layouts/shortcodes/generated/pulsar_source_configuration.html b/docs/layouts/shortcodes/generated/pulsar_source_configuration.html index 3bcdad85e1a04..c79495029b45d 100644 --- a/docs/layouts/shortcodes/generated/pulsar_source_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_source_configuration.html @@ -20,6 +20,12 @@ Boolean Flink commits the consuming position with pulsar transactions on checkpoint. However, if you have disabled the Flink checkpoint or disabled transaction for your Pulsar cluster, ensure that you have set this option to true.
    The source would use pulsar client's internal mechanism and commit cursor in two ways.

    • For Key_Shared and Shared subscription, the cursor would be committed once the message is consumed.
    • For Exclusive and Failover subscription, the cursor would be committed in a given interval.
    + +
    pulsar.source.enableSchemaEvolution
    + false + Boolean + If you enable this option, we would consume and deserialize the message by using Pulsar's Schema. +
    pulsar.source.maxFetchRecords
    100 diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/48c7dd05-c840-4ac4-a3ba-919e07191450 b/flink-connectors/flink-connector-pulsar/archunit-violations/48c7dd05-c840-4ac4-a3ba-919e07191450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 b/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/stored.rules b/flink-connectors/flink-connector-pulsar/archunit-violations/stored.rules new file mode 100644 index 0000000000000..ecdc39b0f1907 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/archunit-violations/stored.rules @@ -0,0 +1,4 @@ +# +#Thu May 12 11:35:25 CST 2022 +Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=48c7dd05-c840-4ac4-a3ba-919e07191450 +ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=a2ce237e-b050-4ba0-8748-d83637a207a8 diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index 7529e533c82df..0d5bb8740a747 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -56,6 +56,34 @@ under the License. ${project.version} + + + org.apache.flink + flink-table-common + ${project.version} + provided + + + + org.apache.flink + flink-table-api-java-bridge + ${project.version} + provided + + + + org.apache.flink + flink-table-runtime + ${project.version} + provided + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + provided + @@ -108,6 +136,62 @@ under the License. test + + + org.apache.flink + flink-table-common + ${project.version} + test-jar + test + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test + + + + org.apache.flink + flink-table-api-scala-bridge_${scala.binary.version} + ${project.version} + test + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${project.version} + test-jar + test + + + + + org.apache.flink + flink-json + ${project.version} + test + + + org.apache.flink + flink-avro + ${project.version} + test + + + org.apache.flink + flink-avro-confluent-registry + ${project.version} + test + + + org.apache.flink + flink-csv + ${project.version} + test + + @@ -236,7 +320,6 @@ under the License. org.apache.maven.plugins maven-surefire-plugin - true 1 -Xms256m -Xmx2048m -Dmvn.forkNumber=${surefire.forkNumber} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java index ab97b4102cc31..8288136288641 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/AvroSchemaFactory.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.pulsar.common.schema.factories; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.AvroUtils; + import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.schema.SchemaDefinition; import org.apache.pulsar.client.impl.schema.AvroSchema; @@ -45,4 +48,14 @@ public Schema createSchema(SchemaInfo info) { return AvroSchema.of(definition); } + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + try { + Class decodeClassInfo = decodeClassInfo(info); + return AvroUtils.getAvroUtils().createAvroTypeInfo(decodeClassInfo); + } catch (Exception e) { + return super.createTypeInfo(info); + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java index 8d1a4944d0c5c..2280d353d95af 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactory.java @@ -18,6 +18,9 @@ package org.apache.flink.connector.pulsar.common.schema.factories; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.AvroUtils; + import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.schema.JSONSchema; import org.apache.pulsar.common.schema.SchemaInfo; @@ -38,4 +41,14 @@ public Schema createSchema(SchemaInfo info) { Class typeClass = decodeClassInfo(info); return JSONSchema.of(typeClass, info.getProperties()); } + + @Override + public TypeInformation createTypeInfo(SchemaInfo info) { + try { + Class decodeClassInfo = decodeClassInfo(info); + return AvroUtils.getAvroUtils().createAvroTypeInfo(decodeClassInfo); + } catch (Exception e) { + return super.createTypeInfo(info); + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java index 893e7e6eeb1db..d60f06d2a7643 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/factories/KeyValueSchemaFactory.java @@ -69,11 +69,8 @@ public Schema> createSchema(SchemaInfo info) { public TypeInformation> createTypeInfo(SchemaInfo info) { KeyValue kvSchemaInfo = decodeKeyValueSchemaInfo(info); - Schema keySchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getKey()); - Class keyClass = decodeClassInfo(keySchema.getSchemaInfo()); - - Schema valueSchema = PulsarSchemaUtils.createSchema(kvSchemaInfo.getValue()); - Class valueClass = decodeClassInfo(valueSchema.getSchemaInfo()); + Class keyClass = decodeClassInfo(kvSchemaInfo.getKey()); + Class valueClass = decodeClassInfo(kvSchemaInfo.getValue()); Schema> schema = createSchema(info); PulsarSchema> pulsarSchema = diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java index 1e4113a8d03b1..30c454455ca9f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java @@ -45,6 +45,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Base64; import java.util.Collection; import java.util.List; import java.util.Map; @@ -133,9 +134,16 @@ public void write(IN element, Context context) throws IOException, InterruptedEx PulsarMessage message = serializationSchema.serialize(element, sinkContext); // Choose the right topic to send. - String key = message.getKey(); + List availableTopics = metadataListener.availableTopics(); - String topic = topicRouter.route(element, key, availableTopics, sinkContext); + String keyString; + // TODO if both keyBytes and key are set, use keyBytes. This is a temporary solution. + if (message.getKeyBytes() == null) { + keyString = message.getKey(); + } else { + keyString = Base64.getEncoder().encodeToString(message.getKeyBytes()); + } + String topic = topicRouter.route(element, keyString, availableTopics, sinkContext); // Create message builder for sending message. TypedMessageBuilder builder = createMessageBuilder(topic, context, message); @@ -206,6 +214,11 @@ private TypedMessageBuilder createMessageBuilder( builder.key(key); } + byte[] keyBytes = message.getKeyBytes(); + if (keyBytes != null) { + builder.keyBytes(keyBytes); + } + long eventTime = message.getEventTime(); if (eventTime > 0) { builder.eventTime(eventTime); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java index 0c45763cdb1ef..0dd52655f9040 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessage.java @@ -37,6 +37,7 @@ public class PulsarMessage { @Nullable private final byte[] orderingKey; @Nullable private final String key; + @Nullable private final byte[] keyBytes; private final long eventTime; private final Schema schema; @Nullable private final T value; @@ -49,6 +50,7 @@ public class PulsarMessage { PulsarMessage( @Nullable byte[] orderingKey, @Nullable String key, + @Nullable byte[] keyBytes, long eventTime, Schema schema, @Nullable T value, @@ -58,6 +60,7 @@ public class PulsarMessage { boolean disableReplication) { this.orderingKey = orderingKey; this.key = key; + this.keyBytes = keyBytes; this.eventTime = eventTime; this.schema = schema; this.value = value; @@ -77,6 +80,11 @@ public String getKey() { return key; } + @Nullable + public byte[] getKeyBytes() { + return keyBytes; + } + public long getEventTime() { return eventTime; } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java index 9330d09a3d03b..f894f2410778c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java @@ -36,6 +36,7 @@ public class PulsarMessageBuilder { private byte[] orderingKey; private String key; + private byte[] keyBytes; private long eventTime; Schema schema; private T value; @@ -59,6 +60,15 @@ public PulsarMessageBuilder key(String key) { return null; } + /** + * Property {@link TypedMessageBuilder#keyBytes(byte[])}. This property would also be used in + * {@link KeyHashTopicRouter}. + */ + public PulsarMessageBuilder keyBytes(byte[] keyBytes) { + this.keyBytes = checkNotNull(keyBytes); + return null; + } + /** Method wrapper of {@link TypedMessageBuilder#eventTime(long)}. */ public PulsarMessageBuilder eventTime(long eventTime) { this.eventTime = eventTime; @@ -116,6 +126,7 @@ public PulsarMessage build() { return new PulsarMessage<>( orderingKey, key, + keyBytes, eventTime, schema, value, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java index a6c48d14bc87e..560b32162f2b3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java @@ -94,7 +94,7 @@ public final class PulsarSource * The constructor for PulsarSource, it's package protected for forcing using {@link * PulsarSourceBuilder}. */ - PulsarSource( + public PulsarSource( SourceConfiguration sourceConfiguration, PulsarSubscriber subscriber, RangeGenerator rangeGenerator, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java index 517286d8341a3..b6e0f408d5bbf 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java @@ -25,6 +25,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder; import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSchemaWrapper; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; @@ -54,6 +55,7 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_CONSUMER_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_SCHEMA_EVOLUTION; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_TRANSACTION_TIMEOUT; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; @@ -273,6 +275,15 @@ public PulsarSourceBuilder setConsumerName(String consumerName) { return setConfig(PULSAR_CONSUMER_NAME, consumerName); } + /** + * If you enable this option, we would consume and deserialize the message by using Pulsar + * {@link Schema}. + */ + public PulsarSourceBuilder enableSchemaEvolution() { + configBuilder.set(PULSAR_READ_SCHEMA_EVOLUTION, true); + return this; + } + /** * Set a topic range generator for Key_Shared subscription. * @@ -470,6 +481,21 @@ public PulsarSource build() { } } + // Schema evolution check. + if (deserializationSchema instanceof PulsarSchemaWrapper + && !Boolean.TRUE.equals(configBuilder.get(PULSAR_READ_SCHEMA_EVOLUTION))) { + LOG.info( + "It seems like you want to read message using Pulsar Schema." + + " You can enableSchemaEvolution for using this feature." + + " We would use Schema.BYTES as the default schema if you don't enable this option."); + } + + if (Boolean.TRUE.equals(configBuilder.get(PULSAR_READ_SCHEMA_EVOLUTION))) { + checkState( + deserializationSchema instanceof PulsarSchemaWrapper, + "When enabling schema evolution, you must use a Pulsar schema."); + } + if (!configBuilder.contains(PULSAR_CONSUMER_NAME)) { LOG.warn( "We recommend set a readable consumer name through setConsumerName(String) in production mode."); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java index 39a73974f5c0a..70ddb6793dfd8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java @@ -188,6 +188,17 @@ private PulsarSourceOptions() { " A possible solution is to adjust the retention settings in Pulsar or ignoring the check result.") .build()); + public static final ConfigOption PULSAR_READ_SCHEMA_EVOLUTION = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "enableSchemaEvolution") + .booleanType() + .defaultValue(false) + .withDescription( + Description.builder() + .text( + "If you enable this option, we would consume and deserialize the message by using Pulsar's %s.", + code("Schema")) + .build()); + /////////////////////////////////////////////////////////////////////////////// // // The configuration for ConsumerConfigurationData part. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java index 806fe4a418b23..bd6c61216acd7 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java @@ -23,10 +23,12 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.pulsar.common.config.PulsarConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSchemaWrapper; import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionMode; import org.apache.pulsar.client.api.SubscriptionType; @@ -39,6 +41,7 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_SCHEMA_EVOLUTION; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_READ_TRANSACTION_TIMEOUT; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_MODE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; @@ -53,6 +56,7 @@ public class SourceConfiguration extends PulsarConfiguration { private final int messageQueueCapacity; private final long partitionDiscoveryIntervalMs; private final boolean enableAutoAcknowledgeMessage; + private final boolean enableSchemaEvolution; private final long autoCommitCursorInterval; private final long transactionTimeoutMillis; private final Duration maxFetchTime; @@ -68,6 +72,7 @@ public SourceConfiguration(Configuration configuration) { this.messageQueueCapacity = getInteger(ELEMENT_QUEUE_CAPACITY); this.partitionDiscoveryIntervalMs = get(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS); this.enableAutoAcknowledgeMessage = get(PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE); + this.enableSchemaEvolution = get(PULSAR_READ_SCHEMA_EVOLUTION); this.autoCommitCursorInterval = get(PULSAR_AUTO_COMMIT_CURSOR_INTERVAL); this.transactionTimeoutMillis = get(PULSAR_READ_TRANSACTION_TIMEOUT); this.maxFetchTime = get(PULSAR_MAX_FETCH_TIME, Duration::ofMillis); @@ -107,6 +112,14 @@ public boolean isEnableAutoAcknowledgeMessage() { return enableAutoAcknowledgeMessage; } + /** + * If we should deserialize the message with a specified Pulsar {@link Schema} instead the + * default {@link Schema#BYTES}. This switch is only used for {@link PulsarSchemaWrapper}. + */ + public boolean isEnableSchemaEvolution() { + return enableSchemaEvolution; + } + /** * The interval in millis for acknowledge message when you enable {@link * #isEnableAutoAcknowledgeMessage} and use {@link SubscriptionType#Failover} or {@link diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java index f8079608cc65a..3705f865ebc01 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -25,6 +25,8 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.impl.MessageIdImpl; +import java.util.Objects; + import static org.apache.flink.util.Preconditions.checkState; /** This cursor would left pulsar start consuming from a specific message id. */ @@ -67,4 +69,21 @@ public MessageIdStartCursor(MessageId messageId, boolean inclusive) { public CursorPosition position(String topic, int partitionId) { return new CursorPosition(messageId); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MessageIdStartCursor that = (MessageIdStartCursor) o; + return Objects.equals(messageId, that.messageId); + } + + @Override + public int hashCode() { + return Objects.hash(messageId); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java index eb4ea32ebb6b9..e03a773281b83 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java @@ -21,7 +21,9 @@ import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; -/** This cursor would left pulsar start consuming from a specific publish timestamp. */ +import java.util.Objects; + +/** A cursor starts consuming from a specific publish timestamp. */ public class TimestampStartCursor implements StartCursor { private static final long serialVersionUID = 5170578885838095320L; @@ -35,4 +37,21 @@ public TimestampStartCursor(long timestamp) { public CursorPosition position(String topic, int partitionId) { return new CursorPosition(timestamp); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TimestampStartCursor that = (TimestampStartCursor) o; + return timestamp == that.timestamp; + } + + @Override + public int hashCode() { + return Objects.hash(timestamp); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java index d266c98df4253..57d5f02ed24c0 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java @@ -20,7 +20,6 @@ import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; @@ -32,6 +31,7 @@ import java.util.stream.IntStream; import static java.util.stream.Collectors.toList; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; /** PulsarSubscriber abstract class to simplify Pulsar admin related operations. */ public abstract class BasePulsarSubscriber implements PulsarSubscriber { @@ -39,7 +39,7 @@ public abstract class BasePulsarSubscriber implements PulsarSubscriber { protected TopicMetadata queryTopicMetadata(PulsarAdmin pulsarAdmin, String topicName) { // Drop the complete topic name for a clean partitioned topic name. - String completeTopicName = TopicNameUtils.topicName(topicName); + String completeTopicName = topicName(topicName); try { PartitionedTopicMetadata metadata = pulsarAdmin.topics().getPartitionedTopicMetadata(completeTopicName); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java index 4a116cdc64dc1..3e3bc4217bf70 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java @@ -76,7 +76,16 @@ default void open(InitializationContext context) throws Exception { * @param message The message decoded by pulsar. * @param out The collector to put the resulting messages. */ - void deserialize(Message message, Collector out) throws Exception; + void deserialize(Message message, Collector out) throws Exception; + + /** + * By default, deserializers will decode bytes array message. + * + * @return The related Pulsar Schema for this serializer. + */ + default Schema schema() { + return Schema.BYTES; + } /** * Create a PulsarDeserializationSchema by using the flink's {@link DeserializationSchema}. It diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java index e9b2779d9a99c..e3f0a7770d2ff 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java @@ -52,10 +52,9 @@ public void open(InitializationContext context, SourceConfiguration configuratio } @Override - public void deserialize(Message message, Collector out) throws Exception { + public void deserialize(Message message, Collector out) throws Exception { byte[] bytes = message.getData(); T instance = deserializationSchema.deserialize(bytes); - out.collect(instance); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java index 7926d803d4a11..2bcc163d198c1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java @@ -18,8 +18,10 @@ package org.apache.flink.connector.pulsar.source.reader.deserializer; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.util.Collector; import org.apache.pulsar.client.api.Message; @@ -41,17 +43,31 @@ class PulsarSchemaWrapper implements PulsarDeserializationSchema { /** The serializable pulsar schema, it wrap the schema with type class. */ private final PulsarSchema pulsarSchema; + private boolean isSchemaEvolutionEnabled; + public PulsarSchemaWrapper(PulsarSchema pulsarSchema) { this.pulsarSchema = pulsarSchema; } @Override - public void deserialize(Message message, Collector out) throws Exception { - Schema schema = this.pulsarSchema.getPulsarSchema(); - byte[] bytes = message.getData(); - T instance = schema.decode(bytes); + public void open( + DeserializationSchema.InitializationContext context, SourceConfiguration configuration) + throws Exception { + this.isSchemaEvolutionEnabled = configuration.isEnableSchemaEvolution(); + } - out.collect(instance); + @Override + public void deserialize(Message message, Collector out) throws Exception { + if (isSchemaEvolutionEnabled) { + @SuppressWarnings("unchecked") + T value = (T) message.getValue(); + out.collect(value); + } else { + Schema schema = this.pulsarSchema.getPulsarSchema(); + byte[] bytes = message.getData(); + T instance = schema.decode(bytes); + out.collect(instance); + } } @Override @@ -59,4 +75,13 @@ public TypeInformation getProducedType() { SchemaInfo info = pulsarSchema.getSchemaInfo(); return createTypeInformation(info); } + + @Override + public Schema schema() { + if (isSchemaEvolutionEnabled) { + return pulsarSchema.getPulsarSchema(); + } else { + return Schema.BYTES; + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java index 0118079f82fa4..5ea9de99dff81 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java @@ -51,7 +51,7 @@ public PulsarTypeInformationWrapper(TypeInformation information, ExecutionCon } @Override - public void deserialize(Message message, Collector out) throws Exception { + public void deserialize(Message message, Collector out) throws Exception { DataInputDeserializer dis = DESERIALIZER.get(); dis.setBuffer(message.getData()); T instance = serializer.deserialize(dis); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java index bb6d79641f503..a6c713cf6577c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java @@ -61,12 +61,12 @@ public PulsarOrderedPartitionSplitReader( } @Override - protected Message pollMessage(Duration timeout) throws PulsarClientException { + protected Message pollMessage(Duration timeout) throws PulsarClientException { return pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS); } @Override - protected void finishedPollMessage(Message message) { + protected void finishedPollMessage(Message message) { // Nothing to do here. LOG.debug("Finished polling message {}", message); @@ -75,7 +75,7 @@ protected void finishedPollMessage(Message message) { } @Override - protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { MessageId latestConsumedId = split.getLatestConsumedId(); // Reset the start position for ordered pulsar consumer. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java index 37b5630a8d113..9778bd183c8bd 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java @@ -72,7 +72,7 @@ abstract class PulsarPartitionSplitReaderBase protected final PulsarDeserializationSchema deserializationSchema; protected final AtomicBoolean wakeup; - protected Consumer pulsarConsumer; + protected Consumer pulsarConsumer; protected PulsarPartitionSplit registeredSplit; protected PulsarPartitionSplitReaderBase( @@ -104,7 +104,7 @@ public RecordsWithSplitIds> fetch() throws IOException { PulsarMessageCollector collector = new PulsarMessageCollector<>(splitId, builder); Deadline deadline = Deadline.fromNow(sourceConfiguration.getMaxFetchTime()); - // Consume message from pulsar until it was woke up by flink reader. + // Consume message from pulsar until it was woken up by flink reader. for (int messageNum = 0; messageNum < sourceConfiguration.getMaxFetchRecords() && deadline.hasTimeLeft() @@ -112,16 +112,17 @@ public RecordsWithSplitIds> fetch() throws IOException { messageNum++) { try { Duration timeout = deadline.timeLeftIfAny(); - Message message = pollMessage(timeout); + Message message = pollMessage(timeout); if (message == null) { break; } - // Deserialize message. collector.setMessage(message); + + // Deserialize message by DeserializationSchema or Pulsar Schema. deserializationSchema.deserialize(message, collector); - // Acknowledge message if need. + // Acknowledge message if needed. finishedPollMessage(message); if (stopCursor.shouldStop(message)) { @@ -166,7 +167,7 @@ public void handleSplitsChanges(SplitsChange splitsChanges PulsarPartitionSplit newSplit = newSplits.get(0); // Create pulsar consumer. - Consumer consumer = createPulsarConsumer(newSplit); + Consumer consumer = createPulsarConsumer(newSplit); // Open start & stop cursor. newSplit.open(pulsarAdmin); @@ -192,12 +193,12 @@ public void close() { } @Nullable - protected abstract Message pollMessage(Duration timeout) + protected abstract Message pollMessage(Duration timeout) throws ExecutionException, InterruptedException, PulsarClientException; - protected abstract void finishedPollMessage(Message message); + protected abstract void finishedPollMessage(Message message); - protected abstract void startConsumer(PulsarPartitionSplit split, Consumer consumer); + protected abstract void startConsumer(PulsarPartitionSplit split, Consumer consumer); // --------------------------- Helper Methods ----------------------------- @@ -205,15 +206,19 @@ protected boolean isNotWakeup() { return !wakeup.get(); } - /** Create a specified {@link Consumer} by the given split information. */ - protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { + /** + * Create a specified {@link Consumer} by the given split information. If using pulsar schema, + * then use the pulsar schema, if using flink schema, then use a Schema.BYTES + */ + protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { return createPulsarConsumer(split.getPartition()); } - /** Create a specified {@link Consumer} by the given topic partition. */ - protected Consumer createPulsarConsumer(TopicPartition partition) { - ConsumerBuilder consumerBuilder = - createConsumerBuilder(pulsarClient, Schema.BYTES, sourceConfiguration); + protected Consumer createPulsarConsumer(TopicPartition partition) { + Schema schema = deserializationSchema.schema(); + + ConsumerBuilder consumerBuilder = + createConsumerBuilder(pulsarClient, schema, sourceConfiguration); consumerBuilder.topic(partition.getFullTopicName()); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java index 5940cc9ac19be..d8dc49c4de2aa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -75,9 +75,9 @@ public PulsarUnorderedPartitionSplitReader( } @Override - protected Message pollMessage(Duration timeout) + protected Message pollMessage(Duration timeout) throws ExecutionException, InterruptedException, PulsarClientException { - Message message = + Message message = pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS); // Skip the message when receive timeout @@ -116,7 +116,7 @@ protected Message pollMessage(Duration timeout) } @Override - protected void finishedPollMessage(Message message) { + protected void finishedPollMessage(Message message) { if (sourceConfiguration.isEnableAutoAcknowledgeMessage()) { sneakyClient(() -> pulsarConsumer.acknowledge(message)); } @@ -126,7 +126,7 @@ protected void finishedPollMessage(Message message) { } @Override - protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); // Abort the uncommitted pulsar transaction. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java new file mode 100644 index 0000000000000..171836b063214 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java @@ -0,0 +1,282 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.sink.PulsarSinkOptions; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.PulsarSourceOptions; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; +import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.source.PulsarTableSource; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createKeyFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createValueFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getKeyDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getKeyEncodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getMessageDelayMillis; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getPulsarProperties; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getSubscriptionType; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicListFromOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRouter; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRoutingMode; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueEncodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SERVICE_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validatePrimaryKeyConstraints; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSinkOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSourceOptions; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT; +import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; + +/** + * Factory for creating {@link DynamicTableSource} and {@link DynamicTableSink}. + * + *

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

    It contains the following functionalities. + * + *

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

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

    By default all {@link org.apache.flink.connector.pulsar.common.config.PulsarOptions} are + * included in the validateExcept() method./p> + */ + public static final ConfigOption SERVICE_URL = + ConfigOptions.key("service-url") + .stringType() + .noDefaultValue() + .withDescription( + Description.builder() + .text("Service URL provider for Pulsar service.") + .linebreak() + .text( + "To connect to Pulsar using client libraries, you need to specify a Pulsar protocol URL.") + .linebreak() + .text( + "You can assign Pulsar protocol URLs to specific clusters and use the Pulsar scheme.") + .linebreak() + .list( + text( + "This is an example of %s: %s.", + code("localhost"), + code("pulsar://localhost:6650")), + text( + "If you have multiple brokers, the URL is as: %s", + code( + "pulsar://localhost:6550,localhost:6651,localhost:6652")), + text( + "A URL for a production Pulsar cluster is as: %s", + code( + "pulsar://pulsar.us-west.example.com:6650")), + text( + "If you use TLS authentication, the URL is as %s", + code( + "pulsar+ssl://pulsar.us-west.example.com:6651"))) + .build()); +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java new file mode 100644 index 0000000000000..12c5ff72e47bc --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.types.RowKind; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.pulsar.common.naming.TopicName.isValid; + +/** + * Util class for source and sink validation rules. TODO refactor to use the PulsarConfigValidator + */ +public class PulsarTableValidationUtils { + + private PulsarTableValidationUtils() {} + + public static void validatePrimaryKeyConstraints( + ObjectIdentifier tableName, + int[] primaryKeyIndexes, + Map options, + FactoryUtil.TableFactoryHelper helper) { + final DecodingFormat> format = + getValueDecodingFormat(helper); + if (primaryKeyIndexes.length > 0 + && format.getChangelogMode().containsOnly(RowKind.INSERT)) { + Configuration configuration = Configuration.fromMap(options); + String formatName = configuration.getOptional(FactoryUtil.FORMAT).get(); + throw new ValidationException( + String.format( + "The Pulsar table '%s' with '%s' format doesn't support defining PRIMARY KEY constraint" + + " on the table, because it can't guarantee the semantic of primary key.", + tableName.asSummaryString(), formatName)); + } + } + + public static void validateTableSourceOptions(ReadableConfig tableOptions) { + validateTopicsConfigs(tableOptions); + validateStartCursorConfigs(tableOptions); + validateSubscriptionTypeConfigs(tableOptions); + validateKeyFormatConfigs(tableOptions); + } + + public static void validateTableSinkOptions(ReadableConfig tableOptions) { + validateTopicsConfigs(tableOptions); + validateKeyFormatConfigs(tableOptions); + validateSinkRoutingConfigs(tableOptions); + } + + protected static void validateTopicsConfigs(ReadableConfig tableOptions) { + if (tableOptions.get(TOPICS).isEmpty()) { + throw new ValidationException("The topics list should not be empty."); + } + + for (String topic : tableOptions.get(TOPICS)) { + if (!isValid(topic)) { + throw new ValidationException( + String.format("The topics name %s is not a valid topic name.", topic)); + } + } + } + + protected static void validateStartCursorConfigs(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SOURCE_START_FROM_MESSAGE_ID).isPresent() + && tableOptions.getOptional(SOURCE_START_FROM_PUBLISH_TIME).isPresent()) { + throw new ValidationException( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME)); + } + } + + protected static void validateSubscriptionTypeConfigs(ReadableConfig tableOptions) { + SubscriptionType subscriptionType = tableOptions.get(SOURCE_SUBSCRIPTION_TYPE); + if (subscriptionType == SubscriptionType.Failover + || subscriptionType == SubscriptionType.Key_Shared) { + throw new ValidationException( + String.format( + "Only %s and %s SubscriptionType is supported. ", + SubscriptionType.Exclusive, SubscriptionType.Shared)); + } + } + + protected static void validateKeyFormatConfigs(ReadableConfig tableOptions) { + final Optional optionalKeyFormat = tableOptions.getOptional(KEY_FORMAT); + final Optional> optionalKeyFields = tableOptions.getOptional(KEY_FIELDS); + if (!optionalKeyFormat.isPresent() && optionalKeyFields.isPresent()) { + throw new ValidationException( + String.format( + "The option '%s' can only be declared if a key format is defined using '%s'.", + KEY_FIELDS.key(), KEY_FORMAT.key())); + } else if (optionalKeyFormat.isPresent() + && (!optionalKeyFields.isPresent() || optionalKeyFields.get().size() == 0)) { + throw new ValidationException( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + } + } + + protected static void validateSinkRoutingConfigs(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SINK_TOPIC_ROUTING_MODE).orElse(TopicRoutingMode.ROUND_ROBIN) + == TopicRoutingMode.CUSTOM) { + throw new ValidationException( + String.format( + "Only %s and %s can be used. For %s, please use sink.custom-topic-router for" + + "custom topic router and do not set this config.", + TopicRoutingMode.ROUND_ROBIN, + TopicRoutingMode.MESSAGE_KEY_HASH, + TopicRoutingMode.CUSTOM)); + } + if (tableOptions.getOptional(SINK_CUSTOM_TOPIC_ROUTER).isPresent() + && tableOptions.getOptional(SINK_TOPIC_ROUTING_MODE).isPresent()) { + throw new ValidationException( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SINK_CUSTOM_TOPIC_ROUTER, SINK_TOPIC_ROUTING_MODE)); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java new file mode 100644 index 0000000000000..0f996193f160f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java @@ -0,0 +1,110 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessage; +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; + +import org.apache.pulsar.client.api.Schema; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * A {@link PulsarSerializationSchema} implementation for Pulsar SQL sink connector. It is + * responsible for retrieving fields from Flink row and serialize into Pulsar message key or body, + * and set necessary metadata fields as required. + */ +public class PulsarTableSerializationSchema implements PulsarSerializationSchema { + + private static final long serialVersionUID = 7314442107082067836L; + + @Nullable private final SerializationSchema keySerialization; + + private final RowData.FieldGetter[] keyFieldGetters; + + private final SerializationSchema valueSerialization; + + private final RowData.FieldGetter[] valueFieldGetters; + + private final PulsarWritableMetadata writableMetadata; + + public PulsarTableSerializationSchema( + @Nullable SerializationSchema keySerialization, + RowData.FieldGetter[] keyFieldGetters, + SerializationSchema valueSerialization, + RowData.FieldGetter[] valueFieldGetters, + PulsarWritableMetadata writableMetadata) { + this.keySerialization = keySerialization; + this.keyFieldGetters = checkNotNull(keyFieldGetters); + this.valueSerialization = checkNotNull(valueSerialization); + this.valueFieldGetters = checkNotNull(valueFieldGetters); + this.writableMetadata = checkNotNull(writableMetadata); + } + + @Override + public void open( + SerializationSchema.InitializationContext initializationContext, + PulsarSinkContext sinkContext, + SinkConfiguration sinkConfiguration) + throws Exception { + valueSerialization.open(initializationContext); + } + + @Override + public PulsarMessage serialize(RowData consumedRow, PulsarSinkContext sinkContext) { + + PulsarMessageBuilder messageBuilder = new PulsarMessageBuilder<>(); + + final RowKind kind = consumedRow.getRowKind(); + final RowData valueRow = createProjectedRow(consumedRow, kind, valueFieldGetters); + + writableMetadata.applyWritableMetadataInMessage(consumedRow, messageBuilder); + + if (keySerialization != null) { + final RowData keyRow = createProjectedRow(consumedRow, RowKind.INSERT, keyFieldGetters); + messageBuilder.keyBytes(keySerialization.serialize(keyRow)); + } + + byte[] serializedData = valueSerialization.serialize(valueRow); + messageBuilder.value(Schema.BYTES, serializedData); + return messageBuilder.build(); + } + + private static RowData createProjectedRow( + RowData consumedRow, RowKind kind, RowData.FieldGetter[] fieldGetters) { + final int arity = fieldGetters.length; + final GenericRowData genericRowData = new GenericRowData(kind, arity); + for (int fieldPos = 0; fieldPos < arity; fieldPos++) { + genericRowData.setField(fieldPos, fieldGetters[fieldPos].getFieldOrNull(consumedRow)); + } + return genericRowData; + } + + /** A class to read fields from Flink row and map to a Pulsar metadata. */ + public interface MetadataConverter extends Serializable { + Object read(RowData consumedRow, int pos); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java new file mode 100644 index 0000000000000..fd3f7c252489f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java @@ -0,0 +1,153 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.connector.Projection; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.utils.DataTypeUtils; + +import javax.annotation.Nullable; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Contains needed field mapping and encoding format information to construct a {@link + * org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchema} instance. + */ +public class PulsarTableSerializationSchemaFactory { + + private final DataType physicalDataType; + + @Nullable private final EncodingFormat> keyEncodingFormat; + + private final int[] keyProjection; + + private final EncodingFormat> valueEncodingFormat; + + private final int[] valueProjection; + + /** Metadata that is appended at the end of a physical sink row. */ + private List writableMetadataKeys; + + public PulsarTableSerializationSchemaFactory( + DataType physicalDataType, + @Nullable EncodingFormat> keyEncodingFormat, + int[] keyProjection, + EncodingFormat> valueEncodingFormat, + int[] valueProjection) { + this.physicalDataType = checkNotNull(physicalDataType); + this.keyEncodingFormat = keyEncodingFormat; + this.keyProjection = checkNotNull(keyProjection); + this.valueEncodingFormat = checkNotNull(valueEncodingFormat); + this.valueProjection = checkNotNull(valueProjection); + this.writableMetadataKeys = Collections.emptyList(); + } + + public PulsarSerializationSchema createPulsarSerializationSchema( + DynamicTableSink.Context context) { + + final SerializationSchema keySerialization = + createSerialization(context, keyEncodingFormat, keyProjection, null); + + final SerializationSchema valueSerialization = + createSerialization(context, valueEncodingFormat, valueProjection, null); + final List physicalChildren = physicalDataType.getLogicalType().getChildren(); + + final RowData.FieldGetter[] keyFieldGetters = + getFieldGetters(physicalChildren, keyProjection); + final RowData.FieldGetter[] valueFieldGetters = + getFieldGetters(physicalChildren, valueProjection); + + final PulsarWritableMetadata writableMetadata = + new PulsarWritableMetadata(writableMetadataKeys, physicalChildren.size()); + + return new PulsarTableSerializationSchema( + keySerialization, + keyFieldGetters, + valueSerialization, + valueFieldGetters, + writableMetadata); + } + + private @Nullable SerializationSchema createSerialization( + DynamicTableSink.Context context, + @Nullable EncodingFormat> format, + int[] projection, + @Nullable String prefix) { + if (format == null) { + return null; + } + DataType physicalFormatDataType = Projection.of(projection).project(this.physicalDataType); + if (prefix != null) { + physicalFormatDataType = DataTypeUtils.stripRowPrefix(physicalFormatDataType, prefix); + } + return format.createRuntimeEncoder(context, physicalFormatDataType); + } + + private RowData.FieldGetter[] getFieldGetters( + List physicalChildren, int[] projection) { + return Arrays.stream(projection) + .mapToObj( + targetField -> + RowData.createFieldGetter( + physicalChildren.get(targetField), targetField)) + .toArray(RowData.FieldGetter[]::new); + } + + public void setWritableMetadataKeys(List writableMetadataKeys) { + this.writableMetadataKeys = writableMetadataKeys; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSerializationSchemaFactory that = (PulsarTableSerializationSchemaFactory) o; + return Objects.equals(physicalDataType, that.physicalDataType) + && Objects.equals(keyEncodingFormat, that.keyEncodingFormat) + && Arrays.equals(keyProjection, that.keyProjection) + && Objects.equals(valueEncodingFormat, that.valueEncodingFormat) + && Arrays.equals(valueProjection, that.valueProjection) + && Objects.equals(writableMetadataKeys, that.writableMetadataKeys); + } + + @Override + public int hashCode() { + int result = + Objects.hash( + physicalDataType, + keyEncodingFormat, + valueEncodingFormat, + writableMetadataKeys); + result = 31 * result + Arrays.hashCode(keyProjection); + result = 31 * result + Arrays.hashCode(valueProjection); + return result; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java new file mode 100644 index 0000000000000..b290967873e5f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSink.java @@ -0,0 +1,171 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.sink.PulsarSink; +import org.apache.flink.connector.pulsar.sink.PulsarSinkBuilder; +import org.apache.flink.connector.pulsar.sink.writer.delayer.FixedMessageDelayer; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.sink.abilities.SupportsWritingMetadata; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import javax.annotation.Nullable; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.stream.Stream; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Pulsar SQL Connector sink. It supports {@link SupportsWritingMetadata}. */ +public class PulsarTableSink implements DynamicTableSink, SupportsWritingMetadata { + + private final PulsarTableSerializationSchemaFactory serializationSchemaFactory; + + private final ChangelogMode changelogMode; + + private final List topics; + + private final Properties properties; + + private final DeliveryGuarantee deliveryGuarantee; + + @Nullable private final TopicRouter topicRouter; + + private final TopicRoutingMode topicRoutingMode; + + private final long messageDelayMillis; + + public PulsarTableSink( + PulsarTableSerializationSchemaFactory serializationSchemaFactory, + ChangelogMode changelogMode, + List topics, + Properties properties, + DeliveryGuarantee deliveryGuarantee, + @Nullable TopicRouter topicRouter, + TopicRoutingMode topicRoutingMode, + long messageDelayMillis) { + this.serializationSchemaFactory = checkNotNull(serializationSchemaFactory); + this.changelogMode = checkNotNull(changelogMode); + this.topics = checkNotNull(topics); + // Mutable attributes + this.properties = checkNotNull(properties); + this.deliveryGuarantee = checkNotNull(deliveryGuarantee); + this.topicRouter = topicRouter; + this.topicRoutingMode = checkNotNull(topicRoutingMode); + this.messageDelayMillis = messageDelayMillis; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + return this.changelogMode; + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + + final PulsarSerializationSchema pulsarSerializationSchema = + serializationSchemaFactory.createPulsarSerializationSchema(context); + + final PulsarSinkBuilder pulsarSinkBuilder = + PulsarSink.builder() + .setSerializationSchema(pulsarSerializationSchema) + .setProperties(properties) + .setDeliveryGuarantee(deliveryGuarantee) + .setTopics(topics) + .setTopicRoutingMode(topicRoutingMode) + .delaySendingMessage(new FixedMessageDelayer<>(messageDelayMillis)); + + if (topicRouter != null) { + pulsarSinkBuilder.setTopicRouter(topicRouter); + } + return SinkV2Provider.of(pulsarSinkBuilder.build()); + } + + @Override + public String asSummaryString() { + return "Pulsar dynamic table sink"; + } + + @Override + public Map listWritableMetadata() { + final Map metadataMap = new LinkedHashMap<>(); + Stream.of(PulsarWritableMetadata.WritableMetadata.values()) + .forEachOrdered(m -> metadataMap.put(m.key, m.dataType)); + return metadataMap; + } + + @Override + public void applyWritableMetadata(List metadataKeys, DataType consumedDataType) { + this.serializationSchemaFactory.setWritableMetadataKeys(metadataKeys); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSink that = (PulsarTableSink) o; + return Objects.equals(serializationSchemaFactory, that.serializationSchemaFactory) + && Objects.equals(changelogMode, that.changelogMode) + && Objects.equals(topics, that.topics) + && Objects.equals(properties, that.properties) + && deliveryGuarantee == that.deliveryGuarantee + && Objects.equals(topicRouter, that.topicRouter) + && topicRoutingMode == that.topicRoutingMode + && messageDelayMillis == that.messageDelayMillis; + } + + @Override + public int hashCode() { + return Objects.hash( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + } + + @Override + public DynamicTableSink copy() { + final PulsarTableSink copy = + new PulsarTableSink( + serializationSchemaFactory, + changelogMode, + topics, + properties, + deliveryGuarantee, + topicRouter, + topicRoutingMode, + messageDelayMillis); + return copy; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java new file mode 100644 index 0000000000000..8f25db73ec442 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarWritableMetadata.java @@ -0,0 +1,140 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.sink; + +import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; + +import org.bouncycastle.util.Arrays; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Stream; + +/** A class used to manage WritableMetadata for Pulsar SQL sink connector. */ +public class PulsarWritableMetadata implements Serializable { + + private static final long serialVersionUID = 8117156158379846715L; + + private final List writableMetadataKeys; + + private final int physicalChildrenSize; + + /** + * Contains the position for each value of {@link WritableMetadata} in the consumed row or -1 if + * this metadata key is not used. + */ + private int[] metadataPositions; + + public PulsarWritableMetadata(List writableMetadataKeys, int physicalChildrenSize) { + this.writableMetadataKeys = writableMetadataKeys; + this.physicalChildrenSize = physicalChildrenSize; + this.metadataPositions = getMetadataPositions(); + } + + public void applyWritableMetadataInMessage( + RowData consumedRow, PulsarMessageBuilder messageBuilder) { + Map properties = readMetadata(consumedRow, WritableMetadata.PROPERTIES); + if (properties != null) { + messageBuilder.properties(properties); + } + final Long eventTime = readMetadata(consumedRow, WritableMetadata.EVENT_TIME); + if (eventTime != null && eventTime >= 0) { + messageBuilder.eventTime(eventTime); + } + } + + @SuppressWarnings("unchecked") + private T readMetadata(RowData consumedRow, WritableMetadata metadata) { + if (Arrays.isNullOrEmpty(metadataPositions)) { + return null; + } + final int pos = metadataPositions[metadata.ordinal()]; + if (pos < 0) { + return null; + } + return (T) metadata.converter.read(consumedRow, pos); + } + + private int[] getMetadataPositions() { + return Stream.of(PulsarWritableMetadata.WritableMetadata.values()) + .mapToInt( + m -> { + final int pos = writableMetadataKeys.indexOf(m.key); + if (pos < 0) { + return -1; + } + return physicalChildrenSize + pos; + }) + .toArray(); + } + + /** A list of writable metadata used by Pulsar SQL sink connector. */ + public enum WritableMetadata { + PROPERTIES( + "properties", + // key and value of the map are nullable to make handling easier in queries + DataTypes.MAP(DataTypes.STRING().nullable(), DataTypes.STRING().nullable()) + .nullable(), + (row, pos) -> { + if (row.isNullAt(pos)) { + return null; + } + final MapData map = row.getMap(pos); + final ArrayData keyArray = map.keyArray(); + final ArrayData valueArray = map.valueArray(); + + final Properties properties = new Properties(); + for (int i = 0; i < keyArray.size(); i++) { + if (!keyArray.isNullAt(i) && !valueArray.isNullAt(i)) { + final String key = keyArray.getString(i).toString(); + final String value = valueArray.getString(i).toString(); + properties.put(key, value); + } + } + return properties; + }), + + EVENT_TIME( + "event_time", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(3).nullable(), + (row, pos) -> { + if (row.isNullAt(pos)) { + return null; + } + return row.getTimestamp(pos, 3).getMillisecond(); + }); + public final String key; + + public final DataType dataType; + + public final PulsarTableSerializationSchema.MetadataConverter converter; + + WritableMetadata( + String key, + DataType dataType, + PulsarTableSerializationSchema.MetadataConverter converter) { + this.key = key; + this.dataType = dataType; + this.converter = converter; + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java new file mode 100644 index 0000000000000..e0a7f9dc77557 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarReadableMetadata.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.source; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.Message; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * Class for reading metadata fields from a Pulsar message and put in corresponding Flink row + * fields. + * + *

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

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

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

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

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

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

    {@link PulsarTableSource} + */ +public class PulsarTableSource implements ScanTableSource, SupportsReadingMetadata { + // -------------------------------------------------------------------------------------------- + // Format attributes + // -------------------------------------------------------------------------------------------- + + private static final String FORMAT_METADATA_PREFIX = "value."; + + private final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory; + + /** + * Usually it is the same as the valueDecodingFormat, but use a different naming to show that it + * is used to list all the format metadata keys. + */ + private final DecodingFormat> decodingFormatForReadingMetadata; + + // -------------------------------------------------------------------------------------------- + // PulsarSource needed attributes + // -------------------------------------------------------------------------------------------- + + private final List topics; + + private final Properties properties; + + private final StartCursor startCursor; + + private final SubscriptionType subscriptionType; + + public PulsarTableSource( + PulsarTableDeserializationSchemaFactory deserializationSchemaFactory, + DecodingFormat> decodingFormatForReadingMetadata, + List topics, + Properties properties, + StartCursor startCursor, + SubscriptionType subscriptionType) { + // Format attributes + this.deserializationSchemaFactory = checkNotNull(deserializationSchemaFactory); + this.decodingFormatForReadingMetadata = checkNotNull(decodingFormatForReadingMetadata); + // DataStream connector attributes + this.topics = topics; + this.properties = checkNotNull(properties); + this.startCursor = checkNotNull(startCursor); + this.subscriptionType = checkNotNull(subscriptionType); + } + + @Override + public ChangelogMode getChangelogMode() { + return decodingFormatForReadingMetadata.getChangelogMode(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { + PulsarDeserializationSchema deserializationSchema = + deserializationSchemaFactory.createPulsarDeserialization(context); + PulsarSource source = + PulsarSource.builder() + .setTopics(topics) + .setStartCursor(startCursor) + .setDeserializationSchema(deserializationSchema) + .setSubscriptionType(subscriptionType) + .setProperties(properties) + .build(); + return SourceProvider.of(source); + } + + /** + * According to convention, the order of the final row must be PHYSICAL + FORMAT METADATA + + * CONNECTOR METADATA where the format metadata has the highest precedence. + * + * @return + */ + @Override + public Map listReadableMetadata() { + final Map allMetadataMap = new LinkedHashMap<>(); + + // add value format metadata with prefix + decodingFormatForReadingMetadata + .listReadableMetadata() + .forEach((key, value) -> allMetadataMap.put(FORMAT_METADATA_PREFIX + key, value)); + // add connector metadata + Stream.of(PulsarReadableMetadata.ReadableMetadata.values()) + .forEachOrdered(m -> allMetadataMap.putIfAbsent(m.key, m.dataType)); + + return allMetadataMap; + } + + @Override + public void applyReadableMetadata(List allMetadataKeys, DataType producedDataType) { + // separate connector and format metadata + final List formatMetadataKeys = + allMetadataKeys.stream() + .filter(k -> k.startsWith(FORMAT_METADATA_PREFIX)) + .collect(Collectors.toList()); + + final List connectorMetadataKeys = new ArrayList<>(allMetadataKeys); + connectorMetadataKeys.removeAll(formatMetadataKeys); + + // push down format metadata + final Map formatMetadata = + decodingFormatForReadingMetadata.listReadableMetadata(); + if (formatMetadata.size() > 0) { + final List requestedFormatMetadataKeys = + formatMetadataKeys.stream() + .map(k -> k.substring(FORMAT_METADATA_PREFIX.length())) + .collect(Collectors.toList()); + decodingFormatForReadingMetadata.applyReadableMetadata(requestedFormatMetadataKeys); + } + + // update the factory attributes. + deserializationSchemaFactory.setConnectorMetadataKeys(connectorMetadataKeys); + deserializationSchemaFactory.setProducedDataType(producedDataType); + } + + @Override + public String asSummaryString() { + return "Pulsar table source"; + } + + @Override + public DynamicTableSource copy() { + final PulsarTableSource copy = + new PulsarTableSource( + deserializationSchemaFactory, + decodingFormatForReadingMetadata, + topics, + properties, + startCursor, + subscriptionType); + return copy; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + PulsarTableSource that = (PulsarTableSource) o; + return Objects.equals(deserializationSchemaFactory, that.deserializationSchemaFactory) + && Objects.equals( + decodingFormatForReadingMetadata, that.decodingFormatForReadingMetadata) + && Objects.equals(topics, that.topics) + && Objects.equals(properties, that.properties) + && Objects.equals(startCursor, that.startCursor) + && subscriptionType == that.subscriptionType; + } + + @Override + public int hashCode() { + return Objects.hash( + deserializationSchemaFactory, + decodingFormatForReadingMetadata, + topics, + properties, + startCursor, + subscriptionType); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 0000000000000..b153b56175670 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.flink.connector.pulsar.table.PulsarTableFactory diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java index 7565193ca9a42..b99a631d92548 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/architecture/TestCodeArchitectureTest.java @@ -34,7 +34,6 @@ ImportOptions.ExcludeShadedImportOption.class }) public class TestCodeArchitectureTest { - @ArchTest public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java index e4b2dc1b9e9cf..d67b64ceac9e7 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/factories/JSONSchemaFactoryTest.java @@ -20,9 +20,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; -import org.apache.flink.connector.pulsar.common.schema.PulsarSchemaTypeInformation; import org.apache.flink.connector.pulsar.testutils.SampleData.FL; import org.apache.flink.connector.pulsar.testutils.SampleData.Foo; +import org.apache.flink.formats.avro.typeutils.AvroTypeInfo; import org.apache.flink.util.InstantiationUtil; import org.apache.pulsar.client.api.Schema; @@ -62,7 +62,7 @@ void createJSONTypeInformationFromSchemaInfo() { TypeInformation typeInfo = factory.createTypeInfo(pulsarSchema.getSchemaInfo()); assertThat(typeInfo) - .isInstanceOf(PulsarSchemaTypeInformation.class) + .isInstanceOf(AvroTypeInfo.class) .hasFieldOrPropertyWithValue("typeClass", FL.class); // TypeInformation serialization. diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java index 9a72c8ace2a19..0259773445ee1 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java @@ -32,14 +32,15 @@ import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.test.junit5.MiniClusterExtension; -import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.extension.ExtendWith; /** Unite test class for {@link PulsarSource}. */ @SuppressWarnings("unused") +@ExtendWith(MiniClusterExtension.class) class PulsarSourceITCase extends SourceTestSuiteBase { - @Disabled // TODO: remove override after FLINK-26177 is fixed @Override public void testScaleUp( TestEnvironment testEnv, @@ -49,7 +50,6 @@ public void testScaleUp( super.testScaleUp(testEnv, externalContext, semantic); } - @Disabled // TODO: remove override after FLINK-26177 is fixed @Override public void testScaleDown( TestEnvironment testEnv, diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java index 18888df64b52a..3478b5c235c0f 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaTest.java @@ -18,13 +18,21 @@ package org.apache.flink.connector.pulsar.source.reader.deserializer; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.connector.pulsar.SampleMessage.TestMessage; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.PulsarSourceOptions; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; import org.apache.flink.connector.testutils.source.deserialization.TestingDeserializationContext; import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.types.StringValue; +import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.function.FunctionWithException; @@ -33,9 +41,12 @@ import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.MessageImpl; import org.apache.pulsar.common.api.proto.MessageMetadata; +import org.apache.pulsar.common.schema.KeyValue; import org.junit.jupiter.api.Test; +import java.io.Serializable; import java.nio.ByteBuffer; +import java.util.Objects; import java.util.concurrent.ThreadLocalRandom; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; @@ -44,13 +55,16 @@ import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; import static org.apache.flink.util.Preconditions.checkState; import static org.apache.pulsar.client.api.Schema.PROTOBUF_NATIVE; +import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.Mockito.mock; /** Unit tests for {@link PulsarDeserializationSchema}. */ -class PulsarDeserializationSchemaTest { +class PulsarDeserializationSchemaTest extends PulsarTestSuiteBase { @Test void createFromFlinkDeserializationSchema() throws Exception { @@ -108,6 +122,243 @@ void createFromFlinkTypeInformation() throws Exception { assertEquals(collector.result, "test-content"); } + @Test + void primitiveStringPulsarSchema() { + final String topicName = + "primitiveString-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + String expectedMessage = randomAlphabetic(10); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.STRING, + expectedMessage); + PulsarSource source = createSource(topicName, pulsarSchema(Schema.STRING)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void unversionedJsonStructPulsarSchema() { + final String topicName = + "unversionedJsonStruct-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + TestingUser expectedMessage = createRandomUser(); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.JSON(TestingUser.class), + expectedMessage); + PulsarSource source = + createSource( + topicName, pulsarSchema(Schema.JSON(TestingUser.class), TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValueJsonStructPulsarSchema() { + final String topicName = + "keyValueJsonStruct-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(createRandomUser(), createRandomUser()); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue( + Schema.JSON(TestingUser.class), Schema.JSON(TestingUser.class)), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue( + Schema.JSON(TestingUser.class), + Schema.JSON(TestingUser.class)), + TestingUser.class, + TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValueAvroStructPulsarSchema() { + final String topicName = + "keyValueAvroStruct-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(createRandomUser(), createRandomUser()); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue( + Schema.AVRO(TestingUser.class), Schema.AVRO(TestingUser.class)), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue( + Schema.AVRO(TestingUser.class), + Schema.AVRO(TestingUser.class)), + TestingUser.class, + TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValuePrimitivePulsarSchema() { + final String topicName = + "keyValuePrimitive-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = new KeyValue<>(randomAlphabetic(5), 5); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue(Schema.STRING, Schema.INT32), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue(Schema.STRING, Schema.INT32), + String.class, + Integer.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValuePrimitiveKeyStructValuePulsarSchema() { + final String topicName = + "primitiveKeyStructValue-" + + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(randomAlphabetic(5), createRandomUser()); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue(Schema.STRING, Schema.JSON(TestingUser.class)), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue(Schema.STRING, Schema.JSON(TestingUser.class)), + String.class, + TestingUser.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void keyValueStructKeyPrimitiveValuePulsarSchema() { + final String topicName = + "structKeyPrimitiveValue-" + + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + KeyValue expectedMessage = + new KeyValue<>(createRandomUser(), randomAlphabetic(5)); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.KeyValue(Schema.JSON(TestingUser.class), Schema.STRING), + expectedMessage); + PulsarSource> source = + createSource( + topicName, + pulsarSchema( + Schema.KeyValue(Schema.JSON(TestingUser.class), Schema.STRING), + TestingUser.class, + String.class)); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + @Test + void simpleFlinkSchema() { + final String topicName = + "simpleFlinkSchema-" + ThreadLocalRandom.current().nextLong(0, Long.MAX_VALUE); + operator().createTopic(topicName, 1); + String expectedMessage = randomAlphabetic(5); + operator() + .sendMessage( + TopicNameUtils.topicNameWithPartition(topicName, 0), + Schema.STRING, + expectedMessage); + PulsarSource source = + createSource(topicName, flinkSchema(new SimpleStringSchema())); + assertThatCode(() -> runPipeline(source, expectedMessage)).doesNotThrowAnyException(); + } + + private PulsarSource createSource( + String topicName, PulsarDeserializationSchema deserializationSchema) { + return PulsarSource.builder() + .setDeserializationSchema(deserializationSchema) + .setServiceUrl(operator().serviceUrl()) + .setAdminUrl(operator().adminUrl()) + .setTopics(topicName) + .setSubscriptionType(Exclusive) + .setSubscriptionName(topicName + "-subscription") + .setBoundedStopCursor(StopCursor.latest()) + .setConfig(PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, -1L) + .build(); + } + + private void runPipeline(PulsarSource source, T expected) throws Exception { + try (CloseableIterator iterator = + StreamExecutionEnvironment.getExecutionEnvironment() + .setParallelism(1) + .fromSource(source, WatermarkStrategy.noWatermarks(), "testSource") + .executeAndCollect()) { + assertThat(iterator).hasNext(); + assertThat(iterator.next()).isEqualTo(expected); + } + } + + /** A test POJO class. */ + public static class TestingUser implements Serializable { + private static final long serialVersionUID = -1123545861004770003L; + public String name; + public Integer age; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Integer getAge() { + return age; + } + + public void setAge(Integer age) { + this.age = age; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestingUser that = (TestingUser) o; + return Objects.equals(name, that.name) && Objects.equals(age, that.age); + } + + @Override + public int hashCode() { + return Objects.hash(name, age); + } + } + + private TestingUser createRandomUser() { + TestingUser user = new TestingUser(); + user.setName(randomAlphabetic(5)); + user.setAge(ThreadLocalRandom.current().nextInt(0, Integer.MAX_VALUE)); + return user; + } + /** Create a test message by given bytes. The message don't contains any meta data. */ private Message getMessage( T message, FunctionWithException decoder) throws Exception { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java index 538e45826d7fc..e1e31b6de41fd 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java @@ -136,7 +136,8 @@ private void seekStartPositionAndHandleSplit( SplitsAddition addition = new SplitsAddition<>(singletonList(split)); // create consumer and seek before split changes - try (Consumer consumer = reader.createPulsarConsumer(partition)) { + try (Consumer consumer = + (Consumer) reader.createPulsarConsumer(partition)) { // inclusive messageId StartCursor startCursor = StartCursor.fromMessageId(startPosition); startCursor.seekPosition(partition.getTopic(), partition.getPartitionId(), consumer); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java new file mode 100644 index 0000000000000..440abfa6f4e3e --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java @@ -0,0 +1,427 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.sink.PulsarSink; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; +import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; +import org.apache.flink.connector.pulsar.table.source.PulsarTableSource; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.catalog.WatermarkSpec; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.format.DecodingFormat; +import org.apache.flink.table.connector.format.EncodingFormat; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkV2Provider; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.SourceProvider; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.utils.ResolvedExpressionMock; +import org.apache.flink.table.factories.TestFormatFactory; +import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; +import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.assertj.core.util.Lists; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SERVICE_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; +import static org.apache.flink.table.factories.FactoryUtil.FORMAT; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; +import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSource; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * This test aims to verify that {@link PulsarTableFactory} can consume proper config options and + * produce expected {@link PulsarTableSource} and {@link PulsarTableSink}. It guarantees that config + * options is used internally by the implementation classes. + */ +public class PulsarTableFactoryTest { + private static final String TEST_TOPIC = "test-topic"; + private static final String TEST_ADMIN_URL = "http://my-broker.example.com:8080"; + private static final String TEST_SERVICE_URL = "pulsar://localhost:6650"; + private static final String TEST_SUBSCRIPTION_NAME = "default-subscription"; + + private static final String NAME = "name"; + private static final String COUNT = "count"; + private static final String TIME = "time"; + private static final String METADATA = "metadata"; + private static final String WATERMARK_EXPRESSION = TIME + " - INTERVAL '5' SECOND"; + private static final DataType WATERMARK_DATATYPE = DataTypes.TIMESTAMP(3); + private static final String COMPUTED_COLUMN_NAME = "computed-column"; + private static final String COMPUTED_COLUMN_EXPRESSION = COUNT + " + 1.0"; + private static final DataType COMPUTED_COLUMN_DATATYPE = DataTypes.DECIMAL(10, 3); + + private static final Properties EXPECTED_PULSAR_SOURCE_PROPERTIES = new Properties(); + private static final Properties EXPECTED_PULSAR_SINK_PROPERTIES = new Properties(); + + private static final String FORMAT_DELIMITER_KEY = + String.format("%s.%s", TestFormatFactory.IDENTIFIER, TestFormatFactory.DELIMITER.key()); + + private static final String FORMAT_FAIL_ON_MISSING_KEY = + String.format( + "%s.%s", TestFormatFactory.IDENTIFIER, TestFormatFactory.FAIL_ON_MISSING.key()); + + static { + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty(PULSAR_ADMIN_URL.key(), TEST_ADMIN_URL); + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty(PULSAR_SERVICE_URL.key(), TEST_SERVICE_URL); + EXPECTED_PULSAR_SOURCE_PROPERTIES.setProperty( + PULSAR_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + + EXPECTED_PULSAR_SINK_PROPERTIES.setProperty(PULSAR_ADMIN_URL.key(), TEST_ADMIN_URL); + EXPECTED_PULSAR_SINK_PROPERTIES.setProperty(PULSAR_SERVICE_URL.key(), TEST_SERVICE_URL); + } + + private static final ResolvedSchema SCHEMA = + new ResolvedSchema( + Arrays.asList( + Column.physical(NAME, DataTypes.STRING().notNull()), + Column.physical(COUNT, DataTypes.DECIMAL(38, 18)), + Column.physical(TIME, DataTypes.TIMESTAMP(3)), + Column.computed( + COMPUTED_COLUMN_NAME, + ResolvedExpressionMock.of( + COMPUTED_COLUMN_DATATYPE, COMPUTED_COLUMN_EXPRESSION))), + Collections.singletonList( + WatermarkSpec.of( + TIME, + ResolvedExpressionMock.of( + WATERMARK_DATATYPE, WATERMARK_EXPRESSION))), + null); + + private static final ResolvedSchema SCHEMA_WITH_METADATA = + new ResolvedSchema( + Arrays.asList( + Column.physical(NAME, DataTypes.STRING()), + Column.physical(COUNT, DataTypes.DECIMAL(38, 18)), + Column.metadata(TIME, DataTypes.TIMESTAMP(3), "publish_time", false), + Column.metadata( + METADATA, DataTypes.STRING(), "value.metadata_2", false)), + Collections.emptyList(), + null); + + private static final DataType SCHEMA_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); + + @Test + public void testTableSource() { + final Map configuration = getBasicSourceOptions(); + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA, configuration); + + final DecodingFormat> valueDecodingFormat = + new TestFormatFactory.DecodingFormatMock(",", true); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_DATA_TYPE, + null, + new int[0], + valueDecodingFormat, + new int[] {0, 1, 2}); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + valueDecodingFormat, + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + SubscriptionType.Exclusive); + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + + ScanTableSource.ScanRuntimeProvider provider = + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + assertPulsarSourceIsSameAsExpected(provider); + } + + @Test + public void testTableSourceWithKeyValue() { + final Map configuration = getSourceKeyValueOptions(); + + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA, configuration); + // initialize stateful testing formats + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + + final TestFormatFactory.DecodingFormatMock keyDecodingFormat = + new TestFormatFactory.DecodingFormatMock("#", false); + keyDecodingFormat.producedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING().notNull())).notNull(); + + final TestFormatFactory.DecodingFormatMock valueDecodingFormat = + new TestFormatFactory.DecodingFormatMock("|", false); + valueDecodingFormat.producedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD(TIME, DataTypes.TIMESTAMP(3))) + .notNull(); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_DATA_TYPE, + keyDecodingFormat, + new int[] {0}, + valueDecodingFormat, + new int[] {1, 2}); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + valueDecodingFormat, + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + SubscriptionType.Exclusive); + + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + } + + @Test + public void testTableSourceWithKeyValueAndMetadata() { + final Map options = getSourceKeyValueOptions(); + options.put("test-format.readable-metadata", "metadata_1:INT, metadata_2:STRING"); + + final PulsarTableSource actualPulsarSource = + (PulsarTableSource) createTableSource(SCHEMA_WITH_METADATA, options); + // initialize stateful testing formats + actualPulsarSource.applyReadableMetadata( + Arrays.asList("publish_time", "value.metadata_2"), + SCHEMA_WITH_METADATA.toSourceRowDataType()); + actualPulsarSource.getScanRuntimeProvider(ScanRuntimeProviderContext.INSTANCE); + + final TestFormatFactory.DecodingFormatMock expectedKeyFormat = + new TestFormatFactory.DecodingFormatMock( + "#", false, ChangelogMode.insertOnly(), Collections.emptyMap()); + expectedKeyFormat.producedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING())).notNull(); + + final Map expectedReadableMetadata = new HashMap<>(); + expectedReadableMetadata.put("metadata_1", DataTypes.INT()); + expectedReadableMetadata.put("metadata_2", DataTypes.STRING()); + + final TestFormatFactory.DecodingFormatMock expectedValueFormat = + new TestFormatFactory.DecodingFormatMock( + "|", false, ChangelogMode.insertOnly(), expectedReadableMetadata); + expectedValueFormat.producedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD("metadata_2", DataTypes.STRING())) + .notNull(); + expectedValueFormat.metadataKeys = Collections.singletonList("metadata_2"); + + final PulsarTableDeserializationSchemaFactory deserializationSchemaFactory = + new PulsarTableDeserializationSchemaFactory( + SCHEMA_WITH_METADATA.toPhysicalRowDataType(), + expectedKeyFormat, + new int[] {0}, + expectedValueFormat, + new int[] {1}); + + final PulsarTableSource expectedPulsarSource = + new PulsarTableSource( + deserializationSchemaFactory, + expectedValueFormat, + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SOURCE_PROPERTIES, + StartCursor.earliest(), + SubscriptionType.Exclusive); + + deserializationSchemaFactory.setProducedDataType( + SCHEMA_WITH_METADATA.toSourceRowDataType()); + deserializationSchemaFactory.setConnectorMetadataKeys( + Collections.singletonList("publish_time")); + + assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); + } + + @Test + public void testTableSink() { + final Map modifiedOptions = getBasicSinkOptions(); + final DynamicTableSink actualPulsarTableSink = createTableSink(SCHEMA, modifiedOptions); + + final EncodingFormat> valueEncodingFormat = + new TestFormatFactory.EncodingFormatMock(","); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + SCHEMA_DATA_TYPE, + null, + new int[0], + valueEncodingFormat, + new int[] {0, 1, 2}); + + final PulsarTableSink expectedPulsarTableSink = + new PulsarTableSink( + serializationSchemaFactory, + valueEncodingFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SINK_PROPERTIES, + DeliveryGuarantee.AT_LEAST_ONCE, + null, + TopicRoutingMode.ROUND_ROBIN, + 0); + assertThat(actualPulsarTableSink).isEqualTo(expectedPulsarTableSink); + + DynamicTableSink.SinkRuntimeProvider provider = + actualPulsarTableSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false)); + assertThat(provider).isInstanceOf(SinkV2Provider.class); + final SinkV2Provider sinkProvider = (SinkV2Provider) provider; + final Sink sinkFunction = sinkProvider.createSink(); + assertThat(sinkFunction).isInstanceOf(PulsarSink.class); + } + + @Test + public void testTableSinkWithKeyValue() { + final Map modifiedOptions = getSinkKeyValueOptions(); + final PulsarTableSink actualPulsarTableSink = + (PulsarTableSink) createTableSink(SCHEMA, modifiedOptions); + // initialize stateful testing formats + actualPulsarTableSink.getSinkRuntimeProvider(new SinkRuntimeProviderContext(false)); + + final TestFormatFactory.EncodingFormatMock keyEncodingFormat = + new TestFormatFactory.EncodingFormatMock("#"); + keyEncodingFormat.consumedDataType = + DataTypes.ROW(DataTypes.FIELD(NAME, DataTypes.STRING().notNull())).notNull(); + + final TestFormatFactory.EncodingFormatMock valueEncodingFormat = + new TestFormatFactory.EncodingFormatMock("|"); + valueEncodingFormat.consumedDataType = + DataTypes.ROW( + DataTypes.FIELD(COUNT, DataTypes.DECIMAL(38, 18)), + DataTypes.FIELD(TIME, DataTypes.TIMESTAMP(3))) + .notNull(); + + final PulsarTableSerializationSchemaFactory serializationSchemaFactory = + new PulsarTableSerializationSchemaFactory( + SCHEMA_DATA_TYPE, + keyEncodingFormat, + new int[] {0}, + valueEncodingFormat, + new int[] {1, 2}); + + final PulsarTableSink expectedPulsarTableSink = + new PulsarTableSink( + serializationSchemaFactory, + valueEncodingFormat.getChangelogMode(), + Lists.list(TEST_TOPIC), + EXPECTED_PULSAR_SINK_PROPERTIES, + DeliveryGuarantee.AT_LEAST_ONCE, + null, + TopicRoutingMode.ROUND_ROBIN, + 0); + assertThat(actualPulsarTableSink).isEqualTo(expectedPulsarTableSink); + } + + private static Map getBasicSourceOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + tableOptions.put(SOURCE_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, ","); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "true"); + return tableOptions; + } + + private static Map getSourceKeyValueOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + tableOptions.put(SOURCE_SUBSCRIPTION_NAME.key(), TEST_SUBSCRIPTION_NAME); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, "|"); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "false"); + + tableOptions.put(KEY_FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put("key." + FORMAT_DELIMITER_KEY, "#"); + tableOptions.put("key." + FORMAT_FAIL_ON_MISSING_KEY, "false"); + tableOptions.put(KEY_FIELDS.key(), NAME); + return tableOptions; + } + + private static Map getBasicSinkOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, ","); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "true"); + return tableOptions; + } + + private static Map getSinkKeyValueOptions() { + Map tableOptions = new HashMap<>(); + tableOptions.put(CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + tableOptions.put(TOPICS.key(), TEST_TOPIC); + tableOptions.put(ADMIN_URL.key(), TEST_ADMIN_URL); + tableOptions.put(SERVICE_URL.key(), TEST_SERVICE_URL); + // Format options. + tableOptions.put(FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put(FORMAT_DELIMITER_KEY, "|"); + tableOptions.put(FORMAT_FAIL_ON_MISSING_KEY, "false"); + + tableOptions.put(KEY_FORMAT.key(), TestFormatFactory.IDENTIFIER); + tableOptions.put("key." + FORMAT_DELIMITER_KEY, "#"); + tableOptions.put("key." + FORMAT_FAIL_ON_MISSING_KEY, "false"); + tableOptions.put(KEY_FIELDS.key(), NAME); + return tableOptions; + } + + private PulsarSource assertPulsarSourceIsSameAsExpected( + ScanTableSource.ScanRuntimeProvider provider) { + assertThat(provider).isInstanceOf(SourceProvider.class); + final SourceProvider sourceProvider = (SourceProvider) provider; + return (PulsarSource) sourceProvider.createSource(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java new file mode 100644 index 0000000000000..08cf89c014ff4 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -0,0 +1,295 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.SuccessException; +import org.apache.flink.types.Row; + +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows; +import static org.apache.flink.util.CollectionUtil.entry; +import static org.apache.flink.util.CollectionUtil.map; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * IT cases for the Pulsar table source and sink. It aims to verify runtime behaviour and certain + * use cases are correct and can produce/consume the desired records as user specifies. + */ +@ExtendWith(MiniClusterExtension.class) +public class PulsarTableITCase extends PulsarTableTestBase { + + private static final String JSON_FORMAT = "json"; + private static final String AVRO_FORMAT = "avro"; + private static final String CSV_FORMAT = "csv"; + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT}) + public void pulsarSourceSink(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "test_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + // ---------- Produce an event time stream into Pulsar ------------------- + + final String createTable = + String.format( + "create table pulsar_source_sink (\n" + + " `computed-price` as price + 1.0,\n" + + " price decimal(38, 18),\n" + + " currency string,\n" + + " log_date date,\n" + + " log_time time(3),\n" + + " log_ts timestamp(3),\n" + + " ts as log_ts + INTERVAL '1' SECOND,\n" + + " watermark for ts as ts\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " %s\n" + + ")", + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + formatOptions(format)); + + tableEnv.executeSql(createTable); + + String initialValues = + "INSERT INTO pulsar_source_sink\n" + + "SELECT CAST(price AS DECIMAL(10, 2)), currency, " + + " CAST(d AS DATE), CAST(t AS TIME(0)), CAST(ts AS TIMESTAMP(3))\n" + + "FROM (VALUES (2.02,'Euro','2019-12-12', '00:00:01', '2019-12-12 00:00:01.001001'), \n" + + " (1.11,'US Dollar','2019-12-12', '00:00:02', '2019-12-12 00:00:02.002001'), \n" + + " (50,'Yen','2019-12-12', '00:00:03', '2019-12-12 00:00:03.004001'), \n" + + " (3.1,'Euro','2019-12-12', '00:00:04', '2019-12-12 00:00:04.005001'), \n" + + " (5.33,'US Dollar','2019-12-12', '00:00:05', '2019-12-12 00:00:05.006001'), \n" + + " (0,'DUMMY','2019-12-12', '00:00:10', '2019-12-12 00:00:10'))\n" + + " AS orders (price, currency, d, t, ts)"; + tableEnv.executeSql(initialValues).await(); + + String query = + "SELECT\n" + + " CAST(TUMBLE_END(ts, INTERVAL '5' SECOND) AS VARCHAR),\n" + + " CAST(MAX(log_date) AS VARCHAR),\n" + + " CAST(MAX(log_time) AS VARCHAR),\n" + + " CAST(MAX(ts) AS VARCHAR),\n" + + " COUNT(*),\n" + + " CAST(MAX(price) AS DECIMAL(10, 2))\n" + + "FROM pulsar_source_sink\n" + + "GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)"; + + DataStream result = + tableEnv.toAppendStream(tableEnv.sqlQuery(query), RowData.class); + TestingSinkFunction sink = new TestingSinkFunction(2); + result.addSink(sink).setParallelism(1); + + try { + env.execute("Job_2"); + } catch (Throwable e) { + if (!isCausedByJobFinished(e)) { + // re-throw + throw e; + } + } + + List expected = + Arrays.asList( + "+I(2019-12-12 00:00:05.000,2019-12-12,00:00:03,2019-12-12 00:00:04.004,3,50.00)", + "+I(2019-12-12 00:00:10.000,2019-12-12,00:00:05,2019-12-12 00:00:06.006,2,5.33)"); + + assertThat(TestingSinkFunction.rows).isEqualTo(expected); + } + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT}) + public void pulsarSourceSinkWithKeyAndPartialValue(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "key_partial_value_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + // ---------- Produce an event time stream into Pulsar ------------------- + + // k_user_id and user_id have different data types to verify the correct mapping, + // fields are reordered on purpose + final String createTable = + String.format( + "CREATE TABLE pulsar_key_value (\n" + + " `user_id` BIGINT,\n" + + " `name` STRING,\n" + + " `event_id` BIGINT,\n" + + " `payload` STRING\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s',\n" + + " 'key.format' = '%s',\n" + + " 'key.fields' = 'user_id; event_id'\n" + + ")", + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format, + format); + + tableEnv.executeSql(createTable); + + String initialValues = + "INSERT INTO pulsar_key_value\n" + + "VALUES\n" + + " (1, 'name 1', 100, 'payload 1'),\n" + + " (2, 'name 2', 101, 'payload 2'),\n" + + " (3, 'name 3', 102, 'payload 3')"; + tableEnv.executeSql(initialValues).await(); + + final List result = + collectRows(tableEnv.sqlQuery("SELECT * FROM pulsar_key_value"), 3); + + assertThat(result) + .containsExactlyInAnyOrder( + Row.of(1L, "name 1", 100L, "payload 1"), + Row.of(2L, "name 2", 101L, "payload 2"), + Row.of(3L, "name 3", 102L, "payload 3")); + } + + @ParameterizedTest + @ValueSource(strings = {JSON_FORMAT}) + public void pulsarSourceSinkWithMetadata(String format) throws Exception { + // we always use a different topic name for each parameterized topic, + // in order to make sure the topic can be created. + final String topic = "metadata_topic_" + format + randomAlphanumeric(3); + createTestTopic(topic, 1); + + final String createTable = + String.format( + "CREATE TABLE pulsar_metadata (\n" + + " `physical_1` STRING,\n" + + " `physical_2` INT,\n" + + " `message_size` INT METADATA VIRTUAL,\n" + + " `event_time` TIMESTAMP(3) METADATA,\n" + + " `properties` MAP METADATA,\n" + + " `physical_3` BOOLEAN\n" + + ") WITH (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'pulsar.producer.producerName' = 'pulsar-table-test',\n" + + " 'format' = '%s'\n" + + ")", + PulsarTableFactory.IDENTIFIER, + topic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + tableEnv.executeSql(createTable); + + String initialValues = + "INSERT INTO pulsar_metadata\n" + + "VALUES\n" + + " ('data 1', 1, TIMESTAMP '2022-03-24 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE),\n" + + " ('data 2', 2, TIMESTAMP '2022-03-25 13:12:11.123', CAST(NULL AS MAP), FALSE),\n" + + " ('data 3', 3, TIMESTAMP '2022-03-26 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE)"; + tableEnv.executeSql(initialValues).await(); + + // ---------- Consume stream from Pulsar ------------------- + + final List result = collectRows(tableEnv.sqlQuery("SELECT * FROM pulsar_metadata"), 3); + assertThat(result) + .containsExactlyInAnyOrder( + Row.of( + "data 1", + 1, + 56, + LocalDateTime.parse("2022-03-24T13:12:11.123"), + map(entry("k1", "C0FFEE"), entry("k2", "BABE01")), + true), + Row.of( + "data 2", + 2, + 57, + LocalDateTime.parse("2022-03-25T13:12:11.123"), + Collections.emptyMap(), + false), + Row.of( + "data 3", + 3, + 56, + LocalDateTime.parse("2022-03-26T13:12:11.123"), + map(entry("k1", "C0FFEE"), entry("k2", "BABE01")), + true)); + } + + private static final class TestingSinkFunction implements SinkFunction { + + private static final long serialVersionUID = 455430015321124493L; + private static List rows = new ArrayList<>(); + + private final int expectedSize; + + private TestingSinkFunction(int expectedSize) { + this.expectedSize = expectedSize; + rows.clear(); + } + + @Override + public void invoke(RowData value, Context context) { + rows.add(value.toString()); + if (rows.size() >= expectedSize) { + // job finish + throw new SuccessException(); + } + } + } + + private static boolean isCausedByJobFinished(Throwable e) { + if (e instanceof SuccessException) { + return true; + } else if (e.getCause() != null) { + return isCausedByJobFinished(e.getCause()); + } else { + return false; + } + } + + private String formatOptions(String format) { + return String.format("'format' = '%s'", format); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java new file mode 100644 index 0000000000000..08e250b7a86db --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java @@ -0,0 +1,376 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; +import org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.factories.utils.FactoryMocks; +import org.apache.flink.table.types.DataType; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.impl.MessageIdImpl; +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_STATS_INTERVAL_SECONDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createKeyFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.createValueFormatProjection; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getMessageDelayMillis; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getPulsarProperties; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getSubscriptionType; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicListFromOptions; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRouter; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRoutingMode; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseMessageIdStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.api.DataTypes.FIELD; +import static org.apache.flink.table.api.DataTypes.INT; +import static org.apache.flink.table.api.DataTypes.ROW; +import static org.apache.flink.table.api.DataTypes.STRING; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + +/** + * Unit test for {@link PulsarTableOptionUtils}. Tests each method and different inputs. Some tests + * have overlapping semantics with {@link PulsarTableOptionsTest} and {@link + * PulsarTableValidationUtilsTest}, but they cover different aspects of the validation, so all of + * them should be kept. + */ +public class PulsarTableOptionUtilsTest { + // -------------------------------------------------------------------------------------------- + // Format and Projection Test + // -------------------------------------------------------------------------------------------- + @Test + void formatProjection() { + final DataType dataType = + DataTypes.ROW( + FIELD("id", INT()), + FIELD("name", STRING()), + FIELD("age", INT()), + FIELD("address", STRING())); + + final Map options = createTestOptions(); + options.put("key.fields", "address; name"); + + final Configuration config = Configuration.fromMap(options); + + assertThat(createKeyFormatProjection(config, dataType)).containsExactly(3, 1); + assertThat(createValueFormatProjection(config, dataType)).containsExactly(0, 2); + } + + @Test + void invalidKeyFormatFieldProjection() { + final DataType dataType = ROW(FIELD("id", INT()), FIELD("name", STRING())); + final Map options = createTestOptions(); + options.put("key.fields", "non_existing"); + + final Configuration config = Configuration.fromMap(options); + + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> createKeyFormatProjection(config, dataType)) + .withMessage( + String.format( + "Could not find the field '%s' in the table schema for usage in the key format. " + + "A key field must be a regular, physical column. " + + "The following columns can be selected in the '%s' option: [id, name]", + "non_existing", KEY_FIELDS.key())); + } + + static Map createTestOptions() { + final Map options = new HashMap<>(); + options.put("key.format", "test-format"); + options.put("key.test-format.delimiter", ","); + options.put("value.format", "test-format"); + options.put("value.test-format.delimiter", "|"); + options.put("value.test-format.fail-on-missing", "true"); + return options; + } + + // -------------------------------------------------------------------------------------------- + // Table Source Option Utils Test + // -------------------------------------------------------------------------------------------- + + @Test + void topicsList() { + final Map options = createDefaultOptions(); + options.put(TOPICS.key(), ";"); + List topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).isEmpty(); + + options.put(TOPICS.key(), "topic1;"); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).hasSize(1); + + options.put(TOPICS.key(), "topic1;topic2"); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).hasSize(2); + + options.put(TOPICS.key(), ""); + topicList = getTopicListFromOptions(Configuration.fromMap(options)); + assertThat(topicList).isEmpty(); + } + + @Test + void pulsarProperties() { + final Map options = createDefaultOptions(); + options.put(PULSAR_STATS_INTERVAL_SECONDS.key(), "30"); + Properties properties = getPulsarProperties(Configuration.fromMap(options)); + assertThat(properties.getProperty(PULSAR_STATS_INTERVAL_SECONDS.key())).isEqualTo("30"); + } + + @Test + void startCursor() { + // TDOO Use isEqualTo() to assert; need equals() method + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "earliest"); + StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:-1"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(MessageIdStartCursor.class); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "other"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage("MessageId format must be ledgerId:entryId:partitionId."); + + options.remove(SOURCE_START_FROM_MESSAGE_ID.key()); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545"); + startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); + + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage( + "Could not parse value '123545L' for key 'source.start.publish-time'."); + } + + @Test + void subscriptionType() { + final Map options = createDefaultOptions(); + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Shared"); + SubscriptionType subscriptionType = getSubscriptionType(Configuration.fromMap(options)); + assertThat(subscriptionType).isEqualTo(SubscriptionType.Shared); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + subscriptionType = getSubscriptionType(Configuration.fromMap(options)); + assertThat(subscriptionType).isEqualTo(SubscriptionType.Exclusive); + } + + @Test + void canParseMessageIdEarliestOrLatest() { + String earliest = "earliest"; + StartCursor startCursor = parseMessageIdStartCursor(earliest); + assertThat(startCursor).isEqualTo(StartCursor.earliest()); + + String latest = "latest"; + startCursor = parseMessageIdStartCursor(latest); + assertThat(startCursor).isEqualTo(StartCursor.latest()); + + String precise = "0:0:100"; + startCursor = parseMessageIdStartCursor(precise); + assertThat(startCursor).isEqualTo(StartCursor.fromMessageId(new MessageIdImpl(0, 0, 100))); + } + + @Test + void canParseMessageIdUsingMessageIdImpl() { + final String invalidFormatMessage = + "MessageId format must be ledgerId:entryId:partitionId."; + final String invalidNumberMessage = + "MessageId format must be ledgerId:entryId:partitionId. Each id should be able to parsed to long type."; + String precise = "0:0:100"; + StartCursor startCursor = parseMessageIdStartCursor(precise); + + String empty = ""; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(empty)) + .withMessage(invalidFormatMessage); + + String noSemicolon = "0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(noSemicolon)) + .withMessage(invalidFormatMessage); + + String oneSemiColon = "0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(oneSemiColon)) + .withMessage(invalidFormatMessage); + + String oneSemiColonComplete = "0:0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(oneSemiColonComplete)) + .withMessage(invalidFormatMessage); + + String twoSemiColon = "0:0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(twoSemiColon)) + .withMessage(invalidNumberMessage); + + String twoSemiColonComplete = "0:0:0"; + startCursor = parseMessageIdStartCursor(twoSemiColonComplete); + + String threeSemicolon = "0:0:0:"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(threeSemicolon)) + .withMessage(invalidNumberMessage); + + String threeSemicolonComplete = "0:0:0:0"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(threeSemicolonComplete)) + .withMessage(invalidNumberMessage); + + String invalidNumber = "0:0:adf"; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> parseMessageIdStartCursor(invalidNumber)) + .withMessage(invalidNumberMessage); + } + + @Test + void publishTimeStartCursor() { + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345"); + StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); + + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage("Could not parse value '12345L' for key 'source.start.publish-time'."); + } + + // -------------------------------------------------------------------------------------------- + // Table Sink Option Utils Test + // -------------------------------------------------------------------------------------------- + + @Test + void topicRouter() { + final Map options = createDefaultOptions(); + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter"); + TopicRouter topicRouter = + getTopicRouter(Configuration.fromMap(options), FactoryMocks.class.getClassLoader()); + assertThat(topicRouter).isInstanceOf(MockTopicRouter.class); + + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.PulsarTableOptionsTest"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + getTopicRouter( + Configuration.fromMap(options), + FactoryMocks.class.getClassLoader())) + .withMessage( + String.format( + "Sink TopicRouter class '%s' should extend from the required class %s", + PulsarTableOptionsTest.class.getName(), + TopicRouter.class.getName())); + + options.put( + SINK_CUSTOM_TOPIC_ROUTER.key(), + "org.apache.flink.connector.pulsar.table.testutils.NonExistMockTopicRouter"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + getTopicRouter( + Configuration.fromMap(options), + FactoryMocks.class.getClassLoader())) + .withMessage( + String.format( + "Could not find and instantiate TopicRouter class '%s'", + "org.apache.flink.connector.pulsar.table.testutils.NonExistMockTopicRouter")); + } + + @Test + void topicRoutingMode() { + final Map options = createDefaultOptions(); + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + TopicRoutingMode topicRoutingMode = getTopicRoutingMode(Configuration.fromMap(options)); + assertThat(topicRoutingMode).isEqualTo(TopicRoutingMode.ROUND_ROBIN); + + options.put(SINK_TOPIC_ROUTING_MODE.key(), "message-key-hash"); + topicRoutingMode = getTopicRoutingMode(Configuration.fromMap(options)); + assertThat(topicRoutingMode).isEqualTo(TopicRoutingMode.MESSAGE_KEY_HASH); + } + + @Test + void messageDelayMillis() { + final Map options = createDefaultOptions(); + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10 s"); + long messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofSeconds(10).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10s"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofSeconds(10).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1000ms"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMillis(1000).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 d"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofDays(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 H"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofHours(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1 min"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMinutes(1).toMillis()); + + options.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "1m"); + messageDelayMillis = getMessageDelayMillis(Configuration.fromMap(options)); + assertThat(messageDelayMillis).isEqualTo(Duration.ofMinutes(1).toMillis()); + } + + private Map createDefaultOptions() { + Map optionMap = new HashMap<>(); + return optionMap; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java new file mode 100644 index 0000000000000..500dd0f49716e --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java @@ -0,0 +1,420 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.connector.pulsar.table.testutils.MockPulsarAuthentication; +import org.apache.flink.table.api.ValidationException; + +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAM_MAP; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_INITIAL_POSITION; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** + * Test config options for Pulsar SQL connector. This test aims to verify legal combination of + * config options will be accepted and do not cause runtime exceptions (but cannot guarantee they + * are taking effect), and illegal combinations of config options will be rejected early. + */ +public class PulsarTableOptionsTest extends PulsarTableTestBase { + @Test + void noTopicsSpecified() { + final String topicName = randomTopicName(); + Map testConfigs = new HashMap<>(); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + "One or more required options are missing.\n\n" + + "Missing required options are:\n\n" + + "topics")); + } + + @Test + void invalidEmptyTopics() { + final String topicName = randomTopicName(); + Map testConfigs = new HashMap<>(); + + testConfigs.put(TOPICS.key(), ""); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, new ValidationException("The topics list should not be empty.")); + } + + @Test + void topicsWithSemicolon() { + final String topicName = randomTopicName(); + Map testConfigs = new HashMap<>(); + testConfigs.put(TOPICS.key(), topicName + ";"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidTopicName() { + final String topicName = randomTopicName(); + Map testConfigs = new HashMap<>(); + String invalidTopicName = "persistent://tenant/no-topic"; + testConfigs.put(TOPICS.key(), invalidTopicName); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + String.format( + "The topics name %s is not a valid topic name.", + invalidTopicName))); + } + + @Test + void topicsList() { + final String topicName = randomTopicName(); + Map testConfigs = new HashMap<>(); + testConfigs.put( + TOPICS.key(), + topicNameWithPartition(topicName, 0) + ";" + topicNameWithPartition(topicName, 1)); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void subscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidUnsupportedSubscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Key_Shared"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new ValidationException( + "Only Exclusive and Shared SubscriptionType is supported. ")); + } + + @Test + void invalidNonExistSubscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "random-subscription-type"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException("Invalid value for option 'source.subscription-type'.")); + } + + @Test + void messageIdStartCursorEarliest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "earliest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStartCursorLatest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStartCursorExact() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:-1"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidMessageIdStartCursorEmptyId() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId. " + + "Each id should be able to parsed to long type.")); + } + + @Test + void invalidMessageIdStartCursorIncomplete() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectException( + topicName, + new IllegalArgumentException( + "MessageId format must be ledgerId:entryId:partitionId.")); + } + + @Test + void timestampStartCursor() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "233010230"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void topicRoutingMode() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SINK_TOPIC_ROUTING_MODE.key(), "message-key-hash"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidTopicRouter() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + String invalidClassName = "invalid class name"; + testConfigs.put(SINK_CUSTOM_TOPIC_ROUTER.key(), invalidClassName); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException( + String.format( + "Could not find and instantiate TopicRouter class '%s'", + invalidClassName))); + } + + @Test + void messageDelay() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10s"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void invalidMessageDelay() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "invalid-duration"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException( + topicName, + new ValidationException("Invalid value for option 'sink.message-delay-interval'.")); + } + + // -------------------------------------------------------------------------------------------- + // PulsarSourceOptions Test + // -------------------------------------------------------------------------------------------- + @Test + void subscriptionInitialPosition() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(PULSAR_SUBSCRIPTION_INITIAL_POSITION.key(), "Earliest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + // -------------------------------------------------------------------------------------------- + // PulsarOptions, PulsarSourceOptions, PulsarSinkOptions Test + // -------------------------------------------------------------------------------------------- + + @Test + void pulsarOptionsAuthParamMap() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put(PULSAR_AUTH_PARAM_MAP.key(), "key1:value1,key2:value2"); + testConfigs.put( + PULSAR_AUTH_PLUGIN_CLASS_NAME.key(), MockPulsarAuthentication.class.getName()); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + // -------------------------------------------------------------------------------------------- + // requiredOptions(), optionalOptions() Test + // -------------------------------------------------------------------------------------------- + + @Test + void unusedConfigOptions() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + + testConfigs.put("random_config", "random_value"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectException(topicName, ValidationException.class); + } + + // -------------------------------------------------------------------------------------------- + // Utils methods + // -------------------------------------------------------------------------------------------- + + private String createTestConfig(Map configMap) { + StringBuilder sb = new StringBuilder(); + for (Map.Entry entry : configMap.entrySet()) { + sb.append(String.format(" '%s' = '%s' ,\n", entry.getKey(), entry.getValue())); + } + return sb.toString(); + } + + private void runSql(String topicName, String testConfigString) { + createTestTopic(topicName, 2); + final String createTable = + String.format( + "CREATE TABLE %s (\n" + + " `physical_1` STRING,\n" + + " `physical_2` INT,\n" + + " `physical_3` BOOLEAN\n" + + ") WITH (\n" + + " 'connector' = 'pulsar',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " %s" + + " 'format' = 'json'\n" + + ")", + topicName, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + testConfigString); + tableEnv.executeSql(createTable); + } + + private void runSinkAndExpectSucceed(String topicName) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatNoException().isThrownBy(() -> tableEnv.executeSql(initialValues).await()); + } + + private void runSinkAndExpectException( + String topicName, final Class exceptionType) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.executeSql(initialValues).await()); + } + + private void runSinkAndExpectException(String topicName, Throwable cause) { + String initialValues = + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TRUE),\n" + + " ('data 2', 2, FALSE),\n" + + " ('data 3', 3, TRUE)", + topicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.executeSql(initialValues).await()) + .withCause(cause); + } + + private void runSourceAndExpectSucceed(String topicName) { + assertThatNoException() + .isThrownBy(() -> tableEnv.sqlQuery(String.format("SELECT * FROM %s", topicName))); + } + + private void runSourceAndExpectException(String topicName, Throwable cause) { + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> tableEnv.sqlQuery(String.format("SELECT * FROM %s", topicName))) + .withCause(cause); + } + + private String randomTopicName() { + final String testTopicPrefix = "test_config_topic"; + return testTopicPrefix + randomAlphabetic(5); + } + + private Map testConfigWithTopic(String tableName) { + Map testConfigs = new HashMap<>(); + testConfigs.put(TOPICS.key(), tableName); + return testConfigs; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java new file mode 100644 index 0000000000000..90ba67a129261 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestInstance; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Base class for Pulsar table integration test. */ +@TestInstance(TestInstance.Lifecycle.PER_CLASS) +public abstract class PulsarTableTestBase { + // TODO fix the archunit violations + private static final Logger LOG = LoggerFactory.getLogger(PulsarTableTestBase.class); + + @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + + // Defines pulsar running environment + @TestExternalSystem + PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.container()); + + @TestSemantics + CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; + + protected StreamExecutionEnvironment env; + + protected StreamTableEnvironment tableEnv; + + @BeforeAll + public void beforeAll() { + pulsar.startUp(); + // run env + env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); + tableEnv = StreamTableEnvironment.create(env); + } + + public void createTestTopic(String topic, int numPartitions) { + pulsar.operator().createTopic(topic, numPartitions); + } + + @AfterAll + public void afterAll() { + pulsar.tearDown(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java new file mode 100644 index 0000000000000..27c1133068aa5 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtilsTest.java @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.ValidationException; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateKeyFormatConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateSinkRoutingConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateStartCursorConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateSubscriptionTypeConfigs; +import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTopicsConfigs; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** Unit test for {@link PulsarTableValidationUtils}. */ +public class PulsarTableValidationUtilsTest extends PulsarTableTestBase { + @Test + void topicsConfigs() { + final Map options = createDefaultOptions(); + options.put(TOPICS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage("The topics list should not be empty."); + + String invalidTopicName = "persistent://tenant/topic"; + String validTopicName = "valid-topic"; + + options.put(TOPICS.key(), invalidTopicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The topics name %s is not a valid topic name.", invalidTopicName)); + + options.put(TOPICS.key(), validTopicName + ";" + invalidTopicName); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The topics name %s is not a valid topic name.", invalidTopicName)); + + options.put(TOPICS.key(), validTopicName + ";" + validTopicName); + assertThatNoException() + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))); + + options.put(TOPICS.key(), validTopicName + ";"); + assertThatNoException() + .isThrownBy(() -> validateTopicsConfigs(Configuration.fromMap(options))); + } + + @Test + void startCursorConfigs() { + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + assertThatNoException() + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))); + + options.remove(SOURCE_START_FROM_MESSAGE_ID.key()); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "2345123234"); + assertThatNoException() + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "2345123234"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateStartCursorConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME)); + } + + @Test + void subscriptionTypeConfigs() { + final Map options = createDefaultOptions(); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); + assertThatNoException() + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Shared"); + assertThatNoException() + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Key_Shared"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only %s and %s SubscriptionType is supported. ", + SubscriptionType.Exclusive, SubscriptionType.Shared)); + + options.put(SOURCE_SUBSCRIPTION_TYPE.key(), "invalid-subscription"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> validateSubscriptionTypeConfigs(Configuration.fromMap(options))) + .withMessage( + "Could not parse value 'invalid-subscription' for key 'source.subscription-type'."); + } + + @Test + void sinkRoutingConfigs() { + final Map options = createDefaultOptions(); + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + assertThatNoException() + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))); + + // validation does not try to create the class + options.remove(SINK_TOPIC_ROUTING_MODE.key()); + options.put(SINK_CUSTOM_TOPIC_ROUTER.key(), "invalid-class-name"); + assertThatNoException() + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))); + + options.put(SINK_TOPIC_ROUTING_MODE.key(), "round-robin"); + options.put(SINK_CUSTOM_TOPIC_ROUTER.key(), "invalid-class-name"); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateSinkRoutingConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "Only one of %s and %s can be specified. Detected both of them", + SINK_CUSTOM_TOPIC_ROUTER, SINK_TOPIC_ROUTING_MODE)); + } + + @Test + void keyFormatConfigs() { + final Map options = createDefaultOptions(); + options.put(KEY_FIELDS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "The option '%s' can only be declared if a key format is defined using '%s'.", + KEY_FIELDS.key(), KEY_FORMAT.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.remove(KEY_FIELDS.key()); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.put(KEY_FIELDS.key(), ""); + assertThatExceptionOfType(ValidationException.class) + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))) + .withMessage( + String.format( + "A key format '%s' requires the declaration of one or more of key fields using '%s'.", + KEY_FORMAT.key(), KEY_FIELDS.key())); + + options.put(KEY_FORMAT.key(), "json"); + options.put(KEY_FIELDS.key(), "k_field1"); + assertThatNoException() + .isThrownBy(() -> validateKeyFormatConfigs(Configuration.fromMap(options))); + } + + private Map createDefaultOptions() { + Map optionMap = new HashMap<>(); + return optionMap; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java new file mode 100644 index 0000000000000..cbb2a867f19b3 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockPulsarAuthentication.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.testutils; + +import org.apache.pulsar.client.api.Authentication; +import org.apache.pulsar.client.api.AuthenticationDataProvider; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.auth.AuthenticationDataNull; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.CompletableFuture; + +/** A class to verify Pulsar authentication auth params map is created properly. way to do this */ +public class MockPulsarAuthentication implements Authentication { + public static String key1 = "key1"; + public static String key2 = "key2"; + public static String value1 = "value1"; + public static String value2 = "value2"; + + @Override + public String getAuthMethodName() { + return "custom authentication"; + } + + @Override + public AuthenticationDataProvider getAuthData() { + return new AuthenticationDataNull(); + } + + @Override + public AuthenticationDataProvider getAuthData(String brokerHostName) { + return new AuthenticationDataNull(); + } + + @Override + public void authenticationStage( + String requestUrl, + AuthenticationDataProvider authData, + Map previousResHeaders, + CompletableFuture> authFuture) { + Authentication.super.authenticationStage( + requestUrl, authData, previousResHeaders, authFuture); + } + + @Override + public Set> newRequestHeader( + String hostName, + AuthenticationDataProvider authData, + Map previousResHeaders) { + return new HashSet<>(); + } + + @Override + public void configure(Map authParams) { + assert Objects.equals(authParams.get(key1), value1); + assert Objects.equals(authParams.get(key2), value2); + } + + @Override + public void start() throws PulsarClientException {} + + @Override + public void close() throws IOException {} +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java new file mode 100644 index 0000000000000..b5ef53841aaee --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/MockTopicRouter.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.testutils; + +import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.table.data.RowData; + +import java.util.List; + +/** A mock topic Router for testing purposes only. */ +public class MockTopicRouter implements TopicRouter { + + private static final long serialVersionUID = 1316133122715449818L; + + @Override + public String route( + RowData rowData, String key, List partitions, PulsarSinkContext context) { + return "never-exist-topic"; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java new file mode 100644 index 0000000000000..f43be2d63eebf --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/PulsarTableTestUtils.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.testutils; + +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** Util class for verify testing results. */ +public class PulsarTableTestUtils { + public static List collectRows(Table table, int expectedSize) throws Exception { + final TableResult result = table.execute(); + final List collectedRows = new ArrayList<>(); + try (CloseableIterator iterator = result.collect()) { + while (collectedRows.size() < expectedSize && iterator.hasNext()) { + collectedRows.add(iterator.next()); + } + } + result.getJobClient() + .ifPresent( + jc -> { + try { + jc.cancel().get(5, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + return collectedRows; + } +} From aa525acc24594a574d37d6d470000b7854a130e9 Mon Sep 17 00:00:00 2001 From: "affe (Yufei Zhang)" Date: Tue, 14 Jun 2022 23:32:51 +0800 Subject: [PATCH 209/258] 98: add PulsarCatalog and tests (#107) * 98: add PulsarCatalog and tests --- .../flink-connector-pulsar/pom.xml | 30 +- .../pulsar/table/PulsarTableFactory.java | 9 +- .../pulsar/table/PulsarTableOptionUtils.java | 9 +- .../pulsar/table/PulsarTableOptions.java | 6 + .../table/PulsarTableValidationUtils.java | 4 +- .../pulsar/table/catalog/PulsarCatalog.java | 447 +++++++++ .../catalog/PulsarCatalogConfiguration.java | 38 + .../table/catalog/PulsarCatalogFactory.java | 103 ++ .../catalog/PulsarCatalogFactoryOptions.java | 65 ++ .../impl/IncompatibleSchemaException.java | 28 + .../table/catalog/impl/PulsarAdminTool.java | 218 +++++ .../catalog/impl/PulsarCatalogSupport.java | 316 ++++++ .../table/catalog/impl/SchemaTranslator.java | 237 +++++ .../catalog/utils/TableSchemaHelper.java | 196 ++++ .../org.apache.flink.table.factories.Factory | 2 + .../pulsar/table/PulsarTableITCase.java | 12 +- .../pulsar/table/PulsarTableTestBase.java | 13 +- .../table/catalog/PulsarCatalogITTest.java | 911 ++++++++++++++++++ .../pulsar/table/testutils/SchemaData.java | 27 + .../pulsar/table/testutils/TestingUser.java | 44 + .../pulsar/testutils/PulsarTestSuiteBase.java | 2 +- .../testutils/runtime/PulsarRuntime.java | 14 +- .../container/PulsarContainerRuntime.java | 16 +- .../flink-sql-connector-pulsar/pom.xml | 1 + 24 files changed, 2710 insertions(+), 38 deletions(-) create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogConfiguration.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactory.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/IncompatibleSchemaException.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarCatalogSupport.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/utils/TableSchemaHelper.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/SchemaData.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index 0d5bb8740a747..57feb0318fe08 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -47,7 +47,6 @@ under the License. - @@ -100,7 +99,22 @@ under the License. protobuf-java ${protoc.version} provided - true + + + + + + org.apache.flink + flink-json + ${project.version} + provided + + + + org.apache.flink + flink-avro + ${project.version} + provided @@ -167,18 +181,6 @@ under the License. - - org.apache.flink - flink-json - ${project.version} - test - - - org.apache.flink - flink-avro - ${project.version} - test - org.apache.flink flink-avro-confluent-registry diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java index 171836b063214..41cfe42599dcf 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java @@ -69,6 +69,7 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueEncodingFormat; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.EXPLICIT; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FORMAT; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SERVICE_URL; @@ -237,16 +238,15 @@ public String factoryIdentifier() { return IDENTIFIER; } - // TODO source_subscription_name has default value so it should be optional ? @Override public Set> requiredOptions() { - return Stream.of(TOPICS, ADMIN_URL, SERVICE_URL, FORMAT, SOURCE_SUBSCRIPTION_NAME) - .collect(Collectors.toSet()); + return Stream.of(TOPICS, ADMIN_URL, SERVICE_URL, FORMAT).collect(Collectors.toSet()); } @Override public Set> optionalOptions() { return Stream.of( + SOURCE_SUBSCRIPTION_NAME, SOURCE_SUBSCRIPTION_TYPE, SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME, @@ -255,7 +255,8 @@ public Set> optionalOptions() { SINK_MESSAGE_DELAY_INTERVAL, SINK_PARALLELISM, KEY_FORMAT, - KEY_FIELDS) + KEY_FIELDS, + EXPLICIT) .collect(Collectors.toSet()); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java index 611bccc75e2d7..c44879cd3a950 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java @@ -182,6 +182,14 @@ public static Properties getPulsarProperties(ReadableConfig tableOptions) { return pulsarProperties; } + public static Properties getPulsarProperties(Map configs) { + final Properties pulsarProperties = new Properties(); + configs.keySet().stream() + .filter(key -> key.startsWith("pulsar")) + .forEach(key -> pulsarProperties.put(key, configs.get(key))); + return pulsarProperties; + } + public static StartCursor getStartCursor(ReadableConfig tableOptions) { if (tableOptions.getOptional(SOURCE_START_FROM_MESSAGE_ID).isPresent()) { return parseMessageIdStartCursor(tableOptions.get(SOURCE_START_FROM_MESSAGE_ID)); @@ -261,7 +269,6 @@ public static TopicRouter getTopicRouter( } } - // TODO what if we use a topicRouter and set TopicRoutingMode to CUSTOM ? public static TopicRoutingMode getTopicRoutingMode(ReadableConfig readableConfig) { return readableConfig.get(SINK_TOPIC_ROUTING_MODE); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java index 91b0b0a1e7fba..d951adb29522f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java @@ -213,4 +213,10 @@ private PulsarTableOptions() {} code( "pulsar+ssl://pulsar.us-west.example.com:6651"))) .build()); + + public static final ConfigOption EXPLICIT = + ConfigOptions.key("explicit") + .booleanType() + .defaultValue(true) + .withDescription("Indicate if the table is an explict flink table"); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java index 12c5ff72e47bc..ca95eb7c6e8fa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java @@ -46,9 +46,7 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; import static org.apache.pulsar.common.naming.TopicName.isValid; -/** - * Util class for source and sink validation rules. TODO refactor to use the PulsarConfigValidator - */ +/** Util class for source and sink validation rules. */ public class PulsarTableValidationUtils { private PulsarTableValidationUtils() {} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java new file mode 100644 index 0000000000000..56a4465145a5b --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java @@ -0,0 +1,447 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog; + +import org.apache.flink.connector.pulsar.table.PulsarTableFactory; +import org.apache.flink.connector.pulsar.table.catalog.impl.IncompatibleSchemaException; +import org.apache.flink.connector.pulsar.table.catalog.impl.PulsarCatalogSupport; +import org.apache.flink.connector.pulsar.table.catalog.impl.SchemaTranslator; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.GenericInMemoryCatalog; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedCatalogView; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.factories.Factory; + +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Optional; + +/** + * Catalog implementation to use Pulsar to store metadatas for Flink tables/databases. + * + *

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

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

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

    Each topic(except Pulsar system topics) is mapped to a native table, and users can create + * arbitrary number of explicit tables that binds to one Pulsar topic besides the native table. + */ +public class PulsarCatalog extends GenericInMemoryCatalog { + private static final Logger LOG = LoggerFactory.getLogger(PulsarCatalog.class); + + private final PulsarCatalogConfiguration catalogConfiguration; + + private PulsarCatalogSupport catalogSupport; + + private final String flinkTenant; + + public static final String DEFAULT_TENANT = "__flink_catalog"; + + public static final String DEFAULT_DB = "default"; + + public PulsarCatalog( + String catalogName, + PulsarCatalogConfiguration catalogConfiguration, + String database, + String flinkTenant) { + super(catalogName, database); + this.catalogConfiguration = catalogConfiguration; + this.flinkTenant = flinkTenant; + } + + @Override + public Optional getFactory() { + return Optional.of(new PulsarTableFactory()); + } + + @Override + public void open() throws CatalogException { + if (catalogSupport == null) { + try { + catalogSupport = + new PulsarCatalogSupport( + catalogConfiguration, flinkTenant, new SchemaTranslator(false)); + } catch (PulsarAdminException e) { + throw new CatalogException( + "Failed to create Pulsar admin with configuration:" + + catalogConfiguration.toString(), + e); + } + } + } + + @Override + public void close() throws CatalogException { + if (catalogSupport != null) { + catalogSupport.close(); + catalogSupport = null; + LOG.info("Closed connection to Pulsar."); + } + } + + @Override + public List listDatabases() throws CatalogException { + try { + return catalogSupport.listDatabases(); + } catch (PulsarAdminException e) { + throw new CatalogException( + String.format("Failed to list all databases in catalog: %s", getName()), e); + } + } + + @Override + public CatalogDatabase getDatabase(String databaseName) throws CatalogException { + try { + return catalogSupport.getDatabase(databaseName); + } catch (PulsarAdminException e) { + throw new CatalogException( + String.format("Failed to get database info in catalog: %s", getName()), e); + } + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + try { + return catalogSupport.databaseExists(databaseName); + } catch (PulsarAdminException e) { + LOG.warn("Failed to check if database exists, encountered PulsarAdminError", e); + return false; + } catch (Exception e) { + LOG.error("Failed to check if database exists", e); + return false; + } + } + + @Override + public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + try { + catalogSupport.createDatabase(name, database); + } catch (PulsarAdminException.ConflictException e) { + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(getName(), name, e); + } + } catch (PulsarAdminException e) { + throw new CatalogException(String.format("Failed to create database %s", name), e); + } + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + try { + if (!listTables(name).isEmpty() && !cascade) { + throw new DatabaseNotEmptyException(getName(), name); + } + + // the cascade deletion relies on the pulsar namespace deletion will clear all topics + catalogSupport.dropDatabase(name); + } catch (PulsarAdminException.NotFoundException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } catch (PulsarAdminException e) { + throw new CatalogException(String.format("Failed to drop database %s", name), e); + } + } + + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException, CatalogException { + try { + return catalogSupport.listTables(databaseName); + } catch (PulsarAdminException.NotFoundException e) { + throw new DatabaseNotExistException(getName(), databaseName, e); + } catch (PulsarAdminException e) { + throw new CatalogException( + String.format("Failed to list tables in database %s", databaseName), e); + } + } + + @Override + public CatalogBaseTable getTable(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + if (tablePath.getObjectName().startsWith("_tmp_table_")) { + return super.getTable(tablePath); + } + try { + return catalogSupport.getTable(tablePath); + } catch (PulsarAdminException.NotFoundException e) { + throw new TableNotExistException(getName(), tablePath, e); + } catch (PulsarAdminException | IncompatibleSchemaException e) { + throw new CatalogException( + String.format("Failed to get table %s schema", tablePath.getFullName()), e); + } + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + if (tablePath.getObjectName().startsWith("_tmp_table_")) { + return super.tableExists(tablePath); + } + try { + return catalogSupport.tableExists(tablePath); + } catch (PulsarAdminException.NotFoundException e) { + return false; + } catch (PulsarAdminException e) { + throw new CatalogException( + String.format("Failed to check table %s existence", tablePath.getFullName()), + e); + } + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + if (tablePath.getObjectName().startsWith("_tmp_table_")) { + super.createTable(tablePath, table, ignoreIfExists); + } + + if (!databaseExists(tablePath.getDatabaseName())) { + throw new DatabaseNotExistException(getName(), tablePath.getDatabaseName()); + } + + if (tableExists(tablePath)) { + if (!ignoreIfExists) { + throw new TableAlreadyExistException(getName(), tablePath); + } else { + return; + } + } + + if (table instanceof ResolvedCatalogTable) { + try { + catalogSupport.createTable(tablePath, (ResolvedCatalogTable) table); + } catch (PulsarAdminException | IncompatibleSchemaException e) { + throw new CatalogException( + String.format("Failed to create table %s", tablePath.getFullName()), e); + } + } else if (table instanceof ResolvedCatalogView) { + throw new CatalogException( + String.format( + "Can't create view %s with catalog %s", + tablePath.getObjectName(), getName())); + } else { + throw new CatalogException( + String.format("Unknown Table Object kind: %s", table.getClass().getName())); + } + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + try { + catalogSupport.dropTable(tablePath); + } catch (PulsarAdminException.NotFoundException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath, e); + } else { + LOG.warn("The table {} does not exist. Drop table operation ignored", tablePath); + } + } catch (PulsarAdminException | RuntimeException e) { + throw new CatalogException( + String.format("Failed to drop table %s", tablePath.getFullName()), e); + } + } + + // ------------------------------------------------------------------------ + // Unsupported catalog operations for Pulsar + // There should not be such permission in the connector, it is very dangerous + // ------------------------------------------------------------------------ + + @Override + public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listViews(String databaseName) + throws DatabaseNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTable( + ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitionsByFilter( + ObjectPath tablePath, List expressions) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void createPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition partition, + boolean ignoreIfExists) + throws TableNotExistException, TableNotPartitionedException, + PartitionSpecInvalidException, PartitionAlreadyExistsException, + CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartition( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition newPartition, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public void alterTableStatistics( + ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableColumnStatistics( + ObjectPath tablePath, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogTableStatistics partitionStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionColumnStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + throw new UnsupportedOperationException(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogConfiguration.java new file mode 100644 index 0000000000000..30cf3482acc8e --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogConfiguration.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.common.config.PulsarConfiguration; + +/** The configuration class for {@link PulsarCatalog}. */ +public class PulsarCatalogConfiguration extends PulsarConfiguration { + private static final long serialVersionUID = 3139935676757015589L; + + /** + * Creates a new PulsarConfiguration, which holds a copy of the given configuration that can't + * be altered. PulsarCatalogConfiguration does not have extra configs besides {@link + * PulsarConfiguration} + * + * @param config The configuration with the original contents. + */ + public PulsarCatalogConfiguration(Configuration config) { + super(config); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactory.java new file mode 100644 index 0000000000000..6b4b091b0973d --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactory.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder; +import org.apache.flink.connector.pulsar.common.config.PulsarConfigValidator; +import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import java.util.Collections; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.AUTH_PARAMS; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.AUTH_PLUGIN; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.CATALOG_ADMIN_URL; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.CATALOG_SERVICE_URL; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.CATALOG_TENANT; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.DEFAULT_DATABASE; + +/** PulsarCatalogFactory implementing {@link CatalogFactory}. */ +public class PulsarCatalogFactory implements CatalogFactory { + + public static final String IDENTIFIER = "pulsar-catalog"; + + public static final PulsarConfigValidator CATALOG_CONFIG_VALIDATOR = + PulsarConfigValidator.builder().build(); + + @Override + public String factoryIdentifier() { + return IDENTIFIER; + } + + @Override + public Catalog createCatalog(Context context) { + final FactoryUtil.CatalogFactoryHelper helper = + FactoryUtil.createCatalogFactoryHelper(this, context); + + helper.validate(); + + PulsarConfigBuilder configBuilder = new PulsarConfigBuilder(); + ReadableConfig tableOptions = helper.getOptions(); + + configBuilder.set(PulsarOptions.PULSAR_ADMIN_URL, tableOptions.get(CATALOG_ADMIN_URL)); + configBuilder.set(PulsarOptions.PULSAR_SERVICE_URL, tableOptions.get(CATALOG_SERVICE_URL)); + if (tableOptions.getOptional(AUTH_PLUGIN).isPresent()) { + configBuilder.set( + PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME, tableOptions.get(AUTH_PLUGIN)); + } + + if (tableOptions.getOptional(AUTH_PARAMS).isPresent()) { + configBuilder.set(PulsarOptions.PULSAR_AUTH_PARAMS, tableOptions.get(AUTH_PARAMS)); + } + + PulsarCatalogConfiguration catalogConfiguration = + configBuilder.build(CATALOG_CONFIG_VALIDATOR, PulsarCatalogConfiguration::new); + + return new PulsarCatalog( + context.getName(), + catalogConfiguration, + helper.getOptions().get(DEFAULT_DATABASE), + helper.getOptions().get(CATALOG_TENANT)); + } + + @Override + public Set> requiredOptions() { + return Collections.emptySet(); + } + + @Override + public Set> optionalOptions() { + // pulsar catalog options + return Stream.of( + CATALOG_ADMIN_URL, + CATALOG_SERVICE_URL, + CATALOG_TENANT, + DEFAULT_DATABASE, + AUTH_PLUGIN, + AUTH_PARAMS) + .collect(Collectors.toSet()); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java new file mode 100644 index 0000000000000..7eecd74b0e77f --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.table.catalog.CommonCatalogOptions; + +/** {@link ConfigOption}s for {@link PulsarCatalog}. */ +@Internal +public final class PulsarCatalogFactoryOptions { + public static final ConfigOption CATALOG_TENANT = + ConfigOptions.key("catalog-tenant") + .stringType() + .defaultValue(PulsarCatalog.DEFAULT_TENANT) + .withDescription("Pulsar tenant used to store all table information"); + + public static final ConfigOption DEFAULT_DATABASE = + ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) + .stringType() + .defaultValue(PulsarCatalog.DEFAULT_DB); + + public static final ConfigOption CATALOG_ADMIN_URL = + ConfigOptions.key("catalog-admin-url") + .stringType() + .defaultValue("http://localhost:8080") + .withDescription("Required pulsar cluster admin url"); + + public static final ConfigOption CATALOG_SERVICE_URL = + ConfigOptions.key("catalog-service-url") + .stringType() + .defaultValue("pulsar://localhost:6650") + .withDescription("Required pulsar cluster service url"); + + public static final ConfigOption AUTH_PLUGIN = + ConfigOptions.key("catalog-auth-plugin") + .stringType() + .noDefaultValue() + .withDescription("Auth plugin name for accessing pulsar cluster"); + + public static final ConfigOption AUTH_PARAMS = + ConfigOptions.key("catalog-auth-params") + .stringType() + .noDefaultValue() + .withDescription("Auth params for accessing pulsar cluster"); + + private PulsarCatalogFactoryOptions() {} +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/IncompatibleSchemaException.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/IncompatibleSchemaException.java new file mode 100644 index 0000000000000..2816e65f13512 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/IncompatibleSchemaException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog.impl; + +/** Exception designates the incompatibility between pulsar and flink type. */ +public class IncompatibleSchemaException extends IllegalStateException { + private static final long serialVersionUID = -2073484724665131680L; + + public IncompatibleSchemaException(String message, Throwable e) { + super(message, e); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java new file mode 100644 index 0000000000000..39024eb4ac455 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java @@ -0,0 +1,218 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog.impl; + +import org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogConfiguration; +import org.apache.flink.connector.pulsar.table.catalog.utils.TableSchemaHelper; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.schema.BytesSchema; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.common.policies.data.PartitionedTopicInternalStats; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.schema.SchemaInfo; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; + +/** A class that wraps Pulsar Admin API. */ +public class PulsarAdminTool implements AutoCloseable { + + // system topics are not filtered out by default in Pulsar 2.10.0 + // this filter is incomplete and should be replaced by SystemTopicNames class + // after 2.10.1 released. + private static final String SYSTEM_TOPIC_PREFIX = "__"; + + private final PulsarAdmin admin; + + public PulsarAdminTool(PulsarCatalogConfiguration catalogConfiguration) { + this.admin = createAdmin(catalogConfiguration); + } + + @Override + public void close() { + admin.close(); + } + + public void createTenant(String tenant) throws PulsarAdminException { + Set clusters = new HashSet<>(admin.clusters().getClusters()); + admin.tenants() + .createTenant(tenant, TenantInfoImpl.builder().allowedClusters(clusters).build()); + } + + public boolean tenantExists(String tenant) throws PulsarAdminException { + try { + admin.tenants().getTenantInfo(tenant); + } catch (PulsarAdminException.NotFoundException e) { + return false; + } + return true; + } + + public List listNamespaces() throws PulsarAdminException { + List tenants = admin.tenants().getTenants(); + List namespaces = new ArrayList(); + for (String tenant : tenants) { + namespaces.addAll(admin.namespaces().getNamespaces(tenant)); + } + return namespaces; + } + + public boolean namespaceExists(String ns) throws PulsarAdminException { + try { + admin.namespaces().getTopics(ns); + } catch (PulsarAdminException.NotFoundException e) { + return false; + } + return true; + } + + public void createNamespace(String ns) throws PulsarAdminException { + createNamespace(ns, false); + } + + public void createNamespace(String ns, boolean retain) throws PulsarAdminException { + String nsName = NamespaceName.get(ns).toString(); + admin.namespaces().createNamespace(nsName); + if (retain) { + // retain the topic infinitely to store the metadata + admin.namespaces().setRetention(nsName, new RetentionPolicies(-1, -1)); + } + } + + public void updateNamespaceProperties(String ns, Map properties) + throws PulsarAdminException { + admin.namespaces().setProperties(ns, properties); + } + + public Map getNamespaceProperties(String ns) throws PulsarAdminException { + return admin.namespaces().getProperties(ns); + } + + public void deleteNamespace(String ns) throws PulsarAdminException { + String nsName = NamespaceName.get(ns).toString(); + admin.namespaces().deleteNamespace(nsName); + } + + public List getTopics(String ns) throws PulsarAdminException { + List nonPartitionedTopics = getNonPartitionedTopics(ns); + List partitionedTopics = admin.topics().getPartitionedTopicList(ns); + List allTopics = new ArrayList<>(); + Stream.of(partitionedTopics, nonPartitionedTopics).forEach(allTopics::addAll); + return allTopics.stream() + .map(t -> TopicName.get(t).getLocalName()) + .filter(topic -> !topic.startsWith(SYSTEM_TOPIC_PREFIX)) + .collect(Collectors.toList()); + } + + public boolean topicExists(String topicName) throws PulsarAdminException { + try { + PartitionedTopicMetadata partitionedTopicMetadata = + admin.topics().getPartitionedTopicMetadata(topicName); + if (partitionedTopicMetadata.partitions > 0) { + return true; + } + } catch (PulsarAdminException.NotFoundException e) { + return false; + } + return false; + } + + public void deleteTopic(String topicName) throws PulsarAdminException { + + try { + PartitionedTopicInternalStats partitionedInternalStats = + admin.topics().getPartitionedInternalStats(topicName); + final Optional any = + partitionedInternalStats.partitions.entrySet().stream() + .map(Map.Entry::getValue) + .filter(p -> !p.cursors.isEmpty()) + .findAny(); + if (any.isPresent()) { + throw new IllegalStateException( + String.format( + "The topic[%s] cannot be deleted because there are subscribers", + topicName)); + } + admin.topics().deletePartitionedTopic(topicName, true); + } catch (PulsarAdminException.NotFoundException e) { + // log.warn("topic<{}> is not exit, try delete force it", topicName); + admin.topics().delete(topicName, true); + } + } + + public void createTopic(String topicName, int partitionNum) + throws PulsarAdminException, IncompatibleSchemaException { + if (partitionNum > 0) { + admin.topics().createPartitionedTopic(topicName, partitionNum); + } else { + admin.topics().createNonPartitionedTopic(topicName); + } + } + + public void uploadSchema(String topicName, SchemaInfo schemaInfo) + throws IncompatibleSchemaException { + TableSchemaHelper.uploadPulsarSchema(admin, topicName, schemaInfo); + } + + public void deleteSchema(String topicName) { + TableSchemaHelper.deletePulsarSchema(admin, topicName); + } + + public SchemaInfo getPulsarSchema(String topic) { + try { + return admin.schemas().getSchemaInfo(TopicName.get(topic).toString()); + } catch (PulsarAdminException e) { + if (e.getStatusCode() == 404) { + return BytesSchema.of().getSchemaInfo(); + } else { + throw new IllegalStateException( + String.format( + "Failed to get schema information for %s", + TopicName.get(topic).toString()), + e); + } + } catch (Throwable e) { + throw new IllegalStateException( + String.format( + "Failed to get schema information for %s", + TopicName.get(topic).toString()), + e); + } + } + + private List getNonPartitionedTopics(String namespace) throws PulsarAdminException { + return admin.topics().getList(namespace).stream() + .filter(t -> !TopicName.get(t).isPartitioned()) + .collect(Collectors.toList()); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarCatalogSupport.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarCatalogSupport.java new file mode 100644 index 0000000000000..9395f3bea3e5b --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarCatalogSupport.java @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog.impl; + +import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.table.PulsarTableFactory; +import org.apache.flink.connector.pulsar.table.PulsarTableOptions; +import org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogConfiguration; +import org.apache.flink.connector.pulsar.table.catalog.utils.TableSchemaHelper; +import org.apache.flink.formats.raw.RawFormatFactory; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.factories.FactoryUtil; + +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.naming.NamespaceName; +import org.apache.pulsar.common.naming.TopicDomain; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.schema.SchemaInfo; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * This class is the implementation layer of catalog operations. It uses {@link PulsarAdminTool} to + * interact with Pulsar topics and manipulates metadata. {@link PulsarCatalogSupport} distinguish + * between explicit and native tables. + */ +public class PulsarCatalogSupport { + + private static final String DATABASE_COMMENT_KEY = "__database_comment"; + private static final String DATABASE_DESCRIPTION_KEY = "__database_description"; + private static final String DATABASE_DETAILED_DESCRIPTION_KEY = + "__database_detailed_description"; + + private static final String TABLE_PREFIX = "table_"; + + PulsarCatalogConfiguration catalogConfiguration; + + private final PulsarAdminTool pulsarAdminTool; + + private final String flinkCatalogTenant; + + private SchemaTranslator schemaTranslator; + + public PulsarCatalogSupport( + PulsarCatalogConfiguration catalogConfiguration, + String flinkTenant, + SchemaTranslator schemaTranslator) + throws PulsarAdminException { + this.catalogConfiguration = catalogConfiguration; + this.pulsarAdminTool = new PulsarAdminTool(catalogConfiguration); + this.schemaTranslator = schemaTranslator; + this.flinkCatalogTenant = flinkTenant; + + // Initialize the dedicated tenant if necessary + if (!pulsarAdminTool.tenantExists(flinkCatalogTenant)) { + pulsarAdminTool.createTenant(flinkCatalogTenant); + } + } + + /** + * A generic database stored in pulsar catalog should consist of alphanumeric characters. A + * pulsar tenant/namespace mapped database should contain the "/" in between tenant and + * namespace + * + * @param name the database name + * @return false if the name contains "/", which indicate it's a pulsar tenant/namespace mapped + * database + */ + private boolean isExplicitDatabase(String name) { + return !name.contains("/"); + } + + private String completeExplicitDatabasePath(String name) { + return this.flinkCatalogTenant + "/" + name; + } + + public List listDatabases() throws PulsarAdminException { + List databases = new ArrayList<>(); + for (String ns : pulsarAdminTool.listNamespaces()) { + if (ns.startsWith(flinkCatalogTenant)) { + // explicit table database + databases.add(ns.substring(flinkCatalogTenant.length() + 1)); + } else { + // pulsar tenant/namespace mapped database + databases.add(ns); + } + } + return databases; + } + + public boolean databaseExists(String name) throws PulsarAdminException { + if (isExplicitDatabase(name)) { + return pulsarAdminTool.namespaceExists(completeExplicitDatabasePath(name)); + } else { + return pulsarAdminTool.namespaceExists(name); + } + } + + public void createDatabase(String name, CatalogDatabase database) throws PulsarAdminException { + if (isExplicitDatabase(name)) { + pulsarAdminTool.createNamespace(completeExplicitDatabasePath(name)); + Map allProperties = database.getProperties(); + allProperties.put(DATABASE_COMMENT_KEY, database.getComment()); + allProperties.put(DATABASE_DESCRIPTION_KEY, database.getDescription().orElse("")); + allProperties.put( + DATABASE_DETAILED_DESCRIPTION_KEY, + database.getDetailedDescription().orElse("")); + pulsarAdminTool.updateNamespaceProperties( + completeExplicitDatabasePath(name), allProperties); + } else { + throw new CatalogException("Can't create pulsar tenant/namespace mapped database"); + } + } + + public CatalogDatabase getDatabase(String name) throws PulsarAdminException { + Map allProperties = + pulsarAdminTool.getNamespaceProperties(completeExplicitDatabasePath(name)); + String comment = allProperties.getOrDefault(DATABASE_COMMENT_KEY, ""); + allProperties.remove(DATABASE_COMMENT_KEY); + return new CatalogDatabaseImpl(allProperties, comment); + } + + public void dropDatabase(String name) throws PulsarAdminException { + if (isExplicitDatabase(name)) { + pulsarAdminTool.deleteNamespace(completeExplicitDatabasePath(name)); + } else { + throw new CatalogException("Can't drop pulsar tenant/namespace mapped database"); + } + } + + public List listTables(String name) throws PulsarAdminException { + if (isExplicitDatabase(name)) { + List tables = new ArrayList<>(); + List topics = pulsarAdminTool.getTopics(completeExplicitDatabasePath(name)); + for (String topic : topics) { + tables.add(topic.substring(TABLE_PREFIX.length())); + } + return tables; + } else { + return pulsarAdminTool.getTopics(name); + } + } + + public boolean tableExists(ObjectPath tablePath) throws PulsarAdminException { + if (isExplicitDatabase(tablePath.getDatabaseName())) { + return pulsarAdminTool.topicExists(findExplicitTablePlaceholderTopic(tablePath)); + } else { + return pulsarAdminTool.topicExists(findTopicForNativeTable(tablePath)); + } + } + + public CatalogTable getTable(ObjectPath tablePath) throws PulsarAdminException { + if (isExplicitDatabase(tablePath.getDatabaseName())) { + try { + String mappedTopic = findExplicitTablePlaceholderTopic(tablePath); + final SchemaInfo metadataSchema = pulsarAdminTool.getPulsarSchema(mappedTopic); + Map tableProperties = + TableSchemaHelper.generateTableProperties(metadataSchema); + CatalogTable table = CatalogTable.fromProperties(tableProperties); + table.getOptions().put(PulsarTableOptions.EXPLICIT.key(), Boolean.TRUE.toString()); + return CatalogTable.of( + table.getUnresolvedSchema(), + table.getComment(), + table.getPartitionKeys(), + fillDefaultOptionsFromCatalogOptions(table.getOptions())); + } catch (Exception e) { + e.printStackTrace(); + throw new CatalogException( + "Failed to fetch metadata for explict table: " + tablePath.getObjectName()); + } + } else { + String existingTopic = findTopicForNativeTable(tablePath); + final SchemaInfo pulsarSchema = pulsarAdminTool.getPulsarSchema(existingTopic); + return schemaToCatalogTable(pulsarSchema, existingTopic); + } + } + + public void dropTable(ObjectPath tablePath) throws PulsarAdminException { + if (isExplicitDatabase(tablePath.getDatabaseName())) { + String mappedTopic = findExplicitTablePlaceholderTopic(tablePath); + // manually clean the schema to avoid affecting new table with same name use old schema + pulsarAdminTool.deleteSchema(mappedTopic); + pulsarAdminTool.deleteTopic(mappedTopic); + } else { + throw new CatalogException("Can't delete native topic"); + } + } + + public void createTable(ObjectPath tablePath, ResolvedCatalogTable table) + throws PulsarAdminException { + // only allow creating table in explict database, the topic is used to save table + // information + if (!isExplicitDatabase(tablePath.getDatabaseName())) { + throw new CatalogException( + String.format( + "Can't create explict table under pulsar tenant/namespace: %s because it's a native database", + tablePath.getDatabaseName())); + } + + String mappedTopic = findExplicitTablePlaceholderTopic(tablePath); + pulsarAdminTool.createTopic(mappedTopic, 1); + + // use pulsar schema to store explicit table information + try { + SchemaInfo schemaInfo = TableSchemaHelper.generateSchemaInfo(table.toProperties()); + pulsarAdminTool.uploadSchema(mappedTopic, schemaInfo); + } catch (Exception e) { + // delete topic if table info cannot be persisted + try { + pulsarAdminTool.deleteTopic(mappedTopic); + } catch (PulsarAdminException ex) { + // do nothing + } + e.printStackTrace(); + throw new CatalogException("Can't store table metadata"); + } + } + + private CatalogTable schemaToCatalogTable(SchemaInfo pulsarSchema, String topicName) { + final Schema schema = schemaTranslator.pulsarSchemaToFlinkSchema(pulsarSchema); + + Map initialTableOptions = new HashMap<>(); + initialTableOptions.put(PulsarTableOptions.TOPICS.key(), topicName); + initialTableOptions.put( + FactoryUtil.FORMAT.key(), schemaTranslator.decideDefaultFlinkFormat(pulsarSchema)); + + Map enrichedTableOptions = + fillDefaultOptionsFromCatalogOptions(initialTableOptions); + + return CatalogTable.of(schema, "", Collections.emptyList(), enrichedTableOptions); + } + + // enrich table properties with proper catalog configs + private Map fillDefaultOptionsFromCatalogOptions( + final Map tableOptions) { + Map enrichedTableOptions = new HashMap<>(); + enrichedTableOptions.put(FactoryUtil.CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + enrichedTableOptions.put( + PulsarTableOptions.ADMIN_URL.key(), + catalogConfiguration.get(PulsarOptions.PULSAR_ADMIN_URL)); + enrichedTableOptions.put( + PulsarTableOptions.SERVICE_URL.key(), + catalogConfiguration.get(PulsarOptions.PULSAR_SERVICE_URL)); + + String authPlugin = catalogConfiguration.get(PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME); + if (authPlugin != null && !authPlugin.isEmpty()) { + enrichedTableOptions.put(PulsarOptions.PULSAR_AUTH_PLUGIN_CLASS_NAME.key(), authPlugin); + } + + String authParams = catalogConfiguration.get(PulsarOptions.PULSAR_AUTH_PARAMS); + if (authParams != null && !authParams.isEmpty()) { + enrichedTableOptions.put(PulsarOptions.PULSAR_AUTH_PARAMS.key(), authParams); + } + + // we always provide RAW format as a default format + if (!enrichedTableOptions.containsKey(FactoryUtil.FORMAT.key())) { + enrichedTableOptions.put(FactoryUtil.FORMAT.key(), RawFormatFactory.IDENTIFIER); + } + + if (tableOptions != null) { + // table options could overwrite the default options provided above + enrichedTableOptions.putAll(tableOptions); + } + return enrichedTableOptions; + } + + private String findExplicitTablePlaceholderTopic(ObjectPath objectPath) { + String database = flinkCatalogTenant + "/" + objectPath.getDatabaseName(); + String topic = TABLE_PREFIX + objectPath.getObjectName(); + + NamespaceName ns = NamespaceName.get(database); + TopicName fullName = TopicName.get(TopicDomain.persistent.toString(), ns, topic); + return fullName.toString(); + } + + private String findTopicForNativeTable(ObjectPath objectPath) { + String database = objectPath.getDatabaseName(); + String topic = objectPath.getObjectName(); + + NamespaceName ns = NamespaceName.get(database); + TopicName fullName = TopicName.get(TopicDomain.persistent.toString(), ns, topic); + return fullName.toString(); + } + + public void close() { + if (pulsarAdminTool != null) { + pulsarAdminTool.close(); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java new file mode 100644 index 0000000000000..1cbde2d4adae2 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java @@ -0,0 +1,237 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog.impl; + +import org.apache.flink.formats.avro.AvroFormatFactory; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.json.JsonFormatFactory; +import org.apache.flink.formats.raw.RawFormatFactory; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.FieldsDataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; + +import com.google.protobuf.Descriptors; +import org.apache.pulsar.client.impl.schema.generic.GenericProtobufNativeSchema; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; + +/** Translate a Pulsar Schema to Flink Table Schema. */ +public class SchemaTranslator { + private static final Logger LOG = LoggerFactory.getLogger(SchemaTranslator.class); + + public static final String SINGLE_FIELD_FIELD_NAME = "value"; + + private final boolean useMetadataFields; + + public SchemaTranslator(boolean useMetadataFields) { + this.useMetadataFields = useMetadataFields; + } + + public org.apache.flink.table.api.Schema pulsarSchemaToFlinkSchema(SchemaInfo pulsarSchema) + throws IncompatibleSchemaException { + final DataType fieldsDataType = pulsarSchemaToPhysicalFields(pulsarSchema); + org.apache.flink.table.api.Schema.Builder schemaBuilder = + org.apache.flink.table.api.Schema.newBuilder().fromRowDataType(fieldsDataType); + + if (useMetadataFields) { + throw new UnsupportedOperationException( + "Querying Pulsar Metadata is not supported yet"); + } + + return schemaBuilder.build(); + } + + public DataType pulsarSchemaToPhysicalFields(SchemaInfo schemaInfo) + throws IncompatibleSchemaException { + List mainSchema = new ArrayList<>(); + DataType dataType = schemaInfo2SqlType(schemaInfo); + // ROW and STRUCTURED are FieldsDataType + if (dataType instanceof FieldsDataType) { + FieldsDataType fieldsDataType = (FieldsDataType) dataType; + RowType rowType = (RowType) fieldsDataType.getLogicalType(); + List fieldNames = rowType.getFieldNames(); + for (int i = 0; i < fieldNames.size(); i++) { + org.apache.flink.table.types.logical.LogicalType logicalType = rowType.getTypeAt(i); + DataTypes.Field field = + DataTypes.FIELD( + fieldNames.get(i), + TypeConversions.fromLogicalToDataType(logicalType)); + mainSchema.add(field); + } + + } else { + mainSchema.add(DataTypes.FIELD(SINGLE_FIELD_FIELD_NAME, dataType)); + } + + return DataTypes.ROW(mainSchema.toArray(new DataTypes.Field[0])); + } + + public DataType schemaInfo2SqlType(SchemaInfo si) throws IncompatibleSchemaException { + switch (si.getType()) { + case NONE: + case BYTES: + return DataTypes.BYTES(); + case BOOLEAN: + return DataTypes.BOOLEAN(); + case LOCAL_DATE: + return DataTypes.DATE(); + case LOCAL_TIME: + return DataTypes.TIME(); + case STRING: + return DataTypes.STRING(); + case LOCAL_DATE_TIME: + return DataTypes.TIMESTAMP(3); + case INT8: + return DataTypes.TINYINT(); + case DOUBLE: + return DataTypes.DOUBLE(); + case FLOAT: + return DataTypes.FLOAT(); + case INT32: + return DataTypes.INT(); + case INT64: + return DataTypes.BIGINT(); + case INT16: + return DataTypes.SMALLINT(); + case AVRO: + case JSON: + String avroSchemaString = new String(si.getSchema(), StandardCharsets.UTF_8); + return AvroSchemaConverter.convertToDataType(avroSchemaString); + case PROTOBUF_NATIVE: + Descriptors.Descriptor descriptor = + ((GenericProtobufNativeSchema) GenericProtobufNativeSchema.of(si)) + .getProtobufNativeSchema(); + return protoDescriptorToSqlType(descriptor); + + default: + throw new UnsupportedOperationException( + String.format("We do not support %s currently.", si.getType())); + } + } + + public static DataType protoDescriptorToSqlType(Descriptors.Descriptor descriptor) + throws IncompatibleSchemaException { + List fields = new ArrayList<>(); + List protoFields = descriptor.getFields(); + + for (Descriptors.FieldDescriptor fieldDescriptor : protoFields) { + DataType fieldType = protoFieldDescriptorToSqlType(fieldDescriptor); + fields.add(DataTypes.FIELD(fieldDescriptor.getName(), fieldType)); + } + + if (fields.isEmpty()) { + throw new IllegalArgumentException("No FieldDescriptors found"); + } + return DataTypes.ROW(fields.toArray(new DataTypes.Field[0])); + } + + private static DataType protoFieldDescriptorToSqlType(Descriptors.FieldDescriptor field) + throws IncompatibleSchemaException { + Descriptors.FieldDescriptor.JavaType type = field.getJavaType(); + DataType dataType; + switch (type) { + case BOOLEAN: + dataType = DataTypes.BOOLEAN(); + break; + case BYTE_STRING: + dataType = DataTypes.BYTES(); + break; + case DOUBLE: + dataType = DataTypes.DOUBLE(); + break; + case ENUM: + dataType = DataTypes.STRING(); + break; + case FLOAT: + dataType = DataTypes.FLOAT(); + break; + case INT: + dataType = DataTypes.INT(); + break; + case LONG: + dataType = DataTypes.BIGINT(); + break; + case MESSAGE: + Descriptors.Descriptor msg = field.getMessageType(); + if (field.isMapField()) { + // map + dataType = + DataTypes.MAP( + protoFieldDescriptorToSqlType(msg.findFieldByName("key")), + protoFieldDescriptorToSqlType(msg.findFieldByName("value"))); + } else { + // row + dataType = protoDescriptorToSqlType(field.getMessageType()); + } + break; + case STRING: + dataType = DataTypes.STRING(); + break; + default: + throw new IllegalArgumentException( + "Unknown type: " + + type.toString() + + " for FieldDescriptor: " + + field.toString()); + } + // list + if (field.isRepeated() && !field.isMapField()) { + dataType = DataTypes.ARRAY(dataType); + } + + return dataType; + } + + /** + * This method is used to determine the Flink format to use for a native table. + * + * @param pulsarSchemaInfo + * @return + */ + public String decideDefaultFlinkFormat(SchemaInfo pulsarSchemaInfo) { + String formatIdentifier = RawFormatFactory.IDENTIFIER; + switch (pulsarSchemaInfo.getType()) { + case JSON: + formatIdentifier = JsonFormatFactory.IDENTIFIER; + break; + case AVRO: + formatIdentifier = AvroFormatFactory.IDENTIFIER; + break; + case PROTOBUF_NATIVE: + case PROTOBUF: + case AUTO_CONSUME: + case AUTO: + case AUTO_PUBLISH: + LOG.error( + "Do not support {} schema, using RAW instead", + pulsarSchemaInfo.getType().toString()); + break; + default: + break; + } + return formatIdentifier; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/utils/TableSchemaHelper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/utils/TableSchemaHelper.java new file mode 100644 index 0000000000000..9961665d2aefa --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/utils/TableSchemaHelper.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.catalog.utils; + +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.types.DataType; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonProcessingException; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.impl.schema.SchemaInfoImpl; +import org.apache.pulsar.client.internal.DefaultImplementation; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.protocol.schema.PostSchemaPayload; +import org.apache.pulsar.common.schema.SchemaInfo; +import org.apache.pulsar.common.schema.SchemaType; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.avro.Schema.Type.RECORD; +import static org.apache.pulsar.shade.com.google.common.base.Preconditions.checkNotNull; + +/** Util to convert between flink table map representation and pulsar SchemaInfo. */ +public final class TableSchemaHelper { + + private TableSchemaHelper() {} + + public static SchemaInfo generateSchemaInfo(Map properties) + throws JsonProcessingException { + ObjectMapper mapper = new ObjectMapper(); + return SchemaInfoImpl.builder() + .name("flink_table_schema") + .type(SchemaType.BYTES) + .schema(mapper.writeValueAsBytes(properties)) + .build(); + } + + public static Map generateTableProperties(SchemaInfo schemaInfo) + throws IOException { + ObjectMapper mapper = new ObjectMapper(); + return mapper.readValue( + schemaInfo.getSchema(), new TypeReference>() {}); + } + + public static void uploadPulsarSchema(PulsarAdmin admin, String topic, SchemaInfo schemaInfo) { + checkNotNull(schemaInfo); + + SchemaInfo existingSchema; + try { + existingSchema = admin.schemas().getSchemaInfo(TopicName.get(topic).toString()); + } catch (PulsarAdminException pae) { + if (pae.getStatusCode() == 404) { + existingSchema = null; + } else { + throw new RuntimeException( + String.format( + "Failed to get schema information for %s", + TopicName.get(topic).toString()), + pae); + } + } catch (Throwable e) { + throw new RuntimeException( + String.format( + "Failed to get schema information for %s", + TopicName.get(topic).toString()), + e); + } + + if (existingSchema == null) { + PostSchemaPayload pl = new PostSchemaPayload(); + try { + pl.setType(schemaInfo.getType().name()); + pl.setSchema(getSchemaString(schemaInfo)); + pl.setProperties(schemaInfo.getProperties()); + admin.schemas().createSchema(TopicName.get(topic).toString(), pl); + } catch (PulsarAdminException pae) { + if (pae.getStatusCode() == 404) { + throw new RuntimeException( + String.format( + "Create schema for %s get 404", + TopicName.get(topic).toString()), + pae); + } else { + throw new RuntimeException( + String.format( + "Failed to create schema information for %s", + TopicName.get(topic).toString()), + pae); + } + } catch (IOException e) { + throw new RuntimeException( + String.format( + "Failed to set schema information for %s", + TopicName.get(topic).toString()), + e); + } catch (Throwable e) { + throw new RuntimeException( + String.format( + "Failed to create schema information for %s", + TopicName.get(topic).toString()), + e); + } + } else if (!schemaEqualsIgnoreProperties(schemaInfo, existingSchema) + && !compatibleSchema(existingSchema, schemaInfo)) { + throw new RuntimeException("Writing to a topic which have incompatible schema"); + } + } + + public static void deletePulsarSchema(PulsarAdmin admin, String topic) { + try { + admin.schemas().deleteSchema(topic); + } catch (PulsarAdminException e) { + e.printStackTrace(); + } + } + + private static boolean schemaEqualsIgnoreProperties( + SchemaInfo schemaInfo, SchemaInfo existingSchema) { + return existingSchema.getType().equals(schemaInfo.getType()) + && Arrays.equals(existingSchema.getSchema(), schemaInfo.getSchema()); + } + + // TODO handle the exception + private static String getSchemaString(SchemaInfo schemaInfo) throws IOException { + final byte[] schemaData = schemaInfo.getSchema(); + if (null == schemaData) { + return null; + } + if (schemaInfo.getType() == SchemaType.KEY_VALUE) { + return DefaultImplementation.getDefaultImplementation() + .convertKeyValueSchemaInfoDataToString( + DefaultImplementation.getDefaultImplementation() + .decodeKeyValueSchemaInfo(schemaInfo)); + } + return new String(schemaData, StandardCharsets.UTF_8); + } + + public static boolean compatibleSchema(SchemaInfo s1, SchemaInfo s2) { + if (s1.getType() == SchemaType.NONE && s2.getType() == SchemaType.BYTES) { + return true; + } else { + return s1.getType() == SchemaType.BYTES && s2.getType() == SchemaType.NONE; + } + } + + public static SchemaInfoImpl getSchemaInfo(SchemaType type, DataType dataType) { + byte[] schemaBytes = getAvroSchema(dataType).toString().getBytes(StandardCharsets.UTF_8); + return SchemaInfoImpl.builder() + .name("Record") + .schema(schemaBytes) + .type(type) + .properties(Collections.emptyMap()) + .build(); + } + + public static org.apache.avro.Schema getAvroSchema(DataType dataType) { + org.apache.avro.Schema schema = + AvroSchemaConverter.convertToSchema(dataType.getLogicalType()); + if (schema.isNullable()) { + schema = + schema.getTypes().stream() + .filter(s -> s.getType() == RECORD) + .findAny() + .orElseThrow( + () -> + new IllegalArgumentException( + "not support DataType: " + + dataType.toString())); + } + return schema; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory index b153b56175670..c2e04bc746218 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ b/flink-connectors/flink-connector-pulsar/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -14,3 +14,5 @@ # limitations under the License. org.apache.flink.connector.pulsar.table.PulsarTableFactory +org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactory + diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java index 08cf89c014ff4..46218572f6735 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -20,7 +20,6 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.table.data.RowData; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.test.util.SuccessException; import org.apache.flink.types.Row; @@ -112,8 +111,7 @@ public void pulsarSourceSink(String format) throws Exception { + "FROM pulsar_source_sink\n" + "GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)"; - DataStream result = - tableEnv.toAppendStream(tableEnv.sqlQuery(query), RowData.class); + DataStream result = tableEnv.toDataStream(tableEnv.sqlQuery(query)); TestingSinkFunction sink = new TestingSinkFunction(2); result.addSink(sink).setParallelism(1); @@ -128,8 +126,8 @@ public void pulsarSourceSink(String format) throws Exception { List expected = Arrays.asList( - "+I(2019-12-12 00:00:05.000,2019-12-12,00:00:03,2019-12-12 00:00:04.004,3,50.00)", - "+I(2019-12-12 00:00:10.000,2019-12-12,00:00:05,2019-12-12 00:00:06.006,2,5.33)"); + "+I[2019-12-12 00:00:05.000, 2019-12-12, 00:00:03, 2019-12-12 00:00:04.004, 3, 50.00]", + "+I[2019-12-12 00:00:10.000, 2019-12-12, 00:00:05, 2019-12-12 00:00:06.006, 2, 5.33]"); assertThat(TestingSinkFunction.rows).isEqualTo(expected); } @@ -257,7 +255,7 @@ public void pulsarSourceSinkWithMetadata(String format) throws Exception { true)); } - private static final class TestingSinkFunction implements SinkFunction { + private static final class TestingSinkFunction implements SinkFunction { private static final long serialVersionUID = 455430015321124493L; private static List rows = new ArrayList<>(); @@ -270,7 +268,7 @@ private TestingSinkFunction(int expectedSize) { } @Override - public void invoke(RowData value, Context context) { + public void invoke(Row value, Context context) { rows.add(value.toString()); if (rows.size() >= expectedSize) { // job finish diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java index 90ba67a129261..32ce79ffccff1 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableTestBase.java @@ -38,22 +38,26 @@ /** Base class for Pulsar table integration test. */ @TestInstance(TestInstance.Lifecycle.PER_CLASS) public abstract class PulsarTableTestBase { - // TODO fix the archunit violations private static final Logger LOG = LoggerFactory.getLogger(PulsarTableTestBase.class); @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); // Defines pulsar running environment @TestExternalSystem - PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.container()); + protected PulsarTestEnvironment pulsar = new PulsarTestEnvironment(runtime()); @TestSemantics - CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; + protected CheckpointingMode[] semantics = + new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; protected StreamExecutionEnvironment env; protected StreamTableEnvironment tableEnv; + protected PulsarRuntime runtime() { + return PulsarRuntime.container(); + } + @BeforeAll public void beforeAll() { pulsar.startUp(); @@ -62,6 +66,9 @@ public void beforeAll() { env.setParallelism(1); env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); tableEnv = StreamTableEnvironment.create(env); + tableEnv.getConfig() + .getConfiguration() + .setString("table.dynamic-table-options.enabled", "true"); } public void createTestTopic(String topic, int numPartitions) { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java new file mode 100644 index 0000000000000..4f9ebae2ae5f0 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java @@ -0,0 +1,911 @@ +package org.apache.flink.connector.pulsar.table.catalog; + +import org.apache.flink.connector.pulsar.common.config.PulsarConfigBuilder; +import org.apache.flink.connector.pulsar.common.config.PulsarOptions; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; +import org.apache.flink.connector.pulsar.table.PulsarTableFactory; +import org.apache.flink.connector.pulsar.table.PulsarTableOptions; +import org.apache.flink.connector.pulsar.table.PulsarTableTestBase; +import org.apache.flink.connector.pulsar.table.testutils.TestingUser; +import org.apache.flink.formats.raw.RawFormatFactory; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.GenericInMemoryCatalog; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.types.Row; + +import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; + +import org.apache.commons.io.IOUtils; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.shade.org.apache.commons.lang3.RandomStringUtils; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactory.CATALOG_CONFIG_VALIDATOR; +import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows; +import static org.apache.flink.connector.pulsar.table.testutils.SchemaData.INTEGER_LIST; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; + +/** Unit test for {@link PulsarCatalog}. */ +public class PulsarCatalogITTest extends PulsarTableTestBase { + private static final Logger LOGGER = LoggerFactory.getLogger(PulsarCatalogITTest.class); + + private static final String INMEMORY_CATALOG = "inmemorycatalog"; + private static final String PULSAR_CATALOG1 = "pulsarcatalog1"; + private static final String PULSAR_CATALOG2 = "pulsarcatalog2"; + + private static final String INMEMORY_DB = "mydatabase"; + private static final String PULSAR1_DB = "public/default"; + private static final String PULSAR2_DB = "tn/ns"; + + private static final String FLINK_TENANT = "__flink_catalog"; + + @BeforeAll + void before() { + registerCatalogs(tableEnv); + } + + // catalog operations + @Test + void createCatalogWithAllConfig() throws PulsarAdminException { + String catalogName = RandomStringUtils.randomAlphabetic(10); + String customTenantPath = "__flink_custom_tenant"; + String defaultDatabase = "my_db"; + tableEnv.executeSql( + String.format( + "CREATE CATALOG %s WITH (" + + "'type' = 'pulsar-catalog',\n" + + "'catalog-admin-url' = '%s',\n" + + "'catalog-service-url' = '%s',\n" + + "'catalog-tenant' = '%s',\n" + + "'default-database' = '%s'\n" + + ")", + catalogName, + pulsar.operator().adminUrl(), + pulsar.operator().serviceUrl(), + customTenantPath, + defaultDatabase)); + Optional catalogOptional = tableEnv.getCatalog(catalogName); + assertThat(catalogOptional).isPresent(); + tableEnv.useCatalog(catalogName); + assertThat(tableEnv.getCurrentDatabase()).isEqualTo(defaultDatabase); + assertThat(pulsar.operator().admin().tenants().getTenants()).contains(customTenantPath); + } + + @Test + void createMultipleCatalog() { + tableEnv.useCatalog(INMEMORY_CATALOG); + assertThat(tableEnv.getCurrentCatalog()).isEqualTo(INMEMORY_CATALOG); + assertThat(tableEnv.getCurrentDatabase()).isEqualTo(INMEMORY_DB); + + Catalog catalog = tableEnv.getCatalog(PULSAR_CATALOG1).orElse(null); + assertThat(catalog).isNotNull(); + assertThat(catalog).isInstanceOf(PulsarCatalog.class); + + tableEnv.useCatalog(PULSAR_CATALOG1); + assertThat(tableEnv.getCurrentDatabase()).isEqualTo(PULSAR1_DB); + + catalog = tableEnv.getCatalog(PULSAR_CATALOG2).orElse(null); + assertThat(catalog).isNotNull(); + assertThat(catalog).isInstanceOf(PulsarCatalog.class); + + tableEnv.useCatalog(PULSAR_CATALOG2); + assertThat(tableEnv.getCurrentDatabase()).isEqualTo(PULSAR2_DB); + } + + // database operations + @Test + void listPulsarNativeDatabase() throws Exception { + List namespaces = Arrays.asList("tn1/ns1", "tn1/ns2"); + List topics = Arrays.asList("tp1", "tp2"); + List topicsFullName = + topics.stream().map(a -> "tn1/ns1/" + a).collect(Collectors.toList()); + List partitionedTopics = Arrays.asList("ptp1", "ptp2"); + List partitionedTopicsFullName = + partitionedTopics.stream().map(a -> "tn1/ns1/" + a).collect(Collectors.toList()); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + assertThat(tableEnv.getCurrentDatabase()).isEqualTo(PULSAR1_DB); + pulsar.operator() + .admin() + .tenants() + .createTenant( + "tn1", + TenantInfo.builder() + .adminRoles(Sets.newHashSet()) + .allowedClusters(Sets.newHashSet("standalone")) + .build()); + + for (String ns : namespaces) { + pulsar.operator().admin().namespaces().createNamespace(ns); + } + + for (String tp : topicsFullName) { + pulsar.operator().admin().topics().createNonPartitionedTopic(tp); + } + + for (String tp : partitionedTopicsFullName) { + pulsar.operator().admin().topics().createPartitionedTopic(tp, 5); + } + + assertThat(Sets.newHashSet(tableEnv.listDatabases())).containsAll(namespaces); + tableEnv.useDatabase("tn1/ns1"); + + Set tableSet = Sets.newHashSet(tableEnv.listTables()); + + assertThat(tableSet) + .containsExactlyInAnyOrderElementsOf(Iterables.concat(topics, partitionedTopics)); + } + + @Test + void createExplicitDatabase() { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + assertThatNoException() + .isThrownBy( + () -> + tableEnv.executeSql( + String.format("CREATE DATABASE %s", explictDatabaseName))); + } + + @Test + void createNativeDatabaseShouldFail() { + tableEnv.useCatalog(PULSAR_CATALOG1); + String nativeDatabaseName = "tn1/ns1"; + assertThatExceptionOfType(TableException.class) + .isThrownBy( + () -> + tableEnv.executeSql( + String.format("CREATE DATABASE `%s`", nativeDatabaseName))) + .withMessageStartingWith("Could not execute CREATE DATABASE"); + } + + @Test + void dropNativeDatabaseShouldFail() { + tableEnv.useCatalog(PULSAR_CATALOG1); + assertThatExceptionOfType(TableException.class) + .isThrownBy( + () -> tableEnv.executeSql(String.format("DROP DATABASE `%s`", PULSAR1_DB))) + .withMessageStartingWith("Could not execute DROP DATABASE"); + } + + @Test + void dropExplicitDatabaseWithTablesShouldFail() { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + String topicName = newTopicName(); + + String dbDDL = "CREATE DATABASE " + explictDatabaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.useDatabase(explictDatabaseName); + + String createTableSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + topicName); + tableEnv.executeSql(createTableSql); + + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + tableEnv.executeSql( + String.format("DROP DATABASE %s", explictDatabaseName))) + .withMessageStartingWith("Could not execute DROP DATABASE") + .withCauseInstanceOf(DatabaseNotEmptyException.class); + } + + @Test + void dropExplicitDatabase() { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + tableEnv.executeSql(String.format("CREATE DATABASE %s", explictDatabaseName)); + + assertThatNoException() + .isThrownBy( + () -> + tableEnv.executeSql( + String.format("DROP DATABASE %s", explictDatabaseName))); + } + + @Test + void createAndGetDetailedDatabase() throws DatabaseNotExistException { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + tableEnv.executeSql( + String.format( + "CREATE DATABASE %s \n" + + "COMMENT 'this is a comment'\n" + + "WITH (" + + "'p1' = 'k1',\n" + + "'p2' = 'k2' \n" + + ")", + explictDatabaseName)); + tableEnv.useDatabase(explictDatabaseName); + Catalog catalog = tableEnv.getCatalog(PULSAR_CATALOG1).get(); + CatalogDatabase database = catalog.getDatabase(explictDatabaseName); + + Map expectedProperties = new HashMap<>(); + expectedProperties.put("p1", "k1"); + expectedProperties.put("p2", "k2"); + assertThat(database.getProperties()).containsAllEntriesOf(expectedProperties); + assertThat(database.getComment()).isEqualTo("this is a comment"); + } + + // table operations + @Test + void createExplicitTable() throws Exception { + String databaseName = newDatabaseName(); + String tableTopic = newTopicName(); + String tableName = TopicName.get(tableTopic).getLocalName(); + + tableEnv.useCatalog(PULSAR_CATALOG1); + + String dbDDL = "CREATE DATABASE " + databaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.useDatabase(databaseName); + + String sinkDDL = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + tableName); + tableEnv.executeSql(sinkDDL).await(10, TimeUnit.SECONDS); + assertThat(tableEnv.listTables()).contains(tableName); + } + + @Test + void createExplicitTableAndRunSourceSink() { + tableEnv.useCatalog(PULSAR_CATALOG1); + + String databaseName = newDatabaseName(); + String dbDDL = "CREATE DATABASE " + databaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.useDatabase(databaseName); + + String topicName = newTopicName(); + String createSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ") with (\n" + + " 'connector' = 'pulsar',\n" + + " 'topics' = '%s'," + + " 'format' = 'avro'\n" + + ")", + topicName, topicName); + assertThatNoException().isThrownBy(() -> tableEnv.executeSql(createSql)); + } + + @Test + void createExplicitTableInNativePulsarDatabaseShouldFail() { + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + String topicName = newTopicName(); + String createSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ") with (\n" + + " 'connector' = 'pulsar',\n" + + " 'topics' = '%s'," + + " 'format' = 'avro'\n" + + ")", + topicName, topicName); + + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tableEnv.executeSql(createSql)) + .withMessage( + String.format( + "Could not execute CreateTable in path `%s`.`%s`.`%s`", + PULSAR_CATALOG1, PULSAR1_DB, topicName)); + } + + @Test + void dropNativeTableShouldFail() { + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + String topicName = newTopicName(); + pulsar.operator().createTopic(topicName, 1); + assertThatExceptionOfType(TableException.class) + .isThrownBy(() -> tableEnv.executeSql(String.format("DROP TABLE %s", topicName))) + .withMessage( + String.format( + "Could not execute DropTable in path `%s`.`%s`.`%s`", + PULSAR_CATALOG1, PULSAR1_DB, topicName)); + } + + @Test + void dropExplicitTable() { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + tableEnv.executeSql(String.format("CREATE DATABASE %s", explictDatabaseName)); + tableEnv.useDatabase(explictDatabaseName); + + String topicName = newTopicName(); + String createTableSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + topicName); + tableEnv.executeSql(createTableSql); + + assertThatNoException() + .isThrownBy(() -> tableEnv.executeSql(String.format("DROP TABLE %s", topicName))); + } + + @Test + void tableExists() throws ExecutionException, InterruptedException { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + tableEnv.executeSql(String.format("CREATE DATABASE %s", explictDatabaseName)); + tableEnv.useDatabase(explictDatabaseName); + + String topicName = newTopicName(); + String createTableSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + topicName); + tableEnv.executeSql(createTableSql).await(); + + Catalog catalog = tableEnv.getCatalog(PULSAR_CATALOG1).get(); + assertThat(catalog.tableExists(new ObjectPath(explictDatabaseName, topicName))).isTrue(); + } + + @Test + void getExplicitTable() throws TableNotExistException { + tableEnv.useCatalog(PULSAR_CATALOG1); + String explictDatabaseName = newDatabaseName(); + tableEnv.executeSql(String.format("CREATE DATABASE %s", explictDatabaseName)); + tableEnv.useDatabase(explictDatabaseName); + + String topicName = newTopicName(); + String createTableSql = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + topicName); + tableEnv.executeSql(createTableSql); + + Catalog catalog = tableEnv.getCatalog(PULSAR_CATALOG1).get(); + CatalogBaseTable table = catalog.getTable(new ObjectPath(explictDatabaseName, topicName)); + + Map expectedOptions = new HashMap<>(); + expectedOptions.put(PulsarTableOptions.EXPLICIT.key(), "true"); + expectedOptions.put(PulsarTableOptions.ADMIN_URL.key(), pulsar.operator().adminUrl()); + expectedOptions.put(PulsarTableOptions.SERVICE_URL.key(), pulsar.operator().serviceUrl()); + expectedOptions.put(FactoryUtil.FORMAT.key(), RawFormatFactory.IDENTIFIER); + expectedOptions.put(FactoryUtil.CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + + assertThat(table.getOptions()).containsExactlyEntriesOf(expectedOptions); + } + + @Test + void getNativeTable() throws Exception { + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + String nativeTopicName = newTopicName(); + pulsar.operator().createTopic(nativeTopicName, 1); + + Catalog catalog = tableEnv.getCatalog(PULSAR_CATALOG1).get(); + CatalogBaseTable table = catalog.getTable(new ObjectPath(PULSAR1_DB, nativeTopicName)); + + Map expectedOptions = new HashMap<>(); + expectedOptions.put( + PulsarTableOptions.TOPICS.key(), TopicNameUtils.topicName(nativeTopicName)); + expectedOptions.put(PulsarTableOptions.ADMIN_URL.key(), pulsar.operator().adminUrl()); + expectedOptions.put(PulsarTableOptions.SERVICE_URL.key(), pulsar.operator().serviceUrl()); + expectedOptions.put(FactoryUtil.FORMAT.key(), RawFormatFactory.IDENTIFIER); + expectedOptions.put(FactoryUtil.CONNECTOR.key(), PulsarTableFactory.IDENTIFIER); + + assertThat(table.getOptions()).containsExactlyEntriesOf(expectedOptions); + } + + // runtime behaviour + + @Test + void readFromNativeTable() throws Exception { + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + String topicName = newTopicName(); + + pulsar.operator().createTopic(topicName, 1); + pulsar.operator().sendMessages(topicName, Schema.INT32, INTEGER_LIST); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", topicName)), + INTEGER_LIST.size()); + assertThat(result) + .containsExactlyElementsOf( + INTEGER_LIST.stream().map(Row::of).collect(Collectors.toList())); + } + + @Test + void readFromNativeTableWithMetadata() { + // TODO this test will be implemented after useMetadata is supported; + } + + @Test + void readFromNativeTableFromEarliest() throws Exception { + String topicName = newTopicName(); + pulsar.operator().sendMessages(topicName, Schema.INT32, INTEGER_LIST); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.getConfig() + .getConfiguration() + .setString("table.dynamic-table-options.enabled", "true"); + tableEnv.useDatabase(PULSAR1_DB); + + final List result = + collectRows( + tableEnv.sqlQuery( + "select `value` from " + + TopicName.get(topicName).getLocalName() + + " /*+ OPTIONS('source.start.message-id'='earliest') */"), + INTEGER_LIST.size()); + assertThat(result) + .containsExactlyElementsOf( + INTEGER_LIST.stream().map(Row::of).collect(Collectors.toList())); + } + + @Test + void readFromNativeTableWithProtobufNativeSchema() { + // TODO implement after protobuf native schema support + } + + @Test + void readFromNativeTableWithJsonSchema() throws Exception { + String topicName = newTopicName(); + TestingUser expectedUser = createRandomUser(); + pulsar.operator().sendMessage(topicName, Schema.JSON(TestingUser.class), expectedUser); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + final List result = + collectRows( + tableEnv.sqlQuery( + String.format( + "select * from %s ", + TopicName.get(topicName).getLocalName())), + 1); + assertThat(result) + .containsExactlyElementsOf( + Collections.singletonList( + Row.of(expectedUser.getAge(), expectedUser.getName()))); + } + + @Test + void readFromNativeTableWithAvroSchema() throws Exception { + String topicName = newTopicName(); + TestingUser expectedUser = createRandomUser(); + pulsar.operator().sendMessage(topicName, Schema.AVRO(TestingUser.class), expectedUser); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + final List result = + collectRows( + tableEnv.sqlQuery( + String.format( + "select * from %s", + TopicName.get(topicName).getLocalName())), + 1); + assertThat(result) + .containsExactlyElementsOf( + Collections.singletonList( + Row.of(expectedUser.getAge(), expectedUser.getName()))); + } + + @Test + void readFromNativeTableWithStringSchemaUsingRawFormat() throws Exception { + String topicName = newTopicName(); + String expectedString = "expected_string"; + pulsar.operator().sendMessage(topicName, Schema.STRING, expectedString); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + final List result = + collectRows( + tableEnv.sqlQuery( + String.format( + "select * from %s", + TopicName.get(topicName).getLocalName())), + 1); + assertThat(result) + .containsExactlyElementsOf(Collections.singletonList(Row.of(expectedString))); + } + + @Test + void readFromNativeTableWithComplexSchemaUsingRawFormatShouldFail() { + String topicName = newTopicName(); + TestingUser expectedUser = createRandomUser(); + pulsar.operator().sendMessage(topicName, Schema.AVRO(TestingUser.class), expectedUser); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + assertThatExceptionOfType(ValidationException.class) + .isThrownBy( + () -> + collectRows( + tableEnv.sqlQuery( + String.format( + "select * from %s /*+ OPTIONS('format'='raw') */", + TopicName.get(topicName).getLocalName())), + 1)) + .withMessageStartingWith("The 'raw' format only supports single physical column."); + } + + @Test + @Disabled("Disabled due to streamnative/flink#100") + void copyDataFromNativeTableToNativeTable() throws Exception { + String sourceTopic = newTopicName(); + String sourceTableName = TopicName.get(sourceTopic).getLocalName(); + pulsar.operator().sendMessages(sourceTopic, Schema.INT32, INTEGER_LIST); + + String sinkTopic = newTopicName(); + String sinkTableName = TopicName.get(sinkTopic).getLocalName(); + pulsar.operator().createTopic(sinkTopic, 1); + pulsar.operator().admin().schemas().createSchema(sinkTopic, Schema.INT32.getSchemaInfo()); + + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + String insertQ = + String.format( + "INSERT INTO %s" + " SELECT * FROM %s", sinkTableName, sourceTableName); + + tableEnv.executeSql(insertQ).await(); + List result = consumeMessage(sinkTableName, Schema.INT32, INTEGER_LIST.size(), 10); + assertThat(result).containsExactlyElementsOf(INTEGER_LIST); + + // final List result = + // collectRows( + // tableEnv.sqlQuery("SELECT * FROM " + sinkTableName), + // INTEGER_LIST.size()); + // + // assertThat(result) + // .containsExactlyElementsOf( + // INTEGER_LIST.stream().map(Row::of).collect(Collectors.toList())); + } + + @Test + void writeToExplicitTableAndReadWithAvroSchema() throws Exception { + String databaseName = newDatabaseName(); + String tableSinkTopic = newTopicName(); + String tableSinkName = TopicName.get(tableSinkTopic).getLocalName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + tableEnv.useCatalog(PULSAR_CATALOG1); + + String dbDDL = "CREATE DATABASE " + databaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.executeSql("USE " + databaseName); + + String sinkDDL = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ") with (\n" + + " 'connector' = 'pulsar',\n" + + " 'topics' = '%s'," + + " 'format' = 'avro'\n" + + ")", + tableSinkName, tableSinkTopic); + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " ('oid1', 10, 'cid1'),\n" + + " ('oid2', 20, 'cid2'),\n" + + " ('oid3', 30, 'cid3'),\n" + + " ('oid4', 10, 'cid4')", + tableSinkName); + + tableEnv.executeSql(sinkDDL).print(); + tableEnv.executeSql(insertQ); + + final List result = + collectRows(tableEnv.sqlQuery(String.format("SELECT * FROM %s", tableSinkName)), 4); + assertThat(result).hasSize(4); + } + + @Test + void writeToExplicitTableAndReadWithJsonSchema() throws Exception { + String databaseName = newDatabaseName(); + String tableSinkTopic = newTopicName(); + String tableSinkName = TopicName.get(tableSinkTopic).getLocalName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + tableEnv.useCatalog(PULSAR_CATALOG1); + + String dbDDL = "CREATE DATABASE " + databaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.executeSql("USE " + databaseName + ""); + + String sinkDDL = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ") with (\n" + + " 'connector' = 'pulsar',\n" + + " 'topics' = '%s',\n" + + " 'format' = 'json'\n" + + ")", + tableSinkName, tableSinkTopic); + tableEnv.executeSql(sinkDDL).await(); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " ('oid1', 10, 'cid1'),\n" + + " ('oid2', 20, 'cid2'),\n" + + " ('oid3', 30, 'cid3'),\n" + + " ('oid4', 10, 'cid4')", + tableSinkName); + tableEnv.executeSql(insertQ).await(); + + final List result = + collectRows(tableEnv.sqlQuery("select * from " + tableSinkName), 4); + assertThat(result).hasSize(4); + } + + @Test + void writeToNativeTableAndReadWithJsonSchema() throws Exception { + String tableSinkTopic = newTopicName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + pulsar.operator() + .admin() + .schemas() + .createSchema(tableSinkTopic, Schema.JSON(TestingUser.class).getSchemaInfo()); + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " (1, 'abc'),\n" + + " (2, 'bcd'),\n" + + " (3, 'cde'),\n" + + " (4, 'def')", + tableSinkTopic); + tableEnv.executeSql(insertQ).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", tableSinkTopic)), 4); + assertThat(result).hasSize(4); + } + + @Test + void writeToNativeTableAndReadWithAvroSchema() throws Exception { + String tableSinkTopic = newTopicName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + pulsar.operator() + .admin() + .schemas() + .createSchema(tableSinkTopic, Schema.AVRO(TestingUser.class).getSchemaInfo()); + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " (1, 'abc'),\n" + + " (2, 'bcd'),\n" + + " (3, 'cde'),\n" + + " (4, 'def')", + tableSinkTopic); + tableEnv.executeSql(insertQ).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", tableSinkTopic)), 4); + assertThat(result).hasSize(4); + } + + @Test + void writeToNativeTableAndReadWithStringSchema() throws Exception { + String tableSinkTopic = newTopicName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + pulsar.operator() + .admin() + .schemas() + .createSchema(tableSinkTopic, Schema.STRING.getSchemaInfo()); + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " ('abc'),\n" + + " ('bcd'),\n" + + " ('cde'),\n" + + " ('def')", + tableSinkTopic); + tableEnv.executeSql(insertQ).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", tableSinkTopic)), 4); + assertThat(result).hasSize(4); + } + + @Test + void writeToNativeTableAndReadWithIntegerSchema() throws Exception { + String tableSinkTopic = newTopicName(); + + pulsar.operator().createTopic(tableSinkTopic, 1); + pulsar.operator() + .admin() + .schemas() + .createSchema(tableSinkTopic, Schema.INT32.getSchemaInfo()); + tableEnv.useCatalog(PULSAR_CATALOG1); + tableEnv.useDatabase(PULSAR1_DB); + + String insertQ = + String.format( + "INSERT INTO %s" + + " VALUES\n" + + " (1),\n" + + " (2),\n" + + " (3),\n" + + " (4)", + tableSinkTopic); + tableEnv.executeSql(insertQ).await(); + + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", tableSinkTopic)), 4); + assertThat(result).hasSize(4); + } + + // utils + private void registerCatalogs(TableEnvironment tableEnvironment) { + tableEnvironment.registerCatalog( + INMEMORY_CATALOG, new GenericInMemoryCatalog(INMEMORY_CATALOG, INMEMORY_DB)); + + PulsarConfigBuilder configBuilder = new PulsarConfigBuilder(); + configBuilder.set(PulsarOptions.PULSAR_ADMIN_URL, pulsar.operator().adminUrl()); + configBuilder.set(PulsarOptions.PULSAR_SERVICE_URL, pulsar.operator().serviceUrl()); + tableEnvironment.registerCatalog( + PULSAR_CATALOG1, + new PulsarCatalog( + PULSAR_CATALOG1, + configBuilder.build( + CATALOG_CONFIG_VALIDATOR, PulsarCatalogConfiguration::new), + PULSAR1_DB, + FLINK_TENANT)); + + tableEnvironment.registerCatalog( + PULSAR_CATALOG2, + new PulsarCatalog( + PULSAR_CATALOG2, + configBuilder.build( + CATALOG_CONFIG_VALIDATOR, PulsarCatalogConfiguration::new), + PULSAR2_DB, + FLINK_TENANT)); + + tableEnvironment.useCatalog(INMEMORY_CATALOG); + } + + private String newDatabaseName() { + return "database" + RandomStringUtils.randomNumeric(8); + } + + private String newTopicName() { + return RandomStringUtils.randomAlphabetic(5); + } + + private TestingUser createRandomUser() { + TestingUser user = new TestingUser(); + user.setName(randomAlphabetic(5)); + user.setAge(ThreadLocalRandom.current().nextInt(0, Integer.MAX_VALUE)); + return user; + } + + private List consumeMessage(String topic, Schema schema, int count, int timeout) + throws InterruptedException, ExecutionException, TimeoutException { + final PulsarClient pulsarClient = pulsar.operator().client(); + return CompletableFuture.supplyAsync( + () -> { + Consumer consumer = null; + try { + consumer = + pulsarClient + .newConsumer(schema) + .topic(topic) + .subscriptionInitialPosition( + SubscriptionInitialPosition.Earliest) + .subscriptionName("test") + .subscribe(); + List result = new ArrayList<>(count); + for (int i = 0; i < count; i++) { + final Message message = consumer.receive(); + result.add(message.getValue()); + consumer.acknowledge(message); + } + consumer.close(); + return result; + } catch (Exception e) { + throw new IllegalStateException(e); + } finally { + IOUtils.closeQuietly(consumer, i -> {}); + } + }) + .get(timeout, TimeUnit.SECONDS); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/SchemaData.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/SchemaData.java new file mode 100644 index 0000000000000..9143af84a3eca --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/SchemaData.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.table.testutils; + +import java.util.Arrays; +import java.util.List; + +/** Data for various test cases. New test data can be added here */ +public class SchemaData { + public static final List INTEGER_LIST = Arrays.asList(1, 2, 3, 4, 5); +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java new file mode 100644 index 0000000000000..4ec6ea60c7879 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java @@ -0,0 +1,44 @@ +package org.apache.flink.connector.pulsar.table.testutils; + +import java.io.Serializable; +import java.util.Objects; + +/** A test POJO class. */ +public class TestingUser implements Serializable { + private static final long serialVersionUID = -1123545861004770003L; + public String name; + public Integer age; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public Integer getAge() { + return age; + } + + public void setAge(Integer age) { + this.age = age; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestingUser that = (TestingUser) o; + return Objects.equals(name, that.name) && Objects.equals(age, that.age); + } + + @Override + public int hashCode() { + return Objects.hash(name, age); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java index c87140ba42762..3bd6c4d1d68dc 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestSuiteBase.java @@ -56,7 +56,7 @@ public abstract class PulsarTestSuiteBase { * pulsar broker. Override this method when needs. */ protected PulsarRuntime runtime() { - return PulsarRuntime.embedded(); + return PulsarRuntime.container(); } /** Operate pulsar by acquiring a runtime operator. */ diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java index 9c1cd01a7f310..49cf95f3e9206 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java @@ -63,7 +63,17 @@ static PulsarRuntime embedded() { * disabled on Pulsar broker. */ static PulsarRuntime container() { - return new PulsarContainerRuntime(); + return new PulsarContainerRuntime(false); + } + + /** + * Create a Pulsar instance in docker. We would start a standalone Pulsar in TestContainers. + * This runtime is often used in end-to-end tests. The performance may be a bit of slower than + * {@link #embedded()}. The stream storage for bookkeeper is disabled. The function worker is + * disabled on Pulsar broker. + */ + static PulsarRuntime authContainer() { + return new PulsarContainerRuntime(true); } /** @@ -76,6 +86,6 @@ static PulsarRuntime container() { * enable the connection for Pulsar and Flink in docker environment. */ static PulsarRuntime container(GenericContainer flinkContainer) { - return new PulsarContainerRuntime().bindWithFlinkContainer(flinkContainer); + return new PulsarContainerRuntime(false).bindWithFlinkContainer(flinkContainer); } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java index 3d66728fded79..06e8cfc67caee 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java @@ -56,6 +56,9 @@ public class PulsarContainerRuntime implements PulsarRuntime { private static final String PULSAR_ADMIN_URL = String.format("http://%s:%d", PULSAR_INTERNAL_HOSTNAME, BROKER_HTTP_PORT); + private static final String TXN_CONFIG_FILE = "containers/txnStandalone.conf"; + private static final String AUTH_CONFIG_FILE = "containers/authStandalone.conf"; + /** * Create a pulsar container provider by a predefined version, this constance {@link * DockerImageVersions#PULSAR} should be bumped after the new pulsar release. @@ -66,6 +69,15 @@ public class PulsarContainerRuntime implements PulsarRuntime { private boolean boundFlink = false; private PulsarRuntimeOperator operator; + private String configFile; + + public PulsarContainerRuntime(boolean authEnabled) { + if (authEnabled) { + configFile = AUTH_CONFIG_FILE; + } else { + configFile = TXN_CONFIG_FILE; + } + } public PulsarContainerRuntime bindWithFlinkContainer(GenericContainer flinkContainer) { checkArgument( @@ -90,9 +102,7 @@ public void startUp() { // Override the default configuration in container for enabling the Pulsar transaction. container.withClasspathResourceMapping( - "containers/txnStandalone.conf", - "/pulsar/conf/standalone.conf", - BindMode.READ_ONLY); + configFile, "/pulsar/conf/standalone.conf", BindMode.READ_ONLY); // Waiting for the Pulsar border is ready. container.waitingFor( forHttp("/admin/v2/namespaces/public/default") diff --git a/flink-connectors/flink-sql-connector-pulsar/pom.xml b/flink-connectors/flink-sql-connector-pulsar/pom.xml index 7324bc33c2e5a..0ec3a32789d8d 100644 --- a/flink-connectors/flink-sql-connector-pulsar/pom.xml +++ b/flink-connectors/flink-sql-connector-pulsar/pom.xml @@ -69,6 +69,7 @@ under the License. org.bouncycastle:bcprov-jdk15on org.bouncycastle:bcutil-jdk15on org.slf4j:jul-to-slf4j + com.google.protobuf:* From 55a84a8467d2ba4d328c4126139ed9afe922c233 Mon Sep 17 00:00:00 2001 From: "affe (Yufei Zhang)" Date: Tue, 14 Jun 2022 23:56:12 +0800 Subject: [PATCH 210/258] 104: add sql-gateway dependencies json (#110) --- scripts/dependencies.json | 12 ++++++++++++ 1 file changed, 12 insertions(+) create mode 100644 scripts/dependencies.json diff --git a/scripts/dependencies.json b/scripts/dependencies.json new file mode 100644 index 0000000000000..e03c3f1e4f077 --- /dev/null +++ b/scripts/dependencies.json @@ -0,0 +1,12 @@ +[ + { + "name": "flink", + "dependencies": [ + { + "repo_owner": "streamnative", + "repo_name": "streamnative-sql-gateway", + "tag": "0.6.1-SNAPSHOT" + } + ] + } +] From 831aa3deb8c2d789cf6979671167b2c890bd2114 Mon Sep 17 00:00:00 2001 From: "affe (Yufei Zhang)" Date: Sun, 19 Jun 2022 09:00:27 +0800 Subject: [PATCH 211/258] 104: add protobuf jar dependency and declare as not provided (#114) --- flink-connectors/flink-connector-pulsar/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index 57feb0318fe08..10633c13836f1 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -98,7 +98,6 @@ under the License. com.google.protobuf protobuf-java ${protoc.version} - provided From f520ca4e29a69ce7360dc2a97df08d7b55d375e4 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Mon, 20 Jun 2022 20:27:28 +0800 Subject: [PATCH 212/258] [FLINK-27881][Connector/Pulsar] The key(String) in PulsarMessageBuilder returns null. --- .../pulsar/sink/writer/message/PulsarMessageBuilder.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java index f894f2410778c..fe4cef02964fa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/message/PulsarMessageBuilder.java @@ -57,7 +57,7 @@ public PulsarMessageBuilder orderingKey(byte[] orderingKey) { */ public PulsarMessageBuilder key(String key) { this.key = checkNotNull(key); - return null; + return this; } /** @@ -66,7 +66,7 @@ public PulsarMessageBuilder key(String key) { */ public PulsarMessageBuilder keyBytes(byte[] keyBytes) { this.keyBytes = checkNotNull(keyBytes); - return null; + return this; } /** Method wrapper of {@link TypedMessageBuilder#eventTime(long)}. */ From d338eeffe18619cc4d0f2b39c252742ad7cc08ca Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Wed, 18 May 2022 04:41:29 +0800 Subject: [PATCH 213/258] [FLINK-27399][Connector/Pulsar] Modify start cursor and stop cursor, unified the behaviors. --- .../docs/connectors/datastream/pulsar.md | 5 +- .../docs/connectors/datastream/pulsar.md | 2 +- .../enumerator/cursor/CursorPosition.java | 21 +- .../source/enumerator/cursor/StartCursor.java | 19 +- .../source/enumerator/cursor/StopCursor.java | 36 +- .../cursor/start/MessageIdStartCursor.java | 41 +- ....java => PublishTimestampStartCursor.java} | 8 +- .../cursor/stop/LatestMessageStopCursor.java | 21 +- .../cursor/stop/MessageIdStopCursor.java | 16 +- .../enumerator/cursor/StopCursorTest.java | 2 +- .../pyflink/datastream/connectors/pulsar.py | 774 ++++++++++++++++++ 11 files changed, 885 insertions(+), 60 deletions(-) rename flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/{TimestampStartCursor.java => PublishTimestampStartCursor.java} (85%) create mode 100644 flink-python/pyflink/datastream/connectors/pulsar.py diff --git a/docs/content.zh/docs/connectors/datastream/pulsar.md b/docs/content.zh/docs/connectors/datastream/pulsar.md index 68bf281510a43..4c402f0e48c61 100644 --- a/docs/content.zh/docs/connectors/datastream/pulsar.md +++ b/docs/content.zh/docs/connectors/datastream/pulsar.md @@ -217,9 +217,8 @@ Pulsar Source 使用 `setStartCursor(StartCursor)` 方法给定开始消费的 ```java StartCursor.fromMessageId(MessageId, boolean); ``` -- 从给定的消息时间开始消费。 - ```java - StartCursor.fromMessageTime(long); +- 从给定的消息发布时间开始消费。 + StartCursor.fromPublishTime(long) ``` {{< hint info >}} diff --git a/docs/content/docs/connectors/datastream/pulsar.md b/docs/content/docs/connectors/datastream/pulsar.md index a8c7057cdddf2..4d616e5f69cf0 100644 --- a/docs/content/docs/connectors/datastream/pulsar.md +++ b/docs/content/docs/connectors/datastream/pulsar.md @@ -258,7 +258,7 @@ The Pulsar connector consumes from the latest available message if the message I ``` - Start from the specified message time by `Message.getPublishTime()`. ```java - StartCursor.fromMessageTime(long); + StartCursor.fromPublishTime(long); ``` {{< hint info >}} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java index a2aaff629066d..e46e1906d63f5 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java @@ -19,11 +19,8 @@ package org.apache.flink.connector.pulsar.source.enumerator.cursor; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClientException; import javax.annotation.Nullable; @@ -52,22 +49,16 @@ public CursorPosition(@Nullable Long timestamp) { this.timestamp = timestamp; } - @VisibleForTesting + public Type getType() { + return type; + } + public MessageId getMessageId() { return messageId; } - /** Pulsar consumer could be subscribed by the position. */ - public void seekPosition(Consumer consumer) throws PulsarClientException { - if (type == Type.MESSAGE_ID) { - consumer.seek(messageId); - } else { - if (timestamp != null) { - consumer.seek(timestamp); - } else { - consumer.seek(System.currentTimeMillis()); - } - } + public Long getTimestamp() { + return timestamp; } @Override diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java index af35319a5a71e..26dfa537fe8d3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java @@ -20,11 +20,9 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.PublishTimestampStartCursor; -import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; import java.io.Serializable; @@ -43,13 +41,6 @@ public interface StartCursor extends Serializable { CursorPosition position(String topic, int partitionId); - /** Helper method for seek the right position for given pulsar consumer. */ - default void seekPosition(String topic, int partitionId, Consumer consumer) - throws PulsarClientException { - CursorPosition position = position(topic, partitionId); - position.seekPosition(consumer); - } - // --------------------------- Static Factory Methods ----------------------------- static StartCursor defaultStartCursor() { @@ -64,6 +55,10 @@ static StartCursor latest() { return fromMessageId(MessageId.latest); } + /** + * Find the available message id and start consuming from it. The given message is included in + * the consuming result by default. + */ static StartCursor fromMessageId(MessageId messageId) { return fromMessageId(messageId, true); } @@ -76,7 +71,7 @@ static StartCursor fromMessageId(MessageId messageId, boolean inclusive) { return new MessageIdStartCursor(messageId, inclusive); } - static StartCursor fromMessageTime(long timestamp) { - return new TimestampStartCursor(timestamp); + static StartCursor fromPublishTime(long timestamp) { + return new PublishTimestampStartCursor(timestamp); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java index 0bf46ce128274..3a55c4fbad73d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java @@ -65,28 +65,48 @@ static StopCursor latest() { } /** - * Stop when the messageId is equal or greater than the specified messageId. Message that is - * equal to the specified messageId will not be consumed. + * Stop consuming when the messageId is equal or greater than the specified messageId. Message + * that is equal to the specified messageId will not be consumed. */ static StopCursor atMessageId(MessageId messageId) { - return new MessageIdStopCursor(messageId); + if (MessageId.latest.equals(messageId)) { + return new LatestMessageStopCursor(true); + } else { + return new MessageIdStopCursor(messageId); + } } /** - * Stop when the messageId is greater than the specified messageId. Message that is equal to the - * specified messageId will be consumed. + * Stop consuming when the messageId is greater than the specified messageId. Message that is + * equal to the specified messageId will be consumed. */ static StopCursor afterMessageId(MessageId messageId) { - return new MessageIdStopCursor(messageId, false); + if (MessageId.latest.equals(messageId)) { + return new LatestMessageStopCursor(false); + } else { + return new MessageIdStopCursor(messageId, false); + } } - @Deprecated + /** Stop consuming when message eventTime is greater than or equals the specified timestamp. */ static StopCursor atEventTime(long timestamp) { return new EventTimestampStopCursor(timestamp); } - /** Stop when message publishTime is greater than the specified timestamp. */ + /** Stop consuming when message eventTime is greater than the specified timestamp. */ + static StopCursor afterEventTime(long timestamp) { + return new EventTimestampStopCursor(timestamp + 1); + } + + /** + * Stop consuming when message publishTime is greater than or equals the specified timestamp. + */ static StopCursor atPublishTime(long timestamp) { return new PublishTimestampStopCursor(timestamp); } + + /** Stop consuming when message publishTime is greater than the specified timestamp. */ + static StopCursor afterPublishTime(long timestamp) { + return new PublishTimestampStopCursor(timestamp + 1); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java index 3705f865ebc01..5ac3d90cbaff8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -23,7 +23,9 @@ import org.apache.pulsar.client.api.ConsumerBuilder; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; +import org.apache.pulsar.client.internal.DefaultImplementation; import java.util.Objects; @@ -48,20 +50,33 @@ public class MessageIdStartCursor implements StartCursor { * @param inclusive Should we include the start message id in consuming result. */ public MessageIdStartCursor(MessageId messageId, boolean inclusive) { - if (inclusive) { - this.messageId = messageId; + MessageIdImpl id = MessageIdImpl.convertToMessageIdImpl(messageId); + checkState( + !(id instanceof BatchMessageIdImpl), + "We only support normal message id currently."); + + if (MessageId.earliest.equals(id) || MessageId.latest.equals(id) || inclusive) { + this.messageId = id; + } else { + this.messageId = getNext(id); + } + } + + /** + * The implementation from the this + * code to get the next message id. + */ + private MessageId getNext(MessageIdImpl messageId) { + if (messageId.getEntryId() < 0) { + return DefaultImplementation.getDefaultImplementation() + .newMessageId(messageId.getLedgerId(), 0, messageId.getPartitionIndex()); } else { - checkState( - messageId instanceof MessageIdImpl, - "We only support normal message id and batch message id."); - MessageIdImpl id = (MessageIdImpl) messageId; - if (MessageId.earliest.equals(messageId) || MessageId.latest.equals(messageId)) { - this.messageId = messageId; - } else { - this.messageId = - new MessageIdImpl( - id.getLedgerId(), id.getEntryId() + 1, id.getPartitionIndex()); - } + return DefaultImplementation.getDefaultImplementation() + .newMessageId( + messageId.getLedgerId(), + messageId.getEntryId() + 1, + messageId.getPartitionIndex()); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/PublishTimestampStartCursor.java similarity index 85% rename from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java rename to flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/PublishTimestampStartCursor.java index e03a773281b83..d0d7428825c39 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/PublishTimestampStartCursor.java @@ -23,13 +23,13 @@ import java.util.Objects; -/** A cursor starts consuming from a specific publish timestamp. */ -public class TimestampStartCursor implements StartCursor { +/** This cursor make pulsar start consuming from a specific publish timestamp. */ +public class PublishTimestampStartCursor implements StartCursor { private static final long serialVersionUID = 5170578885838095320L; private final long timestamp; - public TimestampStartCursor(long timestamp) { + public PublishTimestampStartCursor(long timestamp) { this.timestamp = timestamp; } @@ -46,7 +46,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - TimestampStartCursor that = (TimestampStartCursor) o; + PublishTimestampStartCursor that = (PublishTimestampStartCursor) o; return timestamp == that.timestamp; } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java index 257081f5c8938..29dd68268c0dc 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java @@ -30,13 +30,26 @@ /** * A stop cursor that initialize the position to the latest message id. The offsets initialization * are taken care of by the {@code PulsarPartitionSplitReaderBase} instead of by the {@code - * PulsarSourceEnumerator}. + * PulsarSourceEnumerator}. We would include the latest message available in Pulsar by default. */ public class LatestMessageStopCursor implements StopCursor { private static final long serialVersionUID = 1702059838323965723L; private MessageId messageId; + /** + * Set this to false would include the latest available message when the flink pipeline start. + */ + private final boolean exclusive; + + public LatestMessageStopCursor() { + this.exclusive = false; + } + + public LatestMessageStopCursor(boolean exclusive) { + this.exclusive = exclusive; + } + @Override public void open(PulsarAdmin admin, TopicPartition partition) { if (messageId == null) { @@ -48,6 +61,10 @@ public void open(PulsarAdmin admin, TopicPartition partition) { @Override public boolean shouldStop(Message message) { MessageId id = message.getMessageId(); - return id.compareTo(messageId) >= 0; + if (exclusive) { + return id.compareTo(messageId) > 0; + } else { + return id.compareTo(messageId) >= 0; + } } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java index 7af55a00cc09f..3d7829a80dae0 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java @@ -22,6 +22,11 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.client.impl.MessageIdImpl; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkState; /** * Stop consuming message at a given message id. We use the {@link MessageId#compareTo(Object)} for @@ -39,7 +44,16 @@ public MessageIdStopCursor(MessageId messageId) { } public MessageIdStopCursor(MessageId messageId, boolean exclusive) { - this.messageId = messageId; + MessageIdImpl id = MessageIdImpl.convertToMessageIdImpl(messageId); + checkState( + !(id instanceof BatchMessageIdImpl), + "We only support normal message id currently."); + checkArgument(!MessageId.earliest.equals(id), "MessageId.earliest is not supported."); + checkArgument( + !MessageId.latest.equals(id), + "MessageId.latest is not supported, use LatestMessageStopCursor instead."); + + this.messageId = id; this.exclusive = exclusive; } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java index d003107793a84..d97120229883b 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java @@ -48,7 +48,7 @@ import static org.assertj.core.api.Assertions.assertThat; /** Test different implementation of StopCursor. */ -public class StopCursorTest extends PulsarTestSuiteBase { +class StopCursorTest extends PulsarTestSuiteBase { @Test void publishTimeStopCursor() throws IOException { diff --git a/flink-python/pyflink/datastream/connectors/pulsar.py b/flink-python/pyflink/datastream/connectors/pulsar.py new file mode 100644 index 0000000000000..4f4c60405e274 --- /dev/null +++ b/flink-python/pyflink/datastream/connectors/pulsar.py @@ -0,0 +1,774 @@ +################################################################################ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +################################################################################ +import warnings +from enum import Enum +from typing import Dict, Union, List + +from pyflink.common import DeserializationSchema, TypeInformation, ExecutionConfig, \ + ConfigOptions, Duration, SerializationSchema, ConfigOption +from pyflink.datastream.connectors import Source, Sink, DeliveryGuarantee +from pyflink.java_gateway import get_gateway +from pyflink.util.java_utils import load_java_class + + +# ---- PulsarSource ---- + + +class PulsarDeserializationSchema(object): + """ + A schema bridge for deserializing the pulsar's Message into a flink managed instance. We + support both the pulsar's self managed schema and flink managed schema. + """ + + def __init__(self, _j_pulsar_deserialization_schema): + self._j_pulsar_deserialization_schema = _j_pulsar_deserialization_schema + + @staticmethod + def flink_schema(deserialization_schema: DeserializationSchema) \ + -> 'PulsarDeserializationSchema': + """ + Create a PulsarDeserializationSchema by using the flink's DeserializationSchema. It would + consume the pulsar message as byte array and decode the message by using flink's logic. + """ + JPulsarDeserializationSchema = get_gateway().jvm.org.apache.flink \ + .connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema + _j_pulsar_deserialization_schema = JPulsarDeserializationSchema.flinkSchema( + deserialization_schema._j_deserialization_schema) + return PulsarDeserializationSchema(_j_pulsar_deserialization_schema) + + @staticmethod + def flink_type_info(type_information: TypeInformation, + execution_config: ExecutionConfig = None) -> 'PulsarDeserializationSchema': + """ + Create a PulsarDeserializationSchema by using the given TypeInformation. This method is + only used for treating message that was written into pulsar by TypeInformation. + """ + JPulsarDeserializationSchema = get_gateway().jvm.org.apache.flink \ + .connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema + JExecutionConfig = get_gateway().jvm.org.apache.flink.api.common.ExecutionConfig + _j_execution_config = execution_config._j_execution_config \ + if execution_config is not None else JExecutionConfig() + _j_pulsar_deserialization_schema = JPulsarDeserializationSchema.flinkTypeInfo( + type_information.get_java_type_info(), _j_execution_config) + return PulsarDeserializationSchema(_j_pulsar_deserialization_schema) + + +class SubscriptionType(Enum): + """ + Types of subscription supported by Pulsar. + + :data: `Exclusive`: + + There can be only 1 consumer on the same topic with the same subscription name. + + :data: `Shared`: + + Multiple consumer will be able to use the same subscription name and the messages will be + dispatched according to a round-robin rotation between the connected consumers. In this mode, + the consumption order is not guaranteed. + + :data: `Failover`: + + Multiple consumer will be able to use the same subscription name but only 1 consumer will + receive the messages. If that consumer disconnects, one of the other connected consumers will + start receiving messages. In failover mode, the consumption ordering is guaranteed. In case of + partitioned topics, the ordering is guaranteed on a per-partition basis. The partitions + assignments will be split across the available consumers. On each partition, at most one + consumer will be active at a given point in time. + + :data: `Key_Shared`: + + Multiple consumer will be able to use the same subscription and all messages with the same key + will be dispatched to only one consumer. Use ordering_key to overwrite the message key for + message ordering. + """ + + Exclusive = 0, + Shared = 1, + Failover = 2, + Key_Shared = 3 + + def _to_j_subscription_type(self): + JSubscriptionType = get_gateway().jvm.org.apache.pulsar.client.api.SubscriptionType + return getattr(JSubscriptionType, self.name) + + +class StartCursor(object): + """ + A factory class for users to specify the start position of a pulsar subscription. + Since it would be serialized into split. + The implementation for this interface should be well considered. + I don't recommend adding extra internal state for this implementation. + + This class would be used only for SubscriptionType.Exclusive and SubscriptionType.Failover. + """ + + def __init__(self, _j_start_cursor): + self._j_start_cursor = _j_start_cursor + + @staticmethod + def default_start_cursor() -> 'StartCursor': + return StartCursor.earliest() + + @staticmethod + def earliest() -> 'StartCursor': + JStartCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor + return StartCursor(JStartCursor.earliest()) + + @staticmethod + def latest() -> 'StartCursor': + JStartCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor + return StartCursor(JStartCursor.latest()) + + @staticmethod + def from_publish_time(timestamp: int) -> 'StartCursor': + JStartCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor + return StartCursor(JStartCursor.fromPublishTime(timestamp)) + + @staticmethod + def from_message_id(message_id: bytes, inclusive: bool = True) -> 'StartCursor': + """ + Find the available message id and start consuming from it. User could call pulsar Python + library serialize method to cover messageId bytes. + + Example: + :: + + >>> from pulsar import MessageId + >>> message_id_bytes = MessageId().serialize() + >>> start_cursor = StartCursor.from_message_id(message_id_bytes) + """ + JStartCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor + j_message_id = get_gateway().jvm.org.apache.pulsar.client.api.MessageId \ + .fromByteArray(message_id) + return StartCursor(JStartCursor.fromMessageId(j_message_id, inclusive)) + + +class StopCursor(object): + """ + A factory class for users to specify the stop position of a pulsar subscription. Since it would + be serialized into split. The implementation for this interface should be well considered. I + don't recommend adding extra internal state for this implementation. + """ + + def __init__(self, _j_stop_cursor): + self._j_stop_cursor = _j_stop_cursor + + @staticmethod + def default_stop_cursor() -> 'StopCursor': + return StopCursor.never() + + @staticmethod + def never() -> 'StopCursor': + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + return StopCursor(JStopCursor.never()) + + @staticmethod + def latest() -> 'StopCursor': + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + return StopCursor(JStopCursor.latest()) + + @staticmethod + def at_event_time(timestamp: int) -> 'StopCursor': + warnings.warn( + "at_event_time is deprecated. Use at_publish_time instead.", DeprecationWarning) + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + return StopCursor(JStopCursor.atEventTime(timestamp)) + + @staticmethod + def at_publish_time(timestamp: int) -> 'StopCursor': + """ + Stop when message publishTime is greater than the specified timestamp. + """ + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + return StopCursor(JStopCursor.atPublishTime(timestamp)) + + @staticmethod + def at_message_id(message_id: bytes) -> 'StopCursor': + """ + Stop when the messageId is equal or greater than the specified messageId. Message that is + equal to the specified messageId will not be consumed. User could call pulsar Python + library serialize method to cover messageId bytes. + + Example: + :: + + >>> from pulsar import MessageId + >>> message_id_bytes = MessageId().serialize() + >>> stop_cursor = StopCursor.at_message_id(message_id_bytes) + """ + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + j_message_id = get_gateway().jvm.org.apache.pulsar.client.api.MessageId \ + .fromByteArray(message_id) + return StopCursor(JStopCursor.atMessageId(j_message_id)) + + @staticmethod + def after_message_id(message_id: bytes) -> 'StopCursor': + """ + Stop when the messageId is greater than the specified messageId. Message that is equal to + the specified messageId will be consumed. User could call pulsar Python library serialize + method to cover messageId bytes. + + Example: + :: + + >>> from pulsar import MessageId + >>> message_id_bytes = MessageId().serialize() + >>> stop_cursor = StopCursor.after_message_id(message_id_bytes) + """ + JStopCursor = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor + j_message_id = get_gateway().jvm.org.apache.pulsar.client.api.MessageId \ + .fromByteArray(message_id) + return StopCursor(JStopCursor.afterMessageId(j_message_id)) + + +class PulsarSource(Source): + """ + The Source implementation of Pulsar. Please use a PulsarSourceBuilder to construct a + PulsarSource. The following example shows how to create a PulsarSource emitting records of + String type. + + Example: + :: + + >>> source = PulsarSource() \\ + ... .builder() \\ + ... .set_topics([TOPIC1, TOPIC2]) \\ + ... .set_service_url(get_service_url()) \\ + ... .set_admin_url(get_admin_url()) \\ + ... .set_subscription_name("test") \\ + ... .set_deserialization_schema( + ... PulsarDeserializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .set_bounded_stop_cursor(StopCursor.default_stop_cursor()) \\ + ... .build() + + See PulsarSourceBuilder for more details. + """ + + def __init__(self, j_pulsar_source): + super(PulsarSource, self).__init__(source=j_pulsar_source) + + @staticmethod + def builder() -> 'PulsarSourceBuilder': + """ + Get a PulsarSourceBuilder to builder a PulsarSource. + """ + return PulsarSourceBuilder() + + +class PulsarSourceBuilder(object): + """ + The builder class for PulsarSource to make it easier for the users to construct a PulsarSource. + + The following example shows the minimum setup to create a PulsarSource that reads the String + values from a Pulsar topic. + + Example: + :: + + >>> source = PulsarSource() \\ + ... .builder() \\ + ... .set_service_url(PULSAR_BROKER_URL) \\ + ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \\ + ... .set_subscription_name("flink-source-1") \\ + ... .set_topics([TOPIC1, TOPIC2]) \\ + ... .set_deserialization_schema( + ... PulsarDeserializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .build() + + The service url, admin url, subscription name, topics to consume, and the record deserializer + are required fields that must be set. + + To specify the starting position of PulsarSource, one can call set_start_cursor(StartCursor). + + By default the PulsarSource runs in an Boundedness.CONTINUOUS_UNBOUNDED mode and never stop + until the Flink job is canceled or fails. To let the PulsarSource run in + Boundedness.CONTINUOUS_UNBOUNDED but stops at some given offsets, one can call + set_unbounded_stop_cursor(StopCursor). + + For example the following PulsarSource stops after it consumes up to a event time when the + Flink started. + + Example: + :: + + >>> source = PulsarSource() \\ + ... .builder() \\ + ... .set_service_url(PULSAR_BROKER_URL) \\ + ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \\ + ... .set_subscription_name("flink-source-1") \\ + ... .set_topics([TOPIC1, TOPIC2]) \\ + ... .set_deserialization_schema( + ... PulsarDeserializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .set_bounded_stop_cursor(StopCursor.at_publish_time(int(time.time() * 1000))) + ... .build() + """ + + def __init__(self): + JPulsarSource = \ + get_gateway().jvm.org.apache.flink.connector.pulsar.source.PulsarSource + self._j_pulsar_source_builder = JPulsarSource.builder() + + def set_admin_url(self, admin_url: str) -> 'PulsarSourceBuilder': + """ + Sets the admin endpoint for the PulsarAdmin of the PulsarSource. + """ + self._j_pulsar_source_builder.setAdminUrl(admin_url) + return self + + def set_service_url(self, service_url: str) -> 'PulsarSourceBuilder': + """ + Sets the server's link for the PulsarConsumer of the PulsarSource. + """ + self._j_pulsar_source_builder.setServiceUrl(service_url) + return self + + def set_subscription_name(self, subscription_name: str) -> 'PulsarSourceBuilder': + """ + Sets the name for this pulsar subscription. + """ + self._j_pulsar_source_builder.setSubscriptionName(subscription_name) + return self + + def set_subscription_type(self, subscription_type: SubscriptionType) -> 'PulsarSourceBuilder': + """ + SubscriptionType is the consuming behavior for pulsar, we would generator different split + by the given subscription type. Please take some time to consider which subscription type + matches your application best. Default is SubscriptionType.Shared. + """ + self._j_pulsar_source_builder.setSubscriptionType( + subscription_type._to_j_subscription_type()) + return self + + def set_topics(self, topics: Union[str, List[str]]) -> 'PulsarSourceBuilder': + """ + Set a pulsar topic list for flink source. Some topic may not exist currently, consuming this + non-existed topic wouldn't throw any exception. But the best solution is just consuming by + using a topic regex. You can set topics once either with setTopics or setTopicPattern in + this builder. + """ + if not isinstance(topics, list): + topics = [topics] + self._j_pulsar_source_builder.setTopics(topics) + return self + + def set_topics_pattern(self, topics_pattern: str) -> 'PulsarSourceBuilder': + """ + Set a topic pattern to consume from the java regex str. You can set topics once either with + set_topics or set_topic_pattern in this builder. + """ + warnings.warn("set_topics_pattern is deprecated. Use set_topic_pattern instead.", + DeprecationWarning, stacklevel=2) + self._j_pulsar_source_builder.setTopicPattern(topics_pattern) + return self + + def set_topic_pattern(self, topic_pattern: str) -> 'PulsarSourceBuilder': + """ + Set a topic pattern to consume from the java regex str. You can set topics once either with + set_topics or set_topic_pattern in this builder. + """ + self._j_pulsar_source_builder.setTopicPattern(topic_pattern) + return self + + def set_start_cursor(self, start_cursor: StartCursor) -> 'PulsarSourceBuilder': + """ + Specify from which offsets the PulsarSource should start consume from by providing an + StartCursor. + """ + self._j_pulsar_source_builder.setStartCursor(start_cursor._j_start_cursor) + return self + + def set_unbounded_stop_cursor(self, stop_cursor: StopCursor) -> 'PulsarSourceBuilder': + """ + By default the PulsarSource is set to run in Boundedness.CONTINUOUS_UNBOUNDED manner and + thus never stops until the Flink job fails or is canceled. To let the PulsarSource run as a + streaming source but still stops at some point, one can set an StopCursor to specify the + stopping offsets for each partition. When all the partitions have reached their stopping + offsets, the PulsarSource will then exit. + + This method is different from set_bounded_stop_cursor(StopCursor) that after setting the + stopping offsets with this method, PulsarSource.getBoundedness() will still return + Boundedness.CONTINUOUS_UNBOUNDED even though it will stop at the stopping offsets specified + by the stopping offsets StopCursor. + """ + self._j_pulsar_source_builder.setUnboundedStopCursor(stop_cursor._j_stop_cursor) + return self + + def set_bounded_stop_cursor(self, stop_cursor: StopCursor) -> 'PulsarSourceBuilder': + """ + By default the PulsarSource is set to run in Boundedness.CONTINUOUS_UNBOUNDED manner and + thus never stops until the Flink job fails or is canceled. To let the PulsarSource run in + Boundedness.BOUNDED manner and stops at some point, one can set an StopCursor to specify + the stopping offsets for each partition. When all the partitions have reached their stopping + offsets, the PulsarSource will then exit. + + This method is different from set_unbounded_stop_cursor(StopCursor) that after setting the + stopping offsets with this method, PulsarSource.getBoundedness() will return + Boundedness.BOUNDED instead of Boundedness.CONTINUOUS_UNBOUNDED. + """ + self._j_pulsar_source_builder.setBoundedStopCursor(stop_cursor._j_stop_cursor) + return self + + def set_deserialization_schema(self, + pulsar_deserialization_schema: PulsarDeserializationSchema) \ + -> 'PulsarSourceBuilder': + """ + DeserializationSchema is required for getting the Schema for deserialize message from + pulsar and getting the TypeInformation for message serialization in flink. + + We have defined a set of implementations, using PulsarDeserializationSchema#flink_type_info + or PulsarDeserializationSchema#flink_schema for creating the desired schema. + """ + self._j_pulsar_source_builder.setDeserializationSchema( + pulsar_deserialization_schema._j_pulsar_deserialization_schema) + return self + + def set_config(self, key: Union[str, ConfigOption], value) -> 'PulsarSourceBuilder': + """ + Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be + found in PulsarSourceOptions and PulsarOptions. + + Make sure the option could be set only once or with same value. + """ + if isinstance(key, ConfigOption): + warnings.warn("set_config(key: ConfigOption, value) is deprecated. " + "Use set_config(key: str, value) instead.", + DeprecationWarning, stacklevel=2) + j_config_option = key._j_config_option + else: + j_config_option = \ + ConfigOptions.key(key).string_type().no_default_value()._j_config_option + self._j_pulsar_source_builder.setConfig(j_config_option, value) + return self + + def set_config_with_dict(self, config: Dict) -> 'PulsarSourceBuilder': + """ + Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be + found in PulsarSourceOptions and PulsarOptions. + """ + warnings.warn("set_config_with_dict is deprecated. Use set_properties instead.", + DeprecationWarning, stacklevel=2) + self.set_properties(config) + return self + + def set_properties(self, config: Dict) -> 'PulsarSourceBuilder': + """ + Set arbitrary properties for the PulsarSource and PulsarConsumer. The valid keys can be + found in PulsarSourceOptions and PulsarOptions. + """ + JConfiguration = get_gateway().jvm.org.apache.flink.configuration.Configuration + self._j_pulsar_source_builder.setConfig(JConfiguration.fromMap(config)) + return self + + def build(self) -> 'PulsarSource': + """ + Build the PulsarSource. + """ + return PulsarSource(self._j_pulsar_source_builder.build()) + + +# ---- PulsarSink ---- + + +class PulsarSerializationSchema(object): + """ + The serialization schema for how to serialize records into Pulsar. + """ + + def __init__(self, _j_pulsar_serialization_schema): + self._j_pulsar_serialization_schema = _j_pulsar_serialization_schema + + @staticmethod + def flink_schema(serialization_schema: SerializationSchema) \ + -> 'PulsarSerializationSchema': + """ + Create a PulsarSerializationSchema by using the flink's SerializationSchema. It would + serialize the message into byte array and send it to Pulsar with Schema#BYTES. + """ + JPulsarSerializationSchema = get_gateway().jvm.org.apache.flink \ + .connector.pulsar.sink.writer.serializer.PulsarSerializationSchema + _j_pulsar_serialization_schema = JPulsarSerializationSchema.flinkSchema( + serialization_schema._j_serialization_schema) + return PulsarSerializationSchema(_j_pulsar_serialization_schema) + + +class TopicRoutingMode(Enum): + """ + The routing policy for choosing the desired topic by the given message. + + :data: `ROUND_ROBIN`: + + The producer will publish messages across all partitions in a round-robin fashion to achieve + maximum throughput. Please note that round-robin is not done per individual message but + rather it's set to the same boundary of batching delay, to ensure batching is effective. + + :data: `MESSAGE_KEY_HASH`: + + If no key is provided, The partitioned producer will randomly pick one single topic partition + and publish all the messages into that partition. If a key is provided on the message, the + partitioned producer will hash the key and assign the message to a particular partition. + + :data: `CUSTOM`: + + Use custom topic router implementation that will be called to determine the partition for a + particular message. + """ + + ROUND_ROBIN = 0 + MESSAGE_KEY_HASH = 1 + CUSTOM = 2 + + def _to_j_topic_routing_mode(self): + JTopicRoutingMode = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode + return getattr(JTopicRoutingMode, self.name) + + +class MessageDelayer(object): + """ + A delayer for Pulsar broker passing the sent message to the downstream consumer. This is only + works in :data:`SubscriptionType.Shared` subscription. + + Read delayed message delivery + https://pulsar.apache.org/docs/en/next/concepts-messaging/#delayed-message-delivery for better + understanding this feature. + """ + def __init__(self, _j_message_delayer): + self._j_message_delayer = _j_message_delayer + + @staticmethod + def never() -> 'MessageDelayer': + """ + All the messages should be consumed immediately. + """ + JMessageDelayer = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.sink.writer.delayer.MessageDelayer + return MessageDelayer(JMessageDelayer.never()) + + @staticmethod + def fixed(duration: Duration) -> 'MessageDelayer': + """ + All the messages should be consumed in a fixed duration. + """ + JMessageDelayer = get_gateway().jvm \ + .org.apache.flink.connector.pulsar.sink.writer.delayer.MessageDelayer + return MessageDelayer(JMessageDelayer.fixed(duration._j_duration)) + + +class PulsarSink(Sink): + """ + The Sink implementation of Pulsar. Please use a PulsarSinkBuilder to construct a + PulsarSink. The following example shows how to create a PulsarSink receiving records of + String type. + + Example: + :: + + >>> sink = PulsarSink.builder() \\ + ... .set_service_url(PULSAR_BROKER_URL) \\ + ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \\ + ... .set_topics(topic) \\ + ... .set_serialization_schema( + ... PulsarSerializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .build() + + The sink supports all delivery guarantees described by DeliveryGuarantee. + + DeliveryGuarantee#NONE does not provide any guarantees: messages may be lost in + case of issues on the Pulsar broker and messages may be duplicated in case of a Flink + failure. + + DeliveryGuarantee#AT_LEAST_ONCE the sink will wait for all outstanding records in + the Pulsar buffers to be acknowledged by the Pulsar producer on a checkpoint. No messages + will be lost in case of any issue with the Pulsar brokers but messages may be duplicated + when Flink restarts. + + DeliveryGuarantee#EXACTLY_ONCE: In this mode the PulsarSink will write all messages + in a Pulsar transaction that will be committed to Pulsar on a checkpoint. Thus, no + duplicates will be seen in case of a Flink restart. However, this delays record writing + effectively until a checkpoint is written, so adjust the checkpoint duration accordingly. + Additionally, it is highly recommended to tweak Pulsar transaction timeout (link) >> + maximum checkpoint duration + maximum restart duration or data loss may happen when Pulsar + expires an uncommitted transaction. + + See PulsarSinkBuilder for more details. + """ + + def __init__(self, j_pulsar_sink): + super(PulsarSink, self).__init__(sink=j_pulsar_sink) + + @staticmethod + def builder() -> 'PulsarSinkBuilder': + """ + Get a PulsarSinkBuilder to builder a PulsarSink. + """ + return PulsarSinkBuilder() + + +class PulsarSinkBuilder(object): + """ + The builder class for PulsarSink to make it easier for the users to construct a PulsarSink. + + The following example shows the minimum setup to create a PulsarSink that reads the String + values from a Pulsar topic. + + Example: + :: + + >>> sink = PulsarSink.builder() \\ + ... .set_service_url(PULSAR_BROKER_URL) \\ + ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \\ + ... .set_topics([TOPIC1, TOPIC2]) \\ + ... .set_serialization_schema( + ... PulsarSerializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .build() + + The service url, admin url, and the record serializer are required fields that must be set. If + you don't set the topics, make sure you have provided a custom TopicRouter. Otherwise, + you must provide the topics to produce. + + To specify the delivery guarantees of PulsarSink, one can call + #setDeliveryGuarantee(DeliveryGuarantee). The default value of the delivery guarantee is + DeliveryGuarantee#NONE, and it wouldn't promise the consistence when write the message into + Pulsar. + + Example: + :: + + >>> sink = PulsarSink.builder() \\ + ... .set_service_url(PULSAR_BROKER_URL) \\ + ... .set_admin_url(PULSAR_BROKER_HTTP_URL) \\ + ... .set_topics([TOPIC1, TOPIC2]) \\ + ... .set_serialization_schema( + ... PulsarSerializationSchema.flink_schema(SimpleStringSchema())) \\ + ... .set_delivery_guarantee(DeliveryGuarantee.EXACTLY_ONCE) + ... .build() + """ + + def __init__(self): + JPulsarSink = get_gateway().jvm.org.apache.flink.connector.pulsar.sink.PulsarSink + self._j_pulsar_sink_builder = JPulsarSink.builder() + + def set_admin_url(self, admin_url: str) -> 'PulsarSinkBuilder': + """ + Sets the admin endpoint for the PulsarAdmin of the PulsarSink. + """ + self._j_pulsar_sink_builder.setAdminUrl(admin_url) + return self + + def set_service_url(self, service_url: str) -> 'PulsarSinkBuilder': + """ + Sets the server's link for the PulsarProducer of the PulsarSink. + """ + self._j_pulsar_sink_builder.setServiceUrl(service_url) + return self + + def set_producer_name(self, producer_name: str) -> 'PulsarSinkBuilder': + """ + The producer name is informative, and it can be used to identify a particular producer + instance from the topic stats. + """ + self._j_pulsar_sink_builder.setProducerName(producer_name) + return self + + def set_topics(self, topics: Union[str, List[str]]) -> 'PulsarSinkBuilder': + """ + Set a pulsar topic list for flink sink. Some topic may not exist currently, write to this + non-existed topic wouldn't throw any exception. + """ + if not isinstance(topics, list): + topics = [topics] + self._j_pulsar_sink_builder.setTopics(topics) + return self + + def set_delivery_guarantee(self, delivery_guarantee: DeliveryGuarantee) -> 'PulsarSinkBuilder': + """ + Sets the wanted the DeliveryGuarantee. The default delivery guarantee is + DeliveryGuarantee#NONE. + """ + self._j_pulsar_sink_builder.setDeliveryGuarantee( + delivery_guarantee._to_j_delivery_guarantee()) + return self + + def set_topic_routing_mode(self, topic_routing_mode: TopicRoutingMode) -> 'PulsarSinkBuilder': + """ + Set a routing mode for choosing right topic partition to send messages. + """ + self._j_pulsar_sink_builder.setTopicRoutingMode( + topic_routing_mode._to_j_topic_routing_mode()) + return self + + def set_topic_router(self, topic_router_class_name: str) -> 'PulsarSinkBuilder': + """ + Use a custom topic router instead predefine topic routing. + """ + j_topic_router = load_java_class(topic_router_class_name).newInstance() + self._j_pulsar_sink_builder.setTopicRouter(j_topic_router) + return self + + def set_serialization_schema(self, pulsar_serialization_schema: PulsarSerializationSchema) \ + -> 'PulsarSinkBuilder': + """ + Sets the PulsarSerializationSchema that transforms incoming records to bytes. + """ + self._j_pulsar_sink_builder.setSerializationSchema( + pulsar_serialization_schema._j_pulsar_serialization_schema) + return self + + def delay_sending_message(self, message_delayer: MessageDelayer) -> 'PulsarSinkBuilder': + """ + Set a message delayer for enable Pulsar message delay delivery. + """ + self._j_pulsar_sink_builder.delaySendingMessage(message_delayer._j_message_delayer) + return self + + def set_config(self, key: str, value) -> 'PulsarSinkBuilder': + """ + Set an arbitrary property for the PulsarSink and Pulsar Producer. The valid keys can be + found in PulsarSinkOptions and PulsarOptions. + + Make sure the option could be set only once or with same value. + """ + j_config_option = ConfigOptions.key(key).string_type().no_default_value()._j_config_option + self._j_pulsar_sink_builder.setConfig(j_config_option, value) + return self + + def set_properties(self, config: Dict) -> 'PulsarSinkBuilder': + """ + Set an arbitrary property for the PulsarSink and Pulsar Producer. The valid keys can be + found in PulsarSinkOptions and PulsarOptions. + """ + JConfiguration = get_gateway().jvm.org.apache.flink.configuration.Configuration + self._j_pulsar_sink_builder.setConfig(JConfiguration.fromMap(config)) + return self + + def build(self) -> 'PulsarSink': + """ + Build the PulsarSink. + """ + return PulsarSink(self._j_pulsar_sink_builder.build()) From 9257766c4c5759905da090fed46deac6fdb07a09 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Wed, 15 Jun 2022 05:34:35 +0800 Subject: [PATCH 214/258] [FLINK-27399][Connector/Pulsar] Create the initial subscription instead seek every time. This should fix the wrong position setting. --- .../connector/pulsar/source/PulsarSource.java | 14 +- .../pulsar/source/PulsarSourceBuilder.java | 8 +- .../pulsar/source/PulsarSourceOptions.java | 3 + .../source/config/SourceConfiguration.java | 14 +- .../enumerator/PulsarSourceEnumState.java | 7 +- .../enumerator/PulsarSourceEnumerator.java | 163 ++++++------ .../enumerator/SplitsAssignmentState.java | 239 ------------------ .../assigner/NormalSplitAssigner.java | 135 ++++++++++ .../assigner/SharedSplitAssigner.java | 157 ++++++++++++ .../enumerator/assigner/SplitAssigner.java | 64 +++++ .../assigner/SplitAssignerFactory.java | 72 ++++++ .../source/enumerator/cursor/StartCursor.java | 6 +- .../cursor/start/MessageIdStartCursor.java | 6 +- .../subscriber/PulsarSubscriber.java | 2 +- .../PulsarOrderedPartitionSplitReader.java | 22 +- .../source/split/PulsarPartitionSplit.java | 4 +- .../PulsarSourceEnumeratorTest.java | 10 +- .../enumerator/SplitsAssignmentStateTest.java | 119 --------- .../PulsarPartitionSplitReaderTestBase.java | 4 +- 19 files changed, 562 insertions(+), 487 deletions(-) delete mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NormalSplitAssigner.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java delete mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java index 560b32162f2b3..6b2752ed2970b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java @@ -32,7 +32,8 @@ import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumStateSerializer; import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumerator; -import org.apache.flink.connector.pulsar.source.enumerator.SplitsAssignmentState; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssignerFactory; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; @@ -142,15 +143,14 @@ public SourceReader createReader(SourceReaderContext @Override public SplitEnumerator createEnumerator( SplitEnumeratorContext enumContext) { - SplitsAssignmentState assignmentState = - new SplitsAssignmentState(stopCursor, sourceConfiguration); + SplitAssigner splitAssigner = SplitAssignerFactory.create(stopCursor, sourceConfiguration); return new PulsarSourceEnumerator( subscriber, startCursor, rangeGenerator, sourceConfiguration, enumContext, - assignmentState); + splitAssigner); } @Internal @@ -158,15 +158,15 @@ public SplitEnumerator createEnumer public SplitEnumerator restoreEnumerator( SplitEnumeratorContext enumContext, PulsarSourceEnumState checkpoint) { - SplitsAssignmentState assignmentState = - new SplitsAssignmentState(stopCursor, sourceConfiguration, checkpoint); + SplitAssigner splitAssigner = + SplitAssignerFactory.create(stopCursor, sourceConfiguration, checkpoint); return new PulsarSourceEnumerator( subscriber, startCursor, rangeGenerator, sourceConfiguration, enumContext, - assignmentState); + splitAssigner); } @Internal diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java index b6e0f408d5bbf..a0a0050e1589b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java @@ -99,7 +99,7 @@ *

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

    {@code
      * PulsarSource source = PulsarSource
    @@ -268,7 +268,7 @@ public PulsarSourceBuilder setTopicPattern(
         }
     
         /**
    -     * The consumer name is informative and it can be used to identify a particular consumer
    +     * The consumer name is informative, and it can be used to identify a particular consumer
          * instance from the topic stats.
          */
         public PulsarSourceBuilder setConsumerName(String consumerName) {
    @@ -332,7 +332,7 @@ public PulsarSourceBuilder setStartCursor(StartCursor startCursor) {
          * 

    To stop the connector user has to disable the auto partition discovery. As auto partition * discovery always expected new splits to come and not exiting. To disable auto partition * discovery, use builder.setConfig({@link - * PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). + * PulsarSourceOptions#PULSAR_PARTITION_DISCOVERY_INTERVAL_MS}, -1). * * @param stopCursor The {@link StopCursor} to specify the stopping offset. * @return this PulsarSourceBuilder. @@ -345,7 +345,7 @@ public PulsarSourceBuilder setUnboundedStopCursor(StopCursor stopCursor) { } /** - * By default the PulsarSource is set to run in {@link Boundedness#CONTINUOUS_UNBOUNDED} manner + * By default, the PulsarSource is set to run in {@link Boundedness#CONTINUOUS_UNBOUNDED} manner * and thus never stops until the Flink job fails or is canceled. To let the PulsarSource run in * {@link Boundedness#BOUNDED} manner and stops at some point, one can set an {@link StopCursor} * to specify the stopping offsets for each partition. When all the partitions have reached diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java index 70ddb6793dfd8..1c6808bf01db1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java @@ -26,6 +26,7 @@ import org.apache.flink.configuration.description.Description; import org.apache.flink.connector.pulsar.common.config.PulsarOptions; import org.apache.flink.connector.pulsar.source.config.CursorVerification; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.pulsar.client.api.ConsumerCryptoFailureAction; import org.apache.pulsar.client.api.SubscriptionInitialPosition; @@ -514,6 +515,8 @@ private PulsarSourceOptions() { code("PulsarClientException")) .build()); + /** @deprecated This option would be reset by {@link StartCursor}, no need to use it anymore. */ + @Deprecated public static final ConfigOption PULSAR_SUBSCRIPTION_INITIAL_POSITION = ConfigOptions.key(CONSUMER_CONFIG_PREFIX + "subscriptionInitialPosition") diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java index bd6c61216acd7..4cc5f462861bb 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java @@ -88,6 +88,10 @@ public int getMessageQueueCapacity() { return messageQueueCapacity; } + /** + * We would override the interval into a negative number when we set the connector with bounded + * stop cursor. + */ public boolean isEnablePartitionDiscovery() { return getPartitionDiscoveryIntervalMs() > 0; } @@ -192,16 +196,6 @@ public SubscriptionMode getSubscriptionMode() { return subscriptionMode; } - /** Convert the subscription into a readable str. */ - public String getSubscriptionDesc() { - return getSubscriptionName() - + "(" - + getSubscriptionType() - + "," - + getSubscriptionMode() - + ")"; - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java index dbab9e2178123..0f3c66d40e8d0 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.pulsar.source.enumerator; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; @@ -26,7 +27,7 @@ /** * The state class for pulsar source enumerator, used for storing the split state. This class is - * managed and controlled by {@link SplitsAssignmentState}. + * managed and controlled by {@link SplitAssigner}. */ public class PulsarSourceEnumState { @@ -46,8 +47,8 @@ public class PulsarSourceEnumState { private final Map> sharedPendingPartitionSplits; /** - * A {@link PulsarPartitionSplit} should be assigned for all flink readers. Using this map for - * recording assign status. + * It is used for Shared subscription. A {@link PulsarPartitionSplit} should be assigned for all + * flink readers. Using this map for recording assign status. */ private final Map> readerAssignedSplits; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java index 7890dcf1847f7..84ffbcd1a7224 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumerator.java @@ -22,40 +22,29 @@ import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.CursorPosition; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.util.FlinkRuntimeException; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.api.Consumer; -import org.apache.pulsar.client.api.ConsumerBuilder; -import org.apache.pulsar.client.api.KeySharedPolicy; -import org.apache.pulsar.client.api.KeySharedPolicy.KeySharedPolicySticky; -import org.apache.pulsar.client.api.PulsarClient; -import org.apache.pulsar.client.api.PulsarClientException; -import org.apache.pulsar.client.api.Range; -import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.client.api.MessageId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.HashSet; import java.util.List; import java.util.Set; import static java.util.Collections.singletonList; import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; -import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createClient; -import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; -import static org.apache.flink.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH; -import static org.apache.flink.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; /** The enumerator class for pulsar source. */ @Internal @@ -65,13 +54,12 @@ public class PulsarSourceEnumerator private static final Logger LOG = LoggerFactory.getLogger(PulsarSourceEnumerator.class); private final PulsarAdmin pulsarAdmin; - private final PulsarClient pulsarClient; private final PulsarSubscriber subscriber; private final StartCursor startCursor; private final RangeGenerator rangeGenerator; private final SourceConfiguration sourceConfiguration; private final SplitEnumeratorContext context; - private final SplitsAssignmentState assignmentState; + private final SplitAssigner splitAssigner; public PulsarSourceEnumerator( PulsarSubscriber subscriber, @@ -79,15 +67,14 @@ public PulsarSourceEnumerator( RangeGenerator rangeGenerator, SourceConfiguration sourceConfiguration, SplitEnumeratorContext context, - SplitsAssignmentState assignmentState) { + SplitAssigner splitAssigner) { this.pulsarAdmin = createAdmin(sourceConfiguration); - this.pulsarClient = createClient(sourceConfiguration); this.subscriber = subscriber; this.startCursor = startCursor; this.rangeGenerator = rangeGenerator; this.sourceConfiguration = sourceConfiguration; this.context = context; - this.assignmentState = assignmentState; + this.splitAssigner = splitAssigner; } @Override @@ -99,7 +86,7 @@ public void start() { LOG.info( "Starting the PulsarSourceEnumerator for subscription {} " + "with partition discovery interval of {} ms.", - sourceConfiguration.getSubscriptionDesc(), + subscriptionDesc(), sourceConfiguration.getPartitionDiscoveryIntervalMs()); context.callAsync( this::getSubscribedTopicPartitions, @@ -110,7 +97,7 @@ public void start() { LOG.info( "Starting the PulsarSourceEnumerator for subscription {} " + "without periodic partition discovery.", - sourceConfiguration.getSubscriptionDesc()); + subscriptionDesc()); context.callAsync(this::getSubscribedTopicPartitions, this::checkPartitionChanges); } } @@ -123,9 +110,9 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname @Override public void addSplitsBack(List splits, int subtaskId) { // Put the split back to current pending splits. - assignmentState.putSplitsBackToPendingList(splits, subtaskId); + splitAssigner.addSplitsBack(splits, subtaskId); - // If the failed subtask has already restarted, we need to assign pending splits to it + // If the failed subtask has already restarted, we need to assign pending splits to it. if (context.registeredReaders().containsKey(subtaskId)) { assignPendingPartitionSplits(singletonList(subtaskId)); } @@ -136,13 +123,13 @@ public void addReader(int subtaskId) { LOG.debug( "Adding reader {} to PulsarSourceEnumerator for subscription {}.", subtaskId, - sourceConfiguration.getSubscriptionDesc()); + subscriptionDesc()); assignPendingPartitionSplits(singletonList(subtaskId)); } @Override public PulsarSourceEnumState snapshotState(long checkpointId) { - return assignmentState.snapshotState(); + return splitAssigner.snapshotState(); } @Override @@ -154,6 +141,15 @@ public void close() { // ----------------- private methods ------------------- + /** Convert the subscription into a readable string. */ + private String subscriptionDesc() { + return String.format( + "%s(%s,%s)", + sourceConfiguration.getSubscriptionName(), + sourceConfiguration.getSubscriptionType(), + sourceConfiguration.getSubscriptionMode()); + } + /** * List subscribed topic partitions on Pulsar cluster. * @@ -164,54 +160,7 @@ public void close() { */ private Set getSubscribedTopicPartitions() { int parallelism = context.currentParallelism(); - Set partitions = - subscriber.getSubscribedTopicPartitions(pulsarAdmin, rangeGenerator, parallelism); - - // Seek start position for given partitions. - seekStartPosition(partitions); - - return partitions; - } - - private void seekStartPosition(Set partitions) { - ConsumerBuilder consumerBuilder = consumerBuilder(); - Set seekedTopics = new HashSet<>(); - - for (TopicPartition partition : partitions) { - String topicName = partition.getFullTopicName(); - if (!assignmentState.containsTopic(topicName) && seekedTopics.add(topicName)) { - try (Consumer consumer = - sneakyClient(() -> consumerBuilder.clone().topic(topicName).subscribe())) { - startCursor.seekPosition( - partition.getTopic(), partition.getPartitionId(), consumer); - } catch (PulsarClientException e) { - if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { - throw new IllegalArgumentException(e); - } else { - // WARN_ON_MISMATCH would just print this warning message. - // No need to print the stacktrace. - LOG.warn( - "Failed to set initial consuming position for partition {}", - partition, - e); - } - } - } - } - } - - private ConsumerBuilder consumerBuilder() { - ConsumerBuilder builder = - createConsumerBuilder(pulsarClient, Schema.BYTES, sourceConfiguration); - if (sourceConfiguration.getSubscriptionType() == SubscriptionType.Key_Shared) { - Range range = TopicRange.createFullRange().toPulsarRange(); - KeySharedPolicySticky keySharedPolicy = KeySharedPolicy.stickyHashRange().ranges(range); - // Force this consume use sticky hash range in Key_Shared subscription. - // Pulsar won't remove old message dispatcher before 2.8.2 release. - builder.keySharedPolicy(keySharedPolicy); - } - - return builder; + return subscriber.getSubscribedTopicPartitions(pulsarAdmin, rangeGenerator, parallelism); } /** @@ -230,13 +179,55 @@ private void checkPartitionChanges(Set fetchedPartitions, Throwa } // Append the partitions into current assignment state. - assignmentState.appendTopicPartitions(fetchedPartitions); - List registeredReaders = new ArrayList<>(context.registeredReaders().keySet()); + List newPartitions = + splitAssigner.registerTopicPartitions(fetchedPartitions); + createSubscription(newPartitions); // Assign the new readers. + List registeredReaders = new ArrayList<>(context.registeredReaders().keySet()); assignPendingPartitionSplits(registeredReaders); } + /** Create subscription on topic partition if it doesn't exist. */ + private void createSubscription(List newPartitions) { + for (TopicPartition partition : newPartitions) { + String topicName = partition.getFullTopicName(); + String subscriptionName = sourceConfiguration.getSubscriptionName(); + + List subscriptions = + sneakyAdmin(() -> pulsarAdmin.topics().getSubscriptions(topicName)); + if (!subscriptions.contains(subscriptionName)) { + CursorPosition position = + startCursor.position(partition.getTopic(), partition.getPartitionId()); + MessageId initialPosition = queryInitialPosition(topicName, position); + + sneakyAdmin( + () -> + pulsarAdmin + .topics() + .createSubscription( + topicName, subscriptionName, initialPosition)); + } + } + } + + /** Query the available message id from Pulsar. */ + private MessageId queryInitialPosition(String topicName, CursorPosition position) { + CursorPosition.Type type = position.getType(); + if (type == CursorPosition.Type.TIMESTAMP) { + return sneakyAdmin( + () -> + pulsarAdmin + .topics() + .getMessageIdByTimestamp(topicName, position.getTimestamp())); + } else if (type == CursorPosition.Type.MESSAGE_ID) { + return position.getMessageId(); + } else { + throw new UnsupportedOperationException("We don't support this seek type " + type); + } + } + + /** Query the unassigned splits and assign them to the available readers. */ private void assignPendingPartitionSplits(List pendingReaders) { // Validate the reader. pendingReaders.forEach( @@ -248,17 +239,19 @@ private void assignPendingPartitionSplits(List pendingReaders) { }); // Assign splits to downstream readers. - assignmentState.assignSplits(pendingReaders).ifPresent(context::assignSplits); + splitAssigner.createAssignment(pendingReaders).ifPresent(context::assignSplits); // If periodically partition discovery is disabled and the initializing discovery has done, - // signal NoMoreSplitsEvent to pending readers - if (assignmentState.noMoreNewPartitionSplits()) { - LOG.debug( - "No more PulsarPartitionSplits to assign." - + " Sending NoMoreSplitsEvent to reader {} in subscription {}.", - pendingReaders, - sourceConfiguration.getSubscriptionDesc()); - pendingReaders.forEach(this.context::signalNoMoreSplits); + // signal NoMoreSplitsEvent to pending readers. + for (Integer reader : pendingReaders) { + if (splitAssigner.noMoreSplits(reader)) { + LOG.debug( + "No more PulsarPartitionSplits to assign." + + " Sending NoMoreSplitsEvent to reader {} in subscription {}.", + reader, + subscriptionDesc()); + context.signalNoMoreSplits(reader); + } } } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java deleted file mode 100644 index cbc4826583a53..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentState.java +++ /dev/null @@ -1,239 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.source.enumerator; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.api.connector.source.SplitsAssignment; -import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; -import org.apache.flink.util.InstantiationUtil; - -import org.apache.pulsar.client.api.SubscriptionType; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; - -/** The state class for recording the split assignment. */ -@Internal -public class SplitsAssignmentState { - - private final StopCursor stopCursor; - private final SourceConfiguration sourceConfiguration; - - // The dynamic states for checkpoint. - private final Set appendedPartitions; - // This pending splits is used for Key_Shared, Failover, Exclusive subscription. - private final Set pendingPartitionSplits; - // These two fields are used for Shared subscription. - private final Map> sharedPendingPartitionSplits; - private final Map> readerAssignedSplits; - private boolean initialized; - - public SplitsAssignmentState(StopCursor stopCursor, SourceConfiguration sourceConfiguration) { - this.stopCursor = stopCursor; - this.sourceConfiguration = sourceConfiguration; - this.appendedPartitions = new HashSet<>(); - this.pendingPartitionSplits = new HashSet<>(); - this.sharedPendingPartitionSplits = new HashMap<>(); - this.readerAssignedSplits = new HashMap<>(); - this.initialized = false; - } - - public SplitsAssignmentState( - StopCursor stopCursor, - SourceConfiguration sourceConfiguration, - PulsarSourceEnumState sourceEnumState) { - this.stopCursor = stopCursor; - this.sourceConfiguration = sourceConfiguration; - this.appendedPartitions = sourceEnumState.getAppendedPartitions(); - this.pendingPartitionSplits = sourceEnumState.getPendingPartitionSplits(); - this.sharedPendingPartitionSplits = sourceEnumState.getSharedPendingPartitionSplits(); - this.readerAssignedSplits = sourceEnumState.getReaderAssignedSplits(); - this.initialized = sourceEnumState.isInitialized(); - } - - public PulsarSourceEnumState snapshotState() { - return new PulsarSourceEnumState( - appendedPartitions, - pendingPartitionSplits, - sharedPendingPartitionSplits, - readerAssignedSplits, - initialized); - } - - /** - * Append the new fetched partitions to current state. We would generate pending source split - * for downstream pulsar readers. Since the {@link SplitEnumeratorContext} don't support put the - * split back to enumerator, we don't support partition deletion. - * - * @param fetchedPartitions The partitions from the {@link PulsarSubscriber}. - */ - public void appendTopicPartitions(Set fetchedPartitions) { - for (TopicPartition partition : fetchedPartitions) { - // If this partition is a new partition. - if (!appendedPartitions.contains(partition)) { - if (!sharePartition()) { - // Create a split and add it to pending list. - pendingPartitionSplits.add(createSplit(partition)); - } - - // Shared subscription don't create splits, we just register partitions. - appendedPartitions.add(partition); - } - } - - // Update this initialize flag. - if (!initialized) { - this.initialized = true; - } - } - - public boolean containsTopic(String topicName) { - return appendedPartitions.stream() - .anyMatch(partition -> Objects.equals(partition.getFullTopicName(), topicName)); - } - - /** Put these splits back to pending list. */ - public void putSplitsBackToPendingList(List splits, int readerId) { - if (!sharePartition()) { - // Put these splits back to normal pending list. - pendingPartitionSplits.addAll(splits); - } else { - // Put the splits back to shared pending list. - Set pending = - sharedPendingPartitionSplits.computeIfAbsent(readerId, id -> new HashSet<>()); - pending.addAll(splits); - } - } - - public Optional> assignSplits( - List pendingReaders) { - // Avoid empty readers assign. - if (pendingReaders.isEmpty()) { - return Optional.empty(); - } - - Map> assignMap; - - // We extract the assign logic into two method for better readability. - if (!sharePartition()) { - assignMap = assignNormalSplits(pendingReaders); - } else { - assignMap = assignSharedSplits(pendingReaders); - } - - if (assignMap.isEmpty()) { - return Optional.empty(); - } else { - return Optional.of(new SplitsAssignment<>(assignMap)); - } - } - - /** - * @return It would return true only if periodically partition discovery is disabled, the - * initializing partition discovery has finished AND there is no pending splits for - * assignment. - */ - public boolean noMoreNewPartitionSplits() { - return !sourceConfiguration.isEnablePartitionDiscovery() - && initialized - && pendingPartitionSplits.isEmpty(); - } - - // ----------------- private methods ------------------- - - /** The splits don't shared for all the readers. */ - private Map> assignNormalSplits( - List pendingReaders) { - Map> assignMap = new HashMap<>(); - - // Drain a list of splits. - List pendingSplits = drainPendingPartitionsSplits(); - for (int i = 0; i < pendingSplits.size(); i++) { - PulsarPartitionSplit split = pendingSplits.get(i); - int readerId = pendingReaders.get(i % pendingReaders.size()); - assignMap.computeIfAbsent(readerId, id -> new ArrayList<>()).add(split); - } - - return assignMap; - } - - /** Every split would be shared among available readers. */ - private Map> assignSharedSplits( - List pendingReaders) { - Map> assignMap = new HashMap<>(); - - // Drain the splits from share pending list. - for (Integer reader : pendingReaders) { - Set pendingSplits = sharedPendingPartitionSplits.remove(reader); - if (pendingSplits == null) { - pendingSplits = new HashSet<>(); - } - - Set assignedSplits = - readerAssignedSplits.computeIfAbsent(reader, r -> new HashSet<>()); - - for (TopicPartition partition : appendedPartitions) { - String partitionName = partition.toString(); - if (!assignedSplits.contains(partitionName)) { - pendingSplits.add(createSplit(partition)); - assignedSplits.add(partitionName); - } - } - - if (!pendingSplits.isEmpty()) { - assignMap.put(reader, new ArrayList<>(pendingSplits)); - } - } - - return assignMap; - } - - private PulsarPartitionSplit createSplit(TopicPartition partition) { - try { - StopCursor stop = InstantiationUtil.clone(stopCursor); - return new PulsarPartitionSplit(partition, stop); - } catch (IOException | ClassNotFoundException e) { - throw new IllegalStateException(e); - } - } - - private List drainPendingPartitionsSplits() { - List splits = new ArrayList<>(pendingPartitionSplits); - pendingPartitionSplits.clear(); - - return splits; - } - - /** {@link SubscriptionType#Shared} mode should share a same split for all the readers. */ - private boolean sharePartition() { - return sourceConfiguration.getSubscriptionType() == SubscriptionType.Shared; - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NormalSplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NormalSplitAssigner.java new file mode 100644 index 0000000000000..a03e3e64e6214 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NormalSplitAssigner.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * This assigner is used for {@link SubscriptionType#Failover} and {@link + * SubscriptionType#Exclusive} subscriptions. + */ +public class NormalSplitAssigner implements SplitAssigner { + private static final long serialVersionUID = 8412586087991597092L; + + private final StopCursor stopCursor; + private final SourceConfiguration sourceConfiguration; + + // These states would be saved into checkpoint. + + private final Set appendedPartitions; + private final Set pendingPartitionSplits; + private boolean initialized; + + public NormalSplitAssigner(StopCursor stopCursor, SourceConfiguration sourceConfiguration) { + this.stopCursor = stopCursor; + this.sourceConfiguration = sourceConfiguration; + this.appendedPartitions = new HashSet<>(); + this.pendingPartitionSplits = new HashSet<>(); + this.initialized = false; + } + + public NormalSplitAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + this.stopCursor = stopCursor; + this.sourceConfiguration = sourceConfiguration; + this.appendedPartitions = sourceEnumState.getAppendedPartitions(); + this.pendingPartitionSplits = sourceEnumState.getPendingPartitionSplits(); + this.initialized = sourceEnumState.isInitialized(); + } + + @Override + public List registerTopicPartitions(Set fetchedPartitions) { + List newPartitions = new ArrayList<>(fetchedPartitions.size()); + + for (TopicPartition partition : fetchedPartitions) { + if (!appendedPartitions.contains(partition)) { + pendingPartitionSplits.add(new PulsarPartitionSplit(partition, stopCursor)); + appendedPartitions.add(partition); + newPartitions.add(partition); + } + } + + if (!initialized) { + initialized = true; + } + + return newPartitions; + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + pendingPartitionSplits.addAll(splits); + } + + @Override + public Optional> createAssignment( + List readers) { + if (pendingPartitionSplits.isEmpty() || readers.isEmpty()) { + return Optional.empty(); + } + + Map> assignMap = new HashMap<>(); + List partitionSplits = new ArrayList<>(pendingPartitionSplits); + pendingPartitionSplits.clear(); + int readerCount = readers.size(); + + for (int i = 0; i < partitionSplits.size(); i++) { + int index = i % readerCount; + Integer readerId = readers.get(index); + PulsarPartitionSplit split = partitionSplits.get(i); + assignMap.computeIfAbsent(readerId, id -> new ArrayList<>()).add(split); + } + + return Optional.of(new SplitsAssignment<>(assignMap)); + } + + @Override + public boolean noMoreSplits(Integer reader) { + return !sourceConfiguration.isEnablePartitionDiscovery() + && initialized + && pendingPartitionSplits.isEmpty(); + } + + @Override + public PulsarSourceEnumState snapshotState() { + return new PulsarSourceEnumState( + appendedPartitions, + pendingPartitionSplits, + new HashMap<>(), + new HashMap<>(), + initialized); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java new file mode 100644 index 0000000000000..70d1287f71197 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.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.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** This assigner is used for {@link SubscriptionType#Shared} subscriptions. */ +public class SharedSplitAssigner implements SplitAssigner { + private static final long serialVersionUID = 8468503133499402491L; + + private final StopCursor stopCursor; + private final SourceConfiguration sourceConfiguration; + + // These fields would be saved into checkpoint. + + private final Set appendedPartitions; + private final Map> sharedPendingPartitionSplits; + private final Map> readerAssignedSplits; + private boolean initialized; + + // These fields are used as the dynamic initializing record. + + public SharedSplitAssigner(StopCursor stopCursor, SourceConfiguration sourceConfiguration) { + this.stopCursor = stopCursor; + this.sourceConfiguration = sourceConfiguration; + this.appendedPartitions = new HashSet<>(); + this.sharedPendingPartitionSplits = new HashMap<>(); + this.readerAssignedSplits = new HashMap<>(); + this.initialized = false; + } + + public SharedSplitAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + this.stopCursor = stopCursor; + this.sourceConfiguration = sourceConfiguration; + this.appendedPartitions = sourceEnumState.getAppendedPartitions(); + this.sharedPendingPartitionSplits = sourceEnumState.getSharedPendingPartitionSplits(); + this.readerAssignedSplits = sourceEnumState.getReaderAssignedSplits(); + this.initialized = sourceEnumState.isInitialized(); + } + + @Override + public List registerTopicPartitions(Set fetchedPartitions) { + List newPartitions = new ArrayList<>(fetchedPartitions.size()); + + for (TopicPartition fetchedPartition : fetchedPartitions) { + if (!appendedPartitions.contains(fetchedPartition)) { + newPartitions.add(fetchedPartition); + appendedPartitions.add(fetchedPartition); + } + } + + if (!initialized) { + initialized = true; + } + + return newPartitions; + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + Set pending = + sharedPendingPartitionSplits.computeIfAbsent(subtaskId, id -> new HashSet<>()); + pending.addAll(splits); + } + + @Override + public Optional> createAssignment( + List readers) { + if (readers.isEmpty()) { + return Optional.empty(); + } + + Map> assignMap = new HashMap<>(); + for (Integer reader : readers) { + Set pendingSplits = sharedPendingPartitionSplits.remove(reader); + if (pendingSplits == null) { + pendingSplits = new HashSet<>(); + } + + Set assignedSplits = + readerAssignedSplits.computeIfAbsent(reader, r -> new HashSet<>()); + + for (TopicPartition partition : appendedPartitions) { + String partitionName = partition.toString(); + if (!assignedSplits.contains(partitionName)) { + pendingSplits.add(new PulsarPartitionSplit(partition, stopCursor)); + assignedSplits.add(partitionName); + } + } + + if (!pendingSplits.isEmpty()) { + assignMap.put(reader, new ArrayList<>(pendingSplits)); + } + } + + if (assignMap.isEmpty()) { + return Optional.empty(); + } else { + return Optional.of(new SplitsAssignment<>(assignMap)); + } + } + + @Override + public boolean noMoreSplits(Integer reader) { + Set pendingSplits = sharedPendingPartitionSplits.get(reader); + Set assignedSplits = readerAssignedSplits.get(reader); + + return !sourceConfiguration.isEnablePartitionDiscovery() + && initialized + && (pendingSplits == null || pendingSplits.isEmpty()) + && (assignedSplits != null && assignedSplits.size() == appendedPartitions.size()); + } + + @Override + public PulsarSourceEnumState snapshotState() { + return new PulsarSourceEnumState( + appendedPartitions, + new HashSet<>(), + sharedPendingPartitionSplits, + readerAssignedSplits, + initialized); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java new file mode 100644 index 0000000000000..bc03f5103fd88 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssigner.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import java.io.Serializable; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +/** + * The split assigner for different subscription. We would spread all the splits to different + * readers and store all the state into checkpoint. + */ +@Internal +public interface SplitAssigner extends Serializable { + + /** + * Add the current available partitions into assigner. + * + * @param fetchedPartitions The available partitions queried from Pulsar broker. + * @return New topic partitions compare to previous registered partitions. + */ + List registerTopicPartitions(Set fetchedPartitions); + + /** + * Add a split back to the split assigner if the reader fails. We would try to reassign the + * split or add it to the pending list. + */ + void addSplitsBack(List splits, int subtaskId); + + /** Create a split assignment from the current readers. */ + Optional> createAssignment(List readers); + + /** + * It would return true only if periodically partition discovery is disabled, the initializing + * partition discovery has finished AND there is no pending splits for assignment. + */ + boolean noMoreSplits(Integer reader); + + /** Snapshot the current assign state into checkpoint. */ + PulsarSourceEnumState snapshotState(); +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java new file mode 100644 index 0000000000000..0dc8d23525d90 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; + +import org.apache.pulsar.client.api.SubscriptionType; + +import javax.annotation.Nullable; + +import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; +import static org.apache.pulsar.client.api.SubscriptionType.Failover; +import static org.apache.pulsar.client.api.SubscriptionType.Key_Shared; +import static org.apache.pulsar.client.api.SubscriptionType.Shared; + +/** The factory for creating split assigner. */ +public final class SplitAssignerFactory { + + private SplitAssignerFactory() { + // No public constructor. + } + + /** Create blank assigner. */ + public static SplitAssigner create( + StopCursor stopCursor, SourceConfiguration sourceConfiguration) { + return create(stopCursor, sourceConfiguration, null); + } + + /** Create assigner from checkpoint state. */ + public static SplitAssigner create( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + @Nullable PulsarSourceEnumState sourceEnumState) { + SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType(); + if (subscriptionType == Exclusive + || subscriptionType == Failover + || subscriptionType == Key_Shared) { + if (sourceEnumState == null) { + return new NormalSplitAssigner(stopCursor, sourceConfiguration); + } else { + return new NormalSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } + } else if (subscriptionType == Shared) { + if (sourceEnumState == null) { + return new SharedSplitAssigner(stopCursor, sourceConfiguration); + } else { + return new SharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } + } else { + throw new IllegalArgumentException( + "We don't support this subscription type: " + subscriptionType); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java index 26dfa537fe8d3..da0c4ec739b1f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java @@ -57,7 +57,8 @@ static StartCursor latest() { /** * Find the available message id and start consuming from it. The given message is included in - * the consuming result by default. + * the consuming result by default if you provide a specified message id instead of {@link + * MessageId#earliest} or {@link MessageId#latest}. */ static StartCursor fromMessageId(MessageId messageId) { return fromMessageId(messageId, true); @@ -65,7 +66,8 @@ static StartCursor fromMessageId(MessageId messageId) { /** * @param messageId Find the available message id and start consuming from it. - * @param inclusive {@code true} would include the given message id. + * @param inclusive {@code true} would include the given message id if it's not the {@link + * MessageId#earliest} or {@link MessageId#latest}. */ static StartCursor fromMessageId(MessageId messageId, boolean inclusive) { return new MessageIdStartCursor(messageId, inclusive); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java index 5ac3d90cbaff8..0b2390658d01d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -47,7 +47,9 @@ public class MessageIdStartCursor implements StartCursor { * code for understanding pulsar internal logic. * * @param messageId The message id for start position. - * @param inclusive Should we include the start message id in consuming result. + * @param inclusive Should we include the start message id in consuming result. This is works + * only if we provide a specified message id instead of {@link MessageId#earliest} or {@link + * MessageId#latest}. */ public MessageIdStartCursor(MessageId messageId, boolean inclusive) { MessageIdImpl id = MessageIdImpl.convertToMessageIdImpl(messageId); @@ -67,7 +69,7 @@ public MessageIdStartCursor(MessageId messageId, boolean inclusive) { * href="https://github.com/apache/pulsar/blob/7c8dc3201baad7d02d886dbc26db5c03abce77d6/managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/PositionImpl.java#L85">this * code to get the next message id. */ - private MessageId getNext(MessageIdImpl messageId) { + public static MessageId getNext(MessageIdImpl messageId) { if (messageId.getEntryId() < 0) { return DefaultImplementation.getDefaultImplementation() .newMessageId(messageId.getLedgerId(), 0, messageId.getPartitionIndex()); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java index 08ba1faa44214..b8a55bf8a34b8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/PulsarSubscriber.java @@ -48,7 +48,7 @@ public interface PulsarSubscriber extends Serializable { /** * Get a set of subscribed {@link TopicPartition}s. The method could throw {@link - * IllegalStateException}, a extra try catch is required. + * IllegalStateException}, an extra try catch is required. * * @param pulsarAdmin The admin interface used to retrieve subscribed topic partitions. * @param rangeGenerator The range for different partitions. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java index a6c713cf6577c..b045d0843c4a7 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; @@ -32,6 +32,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.impl.MessageIdImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,12 +81,19 @@ protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { // Reset the start position for ordered pulsar consumer. if (latestConsumedId != null) { - StartCursor startCursor = StartCursor.fromMessageId(latestConsumedId, false); - TopicPartition partition = split.getPartition(); - + LOG.debug("Start seeking from the checkpoint {}", latestConsumedId); try { - startCursor.seekPosition( - partition.getTopic(), partition.getPartitionId(), consumer); + MessageId initialPosition; + if (latestConsumedId == MessageId.latest + || latestConsumedId == MessageId.earliest) { + initialPosition = latestConsumedId; + } else { + MessageIdImpl messageId = + MessageIdImpl.convertToMessageIdImpl(latestConsumedId); + initialPosition = MessageIdStartCursor.getNext(messageId); + } + + consumer.seek(initialPosition); } catch (PulsarClientException e) { if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { throw new IllegalArgumentException(e); @@ -95,7 +103,7 @@ protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { LOG.warn( "Failed to reset cursor to {} on partition {}", latestConsumedId, - partition, + split.getPartition(), e); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java index 0ec693a2b26d2..90e29ca47120f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java @@ -60,8 +60,8 @@ public PulsarPartitionSplit(TopicPartition partition, StopCursor stopCursor) { public PulsarPartitionSplit( TopicPartition partition, StopCursor stopCursor, - MessageId latestConsumedId, - TxnID uncommittedTransactionId) { + @Nullable MessageId latestConsumedId, + @Nullable TxnID uncommittedTransactionId) { this.partition = checkNotNull(partition); this.stopCursor = checkNotNull(stopCursor); this.latestConsumedId = latestConsumedId; diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java index 1dcbe84ba61ee..aebb76119dfdc 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumeratorTest.java @@ -22,6 +22,8 @@ import org.apache.flink.api.connector.source.mocks.MockSplitEnumeratorContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssigner; +import org.apache.flink.connector.pulsar.source.enumerator.assigner.SplitAssignerFactory; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; @@ -51,6 +53,7 @@ import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.latest; import static org.apache.flink.connector.pulsar.source.enumerator.subscriber.PulsarSubscriber.getTopicPatternSubscriber; @@ -368,6 +371,7 @@ private PulsarSourceEnumerator createEnumerator( Configuration configuration = operator().config(); configuration.set(PULSAR_SUBSCRIPTION_TYPE, subscriptionType); + configuration.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10)); if (enablePeriodicPartitionDiscovery) { configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 60L); } else { @@ -375,15 +379,15 @@ private PulsarSourceEnumerator createEnumerator( } SourceConfiguration sourceConfiguration = new SourceConfiguration(configuration); - SplitsAssignmentState assignmentState = - new SplitsAssignmentState(latest(), sourceConfiguration, sourceEnumState); + SplitAssigner assigner = + SplitAssignerFactory.create(latest(), sourceConfiguration, sourceEnumState); return new PulsarSourceEnumerator( subscriber, StartCursor.earliest(), new FullRangeGenerator(), sourceConfiguration, enumContext, - assignmentState); + assigner); } private void registerReader( diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java deleted file mode 100644 index ac811c3dddbfe..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/SplitsAssignmentStateTest.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.source.enumerator; - -import org.apache.flink.api.connector.source.SplitsAssignment; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; -import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; - -import org.apache.flink.shaded.guava30.com.google.common.collect.Lists; -import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; - -import org.apache.pulsar.client.api.SubscriptionType; -import org.junit.jupiter.api.Test; - -import java.util.List; -import java.util.Optional; -import java.util.Set; - -import static java.util.Collections.singletonList; -import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; -import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.MAX_RANGE; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.InstanceOfAssertFactories.map; - -/** Unit tests for {@link SplitsAssignmentState}. */ -class SplitsAssignmentStateTest { - - private final Set partitions = - Sets.newHashSet( - new TopicPartition("some-topic", 1, new TopicRange(1, 30)), - new TopicPartition("some-topic", 2, new TopicRange(31, 60)), - new TopicPartition("some-topic", 3, new TopicRange(61, MAX_RANGE)), - new TopicPartition(randomAlphabetic(10), -1, createFullRange())); - - @Test - void assignSplitsForSharedSubscription() { - SplitsAssignmentState state1 = - new SplitsAssignmentState( - StopCursor.defaultStopCursor(), createConfig(SubscriptionType.Shared)); - state1.appendTopicPartitions(partitions); - Optional> assignment1 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - - assertThat(assignment1) - .isPresent() - .get() - .extracting(SplitsAssignment::assignment) - .asInstanceOf(map(Integer.class, List.class)) - .hasSize(5) - .allSatisfy((idx, list) -> assertThat(list).hasSize(4)); - - Optional> assignment2 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - assertThat(assignment2).isNotPresent(); - - // Reassign reader 3. - state1.putSplitsBackToPendingList(assignment1.get().assignment().get(3), 3); - Optional> assignment3 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 4)); - assertThat(assignment3).isNotPresent(); - - Optional> assignment4 = - state1.assignSplits(singletonList(3)); - assertThat(assignment4) - .isPresent() - .get() - .extracting(SplitsAssignment::assignment) - .asInstanceOf(map(Integer.class, List.class)) - .hasSize(1); - } - - @Test - void assignSplitsForExclusiveSubscription() { - SplitsAssignmentState state1 = - new SplitsAssignmentState( - StopCursor.defaultStopCursor(), createConfig(SubscriptionType.Exclusive)); - state1.appendTopicPartitions(partitions); - Optional> assignment1 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - - assertThat(assignment1).isPresent(); - assertThat(assignment1.get().assignment()) - .hasSize(4) - .allSatisfy((idx, list) -> assertThat(list).hasSize(1)); - - Optional> assignment2 = - state1.assignSplits(Lists.newArrayList(0, 1, 2, 3, 4)); - assertThat(assignment2).isNotPresent(); - } - - private SourceConfiguration createConfig(SubscriptionType type) { - Configuration configuration = new Configuration(); - configuration.set(PULSAR_SUBSCRIPTION_TYPE, type); - - return new SourceConfiguration(configuration); - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java index e1e31b6de41fd..f1715e1f42de9 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java @@ -24,7 +24,6 @@ import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; @@ -139,8 +138,7 @@ private void seekStartPositionAndHandleSplit( try (Consumer consumer = (Consumer) reader.createPulsarConsumer(partition)) { // inclusive messageId - StartCursor startCursor = StartCursor.fromMessageId(startPosition); - startCursor.seekPosition(partition.getTopic(), partition.getPartitionId(), consumer); + consumer.seek(startPosition); } catch (PulsarClientException e) { sneakyThrow(e); } From c803bcae1a309fb3e84e3b8c900adcfd64cd95c0 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 16 Jun 2022 02:42:21 +0800 Subject: [PATCH 215/258] [FLINK-28082][Connector/Pulsar] Add end-to-end encryption support for Pulsar connector. --- .../pulsar_producer_configuration.html | 12 +++++ .../connector/pulsar/sink/PulsarSink.java | 11 ++++- .../pulsar/sink/PulsarSinkBuilder.java | 48 ++++++++++++++++++- .../pulsar/sink/PulsarSinkOptions.java | 18 +++++++ .../sink/config/PulsarSinkConfigUtils.java | 13 ++++- .../pulsar/sink/writer/PulsarWriter.java | 6 ++- .../writer/topic/TopicProducerRegister.java | 14 +++++- .../connector/pulsar/source/PulsarSource.java | 12 ++++- .../pulsar/source/PulsarSourceBuilder.java | 19 +++++++- .../reader/PulsarSourceReaderFactory.java | 10 +++- .../PulsarOrderedPartitionSplitReader.java | 13 ++++- .../split/PulsarPartitionSplitReaderBase.java | 11 ++++- .../PulsarUnorderedPartitionSplitReader.java | 9 +++- .../pulsar/sink/writer/PulsarWriterTest.java | 3 +- .../topic/TopicProducerRegisterTest.java | 4 +- .../enumerator/cursor/StopCursorTest.java | 3 +- .../source/PulsarSourceReaderTestBase.java | 2 +- .../PulsarPartitionSplitReaderTestBase.java | 4 +- 18 files changed, 192 insertions(+), 20 deletions(-) diff --git a/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html b/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html index 044c4b228d47e..22d0de1f5fd3b 100644 --- a/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html @@ -50,6 +50,12 @@

    Enum

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

    Possible values:
    • "NONE"
    • "LZ4"
    • "ZLIB"
    • "ZSTD"
    • "SNAPPY"
    + +
    pulsar.producer.encryptionKeys
    + (none) + List<String> + Add public encryption key, used by producer to encrypt the data key. +
    pulsar.producer.initialSequenceId
    (none) @@ -68,6 +74,12 @@ Integer The maximum number of pending messages across partitions.
    Use the setting to lower the max pending messages for each partition (setMaxPendingMessages) if the total number exceeds the configured value. + +
    pulsar.producer.producerCryptoFailureAction
    + FAIL +

    Enum

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

    Possible values:
    • "FAIL"
    • "SEND"
    +
    pulsar.producer.producerName
    (none) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java index 4c6c4a9b7c7d3..77c1077fdff6a 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java @@ -37,6 +37,10 @@ import org.apache.flink.connector.pulsar.sink.writer.topic.TopicMetadataListener; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.pulsar.client.api.CryptoKeyReader; + +import javax.annotation.Nullable; + import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -86,17 +90,21 @@ public class PulsarSink implements TwoPhaseCommittingSink messageDelayer; private final TopicRouter topicRouter; + @Nullable private final CryptoKeyReader cryptoKeyReader; + PulsarSink( SinkConfiguration sinkConfiguration, PulsarSerializationSchema serializationSchema, TopicMetadataListener metadataListener, TopicRoutingMode topicRoutingMode, TopicRouter topicRouter, - MessageDelayer messageDelayer) { + MessageDelayer messageDelayer, + @Nullable CryptoKeyReader cryptoKeyReader) { this.sinkConfiguration = checkNotNull(sinkConfiguration); this.serializationSchema = checkNotNull(serializationSchema); this.metadataListener = checkNotNull(metadataListener); this.messageDelayer = checkNotNull(messageDelayer); + this.cryptoKeyReader = cryptoKeyReader; checkNotNull(topicRoutingMode); // Create topic router supplier. @@ -128,6 +136,7 @@ public PrecommittingSinkWriter createWriter(InitContext i metadataListener, topicRouter, messageDelayer, + cryptoKeyReader, initContext); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java index 1668e3d126a16..94865e5541973 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java @@ -32,17 +32,23 @@ import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; import org.apache.flink.connector.pulsar.sink.writer.topic.TopicMetadataListener; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Schema; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + +import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Properties; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ENABLE_TRANSACTION; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_ENCRYPTION_KEYS; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_PRODUCER_NAME; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SEND_TIMEOUT_MS; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_DELIVERY_GUARANTEE; @@ -103,6 +109,8 @@ public class PulsarSinkBuilder { private TopicRoutingMode topicRoutingMode; private TopicRouter topicRouter; private MessageDelayer messageDelayer; + @Nullable private CryptoKeyReader cryptoKeyReader; + private List encryptionKeys = new ArrayList<>(); // private builder constructor. PulsarSinkBuilder() { @@ -243,6 +251,36 @@ public PulsarSinkBuilder delaySendingMessage(MessageDelayer messageDelay return this; } + /** + * Sets a {@link CryptoKeyReader}. Configure the key reader to be used to encrypt the message + * payloads. + * + * @param cryptoKeyReader CryptoKeyReader object. + * @return this PulsarSinkBuilder. + */ + public PulsarSinkBuilder setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { + this.cryptoKeyReader = checkNotNull(cryptoKeyReader); + return this; + } + + /** + * Add public encryption key, used by producer to encrypt the data key. + * + *

    At the time of producer creation, Pulsar client checks if there are keys added to + * encryptionKeys. If keys are found, a callback {@link CryptoKeyReader#getPrivateKey(String, + * Map)} and {@link CryptoKeyReader#getPublicKey(String, Map)} is invoked against each key to + * load the values of the key. Application should implement this callback to return the key in + * pkcs8 format. If compression is enabled, message is encrypted after compression. If batch + * messaging is enabled, the batched message is encrypted. + * + * @param keys Encryption keys. + * @return this PulsarSinkBuilder. + */ + public PulsarSinkBuilder setEncryptionKeys(String... keys) { + this.encryptionKeys.addAll(Arrays.asList(keys)); + return this; + } + /** * Set an arbitrary property for the PulsarSink and Pulsar Producer. The valid keys can be found * in {@link PulsarSinkOptions} and {@link PulsarOptions}. @@ -348,6 +386,13 @@ public PulsarSink build() { this.messageDelayer = MessageDelayer.never(); } + // Add the encryption keys if user provides one. + if (cryptoKeyReader != null) { + checkArgument( + !encryptionKeys.isEmpty(), "You should provide at least on encryption key."); + configBuilder.set(PULSAR_ENCRYPTION_KEYS, encryptionKeys); + } + // This is an unmodifiable configuration for Pulsar. // We don't use Pulsar's built-in configure classes for compatible requirement. SinkConfiguration sinkConfiguration = @@ -359,7 +404,8 @@ public PulsarSink build() { metadataListener, topicRoutingMode, topicRouter, - messageDelayer); + messageDelayer, + cryptoKeyReader); } // ------------- private helpers -------------- diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java index 3a7c5bc1a0e95..c9fe82d42aab0 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java @@ -29,8 +29,10 @@ import org.apache.flink.connector.pulsar.sink.writer.router.MessageKeyHash; import org.apache.pulsar.client.api.CompressionType; +import org.apache.pulsar.client.api.ProducerCryptoFailureAction; import java.time.Duration; +import java.util.List; import java.util.Map; import static java.util.Collections.emptyMap; @@ -266,4 +268,20 @@ private PulsarSinkOptions() { .text( " When getting a topic stats, associate this metadata with the consumer stats for easier identification.") .build()); + + public static final ConfigOption + PULSAR_PRODUCER_CRYPTO_FAILURE_ACTION = + ConfigOptions.key(PRODUCER_CONFIG_PREFIX + "producerCryptoFailureAction") + .enumType(ProducerCryptoFailureAction.class) + .defaultValue(ProducerCryptoFailureAction.FAIL) + .withDescription( + "The action the producer will take in case of encryption failures."); + + public static final ConfigOption> PULSAR_ENCRYPTION_KEYS = + ConfigOptions.key(PRODUCER_CONFIG_PREFIX + "encryptionKeys") + .stringType() + .asList() + .noDefaultValue() + .withDescription( + "Add public encryption key, used by producer to encrypt the data key."); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java index 13821fe891852..cb9ba2efd7b67 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java @@ -41,9 +41,11 @@ import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_PARTITION_SWITCH_FREQUENCY_BY_PUBLISH_DELAY; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_CHUNKING_ENABLED; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_COMPRESSION_TYPE; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_ENCRYPTION_KEYS; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_INITIAL_SEQUENCE_ID; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_PENDING_MESSAGES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_PRODUCER_CRYPTO_FAILURE_ACTION; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_PRODUCER_NAME; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_PRODUCER_PROPERTIES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SEND_TIMEOUT_MS; @@ -91,8 +93,17 @@ public static ProducerBuilder createProducerBuilder( configuration.useOption(PULSAR_CHUNKING_ENABLED, builder::enableChunking); configuration.useOption(PULSAR_COMPRESSION_TYPE, builder::compressionType); configuration.useOption(PULSAR_INITIAL_SEQUENCE_ID, builder::initialSequenceId); + configuration.useOption( + PULSAR_PRODUCER_CRYPTO_FAILURE_ACTION, builder::cryptoFailureAction); + + // Create the encryption keys. + if (configuration.contains(PULSAR_ENCRYPTION_KEYS)) { + for (String key : configuration.get(PULSAR_ENCRYPTION_KEYS)) { + builder.addEncryptionKey(key); + } + } - // Set producer properties + // Set producer properties. Map properties = configuration.getProperties(PULSAR_PRODUCER_PROPERTIES); if (!properties.isEmpty()) { builder.properties(properties); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java index 30c454455ca9f..360f742bfd0b1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java @@ -39,11 +39,14 @@ import org.apache.flink.shaded.guava30.com.google.common.base.Strings; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.io.IOException; import java.util.Base64; import java.util.Collection; @@ -96,6 +99,7 @@ public PulsarWriter( TopicMetadataListener metadataListener, TopicRouter topicRouter, MessageDelayer messageDelayer, + @Nullable CryptoKeyReader cryptoKeyReader, InitContext initContext) { this.sinkConfiguration = checkNotNull(sinkConfiguration); this.serializationSchema = checkNotNull(serializationSchema); @@ -126,7 +130,7 @@ public PulsarWriter( } // Create this producer register after opening serialization schema! - this.producerRegister = new TopicProducerRegister(sinkConfiguration); + this.producerRegister = new TopicProducerRegister(sinkConfiguration, cryptoKeyReader); } @Override diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegister.java index 9bb175319d2a1..e83c56c77773c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegister.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegister.java @@ -26,6 +26,7 @@ import org.apache.flink.shaded.guava30.com.google.common.io.Closer; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Producer; import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; @@ -37,6 +38,8 @@ import org.apache.pulsar.client.impl.PulsarClientImpl; import org.apache.pulsar.common.schema.SchemaInfo; +import javax.annotation.Nullable; + import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; @@ -60,12 +63,15 @@ public class TopicProducerRegister implements Closeable { private final PulsarClient pulsarClient; private final SinkConfiguration sinkConfiguration; + @Nullable private final CryptoKeyReader cryptoKeyReader; private final Map>> producerRegister; private final Map transactionRegister; - public TopicProducerRegister(SinkConfiguration sinkConfiguration) { + public TopicProducerRegister( + SinkConfiguration sinkConfiguration, @Nullable CryptoKeyReader cryptoKeyReader) { this.pulsarClient = createClient(sinkConfiguration); this.sinkConfiguration = sinkConfiguration; + this.cryptoKeyReader = cryptoKeyReader; this.producerRegister = new HashMap<>(); this.transactionRegister = new HashMap<>(); } @@ -148,6 +154,12 @@ private Producer getOrCreateProducer(String topic, Schema schema) { } else { ProducerBuilder builder = createProducerBuilder(pulsarClient, schema, sinkConfiguration); + + // Set the message crypto key reader. + if (cryptoKeyReader != null) { + builder.cryptoKeyReader(cryptoKeyReader); + } + // Set the required topic name. builder.topic(topic); Producer producer = sneakyClient(builder::create); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java index 6b2752ed2970b..48c30ff4e7d16 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSource.java @@ -45,6 +45,10 @@ import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitSerializer; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.pulsar.client.api.CryptoKeyReader; + +import javax.annotation.Nullable; + /** * The Source implementation of Pulsar. Please use a {@link PulsarSourceBuilder} to construct a * {@link PulsarSource}. The following example shows how to create a PulsarSource emitting records @@ -91,6 +95,8 @@ public final class PulsarSource /** The pulsar deserialization schema used for deserializing message. */ private final PulsarDeserializationSchema deserializationSchema; + @Nullable private final CryptoKeyReader cryptoKeyReader; + /** * The constructor for PulsarSource, it's package protected for forcing using {@link * PulsarSourceBuilder}. @@ -102,7 +108,8 @@ public PulsarSource( StartCursor startCursor, StopCursor stopCursor, Boundedness boundedness, - PulsarDeserializationSchema deserializationSchema) { + PulsarDeserializationSchema deserializationSchema, + @Nullable CryptoKeyReader cryptoKeyReader) { this.sourceConfiguration = sourceConfiguration; this.subscriber = subscriber; this.rangeGenerator = rangeGenerator; @@ -110,6 +117,7 @@ public PulsarSource( this.stopCursor = stopCursor; this.boundedness = boundedness; this.deserializationSchema = deserializationSchema; + this.cryptoKeyReader = cryptoKeyReader; } /** @@ -136,7 +144,7 @@ public SourceReader createReader(SourceReaderContext deserializationSchema.open(initializationContext, sourceConfiguration); return PulsarSourceReaderFactory.create( - readerContext, deserializationSchema, sourceConfiguration); + readerContext, deserializationSchema, sourceConfiguration, cryptoKeyReader); } @Internal diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java index a0a0050e1589b..37dd01ee946d0 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java @@ -37,12 +37,15 @@ import org.apache.flink.connector.pulsar.source.enumerator.topic.range.UniformRangeGenerator; import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.RegexSubscriptionMode; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.util.Arrays; import java.util.List; import java.util.Properties; @@ -127,6 +130,7 @@ public final class PulsarSourceBuilder { private StopCursor stopCursor; private Boundedness boundedness; private PulsarDeserializationSchema deserializationSchema; + @Nullable private CryptoKeyReader cryptoKeyReader; // private builder constructor. PulsarSourceBuilder() { @@ -380,6 +384,18 @@ public PulsarSourceBuilder setDeserializationSchema( return self; } + /** + * Sets a {@link CryptoKeyReader}. Configure the key reader to be used to decrypt the message + * payloads. + * + * @param cryptoKeyReader CryptoKeyReader object + * @return this PulsarSourceBuilder. + */ + public PulsarSourceBuilder setCryptoKeyReader(CryptoKeyReader cryptoKeyReader) { + this.cryptoKeyReader = checkNotNull(cryptoKeyReader); + return this; + } + /** * Set an arbitrary property for the PulsarSource and Pulsar Consumer. The valid keys can be * found in {@link PulsarSourceOptions} and {@link PulsarOptions}. @@ -517,7 +533,8 @@ public PulsarSource build() { startCursor, stopCursor, boundedness, - deserializationSchema); + deserializationSchema, + cryptoKeyReader); } // ------------- private helpers -------------- diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java index 6a5d51522314e..27d9346c421d2 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java @@ -33,11 +33,14 @@ import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.impl.PulsarClientImpl; +import javax.annotation.Nullable; + import java.util.function.Supplier; import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; @@ -63,7 +66,8 @@ private PulsarSourceReaderFactory() { public static SourceReader create( SourceReaderContext readerContext, PulsarDeserializationSchema deserializationSchema, - SourceConfiguration sourceConfiguration) { + SourceConfiguration sourceConfiguration, + @Nullable CryptoKeyReader cryptoKeyReader) { PulsarClient pulsarClient = createClient(sourceConfiguration); PulsarAdmin pulsarAdmin = createAdmin(sourceConfiguration); @@ -84,7 +88,8 @@ public static SourceReader create( pulsarClient, pulsarAdmin, sourceConfiguration, - deserializationSchema); + deserializationSchema, + cryptoKeyReader); return new PulsarOrderedSourceReader<>( elementsQueue, @@ -109,6 +114,7 @@ public static SourceReader create( pulsarAdmin, sourceConfiguration, deserializationSchema, + cryptoKeyReader, coordinatorClient); return new PulsarUnorderedSourceReader<>( diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java index b045d0843c4a7..29a642969e7e6 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java @@ -28,6 +28,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; @@ -36,6 +37,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nullable; + import java.time.Duration; import java.util.concurrent.TimeUnit; @@ -57,8 +60,14 @@ public PulsarOrderedPartitionSplitReader( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, - PulsarDeserializationSchema deserializationSchema) { - super(pulsarClient, pulsarAdmin, sourceConfiguration, deserializationSchema); + PulsarDeserializationSchema deserializationSchema, + @Nullable CryptoKeyReader cryptoKeyReader) { + super( + pulsarClient, + pulsarAdmin, + sourceConfiguration, + deserializationSchema, + cryptoKeyReader); } @Override diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java index 9778bd183c8bd..97fbe46342101 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java @@ -36,6 +36,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.ConsumerBuilder; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.KeySharedPolicy; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; @@ -70,6 +71,7 @@ abstract class PulsarPartitionSplitReaderBase protected final PulsarAdmin pulsarAdmin; protected final SourceConfiguration sourceConfiguration; protected final PulsarDeserializationSchema deserializationSchema; + @Nullable protected final CryptoKeyReader cryptoKeyReader; protected final AtomicBoolean wakeup; protected Consumer pulsarConsumer; @@ -79,11 +81,13 @@ protected PulsarPartitionSplitReaderBase( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, - PulsarDeserializationSchema deserializationSchema) { + PulsarDeserializationSchema deserializationSchema, + @Nullable CryptoKeyReader cryptoKeyReader) { this.pulsarClient = pulsarClient; this.pulsarAdmin = pulsarAdmin; this.sourceConfiguration = sourceConfiguration; this.deserializationSchema = deserializationSchema; + this.cryptoKeyReader = cryptoKeyReader; this.wakeup = new AtomicBoolean(false); } @@ -222,6 +226,11 @@ protected Consumer createPulsarConsumer(TopicPartition partition) { consumerBuilder.topic(partition.getFullTopicName()); + // Add CryptoKeyReader if it exists for supporting end-to-end encryption. + if (cryptoKeyReader != null) { + consumerBuilder.cryptoKeyReader(cryptoKeyReader); + } + // Add KeySharedPolicy for Key_Shared subscription. if (sourceConfiguration.getSubscriptionType() == SubscriptionType.Key_Shared) { KeySharedPolicy policy = diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java index d8dc49c4de2aa..25e1884adac65 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -27,6 +27,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; @@ -68,8 +69,14 @@ public PulsarUnorderedPartitionSplitReader( PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, PulsarDeserializationSchema deserializationSchema, + @Nullable CryptoKeyReader cryptoKeyReader, TransactionCoordinatorClient coordinatorClient) { - super(pulsarClient, pulsarAdmin, sourceConfiguration, deserializationSchema); + super( + pulsarClient, + pulsarAdmin, + sourceConfiguration, + deserializationSchema, + cryptoKeyReader); this.coordinatorClient = coordinatorClient; } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java index 942b75952b8b2..f71facd994aa6 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java @@ -86,7 +86,8 @@ void writeMessageWithoutGuarantee(DeliveryGuarantee guarantee) throws Exception MockInitContext initContext = new MockInitContext(); PulsarWriter writer = - new PulsarWriter<>(configuration, schema, listener, router, delayer, initContext); + new PulsarWriter<>( + configuration, schema, listener, router, delayer, null, initContext); writer.flush(false); writer.prepareCommit(); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegisterTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegisterTest.java index 2e36bfb0cf758..fa4a1f5109ebd 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegisterTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegisterTest.java @@ -49,7 +49,7 @@ void createMessageBuilderForSendingMessage(DeliveryGuarantee deliveryGuarantee) operator().createTopic(topic, 8); SinkConfiguration configuration = sinkConfiguration(deliveryGuarantee); - TopicProducerRegister register = new TopicProducerRegister(configuration); + TopicProducerRegister register = new TopicProducerRegister(configuration, null); String message = randomAlphabetic(10); register.createMessageBuilder(topic, Schema.STRING).value(message).send(); @@ -76,7 +76,7 @@ void noneAndAtLeastOnceWouldNotCreateTransaction(DeliveryGuarantee deliveryGuara operator().createTopic(topic, 8); SinkConfiguration configuration = sinkConfiguration(deliveryGuarantee); - TopicProducerRegister register = new TopicProducerRegister(configuration); + TopicProducerRegister register = new TopicProducerRegister(configuration, null); String message = randomAlphabetic(10); register.createMessageBuilder(topic, Schema.STRING).value(message).sendAsync(); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java index d97120229883b..255355717bf61 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java @@ -60,7 +60,8 @@ void publishTimeStopCursor() throws IOException { operator().client(), operator().admin(), sourceConfig(), - flinkSchema(new SimpleStringSchema())); + flinkSchema(new SimpleStringSchema()), + null); // send the first message and set the stopCursor to filter any late stopCursor operator() .sendMessage( diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java index a42741d790076..be8eaf1035a94 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java @@ -156,7 +156,7 @@ private PulsarSourceReaderBase sourceReader( SourceConfiguration sourceConfiguration = new SourceConfiguration(configuration); return (PulsarSourceReaderBase) PulsarSourceReaderFactory.create( - context, deserializationSchema, sourceConfiguration); + context, deserializationSchema, sourceConfiguration, null); } public class PulsarSourceReaderInvocationContextProvider diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java index f1715e1f42de9..827eae76506f8 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java @@ -334,13 +334,15 @@ private PulsarPartitionSplitReaderBase splitReader(SubscriptionType subs operator().client(), operator().admin(), sourceConfig(), - flinkSchema(new SimpleStringSchema())); + flinkSchema(new SimpleStringSchema()), + null); } else { return new PulsarUnorderedPartitionSplitReader<>( operator().client(), operator().admin(), sourceConfig(), flinkSchema(new SimpleStringSchema()), + null, null); } } From 86241cd4653d2187abc1027acfac2beecad3261a Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 16 Jun 2022 03:44:00 +0800 Subject: [PATCH 216/258] [FLINK-28083][Connector/Pulsar] PulsarSource work with object-reusing DeserializationSchema. --- .../reader/PulsarSourceReaderFactory.java | 19 +++-- .../reader/emitter/PulsarRecordEmitter.java | 57 ++++++++++++-- .../fetcher/PulsarFetcherManagerBase.java | 25 +++--- .../fetcher/PulsarOrderedFetcherManager.java | 18 ++--- .../PulsarUnorderedFetcherManager.java | 14 ++-- .../source/reader/message/PulsarMessage.java | 74 ------------------ .../message/PulsarMessageCollector.java | 60 -------------- .../source/PulsarOrderedSourceReader.java | 15 ++-- .../reader/source/PulsarSourceReaderBase.java | 16 ++-- .../source/PulsarUnorderedSourceReader.java | 14 ++-- .../PulsarOrderedPartitionSplitReader.java | 15 +--- .../split/PulsarPartitionSplitReaderBase.java | 50 ++++-------- .../PulsarUnorderedPartitionSplitReader.java | 17 +--- .../enumerator/cursor/StopCursorTest.java | 18 ++--- ...PulsarOrderedPartitionSplitReaderTest.java | 8 +- .../PulsarPartitionSplitReaderTestBase.java | 78 ++++++++----------- 16 files changed, 176 insertions(+), 322 deletions(-) delete mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessage.java delete mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessageCollector.java diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java index 27d9346c421d2..043331cec79b6 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java @@ -25,7 +25,7 @@ import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; +import org.apache.flink.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; import org.apache.flink.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; import org.apache.flink.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; @@ -34,6 +34,7 @@ import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.CryptoKeyReader; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; @@ -74,26 +75,28 @@ public static SourceReader create( // Create a message queue with the predefined source option. int queueCapacity = sourceConfiguration.getMessageQueueCapacity(); - FutureCompletingBlockingQueue>> elementsQueue = + FutureCompletingBlockingQueue>> elementsQueue = new FutureCompletingBlockingQueue<>(queueCapacity); + PulsarRecordEmitter recordEmitter = new PulsarRecordEmitter<>(deserializationSchema); + // Create different pulsar source reader by subscription type. SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType(); if (subscriptionType == SubscriptionType.Failover || subscriptionType == SubscriptionType.Exclusive) { // Create a ordered split reader supplier. - Supplier> splitReaderSupplier = + Supplier splitReaderSupplier = () -> - new PulsarOrderedPartitionSplitReader<>( + new PulsarOrderedPartitionSplitReader( pulsarClient, pulsarAdmin, sourceConfiguration, - deserializationSchema, cryptoKeyReader); return new PulsarOrderedSourceReader<>( elementsQueue, splitReaderSupplier, + recordEmitter, readerContext, sourceConfiguration, pulsarClient, @@ -107,19 +110,19 @@ public static SourceReader create( throw new IllegalStateException("Transaction is required but didn't enabled"); } - Supplier> splitReaderSupplier = + Supplier splitReaderSupplier = () -> - new PulsarUnorderedPartitionSplitReader<>( + new PulsarUnorderedPartitionSplitReader( pulsarClient, pulsarAdmin, sourceConfiguration, - deserializationSchema, cryptoKeyReader, coordinatorClient); return new PulsarUnorderedSourceReader<>( elementsQueue, splitReaderSupplier, + recordEmitter, readerContext, sourceConfiguration, pulsarClient, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java index f6607f0990ac9..c74a284f05ccd 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/emitter/PulsarRecordEmitter.java @@ -20,10 +20,13 @@ import org.apache.flink.api.connector.source.SourceOutput; import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; import org.apache.flink.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitState; +import org.apache.flink.util.Collector; + +import org.apache.pulsar.client.api.Message; /** * The {@link RecordEmitter} implementation for both {@link PulsarOrderedSourceReader} and {@link @@ -31,15 +34,55 @@ * emitter. */ public class PulsarRecordEmitter - implements RecordEmitter, T, PulsarPartitionSplitState> { + implements RecordEmitter, T, PulsarPartitionSplitState> { + + private final PulsarDeserializationSchema deserializationSchema; + private final SourceOutputWrapper sourceOutputWrapper = new SourceOutputWrapper<>(); + + public PulsarRecordEmitter(PulsarDeserializationSchema deserializationSchema) { + this.deserializationSchema = deserializationSchema; + } @Override public void emitRecord( - PulsarMessage element, SourceOutput output, PulsarPartitionSplitState splitState) + Message element, SourceOutput output, PulsarPartitionSplitState splitState) throws Exception { - // Sink the record to source output. - output.collect(element.getValue(), element.getEventTime()); - // Update the split state. - splitState.setLatestConsumedId(element.getId()); + // Update the source output. + sourceOutputWrapper.setSourceOutput(output); + sourceOutputWrapper.setTimestamp(element); + + deserializationSchema.deserialize(element, sourceOutputWrapper); + splitState.setLatestConsumedId(element.getMessageId()); + } + + private static class SourceOutputWrapper implements Collector { + private SourceOutput sourceOutput; + private long timestamp; + + @Override + public void collect(T record) { + if (timestamp > 0) { + sourceOutput.collect(record, timestamp); + } else { + sourceOutput.collect(record); + } + } + + @Override + public void close() { + // Nothing to do here. + } + + private void setSourceOutput(SourceOutput sourceOutput) { + this.sourceOutput = sourceOutput; + } + + /** + * Get the event timestamp from Pulsar. Zero means there is no event time. See {@link + * Message#getEventTime()} to get the reason why it returns zero. + */ + private void setTimestamp(Message message) { + this.timestamp = message.getEventTime(); + } } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java index 1f9a35f6c681b..3e2daf1e16c74 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java @@ -25,9 +25,10 @@ import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcherManager; import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.pulsar.client.api.Message; + import java.util.HashMap; import java.util.List; import java.util.Map; @@ -35,14 +36,10 @@ import static java.util.Collections.singletonList; -/** - * Common fetcher manager abstraction for both ordered & unordered message. - * - * @param The decoded message type for flink. - */ +/** Common fetcher manager abstraction for both ordered & unordered message. */ @Internal -public abstract class PulsarFetcherManagerBase - extends SplitFetcherManager, PulsarPartitionSplit> { +public abstract class PulsarFetcherManagerBase + extends SplitFetcherManager, PulsarPartitionSplit> { private final Map splitFetcherMapping = new HashMap<>(); private final Map fetcherStatus = new HashMap<>(); @@ -56,8 +53,8 @@ public abstract class PulsarFetcherManagerBase * @param splitReaderSupplier The factory for the split reader that connects to the source */ protected PulsarFetcherManagerBase( - FutureCompletingBlockingQueue>> elementsQueue, - Supplier, PulsarPartitionSplit>> splitReaderSupplier) { + FutureCompletingBlockingQueue>> elementsQueue, + Supplier, PulsarPartitionSplit>> splitReaderSupplier) { super(elementsQueue, splitReaderSupplier); } @@ -68,7 +65,7 @@ protected PulsarFetcherManagerBase( @Override public void addSplits(List splitsToAdd) { for (PulsarPartitionSplit split : splitsToAdd) { - SplitFetcher, PulsarPartitionSplit> fetcher = + SplitFetcher, PulsarPartitionSplit> fetcher = getOrCreateFetcher(split.splitId()); fetcher.addSplits(singletonList(split)); // This method could be executed multiple times. @@ -77,16 +74,16 @@ public void addSplits(List splitsToAdd) { } @Override - protected void startFetcher(SplitFetcher, PulsarPartitionSplit> fetcher) { + protected void startFetcher(SplitFetcher, PulsarPartitionSplit> fetcher) { if (fetcherStatus.get(fetcher.fetcherId()) != Boolean.TRUE) { fetcherStatus.put(fetcher.fetcherId(), true); super.startFetcher(fetcher); } } - protected SplitFetcher, PulsarPartitionSplit> getOrCreateFetcher( + protected SplitFetcher, PulsarPartitionSplit> getOrCreateFetcher( String splitId) { - SplitFetcher, PulsarPartitionSplit> fetcher; + SplitFetcher, PulsarPartitionSplit> fetcher; Integer fetcherId = splitFetcherMapping.get(splitId); if (fetcherId == null) { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java index f8b89ee98bca9..3178be78fb444 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarOrderedFetcherManager.java @@ -24,11 +24,11 @@ import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,16 +40,14 @@ * Pulsar's FetcherManager implementation for ordered consuming. This class is needed to help * acknowledge the message to Pulsar using the {@link Consumer} inside the {@link * PulsarOrderedPartitionSplitReader}. - * - * @param The message type for pulsar decoded message. */ @Internal -public class PulsarOrderedFetcherManager extends PulsarFetcherManagerBase { +public class PulsarOrderedFetcherManager extends PulsarFetcherManagerBase { private static final Logger LOG = LoggerFactory.getLogger(PulsarOrderedFetcherManager.class); public PulsarOrderedFetcherManager( - FutureCompletingBlockingQueue>> elementsQueue, - Supplier, PulsarPartitionSplit>> splitReaderSupplier) { + FutureCompletingBlockingQueue>> elementsQueue, + Supplier, PulsarPartitionSplit>> splitReaderSupplier) { super(elementsQueue, splitReaderSupplier); } @@ -57,18 +55,18 @@ public void acknowledgeMessages(Map cursorsToCommit) LOG.debug("Acknowledge messages {}", cursorsToCommit); cursorsToCommit.forEach( (partition, messageId) -> { - SplitFetcher, PulsarPartitionSplit> fetcher = + SplitFetcher, PulsarPartitionSplit> fetcher = getOrCreateFetcher(partition.toString()); triggerAcknowledge(fetcher, partition, messageId); }); } private void triggerAcknowledge( - SplitFetcher, PulsarPartitionSplit> splitFetcher, + SplitFetcher, PulsarPartitionSplit> splitFetcher, TopicPartition partition, MessageId messageId) { - PulsarOrderedPartitionSplitReader splitReader = - (PulsarOrderedPartitionSplitReader) splitFetcher.getSplitReader(); + PulsarOrderedPartitionSplitReader splitReader = + (PulsarOrderedPartitionSplitReader) splitFetcher.getSplitReader(); splitReader.notifyCheckpointComplete(partition, messageId); startFetcher(splitFetcher); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java index c086a3fba680e..3af00ab68a96d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarUnorderedFetcherManager.java @@ -23,11 +23,11 @@ import org.apache.flink.connector.base.source.reader.fetcher.SplitFetcher; import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.reader.split.PulsarUnorderedPartitionSplitReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; import java.util.ArrayList; import java.util.List; @@ -39,15 +39,13 @@ * Pulsar's FetcherManager implementation for unordered consuming. This class is needed to help * acknowledge the message to Pulsar using the {@link Consumer} inside the {@link * PulsarUnorderedPartitionSplitReader}. - * - * @param The message type for pulsar decoded message. */ @Internal -public class PulsarUnorderedFetcherManager extends PulsarFetcherManagerBase { +public class PulsarUnorderedFetcherManager extends PulsarFetcherManagerBase { public PulsarUnorderedFetcherManager( - FutureCompletingBlockingQueue>> elementsQueue, - Supplier, PulsarPartitionSplit>> splitReaderSupplier) { + FutureCompletingBlockingQueue>> elementsQueue, + Supplier, PulsarPartitionSplit>> splitReaderSupplier) { super(elementsQueue, splitReaderSupplier); } @@ -59,8 +57,8 @@ public List snapshotState(long checkpointId) { } private PulsarPartitionSplit snapshotReader( - long checkpointId, SplitReader, PulsarPartitionSplit> splitReader) { - return ((PulsarUnorderedPartitionSplitReader) splitReader) + long checkpointId, SplitReader, PulsarPartitionSplit> splitReader) { + return ((PulsarUnorderedPartitionSplitReader) splitReader) .snapshotState(checkpointId) .toPulsarPartitionSplit(); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessage.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessage.java deleted file mode 100644 index 0632e2265d406..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessage.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.source.reader.message; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; - -import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.MessageId; - -/** - * The message instance that contains the required information which would be used for committing - * the consuming status. - */ -@Internal -public class PulsarMessage { - - /** - * The id of a given message. This id could be same for multiple {@link PulsarMessage}, although - * it is unique for every {@link Message}. - */ - private final MessageId id; - - /** The value which deserialized by {@link PulsarDeserializationSchema}. */ - private final T value; - - /** The produce time for this message, it's a event time. */ - private final long eventTime; - - public PulsarMessage(MessageId id, T value, long eventTime) { - this.id = id; - this.value = value; - this.eventTime = eventTime; - } - - public MessageId getId() { - return id; - } - - public T getValue() { - return value; - } - - public long getEventTime() { - return eventTime; - } - - @Override - public String toString() { - return "PulsarMessage{" - + "id=" - + id - + ", value=" - + value - + ", eventTime=" - + eventTime - + '}'; - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessageCollector.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessageCollector.java deleted file mode 100644 index f201425fc71cf..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/message/PulsarMessageCollector.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.source.reader.message; - -import org.apache.flink.connector.base.source.reader.RecordsBySplits; -import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; -import org.apache.flink.util.Collector; - -import org.apache.pulsar.client.api.Message; - -/** - * This collector supplier is providing the {@link Collector} for accepting the deserialized {@link - * PulsarMessage} from pulsar {@link PulsarDeserializationSchema}. - * - * @param The deserialized pulsar message type, aka the source message type. - */ -public class PulsarMessageCollector implements Collector { - - private final String splitId; - private final RecordsBySplits.Builder> builder; - private Message message; - - public PulsarMessageCollector( - String splitId, RecordsBySplits.Builder> builder) { - this.splitId = splitId; - this.builder = builder; - } - - public void setMessage(Message message) { - this.message = message; - } - - @Override - public void collect(T t) { - PulsarMessage result = - new PulsarMessage<>(message.getMessageId(), t, message.getEventTime()); - builder.add(splitId, result); - } - - @Override - public void close() { - // Nothing to do for this collector. - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java index 8c197afdf50c1..c25e6c0822350 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java @@ -26,14 +26,15 @@ import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; import org.apache.flink.connector.pulsar.source.reader.fetcher.PulsarOrderedFetcherManager; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitState; import org.apache.flink.core.io.InputStatus; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.slf4j.Logger; @@ -67,15 +68,17 @@ public class PulsarOrderedSourceReader extends PulsarSourceReaderBase private ScheduledExecutorService cursorScheduler; public PulsarOrderedSourceReader( - FutureCompletingBlockingQueue>> elementsQueue, - Supplier> splitReaderSupplier, + FutureCompletingBlockingQueue>> elementsQueue, + Supplier splitReaderSupplier, + PulsarRecordEmitter recordEmitter, SourceReaderContext context, SourceConfiguration sourceConfiguration, PulsarClient pulsarClient, PulsarAdmin pulsarAdmin) { super( elementsQueue, - new PulsarOrderedFetcherManager<>(elementsQueue, splitReaderSupplier::get), + new PulsarOrderedFetcherManager(elementsQueue, splitReaderSupplier::get), + recordEmitter, context, sourceConfiguration, pulsarClient, @@ -147,7 +150,7 @@ public void notifyCheckpointComplete(long checkpointId) { LOG.debug("Committing cursors for checkpoint {}", checkpointId); Map cursors = cursorsToCommit.get(checkpointId); try { - ((PulsarOrderedFetcherManager) splitFetcherManager).acknowledgeMessages(cursors); + ((PulsarOrderedFetcherManager) splitFetcherManager).acknowledgeMessages(cursors); LOG.debug("Successfully acknowledge cursors for checkpoint {}", checkpointId); // Clean up the cursors. @@ -192,7 +195,7 @@ private void cumulativeAcknowledgmentMessage() { } try { - ((PulsarOrderedFetcherManager) splitFetcherManager).acknowledgeMessages(cursors); + ((PulsarOrderedFetcherManager) splitFetcherManager).acknowledgeMessages(cursors); // Clean up the finish splits. cursorsOfFinishedSplits.keySet().removeAll(cursors.keySet()); } catch (Exception e) { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java index 0122021145439..84e02659380e1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java @@ -25,11 +25,11 @@ import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; import org.apache.flink.connector.pulsar.source.reader.fetcher.PulsarFetcherManagerBase; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitState; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; /** @@ -39,25 +39,21 @@ */ abstract class PulsarSourceReaderBase extends SourceReaderBase< - PulsarMessage, OUT, PulsarPartitionSplit, PulsarPartitionSplitState> { + Message, OUT, PulsarPartitionSplit, PulsarPartitionSplitState> { protected final SourceConfiguration sourceConfiguration; protected final PulsarClient pulsarClient; protected final PulsarAdmin pulsarAdmin; protected PulsarSourceReaderBase( - FutureCompletingBlockingQueue>> elementsQueue, - PulsarFetcherManagerBase splitFetcherManager, + FutureCompletingBlockingQueue>> elementsQueue, + PulsarFetcherManagerBase splitFetcherManager, + PulsarRecordEmitter recordEmitter, SourceReaderContext context, SourceConfiguration sourceConfiguration, PulsarClient pulsarClient, PulsarAdmin pulsarAdmin) { - super( - elementsQueue, - splitFetcherManager, - new PulsarRecordEmitter<>(), - sourceConfiguration, - context); + super(elementsQueue, splitFetcherManager, recordEmitter, sourceConfiguration, context); this.sourceConfiguration = sourceConfiguration; this.pulsarClient = pulsarClient; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java index 2af77d9a8467c..e130031fb37b4 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java @@ -24,13 +24,14 @@ import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; import org.apache.flink.connector.pulsar.source.reader.fetcher.PulsarUnorderedFetcherManager; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.reader.split.PulsarUnorderedPartitionSplitReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitState; import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.api.transaction.TxnID; @@ -60,8 +61,9 @@ public class PulsarUnorderedSourceReader extends PulsarSourceReaderBase transactionsOfFinishedSplits; public PulsarUnorderedSourceReader( - FutureCompletingBlockingQueue>> elementsQueue, - Supplier> splitReaderSupplier, + FutureCompletingBlockingQueue>> elementsQueue, + Supplier splitReaderSupplier, + PulsarRecordEmitter recordEmitter, SourceReaderContext context, SourceConfiguration sourceConfiguration, PulsarClient pulsarClient, @@ -69,7 +71,8 @@ public PulsarUnorderedSourceReader( @Nullable TransactionCoordinatorClient coordinatorClient) { super( elementsQueue, - new PulsarUnorderedFetcherManager<>(elementsQueue, splitReaderSupplier::get), + new PulsarUnorderedFetcherManager(elementsQueue, splitReaderSupplier::get), + recordEmitter, context, sourceConfiguration, pulsarClient, @@ -103,8 +106,7 @@ protected void onSplitFinished(Map finishedSp public List snapshotState(long checkpointId) { LOG.debug("Trigger the new transaction for downstream readers."); List splits = - ((PulsarUnorderedFetcherManager) splitFetcherManager) - .snapshotState(checkpointId); + ((PulsarUnorderedFetcherManager) splitFetcherManager).snapshotState(checkpointId); if (coordinatorClient != null) { // Snapshot the transaction status and commit it after checkpoint finished. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java index 29a642969e7e6..f9416305ecb7c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java @@ -22,7 +22,6 @@ import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; @@ -48,11 +47,9 @@ /** * The split reader a given {@link PulsarPartitionSplit}, it would be closed once the {@link * PulsarOrderedSourceReader} is closed. - * - * @param the type of the pulsar source message that would be serialized to downstream. */ @Internal -public class PulsarOrderedPartitionSplitReader extends PulsarPartitionSplitReaderBase { +public class PulsarOrderedPartitionSplitReader extends PulsarPartitionSplitReaderBase { private static final Logger LOG = LoggerFactory.getLogger(PulsarOrderedPartitionSplitReader.class); @@ -60,18 +57,12 @@ public PulsarOrderedPartitionSplitReader( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, - PulsarDeserializationSchema deserializationSchema, @Nullable CryptoKeyReader cryptoKeyReader) { - super( - pulsarClient, - pulsarAdmin, - sourceConfiguration, - deserializationSchema, - cryptoKeyReader); + super(pulsarClient, pulsarAdmin, sourceConfiguration, cryptoKeyReader); } @Override - protected Message pollMessage(Duration timeout) throws PulsarClientException { + protected Message pollMessage(Duration timeout) throws PulsarClientException { return pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java index 97fbe46342101..963f17c1b1a69 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java @@ -27,9 +27,6 @@ import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessageCollector; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.util.Preconditions; @@ -58,42 +55,36 @@ import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; import static org.apache.flink.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder; -/** - * The common partition split reader. - * - * @param the type of the pulsar source message that would be serialized to downstream. - */ -abstract class PulsarPartitionSplitReaderBase - implements SplitReader, PulsarPartitionSplit> { +/** The common partition split reader. */ +abstract class PulsarPartitionSplitReaderBase + implements SplitReader, PulsarPartitionSplit> { private static final Logger LOG = LoggerFactory.getLogger(PulsarPartitionSplitReaderBase.class); protected final PulsarClient pulsarClient; protected final PulsarAdmin pulsarAdmin; protected final SourceConfiguration sourceConfiguration; - protected final PulsarDeserializationSchema deserializationSchema; @Nullable protected final CryptoKeyReader cryptoKeyReader; + protected final AtomicBoolean wakeup; - protected Consumer pulsarConsumer; + protected Consumer pulsarConsumer; protected PulsarPartitionSplit registeredSplit; protected PulsarPartitionSplitReaderBase( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, - PulsarDeserializationSchema deserializationSchema, @Nullable CryptoKeyReader cryptoKeyReader) { this.pulsarClient = pulsarClient; this.pulsarAdmin = pulsarAdmin; this.sourceConfiguration = sourceConfiguration; - this.deserializationSchema = deserializationSchema; this.cryptoKeyReader = cryptoKeyReader; this.wakeup = new AtomicBoolean(false); } @Override - public RecordsWithSplitIds> fetch() throws IOException { - RecordsBySplits.Builder> builder = new RecordsBySplits.Builder<>(); + public RecordsWithSplitIds> fetch() throws IOException { + RecordsBySplits.Builder> builder = new RecordsBySplits.Builder<>(); // Return when no split registered to this reader. if (pulsarConsumer == null || registeredSplit == null) { @@ -105,7 +96,6 @@ public RecordsWithSplitIds> fetch() throws IOException { StopCursor stopCursor = registeredSplit.getStopCursor(); String splitId = registeredSplit.splitId(); - PulsarMessageCollector collector = new PulsarMessageCollector<>(splitId, builder); Deadline deadline = Deadline.fromNow(sourceConfiguration.getMaxFetchTime()); // Consume message from pulsar until it was woken up by flink reader. @@ -116,15 +106,12 @@ public RecordsWithSplitIds> fetch() throws IOException { messageNum++) { try { Duration timeout = deadline.timeLeftIfAny(); - Message message = pollMessage(timeout); + Message message = pollMessage(timeout); if (message == null) { break; } - collector.setMessage(message); - - // Deserialize message by DeserializationSchema or Pulsar Schema. - deserializationSchema.deserialize(message, collector); + builder.add(splitId, message); // Acknowledge message if needed. finishedPollMessage(message); @@ -171,7 +158,7 @@ public void handleSplitsChanges(SplitsChange splitsChanges PulsarPartitionSplit newSplit = newSplits.get(0); // Create pulsar consumer. - Consumer consumer = createPulsarConsumer(newSplit); + Consumer consumer = createPulsarConsumer(newSplit); // Open start & stop cursor. newSplit.open(pulsarAdmin); @@ -197,7 +184,7 @@ public void close() { } @Nullable - protected abstract Message pollMessage(Duration timeout) + protected abstract Message pollMessage(Duration timeout) throws ExecutionException, InterruptedException, PulsarClientException; protected abstract void finishedPollMessage(Message message); @@ -210,19 +197,16 @@ protected boolean isNotWakeup() { return !wakeup.get(); } - /** - * Create a specified {@link Consumer} by the given split information. If using pulsar schema, - * then use the pulsar schema, if using flink schema, then use a Schema.BYTES - */ - protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { + /** Create a specified {@link Consumer} by the given split information. */ + protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { return createPulsarConsumer(split.getPartition()); } - protected Consumer createPulsarConsumer(TopicPartition partition) { - Schema schema = deserializationSchema.schema(); + protected Consumer createPulsarConsumer(TopicPartition partition) { + // Schema schema = deserializationSchema.schema(); - ConsumerBuilder consumerBuilder = - createConsumerBuilder(pulsarClient, schema, sourceConfiguration); + ConsumerBuilder consumerBuilder = + createConsumerBuilder(pulsarClient, Schema.BYTES, sourceConfiguration); consumerBuilder.topic(partition.getFullTopicName()); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java index 25e1884adac65..bf9d5308d68e3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -20,7 +20,6 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplitState; @@ -50,11 +49,9 @@ /** * The split reader a given {@link PulsarPartitionSplit}, it would be closed once the {@link * PulsarUnorderedSourceReader} is closed. - * - * @param the type of the pulsar source message that would be serialized to downstream. */ @Internal -public class PulsarUnorderedPartitionSplitReader extends PulsarPartitionSplitReaderBase { +public class PulsarUnorderedPartitionSplitReader extends PulsarPartitionSplitReaderBase { private static final Logger LOG = LoggerFactory.getLogger(PulsarUnorderedPartitionSplitReader.class); @@ -68,23 +65,17 @@ public PulsarUnorderedPartitionSplitReader( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, - PulsarDeserializationSchema deserializationSchema, @Nullable CryptoKeyReader cryptoKeyReader, TransactionCoordinatorClient coordinatorClient) { - super( - pulsarClient, - pulsarAdmin, - sourceConfiguration, - deserializationSchema, - cryptoKeyReader); + super(pulsarClient, pulsarAdmin, sourceConfiguration, cryptoKeyReader); this.coordinatorClient = coordinatorClient; } @Override - protected Message pollMessage(Duration timeout) + protected Message pollMessage(Duration timeout) throws ExecutionException, InterruptedException, PulsarClientException { - Message message = + Message message = pulsarConsumer.receive(Math.toIntExact(timeout.toMillis()), TimeUnit.MILLISECONDS); // Skip the message when receive timeout diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java index 255355717bf61..f74c9de8ec670 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java @@ -18,18 +18,17 @@ package org.apache.flink.connector.pulsar.source.enumerator.cursor; -import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Schema; import org.junit.jupiter.api.Test; @@ -44,7 +43,6 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.flinkSchema; import static org.assertj.core.api.Assertions.assertThat; /** Test different implementation of StopCursor. */ @@ -55,13 +53,9 @@ void publishTimeStopCursor() throws IOException { String topicName = randomAlphanumeric(5); operator().createTopic(topicName, 2); - PulsarOrderedPartitionSplitReader splitReader = - new PulsarOrderedPartitionSplitReader<>( - operator().client(), - operator().admin(), - sourceConfig(), - flinkSchema(new SimpleStringSchema()), - null); + PulsarOrderedPartitionSplitReader splitReader = + new PulsarOrderedPartitionSplitReader( + operator().client(), operator().admin(), sourceConfig(), null); // send the first message and set the stopCursor to filter any late stopCursor operator() .sendMessage( @@ -79,7 +73,7 @@ void publishTimeStopCursor() throws IOException { SplitsAddition addition = new SplitsAddition<>(singletonList(split)); splitReader.handleSplitsChanges(addition); // first fetch should have result - RecordsWithSplitIds> firstResult = splitReader.fetch(); + RecordsWithSplitIds> firstResult = splitReader.fetch(); assertThat(firstResult.nextSplit()).isNotNull(); assertThat(firstResult.nextRecordFromSplit()).isNotNull(); assertThat(firstResult.finishedSplits()).isEmpty(); @@ -89,7 +83,7 @@ void publishTimeStopCursor() throws IOException { TopicNameUtils.topicNameWithPartition(topicName, 0), Schema.STRING, randomAlphanumeric(10)); - RecordsWithSplitIds> secondResult = splitReader.fetch(); + RecordsWithSplitIds> secondResult = splitReader.fetch(); assertThat(secondResult.nextSplit()).isNotNull(); assertThat(firstResult.nextRecordFromSplit()).isNull(); assertThat(secondResult.finishedSplits()).isNotEmpty(); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java index 3d58d5efeb4c8..4d93ce92ede36 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReaderTest.java @@ -38,7 +38,7 @@ class PulsarOrderedPartitionSplitReaderTest extends PulsarPartitionSplitReaderTe @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesWithoutSeek( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); handleSplit(splitReader, topicName, 0); @@ -47,7 +47,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesWithoutSeek( @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndUseLatestStartCursorWithoutSeek( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); handleSplit(splitReader, topicName, 0, MessageId.latest); @@ -56,7 +56,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesAndUseLatestStartCursorWithou @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndUseEarliestStartCursorWithoutSeek( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); handleSplit(splitReader, topicName, 0, MessageId.earliest); @@ -65,7 +65,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesAndUseEarliestStartCursorWith @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndUseSecondLastMessageWithoutSeek( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); MessageIdImpl lastMessageId = diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java index 827eae76506f8..261733b5390ae 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java @@ -18,7 +18,6 @@ package org.apache.flink.connector.pulsar.source.reader.split; -import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; @@ -26,13 +25,13 @@ import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; -import org.apache.flink.connector.pulsar.source.reader.message.PulsarMessage; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; import org.apache.flink.connector.pulsar.testutils.extension.TestOrderlinessExtension; import org.apache.flink.util.TestLoggerExtension; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.SubscriptionType; @@ -68,7 +67,6 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.flinkSchema; import static org.apache.flink.connector.pulsar.testutils.PulsarTestCommonUtils.isAssignableFromParameterContext; import static org.apache.flink.connector.pulsar.testutils.extension.TestOrderlinessExtension.PULSAR_SOURCE_READER_SUBSCRIPTION_TYPE_STORE_KEY; import static org.apache.flink.connector.pulsar.testutils.extension.TestOrderlinessExtension.PULSAR_TEST_RESOURCE_NAMESPACE; @@ -103,12 +101,12 @@ private SourceConfiguration sourceConfig() { } protected void handleSplit( - PulsarPartitionSplitReaderBase reader, String topicName, int partitionId) { + PulsarPartitionSplitReaderBase reader, String topicName, int partitionId) { handleSplit(reader, topicName, partitionId, null); } protected void handleSplit( - PulsarPartitionSplitReaderBase reader, + PulsarPartitionSplitReaderBase reader, String topicName, int partitionId, MessageId startPosition) { @@ -120,12 +118,12 @@ protected void handleSplit( } private void seekStartPositionAndHandleSplit( - PulsarPartitionSplitReaderBase reader, String topicName, int partitionId) { + PulsarPartitionSplitReaderBase reader, String topicName, int partitionId) { seekStartPositionAndHandleSplit(reader, topicName, partitionId, MessageId.latest); } private void seekStartPositionAndHandleSplit( - PulsarPartitionSplitReaderBase reader, + PulsarPartitionSplitReaderBase reader, String topicName, int partitionId, MessageId startPosition) { @@ -146,29 +144,29 @@ private void seekStartPositionAndHandleSplit( reader.handleSplitsChanges(addition); } - private PulsarMessage fetchedMessage(PulsarPartitionSplitReaderBase splitReader) { + private Message fetchedMessage(PulsarPartitionSplitReaderBase splitReader) { return fetchedMessages(splitReader, 1, false).stream().findFirst().orElse(null); } - protected List> fetchedMessages( - PulsarPartitionSplitReaderBase splitReader, int expectedCount, boolean verify) { + protected List> fetchedMessages( + PulsarPartitionSplitReaderBase splitReader, int expectedCount, boolean verify) { return fetchedMessages( splitReader, expectedCount, verify, Boundedness.CONTINUOUS_UNBOUNDED); } - private List> fetchedMessages( - PulsarPartitionSplitReaderBase splitReader, + private List> fetchedMessages( + PulsarPartitionSplitReaderBase splitReader, int expectedCount, boolean verify, Boundedness boundedness) { - List> messages = new ArrayList<>(expectedCount); - List finishedSplits = new ArrayList<>(); + List> messages = new ArrayList<>(expectedCount); + List finishedSplits = new ArrayList<>(); for (int i = 0; i < 3; ) { try { - RecordsWithSplitIds> recordsBySplitIds = splitReader.fetch(); + RecordsWithSplitIds> recordsBySplitIds = splitReader.fetch(); if (recordsBySplitIds.nextSplit() != null) { // Collect the records in this split. - PulsarMessage record; + Message record; while ((record = recordsBySplitIds.nextRecordFromSplit()) != null) { messages.add(record); } @@ -194,7 +192,7 @@ private List> fetchedMessages( } @TestTemplate - void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) + void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) throws InterruptedException, TimeoutException { String topicName = randomAlphabetic(10); @@ -202,7 +200,7 @@ void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) seekStartPositionAndHandleSplit(splitReader, topicName, 0); // Poll once with a null message - PulsarMessage message1 = fetchedMessage(splitReader); + Message message1 = fetchedMessage(splitReader); assertThat(message1).isNull(); // Send a message to pulsar @@ -212,7 +210,7 @@ void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) // Poll this message again waitUtil( () -> { - PulsarMessage message2 = fetchedMessage(splitReader); + Message message2 = fetchedMessage(splitReader); return message2 != null; }, ofSeconds(Integer.MAX_VALUE), @@ -221,7 +219,7 @@ void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) @TestTemplate void consumeMessageCreatedAfterHandleSplitChangesAndFetch( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); seekStartPositionAndHandleSplit(splitReader, topicName, 0); operator().sendMessage(topicNameWithPartition(topicName, 0), STRING, randomAlphabetic(10)); @@ -230,7 +228,7 @@ void consumeMessageCreatedAfterHandleSplitChangesAndFetch( @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChanges( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); seekStartPositionAndHandleSplit(splitReader, topicName, 0); @@ -239,7 +237,7 @@ void consumeMessageCreatedBeforeHandleSplitsChanges( @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndResetToEarliestPosition( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); seekStartPositionAndHandleSplit(splitReader, topicName, 0, MessageId.earliest); @@ -248,7 +246,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesAndResetToEarliestPosition( @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndResetToLatestPosition( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); seekStartPositionAndHandleSplit(splitReader, topicName, 0, MessageId.latest); @@ -257,7 +255,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesAndResetToLatestPosition( @TestTemplate void consumeMessageCreatedBeforeHandleSplitsChangesAndUseSecondLastMessageIdCursor( - PulsarPartitionSplitReaderBase splitReader) { + PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().setupTopic(topicName, STRING, () -> randomAlphabetic(10)); @@ -283,7 +281,7 @@ void consumeMessageCreatedBeforeHandleSplitsChangesAndUseSecondLastMessageIdCurs } @TestTemplate - void emptyTopic(PulsarPartitionSplitReaderBase splitReader) { + void emptyTopic(PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().createTopic(topicName, DEFAULT_PARTITIONS); seekStartPositionAndHandleSplit(splitReader, topicName, 0); @@ -291,7 +289,7 @@ void emptyTopic(PulsarPartitionSplitReaderBase splitReader) { } @TestTemplate - void emptyTopicWithoutSeek(PulsarPartitionSplitReaderBase splitReader) { + void emptyTopicWithoutSeek(PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); operator().createTopic(topicName, DEFAULT_PARTITIONS); handleSplit(splitReader, topicName, 0); @@ -299,8 +297,7 @@ void emptyTopicWithoutSeek(PulsarPartitionSplitReaderBase splitReader) { } @TestTemplate - void wakeupSplitReaderShouldNotCauseException( - PulsarPartitionSplitReaderBase splitReader) { + void wakeupSplitReaderShouldNotCauseException(PulsarPartitionSplitReaderBase splitReader) { handleSplit(splitReader, "non-exist", 0); AtomicReference error = new AtomicReference<>(); Thread t = @@ -323,27 +320,18 @@ void wakeupSplitReaderShouldNotCauseException( } @TestTemplate - void assignNoSplits(PulsarPartitionSplitReaderBase splitReader) { + void assignNoSplits(PulsarPartitionSplitReaderBase splitReader) { assertThat(fetchedMessage(splitReader)).isNull(); } /** Create a split reader with max message 1, fetch timeout 1s. */ - private PulsarPartitionSplitReaderBase splitReader(SubscriptionType subscriptionType) { + private PulsarPartitionSplitReaderBase splitReader(SubscriptionType subscriptionType) { if (subscriptionType == SubscriptionType.Failover) { - return new PulsarOrderedPartitionSplitReader<>( - operator().client(), - operator().admin(), - sourceConfig(), - flinkSchema(new SimpleStringSchema()), - null); + return new PulsarOrderedPartitionSplitReader( + operator().client(), operator().admin(), sourceConfig(), null); } else { - return new PulsarUnorderedPartitionSplitReader<>( - operator().client(), - operator().admin(), - sourceConfig(), - flinkSchema(new SimpleStringSchema()), - null, - null); + return new PulsarUnorderedPartitionSplitReader( + operator().client(), operator().admin(), sourceConfig(), null, null); } } @@ -371,9 +359,9 @@ public Stream provideTestTemplateInvocationContex public static class PulsarSplitReaderInvocationContext implements TestTemplateInvocationContext { - private final PulsarPartitionSplitReaderBase splitReader; + private final PulsarPartitionSplitReaderBase splitReader; - public PulsarSplitReaderInvocationContext(PulsarPartitionSplitReaderBase splitReader) { + public PulsarSplitReaderInvocationContext(PulsarPartitionSplitReaderBase splitReader) { this.splitReader = checkNotNull(splitReader); } From de7d75a72036439c3d76c9e39a5be5bbda49ac13 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 31 Mar 2022 01:55:36 +0800 Subject: [PATCH 217/258] [FLINK-26182][Connector/pulsar] Extract common logic from Pulsar source test tools. --- .../source/enumerator/topic/TopicRange.java | 2 +- .../pulsar/source/PulsarSourceITCase.java | 4 +- .../testutils/PulsarTestCommonUtils.java | 17 ++- .../pulsar/testutils/PulsarTestContext.java | 55 ++++--- .../testutils/PulsarTestContextFactory.java | 7 +- .../testutils/PulsarTestEnvironment.java | 2 +- .../pulsar/testutils/PulsarTestSuiteBase.java | 6 +- .../cases/MultipleTopicTemplateContext.java | 134 ---------------- .../cases/SingleTopicConsumingContext.java | 132 ---------------- .../runtime/PulsarRuntimeOperator.java | 52 ++++--- .../container/PulsarContainerRuntime.java | 1 - .../embedded/PulsarEmbeddedRuntime.java | 3 +- .../PulsarPartitionDataWriter.java | 18 ++- .../source/PulsarSourceTestContext.java | 138 +++++++++++++++++ .../cases/MultipleTopicConsumingContext.java | 72 +++++++++ .../cases/SingleTopicConsumingContext.java | 74 +++++++++ .../pulsar/PulsarSourceOrderedE2ECase.java | 5 +- .../pulsar/PulsarSourceUnorderedE2ECase.java | 11 +- .../cases/KeySharedSubscriptionContext.java | 144 ------------------ .../cases/SharedSubscriptionContext.java | 116 -------------- .../FlinkContainerWithPulsarEnvironment.java | 3 +- .../KeyedPulsarPartitionDataWriter.java | 19 ++- .../common/UnorderedSourceTestSuiteBase.java | 108 +++++++------ .../ExclusiveSubscriptionContext.java | 19 +-- .../FailoverSubscriptionContext.java | 19 +-- .../source/KeySharedSubscriptionContext.java | 91 +++++++++++ .../source/SharedSubscriptionContext.java | 29 ++-- .../enumerator/NoOpEnumStateSerializer.java | 2 +- 28 files changed, 573 insertions(+), 710 deletions(-) delete mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java delete mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java rename flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/{ => source}/PulsarPartitionDataWriter.java (69%) create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java delete mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java delete mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java rename flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/{cases => source}/ExclusiveSubscriptionContext.java (71%) rename flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/{cases => source}/FailoverSubscriptionContext.java (71%) create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java rename flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java => flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java (59%) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java index 5b779224b87c7..1508b8732a311 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java @@ -68,7 +68,7 @@ public Range toPulsarRange() { return new Range(start, end); } - /** Create a topic range which contains the fully hash range. */ + /** Create a topic range which contains the full hash range. */ public static TopicRange createFullRange() { return new TopicRange(MIN_RANGE, MAX_RANGE); } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java index 0259773445ee1..50742837c5539 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java @@ -20,9 +20,9 @@ import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory; import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.pulsar.testutils.cases.MultipleTopicConsumingContext; -import org.apache.flink.connector.pulsar.testutils.cases.SingleTopicConsumingContext; import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; +import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; +import org.apache.flink.connector.pulsar.testutils.source.cases.SingleTopicConsumingContext; import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; import org.apache.flink.connector.testframe.environment.TestEnvironment; import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java index 87f3976c6a74c..96b1ca62f2a06 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestCommonUtils.java @@ -19,10 +19,12 @@ package org.apache.flink.connector.pulsar.testutils; import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.pulsar.client.api.MessageId; import org.junit.jupiter.api.extension.ParameterContext; @@ -33,7 +35,18 @@ /** Put static methods that can be used by multiple test classes. */ public class PulsarTestCommonUtils { - // ------- CreateSplits + /** Convert the CheckpointingMode to a connector related DeliveryGuarantee. */ + public static DeliveryGuarantee toDeliveryGuarantee(CheckpointingMode checkpointingMode) { + if (checkpointingMode == CheckpointingMode.AT_LEAST_ONCE) { + return DeliveryGuarantee.AT_LEAST_ONCE; + } else if (checkpointingMode == CheckpointingMode.EXACTLY_ONCE) { + return DeliveryGuarantee.EXACTLY_ONCE; + } else { + throw new IllegalArgumentException( + "Only exactly-once and al-least-once checkpointing mode are supported."); + } + } + /** creates a fullRange() partitionSplit. */ public static PulsarPartitionSplit createPartitionSplit(String topic, int partitionId) { return createPartitionSplit(topic, partitionId, Boundedness.CONTINUOUS_UNBOUNDED); @@ -63,8 +76,6 @@ public static List createPartitionSplits( return splits; } - // -------- InvocationContext Utils - public static boolean isAssignableFromParameterContext( Class requiredType, ParameterContext context) { return requiredType.isAssignableFrom(context.getParameter().getType()); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java index f238a03bfa587..dd7a3fbfbe420 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContext.java @@ -19,37 +19,46 @@ package org.apache.flink.connector.pulsar.testutils; import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; -import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; +import org.apache.flink.connector.testframe.external.ExternalContext; + +import org.apache.pulsar.client.api.Schema; import java.net.URL; -import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; + +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; + +/** + * The implementation for Flink connector test tools. Providing the common test case writing + * constraint for both source, sink and table API. + */ +public abstract class PulsarTestContext implements ExternalContext { -/** Common test context for pulsar based test. */ -public abstract class PulsarTestContext implements DataStreamSourceExternalContext { + private final Set generateTopics = new HashSet<>(); protected final PulsarRuntimeOperator operator; - protected final List connectorJarPaths; + // The schema used for consuming and producing messages between Pulsar and tests. + protected final Schema schema; - protected PulsarTestContext(PulsarTestEnvironment environment, List connectorJarPaths) { + protected PulsarTestContext(PulsarTestEnvironment environment, Schema schema) { this.operator = environment.operator(); - this.connectorJarPaths = connectorJarPaths; + this.schema = schema; } - // Helper methods for generating data. - - protected List generateStringTestData(int splitIndex, long seed) { - int recordNum = 300; - List records = new ArrayList<>(recordNum); - for (int i = 0; i < recordNum; i++) { - records.add(splitIndex + "-" + i); - } + /** Implement this method for providing a more friendly test name in IDE. */ + protected abstract String displayName(); - return records; + /** + * Add the generated topic into the testing context, They would be cleaned after all the cases + * have finished. + */ + protected final void registerTopic(String topic) { + generateTopics.add(topicName(topic)); } - protected abstract String displayName(); - @Override public String toString() { return displayName(); @@ -57,6 +66,14 @@ public String toString() { @Override public List getConnectorJarPaths() { - return connectorJarPaths; + // We don't need any tests jar definition. They are provided in docker related environments. + return Collections.emptyList(); + } + + @Override + public void close() throws Exception { + for (String topic : generateTopics) { + operator.deleteTopic(topic); + } } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java index c634efffc44c1..3c888f59d196d 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestContextFactory.java @@ -18,13 +18,16 @@ package org.apache.flink.connector.pulsar.testutils; +import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext; +import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext; import org.apache.flink.connector.testframe.external.ExternalContextFactory; import java.util.function.Function; /** - * Factory for creating all the test context that extends {@link PulsarTestContext}. Test context - * class should have a constructor with {@link PulsarTestEnvironment} arg. + * Factory for creating all the test context that extends {@link PulsarSourceTestContext} or {@link + * PulsarSinkTestContext}. Test context class should have a constructor with single {@link + * PulsarTestEnvironment} arg. */ public class PulsarTestContextFactory> implements ExternalContextFactory { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java index 0f3fb9e15f9c5..f921e4b92f158 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarTestEnvironment.java @@ -48,7 +48,7 @@ * }

    * *

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

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

    You just need to write a JUnit 5 test class and extends this suite class. All the helper diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java deleted file mode 100644 index 3eca9e7f92636..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicTemplateContext.java +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.testutils.cases; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.pulsar.source.PulsarSource; -import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; -import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter; -import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; -import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; - -import org.apache.pulsar.client.api.RegexSubscriptionMode; -import org.apache.pulsar.client.api.SubscriptionType; - -import java.net.URL; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; -import static org.apache.pulsar.client.api.Schema.STRING; - -/** - * Pulsar external context template that will create multiple topics with only one partitions as - * source splits. - */ -public abstract class MultipleTopicTemplateContext extends PulsarTestContext { - - private int numTopics = 0; - - private final String topicPattern = "pulsar-multiple-topic-[0-9]+-" + randomAlphabetic(8); - - private final Map> topicNameToSplitWriters = - new HashMap<>(); - - public MultipleTopicTemplateContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public MultipleTopicTemplateContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); - } - - @Override - public Source createSource(TestingSourceSettings sourceSettings) { - PulsarSourceBuilder builder = - PulsarSource.builder() - .setDeserializationSchema(pulsarSchema(STRING)) - .setServiceUrl(serviceUrl()) - .setAdminUrl(adminUrl()) - .setTopicPattern(topicPattern, RegexSubscriptionMode.AllTopics) - .setSubscriptionType(subscriptionType()) - .setSubscriptionName(subscriptionName()); - if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { - // Using latest stop cursor for making sure the source could be stopped. - // This is required for SourceTestSuiteBase. - builder.setBoundedStopCursor(StopCursor.latest()); - } - - return builder.build(); - } - - @Override - public ExternalSystemSplitDataWriter createSourceSplitDataWriter( - TestingSourceSettings sourceSettings) { - String topicName = topicPattern.replace("[0-9]+", String.valueOf(numTopics)); - operator.createTopic(topicName, 1); - - String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0); - PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName); - - topicNameToSplitWriters.put(partitionName, writer); - numTopics++; - - return writer; - } - - @Override - public List generateTestData( - TestingSourceSettings sourceSettings, int splitIndex, long seed) { - return generateStringTestData(splitIndex, seed); - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(String.class); - } - - @Override - public void close() throws Exception { - for (ExternalSystemSplitDataWriter writer : topicNameToSplitWriters.values()) { - writer.close(); - } - - topicNameToSplitWriters.clear(); - } - - protected abstract String subscriptionName(); - - protected abstract SubscriptionType subscriptionType(); - - protected String serviceUrl() { - return operator.serviceUrl(); - } - - protected String adminUrl() { - return operator.adminUrl(); - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java deleted file mode 100644 index f5bfa45f32b7a..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/SingleTopicConsumingContext.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.testutils.cases; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.pulsar.source.PulsarSource; -import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; -import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter; -import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; -import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; - -import java.net.URL; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; - -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; -import static org.apache.pulsar.client.api.Schema.STRING; -import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; - -/** - * A Pulsar external context that will create only one topic and use partitions in that topic as - * source splits. - */ -public class SingleTopicConsumingContext extends PulsarTestContext { - - private static final String TOPIC_NAME_PREFIX = "pulsar-single-topic"; - private final String topicName; - private final Map> partitionToSplitWriter = - new HashMap<>(); - - private int numSplits = 0; - - public SingleTopicConsumingContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public SingleTopicConsumingContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); - this.topicName = - TOPIC_NAME_PREFIX + "-" + ThreadLocalRandom.current().nextLong(Long.MAX_VALUE); - } - - @Override - protected String displayName() { - return "consuming message on single topic"; - } - - @Override - public Source createSource(TestingSourceSettings sourceSettings) { - PulsarSourceBuilder builder = - PulsarSource.builder() - .setDeserializationSchema(pulsarSchema(STRING)) - .setServiceUrl(operator.serviceUrl()) - .setAdminUrl(operator.adminUrl()) - .setTopics(topicName) - .setSubscriptionType(Exclusive) - .setSubscriptionName("pulsar-single-topic"); - if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { - // Using latest stop cursor for making sure the source could be stopped. - // This is required for SourceTestSuiteBase. - builder.setBoundedStopCursor(StopCursor.latest()); - } - - return builder.build(); - } - - @Override - public ExternalSystemSplitDataWriter createSourceSplitDataWriter( - TestingSourceSettings sourceSettings) { - if (numSplits == 0) { - // Create the topic first. - operator.createTopic(topicName, 1); - numSplits++; - } else { - numSplits++; - operator.increaseTopicPartitions(topicName, numSplits); - } - - String partitionName = TopicNameUtils.topicNameWithPartition(topicName, numSplits - 1); - PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName); - partitionToSplitWriter.put(numSplits - 1, writer); - - return writer; - } - - @Override - public List generateTestData( - TestingSourceSettings sourceSettings, int splitIndex, long seed) { - return generateStringTestData(splitIndex, seed); - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(String.class); - } - - @Override - public void close() throws Exception { - // Close writer. - for (ExternalSystemSplitDataWriter writer : partitionToSplitWriter.values()) { - writer.close(); - } - - partitionToSplitWriter.clear(); - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java index a78ea992d4b8a..60b5707d9f304 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java @@ -21,6 +21,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.pulsar.common.config.PulsarConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; import org.apache.flink.connector.testframe.external.ExternalContext; @@ -81,6 +82,7 @@ import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.pulsar.client.api.SubscriptionMode.Durable; import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; +import static org.apache.pulsar.common.naming.TopicDomain.persistent; /** * A pulsar cluster operator used for operating pulsar instance. It's serializable for using in @@ -122,6 +124,16 @@ public PulsarRuntimeOperator( this.consumers = new ConcurrentHashMap<>(); } + public boolean topicExists(String topic) { + TopicName topicName = TopicName.get(topic); + String namespace = topicName.getNamespace(); + String parsedTopic = topicName.toString(); + + return sneakyAdmin(() -> admin().topics().getList(namespace, persistent)).stream() + .map(TopicNameUtils::topicName) + .anyMatch(name -> name.equals(parsedTopic)); + } + /** * Create a topic with default {@link #DEFAULT_PARTITIONS} partitions and send a fixed number * {@link #NUM_RECORDS_PER_PARTITION} of records to this topic. @@ -178,10 +190,12 @@ public void setupTopic( */ public void createTopic(String topic, int numberOfPartitions) { checkArgument(numberOfPartitions >= 0); - if (numberOfPartitions <= 0) { - createNonPartitionedTopic(topic); + checkArgument(!topicExists(topic), "Topic %s exists.", topic); + + if (numberOfPartitions == 0) { + sneakyAdmin(() -> admin().topics().createNonPartitionedTopic(topic)); } else { - createPartitionedTopic(topic, numberOfPartitions); + sneakyAdmin(() -> admin().topics().createPartitionedTopic(topic, numberOfPartitions)); } } @@ -195,10 +209,12 @@ public void increaseTopicPartitions(String topic, int newPartitionsNum) { PartitionedTopicMetadata metadata = sneakyAdmin(() -> admin().topics().getPartitionedTopicMetadata(topic)); checkArgument( - metadata.partitions < newPartitionsNum, + metadata.partitions <= newPartitionsNum, "The new partition size which should exceed previous size."); - sneakyAdmin(() -> admin().topics().updatePartitionedTopic(topic, newPartitionsNum)); + if (metadata.partitions < newPartitionsNum) { + sneakyAdmin(() -> admin().topics().updatePartitionedTopic(topic, newPartitionsNum)); + } } /** @@ -361,8 +377,12 @@ public Message receiveMessage(String topic, Schema schema) { public Message receiveMessage(String topic, Schema schema, Duration timeout) { try { Consumer consumer = createConsumer(topic, schema); - Message message = consumer.receiveAsync().get(timeout.toMillis(), MILLISECONDS); - consumer.acknowledgeCumulative(message.getMessageId()); + int millis = Math.toIntExact(timeout.toMillis()); + Message message = consumer.receive(millis, MILLISECONDS); + + if (message != null) { + consumer.acknowledgeCumulative(message.getMessageId()); + } return message; } catch (Exception e) { @@ -487,24 +507,6 @@ public void close() throws IOException { // --------------------------- Private Methods ----------------------------- - private void createNonPartitionedTopic(String topic) { - try { - admin().lookups().lookupTopic(topic); - sneakyAdmin(() -> admin().topics().expireMessagesForAllSubscriptions(topic, 0)); - } catch (PulsarAdminException e) { - sneakyAdmin(() -> admin().topics().createNonPartitionedTopic(topic)); - } - } - - private void createPartitionedTopic(String topic, int numberOfPartitions) { - try { - admin().lookups().lookupPartitionedTopic(topic); - sneakyAdmin(() -> admin().topics().expireMessagesForAllSubscriptions(topic, 0)); - } catch (PulsarAdminException e) { - sneakyAdmin(() -> admin().topics().createPartitionedTopic(topic, numberOfPartitions)); - } - } - @SuppressWarnings("unchecked") private Producer createProducer(String topic, Schema schema) throws PulsarClientException { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java index 06e8cfc67caee..233b01b72962a 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java @@ -134,7 +134,6 @@ public void tearDown() { try { if (operator != null) { operator.close(); - this.operator = null; } container.stop(); started.compareAndSet(true, false); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java index cf080b8f7a638..493fee111058f 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java @@ -53,7 +53,7 @@ import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; import static org.apache.pulsar.common.naming.TopicName.TRANSACTION_COORDINATOR_ASSIGN; -/** Providing a embedded pulsar server. We use this runtime for transaction related tests. */ +/** Providing an embedded pulsar server. We use this runtime for transaction related tests. */ public class PulsarEmbeddedRuntime implements PulsarRuntime { private static final Logger LOG = LoggerFactory.getLogger(PulsarEmbeddedRuntime.class); @@ -95,7 +95,6 @@ public void tearDown() { try { if (operator != null) { operator.close(); - this.operator = null; } if (pulsarService != null) { pulsarService.close(); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarPartitionDataWriter.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarPartitionDataWriter.java similarity index 69% rename from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarPartitionDataWriter.java rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarPartitionDataWriter.java index da48e26cd2589..1ceb2922bc396 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/PulsarPartitionDataWriter.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarPartitionDataWriter.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.connector.pulsar.testutils; +package org.apache.flink.connector.pulsar.testutils.source; import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; @@ -25,20 +25,26 @@ import java.util.List; -/** Source split data writer for writing test data into a Pulsar topic partition. */ -public class PulsarPartitionDataWriter implements ExternalSystemSplitDataWriter { +/** + * Source split data writer for writing test data into a Pulsar topic partition. This writer doesn't + * need to be closed. + */ +public class PulsarPartitionDataWriter implements ExternalSystemSplitDataWriter { private final PulsarRuntimeOperator operator; private final String fullTopicName; + private final Schema schema; - public PulsarPartitionDataWriter(PulsarRuntimeOperator operator, String fullTopicName) { + public PulsarPartitionDataWriter( + PulsarRuntimeOperator operator, String fullTopicName, Schema schema) { this.operator = operator; this.fullTopicName = fullTopicName; + this.schema = schema; } @Override - public void writeRecords(List records) { - operator.sendMessages(fullTopicName, Schema.STRING, records); + public void writeRecords(List records) { + operator.sendMessages(fullTopicName, schema, records); } @Override diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java new file mode 100644 index 0000000000000..8089f8c58e734 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/PulsarSourceTestContext.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.source; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; +import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; +import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; + +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.util.List; +import java.util.Random; +import java.util.stream.IntStream; + +import static java.util.stream.Collectors.toList; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; +import static org.apache.pulsar.client.api.RegexSubscriptionMode.AllTopics; + +/** + * Common source test context for pulsar based test. We use the string text as the basic send + * content. + */ +public abstract class PulsarSourceTestContext extends PulsarTestContext + implements DataStreamSourceExternalContext { + + private static final long DISCOVERY_INTERVAL = 1000L; + private static final int BATCH_DATA_SIZE = 300; + + protected PulsarSourceTestContext(PulsarTestEnvironment environment) { + super(environment, Schema.STRING); + } + + @Override + public Source createSource(TestingSourceSettings sourceSettings) { + PulsarSourceBuilder builder = + PulsarSource.builder() + .setDeserializationSchema(pulsarSchema(schema)) + .setServiceUrl(operator.serviceUrl()) + .setAdminUrl(operator.adminUrl()) + .setTopicPattern(topicPattern(), AllTopics) + .setSubscriptionType(subscriptionType()) + .setSubscriptionName(subscriptionName()) + .setConfig(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, DISCOVERY_INTERVAL); + + // Set extra configuration for source builder. + setSourceBuilder(builder); + + if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { + // Using the latest stop cursor for making sure the source could be stopped. + // This is required for SourceTestSuiteBase. + builder.setBoundedStopCursor(StopCursor.latest()); + } + + return builder.build(); + } + + @Override + public ExternalSystemSplitDataWriter createSourceSplitDataWriter( + TestingSourceSettings sourceSettings) { + String partitionName = generatePartitionName(); + return new PulsarPartitionDataWriter<>(operator, partitionName, schema); + } + + @Override + public List generateTestData( + TestingSourceSettings sourceSettings, int splitIndex, long seed) { + Random random = new Random(seed); + return IntStream.range(0, BATCH_DATA_SIZE) + .boxed() + .map( + index -> { + int length = random.nextInt(20) + 1; + return "split:" + + splitIndex + + "-index:" + + index + + "-content:" + + randomAlphanumeric(length); + }) + .collect(toList()); + } + + @Override + public TypeInformation getProducedType() { + return Types.STRING; + } + + /** Override this method for creating builder. */ + protected void setSourceBuilder(PulsarSourceBuilder builder) { + // Nothing to do by default. + } + + /** + * The topic pattern which is used in Pulsar topic auto discovery. It was discovered every + * {@link #DISCOVERY_INTERVAL} ms; + */ + protected abstract String topicPattern(); + + /** The subscription name used in Pulsar consumer. */ + protected abstract String subscriptionName(); + + /** The subscription type used in Pulsar consumer. */ + protected abstract SubscriptionType subscriptionType(); + + /** + * Dynamic generate a partition related topic in Pulsar. This topic should be pre-created in + * Pulsar. Everytime we call this method, we may get a new partition name. + */ + protected abstract String generatePartitionName(); +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java new file mode 100644 index 0000000000000..d89e153f9f17b --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/MultipleTopicConsumingContext.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.source.cases; + +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext; + +import org.apache.pulsar.client.api.SubscriptionType; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; + +/** + * Pulsar external context that will create multiple topics with only one partitions as source + * splits. + */ +public class MultipleTopicConsumingContext extends PulsarSourceTestContext { + + private final String topicPrefix = "flink-multiple-topic-" + randomAlphabetic(8) + "-"; + + private int index = 0; + + public MultipleTopicConsumingContext(PulsarTestEnvironment environment) { + super(environment); + } + + @Override + protected String displayName() { + return "consume message on multiple topic"; + } + + @Override + protected String topicPattern() { + return topicPrefix + ".+"; + } + + @Override + protected String subscriptionName() { + return "flink-multiple-topic-test"; + } + + @Override + protected SubscriptionType subscriptionType() { + return SubscriptionType.Exclusive; + } + + @Override + protected String generatePartitionName() { + String topic = topicPrefix + index; + operator.createTopic(topic, 1); + registerTopic(topic); + index++; + + return topicNameWithPartition(topic, 0); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java new file mode 100644 index 0000000000000..80e1ff16bff44 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/source/cases/SingleTopicConsumingContext.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.source.cases; + +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.source.PulsarSourceTestContext; + +import org.apache.pulsar.client.api.SubscriptionType; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; + +/** + * A Pulsar external context that will create only one topic and use partitions in that topic as + * source splits. + */ +public class SingleTopicConsumingContext extends PulsarSourceTestContext { + + private final String topicName = "pulsar-single-topic-" + randomAlphanumeric(8); + + private int index = 0; + + public SingleTopicConsumingContext(PulsarTestEnvironment environment) { + super(environment); + registerTopic(topicName); + } + + @Override + protected String displayName() { + return "consume message on single topic"; + } + + @Override + protected String topicPattern() { + return topicName + ".+"; + } + + @Override + protected String subscriptionName() { + return "pulsar-single-topic-test"; + } + + @Override + protected SubscriptionType subscriptionType() { + return SubscriptionType.Exclusive; + } + + @Override + protected String generatePartitionName() { + if (index == 0) { + operator.createTopic(topicName, index + 1); + } else { + operator.increaseTopicPartitions(topicName, index + 1); + } + + return topicNameWithPartition(topicName, index++); + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java index 234c1a01cd623..ea6a982044b8f 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceOrderedE2ECase.java @@ -25,10 +25,10 @@ import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.tests.util.pulsar.cases.ExclusiveSubscriptionContext; -import org.apache.flink.tests.util.pulsar.cases.FailoverSubscriptionContext; import org.apache.flink.tests.util.pulsar.common.FlinkContainerWithPulsarEnvironment; import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment; +import org.apache.flink.tests.util.pulsar.source.ExclusiveSubscriptionContext; +import org.apache.flink.tests.util.pulsar.source.FailoverSubscriptionContext; import org.apache.flink.testutils.junit.FailsOnJava11; import org.junit.experimental.categories.Category; @@ -37,6 +37,7 @@ * Pulsar E2E test based on connector testing framework. It's used for Failover & Exclusive * subscription. */ +@SuppressWarnings("unused") @Category(value = {FailsOnJava11.class}) public class PulsarSourceOrderedE2ECase extends SourceTestSuiteBase { diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java index 50390486dd69c..6bf4fc0c43c7a 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSourceUnorderedE2ECase.java @@ -24,16 +24,21 @@ import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.tests.util.pulsar.cases.KeySharedSubscriptionContext; -import org.apache.flink.tests.util.pulsar.cases.SharedSubscriptionContext; import org.apache.flink.tests.util.pulsar.common.FlinkContainerWithPulsarEnvironment; import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment; import org.apache.flink.tests.util.pulsar.common.UnorderedSourceTestSuiteBase; +import org.apache.flink.tests.util.pulsar.source.KeySharedSubscriptionContext; +import org.apache.flink.tests.util.pulsar.source.SharedSubscriptionContext; +import org.apache.flink.testutils.junit.FailsOnJava11; + +import org.junit.experimental.categories.Category; /** * Pulsar E2E test based on connector testing framework. It's used for Shared & Key_Shared * subscription. */ +@SuppressWarnings("unused") +@Category(value = {FailsOnJava11.class}) public class PulsarSourceUnorderedE2ECase extends UnorderedSourceTestSuiteBase { // Defines the Semantic. @@ -49,12 +54,10 @@ public class PulsarSourceUnorderedE2ECase extends UnorderedSourceTestSuiteBase shared = new PulsarTestContextFactory<>(pulsar, SharedSubscriptionContext::new); - @SuppressWarnings("unused") @TestContext PulsarTestContextFactory keyShared = new PulsarTestContextFactory<>(pulsar, KeySharedSubscriptionContext::new); diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java deleted file mode 100644 index 5ad369bcf03f3..0000000000000 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java +++ /dev/null @@ -1,144 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.tests.util.pulsar.cases; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.pulsar.source.PulsarSource; -import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; -import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FixedRangeGenerator; -import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; -import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; -import org.apache.flink.tests.util.pulsar.common.KeyedPulsarPartitionDataWriter; - -import org.apache.pulsar.client.api.RegexSubscriptionMode; -import org.apache.pulsar.client.api.SubscriptionType; -import org.apache.pulsar.common.util.Murmur3_32Hash; - -import java.net.URL; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static java.util.Collections.singletonList; -import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE; -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; -import static org.apache.pulsar.client.api.Schema.STRING; - -/** We would consume from test splits by using {@link SubscriptionType#Key_Shared} subscription. */ -public class KeySharedSubscriptionContext extends PulsarTestContext { - - private int index = 0; - - private final List writers = new ArrayList<>(); - - // Message keys. - private final String key1; - private final String key2; - - public KeySharedSubscriptionContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public KeySharedSubscriptionContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); - - // Init message keys. - this.key1 = randomAlphabetic(8); - String newKey2; - do { - newKey2 = randomAlphabetic(8); - } while (keyHash(key1) == keyHash(newKey2)); - this.key2 = newKey2; - } - - @Override - protected String displayName() { - return "consuming message by Key_Shared"; - } - - @Override - public Source createSource(TestingSourceSettings sourceSettings) { - int keyHash = keyHash(key1); - TopicRange range = new TopicRange(keyHash, keyHash); - - PulsarSourceBuilder builder = - PulsarSource.builder() - .setDeserializationSchema(pulsarSchema(STRING)) - .setServiceUrl(operator.serviceUrl()) - .setAdminUrl(operator.adminUrl()) - .setTopicPattern( - "pulsar-[0-9]+-key-shared", RegexSubscriptionMode.AllTopics) - .setSubscriptionType(SubscriptionType.Key_Shared) - .setSubscriptionName("pulsar-key-shared") - .setRangeGenerator(new FixedRangeGenerator(singletonList(range))); - if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { - // Using latest stop cursor for making sure the source could be stopped. - builder.setBoundedStopCursor(StopCursor.latest()); - } - - return builder.build(); - } - - @Override - public ExternalSystemSplitDataWriter createSourceSplitDataWriter( - TestingSourceSettings sourceSettings) { - String topicName = "pulsar-" + index + "-key-shared"; - operator.createTopic(topicName, 1); - index++; - - String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0); - KeyedPulsarPartitionDataWriter writer = - new KeyedPulsarPartitionDataWriter(operator, partitionName, key1, key2); - writers.add(writer); - - return writer; - } - - @Override - public List generateTestData( - TestingSourceSettings sourceSettings, int splitIndex, long seed) { - return generateStringTestData(splitIndex, seed); - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(String.class); - } - - @Override - public void close() { - for (KeyedPulsarPartitionDataWriter writer : writers) { - writer.close(); - } - writers.clear(); - } - - private int keyHash(String key) { - return Murmur3_32Hash.getInstance().makeHash(key.getBytes()) % RANGE_SIZE; - } -} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java deleted file mode 100644 index 1a2db6694d1fb..0000000000000 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.tests.util.pulsar.cases; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.pulsar.source.PulsarSource; -import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; -import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter; -import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; -import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; - -import org.apache.pulsar.client.api.RegexSubscriptionMode; -import org.apache.pulsar.client.api.SubscriptionType; - -import java.net.URL; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; -import static org.apache.pulsar.client.api.Schema.STRING; - -/** We would consuming from test splits by using {@link SubscriptionType#Shared} subscription. */ -public class SharedSubscriptionContext extends PulsarTestContext { - - private int index = 0; - - private final List writers = new ArrayList<>(); - - public SharedSubscriptionContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public SharedSubscriptionContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); - } - - @Override - protected String displayName() { - return "consuming message by Shared"; - } - - @Override - public Source createSource(TestingSourceSettings sourceSettings) { - PulsarSourceBuilder builder = - PulsarSource.builder() - .setDeserializationSchema(pulsarSchema(STRING)) - .setServiceUrl(operator.serviceUrl()) - .setAdminUrl(operator.adminUrl()) - .setTopicPattern("pulsar-[0-9]+-shared", RegexSubscriptionMode.AllTopics) - .setSubscriptionType(SubscriptionType.Shared) - .setSubscriptionName("pulsar-shared"); - if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { - // Using latest stop cursor for making sure the source could be stopped. - builder.setBoundedStopCursor(StopCursor.latest()); - } - - return builder.build(); - } - - @Override - public ExternalSystemSplitDataWriter createSourceSplitDataWriter( - TestingSourceSettings sourceSettings) { - String topicName = "pulsar-" + index + "-shared"; - operator.createTopic(topicName, 1); - index++; - - String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0); - PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName); - writers.add(writer); - - return writer; - } - - @Override - public List generateTestData( - TestingSourceSettings sourceSettings, int splitIndex, long seed) { - return generateStringTestData(splitIndex, seed); - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(String.class); - } - - @Override - public void close() { - for (PulsarPartitionDataWriter writer : writers) { - writer.close(); - } - writers.clear(); - } -} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java index 9a1c6dd6f0d6d..14d8cbdd12cec 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/FlinkContainerWithPulsarEnvironment.java @@ -43,7 +43,8 @@ public FlinkContainerWithPulsarEnvironment(int numTaskManagers, int numSlotsPerT resourcePath("bcutil-jdk15on.jar"), resourcePath("bcprov-ext-jdk15on.jar"), resourcePath("jaxb-api.jar"), - resourcePath("jul-to-slf4j.jar")); + resourcePath("jul-to-slf4j.jar"), + resourcePath("flink-connector-testing.jar")); } private static String resourcePath(String jarName) { diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java index e431e4c89d0e1..bdcf8160a968d 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/KeyedPulsarPartitionDataWriter.java @@ -35,23 +35,26 @@ public class KeyedPulsarPartitionDataWriter implements ExternalSystemSplitDataWr private final PulsarRuntimeOperator operator; private final String fullTopicName; - private final String key1; - private final String key2; + private final String keyToRead; + private final String keyToExclude; public KeyedPulsarPartitionDataWriter( - PulsarRuntimeOperator operator, String fullTopicName, String key1, String key2) { + PulsarRuntimeOperator operator, + String fullTopicName, + String keyToRead, + String keyToExclude) { this.operator = operator; this.fullTopicName = fullTopicName; - this.key1 = key1; - this.key2 = key2; + this.keyToRead = keyToRead; + this.keyToExclude = keyToExclude; } @Override public void writeRecords(List records) { - operator.sendMessages(fullTopicName, Schema.STRING, key1, records); + List newRecords = records.stream().map(a -> a + keyToRead).collect(toList()); + operator.sendMessages(fullTopicName, Schema.STRING, keyToExclude, newRecords); - List newRecords = records.stream().map(a -> a + key1).collect(toList()); - operator.sendMessages(fullTopicName, Schema.STRING, key2, newRecords); + operator.sendMessages(fullTopicName, Schema.STRING, keyToRead, records); } @Override diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java index 01527ea54824d..59b05f0955c54 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/common/UnorderedSourceTestSuiteBase.java @@ -18,69 +18,67 @@ package org.apache.flink.tests.util.pulsar.common; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; +import org.apache.flink.connector.testframe.environment.ClusterControllable; import org.apache.flink.connector.testframe.environment.TestEnvironment; -import org.apache.flink.connector.testframe.environment.TestEnvironmentSettings; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; -import org.apache.flink.connector.testframe.junit.extensions.ConnectorTestingExtension; -import org.apache.flink.connector.testframe.junit.extensions.TestCaseInvocationContextProvider; +import org.apache.flink.connector.testframe.testsuites.SourceTestSuiteBase; import org.apache.flink.streaming.api.CheckpointingMode; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.util.TestLoggerExtension; -import org.junit.jupiter.api.DisplayName; -import org.junit.jupiter.api.TestInstance; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.Disabled; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; +/** A source test template for testing the messages which could be consumed in an unordered way. */ +public abstract class UnorderedSourceTestSuiteBase extends SourceTestSuiteBase { -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsInAnyOrder; + private static final String DISABLE_REASON = + "UnorderedSourceTestSuiteBase don't support any test in SourceTestSuiteBase."; -/** A source test template for testing the messages which could be consumed in a unordered way. */ -@ExtendWith({ - ConnectorTestingExtension.class, - TestLoggerExtension.class, - TestCaseInvocationContextProvider.class -}) -@TestInstance(TestInstance.Lifecycle.PER_CLASS) -public abstract class UnorderedSourceTestSuiteBase { + @Override + @Disabled(DISABLE_REASON) + public void testMultipleSplits( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} - @TestTemplate - @DisplayName("Test source with one split and four consumers") - public void testOneSplitWithMultipleConsumers( - TestEnvironment testEnv, DataStreamSourceExternalContext externalContext) - throws Exception { - TestingSourceSettings sourceSettings = - TestingSourceSettings.builder() - .setBoundedness(Boundedness.BOUNDED) - .setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE) - .build(); - TestEnvironmentSettings envOptions = - TestEnvironmentSettings.builder() - .setConnectorJarPaths(externalContext.getConnectorJarPaths()) - .build(); - List testData = - externalContext.generateTestData( - sourceSettings, 0, ThreadLocalRandom.current().nextLong()); - ExternalSystemSplitDataWriter writer = - externalContext.createSourceSplitDataWriter(sourceSettings); - writer.writeRecords(testData); + @Override + @Disabled(DISABLE_REASON) + public void testSavepoint( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} - Source source = externalContext.createSource(sourceSettings); - StreamExecutionEnvironment execEnv = testEnv.createExecutionEnvironment(envOptions); - List results = - execEnv.fromSource(source, WatermarkStrategy.noWatermarks(), "Pulsar source") - .setParallelism(4) - .executeAndCollect( - "Source single split with four readers.", testData.size()); + @Override + @Disabled(DISABLE_REASON) + public void testScaleUp( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} - assertThat(results, containsInAnyOrder(testData.toArray())); - } + @Override + @Disabled(DISABLE_REASON) + public void testScaleDown( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} + + @Override + @Disabled(DISABLE_REASON) + public void testSourceMetrics( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} + + @Override + @Disabled(DISABLE_REASON) + public void testIdleReader( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + CheckpointingMode semantic) {} + + @Override + @Disabled(DISABLE_REASON) + public void testTaskManagerFailure( + TestEnvironment testEnv, + DataStreamSourceExternalContext externalContext, + ClusterControllable controller, + CheckpointingMode semantic) {} } diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/ExclusiveSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/ExclusiveSubscriptionContext.java similarity index 71% rename from flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/ExclusiveSubscriptionContext.java rename to flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/ExclusiveSubscriptionContext.java index 6fea0c9c9e52d..4906ad6cc7e03 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/ExclusiveSubscriptionContext.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/ExclusiveSubscriptionContext.java @@ -16,32 +16,23 @@ * limitations under the License. */ -package org.apache.flink.tests.util.pulsar.cases; +package org.apache.flink.tests.util.pulsar.source; import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.pulsar.testutils.cases.MultipleTopicTemplateContext; +import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; import org.apache.pulsar.client.api.SubscriptionType; -import java.net.URL; -import java.util.Collections; -import java.util.List; - /** We would consume from test splits by using {@link SubscriptionType#Exclusive} subscription. */ -public class ExclusiveSubscriptionContext extends MultipleTopicTemplateContext { +public class ExclusiveSubscriptionContext extends MultipleTopicConsumingContext { public ExclusiveSubscriptionContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public ExclusiveSubscriptionContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); + super(environment); } @Override protected String displayName() { - return "consuming message by Exclusive"; + return "consume message by Exclusive"; } @Override diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/FailoverSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/FailoverSubscriptionContext.java similarity index 71% rename from flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/FailoverSubscriptionContext.java rename to flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/FailoverSubscriptionContext.java index c47348861c11d..3134db4bdda57 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/FailoverSubscriptionContext.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/FailoverSubscriptionContext.java @@ -16,32 +16,23 @@ * limitations under the License. */ -package org.apache.flink.tests.util.pulsar.cases; +package org.apache.flink.tests.util.pulsar.source; import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.pulsar.testutils.cases.MultipleTopicTemplateContext; +import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; import org.apache.pulsar.client.api.SubscriptionType; -import java.net.URL; -import java.util.Collections; -import java.util.List; - /** We would consume from test splits by using {@link SubscriptionType#Failover} subscription. */ -public class FailoverSubscriptionContext extends MultipleTopicTemplateContext { +public class FailoverSubscriptionContext extends MultipleTopicConsumingContext { public FailoverSubscriptionContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public FailoverSubscriptionContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); + super(environment); } @Override protected String displayName() { - return "consuming message by Failover"; + return "consume message by Failover"; } @Override diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java new file mode 100644 index 0000000000000..6d579b5eccfd2 --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/KeySharedSubscriptionContext.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.pulsar.source; + +import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; +import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FixedRangeGenerator; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; +import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; +import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; +import org.apache.flink.tests.util.pulsar.common.KeyedPulsarPartitionDataWriter; + +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.util.Murmur3_32Hash; + +import static java.util.Collections.singletonList; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE; + +/** We would consume from test splits by using {@link SubscriptionType#Key_Shared} subscription. */ +public class KeySharedSubscriptionContext extends MultipleTopicConsumingContext { + + private final String keyToRead; + private final String keyToExclude; + + public KeySharedSubscriptionContext(PulsarTestEnvironment environment) { + super(environment); + + this.keyToRead = randomAlphabetic(8); + + // Make sure they have different hash code. + int readHash = keyHash(keyToRead); + String randomKey; + do { + randomKey = randomAlphabetic(8); + } while (keyHash(randomKey) == readHash); + this.keyToExclude = randomKey; + } + + @Override + public ExternalSystemSplitDataWriter createSourceSplitDataWriter( + TestingSourceSettings sourceSettings) { + String partitionName = generatePartitionName(); + return new KeyedPulsarPartitionDataWriter(operator, partitionName, keyToRead, keyToExclude); + } + + @Override + protected String displayName() { + return "consume message by Key_Shared"; + } + + @Override + protected void setSourceBuilder(PulsarSourceBuilder builder) { + int keyHash = keyHash(keyToRead); + TopicRange range = new TopicRange(keyHash, keyHash); + + builder.setRangeGenerator(new FixedRangeGenerator(singletonList(range))); + } + + @Override + protected String subscriptionName() { + return "pulsar-key-shared-subscription"; + } + + @Override + protected SubscriptionType subscriptionType() { + return SubscriptionType.Key_Shared; + } + + // This method is copied from Pulsar for calculating message key hash. + private int keyHash(String key) { + return Murmur3_32Hash.getInstance().makeHash(key.getBytes()) % RANGE_SIZE; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java similarity index 59% rename from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java rename to flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java index 57027f33e1b5d..3e0730b8f59d3 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/cases/MultipleTopicConsumingContext.java +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/source/SharedSubscriptionContext.java @@ -16,43 +16,32 @@ * limitations under the License. */ -package org.apache.flink.connector.pulsar.testutils.cases; +package org.apache.flink.tests.util.pulsar.source; import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; import org.apache.pulsar.client.api.SubscriptionType; -import java.net.URL; -import java.util.Collections; -import java.util.List; +/** We would consume from test splits by using {@link SubscriptionType#Shared} subscription. */ +public class SharedSubscriptionContext extends MultipleTopicConsumingContext { -/** - * Pulsar external context that will create multiple topics with only one partitions as source - * splits. - */ -public class MultipleTopicConsumingContext extends MultipleTopicTemplateContext { - - public MultipleTopicConsumingContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public MultipleTopicConsumingContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); + public SharedSubscriptionContext(PulsarTestEnvironment environment) { + super(environment); } @Override protected String displayName() { - return "consuming message on multiple topic"; + return "consume message by Shared"; } @Override protected String subscriptionName() { - return "flink-pulsar-multiple-topic-test"; + return "pulsar-shared-subscription"; } @Override protected SubscriptionType subscriptionType() { - return SubscriptionType.Exclusive; + return SubscriptionType.Shared; } } diff --git a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java index 7be0e8d2d88d9..0628e632f19b3 100644 --- a/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java +++ b/flink-test-utils-parent/flink-connector-test-utils/src/main/java/org/apache/flink/connector/testframe/source/enumerator/NoOpEnumStateSerializer.java @@ -22,7 +22,7 @@ import java.io.IOException; -/** Mock enumerator state seializer. */ +/** Mock enumerator state serializer. */ public class NoOpEnumStateSerializer implements SimpleVersionedSerializer { @Override public int getVersion() { From f67888d5c9b9afce7a09ce6c3bee279f2006230d Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 31 Mar 2022 01:55:53 +0800 Subject: [PATCH 218/258] [FLINK-26182][Connector/pulsar] Create a e2e tests for Pulsar sink connector based on connector testing framework. --- .../f4d91193-72ba-4ce4-ad83-98f780dce581 | 12 ++ .../pulsar/sink/PulsarSinkITCase.java | 141 ++++++++++++------ .../sink/PulsarPartitionDataReader.java | 93 ++++++++++++ .../testutils/sink/PulsarSinkTestContext.java | 126 ++++++++++++++++ .../sink/PulsarSinkTestSuiteBase.java | 37 +++++ .../flink-end-to-end-tests-pulsar/pom.xml | 22 ++- .../tests/util/pulsar/PulsarSinkE2ECase.java | 57 +++++++ 7 files changed, 438 insertions(+), 50 deletions(-) create mode 100644 flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 b/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 new file mode 100644 index 0000000000000..7a71d25fc8140 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/archunit-violations/f4d91193-72ba-4ce4-ad83-98f780dce581 @@ -0,0 +1,12 @@ +org.apache.flink.connector.pulsar.sink.PulsarSinkITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule +org.apache.flink.connector.pulsar.source.PulsarSourceITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java index 94e23d7b0977e..1b6fb2e817ca2 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java @@ -20,14 +20,26 @@ import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; import org.apache.flink.connector.pulsar.testutils.function.ControlSource; +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; +import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext; +import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestSuiteBase; +import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; +import org.apache.flink.connector.testframe.junit.annotations.TestContext; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.testutils.junit.SharedObjectsExtension; +import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -43,52 +55,87 @@ /** Tests for using PulsarSink writing to a Pulsar cluster. */ class PulsarSinkITCase extends PulsarTestSuiteBase { - private static final int PARALLELISM = 1; - - @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = - new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - // Using this extension for creating shared reference which would be used in source function. - @RegisterExtension final SharedObjectsExtension sharedObjects = SharedObjectsExtension.create(); - - @ParameterizedTest - @EnumSource(DeliveryGuarantee.class) - void writeRecordsToPulsar(DeliveryGuarantee guarantee) throws Exception { - // A random topic with partition 1. - String topic = randomAlphabetic(8); - operator().createTopic(topic, 4); - int counts = ThreadLocalRandom.current().nextInt(100, 200); - - ControlSource source = - new ControlSource( - sharedObjects, operator(), topic, guarantee, counts, Duration.ofMinutes(5)); - PulsarSink sink = - PulsarSink.builder() - .setServiceUrl(operator().serviceUrl()) - .setAdminUrl(operator().adminUrl()) - .setDeliveryGuarantee(guarantee) - .setTopics(topic) - .setSerializationSchema(flinkSchema(new SimpleStringSchema())) - .build(); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); - env.enableCheckpointing(100L); - env.addSource(source).sinkTo(sink); - env.execute(); - - List expectedRecords = source.getExpectedRecords(); - List consumedRecords = source.getConsumedRecords(); - - assertThat(consumedRecords) - .hasSameSizeAs(expectedRecords) - .containsExactlyInAnyOrderElementsOf(expectedRecords); + /** Integration test based on connector testing framework. */ + @Nested + class IntegrationTest extends PulsarSinkTestSuiteBase { + + @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); + + @TestExternalSystem + PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.embedded()); + + @TestSemantics + CheckpointingMode[] semantics = + new CheckpointingMode[] { + CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE + }; + + @TestContext + PulsarTestContextFactory sinkContext = + new PulsarTestContextFactory<>(pulsar, PulsarSinkTestContext::new); + } + + /** Tests for using PulsarSink writing to a Pulsar cluster. */ + @Nested + class DeliveryGuaranteeTest extends PulsarTestSuiteBase { + + private static final int PARALLELISM = 1; + + @RegisterExtension + private final MiniClusterExtension clusterExtension = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build()); + + // Using this extension for creating shared reference which would be used in source + // function. + @RegisterExtension + final SharedObjectsExtension sharedObjects = SharedObjectsExtension.create(); + + @ParameterizedTest + @EnumSource(DeliveryGuarantee.class) + void writeRecordsToPulsar(DeliveryGuarantee guarantee) throws Exception { + // A random topic with partition 4. + String topic = randomAlphabetic(8); + operator().createTopic(topic, 4); + int counts = ThreadLocalRandom.current().nextInt(100, 200); + + ControlSource source = + new ControlSource( + sharedObjects, + operator(), + topic, + guarantee, + counts, + Duration.ofMillis(50)); + PulsarSink sink = + PulsarSink.builder() + .setServiceUrl(operator().serviceUrl()) + .setAdminUrl(operator().adminUrl()) + .setDeliveryGuarantee(guarantee) + .setTopics(topic) + .setSerializationSchema(flinkSchema(new SimpleStringSchema())) + .build(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.setParallelism(PARALLELISM); + if (guarantee != DeliveryGuarantee.NONE) { + env.enableCheckpointing(500L); + } + env.addSource(source).sinkTo(sink); + env.execute(); + + List expectedRecords = source.getExpectedRecords(); + List consumedRecords = source.getConsumedRecords(); + + assertThat(consumedRecords) + .hasSameSizeAs(expectedRecords) + .containsExactlyInAnyOrderElementsOf(expectedRecords); + } } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java new file mode 100644 index 0000000000000..6ef9510d11683 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarPartitionDataReader.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.sink; + +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; +import org.apache.flink.connector.testframe.external.ExternalSystemDataReader; + +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.api.SubscriptionMode; +import org.apache.pulsar.client.api.SubscriptionType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; + +/** The data reader for a specified topic partition from Pulsar. */ +public class PulsarPartitionDataReader implements ExternalSystemDataReader, Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(PulsarPartitionDataReader.class); + + private final Consumer consumer; + + public PulsarPartitionDataReader( + PulsarRuntimeOperator operator, String fullTopicName, Schema schema) + throws PulsarClientException { + // Create client for supporting the use in E2E test. + String subscriptionName = randomAlphanumeric(12); + this.consumer = + operator.client() + .newConsumer(schema) + .topic(fullTopicName) + .subscriptionName(subscriptionName) + .subscriptionType(SubscriptionType.Exclusive) + .subscriptionMode(SubscriptionMode.Durable) + .subscriptionInitialPosition(SubscriptionInitialPosition.Earliest) + .subscribe(); + } + + @Override + public List poll(Duration timeout) { + List results = new ArrayList<>(); + + while (true) { + try { + int millis = Math.toIntExact(timeout.toMillis()); + Message message = consumer.receive(millis, MILLISECONDS); + + if (message != null) { + consumer.acknowledgeCumulative(message); + results.add(message.getValue()); + } else { + break; + } + } catch (Exception e) { + LOG.error("", e); + break; + } + } + + return results; + } + + @Override + public void close() throws PulsarClientException { + consumer.close(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java new file mode 100644 index 0000000000000..af00b16ddf0b1 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestContext.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.sink; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.connector.base.DeliveryGuarantee; +import org.apache.flink.connector.pulsar.sink.PulsarSink; +import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.testframe.external.ExternalSystemDataReader; +import org.apache.flink.connector.testframe.external.sink.DataStreamSinkV2ExternalContext; +import org.apache.flink.connector.testframe.external.sink.TestingSinkSettings; + +import org.apache.flink.shaded.guava30.com.google.common.io.Closer; + +import org.apache.pulsar.client.api.Schema; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES; +import static org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema.pulsarSchema; +import static org.apache.flink.connector.pulsar.testutils.PulsarTestCommonUtils.toDeliveryGuarantee; + +/** Common sink test context for pulsar based test. */ +public class PulsarSinkTestContext extends PulsarTestContext + implements DataStreamSinkV2ExternalContext { + + private static final String TOPIC_NAME_PREFIX = "flink-sink-topic-"; + private static final int RECORD_SIZE_UPPER_BOUND = 300; + private static final int RECORD_SIZE_LOWER_BOUND = 100; + private static final int RECORD_STRING_SIZE = 20; + + private String topicName = topicName(); + private final Closer closer = Closer.create(); + + public PulsarSinkTestContext(PulsarTestEnvironment environment) { + super(environment, Schema.STRING); + } + + @Override + protected String displayName() { + return "write messages into one topic in Pulsar"; + } + + @Override + public Sink createSink(TestingSinkSettings sinkSettings) { + if (!operator.topicExists(topicName)) { + operator.createTopic(topicName, 4); + } + DeliveryGuarantee guarantee = toDeliveryGuarantee(sinkSettings.getCheckpointingMode()); + + return PulsarSink.builder() + .setServiceUrl(operator.serviceUrl()) + .setAdminUrl(operator.adminUrl()) + .setTopics(topicName) + .setDeliveryGuarantee(guarantee) + .setSerializationSchema(pulsarSchema(Schema.STRING)) + .enableSchemaEvolution() + .setConfig(PULSAR_BATCHING_MAX_MESSAGES, 4) + .build(); + } + + @Override + public ExternalSystemDataReader createSinkDataReader(TestingSinkSettings sinkSettings) { + PulsarPartitionDataReader reader = + sneakyClient( + () -> new PulsarPartitionDataReader<>(operator, topicName, Schema.STRING)); + closer.register(reader); + + return reader; + } + + @Override + public List generateTestData(TestingSinkSettings sinkSettings, long seed) { + Random random = new Random(seed); + int recordSize = + random.nextInt(RECORD_SIZE_UPPER_BOUND - RECORD_SIZE_LOWER_BOUND) + + RECORD_SIZE_LOWER_BOUND; + List records = new ArrayList<>(recordSize); + for (int i = 0; i < recordSize; i++) { + int size = random.nextInt(RECORD_STRING_SIZE) + RECORD_STRING_SIZE; + String record = "index:" + i + "-data:" + randomAlphanumeric(size); + records.add(record); + } + + return records; + } + + @Override + public TypeInformation getProducedType() { + return Types.STRING; + } + + @Override + public void close() throws Exception { + // Change the topic name after finishing a test case. + closer.register(() -> topicName = topicName()); + closer.close(); + } + + private String topicName() { + return TOPIC_NAME_PREFIX + randomAlphanumeric(8); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java new file mode 100644 index 0000000000000..0695a43569b68 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/sink/PulsarSinkTestSuiteBase.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.sink; + +import org.apache.flink.connector.testframe.environment.TestEnvironment; +import org.apache.flink.connector.testframe.external.sink.DataStreamSinkExternalContext; +import org.apache.flink.connector.testframe.testsuites.SinkTestSuiteBase; +import org.apache.flink.streaming.api.CheckpointingMode; + +import org.junit.jupiter.api.Disabled; + +/** Pulsar sink don't expose the monitor metrics now. We have to disable this test. */ +public abstract class PulsarSinkTestSuiteBase extends SinkTestSuiteBase { + + @Override + @Disabled("Enable this test after FLINK-26027 being merged.") + public void testMetrics( + TestEnvironment testEnv, + DataStreamSinkExternalContext externalContext, + CheckpointingMode semantic) {} +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml index f76011f4c6b9f..7a27f6090b44d 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml @@ -53,6 +53,17 @@ under the License. jaxb-api ${jaxb.api.version} + + org.apache.flink + flink-connector-test-utils + ${project.version} + + + com.google.guava + guava + + + org.apache.flink flink-connector-pulsar @@ -70,9 +81,6 @@ under the License. org.apache.maven.plugins maven-surefire-plugin - - true - org.apache.maven.plugins @@ -176,6 +184,14 @@ under the License. jar ${project.build.directory}/dependencies + + org.apache.flink + flink-connector-test-utils + ${project.version} + flink-connector-testing.jar + jar + ${project.build.directory}/dependencies + diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java new file mode 100644 index 0000000000000..a19f5931b587a --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/PulsarSinkE2ECase.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.pulsar; + +import org.apache.flink.connector.pulsar.testutils.PulsarTestContextFactory; +import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestContext; +import org.apache.flink.connector.pulsar.testutils.sink.PulsarSinkTestSuiteBase; +import org.apache.flink.connector.testframe.junit.annotations.TestContext; +import org.apache.flink.connector.testframe.junit.annotations.TestEnv; +import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; +import org.apache.flink.connector.testframe.junit.annotations.TestSemantics; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.tests.util.pulsar.common.FlinkContainerWithPulsarEnvironment; +import org.apache.flink.tests.util.pulsar.common.PulsarContainerTestEnvironment; +import org.apache.flink.testutils.junit.FailsOnJava11; + +import org.junit.experimental.categories.Category; + +/** Pulsar sink E2E test based on connector testing framework. */ +@SuppressWarnings("unused") +@Category(value = {FailsOnJava11.class}) +public class PulsarSinkE2ECase extends PulsarSinkTestSuiteBase { + + @TestSemantics + CheckpointingMode[] semantics = + new CheckpointingMode[] { + CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE + }; + + // Defines TestEnvironment + @TestEnv + FlinkContainerWithPulsarEnvironment flink = new FlinkContainerWithPulsarEnvironment(1, 6); + + // Defines ConnectorExternalSystem. + @TestExternalSystem + PulsarContainerTestEnvironment pulsar = new PulsarContainerTestEnvironment(flink); + + @TestContext + PulsarTestContextFactory sinkContext = + new PulsarTestContextFactory<>(pulsar, PulsarSinkTestContext::new); +} From 4a555984140b62b5238c6e8e1554fdfcd49fdee0 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Fri, 29 Apr 2022 03:58:55 +0800 Subject: [PATCH 219/258] [FLINK-27231][Connector/Pulsar] Bump pulsar to 2.10.0 --- .../pulsar_client_configuration.html | 2 +- .../pulsar_consumer_configuration.html | 6 - .../pulsar_producer_configuration.html | 12 - .../generated/pulsar_sink_configuration.html | 6 + .../flink-connector-pulsar/pom.xml | 41 +- .../pulsar/common/config/PulsarOptions.java | 2 +- .../pulsar/sink/PulsarSinkOptions.java | 27 +- .../sink/config/PulsarSinkConfigUtils.java | 5 + .../pulsar/sink/config/SinkConfiguration.java | 4 +- .../sink/writer/router/MessageKeyHash.java | 4 +- .../source/config/SourceConfiguration.java | 2 +- .../cursor/start/MessageIdStartCursor.java | 2 +- .../pulsar/table/PulsarTableOptionUtils.java | 2 +- .../pulsar/sink/PulsarSinkITCase.java | 2 +- .../pulsar/source/PulsarSourceITCase.java | 2 +- .../table/PulsarTableOptionUtilsTest.java | 6 +- .../testutils/runtime/PulsarRuntime.java | 21 +- .../testutils/runtime/PulsarRuntimeUtils.java | 125 ++ .../container/PulsarContainerRuntime.java | 5 +- .../embedded/PulsarEmbeddedRuntime.java | 272 ----- .../runtime/mock/BlankBrokerInterceptor.java | 61 + .../mock/MockBookKeeperClientFactory.java | 74 ++ .../runtime/mock/MockPulsarService.java | 57 + .../mock/NonClosableMockBookKeeper.java | 55 + .../runtime/mock/PulsarMockRuntime.java | 112 ++ .../mock/SameThreadOrderedSafeExecutor.java | 56 + .../resources/containers/txnStandalone.conf | 1042 ----------------- .../src/test/resources/docker/bootstrap.sh | 28 + .../src/main/resources/META-INF/NOTICE | 10 +- .../flink-end-to-end-tests-pulsar/pom.xml | 2 +- .../cases/KeySharedSubscriptionContext.java | 145 +++ .../cases/SharedSubscriptionContext.java | 117 ++ .../flink/util/DockerImageVersions.java | 2 +- 33 files changed, 922 insertions(+), 1387 deletions(-) create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeUtils.java delete mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/BlankBrokerInterceptor.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockBookKeeperClientFactory.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockPulsarService.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/NonClosableMockBookKeeper.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/SameThreadOrderedSafeExecutor.java delete mode 100644 flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf create mode 100755 flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java create mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java diff --git a/docs/layouts/shortcodes/generated/pulsar_client_configuration.html b/docs/layouts/shortcodes/generated/pulsar_client_configuration.html index 02a6d96867e1a..970b847ac2ab9 100644 --- a/docs/layouts/shortcodes/generated/pulsar_client_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_client_configuration.html @@ -100,7 +100,7 @@

    pulsar.client.memoryLimitBytes
    - 0 + 67108864 Long The limit (in bytes) on the amount of direct memory that will be allocated by this client instance.
    Note: at this moment this is only limiting the memory for producers. Setting this to 0 will disable the limit. diff --git a/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html b/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html index bc8b6df40608b..4e05b270de4b0 100644 --- a/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_consumer_configuration.html @@ -140,12 +140,6 @@ Boolean If enabled, the consumer will automatically retry messages. - -
    pulsar.consumer.subscriptionInitialPosition
    - Latest -

    Enum

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

    Possible values:
    • "Latest"
    • "Earliest"
    -
    pulsar.consumer.subscriptionMode
    Durable diff --git a/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html b/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html index 22d0de1f5fd3b..94739029031a6 100644 --- a/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_producer_configuration.html @@ -62,18 +62,6 @@ Long The sequence id for avoiding the duplication, it's used when Pulsar doesn't have transaction. - -
    pulsar.producer.maxPendingMessages
    - 1000 - Integer - The maximum size of a queue holding pending messages.
    For example, a message waiting to receive an acknowledgment from a https://pulsar.apache.org/docs/en/reference-terminology#broker.
    By default, when the queue is full, all calls to the Send and SendAsync methods fail unless you set BlockIfQueueFull to true. - - -
    pulsar.producer.maxPendingMessagesAcrossPartitions
    - 50000 - Integer - The maximum number of pending messages across partitions.
    Use the setting to lower the max pending messages for each partition (setMaxPendingMessages) if the total number exceeds the configured value. -
    pulsar.producer.producerCryptoFailureAction
    FAIL diff --git a/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html b/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html index cd7f803534a1e..4a3009ae70263 100644 --- a/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html @@ -20,6 +20,12 @@ Boolean If you enable this option and use PulsarSerializationSchema.pulsarSchema(), we would consume and deserialize the message by using Pulsar's Schema. + +
    pulsar.sink.maxPendingMessages
    + 1000 + Integer + The maximum number of pending messages in on sink parallelism. +
    pulsar.sink.maxRecommitTimes
    5 diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index 10633c13836f1..77ad83e12263d 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -36,14 +36,14 @@ under the License. jar - 2.9.1 + 2.10.0 0.6.1 + 1.7.0 3.11 - 3.6.3 - 4.1.72.Final - 1.33.0 + 4.1.74.Final + 1.42.1 @@ -201,8 +201,28 @@ under the License. test
    - + + + org.apache.pulsar + testmocks + ${pulsar.version} + test + + + org.testng + testng + + + org.powermock + powermock-module-testng + + + org.apache.zookeeper + zookeeper + + + org.apache.pulsar pulsar-broker @@ -219,15 +239,6 @@ under the License. test - - - - org.apache.zookeeper - zookeeper - ${pulsar-zookeeper.version} - test - - @@ -312,7 +323,7 @@ under the License. kr.motd.maven os-maven-plugin - 1.7.0 + ${os-maven-plugin.version} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarOptions.java index b06d9ed7c8abd..f878cb4efe099 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarOptions.java @@ -436,7 +436,7 @@ private PulsarOptions() { public static final ConfigOption PULSAR_MEMORY_LIMIT_BYTES = ConfigOptions.key(CLIENT_CONFIG_PREFIX + "memoryLimitBytes") .longType() - .defaultValue(0L) + .defaultValue(64 * 1024 * 1024L) .withDescription( Description.builder() .text( diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java index c9fe82d42aab0..c31097bc441c9 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java @@ -39,12 +39,11 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.flink.configuration.description.LinkElement.link; import static org.apache.flink.configuration.description.TextElement.code; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_MEMORY_LIMIT_BYTES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PRODUCER_CONFIG_PREFIX; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.SINK_CONFIG_PREFIX; import static org.apache.flink.connector.pulsar.sink.writer.router.MessageKeyHash.MURMUR3_32_HASH; import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_BATCHING_MAX_MESSAGES; -import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_MAX_PENDING_MESSAGES; -import static org.apache.pulsar.client.impl.conf.ProducerConfigurationData.DEFAULT_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; /** * Configurations for PulsarSink. All the options list here could be configured in {@link @@ -130,6 +129,16 @@ private PulsarSinkOptions() { "The allowed transaction recommit times if we meet some retryable exception." + " This is used in Pulsar Transaction."); + public static final ConfigOption PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM = + ConfigOptions.key(SINK_CONFIG_PREFIX + "maxPendingMessages") + .intType() + .defaultValue(1000) + .withDescription( + Description.builder() + .text( + "The maximum number of pending messages in on sink parallelism.") + .build()); + /////////////////////////////////////////////////////////////////////////////// // // The configuration for ProducerConfigurationData part. @@ -157,10 +166,12 @@ private PulsarSinkOptions() { code("sendTimeout")) .build()); + /** @deprecated Use {@link PulsarOptions#PULSAR_MEMORY_LIMIT_BYTES} since Pulsar 2.10.0 */ + @Deprecated public static final ConfigOption PULSAR_MAX_PENDING_MESSAGES = ConfigOptions.key(PRODUCER_CONFIG_PREFIX + "maxPendingMessages") .intType() - .defaultValue(DEFAULT_MAX_PENDING_MESSAGES) + .noDefaultValue() .withDescription( Description.builder() .text("The maximum size of a queue holding pending messages.") @@ -176,12 +187,17 @@ private PulsarSinkOptions() { code("Send"), code("SendAsync"), code("BlockIfQueueFull")) + .text( + "Since Pulsar 2.10.0, you shouldn't set this option, use %s instead.", + code(PULSAR_MEMORY_LIMIT_BYTES.key())) .build()); + /** @deprecated Use {@link PulsarOptions#PULSAR_MEMORY_LIMIT_BYTES} since Pulsar 2.10.0 */ + @Deprecated public static final ConfigOption PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS = ConfigOptions.key(PRODUCER_CONFIG_PREFIX + "maxPendingMessagesAcrossPartitions") .intType() - .defaultValue(DEFAULT_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS) + .noDefaultValue() .withDescription( Description.builder() .text( @@ -190,6 +206,9 @@ private PulsarSinkOptions() { .text( "Use the setting to lower the max pending messages for each partition (%s) if the total number exceeds the configured value.", code("setMaxPendingMessages")) + .text( + "Since Pulsar 2.10.0, you shouldn't set this option, use %s instead.", + code(PULSAR_MEMORY_LIMIT_BYTES.key())) .build()); public static final ConfigOption PULSAR_BATCHING_MAX_PUBLISH_DELAY_MICROS = diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java index cb9ba2efd7b67..77155bbea6b74 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java @@ -33,6 +33,7 @@ import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAMS; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_AUTH_PARAM_MAP; +import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_MEMORY_LIMIT_BYTES; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_SERVICE_URL; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_ENABLED; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_MAX_BYTES; @@ -65,6 +66,10 @@ private PulsarSinkConfigUtils() { .requiredOption(PULSAR_SERVICE_URL) .requiredOption(PULSAR_ADMIN_URL) .conflictOptions(PULSAR_AUTH_PARAMS, PULSAR_AUTH_PARAM_MAP) + .conflictOptions(PULSAR_MEMORY_LIMIT_BYTES, PULSAR_MAX_PENDING_MESSAGES) + .conflictOptions( + PULSAR_MEMORY_LIMIT_BYTES, + PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS) .build(); /** Create a pulsar producer builder by using the given Configuration. */ diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java index fe1204ee8791f..768b730f40257 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java @@ -32,7 +32,7 @@ import java.util.Objects; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES; -import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_RECOMMIT_TIMES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MESSAGE_KEY_HASH; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL; @@ -63,7 +63,7 @@ public SinkConfiguration(Configuration configuration) { this.partitionSwitchSize = getInteger(PULSAR_BATCHING_MAX_MESSAGES); this.messageKeyHash = get(PULSAR_MESSAGE_KEY_HASH); this.enableSchemaEvolution = get(PULSAR_WRITE_SCHEMA_EVOLUTION); - this.maxPendingMessages = get(PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS); + this.maxPendingMessages = get(PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM); this.maxRecommitTimes = get(PULSAR_MAX_RECOMMIT_TIMES); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/MessageKeyHash.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/MessageKeyHash.java index 7f35760f2531d..bbac99e4b961d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/MessageKeyHash.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/router/MessageKeyHash.java @@ -25,7 +25,7 @@ import org.apache.pulsar.client.impl.Hash; import org.apache.pulsar.client.impl.JavaStringHash; -import org.apache.pulsar.client.impl.Murmur3_32Hash; +import org.apache.pulsar.client.impl.Murmur3Hash32; import static org.apache.flink.configuration.description.LinkElement.link; import static org.apache.flink.configuration.description.TextElement.code; @@ -57,7 +57,7 @@ public Hash getHash() { link("https://en.wikipedia.org/wiki/MurmurHash", "Murmur3"))) { @Override public Hash getHash() { - return Murmur3_32Hash.getInstance(); + return Murmur3Hash32.getInstance(); } }; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java index 4cc5f462861bb..845cfdceb2baf 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java @@ -48,7 +48,7 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_VERIFY_INITIAL_OFFSETS; -/** The configure class for pulsar source. */ +/** The configuration class for pulsar source. */ @PublicEvolving public class SourceConfiguration extends PulsarConfiguration { private static final long serialVersionUID = 8488507275800787580L; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java index 0b2390658d01d..f4156d4e7acfc 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -31,7 +31,7 @@ import static org.apache.flink.util.Preconditions.checkState; -/** This cursor would left pulsar start consuming from a specific message id. */ +/** This cursor would leave pulsar start consuming from a specific message id. */ public class MessageIdStartCursor implements StartCursor { private static final long serialVersionUID = -8057345435887170111L; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java index c44879cd3a950..6aa7544f4c4b1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java @@ -234,7 +234,7 @@ protected static StartCursor parseMessageIdString(String config) { } protected static StartCursor parsePublishTimeStartCursor(Long config) { - return StartCursor.fromMessageTime(config); + return StartCursor.fromPublishTime(config); } // -------------------------------------------------------------------------------------------- diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java index 1b6fb2e817ca2..63113731c4bab 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/PulsarSinkITCase.java @@ -62,7 +62,7 @@ class IntegrationTest extends PulsarSinkTestSuiteBase { @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); @TestExternalSystem - PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.embedded()); + PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.mock()); @TestSemantics CheckpointingMode[] semantics = diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java index 50742837c5539..497297830767d 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java @@ -64,7 +64,7 @@ public void testScaleDown( // Defines pulsar running environment @TestExternalSystem - PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.embedded()); + PulsarTestEnvironment pulsar = new PulsarTestEnvironment(PulsarRuntime.mock()); @TestSemantics CheckpointingMode[] semantics = new CheckpointingMode[] {CheckpointingMode.EXACTLY_ONCE}; diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java index 08e250b7a86db..5506c236e30a4 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java @@ -23,7 +23,7 @@ import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.PublishTimestampStartCursor; import org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; @@ -179,7 +179,7 @@ void startCursor() { options.remove(SOURCE_START_FROM_MESSAGE_ID.key()); options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545"); startCursor = getStartCursor(Configuration.fromMap(options)); - assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); + assertThat(startCursor).isInstanceOf(PublishTimestampStartCursor.class); options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545L"); assertThatExceptionOfType(IllegalArgumentException.class) @@ -273,7 +273,7 @@ void publishTimeStartCursor() { final Map options = createDefaultOptions(); options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345"); StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); - assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); + assertThat(startCursor).isInstanceOf(PublishTimestampStartCursor.class); options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345L"); assertThatExceptionOfType(IllegalArgumentException.class) diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java index 49cf95f3e9206..613ad270e8733 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntime.java @@ -20,7 +20,7 @@ import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; import org.apache.flink.connector.pulsar.testutils.runtime.container.PulsarContainerRuntime; -import org.apache.flink.connector.pulsar.testutils.runtime.embedded.PulsarEmbeddedRuntime; +import org.apache.flink.connector.pulsar.testutils.runtime.mock.PulsarMockRuntime; import org.testcontainers.containers.GenericContainer; @@ -44,22 +44,15 @@ public interface PulsarRuntime { */ PulsarRuntimeOperator operator(); - /** - * Create a standalone Pulsar instance in test thread. We would start a embedded zookeeper and - * bookkeeper. The stream storage for bookkeeper is disabled. The function worker is disabled on - * Pulsar broker. - * - *

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

    We would link the created Pulsar docker instance with the given flink instance. This would diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeUtils.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeUtils.java new file mode 100644 index 0000000000000..b064cfa2cd7a1 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeUtils.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime; + +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.common.naming.TopicName; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.TenantInfo; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; + +import java.util.Collections; +import java.util.List; + +import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import static org.apache.pulsar.common.naming.TopicName.TRANSACTION_COORDINATOR_ASSIGN; + +/** This class is used to create the basic topics for a standalone Pulsar instance. */ +public final class PulsarRuntimeUtils { + + private PulsarRuntimeUtils() { + // No public constructor + } + + /** Create the system topics. */ + public static void initializePulsarEnvironment( + ServiceConfiguration config, String serviceUrl, String adminUrl) + throws PulsarAdminException, PulsarClientException { + try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl).build()) { + ClusterData clusterData = + ClusterData.builder().serviceUrl(adminUrl).brokerServiceUrl(serviceUrl).build(); + String cluster = config.getClusterName(); + createSampleNameSpace(admin, clusterData, cluster); + + // Create default namespace + createNameSpace( + admin, + cluster, + TopicName.PUBLIC_TENANT, + TopicName.PUBLIC_TENANT + "/" + TopicName.DEFAULT_NAMESPACE); + + // Create Pulsar system namespace + createNameSpace( + admin, cluster, SYSTEM_NAMESPACE.getTenant(), SYSTEM_NAMESPACE.toString()); + + // Enable transaction + if (config.isTransactionCoordinatorEnabled() + && !admin.namespaces() + .getTopics(SYSTEM_NAMESPACE.toString()) + .contains(TRANSACTION_COORDINATOR_ASSIGN.getPartition(0).toString())) { + admin.topics().createPartitionedTopic(TRANSACTION_COORDINATOR_ASSIGN.toString(), 1); + } + } + } + + private static void createSampleNameSpace( + PulsarAdmin admin, ClusterData clusterData, String cluster) + throws PulsarAdminException { + // Create a sample namespace + String tenant = "sample"; + String globalCluster = "global"; + String namespace = tenant + "/ns1"; + + List clusters = admin.clusters().getClusters(); + if (!clusters.contains(cluster)) { + admin.clusters().createCluster(cluster, clusterData); + } else { + admin.clusters().updateCluster(cluster, clusterData); + } + // Create marker for "global" cluster + if (!clusters.contains(globalCluster)) { + admin.clusters().createCluster(globalCluster, ClusterData.builder().build()); + } + + if (!admin.tenants().getTenants().contains(tenant)) { + admin.tenants() + .createTenant( + tenant, + new TenantInfoImpl( + Collections.emptySet(), Collections.singleton(cluster))); + } + + if (!admin.namespaces().getNamespaces(tenant).contains(namespace)) { + admin.namespaces().createNamespace(namespace); + } + } + + private static void createNameSpace( + PulsarAdmin admin, String cluster, String publicTenant, String defaultNamespace) + throws PulsarAdminException { + if (!admin.tenants().getTenants().contains(publicTenant)) { + admin.tenants() + .createTenant( + publicTenant, + TenantInfo.builder() + .adminRoles(Collections.emptySet()) + .allowedClusters(Collections.singleton(cluster)) + .build()); + } + if (!admin.namespaces().getNamespaces(publicTenant).contains(defaultNamespace)) { + admin.namespaces().createNamespace(defaultNamespace); + admin.namespaces() + .setNamespaceReplicationClusters( + defaultNamespace, Collections.singleton(cluster)); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java index 233b01b72962a..9a1aed78b3c76 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java @@ -102,13 +102,16 @@ public void startUp() { // Override the default configuration in container for enabling the Pulsar transaction. container.withClasspathResourceMapping( - configFile, "/pulsar/conf/standalone.conf", BindMode.READ_ONLY); + "docker/bootstrap.sh", "/pulsar/bin/bootstrap.sh", BindMode.READ_ONLY); // Waiting for the Pulsar border is ready. container.waitingFor( forHttp("/admin/v2/namespaces/public/default") .forPort(BROKER_HTTP_PORT) .forStatusCode(200) .withStartupTimeout(Duration.ofMinutes(5))); + // Set custom startup script. + container.withCommand("/pulsar/bin/bootstrap.sh"); + // Start the Pulsar Container. container.start(); // Append the output to this runtime logger. Used for local debug purpose. diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java deleted file mode 100644 index 493fee111058f..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/embedded/PulsarEmbeddedRuntime.java +++ /dev/null @@ -1,272 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.connector.pulsar.testutils.runtime.embedded; - -import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; -import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; -import org.apache.flink.util.FileUtils; - -import org.apache.bookkeeper.conf.ServerConfiguration; -import org.apache.pulsar.broker.PulsarService; -import org.apache.pulsar.broker.ServiceConfiguration; -import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.common.configuration.PulsarConfigurationLoader; -import org.apache.pulsar.common.naming.TopicName; -import org.apache.pulsar.common.policies.data.ClusterData; -import org.apache.pulsar.common.policies.data.TenantInfo; -import org.apache.pulsar.common.policies.data.TenantInfoImpl; -import org.apache.pulsar.zookeeper.LocalBookkeeperEnsemble; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Collections; -import java.util.List; -import java.util.Optional; - -import static org.apache.flink.util.Preconditions.checkNotNull; -import static org.apache.pulsar.broker.ServiceConfigurationUtils.brokerUrl; -import static org.apache.pulsar.broker.ServiceConfigurationUtils.webServiceUrl; -import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; -import static org.apache.pulsar.common.naming.TopicName.TRANSACTION_COORDINATOR_ASSIGN; - -/** Providing an embedded pulsar server. We use this runtime for transaction related tests. */ -public class PulsarEmbeddedRuntime implements PulsarRuntime { - private static final Logger LOG = LoggerFactory.getLogger(PulsarEmbeddedRuntime.class); - - private static final String CONFIG_FILE_PATH; - - static { - // Find the absolute path for containers/txnStandalone.conf - ClassLoader classLoader = PulsarEmbeddedRuntime.class.getClassLoader(); - URL resource = classLoader.getResource("containers/txnStandalone.conf"); - File file = new File(checkNotNull(resource).getFile()); - CONFIG_FILE_PATH = file.getAbsolutePath(); - } - - private final Path tempDir; - - private LocalBookkeeperEnsemble bookkeeper; - private PulsarService pulsarService; - private PulsarRuntimeOperator operator; - - public PulsarEmbeddedRuntime() { - this.tempDir = createTempDir(); - } - - @Override - public void startUp() { - try { - startBookkeeper(); - startPulsarService(); - - // Create the operator. - this.operator = new PulsarRuntimeOperator(getBrokerUrl(), getWebServiceUrl()); - } catch (Exception e) { - throw new IllegalStateException(e); - } - } - - @Override - public void tearDown() { - try { - if (operator != null) { - operator.close(); - } - if (pulsarService != null) { - pulsarService.close(); - } - if (bookkeeper != null) { - bookkeeper.stop(); - } - } catch (Exception e) { - throw new IllegalStateException(e); - } finally { - removeTempDir(tempDir); - } - } - - @Override - public PulsarRuntimeOperator operator() { - return checkNotNull(operator, "You should start this embedded Pulsar first."); - } - - private Path createTempDir() { - try { - return Files.createTempDirectory("pulsar"); - } catch (IOException e) { - throw new IllegalStateException(e); - } - } - - private void removeTempDir(Path tempDir) { - try { - FileUtils.deleteDirectory(tempDir.normalize().toFile()); - } catch (IOException e) { - throw new IllegalStateException(e); - } - } - - public void startBookkeeper() throws Exception { - Path zkPath = Paths.get("data", "standalone", "zookeeper"); - Path bkPath = Paths.get("data", "standalone", "bookkeeper"); - - String zkDir = tempDir.resolve(zkPath).normalize().toString(); - String bkDir = tempDir.resolve(bkPath).normalize().toString(); - - ServerConfiguration bkServerConf = new ServerConfiguration(); - bkServerConf.loadConf(new File(CONFIG_FILE_PATH).toURI().toURL()); - this.bookkeeper = new LocalBookkeeperEnsemble(1, 0, 0, zkDir, bkDir, true, "127.0.0.1"); - - // Start Bookkeeper & zookeeper. - bookkeeper.startStandalone(bkServerConf, false); - } - - private void startPulsarService() throws Exception { - ServiceConfiguration config; - try (FileInputStream inputStream = new FileInputStream(CONFIG_FILE_PATH)) { - config = PulsarConfigurationLoader.create(inputStream, ServiceConfiguration.class); - } catch (IOException e) { - throw new IllegalStateException(e); - } - - // Use runtime dynamic ports for broker. - config.setAdvertisedAddress("127.0.0.1"); - config.setClusterName("standalone"); - - // Use random port. - config.setBrokerServicePort(Optional.of(0)); - config.setWebServicePort(Optional.of(0)); - - // Select available port for bookkeeper and zookeeper. - int zkPort = getZkPort(); - String zkConnect = "127.0.0.1" + ":" + zkPort; - config.setZookeeperServers(zkConnect); - config.setConfigurationStoreServers(zkConnect); - config.setRunningStandalone(true); - - this.pulsarService = new PulsarService(config); - - // Start Pulsar Broker. - pulsarService.start(); - - // Create sample data environment. - String webServiceUrl = getWebServiceUrl(); - String brokerUrl = getBrokerUrl(); - try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(webServiceUrl).build()) { - ClusterData clusterData = - ClusterData.builder() - .serviceUrl(webServiceUrl) - .brokerServiceUrl(brokerUrl) - .build(); - String cluster = config.getClusterName(); - createSampleNameSpace(admin, clusterData, cluster); - - // Create default namespace - createNameSpace( - admin, - cluster, - TopicName.PUBLIC_TENANT, - TopicName.PUBLIC_TENANT + "/" + TopicName.DEFAULT_NAMESPACE); - - // Create Pulsar system namespace - createNameSpace( - admin, cluster, SYSTEM_NAMESPACE.getTenant(), SYSTEM_NAMESPACE.toString()); - // Enable transaction - if (config.isTransactionCoordinatorEnabled() - && !admin.namespaces() - .getTopics(SYSTEM_NAMESPACE.toString()) - .contains(TRANSACTION_COORDINATOR_ASSIGN.getPartition(0).toString())) { - admin.topics().createPartitionedTopic(TRANSACTION_COORDINATOR_ASSIGN.toString(), 1); - } - } - } - - private int getZkPort() { - return checkNotNull(bookkeeper).getZookeeperPort(); - } - - private String getBrokerUrl() { - Integer port = pulsarService.getBrokerListenPort().orElseThrow(IllegalStateException::new); - return brokerUrl("127.0.0.1", port); - } - - private String getWebServiceUrl() { - Integer port = pulsarService.getListenPortHTTP().orElseThrow(IllegalArgumentException::new); - return webServiceUrl("127.0.0.1", port); - } - - private void createSampleNameSpace(PulsarAdmin admin, ClusterData clusterData, String cluster) - throws PulsarAdminException { - // Create a sample namespace - String tenant = "sample"; - String globalCluster = "global"; - String namespace = tenant + "/ns1"; - - List clusters = admin.clusters().getClusters(); - if (!clusters.contains(cluster)) { - admin.clusters().createCluster(cluster, clusterData); - } else { - admin.clusters().updateCluster(cluster, clusterData); - } - // Create marker for "global" cluster - if (!clusters.contains(globalCluster)) { - admin.clusters().createCluster(globalCluster, ClusterData.builder().build()); - } - - if (!admin.tenants().getTenants().contains(tenant)) { - admin.tenants() - .createTenant( - tenant, - new TenantInfoImpl( - Collections.emptySet(), Collections.singleton(cluster))); - } - - if (!admin.namespaces().getNamespaces(tenant).contains(namespace)) { - admin.namespaces().createNamespace(namespace); - } - } - - private void createNameSpace( - PulsarAdmin admin, String cluster, String publicTenant, String defaultNamespace) - throws PulsarAdminException { - if (!admin.tenants().getTenants().contains(publicTenant)) { - admin.tenants() - .createTenant( - publicTenant, - TenantInfo.builder() - .adminRoles(Collections.emptySet()) - .allowedClusters(Collections.singleton(cluster)) - .build()); - } - if (!admin.namespaces().getNamespaces(publicTenant).contains(defaultNamespace)) { - admin.namespaces().createNamespace(defaultNamespace); - admin.namespaces() - .setNamespaceReplicationClusters( - defaultNamespace, Collections.singleton(cluster)); - } - } -} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/BlankBrokerInterceptor.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/BlankBrokerInterceptor.java new file mode 100644 index 0000000000000..8355a23279f09 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/BlankBrokerInterceptor.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.intercept.BrokerInterceptor; +import org.apache.pulsar.broker.service.ServerCnx; +import org.apache.pulsar.common.api.proto.BaseCommand; + +import javax.servlet.ServletRequest; +import javax.servlet.ServletResponse; + +/** No operation for this BrokerInterceptor implementation. */ +public class BlankBrokerInterceptor implements BrokerInterceptor { + + @Override + public void onPulsarCommand(BaseCommand command, ServerCnx cnx) { + // no-op + } + + @Override + public void onConnectionClosed(ServerCnx cnx) { + // no-op + } + + @Override + public void onWebserviceRequest(ServletRequest request) { + // no-op + } + + @Override + public void onWebserviceResponse(ServletRequest request, ServletResponse response) { + // no-op + } + + @Override + public void initialize(PulsarService pulsarService) { + // no-op + } + + @Override + public void close() { + // no-op + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockBookKeeperClientFactory.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockBookKeeperClientFactory.java new file mode 100644 index 0000000000000..d35bd669ab045 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockBookKeeperClientFactory.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import io.netty.channel.EventLoopGroup; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.EnsemblePlacementPolicy; +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.stats.StatsLogger; +import org.apache.pulsar.broker.BookKeeperClientFactory; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended; + +import java.io.IOException; +import java.util.Map; +import java.util.Optional; + +/** A BookKeeperClientFactory implementation which returns a mocked bookkeeper. */ +public class MockBookKeeperClientFactory implements BookKeeperClientFactory { + + private final OrderedExecutor executor = + OrderedExecutor.newBuilder().numThreads(1).name("mock-pulsar-bookkeeper").build(); + + private final BookKeeper bookKeeper = NonClosableMockBookKeeper.create(executor); + + @Override + public BookKeeper create( + ServiceConfiguration conf, + MetadataStoreExtended store, + EventLoopGroup eventLoopGroup, + Optional> ensemblePlacementPolicyClass, + Map ensemblePlacementPolicyProperties) + throws IOException { + return bookKeeper; + } + + @Override + public BookKeeper create( + ServiceConfiguration conf, + MetadataStoreExtended store, + EventLoopGroup eventLoopGroup, + Optional> ensemblePlacementPolicyClass, + Map ensemblePlacementPolicyProperties, + StatsLogger statsLogger) + throws IOException { + return bookKeeper; + } + + @Override + public void close() { + try { + bookKeeper.close(); + executor.shutdown(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockPulsarService.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockPulsarService.java new file mode 100644 index 0000000000000..09015a7db8660 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/MockPulsarService.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.pulsar.broker.BookKeeperClientFactory; +import org.apache.pulsar.broker.PulsarService; +import org.apache.pulsar.broker.ServiceConfiguration; +import org.apache.pulsar.broker.intercept.BrokerInterceptor; +import org.apache.pulsar.broker.namespace.NamespaceService; + +import java.util.function.Supplier; + +/** A Mock pulsar service which would use the mocked zookeeper and bookkeeper. */ +public class MockPulsarService extends PulsarService { + + private final SameThreadOrderedSafeExecutor orderedExecutor = + new SameThreadOrderedSafeExecutor(); + + public MockPulsarService(ServiceConfiguration config) { + super(config); + } + + public BookKeeperClientFactory newBookKeeperClientFactory() { + return new MockBookKeeperClientFactory(); + } + + public Supplier getNamespaceServiceProvider() { + return () -> new NamespaceService(this); + } + + @Override + public OrderedExecutor getOrderedExecutor() { + return orderedExecutor; + } + + @Override + public BrokerInterceptor getBrokerInterceptor() { + return new BlankBrokerInterceptor(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/NonClosableMockBookKeeper.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/NonClosableMockBookKeeper.java new file mode 100644 index 0000000000000..b7001b8ca82f4 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/NonClosableMockBookKeeper.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.bookkeeper.client.PulsarMockBookKeeper; +import org.apache.bookkeeper.common.util.OrderedExecutor; + +/** + * Prevent the MockBookKeeper instance from being closed when the broker is restarted within a test. + */ +public class NonClosableMockBookKeeper extends PulsarMockBookKeeper { + + private NonClosableMockBookKeeper(OrderedExecutor executor) throws Exception { + super(executor); + } + + @Override + public void close() { + // no-op + } + + @Override + public void shutdown() { + // no-op + } + + public void reallyShutdown() { + super.shutdown(); + } + + public static BookKeeper create(OrderedExecutor executor) { + try { + return new NonClosableMockBookKeeper(executor); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java new file mode 100644 index 0000000000000..a86ff5283f558 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/PulsarMockRuntime.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntime; +import org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeOperator; + +import org.apache.pulsar.broker.ServiceConfiguration; + +import java.util.Optional; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.testutils.runtime.PulsarRuntimeUtils.initializePulsarEnvironment; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Providing a mocked pulsar server. */ +public class PulsarMockRuntime implements PulsarRuntime { + + private static final String CLUSTER_NAME = "mock-pulsar-" + randomAlphanumeric(6); + private final ServiceConfiguration configuration; + private final MockPulsarService pulsarService; + private PulsarRuntimeOperator operator; + + public PulsarMockRuntime() { + this(createConfig()); + } + + public PulsarMockRuntime(ServiceConfiguration configuration) { + this.configuration = configuration; + this.pulsarService = new MockPulsarService(configuration); + } + + @Override + public void startUp() { + try { + pulsarService.start(); + + String serviceUrl = pulsarService.getBrokerServiceUrl(); + String adminUrl = pulsarService.getWebServiceAddress(); + initializePulsarEnvironment(configuration, serviceUrl, adminUrl); + + this.operator = new PulsarRuntimeOperator(serviceUrl, adminUrl); + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + + @Override + public void tearDown() { + try { + pulsarService.close(); + operator.close(); + this.operator = null; + } catch (Exception e) { + throw new IllegalStateException(e); + } + } + + @Override + public PulsarRuntimeOperator operator() { + return checkNotNull(operator, "You should start this mock pulsar first."); + } + + private static ServiceConfiguration createConfig() { + ServiceConfiguration configuration = new ServiceConfiguration(); + + configuration.setAdvertisedAddress("localhost"); + configuration.setClusterName(CLUSTER_NAME); + + configuration.setManagedLedgerCacheSizeMB(8); + configuration.setActiveConsumerFailoverDelayTimeMillis(0); + configuration.setDefaultRetentionTimeInMinutes(7); + configuration.setDefaultNumberOfNamespaceBundles(1); + configuration.setMetadataStoreUrl("memory:local"); + configuration.setConfigurationMetadataStoreUrl("memory:local"); + + configuration.setAuthenticationEnabled(false); + configuration.setAuthorizationEnabled(false); + configuration.setAllowAutoTopicCreation(true); + configuration.setBrokerDeleteInactiveTopicsEnabled(false); + + configuration.setWebSocketServiceEnabled(false); + // Use runtime dynamic ports + configuration.setBrokerServicePort(Optional.of(0)); + configuration.setWebServicePort(Optional.of(0)); + + // Enable transactions. + configuration.setSystemTopicEnabled(true); + configuration.setBrokerDeduplicationEnabled(true); + configuration.setTransactionCoordinatorEnabled(true); + configuration.setTransactionMetadataStoreProviderClassName( + "org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider"); + + return configuration; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/SameThreadOrderedSafeExecutor.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/SameThreadOrderedSafeExecutor.java new file mode 100644 index 0000000000000..9667f08414f70 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/mock/SameThreadOrderedSafeExecutor.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.testutils.runtime.mock; + +import org.apache.bookkeeper.common.util.OrderedExecutor; +import org.apache.bookkeeper.common.util.SafeRunnable; +import org.apache.bookkeeper.stats.NullStatsLogger; +import org.apache.pulsar.shade.io.netty.util.concurrent.DefaultThreadFactory; + +/** Override the default bookkeeper executor for executing in one thread executor. */ +public class SameThreadOrderedSafeExecutor extends OrderedExecutor { + + public SameThreadOrderedSafeExecutor() { + super( + "same-thread-executor", + 1, + new DefaultThreadFactory("test"), + NullStatsLogger.INSTANCE, + false, + false, + 100000, + -1, + false); + } + + @Override + public void execute(Runnable r) { + r.run(); + } + + @Override + public void executeOrdered(int orderingKey, SafeRunnable r) { + r.run(); + } + + @Override + public void executeOrdered(long orderingKey, SafeRunnable r) { + r.run(); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf b/flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf deleted file mode 100644 index bf35c59860b23..0000000000000 --- a/flink-connectors/flink-connector-pulsar/src/test/resources/containers/txnStandalone.conf +++ /dev/null @@ -1,1042 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one -# or more contributor license agreements. See the NOTICE file -# distributed with this work for additional information -# regarding copyright ownership. The ASF licenses this file -# to you under the Apache License, Version 2.0 (the -# "License"); you may not use this file except in compliance -# with the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, -# software distributed under the License is distributed on an -# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -# KIND, either express or implied. See the License for the -# specific language governing permissions and limitations -# under the License. -# - -### --- General broker settings --- ### - -# Zookeeper quorum connection string -zookeeperServers= - -# Configuration Store connection string -configurationStoreServers= - -brokerServicePort=6650 - -# Port to use to server HTTP request -webServicePort=8080 - -# Hostname or IP address the service binds on, default is 0.0.0.0. -bindAddress=0.0.0.0 - -# Extra bind addresses for the service: :://:,[...] -bindAddresses= - -# Hostname or IP address the service advertises to the outside world. If not set, the value of InetAddress.getLocalHost().getHostName() is used. -advertisedAddress= - -# Enable or disable the HAProxy protocol. -haProxyProtocolEnabled=false - -# Number of threads to use for Netty IO. Default is set to 2 * Runtime.getRuntime().availableProcessors() -numIOThreads= - -# Number of threads to use for ordered executor. The ordered executor is used to operate with zookeeper, -# such as init zookeeper client, get namespace policies from zookeeper etc. It also used to split bundle. Default is 8 -numOrderedExecutorThreads=8 - -# Number of threads to use for HTTP requests processing. Default is set to 2 * Runtime.getRuntime().availableProcessors() -numHttpServerThreads= - -# Number of thread pool size to use for pulsar broker service. -# The executor in thread pool will do basic broker operation like load/unload bundle, update managedLedgerConfig, -# update topic/subscription/replicator message dispatch rate, do leader election etc. -# Default is Runtime.getRuntime().availableProcessors() -numExecutorThreadPoolSize= - -# Number of thread pool size to use for pulsar zookeeper callback service -# The cache executor thread pool is used for restarting global zookeeper session. -# Default is 10 -numCacheExecutorThreadPoolSize=10 - -# Max concurrent web requests -maxConcurrentHttpRequests=1024 - -# Name of the cluster to which this broker belongs to -clusterName=standalone - -# Enable cluster's failure-domain which can distribute brokers into logical region -failureDomainsEnabled=false - -# Zookeeper session timeout in milliseconds -zooKeeperSessionTimeoutMillis=30000 - -# ZooKeeper operation timeout in seconds -zooKeeperOperationTimeoutSeconds=30 - -# ZooKeeper cache expiry time in seconds -zooKeeperCacheExpirySeconds=300 - -# Time to wait for broker graceful shutdown. After this time elapses, the process will be killed -brokerShutdownTimeoutMs=60000 - -# Flag to skip broker shutdown when broker handles Out of memory error -skipBrokerShutdownOnOOM=false - -# Enable backlog quota check. Enforces action on topic when the quota is reached -backlogQuotaCheckEnabled=true - -# How often to check for topics that have reached the quota -backlogQuotaCheckIntervalInSeconds=60 - -# Default per-topic backlog quota limit -backlogQuotaDefaultLimitGB=10 - -# Default per-topic backlog quota time limit in second, less than 0 means no limitation. default is -1. -backlogQuotaDefaultLimitSecond=-1 - -# Default ttl for namespaces if ttl is not already configured at namespace policies. (disable default-ttl with value 0) -ttlDurationDefaultInSeconds=0 - -# Enable the deletion of inactive topics. This parameter need to cooperate with the allowAutoTopicCreation parameter. -# If brokerDeleteInactiveTopicsEnabled is set to true, we should ensure that allowAutoTopicCreation is also set to true. -brokerDeleteInactiveTopicsEnabled=true - -# How often to check for inactive topics -brokerDeleteInactiveTopicsFrequencySeconds=60 - -# Allow you to delete a tenant forcefully. -forceDeleteTenantAllowed=false - -# Allow you to delete a namespace forcefully. -forceDeleteNamespaceAllowed=false - -# Max pending publish requests per connection to avoid keeping large number of pending -# requests in memory. Default: 1000 -maxPendingPublishRequestsPerConnection=1000 - -# How frequently to proactively check and purge expired messages -messageExpiryCheckIntervalInMinutes=5 - -# Check between intervals to see if max message size in topic policies has been updated. -# Default is 60s -maxMessageSizeCheckIntervalInSeconds=60 - -# How long to delay rewinding cursor and dispatching messages when active consumer is changed -activeConsumerFailoverDelayTimeMillis=1000 - -# How long to delete inactive subscriptions from last consuming -# When it is 0, inactive subscriptions are not deleted automatically -subscriptionExpirationTimeMinutes=0 - -# Enable subscription message redelivery tracker to send redelivery count to consumer (default is enabled) -subscriptionRedeliveryTrackerEnabled=true - -# On KeyShared subscriptions, with default AUTO_SPLIT mode, use splitting ranges or -# consistent hashing to reassign keys to new consumers -subscriptionKeySharedUseConsistentHashing=false - -# On KeyShared subscriptions, number of points in the consistent-hashing ring. -# The higher the number, the more equal the assignment of keys to consumers -subscriptionKeySharedConsistentHashingReplicaPoints=100 - -# How frequently to proactively check and purge expired subscription -subscriptionExpiryCheckIntervalInMinutes=5 - -# Set the default behavior for message deduplication in the broker -# This can be overridden per-namespace. If enabled, broker will reject -# messages that were already stored in the topic -brokerDeduplicationEnabled=false - -# Maximum number of producer information that it's going to be -# persisted for deduplication purposes -brokerDeduplicationMaxNumberOfProducers=10000 - -# Number of entries after which a dedup info snapshot is taken. -# A bigger interval will lead to less snapshots being taken though it would -# increase the topic recovery time, when the entries published after the -# snapshot need to be replayed -brokerDeduplicationEntriesInterval=1000 - -# Time of inactivity after which the broker will discard the deduplication information -# relative to a disconnected producer. Default is 6 hours. -brokerDeduplicationProducerInactivityTimeoutMinutes=360 - -# When a namespace is created without specifying the number of bundle, this -# value will be used as the default -defaultNumberOfNamespaceBundles=4 - -# Max number of topics allowed to be created in the namespace. When the topics reach the max topics of the namespace, -# the broker should reject the new topic request(include topic auto-created by the producer or consumer) -# until the number of connected consumers decrease. -# Using a value of 0, is disabling maxTopicsPerNamespace-limit check. -maxTopicsPerNamespace=0 - -# Allow schema to be auto updated at broker level. User can override this by -# 'is_allow_auto_update_schema' of namespace policy. -isAllowAutoUpdateSchemaEnabled=true - -# Enable check for minimum allowed client library version -clientLibraryVersionCheckEnabled=false - -# Path for the file used to determine the rotation status for the broker when responding -# to service discovery health checks -statusFilePath=/usr/local/apache/htdocs - -# Max number of unacknowledged messages allowed to receive messages by a consumer on a shared subscription. Broker will stop sending -# messages to consumer once, this limit reaches until consumer starts acknowledging messages back -# Using a value of 0, is disabling unackeMessage limit check and consumer can receive messages without any restriction -maxUnackedMessagesPerConsumer=50000 - -# Max number of unacknowledged messages allowed per shared subscription. Broker will stop dispatching messages to -# all consumers of the subscription once this limit reaches until consumer starts acknowledging messages back and -# unack count reaches to limit/2. Using a value of 0, is disabling unackedMessage-limit -# check and dispatcher can dispatch messages without any restriction -maxUnackedMessagesPerSubscription=200000 - -# Max number of unacknowledged messages allowed per broker. Once this limit reaches, broker will stop dispatching -# messages to all shared subscription which has higher number of unack messages until subscriptions start -# acknowledging messages back and unack count reaches to limit/2. Using a value of 0, is disabling -# unackedMessage-limit check and broker doesn't block dispatchers -maxUnackedMessagesPerBroker=0 - -# Once broker reaches maxUnackedMessagesPerBroker limit, it blocks subscriptions which has higher unacked messages -# than this percentage limit and subscription will not receive any new messages until that subscription acks back -# limit/2 messages -maxUnackedMessagesPerSubscriptionOnBrokerBlocked=0.16 - -# Tick time to schedule task that checks topic publish rate limiting across all topics -# Reducing to lower value can give more accuracy while throttling publish but -# it uses more CPU to perform frequent check. (Disable publish throttling with value 0) -topicPublisherThrottlingTickTimeMillis=2 - -# Enable precise rate limit for topic publish -preciseTopicPublishRateLimiterEnable=false - -# Tick time to schedule task that checks broker publish rate limiting across all topics -# Reducing to lower value can give more accuracy while throttling publish but -# it uses more CPU to perform frequent check. (Disable publish throttling with value 0) -brokerPublisherThrottlingTickTimeMillis=50 - -# Max Rate(in 1 seconds) of Message allowed to publish for a broker if broker publish rate limiting enabled -# (Disable message rate limit with value 0) -brokerPublisherThrottlingMaxMessageRate=0 - -# Max Rate(in 1 seconds) of Byte allowed to publish for a broker if broker publish rate limiting enabled -# (Disable byte rate limit with value 0) -brokerPublisherThrottlingMaxByteRate=0 - -# Default messages per second dispatch throttling-limit for every topic. Using a value of 0, is disabling default -# message dispatch-throttling -dispatchThrottlingRatePerTopicInMsg=0 - -# Default bytes per second dispatch throttling-limit for every topic. Using a value of 0, is disabling -# default message-byte dispatch-throttling -dispatchThrottlingRatePerTopicInByte=0 - -# Apply dispatch rate limiting on batch message instead individual -# messages with in batch message. (Default is disabled) -dispatchThrottlingOnBatchMessageEnabled=false - -# Dispatch rate-limiting relative to publish rate. -# (Enabling flag will make broker to dynamically update dispatch-rate relatively to publish-rate: -# throttle-dispatch-rate = (publish-rate + configured dispatch-rate). -dispatchThrottlingRateRelativeToPublishRate=false - -# By default we enable dispatch-throttling for both caught up consumers as well as consumers who have -# backlog. -dispatchThrottlingOnNonBacklogConsumerEnabled=true - -# The read failure backoff initial time in milliseconds. By default it is 15s. -dispatcherReadFailureBackoffInitialTimeInMs=15000 - -# The read failure backoff max time in milliseconds. By default it is 60s. -dispatcherReadFailureBackoffMaxTimeInMs=60000 - -# The read failure backoff mandatory stop time in milliseconds. By default it is 0s. -dispatcherReadFailureBackoffMandatoryStopTimeInMs=0 - -# Precise dispathcer flow control according to history message number of each entry -preciseDispatcherFlowControl=false - -# Max number of concurrent lookup request broker allows to throttle heavy incoming lookup traffic -maxConcurrentLookupRequest=50000 - -# Max number of concurrent topic loading request broker allows to control number of zk-operations -maxConcurrentTopicLoadRequest=5000 - -# Max concurrent non-persistent message can be processed per connection -maxConcurrentNonPersistentMessagePerConnection=1000 - -# Number of worker threads to serve non-persistent topic -numWorkerThreadsForNonPersistentTopic=8 - -# Enable broker to load persistent topics -enablePersistentTopics=true - -# Enable broker to load non-persistent topics -enableNonPersistentTopics=true - -# Max number of producers allowed to connect to topic. Once this limit reaches, Broker will reject new producers -# until the number of connected producers decrease. -# Using a value of 0, is disabling maxProducersPerTopic-limit check. -maxProducersPerTopic=0 - -# Max number of producers with the same IP address allowed to connect to topic. -# Once this limit reaches, Broker will reject new producers until the number of -# connected producers with the same IP address decrease. -# Using a value of 0, is disabling maxSameAddressProducersPerTopic-limit check. -maxSameAddressProducersPerTopic=0 - -# Enforce producer to publish encrypted messages.(default disable). -encryptionRequireOnProducer=false - -# Max number of consumers allowed to connect to topic. Once this limit reaches, Broker will reject new consumers -# until the number of connected consumers decrease. -# Using a value of 0, is disabling maxConsumersPerTopic-limit check. -maxConsumersPerTopic=0 - -# Max number of consumers with the same IP address allowed to connect to topic. -# Once this limit reaches, Broker will reject new consumers until the number of -# connected consumers with the same IP address decrease. -# Using a value of 0, is disabling maxSameAddressConsumersPerTopic-limit check. -maxSameAddressConsumersPerTopic=0 - -# Max number of subscriptions allowed to subscribe to topic. Once this limit reaches, broker will reject -# new subscription until the number of subscribed subscriptions decrease. -# Using a value of 0, is disabling maxSubscriptionsPerTopic limit check. -maxSubscriptionsPerTopic=0 - -# Max number of consumers allowed to connect to subscription. Once this limit reaches, Broker will reject new consumers -# until the number of connected consumers decrease. -# Using a value of 0, is disabling maxConsumersPerSubscription-limit check. -maxConsumersPerSubscription=0 - -# Max number of partitions per partitioned topic -# Use 0 or negative number to disable the check -maxNumPartitionsPerPartitionedTopic=0 - -### --- Metadata Store --- ### - -# Whether we should enable metadata operations batching -metadataStoreBatchingEnabled=true - -# Maximum delay to impose on batching grouping -metadataStoreBatchingMaxDelayMillis=5 - -# Maximum number of operations to include in a singular batch -metadataStoreBatchingMaxOperations=1000 - -# Maximum size of a batch -metadataStoreBatchingMaxSizeKb=128 - -### --- TLS --- ### -# Deprecated - Use webServicePortTls and brokerServicePortTls instead -tlsEnabled=false - -# Tls cert refresh duration in seconds (set 0 to check on every new connection) -tlsCertRefreshCheckDurationSec=300 - -# Path for the TLS certificate file -tlsCertificateFilePath= - -# Path for the TLS private key file -tlsKeyFilePath= - -# Path for the trusted TLS certificate file. -# This cert is used to verify that any certs presented by connecting clients -# are signed by a certificate authority. If this verification -# fails, then the certs are untrusted and the connections are dropped. -tlsTrustCertsFilePath= - -# Accept untrusted TLS certificate from client. -# If true, a client with a cert which cannot be verified with the -# 'tlsTrustCertsFilePath' cert will allowed to connect to the server, -# though the cert will not be used for client authentication. -tlsAllowInsecureConnection=false - -# Specify the tls protocols the broker will use to negotiate during TLS handshake -# (a comma-separated list of protocol names). -# Examples:- [TLSv1.3, TLSv1.2] -tlsProtocols= - -# Specify the tls cipher the broker will use to negotiate during TLS Handshake -# (a comma-separated list of ciphers). -# Examples:- [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256] -tlsCiphers= - -# Trusted client certificates are required for to connect TLS -# Reject the Connection if the Client Certificate is not trusted. -# In effect, this requires that all connecting clients perform TLS client -# authentication. -tlsRequireTrustedClientCertOnConnect=false - -### --- KeyStore TLS config variables --- ### -# Enable TLS with KeyStore type configuration in broker. -tlsEnabledWithKeyStore=false - -# TLS Provider for KeyStore type -tlsProvider= - -# TLS KeyStore type configuration in broker: JKS, PKCS12 -tlsKeyStoreType=JKS - -# TLS KeyStore path in broker -tlsKeyStore= - -# TLS KeyStore password for broker -tlsKeyStorePassword= - -# TLS TrustStore type configuration in broker: JKS, PKCS12 -tlsTrustStoreType=JKS - -# TLS TrustStore path in broker -tlsTrustStore= - -# TLS TrustStore password for broker -tlsTrustStorePassword= - -# Whether internal client use KeyStore type to authenticate with Pulsar brokers -brokerClientTlsEnabledWithKeyStore=false - -# The TLS Provider used by internal client to authenticate with other Pulsar brokers -brokerClientSslProvider= - -# TLS TrustStore type configuration for internal client: JKS, PKCS12 -# used by the internal client to authenticate with Pulsar brokers -brokerClientTlsTrustStoreType=JKS - -# TLS TrustStore path for internal client -# used by the internal client to authenticate with Pulsar brokers -brokerClientTlsTrustStore= - -# TLS TrustStore password for internal client, -# used by the internal client to authenticate with Pulsar brokers -brokerClientTlsTrustStorePassword= - -# Specify the tls cipher the internal client will use to negotiate during TLS Handshake -# (a comma-separated list of ciphers) -# e.g. [TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256]. -# used by the internal client to authenticate with Pulsar brokers -brokerClientTlsCiphers= - -# Specify the tls protocols the broker will use to negotiate during TLS handshake -# (a comma-separated list of protocol names). -# e.g. [TLSv1.3, TLSv1.2] -# used by the internal client to authenticate with Pulsar brokers -brokerClientTlsProtocols= - -# Enable or disable system topic -systemTopicEnabled=true - -# Enable or disable topic level policies, topic level policies depends on the system topic -# Please enable the system topic first. -topicLevelPoliciesEnabled=false - -# If a topic remains fenced for this number of seconds, it will be closed forcefully. -# If it is set to 0 or a negative number, the fenced topic will not be closed. -topicFencingTimeoutSeconds=0 - -### --- Authentication --- ### -# Role names that are treated as "proxy roles". If the broker sees a request with -#role as proxyRoles - it will demand to see a valid original principal. -proxyRoles= - -# If this flag is set then the broker authenticates the original Auth data -# else it just accepts the originalPrincipal and authorizes it (if required). -authenticateOriginalAuthData=false - -# Enable authentication -authenticationEnabled=false - -# Authentication provider name list, which is comma separated list of class names -authenticationProviders= - -# Enforce authorization -authorizationEnabled=false - -# Authorization provider fully qualified class-name -authorizationProvider=org.apache.pulsar.broker.authorization.PulsarAuthorizationProvider - -# Allow wildcard matching in authorization -# (wildcard matching only applicable if wildcard-char: -# * presents at first or last position eg: *.pulsar.service, pulsar.service.*) -authorizationAllowWildcardsMatching=false - -# Role names that are treated as "super-user", meaning they will be able to do all admin -# operations and publish/consume from all topics -superUserRoles= - -# Authentication settings of the broker itself. Used when the broker connects to other brokers, -# either in same or other clusters -brokerClientAuthenticationPlugin= -brokerClientAuthenticationParameters= - -# Supported Athenz provider domain names(comma separated) for authentication -athenzDomainNames= - -# When this parameter is not empty, unauthenticated users perform as anonymousUserRole -anonymousUserRole= - - -### --- Token Authentication Provider --- ### - -## Symmetric key -# Configure the secret key to be used to validate auth tokens -# The key can be specified like: -# tokenSecretKey=data:;base64,xxxxxxxxx -# tokenSecretKey=file:///my/secret.key ( Note: key file must be DER-encoded ) -tokenSecretKey= - -## Asymmetric public/private key pair -# Configure the public key to be used to validate auth tokens -# The key can be specified like: -# tokenPublicKey=data:;base64,xxxxxxxxx -# tokenPublicKey=file:///my/public.key ( Note: key file must be DER-encoded ) -tokenPublicKey= - - -# The token "claim" that will be interpreted as the authentication "role" or "principal" by AuthenticationProviderToken (defaults to "sub" if blank) -tokenAuthClaim= - -# The token audience "claim" name, e.g. "aud", that will be used to get the audience from token. -# If not set, audience will not be verified. -tokenAudienceClaim= - -# The token audience stands for this broker. The field `tokenAudienceClaim` of a valid token, need contains this. -tokenAudience= - -### --- BookKeeper Client --- ### - -# Authentication plugin to use when connecting to bookies -bookkeeperClientAuthenticationPlugin= - -# BookKeeper auth plugin implementation specifics parameters name and values -bookkeeperClientAuthenticationParametersName= -bookkeeperClientAuthenticationParameters= - -# Timeout for BK add / read operations -bookkeeperClientTimeoutInSeconds=30 - -# Number of BookKeeper client worker threads -# Default is Runtime.getRuntime().availableProcessors() -bookkeeperClientNumWorkerThreads= - -# Speculative reads are initiated if a read request doesn't complete within a certain time -# Using a value of 0, is disabling the speculative reads -bookkeeperClientSpeculativeReadTimeoutInMillis=0 - -# Number of channels per bookie -bookkeeperNumberOfChannelsPerBookie=16 - -# Enable bookies health check. Bookies that have more than the configured number of failure within -# the interval will be quarantined for some time. During this period, new ledgers won't be created -# on these bookies -bookkeeperClientHealthCheckEnabled=true -bookkeeperClientHealthCheckIntervalSeconds=60 -bookkeeperClientHealthCheckErrorThresholdPerInterval=5 -bookkeeperClientHealthCheckQuarantineTimeInSeconds=1800 - -#bookie quarantine ratio to avoid all clients quarantine the high pressure bookie servers at the same time -bookkeeperClientQuarantineRatio=1.0 - -# Enable rack-aware bookie selection policy. BK will chose bookies from different racks when -# forming a new bookie ensemble -# This parameter related to ensemblePlacementPolicy in conf/bookkeeper.conf, if enabled, ensemblePlacementPolicy -# should be set to org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy -bookkeeperClientRackawarePolicyEnabled=true - -# Enable region-aware bookie selection policy. BK will chose bookies from -# different regions and racks when forming a new bookie ensemble. -# If enabled, the value of bookkeeperClientRackawarePolicyEnabled is ignored -# This parameter related to ensemblePlacementPolicy in conf/bookkeeper.conf, if enabled, ensemblePlacementPolicy -# should be set to org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy -bookkeeperClientRegionawarePolicyEnabled=false - -# Minimum number of racks per write quorum. BK rack-aware bookie selection policy will try to -# get bookies from at least 'bookkeeperClientMinNumRacksPerWriteQuorum' racks for a write quorum. -bookkeeperClientMinNumRacksPerWriteQuorum=1 - -# Enforces rack-aware bookie selection policy to pick bookies from 'bookkeeperClientMinNumRacksPerWriteQuorum' -# racks for a writeQuorum. -# If BK can't find bookie then it would throw BKNotEnoughBookiesException instead of picking random one. -bookkeeperClientEnforceMinNumRacksPerWriteQuorum=false - -# Enable/disable reordering read sequence on reading entries. -bookkeeperClientReorderReadSequenceEnabled=false - -# Enable bookie isolation by specifying a list of bookie groups to choose from. Any bookie -# outside the specified groups will not be used by the broker -bookkeeperClientIsolationGroups= - -# Enable bookie secondary-isolation group if bookkeeperClientIsolationGroups doesn't -# have enough bookie available. -bookkeeperClientSecondaryIsolationGroups= - -# Minimum bookies that should be available as part of bookkeeperClientIsolationGroups -# else broker will include bookkeeperClientSecondaryIsolationGroups bookies in isolated list. -bookkeeperClientMinAvailableBookiesInIsolationGroups= - -# Set the client security provider factory class name. -# Default: org.apache.bookkeeper.tls.TLSContextFactory -bookkeeperTLSProviderFactoryClass=org.apache.bookkeeper.tls.TLSContextFactory - -# Enable tls authentication with bookie -bookkeeperTLSClientAuthentication=false - -# Supported type: PEM, JKS, PKCS12. Default value: PEM -bookkeeperTLSKeyFileType=PEM - -#Supported type: PEM, JKS, PKCS12. Default value: PEM -bookkeeperTLSTrustCertTypes=PEM - -# Path to file containing keystore password, if the client keystore is password protected. -bookkeeperTLSKeyStorePasswordPath= - -# Path to file containing truststore password, if the client truststore is password protected. -bookkeeperTLSTrustStorePasswordPath= - -# Path for the TLS private key file -bookkeeperTLSKeyFilePath= - -# Path for the TLS certificate file -bookkeeperTLSCertificateFilePath= - -# Path for the trusted TLS certificate file -bookkeeperTLSTrustCertsFilePath= - -# Enable/disable disk weight based placement. Default is false -bookkeeperDiskWeightBasedPlacementEnabled=false - -# Set the interval to check the need for sending an explicit LAC -# A value of '0' disables sending any explicit LACs. Default is 0. -bookkeeperExplicitLacIntervalInMills=0 - -# Use older Bookkeeper wire protocol with bookie -bookkeeperUseV2WireProtocol=true - -# Expose bookkeeper client managed ledger stats to prometheus. default is false -# bookkeeperClientExposeStatsToPrometheus=false - -### --- Managed Ledger --- ### - -# Number of bookies to use when creating a ledger -managedLedgerDefaultEnsembleSize=1 - -# Number of copies to store for each message -managedLedgerDefaultWriteQuorum=1 - -# Number of guaranteed copies (acks to wait before write is complete) -managedLedgerDefaultAckQuorum=1 - -# How frequently to flush the cursor positions that were accumulated due to rate limiting. (seconds). -# Default is 60 seconds -managedLedgerCursorPositionFlushSeconds=60 - -# Default type of checksum to use when writing to BookKeeper. Default is "CRC32C" -# Other possible options are "CRC32", "MAC" or "DUMMY" (no checksum). -managedLedgerDigestType=CRC32C - -# Number of threads to be used for managed ledger tasks dispatching -managedLedgerNumWorkerThreads=4 - -# Number of threads to be used for managed ledger scheduled tasks -managedLedgerNumSchedulerThreads=4 - -# Amount of memory to use for caching data payload in managed ledger. This memory -# is allocated from JVM direct memory and it's shared across all the topics -# running in the same broker. By default, uses 1/5th of available direct memory -managedLedgerCacheSizeMB= - -# Whether we should make a copy of the entry payloads when inserting in cache -managedLedgerCacheCopyEntries=false - -# Threshold to which bring down the cache level when eviction is triggered -managedLedgerCacheEvictionWatermark=0.9 - -# Configure the cache eviction frequency for the managed ledger cache (evictions/sec) -managedLedgerCacheEvictionFrequency=100.0 - -# All entries that have stayed in cache for more than the configured time, will be evicted -managedLedgerCacheEvictionTimeThresholdMillis=1000 - -# Configure the threshold (in number of entries) from where a cursor should be considered 'backlogged' -# and thus should be set as inactive. -managedLedgerCursorBackloggedThreshold=1000 - -# Rate limit the amount of writes generated by consumer acking the messages -managedLedgerDefaultMarkDeleteRateLimit=0.1 - -# Max number of entries to append to a ledger before triggering a rollover -# A ledger rollover is triggered after the min rollover time has passed -# and one of the following conditions is true: -# * The max rollover time has been reached -# * The max entries have been written to the ledger -# * The max ledger size has been written to the ledger -managedLedgerMaxEntriesPerLedger=50000 - -# Minimum time between ledger rollover for a topic -managedLedgerMinLedgerRolloverTimeMinutes=10 - -# Maximum time before forcing a ledger rollover for a topic -managedLedgerMaxLedgerRolloverTimeMinutes=240 - -# Max number of entries to append to a cursor ledger -managedLedgerCursorMaxEntriesPerLedger=50000 - -# Max time before triggering a rollover on a cursor ledger -managedLedgerCursorRolloverTimeInSeconds=14400 - -# Maximum ledger size before triggering a rollover for a topic (MB) -managedLedgerMaxSizePerLedgerMbytes=2048 - -# Max number of "acknowledgment holes" that are going to be persistently stored. -# When acknowledging out of order, a consumer will leave holes that are supposed -# to be quickly filled by acking all the messages. The information of which -# messages are acknowledged is persisted by compressing in "ranges" of messages -# that were acknowledged. After the max number of ranges is reached, the information -# will only be tracked in memory and messages will be redelivered in case of -# crashes. -managedLedgerMaxUnackedRangesToPersist=10000 - -# Max number of "acknowledgment holes" that can be stored in Zookeeper. If number of unack message range is higher -# than this limit then broker will persist unacked ranges into bookkeeper to avoid additional data overhead into -# zookeeper. -managedLedgerMaxUnackedRangesToPersistInZooKeeper=1000 - -# Skip reading non-recoverable/unreadable data-ledger under managed-ledger's list. It helps when data-ledgers gets -# corrupted at bookkeeper and managed-cursor is stuck at that ledger. -autoSkipNonRecoverableData=false - -# operation timeout while updating managed-ledger metadata. -managedLedgerMetadataOperationsTimeoutSeconds=60 - -# Read entries timeout when broker tries to read messages from bookkeeper. -managedLedgerReadEntryTimeoutSeconds=0 - -# Add entry timeout when broker tries to publish message to bookkeeper (0 to disable it). -managedLedgerAddEntryTimeoutSeconds=0 - -# New entries check delay for the cursor under the managed ledger. -# If no new messages in the topic, the cursor will try to check again after the delay time. -# For consumption latency sensitive scenario, can set to a smaller value or set to 0. -# Of course, use a smaller value may degrade consumption throughput. Default is 10ms. -managedLedgerNewEntriesCheckDelayInMillis=10 - -# Use Open Range-Set to cache unacked messages -managedLedgerUnackedRangesOpenCacheSetEnabled=true - -# Managed ledger prometheus stats latency rollover seconds (default: 60s) -managedLedgerPrometheusStatsLatencyRolloverSeconds=60 - -# Whether trace managed ledger task execution time -managedLedgerTraceTaskExecution=true - -# If you want to custom bookie ID or use a dynamic network address for the bookie, -# you can set this option. -# Bookie advertises itself using bookieId rather than -# BookieSocketAddress (hostname:port or IP:port). -# bookieId is a non empty string that can contain ASCII digits and letters ([a-zA-Z9-0]), -# colons, dashes, and dots. -# For more information about bookieId, see http://bookkeeper.apache.org/bps/BP-41-bookieid/. -# bookieId= - -### --- Load balancer --- ### - -loadManagerClassName=org.apache.pulsar.broker.loadbalance.NoopLoadManager - -# Enable load balancer -loadBalancerEnabled=false - -# Percentage of change to trigger load report update -loadBalancerReportUpdateThresholdPercentage=10 - -# maximum interval to update load report -loadBalancerReportUpdateMaxIntervalMinutes=15 - -# Frequency of report to collect -loadBalancerHostUsageCheckIntervalMinutes=1 - -# Load shedding interval. Broker periodically checks whether some traffic should be offload from -# some over-loaded broker to other under-loaded brokers -loadBalancerSheddingIntervalMinutes=1 - -# Prevent the same topics to be shed and moved to other broker more than once within this timeframe -loadBalancerSheddingGracePeriodMinutes=30 - -# Usage threshold to allocate max number of topics to broker -loadBalancerBrokerMaxTopics=50000 - -# Interval to flush dynamic resource quota to ZooKeeper -loadBalancerResourceQuotaUpdateIntervalMinutes=15 - -# enable/disable namespace bundle auto split -loadBalancerAutoBundleSplitEnabled=true - -# enable/disable automatic unloading of split bundles -loadBalancerAutoUnloadSplitBundlesEnabled=true - -# maximum topics in a bundle, otherwise bundle split will be triggered -loadBalancerNamespaceBundleMaxTopics=1000 - -# maximum sessions (producers + consumers) in a bundle, otherwise bundle split will be triggered -loadBalancerNamespaceBundleMaxSessions=1000 - -# maximum msgRate (in + out) in a bundle, otherwise bundle split will be triggered -loadBalancerNamespaceBundleMaxMsgRate=30000 - -# maximum bandwidth (in + out) in a bundle, otherwise bundle split will be triggered -loadBalancerNamespaceBundleMaxBandwidthMbytes=100 - -# maximum number of bundles in a namespace -loadBalancerNamespaceMaximumBundles=128 - -# The broker resource usage threshold. -# When the broker resource usage is greater than the pulsar cluster average resource usage, -# the threshold shedder will be triggered to offload bundles from the broker. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerBrokerThresholdShedderPercentage=10 - -# When calculating new resource usage, the history usage accounts for. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerHistoryResourcePercentage=0.9 - -# The BandWithIn usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerBandwithInResourceWeight=1.0 - -# The BandWithOut usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerBandwithOutResourceWeight=1.0 - -# The CPU usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerCPUResourceWeight=1.0 - -# The heap memory usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerMemoryResourceWeight=1.0 - -# The direct memory usage weight when calculating new resource usage. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerDirectMemoryResourceWeight=1.0 - -# Bundle unload minimum throughput threshold (MB), avoiding bundle unload frequently. -# It only takes effect in the ThresholdShedder strategy. -loadBalancerBundleUnloadMinThroughputThreshold=10 - -# Time to wait for the unloading of a namespace bundle -namespaceBundleUnloadingTimeoutMs=60000 - -### --- Replication --- ### - -# Enable replication metrics -replicationMetricsEnabled=true - -# Max number of connections to open for each broker in a remote cluster -# More connections host-to-host lead to better throughput over high-latency -# links. -replicationConnectionsPerBroker=16 - -# Replicator producer queue size -replicationProducerQueueSize=1000 - -# Duration to check replication policy to avoid replicator inconsistency -# due to missing ZooKeeper watch (disable with value 0) -replicationPolicyCheckDurationSeconds=600 - -# Default message retention time -defaultRetentionTimeInMinutes=0 - -# Default retention size -defaultRetentionSizeInMB=0 - -# How often to check whether the connections are still alive -keepAliveIntervalSeconds=30 - -### --- WebSocket --- ### - -# Enable the WebSocket API service in broker -webSocketServiceEnabled=true - -# Number of IO threads in Pulsar Client used in WebSocket proxy -webSocketNumIoThreads=8 - -# Number of connections per Broker in Pulsar Client used in WebSocket proxy -webSocketConnectionsPerBroker=8 - -# Time in milliseconds that idle WebSocket session times out -webSocketSessionIdleTimeoutMillis=300000 - -# The maximum size of a text message during parsing in WebSocket proxy -webSocketMaxTextFrameSize=1048576 - -### --- Metrics --- ### - -# Enable topic level metrics -exposeTopicLevelMetricsInPrometheus=true - -# Time in milliseconds that metrics endpoint would time out. Default is 30s. -# Increase it if there are a lot of topics to expose topic-level metrics. -# Set it to 0 to disable timeout. -metricsServletTimeoutMs=30000 - -# Classname of Pluggable JVM GC metrics logger that can log GC specific metrics -# jvmGCMetricsLoggerClassName= - -### --- Broker Web Stats --- ### - -# Enable topic level metrics -exposePublisherStats=true - -# Enable expose the precise backlog stats. -# Set false to use published counter and consumed counter to calculate, this would be more efficient but may be inaccurate. -# Default is false. -exposePreciseBacklogInPrometheus=false - -# Enable splitting topic and partition label in Prometheus. -# If enabled, a topic name will split into 2 parts, one is topic name without partition index, -# another one is partition index, e.g. (topic=xxx, partition=0). -# If the topic is a non-partitioned topic, -1 will be used for the partition index. -# If disabled, one label to represent the topic and partition, e.g. (topic=xxx-partition-0) -# Default is false. - -splitTopicAndPartitionLabelInPrometheus=false - -### --- Deprecated config variables --- ### - -# Deprecated. Use configurationStoreServers -globalZookeeperServers= - -# Deprecated. Use brokerDeleteInactiveTopicsFrequencySeconds -brokerServicePurgeInactiveFrequencyInSeconds=60 - -### --- BookKeeper Configuration --- ##### - -ledgerStorageClass=org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage - -# The maximum netty frame size in bytes. Any message received larger than this will be rejected. The default value is 5MB. -nettyMaxFrameSizeBytes=5253120 - -# Size of Write Cache. Memory is allocated from JVM direct memory. -# Write cache is used to buffer entries before flushing into the entry log -# For good performance, it should be big enough to hold a substantial amount -# of entries in the flush interval -# By default it will be allocated to 1/4th of the available direct memory -dbStorage_writeCacheMaxSizeMb= - -# Size of Read cache. Memory is allocated from JVM direct memory. -# This read cache is pre-filled doing read-ahead whenever a cache miss happens -# By default it will be allocated to 1/4th of the available direct memory -dbStorage_readAheadCacheMaxSizeMb= - -# How many entries to pre-fill in cache after a read cache miss -dbStorage_readAheadCacheBatchSize=1000 - -flushInterval=60000 - -## RocksDB specific configurations -## DbLedgerStorage uses RocksDB to store the indexes from -## (ledgerId, entryId) -> (entryLog, offset) - -# Size of RocksDB block-cache. For best performance, this cache -# should be big enough to hold a significant portion of the index -# database which can reach ~2GB in some cases -# Default is to use 10% of the direct memory size -dbStorage_rocksDB_blockCacheSize= - -# Other RocksDB specific tunables -dbStorage_rocksDB_writeBufferSizeMB=4 -dbStorage_rocksDB_sstSizeInMB=4 -dbStorage_rocksDB_blockSize=4096 -dbStorage_rocksDB_bloomFilterBitsPerKey=10 -dbStorage_rocksDB_numLevels=-1 -dbStorage_rocksDB_numFilesInLevel0=4 -dbStorage_rocksDB_maxSizeInLevel1MB=256 - -# Maximum latency to impose on a journal write to achieve grouping -journalMaxGroupWaitMSec=1 - -# Should the data be fsynced on journal before acknowledgment. -journalSyncData=false - - -# For each ledger dir, maximum disk space which can be used. -# Default is 0.95f. i.e. 95% of disk can be used at most after which nothing will -# be written to that partition. If all ledger dir partions are full, then bookie -# will turn to readonly mode if 'readOnlyModeEnabled=true' is set, else it will -# shutdown. -# Valid values should be in between 0 and 1 (exclusive). -diskUsageThreshold=0.99 - -# The disk free space low water mark threshold. -# Disk is considered full when usage threshold is exceeded. -# Disk returns back to non-full state when usage is below low water mark threshold. -# This prevents it from going back and forth between these states frequently -# when concurrent writes and compaction are happening. This also prevent bookie from -# switching frequently between read-only and read-writes states in the same cases. -diskUsageWarnThreshold=0.99 - -# Whether the bookie allowed to use a loopback interface as its primary -# interface(i.e. the interface it uses to establish its identity)? -# By default, loopback interfaces are not allowed as the primary -# interface. -# Using a loopback interface as the primary interface usually indicates -# a configuration error. For example, its fairly common in some VPS setups -# to not configure a hostname, or to have the hostname resolve to -# 127.0.0.1. If this is the case, then all bookies in the cluster will -# establish their identities as 127.0.0.1:3181, and only one will be able -# to join the cluster. For VPSs configured like this, you should explicitly -# set the listening interface. -allowLoopback=true - -# How long the interval to trigger next garbage collection, in milliseconds -# Since garbage collection is running in background, too frequent gc -# will heart performance. It is better to give a higher number of gc -# interval if there is enough disk capacity. -gcWaitTime=300000 - -# Enable topic auto creation if new producer or consumer connected (disable auto creation with value false) -allowAutoTopicCreation=true - -# The type of topic that is allowed to be automatically created.(partitioned/non-partitioned) -allowAutoTopicCreationType=non-partitioned - -# Enable subscription auto creation if new consumer connected (disable auto creation with value false) -allowAutoSubscriptionCreation=true - -# The number of partitioned topics that is allowed to be automatically created if allowAutoTopicCreationType is partitioned. -defaultNumPartitions=1 - -### --- Transaction config variables --- ### -# Enable transaction coordinator in broker -transactionCoordinatorEnabled=true -transactionMetadataStoreProviderClassName=org.apache.pulsar.transaction.coordinator.impl.MLTransactionMetadataStoreProvider - -# Transaction buffer take snapshot transaction count -transactionBufferSnapshotMaxTransactionCount=1000 - -# Transaction buffer take snapshot interval time -# Unit : millisecond -transactionBufferSnapshotMinTimeInMillis=5000 - -### --- Packages management service configuration variables (begin) --- ### - -# Enable the packages management service or not -enablePackagesManagement=false - -# The packages management service storage service provide -packagesManagementStorageProvider=org.apache.pulsar.packages.management.storage.bookkeeper.BookKeeperPackagesStorageProvider - -# When the packages storage provider is bookkeeper, you can use this configuration to -# control the number of replicas for storing the package -packagesReplicas=1 - -# The bookkeeper ledger root path -packagesManagementLedgerRootPath=/ledgers - -### --- Packages management service configuration variables (end) --- ### diff --git a/flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh b/flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh new file mode 100755 index 0000000000000..fa4e2921a75cf --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/resources/docker/bootstrap.sh @@ -0,0 +1,28 @@ +#!/usr/bin/env bash +# ---------------------------------------------------------------------------- +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +# ---------------------------------------------------------------------------- + +# Enable the transaction in standalone config. +sed -i 's/transactionCoordinatorEnabled=false/transactionCoordinatorEnabled=true/g' /pulsar/conf/standalone.conf +sed -i 's/acknowledgmentAtBatchIndexLevelEnabled=false/acknowledgmentAtBatchIndexLevelEnabled=true/g' /pulsar/conf/standalone.conf +sed -i 's/systemTopicEnabled=false/systemTopicEnabled=true/g' /pulsar/conf/standalone.conf +sed -i 's/brokerDeduplicationEnabled=false/brokerDeduplicationEnabled=true/g' /pulsar/conf/standalone.conf + +# Start Pulsar standalone without function worker and streaming storage. +/pulsar/bin/pulsar standalone --no-functions-worker -nss diff --git a/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE index db82a498c9e87..3bcba0e019870 100644 --- a/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE +++ b/flink-connectors/flink-sql-connector-pulsar/src/main/resources/META-INF/NOTICE @@ -6,11 +6,11 @@ The Apache Software Foundation (http://www.apache.org/). This project bundles the following dependencies under the Apache Software License 2.0 (http://www.apache.org/licenses/LICENSE-2.0.txt) -- org.apache.pulsar:bouncy-castle-bc:pkg:2.9.1 -- org.apache.pulsar:pulsar-client-admin-api:2.9.1 -- org.apache.pulsar:pulsar-client-all:2.9.1 -- org.apache.pulsar:pulsar-client-api:2.9.1 -- org.slf4j:jul-to-slf4j:1.7.25 +- org.apache.pulsar:bouncy-castle-bc:pkg:2.10.0 +- org.apache.pulsar:pulsar-client-admin-api:2.10.0 +- org.apache.pulsar:pulsar-client-all:2.10.0 +- org.apache.pulsar:pulsar-client-api:2.10.0 +- org.slf4j:jul-to-slf4j:1.7.32 This project bundles the following dependencies under the Bouncy Castle license. See bundled license files for details. diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml index 7a27f6090b44d..8b8d794cebc2b 100644 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/pom.xml @@ -31,7 +31,7 @@ under the License. Flink : E2E Tests : Pulsar - 2.9.1 + 2.10.0 1.69 diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java new file mode 100644 index 0000000000000..1fdfc6fa18f1a --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.pulsar.cases; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; +import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FixedRangeGenerator; +import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; +import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; +import org.apache.flink.tests.util.pulsar.common.KeyedPulsarPartitionDataWriter; + +import org.apache.pulsar.client.api.RegexSubscriptionMode; +import org.apache.pulsar.client.api.SubscriptionType; +import org.apache.pulsar.common.util.Murmur3_32Hash; + +import java.net.URL; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static java.util.Collections.singletonList; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE; +import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; +import static org.apache.pulsar.client.api.Schema.STRING; + +/** We would consume from test splits by using {@link SubscriptionType#Key_Shared} subscription. */ +public class KeySharedSubscriptionContext extends PulsarTestContext { + + private int index = 0; + + private final List writers = new ArrayList<>(); + + // Message keys. + private final String key1; + private final String key2; + + public KeySharedSubscriptionContext(PulsarTestEnvironment environment) { + this(environment, Collections.emptyList()); + } + + public KeySharedSubscriptionContext( + PulsarTestEnvironment environment, List connectorJarPaths) { + super(environment, connectorJarPaths); + + // Init message keys. + this.key1 = randomAlphabetic(8); + String newKey2; + do { + newKey2 = randomAlphabetic(8); + } while (keyHash(key1) == keyHash(newKey2)); + this.key2 = newKey2; + } + + @Override + protected String displayName() { + return "consuming message by Key_Shared"; + } + + @Override + public Source createSource(TestingSourceSettings sourceSettings) { + int keyHash = keyHash(key1); + TopicRange range = new TopicRange(keyHash, keyHash); + + PulsarSourceBuilder builder = + PulsarSource.builder() + .setDeserializationSchema(pulsarSchema(STRING)) + .setServiceUrl(operator.serviceUrl()) + .setAdminUrl(operator.adminUrl()) + .setTopicPattern( + "pulsar-[0-9]+-key-shared", RegexSubscriptionMode.AllTopics) + .setSubscriptionType(SubscriptionType.Key_Shared) + .setSubscriptionName("pulsar-key-shared") + .setConsumerName("pulsar-key-shared") + .setRangeGenerator(new FixedRangeGenerator(singletonList(range))); + if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { + // Using the latest stop cursor for making sure the source could be stopped. + builder.setBoundedStopCursor(StopCursor.latest()); + } + + return builder.build(); + } + + @Override + public ExternalSystemSplitDataWriter createSourceSplitDataWriter( + TestingSourceSettings sourceSettings) { + String topicName = "pulsar-" + index + "-key-shared"; + operator.createTopic(topicName, 1); + index++; + + String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0); + KeyedPulsarPartitionDataWriter writer = + new KeyedPulsarPartitionDataWriter(operator, partitionName, key1, key2); + writers.add(writer); + + return writer; + } + + @Override + public List generateTestData( + TestingSourceSettings sourceSettings, int splitIndex, long seed) { + return generateStringTestData(splitIndex, seed); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(String.class); + } + + @Override + public void close() { + for (KeyedPulsarPartitionDataWriter writer : writers) { + writer.close(); + } + writers.clear(); + } + + private int keyHash(String key) { + return Murmur3_32Hash.getInstance().makeHash(key.getBytes()) % RANGE_SIZE; + } +} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java new file mode 100644 index 0000000000000..6bdaaa8e339ca --- /dev/null +++ b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.tests.util.pulsar.cases; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.connector.pulsar.source.PulsarSource; +import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; +import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter; +import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; +import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; +import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; +import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; + +import org.apache.pulsar.client.api.RegexSubscriptionMode; +import org.apache.pulsar.client.api.SubscriptionType; + +import java.net.URL; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; +import static org.apache.pulsar.client.api.Schema.STRING; + +/** We would consuming from test splits by using {@link SubscriptionType#Shared} subscription. */ +public class SharedSubscriptionContext extends PulsarTestContext { + + private int index = 0; + + private final List writers = new ArrayList<>(); + + public SharedSubscriptionContext(PulsarTestEnvironment environment) { + this(environment, Collections.emptyList()); + } + + public SharedSubscriptionContext( + PulsarTestEnvironment environment, List connectorJarPaths) { + super(environment, connectorJarPaths); + } + + @Override + protected String displayName() { + return "consuming message by Shared"; + } + + @Override + public Source createSource(TestingSourceSettings sourceSettings) { + PulsarSourceBuilder builder = + PulsarSource.builder() + .setDeserializationSchema(pulsarSchema(STRING)) + .setServiceUrl(operator.serviceUrl()) + .setAdminUrl(operator.adminUrl()) + .setTopicPattern("pulsar-[0-9]+-shared", RegexSubscriptionMode.AllTopics) + .setSubscriptionType(SubscriptionType.Shared) + .setSubscriptionName("pulsar-shared") + .setConsumerName("pulsar-shared"); + if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { + // Using the latest stop cursor for making sure the source could be stopped. + builder.setBoundedStopCursor(StopCursor.latest()); + } + + return builder.build(); + } + + @Override + public ExternalSystemSplitDataWriter createSourceSplitDataWriter( + TestingSourceSettings sourceSettings) { + String topicName = "pulsar-" + index + "-shared"; + operator.createTopic(topicName, 1); + index++; + + String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0); + PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName); + writers.add(writer); + + return writer; + } + + @Override + public List generateTestData( + TestingSourceSettings sourceSettings, int splitIndex, long seed) { + return generateStringTestData(splitIndex, seed); + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(String.class); + } + + @Override + public void close() { + for (PulsarPartitionDataWriter writer : writers) { + writer.close(); + } + writers.clear(); + } +} diff --git a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java index b8422f9d19efd..ec769037dcac5 100644 --- a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java +++ b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/DockerImageVersions.java @@ -42,7 +42,7 @@ public class DockerImageVersions { public static final String LOCALSTACK = "localstack/localstack:0.13.3"; - public static final String PULSAR = "apachepulsar/pulsar:2.9.1"; + public static final String PULSAR = "apachepulsar/pulsar:2.10.0"; public static final String CASSANDRA_3 = "cassandra:3.0"; From 732c57ebd078ba898cb61c500e3589d09cc6ba83 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 16 Jun 2022 04:03:27 +0800 Subject: [PATCH 220/258] [FLINK-27611][Connector/Pulsar] Fix ConcurrentModificationException during Pulsar checkpoint notification. --- .../source/PulsarUnorderedSourceReader.java | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java index e130031fb37b4..33b64be06d417 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java @@ -48,6 +48,8 @@ import java.util.TreeMap; import java.util.function.Supplier; +import static java.util.stream.Collectors.toList; + /** * The source reader for pulsar subscription Shared and Key_Shared, which consumes the unordered * messages. @@ -128,19 +130,19 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { LOG.debug("Committing transactions for checkpoint {}", checkpointId); if (coordinatorClient != null) { - for (Map.Entry> entry : transactionsToCommit.entrySet()) { - Long currentCheckpointId = entry.getKey(); - if (currentCheckpointId > checkpointId) { - continue; - } - - List transactions = entry.getValue(); - for (TxnID transaction : transactions) { - coordinatorClient.commit(transaction); - transactionsOfFinishedSplits.remove(transaction); + List checkpointIds = + transactionsToCommit.keySet().stream() + .filter(id -> id <= checkpointId) + .collect(toList()); + + for (Long id : checkpointIds) { + List transactions = transactionsToCommit.remove(id); + if (transactions != null) { + for (TxnID transaction : transactions) { + coordinatorClient.commit(transaction); + transactionsOfFinishedSplits.remove(transaction); + } } - - transactionsToCommit.remove(currentCheckpointId); } } } From aa0c54a46a2b1654979eec02d3fd75d93bbea10e Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 16 Jun 2022 04:41:46 +0800 Subject: [PATCH 221/258] [FLINK-27400][Connector/Pulsar] Never subscribe the Pulsar internal system name by using regex. --- .../writer/topic/TopicMetadataListener.java | 4 +- .../impl/TopicPatternSubscriber.java | 3 + .../enumerator/topic/TopicNameUtils.java | 55 +++++++++++++++++-- .../topic/TopicMetadataListenerTest.java | 5 +- 4 files changed, 56 insertions(+), 11 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java index fb51ec824a2e1..040438f902349 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java @@ -43,7 +43,7 @@ import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.isPartitioned; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithNonPartition; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; import static org.apache.pulsar.common.partition.PartitionedTopicMetadata.NON_PARTITIONED; @@ -120,7 +120,7 @@ public List availableTopics() { int partitionNums = entry.getValue(); // Get all topics from partitioned and non-partitioned topic names if (partitionNums == NON_PARTITIONED) { - results.add(topicNameWithNonPartition(entry.getKey())); + results.add(topicName(entry.getKey())); } else { for (int i = 0; i < partitionNums; i++) { results.add(topicNameWithPartition(entry.getKey(), i)); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java index 472dbde3e352b..d08022f067de3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/TopicPatternSubscriber.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.pulsar.source.enumerator.subscriber.impl; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; import org.apache.flink.connector.pulsar.source.enumerator.topic.range.RangeGenerator; @@ -35,6 +36,7 @@ import java.util.regex.Pattern; import static java.util.stream.Collectors.toSet; +import static org.apache.flink.shaded.guava30.com.google.common.base.Predicates.not; /** Subscribe to matching topics based on topic pattern. */ public class TopicPatternSubscriber extends BasePulsarSubscriber { @@ -63,6 +65,7 @@ public Set getSubscribedTopicPartitions( .namespaces() .getTopics(namespace) .parallelStream() + .filter(not(TopicNameUtils::isInternal)) .filter(this::matchesSubscriptionMode) .filter(topic -> topicPattern.matcher(topic).find()) .map(topic -> queryTopicMetadata(pulsarAdmin, topic)) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java index f5c71159deb80..94bb144686e2f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; +import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableSet; import org.apache.pulsar.common.naming.TopicName; @@ -30,13 +31,35 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.regex.Pattern; import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.pulsar.common.naming.NamespaceName.SYSTEM_NAMESPACE; +import static org.apache.pulsar.common.naming.TopicDomain.persistent; /** util for topic name. */ @Internal public final class TopicNameUtils { + private static final Pattern HEARTBEAT_NAMESPACE_PATTERN = + Pattern.compile("pulsar/[^/]+/([^:]+:\\d+)"); + private static final Pattern HEARTBEAT_NAMESPACE_PATTERN_V2 = + Pattern.compile("pulsar/([^:]+:\\d+)"); + private static final Pattern SLA_NAMESPACE_PATTERN = + Pattern.compile("sla-monitor" + "/[^/]+/([^:]+:\\d+)"); + private static final Set SYSTEM_TOPIC_NAMES = + ImmutableSet.of( + "__change_events", + "__transaction_buffer_snapshot", + "__pending_ack_state", + "__transaction_pending_ack"); + + private static final String TRANSACTION_COORDINATOR_ASSIGN_PREFIX = + TopicName.get(persistent.value(), SYSTEM_NAMESPACE, "transaction_coordinator_assign") + .toString(); + private static final String TRANSACTION_COORDINATOR_LOG_PREFIX = + TopicName.get(persistent.value(), SYSTEM_NAMESPACE, "__transaction_log_").toString(); + private TopicNameUtils() { // No public constructor. } @@ -52,12 +75,7 @@ public static String topicNameWithPartition(String topic, int partitionId) { return TopicName.get(topic).getPartition(partitionId).toString(); } - /** Get a non-partitioned topic name that does not belong to any partitioned topic. */ - public static String topicNameWithNonPartition(String topic) { - return TopicName.get(topic).toString(); - } - - public static boolean isPartitioned(String topic) { + public static boolean isPartition(String topic) { return TopicName.get(topic).isPartitioned(); } @@ -92,4 +110,29 @@ public static List distinctTopics(List topics) { return builder.build(); } + + /** + * This method is refactored from {@code BrokerService} in pulsar-broker which is not available + * in Pulsar client. We have to put it here and self maintained. Since these topic names would + * never be changed for backward compatible, we only need to add new topic names after version + * bump. + * + * @see BrokerService#isSystemTopic + */ + public static boolean isInternal(String topic) { + // A topic name instance without partition information. + String topicName = topicName(topic); + TopicName topicInstance = TopicName.get(topicName); + String localName = topicInstance.getLocalName(); + String namespace = topicInstance.getNamespace(); + + return namespace.equals(SYSTEM_NAMESPACE.toString()) + || SLA_NAMESPACE_PATTERN.matcher(namespace).matches() + || HEARTBEAT_NAMESPACE_PATTERN.matcher(namespace).matches() + || HEARTBEAT_NAMESPACE_PATTERN_V2.matcher(namespace).matches() + || SYSTEM_TOPIC_NAMES.contains(localName) + || topicName.startsWith(TRANSACTION_COORDINATOR_ASSIGN_PREFIX) + || topicName.startsWith(TRANSACTION_COORDINATOR_LOG_PREFIX); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListenerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListenerTest.java index b4aba9617c9a4..b34d229c8feac 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListenerTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListenerTest.java @@ -34,7 +34,7 @@ import static java.util.stream.Collectors.toList; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithNonPartition; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -130,8 +130,7 @@ void fetchTopicPartitionUpdate() throws Exception { void fetchNonPartitionTopic() { String topic = randomAlphabetic(10); operator().createTopic(topic, 0); - List nonPartitionTopic = - Collections.singletonList(topicNameWithNonPartition(topic)); + List nonPartitionTopic = Collections.singletonList(topicName(topic)); TopicMetadataListener listener = new TopicMetadataListener(nonPartitionTopic); long interval = Duration.ofMinutes(15).toMillis(); From 35c90e78779790dede80deb28790cd697c7aee63 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 16 Jun 2022 05:45:54 +0800 Subject: [PATCH 222/258] [FLINK-28084][Connector/Pulsar] Remove the reconsume later logic for Pulsar unordered reader. --- .../split/PulsarUnorderedPartitionSplitReader.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java index bf9d5308d68e3..a070fd2976090 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -95,18 +95,8 @@ protected Message pollMessage(Duration timeout) .acknowledgeAsync(message.getMessageId(), uncommittedTransaction) .get(); } catch (InterruptedException e) { - sneakyClient( - () -> - pulsarConsumer.reconsumeLater( - message, REDELIVER_TIME.toMillis(), TimeUnit.MILLISECONDS)); Thread.currentThread().interrupt(); throw e; - } catch (ExecutionException e) { - sneakyClient( - () -> - pulsarConsumer.reconsumeLater( - message, REDELIVER_TIME.toMillis(), TimeUnit.MILLISECONDS)); - throw e; } } From 4506272accea1f05a2d90b69977d9b636298a695 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Mon, 20 Jun 2022 17:45:43 +0800 Subject: [PATCH 223/258] [FLINK-28085][Connector/Pulsar] Close the finished splits properly for Pulsar reader. --- .../reader/fetcher/PulsarFetcherManagerBase.java | 6 ++++++ .../reader/source/PulsarOrderedSourceReader.java | 3 +++ .../source/reader/source/PulsarSourceReaderBase.java | 8 ++++++++ .../reader/source/PulsarUnorderedSourceReader.java | 11 +++++++++++ 4 files changed, 28 insertions(+) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java index 3e2daf1e16c74..e4f996200b5b7 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java @@ -73,6 +73,12 @@ public void addSplits(List splitsToAdd) { } } + /** Close the finished split related fetcher. */ + public void closeFetcher(String splitId) { + SplitFetcher, PulsarPartitionSplit> fetcher = getOrCreateFetcher(splitId); + fetcher.shutdown(); + } + @Override protected void startFetcher(SplitFetcher, PulsarPartitionSplit> fetcher) { if (fetcherStatus.get(fetcher.fetcherId()) != Boolean.TRUE) { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java index c25e6c0822350..7280cd52c12f4 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarOrderedSourceReader.java @@ -111,6 +111,9 @@ public InputStatus pollNext(ReaderOutput output) throws Exception { @Override protected void onSplitFinished(Map finishedSplitIds) { + // Close all the finished splits. + closeFinishedSplits(finishedSplitIds.keySet()); + // We don't require new splits, all the splits are pre-assigned by source enumerator. if (LOG.isDebugEnabled()) { LOG.debug("onSplitFinished event: {}", finishedSplitIds); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java index 84e02659380e1..8ca4b3fd035b4 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java @@ -32,6 +32,8 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; +import java.util.Set; + /** * The common pulsar source reader for both ordered & unordered message consuming. * @@ -71,6 +73,12 @@ protected PulsarPartitionSplit toSplitType( return splitState.toPulsarPartitionSplit(); } + protected void closeFinishedSplits(Set finishedSplitIds) { + for (String splitId : finishedSplitIds) { + ((PulsarFetcherManagerBase) splitFetcherManager).closeFetcher(splitId); + } + } + @Override public void close() throws Exception { // Close the all the consumers first. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java index 33b64be06d417..90745a6225a4b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java @@ -87,6 +87,9 @@ public PulsarUnorderedSourceReader( @Override protected void onSplitFinished(Map finishedSplitIds) { + // Close all the finished splits. + closeFinishedSplits(finishedSplitIds.keySet()); + // We don't require new splits, all the splits are pre-assigned by source enumerator. if (LOG.isDebugEnabled()) { LOG.debug("onSplitFinished event: {}", finishedSplitIds); @@ -146,4 +149,12 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { } } } + + @Override + public void close() throws Exception { + // Abort all the pending transactions. + + // Close the pulsar client finally. + super.close(); + } } From 4c28b9af78dac1f06e69fdd82f92a6b0c7e7ecc4 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Mon, 20 Jun 2022 19:04:11 +0800 Subject: [PATCH 224/258] [FLINK-28085][Connector/Pulsar] Close the pending transaction before close the reader. --- .../source/PulsarUnorderedSourceReader.java | 17 ++++++++++++++++- .../PulsarUnorderedPartitionSplitReader.java | 14 ++++++++++---- 2 files changed, 26 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java index 90745a6225a4b..caa7199254139 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarUnorderedSourceReader.java @@ -123,6 +123,11 @@ public List snapshotState(long checkpointId) { txnIDs.add(uncommittedTransactionId); } } + + // Add finished splits' transactions. + txnIDs.addAll(transactionsOfFinishedSplits); + // Purge the transactions. + transactionsOfFinishedSplits.clear(); } return splits; @@ -143,7 +148,6 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { if (transactions != null) { for (TxnID transaction : transactions) { coordinatorClient.commit(transaction); - transactionsOfFinishedSplits.remove(transaction); } } } @@ -153,6 +157,17 @@ public void notifyCheckpointComplete(long checkpointId) throws Exception { @Override public void close() throws Exception { // Abort all the pending transactions. + if (coordinatorClient != null) { + for (List transactions : transactionsToCommit.values()) { + for (TxnID transaction : transactions) { + try { + coordinatorClient.abort(transaction); + } catch (Exception e) { + LOG.warn("Error in aborting transaction {}", transaction, e); + } + } + } + } // Close the pulsar client finally. super.close(); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java index a070fd2976090..bbae002d8f3a3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -33,6 +33,7 @@ import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException; +import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException.TransactionNotFoundException; import org.apache.pulsar.client.api.transaction.TxnID; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,6 +46,7 @@ import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; import static org.apache.flink.connector.pulsar.common.utils.PulsarTransactionUtils.createTransaction; +import static org.apache.flink.connector.pulsar.common.utils.PulsarTransactionUtils.unwrap; /** * The split reader a given {@link PulsarPartitionSplit}, it would be closed once the {@link @@ -123,10 +125,14 @@ protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { try { coordinatorClient.abort(uncommittedTransactionId); } catch (TransactionCoordinatorClientException e) { - LOG.error( - "Failed to abort the uncommitted transaction {} when restart the reader", - uncommittedTransactionId, - e); + TransactionCoordinatorClientException exception = unwrap(e); + // The aborted transaction would return a not found exception. + if (!(exception instanceof TransactionNotFoundException)) { + LOG.error( + "Failed to abort the uncommitted transaction {} when restart the reader", + uncommittedTransactionId, + e); + } } } From 8fe10ab44b00bbd4eff60791b62ae5196216562f Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Tue, 21 Jun 2022 10:15:35 +0800 Subject: [PATCH 225/258] Fix the compile error in code. --- .../pulsar/source/enumerator/topic/TopicNameUtils.java | 2 +- .../pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java | 2 +- .../pulsar/source/reader/source/PulsarSourceReaderBase.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java index 94bb144686e2f..6a35ad6a58e37 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicNameUtils.java @@ -75,7 +75,7 @@ public static String topicNameWithPartition(String topic, int partitionId) { return TopicName.get(topic).getPartition(partitionId).toString(); } - public static boolean isPartition(String topic) { + public static boolean isPartitioned(String topic) { return TopicName.get(topic).isPartitioned(); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java index e4f996200b5b7..d4ac8a721960b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/fetcher/PulsarFetcherManagerBase.java @@ -75,7 +75,7 @@ public void addSplits(List splitsToAdd) { /** Close the finished split related fetcher. */ public void closeFetcher(String splitId) { - SplitFetcher, PulsarPartitionSplit> fetcher = getOrCreateFetcher(splitId); + SplitFetcher, PulsarPartitionSplit> fetcher = getOrCreateFetcher(splitId); fetcher.shutdown(); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java index 8ca4b3fd035b4..bf95ff858af18 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderBase.java @@ -75,7 +75,7 @@ protected PulsarPartitionSplit toSplitType( protected void closeFinishedSplits(Set finishedSplitIds) { for (String splitId : finishedSplitIds) { - ((PulsarFetcherManagerBase) splitFetcherManager).closeFetcher(splitId); + ((PulsarFetcherManagerBase) splitFetcherManager).closeFetcher(splitId); } } From 4ecac1a57aefc88c5702d21ef3f0b44b935e7a87 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Tue, 5 Jul 2022 05:20:21 +0800 Subject: [PATCH 226/258] [FLINK-28351][Connector/Pulsar] Add dynamic sink topic support for Pulsar (#129) --- .github/workflows/run-pr-check.yml | 12 +- .../docs/connectors/datastream/pulsar.md | 11 ++ .../docs/connectors/datastream/pulsar.md | 12 ++ .../a2ce237e-b050-4ba0-8748-d83637a207a8 | 6 + .../flink-connector-pulsar/pom.xml | 14 ++ .../connector/pulsar/sink/PulsarSink.java | 10 +- .../pulsar/sink/PulsarSinkBuilder.java | 34 ++++- .../pulsar/sink/writer/PulsarWriter.java | 29 ++-- ...cerRegister.java => ProducerRegister.java} | 14 +- .../sink/writer/topic/TopicExtractor.java | 53 +++++++ .../sink/writer/topic/TopicRegister.java | 42 +++++ .../topic/register/DynamicTopicRegister.java | 144 ++++++++++++++++++ .../topic/register/EmptyTopicRegister.java | 49 ++++++ .../FixedTopicRegister.java} | 26 +--- .../enumerator/cursor/CursorPosition.java | 1 + .../subscriber/impl/BasePulsarSubscriber.java | 26 ++-- .../enumerator/topic/TopicPartition.java | 23 ++- .../source/enumerator/topic/TopicRange.java | 5 +- .../pulsar/sink/writer/PulsarWriterTest.java | 4 +- ...terTest.java => ProducerRegisterTest.java} | 8 +- .../register/DynamicTopicRegisterTest.java | 100 ++++++++++++ .../FixedTopicRegisterTest.java} | 54 ++++--- .../container/PulsarContainerRuntime.java | 12 +- 23 files changed, 579 insertions(+), 110 deletions(-) rename flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/{TopicProducerRegister.java => ProducerRegister.java} (94%) create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicRegister.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/EmptyTopicRegister.java rename flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/{TopicMetadataListener.java => register/FixedTopicRegister.java} (89%) rename flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/{TopicProducerRegisterTest.java => ProducerRegisterTest.java} (92%) create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegisterTest.java rename flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/{TopicMetadataListenerTest.java => register/FixedTopicRegisterTest.java} (78%) diff --git a/.github/workflows/run-pr-check.yml b/.github/workflows/run-pr-check.yml index 54c7ef67b2a11..c4b097e3dffb0 100644 --- a/.github/workflows/run-pr-check.yml +++ b/.github/workflows/run-pr-check.yml @@ -12,12 +12,12 @@ jobs: uses: actions/checkout@v2 with: ref: ${{ github.event.pull_request.head.sha }} - - name: Set up Maven - uses: apache/pulsar-test-infra/setup-maven@master + - name: Set up JDK 8 + uses: actions/setup-java@v2 with: - maven-version: 3.6.2 + java-version: '8' + cache: 'maven' + distribution: 'zulu' - name: Run Unit Test and Install run: | - mvn -ntp -pl 'flink-connectors/flink-connector-pulsar,flink-connectors/flink-sql-connector-pulsar' \ - clean install - + mvn -ntp -pl 'flink-connectors/flink-connector-pulsar,flink-connectors/flink-sql-connector-pulsar' clean install diff --git a/docs/content.zh/docs/connectors/datastream/pulsar.md b/docs/content.zh/docs/connectors/datastream/pulsar.md index 4c402f0e48c61..6c428f2a4166a 100644 --- a/docs/content.zh/docs/connectors/datastream/pulsar.md +++ b/docs/content.zh/docs/connectors/datastream/pulsar.md @@ -404,6 +404,17 @@ PulsarSink.builder().setTopics("topic-a-partition-0", "topic-a-partition-2", "so 举个例子,如果通过 `PulsarSink.builder().setTopics("some-topic1", "some-topic1-partition-0")` 来指定写入的 Topic,那么其结果等价于 `PulsarSink.builder().setTopics("some-topic1")`。 {{< /hint >}} +#### 基于消息实例的动态 Topic 指定 + +除了前面说的一开始就指定 Topic 或者是 Topic 分区,你还可以在程序启动后基于消息内容动态指定 Topic,只需要实现 `TopicExtractor` 接口即可。 +`TopicExtractor` 接口还提供了 `TopicMetadataProvider` 用于查询某个 Topic 在 Pulsar 上有多少个分区, +查询结果会缓存并在 `PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL` 之后失效。 + +`TopicExtractor` 的返回结果支持带分区信息和不带分区信息的 Topic。 + +1. 当返回结果里没有分区信息时,我们会查询对应的分区大小,生成所有的分区 Topic,然后传递给 `TopicRouter` 用于路由。分区信息将会被缓存 `PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL`。 +2. 如果你的返回结果里面提供了分区信息,我们则会什么都不做,直接传递给下游。 + ### 序列化器 序列化器(`PulsarSerializationSchema`)负责将 Flink 中的每条记录序列化成 byte 数组,并通过网络发送至指定的写入 Topic。和 Pulsar Source 类似的是,序列化器同时支持使用基于 Flink 的 `SerializationSchema` 接口实现序列化器和使用 Pulsar 原生的 `Schema` 类型实现的序列化器。不过序列化器并不支持 Pulsar 的 `Schema.AUTO_PRODUCE_BYTES()`。 diff --git a/docs/content/docs/connectors/datastream/pulsar.md b/docs/content/docs/connectors/datastream/pulsar.md index 4d616e5f69cf0..15115c673b10b 100644 --- a/docs/content/docs/connectors/datastream/pulsar.md +++ b/docs/content/docs/connectors/datastream/pulsar.md @@ -493,6 +493,18 @@ For example, when using the `PulsarSink.builder().setTopics("some-topic1", "some this is simplified to `PulsarSink.builder().setTopics("some-topic1")`. {{< /hint >}} +#### Dynamic Topics by income messages + +Topics could be defined by the message content instead of providing the fix topic set. You can dynamically +provide the topic by implementing `TopicExtractor`. The topic metadata in `TopicExtractor` can be queried +by using `TopicMetadataProvider` and the query result would be expired after we have queried for +`PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL` time. + +You can return two types of value in `TopicExtractor`. A topic name with or without partition information. + +1. If you don't want to provide the partition, we would query the partition size and passing all the partitions to `TopicRouter`. The partition size would be cached in `PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL`. +2. If you provided the topic partition, we would do nothing but just pass it to downstream. + ### Serializer A serializer (`PulsarSerializationSchema`) is required for serializing the record instance into bytes. diff --git a/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 b/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 index e69de29bb2d1d..36b34896247f5 100644 --- a/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 +++ b/flink-connectors/flink-connector-pulsar/archunit-violations/a2ce237e-b050-4ba0-8748-d83637a207a8 @@ -0,0 +1,6 @@ +org.apache.flink.connector.pulsar.sink.PulsarSinkITCase does not satisfy: only one of the following predicates match:\ +* reside in a package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type InternalMiniClusterExtension and annotated with @RegisterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and contain any fields that are static, final, and of type MiniClusterExtension and annotated with @RegisterExtension\ +* reside in a package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class InternalMiniClusterExtension\ +* reside outside of package 'org.apache.flink.runtime.*' and is annotated with @ExtendWith with class MiniClusterExtension\ + or contain any fields that are public, static, and of type MiniClusterWithClientResource and final and annotated with @ClassRule or contain any fields that is of type MiniClusterWithClientResource and public and final and not static and annotated with @Rule \ No newline at end of file diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index 77ad83e12263d..b452d0f7c3459 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -293,6 +293,20 @@ under the License. flink-architecture-tests-test test + + + + net.java.dev.jna + jna + 5.5.0 + test + + + net.java.dev.jna + jna-platform + 5.5.0 + test + diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java index 77c1077fdff6a..d952c5f016b32 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSink.java @@ -34,7 +34,7 @@ import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicMetadataListener; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; import org.apache.flink.core.io.SimpleVersionedSerializer; import org.apache.pulsar.client.api.CryptoKeyReader; @@ -86,7 +86,7 @@ public class PulsarSink implements TwoPhaseCommittingSink serializationSchema; - private final TopicMetadataListener metadataListener; + private final TopicRegister topicRegister; private final MessageDelayer messageDelayer; private final TopicRouter topicRouter; @@ -95,14 +95,14 @@ public class PulsarSink implements TwoPhaseCommittingSink serializationSchema, - TopicMetadataListener metadataListener, + TopicRegister topicRegister, TopicRoutingMode topicRoutingMode, TopicRouter topicRouter, MessageDelayer messageDelayer, @Nullable CryptoKeyReader cryptoKeyReader) { this.sinkConfiguration = checkNotNull(sinkConfiguration); this.serializationSchema = checkNotNull(serializationSchema); - this.metadataListener = checkNotNull(metadataListener); + this.topicRegister = checkNotNull(topicRegister); this.messageDelayer = checkNotNull(messageDelayer); this.cryptoKeyReader = cryptoKeyReader; checkNotNull(topicRoutingMode); @@ -133,7 +133,7 @@ public PrecommittingSinkWriter createWriter(InitContext i return new PulsarWriter<>( sinkConfiguration, serializationSchema, - metadataListener, + topicRegister, topicRouter, messageDelayer, cryptoKeyReader, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java index 94865e5541973..96ca03876a4fd 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java @@ -30,7 +30,11 @@ import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSchemaWrapper; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicMetadataListener; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.register.DynamicTopicRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.register.EmptyTopicRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.register.FixedTopicRegister; import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Schema; @@ -105,7 +109,7 @@ public class PulsarSinkBuilder { private final PulsarConfigBuilder configBuilder; private PulsarSerializationSchema serializationSchema; - private TopicMetadataListener metadataListener; + private TopicRegister topicRegister; private TopicRoutingMode topicRoutingMode; private TopicRouter topicRouter; private MessageDelayer messageDelayer; @@ -167,10 +171,26 @@ public PulsarSinkBuilder setTopics(String... topics) { * @return this PulsarSinkBuilder. */ public PulsarSinkBuilder setTopics(List topics) { - checkState(metadataListener == null, "setTopics couldn't be set twice."); + checkState(topicRegister == null, "setTopics couldn't be set twice."); // Making sure the topic should be distinct. List topicSet = distinctTopics(topics); - this.metadataListener = new TopicMetadataListener(topicSet); + if (topicSet.isEmpty()) { + this.topicRegister = new EmptyTopicRegister<>(); + } else { + this.topicRegister = new FixedTopicRegister<>(topicSet); + } + return this; + } + + /** + * Set a dynamic topic extractor for extracting the topic information. + * + * @return this PulsarSinkBuilder. + */ + public PulsarSinkBuilder setTopics(TopicExtractor extractor) { + checkState(topicRegister == null, "setTopics couldn't be set twice."); + this.topicRegister = new DynamicTopicRegister<>(extractor); + return this; } @@ -365,14 +385,14 @@ public PulsarSink build() { } // Topic metadata listener validation. - if (metadataListener == null) { + if (topicRegister == null) { if (topicRouter == null) { throw new NullPointerException( "No topic names or custom topic router are provided."); } else { LOG.warn( "No topic set has been provided, make sure your custom topic router support empty topic set."); - this.metadataListener = new TopicMetadataListener(); + this.topicRegister = new EmptyTopicRegister<>(); } } @@ -401,7 +421,7 @@ public PulsarSink build() { return new PulsarSink<>( sinkConfiguration, serializationSchema, - metadataListener, + topicRegister, topicRoutingMode, topicRouter, messageDelayer, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java index 360f742bfd0b1..b0ce536139cb1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java @@ -33,8 +33,8 @@ import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessage; import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicMetadataListener; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicProducerRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.ProducerRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; import org.apache.flink.util.FlinkRuntimeException; import org.apache.flink.shaded.guava30.com.google.common.base.Strings; @@ -70,13 +70,13 @@ public class PulsarWriter implements PrecommittingSinkWriter serializationSchema; - private final TopicMetadataListener metadataListener; + private final TopicRegister topicRegister; private final TopicRouter topicRouter; private final MessageDelayer messageDelayer; private final DeliveryGuarantee deliveryGuarantee; private final PulsarSinkContext sinkContext; private final MailboxExecutor mailboxExecutor; - private final TopicProducerRegister producerRegister; + private final ProducerRegister producerRegister; private long pendingMessages = 0; @@ -89,21 +89,21 @@ public class PulsarWriter implements PrecommittingSinkWriter serializationSchema, - TopicMetadataListener metadataListener, + TopicRegister topicRegister, TopicRouter topicRouter, MessageDelayer messageDelayer, @Nullable CryptoKeyReader cryptoKeyReader, InitContext initContext) { this.sinkConfiguration = checkNotNull(sinkConfiguration); this.serializationSchema = checkNotNull(serializationSchema); - this.metadataListener = checkNotNull(metadataListener); + this.topicRegister = checkNotNull(topicRegister); this.topicRouter = checkNotNull(topicRouter); this.messageDelayer = checkNotNull(messageDelayer); checkNotNull(initContext); @@ -115,7 +115,7 @@ public PulsarWriter( // Initialize topic metadata listener. LOG.debug("Initialize topic metadata after creating Pulsar writer."); ProcessingTimeService timeService = initContext.getProcessingTimeService(); - this.metadataListener.open(sinkConfiguration, timeService); + this.topicRegister.open(sinkConfiguration, timeService); // Initialize topic router. this.topicRouter.open(sinkConfiguration); @@ -130,7 +130,7 @@ public PulsarWriter( } // Create this producer register after opening serialization schema! - this.producerRegister = new TopicProducerRegister(sinkConfiguration, cryptoKeyReader); + this.producerRegister = new ProducerRegister(sinkConfiguration, cryptoKeyReader); } @Override @@ -138,16 +138,17 @@ public void write(IN element, Context context) throws IOException, InterruptedEx PulsarMessage message = serializationSchema.serialize(element, sinkContext); // Choose the right topic to send. + List topics = topicRegister.topics(element); - List availableTopics = metadataListener.availableTopics(); - String keyString; // TODO if both keyBytes and key are set, use keyBytes. This is a temporary solution. + String keyString; if (message.getKeyBytes() == null) { keyString = message.getKey(); } else { keyString = Base64.getEncoder().encodeToString(message.getKeyBytes()); } - String topic = topicRouter.route(element, keyString, availableTopics, sinkContext); + + String topic = topicRouter.route(element, keyString, topics, sinkContext); // Create message builder for sending message. TypedMessageBuilder builder = createMessageBuilder(topic, context, message); @@ -201,7 +202,7 @@ private void releasePermits() { this.pendingMessages -= 1; } - @SuppressWarnings("rawtypes") + @SuppressWarnings({"rawtypes", "unchecked"}) private TypedMessageBuilder createMessageBuilder( String topic, Context context, PulsarMessage message) { @@ -286,6 +287,6 @@ public Collection prepareCommit() { @Override public void close() throws Exception { // Close all the resources and throw the exception at last. - closeAll(metadataListener, producerRegister); + closeAll(topicRegister, producerRegister); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegister.java similarity index 94% rename from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegister.java rename to flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegister.java index e83c56c77773c..d25b8ef00521b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegister.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegister.java @@ -59,20 +59,20 @@ * we have to create different instances for different topics. */ @Internal -public class TopicProducerRegister implements Closeable { +public class ProducerRegister implements Closeable { private final PulsarClient pulsarClient; private final SinkConfiguration sinkConfiguration; @Nullable private final CryptoKeyReader cryptoKeyReader; - private final Map>> producerRegister; + private final Map>> register; private final Map transactionRegister; - public TopicProducerRegister( + public ProducerRegister( SinkConfiguration sinkConfiguration, @Nullable CryptoKeyReader cryptoKeyReader) { this.pulsarClient = createClient(sinkConfiguration); this.sinkConfiguration = sinkConfiguration; this.cryptoKeyReader = cryptoKeyReader; - this.producerRegister = new HashMap<>(); + this.register = new HashMap<>(); this.transactionRegister = new HashMap<>(); } @@ -116,7 +116,7 @@ public List prepareCommit() { * successfully persisted. */ public void flush() throws IOException { - Collection>> collection = producerRegister.values(); + Collection>> collection = register.values(); for (Map> producers : collection) { for (Producer producer : producers.values()) { producer.flush(); @@ -134,7 +134,7 @@ public void close() throws IOException { closer.register(this::abortTransactions); // Remove all the producers. - closer.register(producerRegister::clear); + closer.register(register::clear); // All the producers would be closed by this method. // We would block until all the producers have been successfully closed. @@ -146,7 +146,7 @@ public void close() throws IOException { @SuppressWarnings("unchecked") private Producer getOrCreateProducer(String topic, Schema schema) { Map> producers = - producerRegister.computeIfAbsent(topic, key -> new HashMap<>()); + register.computeIfAbsent(topic, key -> new HashMap<>()); SchemaInfo schemaInfo = schema.getSchemaInfo(); if (producers.containsKey(schemaInfo)) { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java new file mode 100644 index 0000000000000..4d8de8037a07a --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; + +import org.apache.pulsar.client.admin.PulsarAdmin; + +import java.io.Serializable; + +/** Choose topics from the message, used for dynamic generate topics in Pulsar sink. */ +@PublicEvolving +public interface TopicExtractor extends Serializable { + + /** + * @param in The message would be written to Pulsar. + * @param provider Used for query topic metadata. + * @return The topic you want to use. You can use both partitioned topic name or a topic name + * without partition information. We would query the partition information and pass it to + * {@link TopicRouter} if you return a topic name without partition information. + */ + TopicPartition extract(IN in, TopicMetadataProvider provider); + + /** + * A wrapper for {@link PulsarAdmin} instance, we won't expose the Pulsar admin interface for + * better control the abstraction. And add cache support. + */ + @PublicEvolving + interface TopicMetadataProvider { + + /** @throws Exception Failed to query Pulsar metadata would throw this exception. */ + TopicMetadata query(String topic) throws Exception; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicRegister.java new file mode 100644 index 0000000000000..925334923e72c --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicRegister.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; + +import java.io.Closeable; +import java.io.Serializable; +import java.util.List; + +/** The topic register for returning the available topic partitions. */ +@Internal +public interface TopicRegister extends Serializable, Closeable { + + /** + * Return all the available topic partitions. We would recalculate the partitions if the topic + * metadata has been changed. Otherwise, we would return the cached result for better + * performance. + */ + List topics(IN in); + + /** Register the topic metadata update in process time service. */ + void open(SinkConfiguration sinkConfiguration, ProcessingTimeService timeService); +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java new file mode 100644 index 0000000000000..a1a3e3ae14d00 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.register; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor.TopicMetadataProvider; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.util.FlinkRuntimeException; + +import org.apache.flink.shaded.guava30.com.google.common.cache.CacheBuilder; +import org.apache.flink.shaded.guava30.com.google.common.cache.CacheLoader; +import org.apache.flink.shaded.guava30.com.google.common.cache.LoadingCache; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; + +import static java.util.Collections.singletonList; +import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The register for returning dynamic topic partitions information. */ +@Internal +public class DynamicTopicRegister implements TopicRegister { + private static final long serialVersionUID = 4374769306761301456L; + + private final TopicExtractor topicExtractor; + + // Dynamic fields. + private transient PulsarAdmin pulsarAdmin; + private transient TopicMetadataProvider metadataProvider; + private transient LoadingCache> partitionsCache; + + public DynamicTopicRegister(TopicExtractor topicExtractor) { + this.topicExtractor = checkNotNull(topicExtractor); + } + + @Override + public List topics(IN in) { + TopicPartition partition = topicExtractor.extract(in, metadataProvider); + String topicName = partition.getFullTopicName(); + + if (partition.isPartition()) { + return singletonList(topicName); + } else { + try { + return partitionsCache.get(topicName); + } catch (ExecutionException e) { + throw new FlinkRuntimeException("Failed to query Pulsar topic partitions.", e); + } + } + } + + @Override + public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService timeService) { + long refreshInterval = sinkConfiguration.getTopicMetadataRefreshInterval(); + + // Initialize Pulsar admin instance. + this.pulsarAdmin = createAdmin(sinkConfiguration); + this.metadataProvider = new DefaultTopicMetadataProvider(pulsarAdmin, refreshInterval); + this.partitionsCache = + CacheBuilder.newBuilder() + .expireAfterWrite(refreshInterval, TimeUnit.MILLISECONDS) + .build( + new CacheLoader>() { + @Override + public List load(String topic) throws Exception { + TopicMetadata metadata = metadataProvider.query(topic); + if (metadata.isPartitioned()) { + int partitionSize = metadata.getPartitionSize(); + List partitions = + new ArrayList<>(partitionSize); + for (int i = 0; i < partitionSize; i++) { + partitions.add(topicNameWithPartition(topic, i)); + } + return partitions; + } else { + return singletonList(topic); + } + } + }); + } + + @Override + public void close() throws IOException { + if (pulsarAdmin != null) { + pulsarAdmin.close(); + } + } + + private static class DefaultTopicMetadataProvider implements TopicMetadataProvider { + + private final LoadingCache metadataCache; + + private DefaultTopicMetadataProvider(PulsarAdmin pulsarAdmin, long refreshInterval) { + this.metadataCache = + CacheBuilder.newBuilder() + .expireAfterWrite(refreshInterval, TimeUnit.MILLISECONDS) + .build( + new CacheLoader() { + @Override + public TopicMetadata load(String topic) throws Exception { + PartitionedTopicMetadata metadata = + pulsarAdmin + .topics() + .getPartitionedTopicMetadata(topic); + return new TopicMetadata(topic, metadata.partitions); + } + }); + } + + @Override + public TopicMetadata query(String topic) throws ExecutionException { + return metadataCache.get(topic); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/EmptyTopicRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/EmptyTopicRegister.java new file mode 100644 index 0000000000000..b350b3cac3471 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/EmptyTopicRegister.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.register; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +/** The topic register which would do nothing for just return an empty topic partitions. */ +@Internal +public class EmptyTopicRegister implements TopicRegister { + private static final long serialVersionUID = -9199261243659491097L; + + @Override + public List topics(IN in) { + return Collections.emptyList(); + } + + @Override + public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService timeService) { + // Nothing to do. + } + + @Override + public void close() throws IOException { + // Nothing to do. + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegister.java similarity index 89% rename from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java rename to flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegister.java index 040438f902349..ec0d45a4919de 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListener.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegister.java @@ -16,11 +16,12 @@ * limitations under the License. */ -package org.apache.flink.connector.pulsar.sink.writer.topic; +package org.apache.flink.connector.pulsar.sink.writer.topic.register; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; import org.apache.flink.shaded.guava30.com.google.common.base.Objects; import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; @@ -31,15 +32,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Closeable; import java.io.IOException; -import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import static java.util.Collections.emptyList; import static org.apache.flink.connector.pulsar.common.config.PulsarClientFactory.createAdmin; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.isPartitioned; @@ -53,10 +51,10 @@ * built-in logic. We use Flink's ProcessingTimer as the executor. */ @Internal -public class TopicMetadataListener implements Serializable, Closeable { +public class FixedTopicRegister implements TopicRegister { private static final long serialVersionUID = 6186948471557507522L; - private static final Logger LOG = LoggerFactory.getLogger(TopicMetadataListener.class); + private static final Logger LOG = LoggerFactory.getLogger(FixedTopicRegister.class); private final ImmutableList partitionedTopics; private final Map topicMetadata; @@ -67,11 +65,7 @@ public class TopicMetadataListener implements Serializable, Closeable { private transient Long topicMetadataRefreshInterval; private transient ProcessingTimeService timeService; - public TopicMetadataListener() { - this(emptyList()); - } - - public TopicMetadataListener(List topics) { + public FixedTopicRegister(List topics) { List partitions = new ArrayList<>(topics.size()); Map metadata = new HashMap<>(topics.size()); for (String topic : topics) { @@ -88,7 +82,7 @@ public TopicMetadataListener(List topics) { this.availableTopics = ImmutableList.of(); } - /** Register the topic metadata update in process time service. */ + @Override public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService timeService) { if (topicMetadata.isEmpty()) { LOG.info("No topics have been provided, skip listener initialize."); @@ -107,12 +101,8 @@ public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService time triggerNextTopicMetadataUpdate(true); } - /** - * Return all the available topic partitions. We would recalculate the partitions if the topic - * metadata has been changed. Otherwise, we would return the cached result for better - * performance. - */ - public List availableTopics() { + @Override + public List topics(IN in) { if (availableTopics.isEmpty() && (!partitionedTopics.isEmpty() || !topicMetadata.isEmpty())) { List results = new ArrayList<>(); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java index e46e1906d63f5..769585182e254 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java @@ -73,6 +73,7 @@ public String toString() { /** * The position type for reader to choose whether timestamp or message id as the start position. */ + @PublicEvolving public enum Type { TIMESTAMP, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java index 57d5f02ed24c0..f9ff0618215b8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/subscriber/impl/BasePulsarSubscriber.java @@ -27,11 +27,12 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import java.util.ArrayList; import java.util.List; -import java.util.stream.IntStream; import static java.util.stream.Collectors.toList; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition.NON_PARTITION_ID; /** PulsarSubscriber abstract class to simplify Pulsar admin related operations. */ public abstract class BasePulsarSubscriber implements PulsarSubscriber { @@ -60,21 +61,18 @@ protected List toTopicPartitions( if (!metadata.isPartitioned()) { // For non-partitioned topic. return ranges.stream() - .map(range -> new TopicPartition(metadata.getName(), -1, range)) + .map(range -> new TopicPartition(metadata.getName(), NON_PARTITION_ID, range)) .collect(toList()); } else { - return IntStream.range(0, metadata.getPartitionSize()) - .boxed() - .flatMap( - partitionId -> - ranges.stream() - .map( - range -> - new TopicPartition( - metadata.getName(), - partitionId, - range))) - .collect(toList()); + List partitions = new ArrayList<>(); + for (int i = 0; i < metadata.getPartitionSize(); i++) { + for (TopicRange range : ranges) { + TopicPartition partition = new TopicPartition(metadata.getName(), i, range); + partitions.add(partition); + } + } + + return partitions; } } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java index b3035cde8485e..64901e24d303c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java @@ -29,6 +29,7 @@ import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -39,6 +40,12 @@ public class TopicPartition implements Serializable { private static final long serialVersionUID = -1474354741550810953L; + /** + * If {@link TopicPartition#getPartitionId()} is equal to this. This topic partition wouldn't be + * a partition instance. It would be a top topic name. + */ + public static final int NON_PARTITION_ID = -1; + /** * The topic name of the pulsar. It would be a full topic name, if your don't provide the tenant * and namespace, we would add them automatically. @@ -58,6 +65,16 @@ public class TopicPartition implements Serializable { */ private final TopicRange range; + /** Create a top level topic without partition information. */ + public TopicPartition(String topic) { + this(topic, NON_PARTITION_ID); + } + + /** Create a topic partition without key hash range. */ + public TopicPartition(String topic, int partitionId) { + this(topic, partitionId, createFullRange()); + } + public TopicPartition(String topic, int partitionId, TopicRange range) { this.topic = topicName(checkNotNull(topic)); this.partitionId = partitionId; @@ -72,12 +89,16 @@ public int getPartitionId() { return partitionId; } + public boolean isPartition() { + return partitionId != NON_PARTITION_ID; + } + /** * Pulsar split the topic partition into a bunch of small topics, we would get the real topic * name by using this method. */ public String getFullTopicName() { - if (partitionId >= 0) { + if (isPartition()) { return topicNameWithPartition(topic, partitionId); } else { return topic; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java index 1508b8732a311..09d4882ffe9da 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicRange.java @@ -48,6 +48,9 @@ public class TopicRange implements Serializable { /** The end position for hash range, it's 65535. */ public static final int MAX_RANGE = RANGE_SIZE - 1; + /** A full topic range instance for avoiding multiple instance creation. */ + private static final TopicRange FULL_RANGE = new TopicRange(MIN_RANGE, MAX_RANGE); + /** The start of the range, default is zero. */ private final int start; @@ -70,7 +73,7 @@ public Range toPulsarRange() { /** Create a topic range which contains the full hash range. */ public static TopicRange createFullRange() { - return new TopicRange(MIN_RANGE, MAX_RANGE); + return FULL_RANGE; } public int getStart() { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java index f71facd994aa6..9fc272a7338c8 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriterTest.java @@ -31,7 +31,7 @@ import org.apache.flink.connector.pulsar.sink.writer.delayer.MessageDelayer; import org.apache.flink.connector.pulsar.sink.writer.router.RoundRobinTopicRouter; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicMetadataListener; +import org.apache.flink.connector.pulsar.sink.writer.topic.register.FixedTopicRegister; import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.OperatorIOMetricGroup; @@ -80,7 +80,7 @@ void writeMessageWithoutGuarantee(DeliveryGuarantee guarantee) throws Exception SinkConfiguration configuration = sinkConfiguration(guarantee); PulsarSerializationSchema schema = pulsarSchema(STRING); - TopicMetadataListener listener = new TopicMetadataListener(singletonList(topic)); + FixedTopicRegister listener = new FixedTopicRegister(singletonList(topic)); RoundRobinTopicRouter router = new RoundRobinTopicRouter<>(configuration); FixedMessageDelayer delayer = MessageDelayer.never(); MockInitContext initContext = new MockInitContext(); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegisterTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegisterTest.java similarity index 92% rename from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegisterTest.java rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegisterTest.java index fa4a1f5109ebd..135ce8779b3f3 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicProducerRegisterTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/ProducerRegisterTest.java @@ -38,8 +38,8 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; -/** Unit tests for {@link TopicProducerRegister}. */ -class TopicProducerRegisterTest extends PulsarTestSuiteBase { +/** Unit tests for {@link ProducerRegister}. */ +class ProducerRegisterTest extends PulsarTestSuiteBase { @ParameterizedTest @EnumSource(DeliveryGuarantee.class) @@ -49,7 +49,7 @@ void createMessageBuilderForSendingMessage(DeliveryGuarantee deliveryGuarantee) operator().createTopic(topic, 8); SinkConfiguration configuration = sinkConfiguration(deliveryGuarantee); - TopicProducerRegister register = new TopicProducerRegister(configuration, null); + ProducerRegister register = new ProducerRegister(configuration, null); String message = randomAlphabetic(10); register.createMessageBuilder(topic, Schema.STRING).value(message).send(); @@ -76,7 +76,7 @@ void noneAndAtLeastOnceWouldNotCreateTransaction(DeliveryGuarantee deliveryGuara operator().createTopic(topic, 8); SinkConfiguration configuration = sinkConfiguration(deliveryGuarantee); - TopicProducerRegister register = new TopicProducerRegister(configuration, null); + ProducerRegister register = new ProducerRegister(configuration, null); String message = randomAlphabetic(10); register.createMessageBuilder(topic, Schema.STRING).value(message).sendAsync(); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegisterTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegisterTest.java new file mode 100644 index 0000000000000..c5290d6bf78a8 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegisterTest.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.register; + +import org.apache.flink.api.common.operators.ProcessingTimeService; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; + +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.List; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; + +/** Unit tests for {@link DynamicTopicRegister}. */ +class DynamicTopicRegisterTest extends PulsarTestSuiteBase { + + private final MockTopicExtractor extractor = new MockTopicExtractor(); + + private static final class MockTopicExtractor implements TopicExtractor { + private static final long serialVersionUID = 2456172645787498006L; + + private TopicPartition partition; + + @Override + public TopicPartition extract(String s, TopicMetadataProvider provider) { + return partition; + } + + public void setPartition(TopicPartition partition) { + this.partition = partition; + } + } + + @Test + void partitionedTopicWouldBeReturnedDirectly() throws IOException { + DynamicTopicRegister register = topicRegister(50000); + TopicPartition partition = new TopicPartition("some", 1); + extractor.setPartition(partition); + List topics = register.topics(randomAlphabetic(10)); + + assertThat(topics) + .hasSize(1) + .allSatisfy(topic -> assertThat(topic).isEqualTo(partition.getFullTopicName())); + + register.close(); + } + + @Test + void rootTopicWillReturnAllThePartitions() throws IOException { + DynamicTopicRegister register = topicRegister(50000); + TopicPartition partition = new TopicPartition("root-topic" + randomAlphabetic(10)); + extractor.setPartition(partition); + operator().createTopic(partition.getFullTopicName(), 10); + List topics = register.topics(randomAlphabetic(10)); + + assertThat(topics) + .hasSize(10) + .allSatisfy(topic -> assertThat(topic).startsWith(partition.getTopic())); + + register.close(); + } + + private DynamicTopicRegister topicRegister(long interval) { + DynamicTopicRegister register = new DynamicTopicRegister<>(extractor); + register.open(sinkConfiguration(interval), mock(ProcessingTimeService.class)); + + return register; + } + + private SinkConfiguration sinkConfiguration(long interval) { + Configuration configuration = operator().config(); + configuration.set(PULSAR_TOPIC_METADATA_REFRESH_INTERVAL, interval); + + return new SinkConfiguration(configuration); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListenerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegisterTest.java similarity index 78% rename from flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListenerTest.java rename to flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegisterTest.java index b34d229c8feac..b0ee0c5ddc7cc 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicMetadataListenerTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegisterTest.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.flink.connector.pulsar.sink.writer.topic; +package org.apache.flink.connector.pulsar.sink.writer.topic.register; import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; @@ -25,11 +25,13 @@ import org.junit.jupiter.api.Test; +import java.io.IOException; import java.time.Duration; import java.util.Collections; import java.util.List; import java.util.stream.IntStream; +import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static java.util.stream.Collectors.toList; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; @@ -39,21 +41,23 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; -/** Unit tests for {@link TopicMetadataListener}. */ -class TopicMetadataListenerTest extends PulsarTestSuiteBase { +/** Unit tests for {@link FixedTopicRegister}. */ +class FixedTopicRegisterTest extends PulsarTestSuiteBase { @Test - void listenEmptyTopics() { - TopicMetadataListener listener = new TopicMetadataListener(); + void listenEmptyTopics() throws IOException { + FixedTopicRegister listener = new FixedTopicRegister<>(emptyList()); SinkConfiguration configuration = sinkConfiguration(Duration.ofMinutes(5).toMillis()); TestProcessingTimeService timeService = new TestProcessingTimeService(); - List topics = listener.availableTopics(); + List topics = listener.topics(""); assertThat(topics).isEmpty(); listener.open(configuration, timeService); - topics = listener.availableTopics(); + topics = listener.topics(""); assertThat(topics).isEmpty(); + + listener.close(); } @Test @@ -62,41 +66,45 @@ void listenOnPartitions() throws Exception { operator().createTopic(topic, 6); List partitions = topicPartitions(topic, 6); - TopicMetadataListener listener = new TopicMetadataListener(partitions); + FixedTopicRegister listener = new FixedTopicRegister<>(partitions); long interval = Duration.ofMinutes(15).toMillis(); SinkConfiguration configuration = sinkConfiguration(interval); TestProcessingTimeService timeService = new TestProcessingTimeService(); - List topics = listener.availableTopics(); + List topics = listener.topics(""); assertEquals(topics, partitions); listener.open(configuration, timeService); - topics = listener.availableTopics(); + topics = listener.topics(""); assertEquals(topics, partitions); operator().increaseTopicPartitions(topic, 12); timeService.advance(interval); - topics = listener.availableTopics(); + topics = listener.topics(""); assertEquals(topics, partitions); + + listener.close(); } @Test - void fetchTopicPartitionInformation() { + void fetchTopicPartitionInformation() throws IOException { String topic = randomAlphabetic(10); operator().createTopic(topic, 8); - TopicMetadataListener listener = new TopicMetadataListener(singletonList(topic)); + FixedTopicRegister listener = new FixedTopicRegister<>(singletonList(topic)); SinkConfiguration configuration = sinkConfiguration(Duration.ofMinutes(10).toMillis()); TestProcessingTimeService timeService = new TestProcessingTimeService(); - List topics = listener.availableTopics(); + List topics = listener.topics(""); assertThat(topics).isEmpty(); listener.open(configuration, timeService); - topics = listener.availableTopics(); + topics = listener.topics(""); List desiredTopics = topicPartitions(topic, 8); assertThat(topics).hasSize(8).isEqualTo(desiredTopics); + + listener.close(); } @Test @@ -106,13 +114,13 @@ void fetchTopicPartitionUpdate() throws Exception { long interval = Duration.ofMinutes(20).toMillis(); - TopicMetadataListener listener = new TopicMetadataListener(singletonList(topic)); + FixedTopicRegister listener = new FixedTopicRegister<>(singletonList(topic)); SinkConfiguration configuration = sinkConfiguration(interval); TestProcessingTimeService timeService = new TestProcessingTimeService(); timeService.setCurrentTime(System.currentTimeMillis()); listener.open(configuration, timeService); - List topics = listener.availableTopics(); + List topics = listener.topics(""); List desiredTopics = topicPartitions(topic, 8); assertThat(topics).isEqualTo(desiredTopics); @@ -121,25 +129,29 @@ void fetchTopicPartitionUpdate() throws Exception { operator().increaseTopicPartitions(topic, 16); timeService.advance(interval); - topics = listener.availableTopics(); + topics = listener.topics(""); desiredTopics = topicPartitions(topic, 16); assertThat(topics).isEqualTo(desiredTopics); + + listener.close(); } @Test - void fetchNonPartitionTopic() { + void fetchNonPartitionTopic() throws IOException { String topic = randomAlphabetic(10); operator().createTopic(topic, 0); List nonPartitionTopic = Collections.singletonList(topicName(topic)); - TopicMetadataListener listener = new TopicMetadataListener(nonPartitionTopic); + FixedTopicRegister listener = new FixedTopicRegister<>(nonPartitionTopic); long interval = Duration.ofMinutes(15).toMillis(); SinkConfiguration configuration = sinkConfiguration(interval); TestProcessingTimeService timeService = new TestProcessingTimeService(); listener.open(configuration, timeService); - List topics = listener.availableTopics(); + List topics = listener.topics(""); assertEquals(topics, nonPartitionTopic); + + listener.close(); } private List topicPartitions(String topic, int partitionSize) { diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java index 9a1aed78b3c76..2a2d4d34ecdfc 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/container/PulsarContainerRuntime.java @@ -56,9 +56,6 @@ public class PulsarContainerRuntime implements PulsarRuntime { private static final String PULSAR_ADMIN_URL = String.format("http://%s:%d", PULSAR_INTERNAL_HOSTNAME, BROKER_HTTP_PORT); - private static final String TXN_CONFIG_FILE = "containers/txnStandalone.conf"; - private static final String AUTH_CONFIG_FILE = "containers/authStandalone.conf"; - /** * Create a pulsar container provider by a predefined version, this constance {@link * DockerImageVersions#PULSAR} should be bumped after the new pulsar release. @@ -69,14 +66,9 @@ public class PulsarContainerRuntime implements PulsarRuntime { private boolean boundFlink = false; private PulsarRuntimeOperator operator; - private String configFile; public PulsarContainerRuntime(boolean authEnabled) { - if (authEnabled) { - configFile = AUTH_CONFIG_FILE; - } else { - configFile = TXN_CONFIG_FILE; - } + // TODO Add authentication support. } public PulsarContainerRuntime bindWithFlinkContainer(GenericContainer flinkContainer) { @@ -110,7 +102,7 @@ public void startUp() { .forStatusCode(200) .withStartupTimeout(Duration.ofMinutes(5))); // Set custom startup script. - container.withCommand("/pulsar/bin/bootstrap.sh"); + container.withCommand("sh /pulsar/bin/bootstrap.sh"); // Start the Pulsar Container. container.start(); From 87de6e75562bab7a41ed51acdce57edc5caadc99 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Tue, 5 Jul 2022 21:15:18 +0800 Subject: [PATCH 227/258] feat: Add open method for dynamic topic extractor. (#130) --- .../connector/pulsar/sink/writer/topic/TopicExtractor.java | 6 ++++++ .../sink/writer/topic/register/DynamicTopicRegister.java | 2 ++ 2 files changed, 8 insertions(+) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java index 4d8de8037a07a..32f8bf913cbe6 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java @@ -19,6 +19,7 @@ package org.apache.flink.connector.pulsar.sink.writer.topic; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; @@ -40,6 +41,11 @@ public interface TopicExtractor extends Serializable { */ TopicPartition extract(IN in, TopicMetadataProvider provider); + /** Implement this method if you have some non-serializable field. */ + default void open(SinkConfiguration sinkConfiguration) { + // Nothing to do by default. + } + /** * A wrapper for {@link PulsarAdmin} instance, we won't expose the Pulsar admin interface for * better control the abstraction. And add cache support. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java index a1a3e3ae14d00..616ecda6c2b68 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java @@ -106,6 +106,8 @@ public List load(String topic) throws Exception { } } }); + // Open the topic extractor instance. + topicExtractor.open(sinkConfiguration); } @Override From 2992c0ef1d88374419d88052a59532e57f147a68 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 5 Jul 2022 22:52:17 +0800 Subject: [PATCH 228/258] fix: always delete topic even when there are subscribers --- .../table/catalog/impl/PulsarAdminTool.java | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java index 39024eb4ac455..f70a2f5c2f379 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/PulsarAdminTool.java @@ -27,8 +27,6 @@ import org.apache.pulsar.common.naming.NamespaceName; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.partition.PartitionedTopicMetadata; -import org.apache.pulsar.common.policies.data.PartitionedTopicInternalStats; -import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; import org.apache.pulsar.common.policies.data.RetentionPolicies; import org.apache.pulsar.common.policies.data.TenantInfoImpl; import org.apache.pulsar.common.schema.SchemaInfo; @@ -37,7 +35,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -150,22 +147,8 @@ public boolean topicExists(String topicName) throws PulsarAdminException { public void deleteTopic(String topicName) throws PulsarAdminException { try { - PartitionedTopicInternalStats partitionedInternalStats = - admin.topics().getPartitionedInternalStats(topicName); - final Optional any = - partitionedInternalStats.partitions.entrySet().stream() - .map(Map.Entry::getValue) - .filter(p -> !p.cursors.isEmpty()) - .findAny(); - if (any.isPresent()) { - throw new IllegalStateException( - String.format( - "The topic[%s] cannot be deleted because there are subscribers", - topicName)); - } admin.topics().deletePartitionedTopic(topicName, true); } catch (PulsarAdminException.NotFoundException e) { - // log.warn("topic<{}> is not exit, try delete force it", topicName); admin.topics().delete(topicName, true); } } From 3c81f7a785cbab0670832981ca8a7bcf99bac890 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Thu, 23 Jun 2022 12:59:03 +0800 Subject: [PATCH 229/258] 121: add new tests for documentation 100: timestamp should be larger than 0 --- .../pulsar/sink/writer/PulsarWriter.java | 2 +- .../pulsar/table/PulsarTableITCase.java | 310 +++++++++++++++--- .../table/catalog/PulsarCatalogITTest.java | 223 ++++++------- .../pulsar/table/testutils/TestingUser.java | 17 + 4 files changed, 388 insertions(+), 164 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java index b0ce536139cb1..c0ebc926ee8fa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java @@ -230,7 +230,7 @@ private TypedMessageBuilder createMessageBuilder( } else { // Set default message timestamp if flink has provided one. Long timestamp = context.timestamp(); - if (timestamp != null) { + if (timestamp != null && timestamp > 0) { builder.eventTime(timestamp); } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java index 46218572f6735..3bdde02f8fe9c 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -18,14 +18,21 @@ package org.apache.flink.connector.pulsar.table; +import org.apache.flink.connector.pulsar.table.testutils.TestingUser; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.functions.sink.SinkFunction; +import org.apache.flink.table.api.DataTypes; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.test.util.SuccessException; import org.apache.flink.types.Row; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Schema; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; import java.time.LocalDateTime; @@ -33,9 +40,12 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.stream.Stream; +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows; +import static org.apache.flink.connector.pulsar.table.testutils.TestingUser.createRandomUser; import static org.apache.flink.util.CollectionUtil.entry; import static org.apache.flink.util.CollectionUtil.map; import static org.assertj.core.api.Assertions.assertThat; @@ -50,20 +60,21 @@ public class PulsarTableITCase extends PulsarTableTestBase { private static final String JSON_FORMAT = "json"; private static final String AVRO_FORMAT = "avro"; private static final String CSV_FORMAT = "csv"; + private static final String RAW_FORMAT = "raw"; @ParameterizedTest - @ValueSource(strings = {JSON_FORMAT}) - public void pulsarSourceSink(String format) throws Exception { + @ValueSource(strings = {JSON_FORMAT, AVRO_FORMAT, CSV_FORMAT}) + void pulsarSourceSink(String format) throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "test_topic_" + format + randomAlphanumeric(3); createTestTopic(topic, 1); // ---------- Produce an event time stream into Pulsar ------------------- - + String randomTableName = randomAlphabetic(5); final String createTable = String.format( - "create table pulsar_source_sink (\n" + "create table %s (\n" + " `computed-price` as price + 1.0,\n" + " price decimal(38, 18),\n" + " currency string,\n" @@ -77,39 +88,44 @@ public void pulsarSourceSink(String format) throws Exception { + " 'topics' = '%s',\n" + " 'service-url' = '%s',\n" + " 'admin-url' = '%s',\n" - + " %s\n" + + " 'format' = '%s'\n" + ")", + randomTableName, PulsarTableFactory.IDENTIFIER, topic, pulsar.operator().serviceUrl(), pulsar.operator().adminUrl(), - formatOptions(format)); + format); tableEnv.executeSql(createTable); String initialValues = - "INSERT INTO pulsar_source_sink\n" - + "SELECT CAST(price AS DECIMAL(10, 2)), currency, " - + " CAST(d AS DATE), CAST(t AS TIME(0)), CAST(ts AS TIMESTAMP(3))\n" - + "FROM (VALUES (2.02,'Euro','2019-12-12', '00:00:01', '2019-12-12 00:00:01.001001'), \n" - + " (1.11,'US Dollar','2019-12-12', '00:00:02', '2019-12-12 00:00:02.002001'), \n" - + " (50,'Yen','2019-12-12', '00:00:03', '2019-12-12 00:00:03.004001'), \n" - + " (3.1,'Euro','2019-12-12', '00:00:04', '2019-12-12 00:00:04.005001'), \n" - + " (5.33,'US Dollar','2019-12-12', '00:00:05', '2019-12-12 00:00:05.006001'), \n" - + " (0,'DUMMY','2019-12-12', '00:00:10', '2019-12-12 00:00:10'))\n" - + " AS orders (price, currency, d, t, ts)"; + String.format( + "INSERT INTO %s\n" + + "SELECT CAST(price AS DECIMAL(10, 2)), currency, " + + " CAST(d AS DATE), CAST(t AS TIME(0)), CAST(ts AS TIMESTAMP(3))\n" + + "FROM (VALUES (2.02,'Euro','2019-12-12', '00:00:01', '2019-12-12 00:00:01.001001'), \n" + + " (1.11,'US Dollar','2019-12-12', '00:00:02', '2019-12-12 00:00:02.002001'), \n" + + " (50,'Yen','2019-12-12', '00:00:03', '2019-12-12 00:00:03.004001'), \n" + + " (3.1,'Euro','2019-12-12', '00:00:04', '2019-12-12 00:00:04.005001'), \n" + + " (5.33,'US Dollar','2019-12-12', '00:00:05', '2019-12-12 00:00:05.006001'), \n" + + " (0,'DUMMY','2019-12-12', '00:00:10', '2019-12-12 00:00:10'))\n" + + " AS orders (price, currency, d, t, ts)", + randomTableName); tableEnv.executeSql(initialValues).await(); String query = - "SELECT\n" - + " CAST(TUMBLE_END(ts, INTERVAL '5' SECOND) AS VARCHAR),\n" - + " CAST(MAX(log_date) AS VARCHAR),\n" - + " CAST(MAX(log_time) AS VARCHAR),\n" - + " CAST(MAX(ts) AS VARCHAR),\n" - + " COUNT(*),\n" - + " CAST(MAX(price) AS DECIMAL(10, 2))\n" - + "FROM pulsar_source_sink\n" - + "GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)"; + String.format( + "SELECT\n" + + " CAST(TUMBLE_END(ts, INTERVAL '5' SECOND) AS VARCHAR),\n" + + " CAST(MAX(log_date) AS VARCHAR),\n" + + " CAST(MAX(log_time) AS VARCHAR),\n" + + " CAST(MAX(ts) AS VARCHAR),\n" + + " COUNT(*),\n" + + " CAST(MAX(price) AS DECIMAL(10, 2))\n" + + "FROM %s\n" + + "GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)", + randomTableName); DataStream result = tableEnv.toDataStream(tableEnv.sqlQuery(query)); TestingSinkFunction sink = new TestingSinkFunction(2); @@ -133,8 +149,8 @@ public void pulsarSourceSink(String format) throws Exception { } @ParameterizedTest - @ValueSource(strings = {JSON_FORMAT}) - public void pulsarSourceSinkWithKeyAndPartialValue(String format) throws Exception { + @ValueSource(strings = {JSON_FORMAT, CSV_FORMAT}) + void pulsarSourceSinkWithKeyAndPartialValue(String format) throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "key_partial_value_topic_" + format + randomAlphanumeric(3); @@ -144,9 +160,10 @@ public void pulsarSourceSinkWithKeyAndPartialValue(String format) throws Excepti // k_user_id and user_id have different data types to verify the correct mapping, // fields are reordered on purpose + String randomTableName = randomAlphabetic(5); final String createTable = String.format( - "CREATE TABLE pulsar_key_value (\n" + "CREATE TABLE %s (\n" + " `user_id` BIGINT,\n" + " `name` STRING,\n" + " `event_id` BIGINT,\n" @@ -160,6 +177,7 @@ public void pulsarSourceSinkWithKeyAndPartialValue(String format) throws Excepti + " 'key.format' = '%s',\n" + " 'key.fields' = 'user_id; event_id'\n" + ")", + randomTableName, PulsarTableFactory.IDENTIFIER, topic, pulsar.operator().serviceUrl(), @@ -170,15 +188,18 @@ public void pulsarSourceSinkWithKeyAndPartialValue(String format) throws Excepti tableEnv.executeSql(createTable); String initialValues = - "INSERT INTO pulsar_key_value\n" - + "VALUES\n" - + " (1, 'name 1', 100, 'payload 1'),\n" - + " (2, 'name 2', 101, 'payload 2'),\n" - + " (3, 'name 3', 102, 'payload 3')"; + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " (1, 'name 1', 100, 'payload 1'),\n" + + " (2, 'name 2', 101, 'payload 2'),\n" + + " (3, 'name 3', 102, 'payload 3')", + randomTableName); tableEnv.executeSql(initialValues).await(); final List result = - collectRows(tableEnv.sqlQuery("SELECT * FROM pulsar_key_value"), 3); + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", randomTableName)), 3); assertThat(result) .containsExactlyInAnyOrder( @@ -188,19 +209,19 @@ public void pulsarSourceSinkWithKeyAndPartialValue(String format) throws Excepti } @ParameterizedTest - @ValueSource(strings = {JSON_FORMAT}) - public void pulsarSourceSinkWithMetadata(String format) throws Exception { + @ValueSource(strings = {JSON_FORMAT, AVRO_FORMAT, CSV_FORMAT}) + void pulsarSourceSinkWithMetadata(String format) throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "metadata_topic_" + format + randomAlphanumeric(3); createTestTopic(topic, 1); + String randomTableName = randomAlphabetic(5); final String createTable = String.format( - "CREATE TABLE pulsar_metadata (\n" + "CREATE TABLE %s (\n" + " `physical_1` STRING,\n" + " `physical_2` INT,\n" - + " `message_size` INT METADATA VIRTUAL,\n" + " `event_time` TIMESTAMP(3) METADATA,\n" + " `properties` MAP METADATA,\n" + " `physical_3` BOOLEAN\n" @@ -212,6 +233,7 @@ public void pulsarSourceSinkWithMetadata(String format) throws Exception { + " 'pulsar.producer.producerName' = 'pulsar-table-test',\n" + " 'format' = '%s'\n" + ")", + randomTableName, PulsarTableFactory.IDENTIFIER, topic, pulsar.operator().serviceUrl(), @@ -220,41 +242,205 @@ public void pulsarSourceSinkWithMetadata(String format) throws Exception { tableEnv.executeSql(createTable); String initialValues = - "INSERT INTO pulsar_metadata\n" - + "VALUES\n" - + " ('data 1', 1, TIMESTAMP '2022-03-24 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE),\n" - + " ('data 2', 2, TIMESTAMP '2022-03-25 13:12:11.123', CAST(NULL AS MAP), FALSE),\n" - + " ('data 3', 3, TIMESTAMP '2022-03-26 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE)"; + String.format( + "INSERT INTO %s\n" + + "VALUES\n" + + " ('data 1', 1, TIMESTAMP '2022-03-24 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE),\n" + + " ('data 2', 2, TIMESTAMP '2022-03-25 13:12:11.123', CAST(NULL AS MAP), FALSE),\n" + + " ('data 3', 3, TIMESTAMP '2022-03-26 13:12:11.123', MAP['k1', 'C0FFEE', 'k2', 'BABE01'], TRUE)", + randomTableName); tableEnv.executeSql(initialValues).await(); // ---------- Consume stream from Pulsar ------------------- - final List result = collectRows(tableEnv.sqlQuery("SELECT * FROM pulsar_metadata"), 3); + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", randomTableName)), 3); assertThat(result) .containsExactlyInAnyOrder( Row.of( "data 1", 1, - 56, LocalDateTime.parse("2022-03-24T13:12:11.123"), map(entry("k1", "C0FFEE"), entry("k2", "BABE01")), true), Row.of( "data 2", 2, - 57, LocalDateTime.parse("2022-03-25T13:12:11.123"), Collections.emptyMap(), false), Row.of( "data 3", 3, - 56, LocalDateTime.parse("2022-03-26T13:12:11.123"), map(entry("k1", "C0FFEE"), entry("k2", "BABE01")), true)); } + // TODO split this into two tests + @ParameterizedTest + @MethodSource("provideSchemaData") + void readAndSelectIntoTableUsingSimpleSchema( + String format, Schema schema, T value, String flinkTableDataType) throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, schema, value); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " `field_1` %s\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + flinkTableDataType, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSourceTable); + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1); + assertThat(result).containsExactlyInAnyOrder(Row.of(value)); + + // insert into ... select from + + final String sinkTopic = "sink_topic_" + randomAlphanumeric(3); + createTestTopic(sinkTopic, 1); + + String sinkTableName = randomAlphabetic(5); + pulsar.operator().sendMessage(sourceTopic, schema, value); + final String createSinkTable = + String.format( + "create table %s (\n" + + " `field_1` %s\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sinkTableName, + flinkTableDataType, + PulsarTableFactory.IDENTIFIER, + sinkTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSinkTable); + tableEnv.executeSql( + String.format("INSERT INTO %s SELECT * FROM %s", sinkTableName, sourceTableName)); + Message sinkResult = pulsar.operator().receiveMessage(sinkTopic, schema); + assertThat(sinkResult.getValue()).isEqualTo(value); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + @Disabled("flink-128") + void sendMessageToTopicAndReadUsingAvroBasedSchema( + String format, Schema schema, TestingUser value) throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, schema, value); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING,\n" + + " age INT\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSourceTable); + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1); + assertThat(result).containsExactlyInAnyOrder(Row.of(value.getName(), value.getAge())); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void selectIntoTableUsingAvroBasedSchema( + String format, Schema schema, TestingUser value) throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, schema, value); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " age INT,\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSourceTable); + + final String sinkTopic = "sink_topic_" + randomAlphanumeric(3); + createTestTopic(sinkTopic, 1); + String sinkTableName = randomAlphabetic(5); + final String createSinkTable = + String.format( + "create table %s (\n" + + " age INT,\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sinkTableName, + PulsarTableFactory.IDENTIFIER, + sinkTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + format); + + tableEnv.executeSql(createSinkTable); + tableEnv.executeSql( + String.format("INSERT INTO %s SELECT * FROM %s", sinkTableName, sourceTableName)); + Message sinkResult = pulsar.operator().receiveMessage(sinkTopic, schema); + assertThat(sinkResult.getValue()).isEqualTo(value); + } + private static final class TestingSinkFunction implements SinkFunction { private static final long serialVersionUID = 455430015321124493L; @@ -287,7 +473,33 @@ private static boolean isCausedByJobFinished(Throwable e) { } } - private String formatOptions(String format) { - return String.format("'format' = '%s'", format); + private static Stream provideSchemaData() { + return Stream.of( + Arguments.of(RAW_FORMAT, Schema.INT8, (byte) 0xa, DataTypes.TINYINT().toString()), + Arguments.of( + RAW_FORMAT, Schema.INT16, Short.MAX_VALUE, DataTypes.SMALLINT().toString()), + Arguments.of( + RAW_FORMAT, Schema.INT32, Integer.MAX_VALUE, DataTypes.INT().toString()), + Arguments.of( + RAW_FORMAT, Schema.INT64, Long.MAX_VALUE, DataTypes.BIGINT().toString()), + Arguments.of( + RAW_FORMAT, Schema.FLOAT, Float.MAX_VALUE, DataTypes.FLOAT().toString()), + Arguments.of( + RAW_FORMAT, Schema.DOUBLE, Double.MAX_VALUE, DataTypes.DOUBLE().toString()), + Arguments.of(RAW_FORMAT, Schema.BOOL, Boolean.TRUE, DataTypes.BOOLEAN().toString()), + Arguments.of(RAW_FORMAT, Schema.BYTES, new byte[1], DataTypes.BYTES().toString()), + Arguments.of( + RAW_FORMAT, + Schema.STRING, + "this is a string", + DataTypes.STRING().toString())); + } + + private static Stream provideAvroBasedSchemaData() { + return Stream.of( + Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class), createRandomUser()) + // Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class), + // createRandomUser()) + ); } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java index 4f9ebae2ae5f0..4cd9adae235a6 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java @@ -25,23 +25,21 @@ import org.apache.flink.shaded.guava30.com.google.common.collect.Iterables; import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; -import org.apache.commons.io.IOUtils; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.client.api.Consumer; import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.common.naming.TopicName; import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.shade.org.apache.commons.lang3.RandomStringUtils; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -49,17 +47,16 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; +import java.util.stream.Stream; -import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactory.CATALOG_CONFIG_VALIDATOR; import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows; import static org.apache.flink.connector.pulsar.table.testutils.SchemaData.INTEGER_LIST; +import static org.apache.flink.connector.pulsar.table.testutils.TestingUser.createRandomUser; +import static org.apache.flink.connector.pulsar.table.testutils.TestingUser.createUser; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.assertj.core.api.Assertions.assertThatNoException; @@ -68,6 +65,9 @@ public class PulsarCatalogITTest extends PulsarTableTestBase { private static final Logger LOGGER = LoggerFactory.getLogger(PulsarCatalogITTest.class); + private static final String AVRO_FORMAT = "avro"; + private static final String JSON_FORMAT = "json"; + private static final String INMEMORY_CATALOG = "inmemorycatalog"; private static final String PULSAR_CATALOG1 = "pulsarcatalog1"; private static final String PULSAR_CATALOG2 = "pulsarcatalog2"; @@ -514,11 +514,14 @@ void readFromNativeTableWithProtobufNativeSchema() { // TODO implement after protobuf native schema support } - @Test - void readFromNativeTableWithJsonSchema() throws Exception { + // TODO we didn't create the topic, how can we send to it ? + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void readFromNativeTableWithAvroBasedSchema(String format, Schema schema) + throws Exception { String topicName = newTopicName(); TestingUser expectedUser = createRandomUser(); - pulsar.operator().sendMessage(topicName, Schema.JSON(TestingUser.class), expectedUser); + pulsar.operator().sendMessage(topicName, schema, expectedUser); tableEnv.useCatalog(PULSAR_CATALOG1); tableEnv.useDatabase(PULSAR1_DB); @@ -536,20 +539,40 @@ void readFromNativeTableWithJsonSchema() throws Exception { Row.of(expectedUser.getAge(), expectedUser.getName()))); } - @Test - void readFromNativeTableWithAvroSchema() throws Exception { + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void readFromExplicitTableWithAvroBasedSchema(String format, Schema schema) + throws Exception { + // TODO add this test + String databaseName = newDatabaseName(); String topicName = newTopicName(); TestingUser expectedUser = createRandomUser(); - pulsar.operator().sendMessage(topicName, Schema.AVRO(TestingUser.class), expectedUser); + pulsar.operator().sendMessage(topicName, schema, expectedUser); tableEnv.useCatalog(PULSAR_CATALOG1); - tableEnv.useDatabase(PULSAR1_DB); + + String dbDDL = "CREATE DATABASE " + databaseName; + tableEnv.executeSql(dbDDL).print(); + tableEnv.useDatabase(databaseName); + + String sourceTableDDL = + String.format( + "CREATE TABLE %s (\n" + + " age INT,\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = 'pulsar',\n" + + " 'topics' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + topicName, topicName, format); + tableEnv.executeSql(sourceTableDDL).await(); final List result = collectRows( tableEnv.sqlQuery( String.format( - "select * from %s", + "select * from %s ", TopicName.get(topicName).getLocalName())), 1); assertThat(result) @@ -600,7 +623,6 @@ void readFromNativeTableWithComplexSchemaUsingRawFormatShouldFail() { } @Test - @Disabled("Disabled due to streamnative/flink#100") void copyDataFromNativeTableToNativeTable() throws Exception { String sourceTopic = newTopicName(); String sourceTableName = TopicName.get(sourceTopic).getLocalName(); @@ -614,25 +636,21 @@ void copyDataFromNativeTableToNativeTable() throws Exception { tableEnv.useCatalog(PULSAR_CATALOG1); tableEnv.useDatabase(PULSAR1_DB); String insertQ = - String.format( - "INSERT INTO %s" + " SELECT * FROM %s", sinkTableName, sourceTableName); + String.format("INSERT INTO %s SELECT * FROM %s", sinkTableName, sourceTableName); - tableEnv.executeSql(insertQ).await(); - List result = consumeMessage(sinkTableName, Schema.INT32, INTEGER_LIST.size(), 10); + tableEnv.executeSql(insertQ); + List result = + pulsar.operator().receiveMessages(sinkTopic, Schema.INT32, INTEGER_LIST.size()) + .stream() + .map(Message::getValue) + .collect(Collectors.toList()); assertThat(result).containsExactlyElementsOf(INTEGER_LIST); - - // final List result = - // collectRows( - // tableEnv.sqlQuery("SELECT * FROM " + sinkTableName), - // INTEGER_LIST.size()); - // - // assertThat(result) - // .containsExactlyElementsOf( - // INTEGER_LIST.stream().map(Row::of).collect(Collectors.toList())); } - @Test - void writeToExplicitTableAndReadWithAvroSchema() throws Exception { + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void writeToExplicitTableAndReadWithAvroBasedSchema(String format, Schema schema) + throws Exception { String databaseName = newDatabaseName(); String tableSinkTopic = newTopicName(); String tableSinkName = TopicName.get(tableSinkTopic).getLocalName(); @@ -642,7 +660,7 @@ void writeToExplicitTableAndReadWithAvroSchema() throws Exception { String dbDDL = "CREATE DATABASE " + databaseName; tableEnv.executeSql(dbDDL).print(); - tableEnv.executeSql("USE " + databaseName); + tableEnv.useDatabase(databaseName); String sinkDDL = String.format( @@ -652,10 +670,12 @@ void writeToExplicitTableAndReadWithAvroSchema() throws Exception { + " customerid STRING\n" + ") with (\n" + " 'connector' = 'pulsar',\n" - + " 'topics' = '%s'," - + " 'format' = 'avro'\n" + + " 'topics' = '%s',\n" + + " 'format' = '%s'\n" + ")", - tableSinkName, tableSinkTopic); + tableSinkName, tableSinkTopic, format); + tableEnv.executeSql(sinkDDL).await(); + String insertQ = String.format( "INSERT INTO %s" @@ -665,17 +685,18 @@ void writeToExplicitTableAndReadWithAvroSchema() throws Exception { + " ('oid3', 30, 'cid3'),\n" + " ('oid4', 10, 'cid4')", tableSinkName); - - tableEnv.executeSql(sinkDDL).print(); - tableEnv.executeSql(insertQ); + tableEnv.executeSql(insertQ).await(); final List result = - collectRows(tableEnv.sqlQuery(String.format("SELECT * FROM %s", tableSinkName)), 4); + collectRows(tableEnv.sqlQuery("select * from " + tableSinkName), 4); assertThat(result).hasSize(4); } - @Test - void writeToExplicitTableAndReadWithJsonSchema() throws Exception { + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + @Disabled("flink-128") + void writeToExplicitTableAndReadWithAvroBasedSchemaUsingPulsarConsumer( + String format, Schema schema) throws Exception { String databaseName = newDatabaseName(); String tableSinkTopic = newTopicName(); String tableSinkName = TopicName.get(tableSinkTopic).getLocalName(); @@ -690,42 +711,47 @@ void writeToExplicitTableAndReadWithJsonSchema() throws Exception { String sinkDDL = String.format( "CREATE TABLE %s (\n" - + " oid STRING,\n" - + " totalprice INT,\n" - + " customerid STRING\n" + + " name STRING,\n" + + " age INT\n" + ") with (\n" + " 'connector' = 'pulsar',\n" + " 'topics' = '%s',\n" - + " 'format' = 'json'\n" + + " 'format' = '%s'\n" + ")", - tableSinkName, tableSinkTopic); + tableSinkName, tableSinkTopic, format); tableEnv.executeSql(sinkDDL).await(); String insertQ = String.format( "INSERT INTO %s" + " VALUES\n" - + " ('oid1', 10, 'cid1'),\n" - + " ('oid2', 20, 'cid2'),\n" - + " ('oid3', 30, 'cid3'),\n" - + " ('oid4', 10, 'cid4')", + + " ('oid1', 10),\n" + + " ('oid2', 20),\n" + + " ('oid3', 30),\n" + + " ('oid4', 10)", tableSinkName); tableEnv.executeSql(insertQ).await(); - final List result = - collectRows(tableEnv.sqlQuery("select * from " + tableSinkName), 4); - assertThat(result).hasSize(4); - } - - @Test - void writeToNativeTableAndReadWithJsonSchema() throws Exception { + List sinkResult = + pulsar.operator().receiveMessages(tableSinkTopic, schema, 4).stream() + .map(Message::getValue) + .collect(Collectors.toList()); + assertThat(sinkResult) + .containsExactly( + createUser("oid1", 10), + createUser("oid2", 20), + createUser("oid3", 30), + createUser("oid4", 40)); + } + + @ParameterizedTest + @MethodSource("provideAvroBasedSchemaData") + void writeToNativeTableAndReadWithAvroBasedSchema(String format, Schema schema) + throws Exception { String tableSinkTopic = newTopicName(); pulsar.operator().createTopic(tableSinkTopic, 1); - pulsar.operator() - .admin() - .schemas() - .createSchema(tableSinkTopic, Schema.JSON(TestingUser.class).getSchemaInfo()); + pulsar.operator().admin().schemas().createSchema(tableSinkTopic, schema.getSchemaInfo()); tableEnv.useCatalog(PULSAR_CATALOG1); tableEnv.useDatabase(PULSAR1_DB); @@ -747,14 +773,14 @@ void writeToNativeTableAndReadWithJsonSchema() throws Exception { } @Test - void writeToNativeTableAndReadWithAvroSchema() throws Exception { + void writeToNativeTableAndReadWithStringSchema() throws Exception { String tableSinkTopic = newTopicName(); pulsar.operator().createTopic(tableSinkTopic, 1); pulsar.operator() .admin() .schemas() - .createSchema(tableSinkTopic, Schema.AVRO(TestingUser.class).getSchemaInfo()); + .createSchema(tableSinkTopic, Schema.STRING.getSchemaInfo()); tableEnv.useCatalog(PULSAR_CATALOG1); tableEnv.useDatabase(PULSAR1_DB); @@ -762,10 +788,10 @@ void writeToNativeTableAndReadWithAvroSchema() throws Exception { String.format( "INSERT INTO %s" + " VALUES\n" - + " (1, 'abc'),\n" - + " (2, 'bcd'),\n" - + " (3, 'cde'),\n" - + " (4, 'def')", + + " ('abc'),\n" + + " ('bcd'),\n" + + " ('cde'),\n" + + " ('def')", tableSinkTopic); tableEnv.executeSql(insertQ).await(); @@ -776,14 +802,14 @@ void writeToNativeTableAndReadWithAvroSchema() throws Exception { } @Test - void writeToNativeTableAndReadWithStringSchema() throws Exception { + void writeToNativeTableAndReadWithIntegerSchema() throws Exception { String tableSinkTopic = newTopicName(); pulsar.operator().createTopic(tableSinkTopic, 1); pulsar.operator() .admin() .schemas() - .createSchema(tableSinkTopic, Schema.STRING.getSchemaInfo()); + .createSchema(tableSinkTopic, Schema.INT32.getSchemaInfo()); tableEnv.useCatalog(PULSAR_CATALOG1); tableEnv.useDatabase(PULSAR1_DB); @@ -791,10 +817,10 @@ void writeToNativeTableAndReadWithStringSchema() throws Exception { String.format( "INSERT INTO %s" + " VALUES\n" - + " ('abc'),\n" - + " ('bcd'),\n" - + " ('cde'),\n" - + " ('def')", + + " (1),\n" + + " (2),\n" + + " (3),\n" + + " (4)", tableSinkTopic); tableEnv.executeSql(insertQ).await(); @@ -805,7 +831,7 @@ void writeToNativeTableAndReadWithStringSchema() throws Exception { } @Test - void writeToNativeTableAndReadWithIntegerSchema() throws Exception { + void writeToNativeTableAndReadWithIntegerSchemaUsingValueField() throws Exception { String tableSinkTopic = newTopicName(); pulsar.operator().createTopic(tableSinkTopic, 1); @@ -829,7 +855,8 @@ void writeToNativeTableAndReadWithIntegerSchema() throws Exception { final List result = collectRows( - tableEnv.sqlQuery(String.format("SELECT * FROM %s", tableSinkTopic)), 4); + tableEnv.sqlQuery(String.format("SELECT `value` FROM %s", tableSinkTopic)), + 4); assertThat(result).hasSize(4); } @@ -870,42 +897,10 @@ private String newTopicName() { return RandomStringUtils.randomAlphabetic(5); } - private TestingUser createRandomUser() { - TestingUser user = new TestingUser(); - user.setName(randomAlphabetic(5)); - user.setAge(ThreadLocalRandom.current().nextInt(0, Integer.MAX_VALUE)); - return user; - } - - private List consumeMessage(String topic, Schema schema, int count, int timeout) - throws InterruptedException, ExecutionException, TimeoutException { - final PulsarClient pulsarClient = pulsar.operator().client(); - return CompletableFuture.supplyAsync( - () -> { - Consumer consumer = null; - try { - consumer = - pulsarClient - .newConsumer(schema) - .topic(topic) - .subscriptionInitialPosition( - SubscriptionInitialPosition.Earliest) - .subscriptionName("test") - .subscribe(); - List result = new ArrayList<>(count); - for (int i = 0; i < count; i++) { - final Message message = consumer.receive(); - result.add(message.getValue()); - consumer.acknowledge(message); - } - consumer.close(); - return result; - } catch (Exception e) { - throw new IllegalStateException(e); - } finally { - IOUtils.closeQuietly(consumer, i -> {}); - } - }) - .get(timeout, TimeUnit.SECONDS); + private static Stream provideAvroBasedSchemaData() { + return Stream.of( + Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class)) + // Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class)) + ); } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java index 4ec6ea60c7879..b597f3fa3d875 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/testutils/TestingUser.java @@ -2,6 +2,9 @@ import java.io.Serializable; import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; /** A test POJO class. */ public class TestingUser implements Serializable { @@ -41,4 +44,18 @@ public boolean equals(Object o) { public int hashCode() { return Objects.hash(name, age); } + + public static TestingUser createUser(String name, Integer age) { + TestingUser user = new TestingUser(); + user.setAge(age); + user.setName(name); + return user; + } + + public static TestingUser createRandomUser() { + TestingUser user = new TestingUser(); + user.setName(randomAlphabetic(5)); + user.setAge(ThreadLocalRandom.current().nextInt(0, Integer.MAX_VALUE)); + return user; + } } From 87e1c3c185408915c183d1738e09987f614ecb2e Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Sun, 17 Jul 2022 12:49:04 +0800 Subject: [PATCH 230/258] 143: create default database is not exist --- .../pulsar/table/catalog/PulsarCatalog.java | 15 +++++++++- .../catalog/PulsarCatalogFactoryOptions.java | 4 ++- .../table/catalog/PulsarCatalogITTest.java | 29 +++++++++++++++++-- 3 files changed, 44 insertions(+), 4 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java index 56a4465145a5b..1ff1bb0353d2f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalog.java @@ -24,6 +24,7 @@ import org.apache.flink.connector.pulsar.table.catalog.impl.SchemaTranslator; import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; import org.apache.flink.table.catalog.CatalogPartition; import org.apache.flink.table.catalog.CatalogPartitionSpec; import org.apache.flink.table.catalog.GenericInMemoryCatalog; @@ -49,9 +50,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashMap; import java.util.List; import java.util.Optional; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactoryOptions.DEFAULT_DATABASE; + /** * Catalog implementation to use Pulsar to store metadatas for Flink tables/databases. * @@ -85,7 +89,7 @@ public class PulsarCatalog extends GenericInMemoryCatalog { public static final String DEFAULT_TENANT = "__flink_catalog"; - public static final String DEFAULT_DB = "default"; + public static final String DEFAULT_DB = "default_database"; public PulsarCatalog( String catalogName, @@ -116,6 +120,15 @@ public void open() throws CatalogException { e); } } + + CatalogDatabaseImpl defaultDatabase = + new CatalogDatabaseImpl(new HashMap<>(), "The default database for PulsarCatalog"); + try { + createDatabase(catalogConfiguration.get(DEFAULT_DATABASE), defaultDatabase, true); + } catch (DatabaseAlreadyExistException e) { + throw new CatalogException( + "Error: should ignore default database if not exist instead of throwing exception"); + } } @Override diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java index 7eecd74b0e77f..10005c6cad407 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java @@ -35,7 +35,9 @@ public final class PulsarCatalogFactoryOptions { public static final ConfigOption DEFAULT_DATABASE = ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) .stringType() - .defaultValue(PulsarCatalog.DEFAULT_DB); + .defaultValue(PulsarCatalog.DEFAULT_DB) + .withDescription( + "The default database when using PulsarCatalog. It will be created if not exist."); public static final ConfigOption CATALOG_ADMIN_URL = ConfigOptions.key("catalog-admin-url") diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java index 4cd9adae235a6..e1be82c5ef75d 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java @@ -49,9 +49,11 @@ import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalog.DEFAULT_DB; import static org.apache.flink.connector.pulsar.table.catalog.PulsarCatalogFactory.CATALOG_CONFIG_VALIDATOR; import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows; import static org.apache.flink.connector.pulsar.table.testutils.SchemaData.INTEGER_LIST; @@ -187,6 +189,29 @@ void createExplicitDatabase() { String.format("CREATE DATABASE %s", explictDatabaseName))); } + @Test + void createCatalogAndExpectDefaultDatabase() + throws ExecutionException, InterruptedException, TimeoutException { + tableEnv.useCatalog(PULSAR_CATALOG1); + assertThatNoException() + .isThrownBy(() -> tableEnv.executeSql(String.format("USE %s", DEFAULT_DB))); + + assertThatNoException() + .isThrownBy(() -> tableEnv.executeSql(String.format("SHOW TABLES", DEFAULT_DB))); + + String tableName = newTopicName(); + String tableDDL = + String.format( + "CREATE TABLE %s (\n" + + " oid STRING,\n" + + " totalprice INT,\n" + + " customerid STRING\n" + + ")", + tableName); + tableEnv.executeSql(tableDDL).await(10, TimeUnit.SECONDS); + assertThat(tableEnv.listTables()).contains(tableName); + } + @Test void createNativeDatabaseShouldFail() { tableEnv.useCatalog(PULSAR_CATALOG1); @@ -287,7 +312,7 @@ void createExplicitTable() throws Exception { tableEnv.executeSql(dbDDL).print(); tableEnv.useDatabase(databaseName); - String sinkDDL = + String tableDDL = String.format( "CREATE TABLE %s (\n" + " oid STRING,\n" @@ -295,7 +320,7 @@ void createExplicitTable() throws Exception { + " customerid STRING\n" + ")", tableName); - tableEnv.executeSql(sinkDDL).await(10, TimeUnit.SECONDS); + tableEnv.executeSql(tableDDL).await(10, TimeUnit.SECONDS); assertThat(tableEnv.listTables()).contains(tableName); } From 52a72d433906a4e7095e37b16d9afc679972c085 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Sun, 17 Jul 2022 12:30:28 +0800 Subject: [PATCH 231/258] 136: support value.format config --- .../pulsar/table/PulsarTableFactory.java | 16 ++-- .../pulsar/table/PulsarTableOptionUtils.java | 18 +++- .../pulsar/table/PulsarTableOptions.java | 8 ++ .../table/PulsarTableValidationUtils.java | 7 +- .../pulsar/table/PulsarTableOptionsTest.java | 94 ++++++++++++++----- 5 files changed, 97 insertions(+), 46 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java index 41cfe42599dcf..c4b32c83c0e52 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java @@ -81,10 +81,10 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.VALUE_FORMAT; import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validatePrimaryKeyConstraints; import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSinkOptions; import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSourceOptions; -import static org.apache.flink.table.factories.FactoryUtil.FORMAT; import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; /** @@ -118,10 +118,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { PulsarSinkOptions.SINK_CONFIG_PREFIX); validatePrimaryKeyConstraints( - context.getObjectIdentifier(), - context.getPrimaryKeyIndexes(), - context.getCatalogTable().getOptions(), - helper); + context.getObjectIdentifier(), context.getPrimaryKeyIndexes(), helper); validateTableSourceOptions(tableOptions); @@ -185,10 +182,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { PulsarSinkOptions.SINK_CONFIG_PREFIX); validatePrimaryKeyConstraints( - context.getObjectIdentifier(), - context.getPrimaryKeyIndexes(), - context.getCatalogTable().getOptions(), - helper); + context.getObjectIdentifier(), context.getPrimaryKeyIndexes(), helper); validateTableSinkOptions(tableOptions); @@ -240,12 +234,14 @@ public String factoryIdentifier() { @Override public Set> requiredOptions() { - return Stream.of(TOPICS, ADMIN_URL, SERVICE_URL, FORMAT).collect(Collectors.toSet()); + return Stream.of(TOPICS, ADMIN_URL, SERVICE_URL).collect(Collectors.toSet()); } @Override public Set> optionalOptions() { return Stream.of( + FactoryUtil.FORMAT, + VALUE_FORMAT, SOURCE_SUBSCRIPTION_NAME, SOURCE_SUBSCRIPTION_TYPE, SOURCE_START_FROM_MESSAGE_ID, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java index 6aa7544f4c4b1..b89c14edb9db3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java @@ -61,7 +61,7 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; -import static org.apache.flink.table.factories.FactoryUtil.FORMAT; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.VALUE_FORMAT; /** * A util class for getting fields from config options, getting formats and other useful @@ -103,14 +103,22 @@ public static EncodingFormat> getKeyEncodingFormat( public static DecodingFormat> getValueDecodingFormat( FactoryUtil.TableFactoryHelper helper) { - return helper.discoverOptionalDecodingFormat(DeserializationFormatFactory.class, FORMAT) - .get(); + return helper.discoverOptionalDecodingFormat( + DeserializationFormatFactory.class, FactoryUtil.FORMAT) + .orElseGet( + () -> + helper.discoverDecodingFormat( + DeserializationFormatFactory.class, VALUE_FORMAT)); } public static EncodingFormat> getValueEncodingFormat( FactoryUtil.TableFactoryHelper helper) { - return helper.discoverOptionalEncodingFormat(SerializationFormatFactory.class, FORMAT) - .get(); + return helper.discoverOptionalEncodingFormat( + SerializationFormatFactory.class, FactoryUtil.FORMAT) + .orElseGet( + () -> + helper.discoverEncodingFormat( + SerializationFormatFactory.class, VALUE_FORMAT)); } /** diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java index d951adb29522f..df131ea225eea 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java @@ -147,6 +147,14 @@ private PulsarTableOptions() {} + "from the key bytes of a Pulsar message. By default, " + "this list is empty and thus a key is undefined."); + public static final ConfigOption VALUE_FORMAT = + ConfigOptions.key("value" + FORMAT_SUFFIX) + .stringType() + .noDefaultValue() + .withDescription( + "Defines the format identifier for decoding/encoding value data. " + + "The identifier is used to discover a suitable format factory."); + // -------------------------------------------------------------------------------------------- // Pulsar Options // -------------------------------------------------------------------------------------------- diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java index ca95eb7c6e8fa..6ce78337b6fae 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java @@ -19,7 +19,6 @@ package org.apache.flink.connector.pulsar.table; import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.table.api.ValidationException; @@ -32,7 +31,6 @@ import org.apache.pulsar.client.api.SubscriptionType; import java.util.List; -import java.util.Map; import java.util.Optional; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; @@ -54,19 +52,16 @@ private PulsarTableValidationUtils() {} public static void validatePrimaryKeyConstraints( ObjectIdentifier tableName, int[] primaryKeyIndexes, - Map options, FactoryUtil.TableFactoryHelper helper) { final DecodingFormat> format = getValueDecodingFormat(helper); if (primaryKeyIndexes.length > 0 && format.getChangelogMode().containsOnly(RowKind.INSERT)) { - Configuration configuration = Configuration.fromMap(options); - String formatName = configuration.getOptional(FactoryUtil.FORMAT).get(); throw new ValidationException( String.format( "The Pulsar table '%s' with '%s' format doesn't support defining PRIMARY KEY constraint" + " on the table, because it can't guarantee the semantic of primary key.", - tableName.asSummaryString(), formatName)); + tableName.asSummaryString(), format)); } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java index 500dd0f49716e..413acad203b7d 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java @@ -20,6 +20,7 @@ import org.apache.flink.connector.pulsar.table.testutils.MockPulsarAuthentication; import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.factories.FactoryUtil; import org.junit.jupiter.api.Test; @@ -38,6 +39,7 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; +import static org.apache.flink.table.factories.TestDynamicTableFactory.VALUE_FORMAT; import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.assertj.core.api.Assertions.assertThatNoException; @@ -50,7 +52,7 @@ public class PulsarTableOptionsTest extends PulsarTableTestBase { @Test void noTopicsSpecified() { final String topicName = randomTopicName(); - Map testConfigs = new HashMap<>(); + Map testConfigs = testConfigWithFormat(); runSql(topicName, createTestConfig(testConfigs)); runSinkAndExpectException( topicName, @@ -63,8 +65,7 @@ void noTopicsSpecified() { @Test void invalidEmptyTopics() { final String topicName = randomTopicName(); - Map testConfigs = new HashMap<>(); - + Map testConfigs = testConfigWithFormat(); testConfigs.put(TOPICS.key(), ""); runSql(topicName, createTestConfig(testConfigs)); runSinkAndExpectException( @@ -74,7 +75,7 @@ void invalidEmptyTopics() { @Test void topicsWithSemicolon() { final String topicName = randomTopicName(); - Map testConfigs = new HashMap<>(); + Map testConfigs = testConfigWithFormat(); testConfigs.put(TOPICS.key(), topicName + ";"); runSql(topicName, createTestConfig(testConfigs)); runSinkAndExpectSucceed(topicName); @@ -84,7 +85,7 @@ void topicsWithSemicolon() { @Test void invalidTopicName() { final String topicName = randomTopicName(); - Map testConfigs = new HashMap<>(); + Map testConfigs = testConfigWithFormat(); String invalidTopicName = "persistent://tenant/no-topic"; testConfigs.put(TOPICS.key(), invalidTopicName); runSql(topicName, createTestConfig(testConfigs)); @@ -99,7 +100,7 @@ void invalidTopicName() { @Test void topicsList() { final String topicName = randomTopicName(); - Map testConfigs = new HashMap<>(); + Map testConfigs = testConfigWithFormat(); testConfigs.put( TOPICS.key(), topicNameWithPartition(topicName, 0) + ";" + topicNameWithPartition(topicName, 1)); @@ -109,9 +110,40 @@ void topicsList() { } @Test - void subscriptionType() { + void usingFormat() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(FactoryUtil.FORMAT.key(), "json"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void usingValueFormat() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(VALUE_FORMAT.key(), "json"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void usingValueFormatAndFormatOptions() { final String topicName = randomTopicName(); Map testConfigs = testConfigWithTopic(topicName); + testConfigs.put(VALUE_FORMAT.key(), "json"); + testConfigs.put("value.json.fail-on-missing-field", "false"); + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void subscriptionType() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Exclusive"); runSql(topicName, createTestConfig(testConfigs)); runSinkAndExpectSucceed(topicName); @@ -121,7 +153,7 @@ void subscriptionType() { @Test void invalidUnsupportedSubscriptionType() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "Key_Shared"); runSql(topicName, createTestConfig(testConfigs)); runSinkAndExpectSucceed(topicName); @@ -134,7 +166,7 @@ void invalidUnsupportedSubscriptionType() { @Test void invalidNonExistSubscriptionType() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SOURCE_SUBSCRIPTION_TYPE.key(), "random-subscription-type"); runSql(topicName, createTestConfig(testConfigs)); runSinkAndExpectException( @@ -145,7 +177,7 @@ void invalidNonExistSubscriptionType() { @Test void messageIdStartCursorEarliest() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "earliest"); @@ -157,7 +189,7 @@ void messageIdStartCursorEarliest() { @Test void messageIdStartCursorLatest() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "latest"); @@ -169,7 +201,7 @@ void messageIdStartCursorLatest() { @Test void messageIdStartCursorExact() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:-1"); @@ -181,7 +213,7 @@ void messageIdStartCursorExact() { @Test void invalidMessageIdStartCursorEmptyId() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0:"); @@ -197,7 +229,7 @@ void invalidMessageIdStartCursorEmptyId() { @Test void invalidMessageIdStartCursorIncomplete() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SOURCE_START_FROM_MESSAGE_ID.key(), "0:0"); @@ -212,7 +244,7 @@ void invalidMessageIdStartCursorIncomplete() { @Test void timestampStartCursor() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "233010230"); @@ -224,7 +256,7 @@ void timestampStartCursor() { @Test void topicRoutingMode() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SINK_TOPIC_ROUTING_MODE.key(), "message-key-hash"); @@ -236,7 +268,7 @@ void topicRoutingMode() { @Test void invalidTopicRouter() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); String invalidClassName = "invalid class name"; testConfigs.put(SINK_CUSTOM_TOPIC_ROUTER.key(), invalidClassName); @@ -253,7 +285,7 @@ void invalidTopicRouter() { @Test void messageDelay() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "10s"); @@ -265,7 +297,7 @@ void messageDelay() { @Test void invalidMessageDelay() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(SINK_MESSAGE_DELAY_INTERVAL.key(), "invalid-duration"); @@ -281,7 +313,7 @@ void invalidMessageDelay() { @Test void subscriptionInitialPosition() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(PULSAR_SUBSCRIPTION_INITIAL_POSITION.key(), "Earliest"); @@ -297,7 +329,7 @@ void subscriptionInitialPosition() { @Test void pulsarOptionsAuthParamMap() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put(PULSAR_AUTH_PARAM_MAP.key(), "key1:value1,key2:value2"); testConfigs.put( @@ -315,7 +347,7 @@ void pulsarOptionsAuthParamMap() { @Test void unusedConfigOptions() { final String topicName = randomTopicName(); - Map testConfigs = testConfigWithTopic(topicName); + Map testConfigs = testConfigWithTopicAndFormat(topicName); testConfigs.put("random_config", "random_value"); runSql(topicName, createTestConfig(testConfigs)); @@ -343,11 +375,10 @@ private void runSql(String topicName, String testConfigString) { + " `physical_2` INT,\n" + " `physical_3` BOOLEAN\n" + ") WITH (\n" - + " 'connector' = 'pulsar',\n" + " 'service-url' = '%s',\n" + " 'admin-url' = '%s',\n" - + " %s" - + " 'format' = 'json'\n" + + " %s\n" + + " 'connector' = 'pulsar'" + ")", topicName, pulsar.operator().serviceUrl(), @@ -412,6 +443,19 @@ private String randomTopicName() { return testTopicPrefix + randomAlphabetic(5); } + private Map testConfigWithTopicAndFormat(String tableName) { + Map testConfigs = new HashMap<>(); + testConfigs.put(TOPICS.key(), tableName); + testConfigs.put(FactoryUtil.FORMAT.key(), "json"); + return testConfigs; + } + + private Map testConfigWithFormat() { + Map testConfigs = new HashMap<>(); + testConfigs.put(FactoryUtil.FORMAT.key(), "json"); + return testConfigs; + } + private Map testConfigWithTopic(String tableName) { Map testConfigs = new HashMap<>(); testConfigs.put(TOPICS.key(), tableName); From b829d735433e5e6c93a87e1a110d1d9be8f6d9c4 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Wed, 20 Jul 2022 15:33:17 +0800 Subject: [PATCH 232/258] [FLINK-26931][Connector/Pulsar] Make the producer name and consumer name unique for each instance. (#152) --- .../common/config/PulsarClientFactory.java | 18 ++++++++++-------- .../common/config/PulsarConfiguration.java | 2 +- .../pulsar/sink/PulsarSinkBuilder.java | 5 +++++ .../sink/config/PulsarSinkConfigUtils.java | 6 +++++- .../pulsar/source/PulsarSourceBuilder.java | 7 ++++++- .../source/config/PulsarSourceConfigUtils.java | 6 +++++- 6 files changed, 32 insertions(+), 12 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarClientFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarClientFactory.java index b1214b57c04f6..1f01b242143ca 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarClientFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarClientFactory.java @@ -153,7 +153,7 @@ public static PulsarClient createClient(PulsarConfiguration configuration) { /** * PulsarAdmin shares almost the same configuration with PulsarClient, but we separate this - * create method for directly creating it. + * creating method for directly use it. */ public static PulsarAdmin createAdmin(PulsarConfiguration configuration) { PulsarAdminBuilder builder = PulsarAdmin.builder(); @@ -200,15 +200,17 @@ private static Authentication createAuthentication(PulsarConfiguration configura String authParamsString = configuration.get(PULSAR_AUTH_PARAMS); return sneakyClient( () -> AuthenticationFactory.create(authPluginClassName, authParamsString)); - } else if (configuration.contains(PULSAR_AUTH_PARAM_MAP)) { - Map paramsMap = configuration.get(PULSAR_AUTH_PARAM_MAP); + } else { + Map paramsMap = configuration.getProperties(PULSAR_AUTH_PARAM_MAP); + if (paramsMap.isEmpty()) { + throw new IllegalArgumentException( + String.format( + "No %s or %s provided", + PULSAR_AUTH_PARAMS.key(), PULSAR_AUTH_PARAM_MAP.key())); + } + return sneakyClient( () -> AuthenticationFactory.create(authPluginClassName, paramsMap)); - } else { - throw new IllegalArgumentException( - String.format( - "No %s or %s provided", - PULSAR_AUTH_PARAMS.key(), PULSAR_AUTH_PARAM_MAP.key())); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfiguration.java index 3e64c6643cd62..0681a3ef4cba4 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/config/PulsarConfiguration.java @@ -76,7 +76,7 @@ public Map getProperties(ConfigOption> optio return properties; } - /** Get an option value from the given config, convert it into the a new value instance. */ + /** Get an option value from the given config, convert it into a new value instance. */ public T get(ConfigOption option, Function convertor) { F value = get(option); if (value != null) { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java index 96ca03876a4fd..e623f29406725 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java @@ -373,6 +373,11 @@ public PulsarSink build() { if (!configBuilder.contains(PULSAR_PRODUCER_NAME)) { LOG.warn( "We recommend set a readable producer name through setProducerName(String) in production mode."); + } else { + String producerName = configBuilder.get(PULSAR_PRODUCER_NAME); + if (!producerName.contains("%s")) { + configBuilder.override(PULSAR_PRODUCER_NAME, producerName + " - %s"); + } } checkNotNull(serializationSchema, "serializationSchema must be set."); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java index 77155bbea6b74..e4c48162589f3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java @@ -27,6 +27,7 @@ import org.apache.pulsar.client.api.Schema; import java.util.Map; +import java.util.UUID; import static java.util.concurrent.TimeUnit.MICROSECONDS; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -77,7 +78,10 @@ public static ProducerBuilder createProducerBuilder( PulsarClient client, Schema schema, SinkConfiguration configuration) { ProducerBuilder builder = client.newProducer(schema); - configuration.useOption(PULSAR_PRODUCER_NAME, builder::producerName); + configuration.useOption( + PULSAR_PRODUCER_NAME, + producerName -> String.format(producerName, UUID.randomUUID()), + builder::producerName); configuration.useOption( PULSAR_SEND_TIMEOUT_MS, Math::toIntExact, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java index 37dd01ee946d0..7e6842a76cdc9 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java @@ -515,9 +515,14 @@ public PulsarSource build() { if (!configBuilder.contains(PULSAR_CONSUMER_NAME)) { LOG.warn( "We recommend set a readable consumer name through setConsumerName(String) in production mode."); + } else { + String consumerName = configBuilder.get(PULSAR_CONSUMER_NAME); + if (!consumerName.contains("%s")) { + configBuilder.override(PULSAR_CONSUMER_NAME, consumerName + " - %s"); + } } - // Since these implementation could be a lambda, make sure they are serializable. + // Since these implementations could be a lambda, make sure they are serializable. checkState(isSerializable(startCursor), "StartCursor isn't serializable"); checkState(isSerializable(stopCursor), "StopCursor isn't serializable"); checkState(isSerializable(rangeGenerator), "RangeGenerator isn't serializable"); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java index adb8a03df0328..602a1577938e1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java @@ -29,6 +29,7 @@ import java.util.Map; import java.util.Optional; +import java.util.UUID; import static java.util.concurrent.TimeUnit.MICROSECONDS; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -106,7 +107,10 @@ public static ConsumerBuilder createConsumerBuilder( configuration.useOption( PULSAR_MAX_TOTAL_RECEIVER_QUEUE_SIZE_ACROSS_PARTITIONS, builder::maxTotalReceiverQueueSizeAcrossPartitions); - configuration.useOption(PULSAR_CONSUMER_NAME, builder::consumerName); + configuration.useOption( + PULSAR_CONSUMER_NAME, + consumerName -> String.format(consumerName, UUID.randomUUID()), + builder::consumerName); configuration.useOption(PULSAR_READ_COMPACTED, builder::readCompacted); configuration.useOption(PULSAR_PRIORITY_LEVEL, builder::priorityLevel); configuration.useOption( From 2c8412d5a29c7e5d110e8f04dfe9b41d06aed670 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 21 Jul 2022 23:04:08 +0800 Subject: [PATCH 233/258] [FLINK-28609][Connector/Pulsar] PulsarSchema didn't get properly serialized. --- .../pulsar/common/schema/PulsarSchema.java | 100 ++++++------------ .../common/schema/PulsarSchemaTest.java | 28 +++++ 2 files changed, 61 insertions(+), 67 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java index 6ce91cdc67415..5fd176e73267b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java @@ -27,12 +27,8 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Arrays; -import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -61,6 +57,11 @@ public final class PulsarSchema implements Serializable { private transient Schema schema; private transient SchemaInfo schemaInfo; + private String schemaName; + private byte[] schemaBytes; + private SchemaType schemaType; + private Map schemaProperties; + /** Create serializable pulsar schema for primitive types. */ public PulsarSchema(Schema schema) { SchemaInfo info = schema.getSchemaInfo(); @@ -77,8 +78,7 @@ public PulsarSchema(Schema schema) { // Primitive type information could be reflected from the schema class. Class typeClass = getTemplateType1(schema.getClass()); - this.schemaInfo = encodeClassInfo(info, typeClass); - this.schema = createSchema(schemaInfo); + setSchemaInfo(encodeClassInfo(info, typeClass)); } /** @@ -94,8 +94,7 @@ public PulsarSchema(Schema schema, Class typeClass) { "Key Value Schema should provide the type classes of key and value"); validateSchemaInfo(info); - this.schemaInfo = encodeClassInfo(info, typeClass); - this.schema = createSchema(schemaInfo); + setSchemaInfo(encodeClassInfo(info, typeClass)); } /** Create serializable pulsar schema for key value type. */ @@ -118,67 +117,37 @@ public PulsarSchema( SchemaInfo encodedInfo = encodeKeyValueSchemaInfo(info.getName(), infoKey, infoValue, encodingType); - this.schemaInfo = encodeClassInfo(encodedInfo, KeyValue.class); - this.schema = createSchema(this.schemaInfo); + setSchemaInfo(encodeClassInfo(encodedInfo, KeyValue.class)); + } + + /** Validate the schema for having the required class info. */ + private void setSchemaInfo(SchemaInfo schemaInfo) { + this.schema = createSchema(schemaInfo); + this.schemaInfo = schemaInfo; + + this.schemaName = schemaInfo.getName(); + this.schemaBytes = schemaInfo.getSchema(); + this.schemaType = schemaInfo.getType(); + this.schemaProperties = schemaInfo.getProperties(); } public Schema getPulsarSchema() { + if (schema == null) { + this.schema = createSchema(getSchemaInfo()); + } return schema; } public SchemaInfo getSchemaInfo() { + if (schemaInfo == null) { + this.schemaInfo = + new SchemaInfoImpl(schemaName, schemaBytes, schemaType, schemaProperties); + } return schemaInfo; } public Class getRecordClass() { - return decodeClassInfo(schemaInfo); - } - - private void writeObject(ObjectOutputStream oos) throws IOException { - // Name - oos.writeUTF(schemaInfo.getName()); - - // Schema - byte[] schemaBytes = schemaInfo.getSchema(); - oos.writeInt(schemaBytes.length); - oos.write(schemaBytes); - - // Type - SchemaType type = schemaInfo.getType(); - oos.writeInt(type.getValue()); - - // Properties - Map properties = schemaInfo.getProperties(); - oos.writeInt(properties.size()); - for (Map.Entry entry : properties.entrySet()) { - oos.writeUTF(entry.getKey()); - oos.writeUTF(entry.getValue()); - } - } - - private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException { - // Name - String name = ois.readUTF(); - - // Schema - int byteLen = ois.readInt(); - byte[] schemaBytes = new byte[byteLen]; - int read = ois.read(schemaBytes); - checkState(read == byteLen); - - // Type - int typeIdx = ois.readInt(); - SchemaType type = SchemaType.valueOf(typeIdx); - - // Properties - int propSize = ois.readInt(); - Map properties = new HashMap<>(propSize); - for (int i = 0; i < propSize; i++) { - properties.put(ois.readUTF(), ois.readUTF()); - } - - this.schemaInfo = new SchemaInfoImpl(name, schemaBytes, type, properties); - this.schema = createSchema(schemaInfo); + return decodeClassInfo(getSchemaInfo()); } @Override @@ -189,24 +158,21 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - SchemaInfo that = ((PulsarSchema) o).getPulsarSchema().getSchemaInfo(); + PulsarSchema that = ((PulsarSchema) o); - return Objects.equals(schemaInfo.getType(), that.getType()) - && Arrays.equals(schemaInfo.getSchema(), that.getSchema()) - && Objects.equals(schemaInfo.getProperties(), that.getProperties()); + return Objects.equals(schemaType, that.schemaType) + && Arrays.equals(schemaBytes, that.schemaBytes) + && Objects.equals(schemaProperties, that.schemaProperties); } @Override public int hashCode() { - return Objects.hash( - schemaInfo.getType(), - Arrays.hashCode(schemaInfo.getSchema()), - schemaInfo.getProperties()); + return Objects.hash(schemaType, Arrays.hashCode(schemaBytes), schemaProperties); } @Override public String toString() { - return schemaInfo.toString(); + return getSchemaInfo().toString(); } /** diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaTest.java index 7011e169656d3..81074c993708f 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchemaTest.java @@ -36,6 +36,8 @@ import org.apache.pulsar.common.schema.SchemaType; import org.junit.jupiter.api.Test; +import java.io.Serializable; + import static org.junit.jupiter.api.Assertions.assertAll; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -117,9 +119,35 @@ void pulsarSchemaSerialization() throws Exception { assertPulsarSchemaIsSerializable(new PulsarSchema<>(KV, Foo.class, FA.class)); } + @Test + void largeAvroSchemaSerialization() throws Exception { + Schema largeMessageSchema = Schema.AVRO(LargeMessage.class); + assertPulsarSchemaIsSerializable( + new PulsarSchema<>(largeMessageSchema, LargeMessage.class)); + } + private void assertPulsarSchemaIsSerializable(PulsarSchema schema) throws Exception { PulsarSchema clonedSchema = InstantiationUtil.clone(schema); assertEquals(clonedSchema.getSchemaInfo(), schema.getSchemaInfo()); assertEquals(clonedSchema.getRecordClass(), schema.getRecordClass()); } + + /** A POJO Class which would generate a large schema by Avro. */ + public static class LargeMessage implements Serializable { + private static final long serialVersionUID = 5364494369740402518L; + + public String + aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa; + public String + bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb; + public String + cccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccccc; + public String + dddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddddd; + public String + eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee; + // the problem begins + public String + ffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffffff; + } } From f7c60c41c6f1042b3f8b7999183133716bc5f203 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 21 Jul 2022 23:20:45 +0800 Subject: [PATCH 234/258] [FLINK-28609][Connector/Pulsar] Drop useless end-to-end test classes. --- .../cases/KeySharedSubscriptionContext.java | 145 ------------------ .../cases/SharedSubscriptionContext.java | 117 -------------- 2 files changed, 262 deletions(-) delete mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java delete mode 100644 flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java deleted file mode 100644 index 1fdfc6fa18f1a..0000000000000 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/KeySharedSubscriptionContext.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.tests.util.pulsar.cases; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.pulsar.source.PulsarSource; -import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange; -import org.apache.flink.connector.pulsar.source.enumerator.topic.range.FixedRangeGenerator; -import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; -import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; -import org.apache.flink.tests.util.pulsar.common.KeyedPulsarPartitionDataWriter; - -import org.apache.pulsar.client.api.RegexSubscriptionMode; -import org.apache.pulsar.client.api.SubscriptionType; -import org.apache.pulsar.common.util.Murmur3_32Hash; - -import java.net.URL; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static java.util.Collections.singletonList; -import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; -import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.RANGE_SIZE; -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; -import static org.apache.pulsar.client.api.Schema.STRING; - -/** We would consume from test splits by using {@link SubscriptionType#Key_Shared} subscription. */ -public class KeySharedSubscriptionContext extends PulsarTestContext { - - private int index = 0; - - private final List writers = new ArrayList<>(); - - // Message keys. - private final String key1; - private final String key2; - - public KeySharedSubscriptionContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public KeySharedSubscriptionContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); - - // Init message keys. - this.key1 = randomAlphabetic(8); - String newKey2; - do { - newKey2 = randomAlphabetic(8); - } while (keyHash(key1) == keyHash(newKey2)); - this.key2 = newKey2; - } - - @Override - protected String displayName() { - return "consuming message by Key_Shared"; - } - - @Override - public Source createSource(TestingSourceSettings sourceSettings) { - int keyHash = keyHash(key1); - TopicRange range = new TopicRange(keyHash, keyHash); - - PulsarSourceBuilder builder = - PulsarSource.builder() - .setDeserializationSchema(pulsarSchema(STRING)) - .setServiceUrl(operator.serviceUrl()) - .setAdminUrl(operator.adminUrl()) - .setTopicPattern( - "pulsar-[0-9]+-key-shared", RegexSubscriptionMode.AllTopics) - .setSubscriptionType(SubscriptionType.Key_Shared) - .setSubscriptionName("pulsar-key-shared") - .setConsumerName("pulsar-key-shared") - .setRangeGenerator(new FixedRangeGenerator(singletonList(range))); - if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { - // Using the latest stop cursor for making sure the source could be stopped. - builder.setBoundedStopCursor(StopCursor.latest()); - } - - return builder.build(); - } - - @Override - public ExternalSystemSplitDataWriter createSourceSplitDataWriter( - TestingSourceSettings sourceSettings) { - String topicName = "pulsar-" + index + "-key-shared"; - operator.createTopic(topicName, 1); - index++; - - String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0); - KeyedPulsarPartitionDataWriter writer = - new KeyedPulsarPartitionDataWriter(operator, partitionName, key1, key2); - writers.add(writer); - - return writer; - } - - @Override - public List generateTestData( - TestingSourceSettings sourceSettings, int splitIndex, long seed) { - return generateStringTestData(splitIndex, seed); - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(String.class); - } - - @Override - public void close() { - for (KeyedPulsarPartitionDataWriter writer : writers) { - writer.close(); - } - writers.clear(); - } - - private int keyHash(String key) { - return Murmur3_32Hash.getInstance().makeHash(key.getBytes()) % RANGE_SIZE; - } -} diff --git a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java b/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java deleted file mode 100644 index 6bdaaa8e339ca..0000000000000 --- a/flink-end-to-end-tests/flink-end-to-end-tests-pulsar/src/test/java/org/apache/flink/tests/util/pulsar/cases/SharedSubscriptionContext.java +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.tests.util.pulsar.cases; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.connector.pulsar.source.PulsarSource; -import org.apache.flink.connector.pulsar.source.PulsarSourceBuilder; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; -import org.apache.flink.connector.pulsar.testutils.PulsarPartitionDataWriter; -import org.apache.flink.connector.pulsar.testutils.PulsarTestContext; -import org.apache.flink.connector.pulsar.testutils.PulsarTestEnvironment; -import org.apache.flink.connector.testframe.external.ExternalSystemSplitDataWriter; -import org.apache.flink.connector.testframe.external.source.TestingSourceSettings; - -import org.apache.pulsar.client.api.RegexSubscriptionMode; -import org.apache.pulsar.client.api.SubscriptionType; - -import java.net.URL; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -import static org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema.pulsarSchema; -import static org.apache.pulsar.client.api.Schema.STRING; - -/** We would consuming from test splits by using {@link SubscriptionType#Shared} subscription. */ -public class SharedSubscriptionContext extends PulsarTestContext { - - private int index = 0; - - private final List writers = new ArrayList<>(); - - public SharedSubscriptionContext(PulsarTestEnvironment environment) { - this(environment, Collections.emptyList()); - } - - public SharedSubscriptionContext( - PulsarTestEnvironment environment, List connectorJarPaths) { - super(environment, connectorJarPaths); - } - - @Override - protected String displayName() { - return "consuming message by Shared"; - } - - @Override - public Source createSource(TestingSourceSettings sourceSettings) { - PulsarSourceBuilder builder = - PulsarSource.builder() - .setDeserializationSchema(pulsarSchema(STRING)) - .setServiceUrl(operator.serviceUrl()) - .setAdminUrl(operator.adminUrl()) - .setTopicPattern("pulsar-[0-9]+-shared", RegexSubscriptionMode.AllTopics) - .setSubscriptionType(SubscriptionType.Shared) - .setSubscriptionName("pulsar-shared") - .setConsumerName("pulsar-shared"); - if (sourceSettings.getBoundedness() == Boundedness.BOUNDED) { - // Using the latest stop cursor for making sure the source could be stopped. - builder.setBoundedStopCursor(StopCursor.latest()); - } - - return builder.build(); - } - - @Override - public ExternalSystemSplitDataWriter createSourceSplitDataWriter( - TestingSourceSettings sourceSettings) { - String topicName = "pulsar-" + index + "-shared"; - operator.createTopic(topicName, 1); - index++; - - String partitionName = TopicNameUtils.topicNameWithPartition(topicName, 0); - PulsarPartitionDataWriter writer = new PulsarPartitionDataWriter(operator, partitionName); - writers.add(writer); - - return writer; - } - - @Override - public List generateTestData( - TestingSourceSettings sourceSettings, int splitIndex, long seed) { - return generateStringTestData(splitIndex, seed); - } - - @Override - public TypeInformation getProducedType() { - return TypeInformation.of(String.class); - } - - @Override - public void close() { - for (PulsarPartitionDataWriter writer : writers) { - writer.close(); - } - writers.clear(); - } -} From 0e86a8696746d36f132362fc65d7c0a96c2ac04d Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Fri, 22 Jul 2022 03:04:56 +0800 Subject: [PATCH 235/258] [FLINK-25686][Connector/Pulsar]: Add schema evolution support for pulsar source connector. --- .../pulsar/common/schema/BytesSchema.java | 82 +++++++++++++++++++ .../pulsar/source/PulsarSourceBuilder.java | 15 ++-- .../reader/PulsarSourceReaderFactory.java | 16 ++++ .../PulsarDeserializationSchema.java | 11 +-- .../PulsarDeserializationSchemaWrapper.java | 2 +- .../deserializer/PulsarSchemaWrapper.java | 39 ++------- .../PulsarTypeInformationWrapper.java | 2 +- .../PulsarOrderedPartitionSplitReader.java | 4 +- .../split/PulsarPartitionSplitReaderBase.java | 7 +- .../PulsarUnorderedPartitionSplitReader.java | 6 +- .../PulsarTableDeserializationSchema.java | 10 +-- .../enumerator/cursor/StopCursorTest.java | 6 +- .../PulsarPartitionSplitReaderTestBase.java | 10 ++- 13 files changed, 141 insertions(+), 69 deletions(-) create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/BytesSchema.java diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/BytesSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/BytesSchema.java new file mode 100644 index 0000000000000..674c971b825f7 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/BytesSchema.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.common.schema; + +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.common.schema.SchemaInfo; + +import java.io.Serializable; +import java.nio.ByteBuffer; + +import static org.apache.pulsar.client.internal.PulsarClientImplementationBinding.getBytes; + +/** + * The schema is a wrapper for the original schema. It will send the schema info to Pulsar for + * compatibility check. And didn't deserialize messages. + */ +public class BytesSchema implements Schema, Serializable { + private static final long serialVersionUID = -539752264675729127L; + + private final PulsarSchema pulsarSchema; + + public BytesSchema(PulsarSchema pulsarSchema) { + this.pulsarSchema = pulsarSchema; + } + + @Override + public void validate(byte[] message) { + pulsarSchema.getPulsarSchema().validate(message); + } + + @Override + public byte[] encode(byte[] message) { + return message; + } + + @Override + public boolean supportSchemaVersioning() { + return pulsarSchema.getPulsarSchema().supportSchemaVersioning(); + } + + @Override + public byte[] decode(byte[] bytes) { + return bytes; + } + + @Override + public byte[] decode(byte[] bytes, byte[] schemaVersion) { + // None of Pulsar's schema implementations have implemented this method. + return bytes; + } + + @Override + public byte[] decode(ByteBuffer data, byte[] schemaVersion) { + return getBytes(data); + } + + @Override + public SchemaInfo getSchemaInfo() { + return pulsarSchema.getSchemaInfo(); + } + + @Override + public Schema clone() { + return this; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java index 7e6842a76cdc9..5c6bb624cdb1a 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceBuilder.java @@ -497,21 +497,18 @@ public PulsarSource build() { } } - // Schema evolution check. - if (deserializationSchema instanceof PulsarSchemaWrapper - && !Boolean.TRUE.equals(configBuilder.get(PULSAR_READ_SCHEMA_EVOLUTION))) { + // Schema evolution validation. + if (Boolean.TRUE.equals(configBuilder.get(PULSAR_READ_SCHEMA_EVOLUTION))) { + checkState( + deserializationSchema instanceof PulsarSchemaWrapper, + "When enabling schema evolution, you must provide a Pulsar Schema in PulsarDeserializationSchema."); + } else if (deserializationSchema instanceof PulsarSchemaWrapper) { LOG.info( "It seems like you want to read message using Pulsar Schema." + " You can enableSchemaEvolution for using this feature." + " We would use Schema.BYTES as the default schema if you don't enable this option."); } - if (Boolean.TRUE.equals(configBuilder.get(PULSAR_READ_SCHEMA_EVOLUTION))) { - checkState( - deserializationSchema instanceof PulsarSchemaWrapper, - "When enabling schema evolution, you must use a Pulsar schema."); - } - if (!configBuilder.contains(PULSAR_CONSUMER_NAME)) { LOG.warn( "We recommend set a readable consumer name through setConsumerName(String) in production mode."); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java index 043331cec79b6..d86e6a95166dd 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java @@ -23,8 +23,11 @@ import org.apache.flink.api.connector.source.SourceReaderContext; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.pulsar.common.schema.BytesSchema; +import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; +import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarSchemaWrapper; import org.apache.flink.connector.pulsar.source.reader.emitter.PulsarRecordEmitter; import org.apache.flink.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; import org.apache.flink.connector.pulsar.source.reader.source.PulsarUnorderedSourceReader; @@ -36,6 +39,7 @@ import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.impl.PulsarClientImpl; @@ -73,6 +77,16 @@ public static SourceReader create( PulsarClient pulsarClient = createClient(sourceConfiguration); PulsarAdmin pulsarAdmin = createAdmin(sourceConfiguration); + // Choose the right schema to use. + Schema schema; + if (sourceConfiguration.isEnableSchemaEvolution()) { + PulsarSchema pulsarSchema = + ((PulsarSchemaWrapper) deserializationSchema).pulsarSchema(); + schema = new BytesSchema(pulsarSchema); + } else { + schema = Schema.BYTES; + } + // Create a message queue with the predefined source option. int queueCapacity = sourceConfiguration.getMessageQueueCapacity(); FutureCompletingBlockingQueue>> elementsQueue = @@ -91,6 +105,7 @@ public static SourceReader create( pulsarClient, pulsarAdmin, sourceConfiguration, + schema, cryptoKeyReader); return new PulsarOrderedSourceReader<>( @@ -116,6 +131,7 @@ public static SourceReader create( pulsarClient, pulsarAdmin, sourceConfiguration, + schema, cryptoKeyReader, coordinatorClient); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java index 3e3bc4217bf70..4a116cdc64dc1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchema.java @@ -76,16 +76,7 @@ default void open(InitializationContext context) throws Exception { * @param message The message decoded by pulsar. * @param out The collector to put the resulting messages. */ - void deserialize(Message message, Collector out) throws Exception; - - /** - * By default, deserializers will decode bytes array message. - * - * @return The related Pulsar Schema for this serializer. - */ - default Schema schema() { - return Schema.BYTES; - } + void deserialize(Message message, Collector out) throws Exception; /** * Create a PulsarDeserializationSchema by using the flink's {@link DeserializationSchema}. It diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java index e3f0a7770d2ff..2b9841656da36 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarDeserializationSchemaWrapper.java @@ -52,7 +52,7 @@ public void open(InitializationContext context, SourceConfiguration configuratio } @Override - public void deserialize(Message message, Collector out) throws Exception { + public void deserialize(Message message, Collector out) throws Exception { byte[] bytes = message.getData(); T instance = deserializationSchema.deserialize(bytes); out.collect(instance); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java index 2bcc163d198c1..c515008e4f020 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarSchemaWrapper.java @@ -18,10 +18,8 @@ package org.apache.flink.connector.pulsar.source.reader.deserializer; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; -import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.util.Collector; import org.apache.pulsar.client.api.Message; @@ -37,37 +35,23 @@ * @param The output type of the message. */ @Internal -class PulsarSchemaWrapper implements PulsarDeserializationSchema { +public class PulsarSchemaWrapper implements PulsarDeserializationSchema { private static final long serialVersionUID = -4864701207257059158L; - /** The serializable pulsar schema, it wrap the schema with type class. */ + /** The serializable pulsar schema, it wraps the schema with type class. */ private final PulsarSchema pulsarSchema; - private boolean isSchemaEvolutionEnabled; - public PulsarSchemaWrapper(PulsarSchema pulsarSchema) { this.pulsarSchema = pulsarSchema; } @Override - public void open( - DeserializationSchema.InitializationContext context, SourceConfiguration configuration) - throws Exception { - this.isSchemaEvolutionEnabled = configuration.isEnableSchemaEvolution(); - } + public void deserialize(Message message, Collector out) throws Exception { + Schema schema = this.pulsarSchema.getPulsarSchema(); + byte[] bytes = message.getData(); + T instance = schema.decode(bytes); - @Override - public void deserialize(Message message, Collector out) throws Exception { - if (isSchemaEvolutionEnabled) { - @SuppressWarnings("unchecked") - T value = (T) message.getValue(); - out.collect(value); - } else { - Schema schema = this.pulsarSchema.getPulsarSchema(); - byte[] bytes = message.getData(); - T instance = schema.decode(bytes); - out.collect(instance); - } + out.collect(instance); } @Override @@ -76,12 +60,7 @@ public TypeInformation getProducedType() { return createTypeInformation(info); } - @Override - public Schema schema() { - if (isSchemaEvolutionEnabled) { - return pulsarSchema.getPulsarSchema(); - } else { - return Schema.BYTES; - } + public PulsarSchema pulsarSchema() { + return pulsarSchema; } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java index 5ea9de99dff81..0118079f82fa4 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/deserializer/PulsarTypeInformationWrapper.java @@ -51,7 +51,7 @@ public PulsarTypeInformationWrapper(TypeInformation information, ExecutionCon } @Override - public void deserialize(Message message, Collector out) throws Exception { + public void deserialize(Message message, Collector out) throws Exception { DataInputDeserializer dis = DESERIALIZER.get(); dis.setBuffer(message.getData()); T instance = serializer.deserialize(dis); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java index f9416305ecb7c..3131c00cb8524 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java @@ -32,6 +32,7 @@ import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.impl.MessageIdImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,8 +58,9 @@ public PulsarOrderedPartitionSplitReader( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, + Schema schema, @Nullable CryptoKeyReader cryptoKeyReader) { - super(pulsarClient, pulsarAdmin, sourceConfiguration, cryptoKeyReader); + super(pulsarClient, pulsarAdmin, sourceConfiguration, schema, cryptoKeyReader); } @Override diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java index 963f17c1b1a69..52033c2c7df58 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java @@ -63,6 +63,7 @@ abstract class PulsarPartitionSplitReaderBase protected final PulsarClient pulsarClient; protected final PulsarAdmin pulsarAdmin; protected final SourceConfiguration sourceConfiguration; + protected final Schema schema; @Nullable protected final CryptoKeyReader cryptoKeyReader; protected final AtomicBoolean wakeup; @@ -74,10 +75,12 @@ protected PulsarPartitionSplitReaderBase( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, + Schema schema, @Nullable CryptoKeyReader cryptoKeyReader) { this.pulsarClient = pulsarClient; this.pulsarAdmin = pulsarAdmin; this.sourceConfiguration = sourceConfiguration; + this.schema = schema; this.cryptoKeyReader = cryptoKeyReader; this.wakeup = new AtomicBoolean(false); } @@ -203,10 +206,8 @@ protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { } protected Consumer createPulsarConsumer(TopicPartition partition) { - // Schema schema = deserializationSchema.schema(); - ConsumerBuilder consumerBuilder = - createConsumerBuilder(pulsarClient, Schema.BYTES, sourceConfiguration); + createConsumerBuilder(pulsarClient, schema, sourceConfiguration); consumerBuilder.topic(partition.getFullTopicName()); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java index bbae002d8f3a3..eb7064872db1b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -30,6 +30,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.transaction.Transaction; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClientException; @@ -57,8 +58,6 @@ public class PulsarUnorderedPartitionSplitReader extends PulsarPartitionSplitRea private static final Logger LOG = LoggerFactory.getLogger(PulsarUnorderedPartitionSplitReader.class); - private static final Duration REDELIVER_TIME = Duration.ofSeconds(3); - private final TransactionCoordinatorClient coordinatorClient; @Nullable private Transaction uncommittedTransaction; @@ -67,9 +66,10 @@ public PulsarUnorderedPartitionSplitReader( PulsarClient pulsarClient, PulsarAdmin pulsarAdmin, SourceConfiguration sourceConfiguration, + Schema schema, @Nullable CryptoKeyReader cryptoKeyReader, TransactionCoordinatorClient coordinatorClient) { - super(pulsarClient, pulsarAdmin, sourceConfiguration, cryptoKeyReader); + super(pulsarClient, pulsarAdmin, sourceConfiguration, schema, cryptoKeyReader); this.coordinatorClient = coordinatorClient; } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java index 6c4d4caa1e3f8..ae19e222daa5b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableDeserializationSchema.java @@ -27,7 +27,6 @@ import org.apache.flink.util.Collector; import org.apache.pulsar.client.api.Message; -import org.apache.pulsar.client.api.Schema; import javax.annotation.Nullable; @@ -49,7 +48,6 @@ * PulsarRowDataConverter} instance. */ public class PulsarTableDeserializationSchema implements PulsarDeserializationSchema { - private static final long serialVersionUID = -3298784447432136216L; private final TypeInformation producedTypeInfo; @@ -82,7 +80,8 @@ public void open( } @Override - public void deserialize(Message message, Collector collector) throws IOException { + public void deserialize(Message message, Collector collector) + throws IOException { // Get the value row data List valueRowData = new ArrayList<>(); valueDeserialization.deserialize(message.getData(), new ListCollector<>(valueRowData)); @@ -101,9 +100,4 @@ public void deserialize(Message message, Collector collector) throws public TypeInformation getProducedType() { return producedTypeInfo; } - - @Override - public Schema schema() { - return Schema.BYTES; - } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java index f74c9de8ec670..78f849efa7385 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java @@ -55,7 +55,11 @@ void publishTimeStopCursor() throws IOException { PulsarOrderedPartitionSplitReader splitReader = new PulsarOrderedPartitionSplitReader( - operator().client(), operator().admin(), sourceConfig(), null); + operator().client(), + operator().admin(), + sourceConfig(), + Schema.BYTES, + null); // send the first message and set the stopCursor to filter any late stopCursor operator() .sendMessage( diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java index 261733b5390ae..346b05bde0fac 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java @@ -34,6 +34,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; import org.junit.jupiter.api.TestTemplate; @@ -328,10 +329,15 @@ void assignNoSplits(PulsarPartitionSplitReaderBase splitReader) { private PulsarPartitionSplitReaderBase splitReader(SubscriptionType subscriptionType) { if (subscriptionType == SubscriptionType.Failover) { return new PulsarOrderedPartitionSplitReader( - operator().client(), operator().admin(), sourceConfig(), null); + operator().client(), operator().admin(), sourceConfig(), Schema.BYTES, null); } else { return new PulsarUnorderedPartitionSplitReader( - operator().client(), operator().admin(), sourceConfig(), null, null); + operator().client(), + operator().admin(), + sourceConfig(), + Schema.BYTES, + null, + null); } } From a82ab8fc1f290855a72b1ff9ba60a1b80bf6cdc2 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Mon, 1 Aug 2022 11:11:44 +0800 Subject: [PATCH 236/258] 137: use a random default subscription name in Pulsar SQL Connector --- .../flink/connector/pulsar/table/PulsarTableFactory.java | 9 ++++++++- .../flink/connector/pulsar/table/PulsarTableOptions.java | 4 ++-- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java index c4b32c83c0e52..97d76bf8da165 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java @@ -86,6 +86,7 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSinkOptions; import static org.apache.flink.connector.pulsar.table.PulsarTableValidationUtils.validateTableSourceOptions; import static org.apache.flink.table.factories.FactoryUtil.SINK_PARALLELISM; +import static org.apache.pulsar.shade.org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; /** * Factory for creating {@link DynamicTableSource} and {@link DynamicTableSink}. @@ -97,6 +98,8 @@ public class PulsarTableFactory implements DynamicTableSourceFactory, DynamicTab public static final String IDENTIFIER = "pulsar"; + public static final String DEFAULT_SUBSCRIPTION_NAME_PREFIX = "flink-sql-connector-pulsar-"; + @Override public DynamicTableSource createDynamicTableSource(Context context) { FactoryUtil.TableFactoryHelper helper = FactoryUtil.createTableFactoryHelper(this, context); @@ -131,8 +134,12 @@ public DynamicTableSource createDynamicTableSource(Context context) { final Properties properties = getPulsarProperties(tableOptions); properties.setProperty(PULSAR_ADMIN_URL.key(), tableOptions.get(ADMIN_URL)); properties.setProperty(PULSAR_SERVICE_URL.key(), tableOptions.get(SERVICE_URL)); + // Set random subscriptionName if not provided properties.setProperty( - PULSAR_SUBSCRIPTION_NAME.key(), tableOptions.get(SOURCE_SUBSCRIPTION_NAME)); + PULSAR_SUBSCRIPTION_NAME.key(), + tableOptions + .getOptional(SOURCE_SUBSCRIPTION_NAME) + .orElse(DEFAULT_SUBSCRIPTION_NAME_PREFIX + randomAlphabetic(5))); // Retrieve physical fields (not including computed or metadata fields), // and projections and create a schema factory based on such information. final DataType physicalDataType = context.getPhysicalRowDataType(); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java index df131ea225eea..b1b2bd835ad5f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java @@ -70,11 +70,11 @@ private PulsarTableOptions() {} public static final ConfigOption SOURCE_SUBSCRIPTION_NAME = ConfigOptions.key("source.subscription-name") .stringType() - .defaultValue("flink-sql-connector-pulsar") + .noDefaultValue() .withDescription( Description.builder() .text( - "Specify the subscription name consumer used by runtime PulsarSource.") + "Specify the subscription name consumer used by runtime PulsarSource. If not provided, a random subscription name will be generated") .text( " This argument is required when constructing the consumer.") .build()); From c23bca67cebd0545276f45710e023868da129047 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Mon, 13 Jun 2022 11:19:50 +0800 Subject: [PATCH 237/258] 138: update the documentation --- .github/README.md | 11 + docs/content/docs/connectors/table/pulsar.md | 414 ++++++++++++++++++ .../pulsar_catalog_factory_configuration.html | 48 ++ .../generated/pulsar_table_configuration.html | 90 ++++ .../pulsar/table/PulsarTableOptions.java | 70 ++- .../catalog/PulsarCatalogFactoryOptions.java | 18 +- .../table/catalog/impl/SchemaTranslator.java | 8 +- .../ConfigOptionsDocGenerator.java | 6 + 8 files changed, 630 insertions(+), 35 deletions(-) create mode 100644 .github/README.md create mode 100644 docs/content/docs/connectors/table/pulsar.md create mode 100644 docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html create mode 100644 docs/layouts/shortcodes/generated/pulsar_table_configuration.html diff --git a/.github/README.md b/.github/README.md new file mode 100644 index 0000000000000..cb97d3c8652f4 --- /dev/null +++ b/.github/README.md @@ -0,0 +1,11 @@ +# StreamNative's fork of Apache Flink + +This repository is maintained by StreamNative to host a StreamNative fork of the Flink Connector for Pulsar. Only the Flink Connector for Pulsar (including the Flink SQL Connector for Pulsar and the Flink DataStream Connector for Pulsar) diverges with the official Apache Flink repository. + +The following is some useful documentation and resources for the DataStream and SQL Connectors + +- [Flink SQL Connector for Pulsar documentation](../docs/content/docs/connectors/table/pulsar.md) +- [flink-example: cookbooks, examples](https://github.com/streamnative/flink-example) +- [StreamNative Hub documentation](https://hub.streamnative.io/data-processing/pulsar-flink/1.15.0.1) + +> The legacy [pulsar-flink connector](https://github.com/streamnative/pulsar-flink) is deprecated. It will not be actively maintained and users are recommended to use this repository. diff --git a/docs/content/docs/connectors/table/pulsar.md b/docs/content/docs/connectors/table/pulsar.md new file mode 100644 index 0000000000000..74d81491082fe --- /dev/null +++ b/docs/content/docs/connectors/table/pulsar.md @@ -0,0 +1,414 @@ +# Pulsar SQL connector + +Currently, the Pulsar SQL connector is not delivered as a part of the official Flink binary. It is maintained by [StreamNative](https://streamnative.io/). + +The Pulsar SQL connector allows you to query data from or write data into Pulsar topics using simple SQL queries or the [Flink Table API](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/tableapi/). + +## Dependencies + +This table outlines the Maven dependency that is added to the Pulsar SQL connector. + + + + + + + + + +
    Maven Dependency SQL JAR
    + +```xml + + + io.streamnative.connectors + flink-sql-connector-pulsar + 1.15.0.1 + +``` + + + +[SQL JAR](https://repo1.maven.org/maven2/io/streamnative/connectors/flink-sql-connector-pulsar/) + +
    + +## Pulsar SQL connector options + +This table outlines options for the Pulsar SQL connector. + +| Key | Default | Type | Description | +| --- | --- | --- | --- | +| `admin-url` | (none) | String | The Pulsar service HTTP URL for the admin endpoint. For example, `http://my-broker.example.com:8080` or `https://my-broker.example.com:8443` for TLS. | +| `explicit` | true | Boolean | Indicate if the table is an explicit Flink table. | +| `key.fields` | | List<String> | An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined. | +| `key.format` | (none) | String | The format that is used to deserialize and serialize the key bytes of Pulsar messages. The format identifier is used to discover a suitable format factory. | +| `service-url` | (none) | String | The Service URL for the Pulsar service. To connect to a Pulsar cluster using a client library, you need to specify a Pulsar protocol URL. You can assign a Pulsar protocol URL to a specific cluster.
    -This is an example URL of the `localhost:pulsar://localhost:6650`.
    - If you have multiple brokers, the URL is something like `pulsar://localhost:6550,localhost:6651,localhost:6652`.
    - A URL for a production Pulsar cluster is something like `pulsar://pulsar.us-west.example.com:6650`.
    - If TLS authentication is enabled, the URL is something like `pulsar+ssl://pulsar.us-west.example.com:6651`. | +| `sink.custom-topic-router` | (none) | String | (Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the `sink.topic-routing-mode` option will be ignored. | +| `sink.message-delay-interval` | 0 ms | Duration | (Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). | +| `sink.topic-routing-mode` | round-robin | Enum | (Optional) the topic routing mode. Available options are `round-robin` and `message-key-hash`. By default, it is set to `round-robin`. If you want to use a custom topic router, use the `sink.custom-topic-router` option to determine the partition for a particular message.
    - `round-robin`: the producer publishes messages across all partitions in a round-robin fashion to achieve the maximum throughput. The round-robin method is not implemented for individual messages. However, it is set to the same boundary of the `pulsar.producer.batchingMaxMessages` option to make batching take effect.
    - `message-key-hash`: if no key is provided, the partitioned producer will randomly pick one single topic partition and publish all messages to that partition. If a key is provided for a message, the partitioned producer will hash the key and assign the message to a particular partition. | +| `source.start.message-id` | (none) | String | (Optional) the message ID that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data. Available options are `earliest`, `latest`, and the message ID (in a format of `ledgerId:entryId:partitionId`, such as "12:2:-1"). | +| `source.start.publish-time` | (none) | Long | (Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data. | +| `source.subscription-name` | flink-sql-connector-pulsar | String | The subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer. | +| `source.subscription-type` | Exclusive | Enum | The [subscription type](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-subscriptions) that is supported by the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). Currently, only `Exclusive` and `Shared` subscription types are supported. | +| `topics` | (none) | List<String> | Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (`;`) like `topic-1;topic-2`. | + +## Features + +This section describes some significant features for the Pulsar SQL connector. + +### Available metadata + +The connector metadata are the metadata specific to the external system in use. [Flink SQL](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/fsql/overview/) supports using metadata from the external system or their decoding/encoding formats as a table column. + +This table outlines the Pulsar topic metadata that can be mapped by the Pulsar SQL connector to Flink table fields. + +> **Note** +> +> - The `R/W` column defines whether a metadata is readable from the Pulsar topics (`R`) and/or writable to the Pulsar topics (`W`). +> - The `R` column defines that a metadata is read-only. The read-only metadata must be declared `VIRTUAL` to exclude them during an `INSERT INTO` operation. + +| Key | Data Type | Description | R/W | +|:--------------|:--------------------------|:------------|-----| +| topic | STRING NOT NULL | | R | +| message_size | INT NOT NULL | | R | +| producer_name | STRING NOT NULL | | R | +| message_id | BYTES NOT NULL | | R | +| sequenceId | BIGINT NOT NULL | | R | +| publish_time | TIMESTAMP_LTZ(3) NOT NULL | | R | +| event_time | TIMESTAMP_LTZ(3) NOT NULL | | R/W | + +> **Note** +> +> Currently, the `properties` metadata field is not supported yet. + +The extended `CREATE TABLE` example demonstrates the syntax for exposing `publish_time`, `producer_name`, and `topic` metadata fields. + +```sql +CREATE TABLE user +( + `publish_time` TIMESTAMP(3) METADATA VIRTUAL FROM 'publish_time', + `producer_name` STRING METADATA VIRTUAL, + `topic` STRING METADATA VIRTUAL, + `uid` BIGINT, + `item_id` BIGINT, + `description` STRING +) WITH ( + 'connector' = 'pulsar', + 'topics' = 'persistent://public/default/user', + 'service-url' = 'pulsar://localhost:6650', + 'admin-url' = 'http://localhost:8080', + 'source.start.message-id' = 'earliest' , + 'format' = 'json' +); +``` + +### Key and value formats + +You can serialize or deserialize the key and the value of a Pulsar message to or from raw bytes using one of the Flink official [formats](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/overview/). For details about how to configure the key and value formats, see [configure key and value formats](#configure-key-and-value-formats). + +#### Value format + +If the key of a Pulsar message is absent or ignored, you should use the `format` option (a synonym for `value.format`) to designate a decoded/encoded format for the Pulsar message. For details, see the [formats documentation](https://docs.google.com/document/d/1itz4cyTYVmxf3g_iBYyh6rN092d4Se5-66X0kAP6ERo/edit#). All format options are prefixed with the format identifier, such as `json.ignore-parse-errors`. + +```sql +CREATE TABLE users +( + `uid` BIGINT, + `item_id` BIGINT, + `description` STRING +) WITH ( + 'connector' = 'pulsar', + 'topics' = 'persistent://public/default/users', + 'service-url' = 'pulsar://localhost:6650', + ... + 'format' = 'json', + 'json.ignore-parse-errors' = 'true' +); +``` + +You can use the value format to deserialize a Pulsar message to the following Flink data types: + +```text +ROW<`uid` BIGINT, `item_id` BIGINT, `description` STRING> +``` + +#### Key format + +The deserialized or serialized key of a Pulsar message can be mapped to a Flink table field. The key format includes the fields that are listed in the `key.fields` option. You can use the semicolon symbol (`;`) to separate multiple fields. Thus, the key format is configured with the following data type: + +```text +ROW<`uid` BIGINT, `item_id` BIGINT> +``` + +And all the key fields are excluded in the value format's data type: + +```text +ROW<`description` STRING> +``` + +### Topic and partition discovery + +The `topics` and `topic-pattern` options specify the topics and topic pattern for the [Pulsar DataStream connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/) to consume or produce data. + +- `topics`: a list of topics that are separated using the semicolon symbol (`;`) like `topic-1;topic-2`. + +- `topic-pattern`: uses the regular expression to discover the matched topics. For example, if the `topic-pattern` is set to `test-topic-[0-9]`, then all topics whose names match the specified regular expression (starting with `test-topic-` and ending with a single digit) will be subscribed to by the consumer when running a query job. + +By default, the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) dynamically checks topic changes every 30 seconds, such as the number of topic partitions and topics. To disable automatic topic and partition discovery, you can set a negative value for the `pulsar.source.partitionDiscoveryIntervalMs` option. For details, see [Pulsar DataStream connector documentation](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/datastream/pulsar/). + +### Starting position + +You can use the `source.start.message-id` or `source.start-publish-time` option to designate the starting position for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). + +You can configure one of the following values for the `source.start.message-id` option: + +- `earliest` +- `latest` +- A message ID (in `::` format) + +You can configure the `source.start.publish-time` option with a Long type timestamp value. + +The `source.start.message-id` and `source.start-publish-time` options are exclusive. If both options are configured, the validation will fail. By default, the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) uses the `StartCursor.earliest()` option. + +### Sink topic router + +The Pulsar DataStream connector supports designating a topic router. In the Pulsar SQL connector, you can configure the sink topic router using the `sink.topic-routing-mode` or `sink.custom-topic-router` option. + +- `sink.topic-routing-mode`: the built-in topic router implementation. Available values are `round-robin` and `message-key-hash`. +- `sink.custom-topic-router`: the full class name of the custom topic router. Make sure that the custom topic router implementation is set on the classpath. + +The `sink.topic-routing-mode` and `sink.custom-topic-router` options are exclusive. If both options are configured, the validation will fail. + +### Data type mapping + +[Pulsar Schema](https://pulsar.apache.org/docs/schema-get-started/) describes how to deserialize data from and serialize data to a message stored in Pulsar topics. Flink SQL also has its own data types as well as the deserialization and serialization framework. The Pulsar SQL connector provides multiple choices to map a Pulsar schema to Flink data types to transfer data between Flink SQL and Pulsar. + +Flink SQL uses [formats](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/overview/) to decode and encode data from external systems, such as [flink-csvs](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/csv/), [flink-json](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/json/), and [flink-avro](https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/connectors/table/formats/avro/). If you want to read a Pulsar topic with a predefined Pulsar schema (JSON, CSV, or Avro schema), you can use the related Flink format. + +This table outlines the recommended Flink format for each Pulsar Schema. To use the `avro` and `json` format, you should add `flink-avro` and `flink-json` to your dependencies if they are not added yet. (By default, the JAR package of the Pulsar SQL connector is bundled with them). + +| Pulsar schema | Flink format | +|-----------------|-------------------| +| AVRO | avro | +| JSON | json | +| PROTOBUF | Not supported yet | +| PROTOBUF_NATIVE | Not supported yet | +| AUTO_CONSUME | Not supported yet | +| AUTO_PUBLISH | Not supported yet | +| NONE/BYTES | raw | +| BOOLEAN | raw | +| STRING | raw | +| DOUBLE | raw | +| FLOAT | raw | +| INT8 | raw | +| INT16 | raw | +| INT32 | raw | +| INT64 | raw | +| LOCAL_DATE | Not supported yet | +| LOCAL_TIME | Not supported yet | +| LOCAL_DATE_TIME | Not supported yet | + +> **Note** +> +> When Flink writes data to a Pulsar topic with the Avro format, the Pulsar consumer cannot consume data with the Avro schema. + +## PulsarCatalog + +PulsarCatalog supports configuring the Pulsar cluster as the metadata storage for Flink tables. + +### Explicit tables and native tables + +PulsarCatalog defines two different kinds of tables: `explicit` tables and `native` tables. + +- An `explicit` table is a table explicitly created using the `CREATE` statements or table API. It is like the common use pattern in other SQL connectors. You can create a table and then query data from or write data to the table. +- A `native` table is automatically created by PulsarCatalog. PulsarCatalog scans all non-system topics in a Pulsar cluster, and then maps each topic into a Flink table without using the `CREATE` statements. + +#### Explicit tables + +PulsarCatalog uses the Pulsar `SchemaInfo` to store the Flink schema bytes for an `explicit table`. + +For each `explicit` table, PulsarCatalog creates a **placehoder** topic under a preconfigured tenant in the Pulsar cluster. The default value for such a preconfigured tenant is named `__flink_catalog`, but you can use the `catalog-tenant` option to specify a different tenant name. The Flink database is then mapped to a namespace with the same name. Finally, the **placehoder** topic that saves the Flink table schema information is named as `table_`. + +For example, if you create a table `users` under the `testdb` database, then a topic named `table_users` is created under the `__flink_catalog` tenant in the`testdb` namespace. + +The table like `table_users` is called a **placeholder topic**, because these topics do not have any producer or consumer. Therefore, no data would flow in or out of these topics. You can use the schema information of such topics to store the Flink table metadata. + +For the JSON format, the Flink table schema is serialized and then stored under the `schema` key. Other table options from `CREATE TABLE xxx WITH ()` statements are stored under the `properties` key. + +You can use the `pulsar-admin` CLI tool to retrieve the JSON format of a topic schema: + +```bash +pulsar-admin schemas get persistent://// +``` + +#### Native tables + +A `native` table does not have any **placeholder topic**. Instead, PulsarCatalog maps the Pulsar schema of the `native table` to a Flink table schema when creating a SQL job. + +This table outlines the Flink formats that are used for a native table's Pulsar schema. + +| Pulsar schema | Flink data type | Flink format | Work or not | +|-----------------|-------------------------------------------|-------------------|-------------| +| AVRO | It is decided by the Avro format. | avro | Yes | +| JSON | It is decided by the JSON format. | json | Yes | +| PROTOBUF | Not supported yet | / | No | +| PROTOBUF_NATIVE | It is decided by the Protobuf definition. | Not supported yet | No | +| AUTO_CONSUME | Not supported yet | / | No | +| AUTO_PUBLISH | Not supported yet | / | No | +| NONE/BYTES | DataTypes.BYTES() | raw | Yes | +| BOOLEAN | DataTypes.BOOLEAN() | raw | Yes | +| LOCAL_DATE | DataTypes.DATE() | / | No | +| LOCAL_TIME | DataTypes.TIME() | / | No | +| LOCAL_DATE_TIME | DataTypes.TIMESTAMP(3) | / | No | +| STRING | DataTypes.STRING() | raw | Yes | +| DOUBLE | DataTypes.DOUBLE() | raw | Yes | +| FLOAT | DataTypes.FLOAT() | raw | Yes | +| INT8 | DataTypes.TINYINT() | raw | Yes | +| INT16 | DataTypes.SMALLINT() | raw | Yes | +| INT32 | DataTypes.INT() | raw | Yes | +| INT64 | DataTypes.BIGINT() | raw | Yes | + +> **Note** +> +> Even if there are corresponding Flink data formats for the `LOCAL_DATE`, `LOCAL_TIME`, and `LOCAL_DATE_TIME` options, the `raw` format is not able to decode messages with these Pulsar schemas. In this case, the auto schema mapping will fail. + +#### Comparison + +After a `native` table is created, you can query data from existing Pulsar topics. PulsarCatalog automatically reads the topic's schema and decides which decoded/encoded format to use. However, the `native` table does not support watermarks and primary keys. Therefore, you cannot use the `native` table to do window aggregation. A `native` table maps `tenant/namespace` to a database and the topic name to the table name. + +To fully manage a table, you can use the `explicit` table to define watermark fields, specify metadata fields, and specify a custom format. The usage is similar to creating a Pulsar table in `GenericInMemoryCatalog`. You can bind an `explicit` table to a Pulsar topic and each Pulsar topic can be bound to multiple Flink tables (including the `native` table). + +### PulsarCatalog options + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    KeyDefaultTypeDescriptionRequired
    catalog-admin-url
    "http://localhost:8080"String(Required) The admin URL of the Pulsar cluster.Yes
    catalog-auth-params
    (none)StringThe authentication parameters for accessing the Pulsar cluster.
    catalog-auth-plugin
    (none)StringThe name of the authentication plugin for accessing the Pulsar cluster.
    catalog-service-url
    "pulsar://localhost:6650"String(Required) The service URL of the Pulsar cluster.Yes
    catalog-tenant
    "__flink_catalog"StringThe Pulsar tenant that stores all table information.
    default-database
    "default"StringThe default database when using PulsarCatalog. It will be created if it does not exist.
    + +### Create PulsarCatalog + +You can create PulsarCatalog using any of the following ways: + +- SQL query statements +- Flink [Table API](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/common/) + +This example shows how to create PulsarCatalog using the SQL query statements. + +```sql +CREATE +CATALOG pulsar + WITH ( + 'type' = 'pulsar-catalog', + 'catalog-admin-url' = '', + 'catalog-service-url' = '' +); +``` + +This example shows how to create PulsarCatalog using the Flink [Table API](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/common/). + +```java +Catalog pulsarCatalog = new PulsarCatalog(); +tableEnv.registerCatalog("pulsar",pulsarCatalog); +``` + +> **Note** +> +> If you use the SQL Client and configure catalogs in `sql-clients-defaults.yaml` YAML file, make sure that this file has been removed due to [FLIP-163](https://cwiki.apache.org/confluence/display/FLINK/FLIP-163%3A+SQL+Client+Improvements) and this file is no longer available for Flink 1.15. + +## Examples + +This section lists some examples about the Pulsar SQL connector. + +### Create a table + +This example shows how to use the Pulsar SQL connector to create a table in a JSON encoding/decoding format. + +```sql +CREATE TABLE users +( + `user_id` BIGINT, + `item_id` BIGINT, + `description` STRING, + `ts` TIMESTAMP(3) METADATA FROM 'timestamp' +) WITH ( + 'connector' = 'pulsar', + 'topics' = 'persistent://public/default/users', + 'service-url' = 'pulsar://localhost:6650', + 'admin-url' = 'http://localhost:8080', + 'format' = 'json' +) +``` + +### Configure key and value formats + +This example shows how to specify and configure the key format and value formats. The format options are prefixed with either the `key` or `value` plus the format identifier. + +```sql +CREATE TABLE users +( + `uid` BIGINT, + `item_id` BIGINT, + `description` STRING +) WITH ( + 'connector' = 'pulsar', + 'topics' = 'persistent://public/default/users', + 'service-url' = 'pulsar://localhost:6650', + 'admin-url' = 'http://localhost:8080', + 'key.format' = 'json', + 'key.json.ignore-parse-errors' = 'true', + 'key.fields' = 'uid;item_id', + 'value.format' = 'json', + 'value.json.fail-on-missing-field' = 'false', +``` diff --git a/docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html b/docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html new file mode 100644 index 0000000000000..ee205368fdf6d --- /dev/null +++ b/docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html @@ -0,0 +1,48 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    KeyDefaultTypeDescription
    catalog-admin-url
    "http://localhost:8080"String(Required) The admin URL of the Pulsar cluster.
    catalog-auth-params
    (none)StringThe authentication parameters for accessing the Pulsar cluster.
    catalog-auth-plugin
    (none)StringThe name of the authentication plugin for accessing the Pulsar cluster.
    catalog-service-url
    "pulsar://localhost:6650"String(Required) The service URL of the Pulsar cluster.
    catalog-tenant
    "__flink_catalog"StringThe Pulsar tenant that stores all table information.
    default-database
    "default"StringThe default database when using PulsarCatalog. It will be created if it does not exist.
    diff --git a/docs/layouts/shortcodes/generated/pulsar_table_configuration.html b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html new file mode 100644 index 0000000000000..c84bc6bb1382c --- /dev/null +++ b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html @@ -0,0 +1,90 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    KeyDefaultTypeDescription
    admin-url
    (none)StringThe Pulsar service HTTP URL for the admin endpoint. For example, http://my-broker.example.com:8080, or https://my-broker.example.com:8443 for TLS.
    explicit
    trueBooleanIndicate if the table is an explicit Flink table.
    key.fields
    List<String>An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined.
    key.format
    (none)StringDefines the format identifier for decoding/encoding key bytes in Pulsar message. The identifier is used to discover a suitable format factory.
    service-url
    (none)StringService URL provider for Pulsar service.
    To connect to Pulsar using client libraries, you need to specify a Pulsar protocol URL.
    You can assign Pulsar protocol URLs to specific clusters and use the Pulsar scheme.
    • This is an example of localhost: pulsar://localhost:6650.
    • If you have multiple brokers, the URL is as: pulsar://localhost:6550,localhost:6651,localhost:6652
    • A URL for a production Pulsar cluster is as: pulsar://pulsar.us-west.example.com:6650
    • If you use TLS authentication, the URL is as pulsar+ssl://pulsar.us-west.example.com:6651
    sink.custom-topic-router
    (none)String(Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the sink.topic-routing-mode option will be ignored.
    sink.message-delay-interval
    0 msDuration(Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink).
    sink.topic-routing-mode
    round-robin

    Enum

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

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

    Enum

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

    Possible values:
    • "Exclusive"
    • "Shared"
    • "Failover"
    • "Key_Shared"
    topics
    (none)List<String>Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (;) like topic-1;topic-2.
    diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java index b1b2bd835ad5f..b805dd47153ec 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.pulsar.table; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.description.Description; @@ -39,7 +40,8 @@ * org.apache.flink.connector.pulsar.source.PulsarSourceOptions}, and {@link * org.apache.flink.connector.pulsar.sink.PulsarSinkOptions}. */ -public class PulsarTableOptions { +@PublicEvolving +public final class PulsarTableOptions { private PulsarTableOptions() {} @@ -49,7 +51,11 @@ private PulsarTableOptions() {} .asList() .noDefaultValue() .withDescription( - "Topic names from which the table is read. It is required for both source and sink"); + Description.builder() + .text( + "Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (%s) like %s.", + code(";"), code("topic-1;topic-2")) + .build()); // -------------------------------------------------------------------------------------------- // Table Source Options @@ -60,7 +66,11 @@ private PulsarTableOptions() {} .enumType(SubscriptionType.class) .defaultValue(SubscriptionType.Exclusive) .withDescription( - "Subscription type for Pulsar source to use. Only \"Exclusive\" and \"Shared\" are allowed."); + Description.builder() + .text( + "The [subscription type](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-subscriptions) that is supported by the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). Currently, only %s and %s subscription types are supported.", + code("Exclusive"), code("Shared")) + .build()); /** * Exactly same as {@link @@ -74,9 +84,7 @@ private PulsarTableOptions() {} .withDescription( Description.builder() .text( - "Specify the subscription name consumer used by runtime PulsarSource. If not provided, a random subscription name will be generated") - .text( - " This argument is required when constructing the consumer.") + "The subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer.") .build()); public static final ConfigOption SOURCE_START_FROM_MESSAGE_ID = @@ -84,18 +92,27 @@ private PulsarTableOptions() {} .stringType() .noDefaultValue() .withDescription( - "Optional message id used to specify a consuming starting point for " - + "source. Use \"earliest\", \"latest\" or pass in a message id " - + "representation in \"ledgerId:entryId:partitionId\", " - + "such as \"12:2:-1\""); + Description.builder() + .text( + "Optional message id used to specify a consuming starting point for " + + "source. Use %s, %s or pass in a message id " + + "representation in %s, " + + "such as %s", + code("earliest"), + code("latest"), + code("ledgerId:entryId:partitionId"), + code("12:2:-1")) + .build()); public static final ConfigOption SOURCE_START_FROM_PUBLISH_TIME = ConfigOptions.key("source.start.publish-time") .longType() .noDefaultValue() .withDescription( - "Optional publish timestamp used to specify a consuming starting point for source."); - + Description.builder() + .text( + "(Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data.") + .build()); // -------------------------------------------------------------------------------------------- // Table Sink Options // -------------------------------------------------------------------------------------------- @@ -105,24 +122,32 @@ private PulsarTableOptions() {} .stringType() .noDefaultValue() .withDescription( - "Optional custom TopicRouter implementation class URL to use in sink. If this option" - + "is provided, \"sink.topic-routing-mode\" will be ignored."); + Description.builder() + .text( + "(Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the %s option will be ignored.", + code("sink.topic-routing-mode")) + .build()); public static final ConfigOption SINK_TOPIC_ROUTING_MODE = ConfigOptions.key("sink.topic-routing-mode") .enumType(TopicRoutingMode.class) .defaultValue(TopicRoutingMode.ROUND_ROBIN) .withDescription( - "Optional TopicRoutingMode. There are \"round-robin\" and " - + "\"message-key-hash\" two options. Default use" - + "\"round-robin\", if you want to use a custom" - + "TopicRouter implementation, use \"sink.custom-topic-router\""); + Description.builder() + .text( + "(Optional) the topic routing mode. Available options are %s and %s. By default, it is set to %s. If you want to use a custom topic router, use the %s option to determine the partition for a particular message.", + code("round-robin"), + code("message-key-hash"), + code("round-robin"), + code("sink.custom-topic-router")) + .build()); public static final ConfigOption SINK_MESSAGE_DELAY_INTERVAL = ConfigOptions.key("sink.message-delay-interval") .durationType() .defaultValue(Duration.ZERO) - .withDescription("Optional sink message delay delivery interval."); + .withDescription( + "(Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink)."); // -------------------------------------------------------------------------------------------- // Format Options @@ -142,10 +167,7 @@ private PulsarTableOptions() {} .asList() .defaultValues() .withDescription( - "Defines an explicit list of physical columns from the " - + "table schema which should be decoded/encoded " - + "from the key bytes of a Pulsar message. By default, " - + "this list is empty and thus a key is undefined."); + "An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined."); public static final ConfigOption VALUE_FORMAT = ConfigOptions.key("value" + FORMAT_SUFFIX) @@ -226,5 +248,5 @@ private PulsarTableOptions() {} ConfigOptions.key("explicit") .booleanType() .defaultValue(true) - .withDescription("Indicate if the table is an explict flink table"); + .withDescription("Indicate if the table is an explicit Flink table."); } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java index 10005c6cad407..6ee4f3e66dc83 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogFactoryOptions.java @@ -18,50 +18,52 @@ package org.apache.flink.connector.pulsar.table.catalog; -import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.table.catalog.CommonCatalogOptions; /** {@link ConfigOption}s for {@link PulsarCatalog}. */ -@Internal +@PublicEvolving public final class PulsarCatalogFactoryOptions { public static final ConfigOption CATALOG_TENANT = ConfigOptions.key("catalog-tenant") .stringType() .defaultValue(PulsarCatalog.DEFAULT_TENANT) - .withDescription("Pulsar tenant used to store all table information"); + .withDescription("The Pulsar tenant that stores all table information."); public static final ConfigOption DEFAULT_DATABASE = ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) .stringType() .defaultValue(PulsarCatalog.DEFAULT_DB) .withDescription( - "The default database when using PulsarCatalog. It will be created if not exist."); + "The default database when using PulsarCatalog. It will be created if it does not exist."); public static final ConfigOption CATALOG_ADMIN_URL = ConfigOptions.key("catalog-admin-url") .stringType() .defaultValue("http://localhost:8080") - .withDescription("Required pulsar cluster admin url"); + .withDescription("(Required) The admin URL of the Pulsar cluster."); public static final ConfigOption CATALOG_SERVICE_URL = ConfigOptions.key("catalog-service-url") .stringType() .defaultValue("pulsar://localhost:6650") - .withDescription("Required pulsar cluster service url"); + .withDescription("(Required) The service URL of the Pulsar cluster."); public static final ConfigOption AUTH_PLUGIN = ConfigOptions.key("catalog-auth-plugin") .stringType() .noDefaultValue() - .withDescription("Auth plugin name for accessing pulsar cluster"); + .withDescription( + "The name of the authentication plugin for accessing the Pulsar cluster."); public static final ConfigOption AUTH_PARAMS = ConfigOptions.key("catalog-auth-params") .stringType() .noDefaultValue() - .withDescription("Auth params for accessing pulsar cluster"); + .withDescription( + "The authentication parameters for accessing the Pulsar cluster."); private PulsarCatalogFactoryOptions() {} } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java index 1cbde2d4adae2..b5df200a5b93c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/catalog/impl/SchemaTranslator.java @@ -226,9 +226,11 @@ public String decideDefaultFlinkFormat(SchemaInfo pulsarSchemaInfo) { case AUTO: case AUTO_PUBLISH: LOG.error( - "Do not support {} schema, using RAW instead", - pulsarSchemaInfo.getType().toString()); - break; + "Can't decide format for {} schema", pulsarSchemaInfo.getType().toString()); + throw new UnsupportedOperationException( + String.format( + "Can't decide format for %s schema", + pulsarSchemaInfo.getType().toString())); default: break; } diff --git a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java index 38b5121f4f45e..d86959e5d9232 100644 --- a/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java +++ b/flink-docs/src/main/java/org/apache/flink/docs/configuration/ConfigOptionsDocGenerator.java @@ -107,6 +107,12 @@ public class ConfigOptionsDocGenerator { new OptionsClassLocation( "flink-connectors/flink-connector-pulsar", "org.apache.flink.connector.pulsar.sink"), + new OptionsClassLocation( + "flink-connectors/flink-connector-pulsar", + "org.apache.flink.connector.pulsar.table.catalog"), + new OptionsClassLocation( + "flink-connectors/flink-connector-pulsar", + "org.apache.flink.connector.pulsar.table"), new OptionsClassLocation( "flink-libraries/flink-cep", "org.apache.flink.cep.configuration"), new OptionsClassLocation( From dd52584d1329a856c3f42c27ab28e5a17a6fac49 Mon Sep 17 00:00:00 2001 From: StreamNative Bot <44651360+streamnativebot@users.noreply.github.com> Date: Wed, 3 Aug 2022 12:55:56 -0700 Subject: [PATCH 238/258] Create new CODEOWNERS file --- .github/CODEOWNERS | 1 + 1 file changed, 1 insertion(+) create mode 100644 .github/CODEOWNERS diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS new file mode 100644 index 0000000000000..40b578fe59555 --- /dev/null +++ b/.github/CODEOWNERS @@ -0,0 +1 @@ +* @streamnative/compute From 6293907968feafe4722d46d25bbbf8c992ae1c86 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 4 Aug 2022 22:11:55 +0800 Subject: [PATCH 239/258] [FLINK-28820][Connector/Pulsar] Drop MailboxExecutor, improve the writing performance for PulsarSink. --- .../pulsar/sink/writer/PulsarWriter.java | 76 +++++++------------ 1 file changed, 26 insertions(+), 50 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java index c0ebc926ee8fa..fc5c73baa413b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/PulsarWriter.java @@ -19,7 +19,6 @@ package org.apache.flink.connector.pulsar.sink.writer; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.api.common.serialization.SerializationSchema.InitializationContext; import org.apache.flink.api.connector.sink2.Sink.InitContext; @@ -52,7 +51,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicLong; import static java.util.Collections.emptyList; import static org.apache.flink.util.IOUtils.closeAll; @@ -68,17 +67,14 @@ public class PulsarWriter implements PrecommittingSinkWriter { private static final Logger LOG = LoggerFactory.getLogger(PulsarWriter.class); - private final SinkConfiguration sinkConfiguration; private final PulsarSerializationSchema serializationSchema; private final TopicRegister topicRegister; private final TopicRouter topicRouter; private final MessageDelayer messageDelayer; private final DeliveryGuarantee deliveryGuarantee; private final PulsarSinkContext sinkContext; - private final MailboxExecutor mailboxExecutor; private final ProducerRegister producerRegister; - - private long pendingMessages = 0; + private final AtomicLong pendingMessages = new AtomicLong(0); /** * Constructor creating a Pulsar writer. @@ -101,7 +97,7 @@ public PulsarWriter( MessageDelayer messageDelayer, @Nullable CryptoKeyReader cryptoKeyReader, InitContext initContext) { - this.sinkConfiguration = checkNotNull(sinkConfiguration); + checkNotNull(sinkConfiguration); this.serializationSchema = checkNotNull(serializationSchema); this.topicRegister = checkNotNull(topicRegister); this.topicRouter = checkNotNull(topicRouter); @@ -110,7 +106,6 @@ public PulsarWriter( this.deliveryGuarantee = sinkConfiguration.getDeliveryGuarantee(); this.sinkContext = new PulsarSinkContextImpl(initContext, sinkConfiguration); - this.mailboxExecutor = initContext.getMailboxExecutor(); // Initialize topic metadata listener. LOG.debug("Initialize topic metadata after creating Pulsar writer."); @@ -164,42 +159,23 @@ public void write(IN element, Context context) throws IOException, InterruptedEx // We would just ignore the sending exception. This may cause data loss. builder.sendAsync(); } else { - // Waiting for permits to write message. - requirePermits(); - mailboxExecutor.execute( - () -> enqueueMessageSending(topic, builder), - "Failed to send message to Pulsar"); - } - } - - private void enqueueMessageSending(String topic, TypedMessageBuilder builder) - throws ExecutionException, InterruptedException { - // Block the mailbox executor for yield method. - builder.sendAsync() - .whenComplete( - (id, ex) -> { - this.releasePermits(); - if (ex != null) { - throw new FlinkRuntimeException( - "Failed to send data to Pulsar " + topic, ex); - } else { - LOG.debug( - "Sent message to Pulsar {} with message id {}", topic, id); - } - }) - .get(); - } - - private void requirePermits() throws InterruptedException { - while (pendingMessages >= sinkConfiguration.getMaxPendingMessages()) { - LOG.info("Waiting for the available permits."); - mailboxExecutor.yield(); + // Increase the pending message count. + pendingMessages.incrementAndGet(); + builder.sendAsync() + .whenComplete( + (id, ex) -> { + pendingMessages.decrementAndGet(); + if (ex != null) { + throw new FlinkRuntimeException( + "Failed to send data to Pulsar " + topic, ex); + } else { + LOG.debug( + "Sent message to Pulsar {} with message id {}", + topic, + id); + } + }); } - pendingMessages++; - } - - private void releasePermits() { - this.pendingMessages -= 1; } @SuppressWarnings({"rawtypes", "unchecked"}) @@ -262,15 +238,15 @@ private TypedMessageBuilder createMessageBuilder( } @Override - public void flush(boolean endOfInput) throws IOException, InterruptedException { - if (endOfInput) { - // Try flush only once when we meet the end of the input. + public void flush(boolean endOfInput) throws IOException { + if (endOfInput || deliveryGuarantee != DeliveryGuarantee.NONE) { + LOG.info("Flush the pending messages to Pulsar."); + + // Try to flush pending messages. producerRegister.flush(); - } else { - while (pendingMessages != 0 && deliveryGuarantee != DeliveryGuarantee.NONE) { + // Make sure all the pending messages should be flushed to Pulsar. + while (pendingMessages.longValue() > 0) { producerRegister.flush(); - LOG.info("Flush the pending messages to Pulsar."); - mailboxExecutor.yield(); } } } From 06375237a3e49311b8e067c80707f694b6f5ad98 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Mon, 8 Aug 2022 20:12:46 +0800 Subject: [PATCH 240/258] [FLINK-28820][Connector/Pulsar] Deprecated unused message writing queue length option. --- .../pulsar_catalog_factory_configuration.html | 2 +- .../generated/pulsar_sink_configuration.html | 6 ------ .../generated/pulsar_table_configuration.html | 8 +++++++- .../connector/pulsar/sink/PulsarSinkOptions.java | 2 ++ .../pulsar/sink/config/PulsarSinkConfigUtils.java | 4 ---- .../pulsar/sink/config/SinkConfiguration.java | 13 ------------- 6 files changed, 10 insertions(+), 25 deletions(-) diff --git a/docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html b/docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html index ee205368fdf6d..3954b7735ea13 100644 --- a/docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_catalog_factory_configuration.html @@ -40,7 +40,7 @@

    default-database
    - "default" + "default_database" String The default database when using PulsarCatalog. It will be created if it does not exist. diff --git a/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html b/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html index 4a3009ae70263..cd7f803534a1e 100644 --- a/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_sink_configuration.html @@ -20,12 +20,6 @@ Boolean If you enable this option and use PulsarSerializationSchema.pulsarSchema(), we would consume and deserialize the message by using Pulsar's Schema. - -
    pulsar.sink.maxPendingMessages
    - 1000 - Integer - The maximum number of pending messages in on sink parallelism. -
    pulsar.sink.maxRecommitTimes
    5 diff --git a/docs/layouts/shortcodes/generated/pulsar_table_configuration.html b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html index c84bc6bb1382c..738bd7384ed45 100644 --- a/docs/layouts/shortcodes/generated/pulsar_table_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html @@ -70,7 +70,7 @@
    source.subscription-name
    - "flink-sql-connector-pulsar" + (none) String The subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer. @@ -86,5 +86,11 @@ List<String> Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (;) like topic-1;topic-2. + +
    value.format
    + (none) + String + Defines the format identifier for decoding/encoding value data. The identifier is used to discover a suitable format factory. + diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java index c31097bc441c9..90b0c325c00fc 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java @@ -129,6 +129,8 @@ private PulsarSinkOptions() { "The allowed transaction recommit times if we meet some retryable exception." + " This is used in Pulsar Transaction."); + /** @deprecated This config option was removed sink Flink 1.16 */ + @Deprecated public static final ConfigOption PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM = ConfigOptions.key(SINK_CONFIG_PREFIX + "maxPendingMessages") .intType() diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java index e4c48162589f3..f7f4ae89f9555 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/PulsarSinkConfigUtils.java @@ -86,10 +86,6 @@ public static ProducerBuilder createProducerBuilder( PULSAR_SEND_TIMEOUT_MS, Math::toIntExact, ms -> builder.sendTimeout(ms, MILLISECONDS)); - configuration.useOption(PULSAR_MAX_PENDING_MESSAGES, builder::maxPendingMessages); - configuration.useOption( - PULSAR_MAX_PENDING_MESSAGES_ACROSS_PARTITIONS, - builder::maxPendingMessagesAcrossPartitions); configuration.useOption( PULSAR_BATCHING_MAX_PUBLISH_DELAY_MICROS, s -> builder.batchingMaxPublishDelay(s, MICROSECONDS)); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java index 768b730f40257..d6a6ee353519c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java @@ -32,7 +32,6 @@ import java.util.Objects; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES; -import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_RECOMMIT_TIMES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MESSAGE_KEY_HASH; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL; @@ -51,7 +50,6 @@ public class SinkConfiguration extends PulsarConfiguration { private final int partitionSwitchSize; private final MessageKeyHash messageKeyHash; private final boolean enableSchemaEvolution; - private final int maxPendingMessages; private final int maxRecommitTimes; public SinkConfiguration(Configuration configuration) { @@ -63,7 +61,6 @@ public SinkConfiguration(Configuration configuration) { this.partitionSwitchSize = getInteger(PULSAR_BATCHING_MAX_MESSAGES); this.messageKeyHash = get(PULSAR_MESSAGE_KEY_HASH); this.enableSchemaEvolution = get(PULSAR_WRITE_SCHEMA_EVOLUTION); - this.maxPendingMessages = get(PULSAR_MAX_PENDING_MESSAGES_ON_PARALLELISM); this.maxRecommitTimes = get(PULSAR_MAX_RECOMMIT_TIMES); } @@ -111,14 +108,6 @@ public boolean isEnableSchemaEvolution() { return enableSchemaEvolution; } - /** - * Pulsar message is sent asynchronously. Set this option for limiting the pending messages in a - * Pulsar writer instance. - */ - public int getMaxPendingMessages() { - return maxPendingMessages; - } - /** The maximum allowed recommitting time for a Pulsar transaction. */ public int getMaxRecommitTimes() { return maxRecommitTimes; @@ -141,7 +130,6 @@ public boolean equals(Object o) { && partitionSwitchSize == that.partitionSwitchSize && enableSchemaEvolution == that.enableSchemaEvolution && messageKeyHash == that.messageKeyHash - && maxPendingMessages == that.maxPendingMessages && maxRecommitTimes == that.maxRecommitTimes; } @@ -154,7 +142,6 @@ public int hashCode() { partitionSwitchSize, messageKeyHash, enableSchemaEvolution, - maxPendingMessages, maxRecommitTimes); } } From 603b1bf3ef687fa75dc4f2945f8dcf2ef707dff7 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Mon, 8 Aug 2022 20:35:02 +0800 Subject: [PATCH 241/258] [FLINK-28870][Connector/Pulsar] Add fetch time option for Pulsar Source. Avoid hanging on small message income rates. --- .../generated/pulsar_source_configuration.html | 6 ++++++ .../pulsar/source/PulsarSourceOptions.java | 8 ++++++++ .../pulsar/source/config/SourceConfiguration.java | 15 ++++++++++++++- .../split/PulsarPartitionSplitReaderBase.java | 9 +++------ 4 files changed, 31 insertions(+), 7 deletions(-) diff --git a/docs/layouts/shortcodes/generated/pulsar_source_configuration.html b/docs/layouts/shortcodes/generated/pulsar_source_configuration.html index c79495029b45d..5f25117a45a9a 100644 --- a/docs/layouts/shortcodes/generated/pulsar_source_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_source_configuration.html @@ -14,6 +14,12 @@ Long This option is used only when the user disables the checkpoint and uses Exclusive or Failover subscription. We would automatically commit the cursor using the given period (in ms). + +
    pulsar.source.defaultFetchTime
    + 100 + Long + The time (in ms) for fetching messages from Pulsar. If time exceed and no message returned from Pulsar. We would consider there is no record at the current topic and stop fetch until next switch. +
    pulsar.source.enableAutoAcknowledgeMessage
    false diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java index 1c6808bf01db1..74e39ac005c54 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java @@ -148,6 +148,14 @@ private PulsarSourceOptions() { public static final ConfigOption PULSAR_TRANSACTION_TIMEOUT_MILLIS = PULSAR_READ_TRANSACTION_TIMEOUT; + public static final ConfigOption PULSAR_DEFAULT_FETCH_TIME = + ConfigOptions.key(SOURCE_CONFIG_PREFIX + "defaultFetchTime") + .longType() + .defaultValue(100L) + .withDescription( + "The time (in ms) for fetching messages from Pulsar. If time exceed and no message returned from Pulsar." + + " We would consider there is no record at the current topic and stop fetch until next switch."); + public static final ConfigOption PULSAR_MAX_FETCH_TIME = ConfigOptions.key(SOURCE_CONFIG_PREFIX + "maxFetchTime") .longType() diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java index 845cfdceb2baf..b3291531b2852 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/SourceConfiguration.java @@ -37,6 +37,7 @@ import static org.apache.flink.connector.base.source.reader.SourceReaderOptions.ELEMENT_QUEUE_CAPACITY; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_AUTO_COMMIT_CURSOR_INTERVAL; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_DEFAULT_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; @@ -59,6 +60,7 @@ public class SourceConfiguration extends PulsarConfiguration { private final boolean enableSchemaEvolution; private final long autoCommitCursorInterval; private final long transactionTimeoutMillis; + private final Duration defaultFetchTime; private final Duration maxFetchTime; private final int maxFetchRecords; private final CursorVerification verifyInitialOffsets; @@ -75,6 +77,7 @@ public SourceConfiguration(Configuration configuration) { this.enableSchemaEvolution = get(PULSAR_READ_SCHEMA_EVOLUTION); this.autoCommitCursorInterval = get(PULSAR_AUTO_COMMIT_CURSOR_INTERVAL); this.transactionTimeoutMillis = get(PULSAR_READ_TRANSACTION_TIMEOUT); + this.defaultFetchTime = get(PULSAR_DEFAULT_FETCH_TIME, Duration::ofMillis); this.maxFetchTime = get(PULSAR_MAX_FETCH_TIME, Duration::ofMillis); this.maxFetchRecords = get(PULSAR_MAX_FETCH_RECORDS); this.verifyInitialOffsets = get(PULSAR_VERIFY_INITIAL_OFFSETS); @@ -144,6 +147,14 @@ public long getTransactionTimeoutMillis() { return transactionTimeoutMillis; } + /** + * The fetch time for polling one message. We would stop polling message and return the message + * in {@link RecordsWithSplitIds} when timeout and no message consumed. + */ + public Duration getDefaultFetchTime() { + return defaultFetchTime; + } + /** * The fetch time for flink split reader polling message. We would stop polling message and * return the message in {@link RecordsWithSplitIds} when timeout or exceed the {@link @@ -212,8 +223,9 @@ public boolean equals(Object o) { && enableAutoAcknowledgeMessage == that.enableAutoAcknowledgeMessage && autoCommitCursorInterval == that.autoCommitCursorInterval && transactionTimeoutMillis == that.transactionTimeoutMillis - && maxFetchRecords == that.maxFetchRecords + && Objects.equals(defaultFetchTime, that.defaultFetchTime) && Objects.equals(maxFetchTime, that.maxFetchTime) + && maxFetchRecords == that.maxFetchRecords && verifyInitialOffsets == that.verifyInitialOffsets && Objects.equals(subscriptionName, that.subscriptionName) && subscriptionType == that.subscriptionType @@ -228,6 +240,7 @@ public int hashCode() { enableAutoAcknowledgeMessage, autoCommitCursorInterval, transactionTimeoutMillis, + defaultFetchTime, maxFetchTime, maxFetchRecords, verifyInitialOffsets, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java index 52033c2c7df58..abba214763f41 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java @@ -49,7 +49,6 @@ import java.time.Duration; import java.util.List; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; @@ -101,15 +100,14 @@ public RecordsWithSplitIds> fetch() throws IOException { String splitId = registeredSplit.splitId(); Deadline deadline = Deadline.fromNow(sourceConfiguration.getMaxFetchTime()); - // Consume message from pulsar until it was woken up by flink reader. + // Consume messages from pulsar until it was waked up by flink reader. for (int messageNum = 0; messageNum < sourceConfiguration.getMaxFetchRecords() && deadline.hasTimeLeft() && isNotWakeup(); messageNum++) { try { - Duration timeout = deadline.timeLeftIfAny(); - Message message = pollMessage(timeout); + Message message = pollMessage(sourceConfiguration.getDefaultFetchTime()); if (message == null) { break; } @@ -126,8 +124,6 @@ public RecordsWithSplitIds> fetch() throws IOException { } catch (InterruptedException e) { Thread.currentThread().interrupt(); break; - } catch (TimeoutException e) { - break; } catch (ExecutionException e) { LOG.error("Error in polling message from pulsar consumer.", e); break; @@ -205,6 +201,7 @@ protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { return createPulsarConsumer(split.getPartition()); } + /** Create a specified {@link Consumer} by the given topic partition. */ protected Consumer createPulsarConsumer(TopicPartition partition) { ConsumerBuilder consumerBuilder = createConsumerBuilder(pulsarClient, schema, sourceConfiguration); From 24c70588d90c97faab32d4a96fa2d60741f861a0 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Mon, 1 Aug 2022 19:03:38 +0800 Subject: [PATCH 242/258] [FLINK-28505][Connector/Pulsar] Support topic auto creation for Pulsar sink. --- .../pulsar/sink/PulsarSinkBuilder.java | 22 ++++ .../pulsar/sink/PulsarSinkOptions.java | 14 +++ .../pulsar/sink/config/SinkConfiguration.java | 30 ++++- .../sink/writer/topic/TopicExtractor.java | 11 +- .../metadata/CachedTopicMetadataProvider.java | 76 +++++++++++++ .../NotExistedTopicMetadataProvider.java | 74 +++++++++++++ .../topic/register/DynamicTopicRegister.java | 103 ++++++++---------- .../topic/register/FixedTopicRegister.java | 24 ++-- .../enumerator/topic/TopicPartition.java | 5 +- .../CachedTopicMetadataProviderTest.java | 92 ++++++++++++++++ .../NotExistedTopicMetadataProviderTest.java | 69 ++++++++++++ 11 files changed, 443 insertions(+), 77 deletions(-) create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProvider.java create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProvider.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProviderTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProviderTest.java diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java index e623f29406725..2d886ef7947ad 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkBuilder.java @@ -55,6 +55,8 @@ import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_ENCRYPTION_KEYS; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_PRODUCER_NAME; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SEND_TIMEOUT_MS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_TOPIC_AUTO_CREATION; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_DELIVERY_GUARANTEE; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_SCHEMA_EVOLUTION; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_TRANSACTION_TIMEOUT; @@ -301,6 +303,26 @@ public PulsarSinkBuilder setEncryptionKeys(String... keys) { return this; } + /** + * Pulsar sink disable the topic creation if the sink topic doesn't exist. You should explicitly + * set the default partition size for enabling topic creation. Make sure you have the authority + * on the given Pulsar admin token. + * + * @param partitionSize The partition size used on topic creation. It should be above to zero. + *
      + *
    • 0: we would create a non-partitioned topic. + *
    • above 0: we would create a partitioned topic with the given size. + *
    + * + * @return this PulsarSinkBuilder. + */ + public PulsarSinkBuilder enableTopicAutoCreation(int partitionSize) { + checkArgument(partitionSize >= 0); + configBuilder.set(PULSAR_SINK_TOPIC_AUTO_CREATION, true); + configBuilder.set(PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS, partitionSize); + return this; + } + /** * Set an arbitrary property for the PulsarSink and Pulsar Producer. The valid keys can be found * in {@link PulsarSinkOptions} and {@link PulsarOptions}. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java index 90b0c325c00fc..36aab9c8d051b 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/PulsarSinkOptions.java @@ -141,6 +141,20 @@ private PulsarSinkOptions() { "The maximum number of pending messages in on sink parallelism.") .build()); + public static final ConfigOption PULSAR_SINK_TOPIC_AUTO_CREATION = + ConfigOptions.key(SINK_CONFIG_PREFIX + "topicAutoCreation") + .booleanType() + .defaultValue(false) + .withDescription( + "Enable the topic auto creation if the topic doesn't exist in Pulsar."); + + public static final ConfigOption PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS = + ConfigOptions.key(SINK_CONFIG_PREFIX + "defaultTopicPartitions") + .intType() + .defaultValue(4) + .withDescription( + "If you enable the topic auto creation, you should also configure the default partition number here"); + /////////////////////////////////////////////////////////////////////////////// // // The configuration for ProducerConfigurationData part. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java index d6a6ee353519c..66c88e8a77b9d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/config/SinkConfiguration.java @@ -34,6 +34,8 @@ import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_BATCHING_MAX_MESSAGES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MAX_RECOMMIT_TIMES; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_MESSAGE_KEY_HASH; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_TOPIC_AUTO_CREATION; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_DELIVERY_GUARANTEE; import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_WRITE_SCHEMA_EVOLUTION; @@ -51,6 +53,8 @@ public class SinkConfiguration extends PulsarConfiguration { private final MessageKeyHash messageKeyHash; private final boolean enableSchemaEvolution; private final int maxRecommitTimes; + private final boolean enableTopicAutoCreation; + private final int defaultTopicPartitions; public SinkConfiguration(Configuration configuration) { super(configuration); @@ -62,6 +66,8 @@ public SinkConfiguration(Configuration configuration) { this.messageKeyHash = get(PULSAR_MESSAGE_KEY_HASH); this.enableSchemaEvolution = get(PULSAR_WRITE_SCHEMA_EVOLUTION); this.maxRecommitTimes = get(PULSAR_MAX_RECOMMIT_TIMES); + this.enableTopicAutoCreation = get(PULSAR_SINK_TOPIC_AUTO_CREATION); + this.defaultTopicPartitions = get(PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS); } /** The delivery guarantee changes the behavior of {@link PulsarWriter}. */ @@ -113,6 +119,16 @@ public int getMaxRecommitTimes() { return maxRecommitTimes; } + /** Could the connector auto create the non-existed topics on the Pulsar? */ + public boolean isEnableTopicAutoCreation() { + return enableTopicAutoCreation; + } + + /** The default partition size when we enable the topic auto creation. */ + public int getDefaultTopicPartitions() { + return defaultTopicPartitions; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -125,23 +141,29 @@ public boolean equals(Object o) { return false; } SinkConfiguration that = (SinkConfiguration) o; - return transactionTimeoutMillis == that.transactionTimeoutMillis + return deliveryGuarantee == that.deliveryGuarantee + && transactionTimeoutMillis == that.transactionTimeoutMillis && topicMetadataRefreshInterval == that.topicMetadataRefreshInterval && partitionSwitchSize == that.partitionSwitchSize - && enableSchemaEvolution == that.enableSchemaEvolution && messageKeyHash == that.messageKeyHash - && maxRecommitTimes == that.maxRecommitTimes; + && enableSchemaEvolution == that.enableSchemaEvolution + && maxRecommitTimes == that.maxRecommitTimes + && enableTopicAutoCreation == that.enableTopicAutoCreation + && defaultTopicPartitions == that.defaultTopicPartitions; } @Override public int hashCode() { return Objects.hash( super.hashCode(), + deliveryGuarantee, transactionTimeoutMillis, topicMetadataRefreshInterval, partitionSwitchSize, messageKeyHash, enableSchemaEvolution, - maxRecommitTimes); + maxRecommitTimes, + enableTopicAutoCreation, + defaultTopicPartitions); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java index 32f8bf913cbe6..853698220f46d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/TopicExtractor.java @@ -28,7 +28,7 @@ import java.io.Serializable; -/** Choose topics from the message, used for dynamic generate topics in Pulsar sink. */ +/** Choose topics from the message, used for dynamically generate topics in Pulsar sink. */ @PublicEvolving public interface TopicExtractor extends Serializable { @@ -48,12 +48,17 @@ default void open(SinkConfiguration sinkConfiguration) { /** * A wrapper for {@link PulsarAdmin} instance, we won't expose the Pulsar admin interface for - * better control the abstraction. And add cache support. + * better control the abstraction and cache support. */ @PublicEvolving interface TopicMetadataProvider { - /** @throws Exception Failed to query Pulsar metadata would throw this exception. */ + /** + * The query result from Pulsar would be cached, if you set positive value in {@link + * SinkConfiguration#getTopicMetadataRefreshInterval()}. + * + * @throws Exception Failed to query Pulsar metadata would throw this exception. + */ TopicMetadata query(String topic) throws Exception; } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProvider.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProvider.java new file mode 100644 index 0000000000000..5d3a3729f2d54 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProvider.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.metadata; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor.TopicMetadataProvider; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; + +import org.apache.flink.shaded.guava30.com.google.common.cache.Cache; +import org.apache.flink.shaded.guava30.com.google.common.cache.CacheBuilder; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; + +import java.util.concurrent.TimeUnit; + +/** + * The default implementation for querying topic metadata. The query result would be cached by + * positive {@link SinkConfiguration#getTopicMetadataRefreshInterval()}. + */ +@Internal +public class CachedTopicMetadataProvider implements TopicMetadataProvider { + + private final PulsarAdmin pulsarAdmin; + private final Cache metadataCache; + + public CachedTopicMetadataProvider( + PulsarAdmin pulsarAdmin, SinkConfiguration sinkConfiguration) { + this.pulsarAdmin = pulsarAdmin; + + long refreshInterval = sinkConfiguration.getTopicMetadataRefreshInterval(); + if (refreshInterval <= 0) { + // Disable cache expires, the query result will never be kept in the cache. + this.metadataCache = null; + } else { + this.metadataCache = + CacheBuilder.newBuilder() + .expireAfterWrite(refreshInterval, TimeUnit.MILLISECONDS) + .maximumSize(1000) + .build(); + } + } + + @Override + public TopicMetadata query(String topic) throws PulsarAdminException { + TopicMetadata metadata = metadataCache == null ? null : metadataCache.getIfPresent(topic); + + if (metadata == null) { + PartitionedTopicMetadata meta = pulsarAdmin.topics().getPartitionedTopicMetadata(topic); + metadata = new TopicMetadata(topic, meta.partitions); + if (metadataCache != null) { + metadataCache.put(topic, metadata); + } + } + + return metadata; + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProvider.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProvider.java new file mode 100644 index 0000000000000..15926916ed365 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProvider.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.metadata; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor.TopicMetadataProvider; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; + +import org.apache.pulsar.client.admin.PulsarAdmin; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; +import org.apache.pulsar.common.partition.PartitionedTopicMetadata; + +import static org.apache.flink.util.ExceptionUtils.findThrowable; +import static org.apache.pulsar.common.partition.PartitionedTopicMetadata.NON_PARTITIONED; + +/** + * Shared topic metadata query for {@code TopicRegister}. We would auto create the topics if you + * enable this feature. + */ +@Internal +public class NotExistedTopicMetadataProvider implements TopicMetadataProvider { + + private final PulsarAdmin pulsarAdmin; + private final boolean enableTopicAutoCreation; + private final int defaultTopicPartitions; + + public NotExistedTopicMetadataProvider( + PulsarAdmin pulsarAdmin, SinkConfiguration sinkConfiguration) { + this.pulsarAdmin = pulsarAdmin; + this.enableTopicAutoCreation = sinkConfiguration.isEnableTopicAutoCreation(); + this.defaultTopicPartitions = sinkConfiguration.getDefaultTopicPartitions(); + } + + @Override + public TopicMetadata query(String topic) throws PulsarAdminException { + try { + PartitionedTopicMetadata meta = pulsarAdmin.topics().getPartitionedTopicMetadata(topic); + return new TopicMetadata(topic, meta.partitions); + } catch (PulsarAdminException e) { + if (findThrowable(e, NotFoundException.class).isPresent() && enableTopicAutoCreation) { + createTopic(topic); + return new TopicMetadata(topic, defaultTopicPartitions); + } else { + throw e; + } + } + } + + private void createTopic(String topic) throws PulsarAdminException { + if (defaultTopicPartitions == NON_PARTITIONED) { + pulsarAdmin.topics().createNonPartitionedTopic(topic); + } else { + pulsarAdmin.topics().createPartitionedTopic(topic, defaultTopicPartitions); + } + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java index 616ecda6c2b68..7dea2a159d4d3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/DynamicTopicRegister.java @@ -22,23 +22,22 @@ import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor; -import org.apache.flink.connector.pulsar.sink.writer.topic.TopicExtractor.TopicMetadataProvider; import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.metadata.CachedTopicMetadataProvider; +import org.apache.flink.connector.pulsar.sink.writer.topic.metadata.NotExistedTopicMetadataProvider; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.shaded.guava30.com.google.common.cache.Cache; import org.apache.flink.shaded.guava30.com.google.common.cache.CacheBuilder; -import org.apache.flink.shaded.guava30.com.google.common.cache.CacheLoader; -import org.apache.flink.shaded.guava30.com.google.common.cache.LoadingCache; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; +import org.apache.pulsar.client.admin.PulsarAdminException; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import static java.util.Collections.singletonList; @@ -55,8 +54,9 @@ public class DynamicTopicRegister implements TopicRegister { // Dynamic fields. private transient PulsarAdmin pulsarAdmin; - private transient TopicMetadataProvider metadataProvider; - private transient LoadingCache> partitionsCache; + private transient CachedTopicMetadataProvider cachedMetadataProvider; + private transient NotExistedTopicMetadataProvider notExistedMetadataProvider; + private transient Cache> partitionsCache; public DynamicTopicRegister(TopicExtractor topicExtractor) { this.topicExtractor = checkNotNull(topicExtractor); @@ -64,48 +64,60 @@ public DynamicTopicRegister(TopicExtractor topicExtractor) { @Override public List topics(IN in) { - TopicPartition partition = topicExtractor.extract(in, metadataProvider); + TopicPartition partition = topicExtractor.extract(in, cachedMetadataProvider); String topicName = partition.getFullTopicName(); if (partition.isPartition()) { return singletonList(topicName); } else { try { - return partitionsCache.get(topicName); - } catch (ExecutionException e) { - throw new FlinkRuntimeException("Failed to query Pulsar topic partitions.", e); + List topics = partitionsCache.getIfPresent(topicName); + if (topics == null) { + topics = queryTopics(topicName); + partitionsCache.put(topicName, topics); + } + + return topics; + } catch (PulsarAdminException e) { + throw new FlinkRuntimeException( + "Failed to query Pulsar topic partitions.", e.getCause()); } } } + private List queryTopics(String topic) throws PulsarAdminException { + TopicMetadata metadata = notExistedMetadataProvider.query(topic); + if (metadata.isPartitioned()) { + int partitionSize = metadata.getPartitionSize(); + List partitions = new ArrayList<>(partitionSize); + for (int i = 0; i < partitionSize; i++) { + partitions.add(topicNameWithPartition(topic, i)); + } + return partitions; + } else { + return singletonList(topic); + } + } + @Override public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService timeService) { - long refreshInterval = sinkConfiguration.getTopicMetadataRefreshInterval(); - // Initialize Pulsar admin instance. this.pulsarAdmin = createAdmin(sinkConfiguration); - this.metadataProvider = new DefaultTopicMetadataProvider(pulsarAdmin, refreshInterval); + this.cachedMetadataProvider = + new CachedTopicMetadataProvider(pulsarAdmin, sinkConfiguration); + this.notExistedMetadataProvider = + new NotExistedTopicMetadataProvider(pulsarAdmin, sinkConfiguration); + + long refreshInterval = sinkConfiguration.getTopicMetadataRefreshInterval(); + if (refreshInterval <= 0) { + refreshInterval = Long.MAX_VALUE; + } this.partitionsCache = CacheBuilder.newBuilder() .expireAfterWrite(refreshInterval, TimeUnit.MILLISECONDS) - .build( - new CacheLoader>() { - @Override - public List load(String topic) throws Exception { - TopicMetadata metadata = metadataProvider.query(topic); - if (metadata.isPartitioned()) { - int partitionSize = metadata.getPartitionSize(); - List partitions = - new ArrayList<>(partitionSize); - for (int i = 0; i < partitionSize; i++) { - partitions.add(topicNameWithPartition(topic, i)); - } - return partitions; - } else { - return singletonList(topic); - } - } - }); + .maximumSize(1000) + .build(); + // Open the topic extractor instance. topicExtractor.open(sinkConfiguration); } @@ -116,31 +128,4 @@ public void close() throws IOException { pulsarAdmin.close(); } } - - private static class DefaultTopicMetadataProvider implements TopicMetadataProvider { - - private final LoadingCache metadataCache; - - private DefaultTopicMetadataProvider(PulsarAdmin pulsarAdmin, long refreshInterval) { - this.metadataCache = - CacheBuilder.newBuilder() - .expireAfterWrite(refreshInterval, TimeUnit.MILLISECONDS) - .build( - new CacheLoader() { - @Override - public TopicMetadata load(String topic) throws Exception { - PartitionedTopicMetadata metadata = - pulsarAdmin - .topics() - .getPartitionedTopicMetadata(topic); - return new TopicMetadata(topic, metadata.partitions); - } - }); - } - - @Override - public TopicMetadata query(String topic) throws ExecutionException { - return metadataCache.get(topic); - } - } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegister.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegister.java index ec0d45a4919de..fb5d3658060aa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegister.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/sink/writer/topic/register/FixedTopicRegister.java @@ -22,13 +22,14 @@ import org.apache.flink.api.common.operators.ProcessingTimeService; import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; import org.apache.flink.connector.pulsar.sink.writer.topic.TopicRegister; +import org.apache.flink.connector.pulsar.sink.writer.topic.metadata.NotExistedTopicMetadataProvider; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; import org.apache.flink.shaded.guava30.com.google.common.base.Objects; import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList; import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.admin.PulsarAdminException; -import org.apache.pulsar.common.partition.PartitionedTopicMetadata; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -46,9 +47,9 @@ import static org.apache.pulsar.common.partition.PartitionedTopicMetadata.NON_PARTITIONED; /** - * We need the latest topic metadata for making sure the newly created topic partitions would be - * used by the Pulsar sink. This routing policy would be different compared with Pulsar Client - * built-in logic. We use Flink's ProcessingTimer as the executor. + * We need the latest topic metadata for making sure the Pulsar sink would use the newly created + * topic partitions. This routing policy would be different compared with Pulsar Client built-in + * logic. We use Flink's ProcessingTimer as the executor. */ @Internal public class FixedTopicRegister implements TopicRegister { @@ -64,6 +65,7 @@ public class FixedTopicRegister implements TopicRegister { private transient PulsarAdmin pulsarAdmin; private transient Long topicMetadataRefreshInterval; private transient ProcessingTimeService timeService; + private transient NotExistedTopicMetadataProvider metadataProvider; public FixedTopicRegister(List topics) { List partitions = new ArrayList<>(topics.size()); @@ -93,12 +95,15 @@ public void open(SinkConfiguration sinkConfiguration, ProcessingTimeService time this.pulsarAdmin = createAdmin(sinkConfiguration); this.topicMetadataRefreshInterval = sinkConfiguration.getTopicMetadataRefreshInterval(); this.timeService = timeService; + this.metadataProvider = new NotExistedTopicMetadataProvider(pulsarAdmin, sinkConfiguration); // Initialize the topic metadata. Quit if fail to connect to Pulsar. sneakyAdmin(this::updateTopicMetadata); - // Register time service. - triggerNextTopicMetadataUpdate(true); + // Register time service, if user enable the topic metadata update. + if (topicMetadataRefreshInterval > 0) { + triggerNextTopicMetadataUpdate(true); + } } @Override @@ -153,12 +158,11 @@ private void updateTopicMetadata() throws PulsarAdminException { for (Map.Entry entry : topicMetadata.entrySet()) { String topic = entry.getKey(); - PartitionedTopicMetadata metadata = - pulsarAdmin.topics().getPartitionedTopicMetadata(topic); + TopicMetadata metadata = metadataProvider.query(topic); // Update topic metadata if it has been changed. - if (!Objects.equal(entry.getValue(), metadata.partitions)) { - entry.setValue(metadata.partitions); + if (!Objects.equal(entry.getValue(), metadata.getPartitionSize())) { + entry.setValue(metadata.getPartitionSize()); shouldUpdate = true; } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java index 64901e24d303c..371e7dfabf707 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/topic/TopicPartition.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.pulsar.source.enumerator.topic; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; @@ -65,7 +66,7 @@ public class TopicPartition implements Serializable { */ private final TopicRange range; - /** Create a top level topic without partition information. */ + /** Create a top-level topic without partition information. */ public TopicPartition(String topic) { this(topic, NON_PARTITION_ID); } @@ -75,6 +76,7 @@ public TopicPartition(String topic, int partitionId) { this(topic, partitionId, createFullRange()); } + @Internal public TopicPartition(String topic, int partitionId, TopicRange range) { this.topic = topicName(checkNotNull(topic)); this.partitionId = partitionId; @@ -89,6 +91,7 @@ public int getPartitionId() { return partitionId; } + /** @return Is this a partition instance or a topic instance? */ public boolean isPartition() { return partitionId != NON_PARTITION_ID; } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProviderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProviderTest.java new file mode 100644 index 0000000000000..1bef64b8857af --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/CachedTopicMetadataProviderTest.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.metadata; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; + +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_TOPIC_METADATA_REFRESH_INTERVAL; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; + +/** Unit tests for {@link CachedTopicMetadataProvider}. */ +class CachedTopicMetadataProviderTest extends PulsarTestSuiteBase { + + @Test + void queryTopicsWhichIsNotExisted() { + CachedTopicMetadataProvider provider = + new CachedTopicMetadataProvider( + operator().admin(), new SinkConfiguration(new Configuration())); + + String notExistedTopic = "not-existed-topic-" + randomAlphanumeric(8); + + assertFalse(operator().topicExists(notExistedTopic)); + assertThrows(PulsarAdminException.class, () -> provider.query(notExistedTopic)); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void queryTopicsWhichIsExisted(boolean enableCache) throws PulsarAdminException { + String topicName = "existed-topic-" + randomAlphanumeric(8); + operator().createTopic(topicName, 8); + + Configuration configuration = new Configuration(); + if (!enableCache) { + configuration.set(PULSAR_TOPIC_METADATA_REFRESH_INTERVAL, -1L); + } + + CachedTopicMetadataProvider provider = + new CachedTopicMetadataProvider( + operator().admin(), new SinkConfiguration(configuration)); + + TopicMetadata metadata1 = provider.query(topicName); + assertThat(metadata1).hasFieldOrPropertyWithValue("partitionSize", 8); + + // Increase topic partition, but the query result didn't get changed immediately with cache. + operator().increaseTopicPartitions(topicName, 16); + + TopicMetadata metadata2 = provider.query(topicName); + if (enableCache) { + assertThat(metadata2).hasFieldOrPropertyWithValue("partitionSize", 8); + } else { + assertThat(metadata2).hasFieldOrPropertyWithValue("partitionSize", 16); + } + } + + @Test + void queryTopicsWhichIsExistedWithoutCache() { + String topicName = "existed-topic-" + randomAlphanumeric(8); + operator().createTopic(topicName, 8); + + Configuration configuration = new Configuration(); + configuration.set(PULSAR_TOPIC_METADATA_REFRESH_INTERVAL, -1L); + CachedTopicMetadataProvider provider = + new CachedTopicMetadataProvider( + operator().admin(), new SinkConfiguration(configuration)); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProviderTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProviderTest.java new file mode 100644 index 0000000000000..770b141a49697 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/sink/writer/topic/metadata/NotExistedTopicMetadataProviderTest.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.sink.writer.topic.metadata; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicMetadata; +import org.apache.flink.connector.pulsar.testutils.PulsarTestSuiteBase; + +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.junit.jupiter.api.Test; + +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS; +import static org.apache.flink.connector.pulsar.sink.PulsarSinkOptions.PULSAR_SINK_TOPIC_AUTO_CREATION; +import static org.assertj.core.api.Assertions.assertThat; + +/** Unit tests for {@link NotExistedTopicMetadataProvider}. */ +class NotExistedTopicMetadataProviderTest extends PulsarTestSuiteBase { + + @Test + void autoTopicCreationForNotExisted() throws PulsarAdminException { + // Create an existed topic + operator().createTopic("existed-topic", 10); + + // This provider will create a topic with 5 partitions. + NotExistedTopicMetadataProvider provider1 = + new NotExistedTopicMetadataProvider(operator().admin(), configuration(5)); + + TopicMetadata metadata1 = provider1.query("existed-topic"); + assertThat(metadata1).hasFieldOrPropertyWithValue("partitionSize", 10); + + TopicMetadata metadata2 = provider1.query("not-existed-topic-1"); + assertThat(metadata2).hasFieldOrPropertyWithValue("partitionSize", 5); + + // This provider will create a topic with 8 partitions. + NotExistedTopicMetadataProvider provider2 = + new NotExistedTopicMetadataProvider(operator().admin(), configuration(8)); + + TopicMetadata metadata3 = provider2.query("not-existed-topic-1"); + assertThat(metadata3).hasFieldOrPropertyWithValue("partitionSize", 5); + + TopicMetadata metadata4 = provider2.query("not-existed-topic-2"); + assertThat(metadata4).hasFieldOrPropertyWithValue("partitionSize", 8); + } + + private SinkConfiguration configuration(int partitions) { + Configuration configuration = new Configuration(); + configuration.set(PULSAR_SINK_TOPIC_AUTO_CREATION, true); + configuration.set(PULSAR_SINK_DEFAULT_TOPIC_PARTITIONS, partitions); + + return new SinkConfiguration(configuration); + } +} From 582eb04b5238b5ecf1d3e3f697d355169cc7f7d2 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Tue, 9 Aug 2022 07:11:59 +0800 Subject: [PATCH 243/258] [FLINK-28870][Connector/Pulsar] Increase fetch time for tests. --- .../pulsar/source/enumerator/cursor/StopCursorTest.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java index 78f849efa7385..02ef370da9a99 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java @@ -38,6 +38,7 @@ import static java.util.Collections.singletonList; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_DEFAULT_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; @@ -96,7 +97,8 @@ void publishTimeStopCursor() throws IOException { private SourceConfiguration sourceConfig() { Configuration config = operator().config(); config.set(PULSAR_MAX_FETCH_RECORDS, 1); - config.set(PULSAR_MAX_FETCH_TIME, 1000L); + config.set(PULSAR_DEFAULT_FETCH_TIME, 2000L); + config.set(PULSAR_MAX_FETCH_TIME, 3000L); config.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10)); config.set(PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE, true); return new SourceConfiguration(config); From c61e05a54ae05f9824d27dc21bdc0ad571c1a369 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Tue, 9 Aug 2022 11:07:46 +0800 Subject: [PATCH 244/258] [FLINK-28870][Connector/Pulsar] Increase fetch time for tests. --- .../source/reader/source/PulsarSourceReaderTestBase.java | 4 +++- .../reader/split/PulsarPartitionSplitReaderTestBase.java | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java index be8eaf1035a94..e0720b7375c20 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/source/PulsarSourceReaderTestBase.java @@ -61,6 +61,7 @@ import java.util.stream.Stream; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_DEFAULT_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; @@ -136,7 +137,8 @@ private PulsarSourceReaderBase sourceReader( boolean autoAcknowledgementEnabled, SubscriptionType subscriptionType) { Configuration configuration = operator().config(); configuration.set(PULSAR_MAX_FETCH_RECORDS, 1); - configuration.set(PULSAR_MAX_FETCH_TIME, 1000L); + configuration.set(PULSAR_DEFAULT_FETCH_TIME, 2000L); + configuration.set(PULSAR_MAX_FETCH_TIME, 3000L); configuration.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10)); configuration.set(PULSAR_SUBSCRIPTION_TYPE, subscriptionType); if (autoAcknowledgementEnabled diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java index 346b05bde0fac..f5c89fc2b6939 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java @@ -62,6 +62,7 @@ import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyThrow; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_DEFAULT_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; @@ -94,7 +95,8 @@ public abstract class PulsarPartitionSplitReaderTestBase extends PulsarTestSuite private SourceConfiguration sourceConfig() { Configuration config = operator().config(); config.set(PULSAR_MAX_FETCH_RECORDS, 1); - config.set(PULSAR_MAX_FETCH_TIME, 1000L); + config.set(PULSAR_DEFAULT_FETCH_TIME, 2000L); + config.set(PULSAR_MAX_FETCH_TIME, 3000L); config.set(PULSAR_SUBSCRIPTION_NAME, randomAlphabetic(10)); config.set(PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE, true); From e28eb598bf8a6bc263099ff798a32f9533199c37 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Mon, 1 Aug 2022 11:25:25 +0800 Subject: [PATCH 245/258] 163: support unbounded stop cursor in SQL Connector --- docs/content/docs/connectors/table/pulsar.md | 33 +++---- .../generated/pulsar_table_configuration.html | 18 ++++ .../cursor/stop/EventTimestampStopCursor.java | 19 ++++ .../cursor/stop/LatestMessageStopCursor.java | 19 ++++ .../cursor/stop/MessageIdStopCursor.java | 19 ++++ .../cursor/stop/NeverStopCursor.java | 11 +++ .../stop/PublishTimestampStopCursor.java | 19 ++++ .../pulsar/table/PulsarTableFactory.java | 13 +++ .../pulsar/table/PulsarTableOptionUtils.java | 62 +++++++++---- .../pulsar/table/PulsarTableOptions.java | 30 +++++++ .../table/PulsarTableValidationUtils.java | 31 +++++++ .../table/source/PulsarTableSource.java | 9 ++ .../pulsar/table/PulsarTableFactoryTest.java | 4 + .../table/PulsarTableOptionUtilsTest.java | 88 ++++++++++++++----- .../pulsar/table/PulsarTableOptionsTest.java | 50 +++++++++++ 15 files changed, 370 insertions(+), 55 deletions(-) diff --git a/docs/content/docs/connectors/table/pulsar.md b/docs/content/docs/connectors/table/pulsar.md index 74d81491082fe..42190c8c1cab0 100644 --- a/docs/content/docs/connectors/table/pulsar.md +++ b/docs/content/docs/connectors/table/pulsar.md @@ -37,21 +37,24 @@ This table outlines the Maven dependency that is added to the Pulsar SQL connect This table outlines options for the Pulsar SQL connector. -| Key | Default | Type | Description | -| --- | --- | --- | --- | -| `admin-url` | (none) | String | The Pulsar service HTTP URL for the admin endpoint. For example, `http://my-broker.example.com:8080` or `https://my-broker.example.com:8443` for TLS. | -| `explicit` | true | Boolean | Indicate if the table is an explicit Flink table. | -| `key.fields` | | List<String> | An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined. | -| `key.format` | (none) | String | The format that is used to deserialize and serialize the key bytes of Pulsar messages. The format identifier is used to discover a suitable format factory. | -| `service-url` | (none) | String | The Service URL for the Pulsar service. To connect to a Pulsar cluster using a client library, you need to specify a Pulsar protocol URL. You can assign a Pulsar protocol URL to a specific cluster.
    -This is an example URL of the `localhost:pulsar://localhost:6650`.
    - If you have multiple brokers, the URL is something like `pulsar://localhost:6550,localhost:6651,localhost:6652`.
    - A URL for a production Pulsar cluster is something like `pulsar://pulsar.us-west.example.com:6650`.
    - If TLS authentication is enabled, the URL is something like `pulsar+ssl://pulsar.us-west.example.com:6651`. | -| `sink.custom-topic-router` | (none) | String | (Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the `sink.topic-routing-mode` option will be ignored. | -| `sink.message-delay-interval` | 0 ms | Duration | (Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). | -| `sink.topic-routing-mode` | round-robin | Enum | (Optional) the topic routing mode. Available options are `round-robin` and `message-key-hash`. By default, it is set to `round-robin`. If you want to use a custom topic router, use the `sink.custom-topic-router` option to determine the partition for a particular message.
    - `round-robin`: the producer publishes messages across all partitions in a round-robin fashion to achieve the maximum throughput. The round-robin method is not implemented for individual messages. However, it is set to the same boundary of the `pulsar.producer.batchingMaxMessages` option to make batching take effect.
    - `message-key-hash`: if no key is provided, the partitioned producer will randomly pick one single topic partition and publish all messages to that partition. If a key is provided for a message, the partitioned producer will hash the key and assign the message to a particular partition. | -| `source.start.message-id` | (none) | String | (Optional) the message ID that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data. Available options are `earliest`, `latest`, and the message ID (in a format of `ledgerId:entryId:partitionId`, such as "12:2:-1"). | -| `source.start.publish-time` | (none) | Long | (Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data. | -| `source.subscription-name` | flink-sql-connector-pulsar | String | The subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer. | -| `source.subscription-type` | Exclusive | Enum | The [subscription type](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-subscriptions) that is supported by the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). Currently, only `Exclusive` and `Shared` subscription types are supported. | -| `topics` | (none) | List<String> | Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (`;`) like `topic-1;topic-2`. | +| Key | Default | Type | Description | +|--------------------------------|----------------------------|--------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| `admin-url` | (none) | String | The Pulsar service HTTP URL for the admin endpoint. For example, `http://my-broker.example.com:8080` or `https://my-broker.example.com:8443` for TLS. | +| `explicit` | true | Boolean | Indicate if the table is an explicit Flink table. | +| `key.fields` | | List<String> | An explicit list of physical columns from the table schema that are decoded/encoded from the key bytes of a Pulsar message. By default, this list is empty and thus a key is undefined. | +| `key.format` | (none) | String | The format that is used to deserialize and serialize the key bytes of Pulsar messages. The format identifier is used to discover a suitable format factory. | +| `service-url` | (none) | String | The Service URL for the Pulsar service. To connect to a Pulsar cluster using a client library, you need to specify a Pulsar protocol URL. You can assign a Pulsar protocol URL to a specific cluster.
    -This is an example URL of the `localhost:pulsar://localhost:6650`.
    - If you have multiple brokers, the URL is something like `pulsar://localhost:6550,localhost:6651,localhost:6652`.
    - A URL for a production Pulsar cluster is something like `pulsar://pulsar.us-west.example.com:6650`.
    - If TLS authentication is enabled, the URL is something like `pulsar+ssl://pulsar.us-west.example.com:6651`. | +| `sink.custom-topic-router` | (none) | String | (Optional) the custom topic router class URL that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). If this option is provided, the `sink.topic-routing-mode` option will be ignored. | +| `sink.message-delay-interval` | 0 ms | Duration | (Optional) the message delay delivery interval that is used in the [Pulsar DataStream sink connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-sink). | +| `sink.topic-routing-mode` | round-robin | Enum | (Optional) the topic routing mode. Available options are `round-robin` and `message-key-hash`. By default, it is set to `round-robin`. If you want to use a custom topic router, use the `sink.custom-topic-router` option to determine the partition for a particular message.
    - `round-robin`: the producer publishes messages across all partitions in a round-robin fashion to achieve the maximum throughput. The round-robin method is not implemented for individual messages. However, it is set to the same boundary of the `pulsar.producer.batchingMaxMessages` option to make batching take effect.
    - `message-key-hash`: if no key is provided, the partitioned producer will randomly pick one single topic partition and publish all messages to that partition. If a key is provided for a message, the partitioned producer will hash the key and assign the message to a particular partition. | +| `source.start.message-id` | (none) | String | (Optional) the message ID that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data. Available options are `earliest`, `latest`, and the message ID (in a format of `ledgerId:entryId:partitionId`, such as "12:2:-1"). | +| `source.start.publish-time` | (none) | Long | (Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data. | +| `source.subscription-name` | flink-sql-connector-pulsar | String | The subscription name of the consumer that is used by the runtime [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). This argument is required for constructing the consumer. | +| `source.subscription-type` | Exclusive | Enum | The [subscription type](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-subscriptions) that is supported by the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source). Currently, only `Exclusive` and `Shared` subscription types are supported. | +| `source.stop.at-message-id` | (none) | String | (Optional) message id used to specify a stop cursor for the unbounded sql source. Use `never"`, `latest` or pass in a message id representation in `ledgerId:entryId:partitionId`, such as `12:2:-1` | +| `source.stop.at-publish-time` | (none) | Long | (Optional) publish timestamp used to specify a stop cursor for the unbounded sql source. | +| `source.stop.after-message-id` | (none) | String | (Optional) message id used to specify a stop position but include the given message in the consuming result for the unbounded sql source. Pass in a message id representation in `ledgerId:entryId:partitionId`, such as `12:2:-1`. | +| `topics` | (none) | List<String> | Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (`;`) like `topic-1;topic-2`. | ## Features diff --git a/docs/layouts/shortcodes/generated/pulsar_table_configuration.html b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html index 738bd7384ed45..f87fa1766d416 100644 --- a/docs/layouts/shortcodes/generated/pulsar_table_configuration.html +++ b/docs/layouts/shortcodes/generated/pulsar_table_configuration.html @@ -68,6 +68,24 @@ Long (Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data. + +
    source.stop.after-message-id
    + (none) + String + Optional message id used to specify a stop position but include the given message in the consuming result for the unbounded sql source. Pass in a message id representation in "ledgerId:entryId:partitionId", such as "12:2:-1". + + +
    source.stop.at-message-id
    + (none) + String + Optional message id used to specify a stop cursor for the unbounded sql source. Use "never", "latest" or pass in a message id representation in "ledgerId:entryId:partitionId", such as "12:2:-1" + + +
    source.stop.at-publish-time
    + (none) + Long + Optional publish timestamp used to specify a stop cursor for the unbounded sql source. +
    source.subscription-name
    (none) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java index e425545de4412..c569fe1e3b317 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java @@ -22,6 +22,8 @@ import org.apache.pulsar.client.api.Message; +import java.util.Objects; + /** Stop consuming message at the given event time. */ public class EventTimestampStopCursor implements StopCursor { private static final long serialVersionUID = 2391576769339369027L; @@ -36,4 +38,21 @@ public EventTimestampStopCursor(long timestamp) { public boolean shouldStop(Message message) { return message.getEventTime() >= timestamp; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + EventTimestampStopCursor that = (EventTimestampStopCursor) o; + return timestamp == that.timestamp; + } + + @Override + public int hashCode() { + return Objects.hash(timestamp); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java index 29dd68268c0dc..bdd054a3c442d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java @@ -25,6 +25,8 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; +import java.util.Objects; + import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; /** @@ -67,4 +69,21 @@ public boolean shouldStop(Message message) { return id.compareTo(messageId) >= 0; } } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LatestMessageStopCursor that = (LatestMessageStopCursor) o; + return exclusive == that.exclusive && Objects.equals(messageId, that.messageId); + } + + @Override + public int hashCode() { + return Objects.hash(messageId, exclusive); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java index 3d7829a80dae0..7ea6c50c6dbc9 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java @@ -25,6 +25,8 @@ import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; +import java.util.Objects; + import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkState; @@ -66,4 +68,21 @@ public boolean shouldStop(Message message) { return id.compareTo(messageId) >= 0; } } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + MessageIdStopCursor that = (MessageIdStopCursor) o; + return exclusive == that.exclusive && messageId.equals(that.messageId); + } + + @Override + public int hashCode() { + return Objects.hash(messageId, exclusive); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java index ff2c619afb8b0..8d646c572c77f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java @@ -30,4 +30,15 @@ public class NeverStopCursor implements StopCursor { public boolean shouldStop(Message message) { return false; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + return true; + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java index b598e7addd422..9dc914de8cb35 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java @@ -22,6 +22,8 @@ import org.apache.pulsar.client.api.Message; +import java.util.Objects; + /** Stop consuming message at the given publish time. */ public class PublishTimestampStopCursor implements StopCursor { private static final long serialVersionUID = 4386276745339324527L; @@ -36,4 +38,21 @@ public PublishTimestampStopCursor(long timestamp) { public boolean shouldStop(Message message) { return message.getPublishTime() >= timestamp; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PublishTimestampStopCursor that = (PublishTimestampStopCursor) o; + return timestamp == that.timestamp; + } + + @Override + public int hashCode() { + return Objects.hash(timestamp); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java index 97d76bf8da165..b6ae8fc949869 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableFactory.java @@ -29,6 +29,7 @@ import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.connector.pulsar.source.PulsarSourceOptions; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; @@ -62,6 +63,7 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getMessageDelayMillis; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getPulsarProperties; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStopCursor; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getSubscriptionType; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicListFromOptions; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRouter; @@ -78,6 +80,9 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AFTER_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; @@ -128,6 +133,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { // Retrieve configs final List topics = getTopicListFromOptions(tableOptions); final StartCursor startCursor = getStartCursor(tableOptions); + final StopCursor stopCursor = getStopCursor(tableOptions); final SubscriptionType subscriptionType = getSubscriptionType(tableOptions); // Forward source configs @@ -165,6 +171,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { topics, properties, startCursor, + stopCursor, subscriptionType); } @@ -253,6 +260,9 @@ public Set> optionalOptions() { SOURCE_SUBSCRIPTION_TYPE, SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME, + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME, SINK_CUSTOM_TOPIC_ROUTER, SINK_TOPIC_ROUTING_MODE, SINK_MESSAGE_DELAY_INTERVAL, @@ -278,6 +288,9 @@ public Set> forwardOptions() { SOURCE_SUBSCRIPTION_NAME, SOURCE_START_FROM_MESSAGE_ID, SOURCE_START_FROM_PUBLISH_TIME, + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME, SINK_CUSTOM_TOPIC_ROUTER, SINK_TOPIC_ROUTING_MODE, SINK_MESSAGE_DELAY_INTERVAL) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java index b89c14edb9db3..8df1346d7f636 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtils.java @@ -25,6 +25,7 @@ import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.connector.format.DecodingFormat; import org.apache.flink.table.connector.format.EncodingFormat; @@ -38,7 +39,6 @@ import org.apache.flink.table.types.logical.utils.LogicalTypeChecks; import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; -import org.apache.flink.util.Preconditions; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; @@ -59,9 +59,13 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AFTER_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.VALUE_FORMAT; +import static org.apache.flink.util.Preconditions.checkArgument; /** * A util class for getting fields from config options, getting formats and other useful @@ -128,8 +132,7 @@ public static EncodingFormat> getValueEncodingForma public static int[] createKeyFormatProjection( ReadableConfig options, DataType physicalDataType) { final LogicalType physicalType = physicalDataType.getLogicalType(); - Preconditions.checkArgument( - physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + checkArgument(physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); final Optional optionalKeyFormat = options.getOptional(KEY_FORMAT); final Optional> optionalKeyFields = options.getOptional(KEY_FIELDS); @@ -161,8 +164,7 @@ public static int[] createKeyFormatProjection( public static int[] createValueFormatProjection( ReadableConfig options, DataType physicalDataType) { final LogicalType physicalType = physicalDataType.getLogicalType(); - Preconditions.checkArgument( - physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); + checkArgument(physicalType.is(LogicalTypeRoot.ROW), "Row data type expected."); final int physicalFieldCount = LogicalTypeChecks.getFieldCount(physicalType); final IntStream physicalFields = IntStream.range(0, physicalFieldCount); @@ -208,6 +210,18 @@ public static StartCursor getStartCursor(ReadableConfig tableOptions) { } } + public static StopCursor getStopCursor(ReadableConfig tableOptions) { + if (tableOptions.getOptional(SOURCE_STOP_AT_MESSAGE_ID).isPresent()) { + return parseAtMessageIdStopCursor(tableOptions.get(SOURCE_STOP_AT_MESSAGE_ID)); + } else if (tableOptions.getOptional(SOURCE_STOP_AFTER_MESSAGE_ID).isPresent()) { + return parseAfterMessageIdStopCursor(tableOptions.get(SOURCE_STOP_AFTER_MESSAGE_ID)); + } else if (tableOptions.getOptional(SOURCE_STOP_AT_PUBLISH_TIME).isPresent()) { + return parseAtPublishTimeStopCursor(tableOptions.get(SOURCE_STOP_AT_PUBLISH_TIME)); + } else { + return StopCursor.never(); + } + } + public static SubscriptionType getSubscriptionType(ReadableConfig tableOptions) { return tableOptions.get(SOURCE_SUBSCRIPTION_TYPE); } @@ -218,22 +232,42 @@ protected static StartCursor parseMessageIdStartCursor(String config) { } else if (Objects.equals(config, "latest")) { return StartCursor.latest(); } else { - return parseMessageIdString(config); + return StartCursor.fromMessageId(parseMessageIdString(config)); } } - protected static StartCursor parseMessageIdString(String config) { - String[] tokens = config.split(":", 3); - if (tokens.length != 3) { - throw new IllegalArgumentException( - "MessageId format must be ledgerId:entryId:partitionId."); + protected static StartCursor parsePublishTimeStartCursor(Long config) { + return StartCursor.fromPublishTime(config); + } + + protected static StopCursor parseAtMessageIdStopCursor(String config) { + if (Objects.equals(config, "never")) { + return StopCursor.never(); + } else if (Objects.equals(config, "latest")) { + return StopCursor.latest(); + } else { + return StopCursor.atMessageId(parseMessageIdString(config)); } + } + + protected static StopCursor parseAfterMessageIdStopCursor(String config) { + return StopCursor.afterMessageId(parseMessageIdString(config)); + } + + protected static StopCursor parseAtPublishTimeStopCursor(Long config) { + return StopCursor.atPublishTime(config); + } + + protected static MessageIdImpl parseMessageIdString(String config) { + String[] tokens = config.split(":", 3); + checkArgument(tokens.length == 3, "MessageId format must be ledgerId:entryId:partitionId."); + try { long ledgerId = Long.parseLong(tokens[0]); long entryId = Long.parseLong(tokens[1]); int partitionId = Integer.parseInt(tokens[2]); MessageIdImpl messageId = new MessageIdImpl(ledgerId, entryId, partitionId); - return StartCursor.fromMessageId(messageId); + return messageId; } catch (NumberFormatException e) { throw new IllegalArgumentException( "MessageId format must be ledgerId:entryId:partitionId. " @@ -241,10 +275,6 @@ protected static StartCursor parseMessageIdString(String config) { } } - protected static StartCursor parsePublishTimeStartCursor(Long config) { - return StartCursor.fromPublishTime(config); - } - // -------------------------------------------------------------------------------------------- // Table Sink Option Utils // -------------------------------------------------------------------------------------------- diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java index b805dd47153ec..2f3d9d602a3db 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableOptions.java @@ -113,6 +113,36 @@ private PulsarTableOptions() {} .text( "(Optional) the publish timestamp that is used to specify a starting point for the [Pulsar DataStream source connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#pulsar-source) to consume data.") .build()); + + public static final ConfigOption SOURCE_STOP_AT_MESSAGE_ID = + ConfigOptions.key("source.stop.at-message-id") + .stringType() + .noDefaultValue() + .withDescription( + "Optional message id used to specify a stop cursor for the unbounded sql " + + "source. Use \"never\", \"latest\" or pass in a message id " + + "representation in \"ledgerId:entryId:partitionId\", " + + "such as \"12:2:-1\""); + + public static final ConfigOption SOURCE_STOP_AFTER_MESSAGE_ID = + ConfigOptions.key("source.stop.after-message-id") + .stringType() + .noDefaultValue() + .withDescription( + "Optional message id used to specify a stop position but include the " + + "given message in the consuming result for the unbounded sql " + + "source. Pass in a message id " + + "representation in \"ledgerId:entryId:partitionId\", " + + "such as \"12:2:-1\". "); + + public static final ConfigOption SOURCE_STOP_AT_PUBLISH_TIME = + ConfigOptions.key("source.stop.at-publish-time") + .longType() + .noDefaultValue() + .withDescription( + "Optional publish timestamp used to specify a stop cursor" + + " for the unbounded sql source."); + // -------------------------------------------------------------------------------------------- // Table Sink Options // -------------------------------------------------------------------------------------------- diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java index 6ce78337b6fae..8d7e01825ca09 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/PulsarTableValidationUtils.java @@ -19,6 +19,7 @@ package org.apache.flink.connector.pulsar.table; import org.apache.flink.api.common.serialization.DeserializationSchema; +import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.table.api.ValidationException; @@ -28,10 +29,13 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.types.RowKind; +import org.apache.flink.shaded.guava30.com.google.common.collect.Sets; + import org.apache.pulsar.client.api.SubscriptionType; import java.util.List; import java.util.Optional; +import java.util.Set; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getValueDecodingFormat; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; @@ -40,6 +44,9 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AFTER_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; import static org.apache.pulsar.common.naming.TopicName.isValid; @@ -68,6 +75,7 @@ public static void validatePrimaryKeyConstraints( public static void validateTableSourceOptions(ReadableConfig tableOptions) { validateTopicsConfigs(tableOptions); validateStartCursorConfigs(tableOptions); + validateStopCursorConfigs(tableOptions); validateSubscriptionTypeConfigs(tableOptions); validateKeyFormatConfigs(tableOptions); } @@ -101,6 +109,29 @@ protected static void validateStartCursorConfigs(ReadableConfig tableOptions) { } } + protected static void validateStopCursorConfigs(ReadableConfig tableOptions) { + Set> conflictConfigOptions = + Sets.newHashSet( + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME); + + long configsNums = + conflictConfigOptions.stream() + .map(tableOptions::getOptional) + .filter(Optional::isPresent) + .count(); + + if (configsNums > 1) { + throw new ValidationException( + String.format( + "Only one of %s, %s and %s can be specified. Detected more than 1 of them", + SOURCE_STOP_AT_MESSAGE_ID, + SOURCE_STOP_AFTER_MESSAGE_ID, + SOURCE_STOP_AT_PUBLISH_TIME)); + } + } + protected static void validateSubscriptionTypeConfigs(ReadableConfig tableOptions) { SubscriptionType subscriptionType = tableOptions.get(SOURCE_SUBSCRIPTION_TYPE); if (subscriptionType == SubscriptionType.Failover diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java index b8ba6d401f9b4..a677256471f2f 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/source/PulsarTableSource.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.serialization.DeserializationSchema; import org.apache.flink.connector.pulsar.source.PulsarSource; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.reader.deserializer.PulsarDeserializationSchema; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.format.DecodingFormat; @@ -76,6 +77,8 @@ public class PulsarTableSource implements ScanTableSource, SupportsReadingMetada private final StartCursor startCursor; + private final StopCursor stopCursor; + private final SubscriptionType subscriptionType; public PulsarTableSource( @@ -84,6 +87,7 @@ public PulsarTableSource( List topics, Properties properties, StartCursor startCursor, + StopCursor stopCursor, SubscriptionType subscriptionType) { // Format attributes this.deserializationSchemaFactory = checkNotNull(deserializationSchemaFactory); @@ -92,6 +96,7 @@ public PulsarTableSource( this.topics = topics; this.properties = checkNotNull(properties); this.startCursor = checkNotNull(startCursor); + this.stopCursor = checkNotNull(stopCursor); this.subscriptionType = checkNotNull(subscriptionType); } @@ -108,6 +113,7 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) { PulsarSource.builder() .setTopics(topics) .setStartCursor(startCursor) + .setUnboundedStopCursor(stopCursor) .setDeserializationSchema(deserializationSchema) .setSubscriptionType(subscriptionType) .setProperties(properties) @@ -177,6 +183,7 @@ public DynamicTableSource copy() { topics, properties, startCursor, + stopCursor, subscriptionType); return copy; } @@ -197,6 +204,7 @@ public boolean equals(Object o) { && Objects.equals(topics, that.topics) && Objects.equals(properties, that.properties) && Objects.equals(startCursor, that.startCursor) + && Objects.equals(stopCursor, that.stopCursor) && subscriptionType == that.subscriptionType; } @@ -208,6 +216,7 @@ public int hashCode() { topics, properties, startCursor, + stopCursor, subscriptionType); } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java index 440abfa6f4e3e..0d6a59f34e798 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableFactoryTest.java @@ -26,6 +26,7 @@ import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.connector.pulsar.source.PulsarSource; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.table.sink.PulsarTableSerializationSchemaFactory; import org.apache.flink.connector.pulsar.table.sink.PulsarTableSink; import org.apache.flink.connector.pulsar.table.source.PulsarTableDeserializationSchemaFactory; @@ -168,6 +169,7 @@ public void testTableSource() { Lists.list(TEST_TOPIC), EXPECTED_PULSAR_SOURCE_PROPERTIES, StartCursor.earliest(), + StopCursor.never(), SubscriptionType.Exclusive); assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); @@ -213,6 +215,7 @@ public void testTableSourceWithKeyValue() { Lists.list(TEST_TOPIC), EXPECTED_PULSAR_SOURCE_PROPERTIES, StartCursor.earliest(), + StopCursor.never(), SubscriptionType.Exclusive); assertThat(actualPulsarSource).isEqualTo(expectedPulsarSource); @@ -266,6 +269,7 @@ public void testTableSourceWithKeyValueAndMetadata() { Lists.list(TEST_TOPIC), EXPECTED_PULSAR_SOURCE_PROPERTIES, StartCursor.earliest(), + StopCursor.never(), SubscriptionType.Exclusive); deserializationSchemaFactory.setProducedDataType( diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java index 5506c236e30a4..4e1e42241d02c 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java @@ -22,8 +22,10 @@ import org.apache.flink.connector.pulsar.sink.writer.router.TopicRouter; import org.apache.flink.connector.pulsar.sink.writer.router.TopicRoutingMode; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.PublishTimestampStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.PublishTimestampStopCursor; import org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.ValidationException; @@ -47,17 +49,22 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getMessageDelayMillis; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getPulsarProperties; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getStopCursor; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getSubscriptionType; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicListFromOptions; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRouter; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.getTopicRoutingMode; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseAfterMessageIdStopCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseAtMessageIdStopCursor; import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseMessageIdStartCursor; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptionUtils.parseMessageIdString; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.KEY_FIELDS; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_CUSTOM_TOPIC_ROUTER; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_MESSAGE_DELAY_INTERVAL; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; import static org.apache.flink.table.api.DataTypes.FIELD; @@ -66,6 +73,7 @@ import static org.apache.flink.table.api.DataTypes.STRING; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; /** * Unit test for {@link PulsarTableOptionUtils}. Tests each method and different inputs. Some tests @@ -201,7 +209,7 @@ void subscriptionType() { } @Test - void canParseMessageIdEarliestOrLatest() { + void canParseMessageIdEarliestOrLatestStartCursor() { String earliest = "earliest"; StartCursor startCursor = parseMessageIdStartCursor(earliest); assertThat(startCursor).isEqualTo(StartCursor.earliest()); @@ -215,6 +223,51 @@ void canParseMessageIdEarliestOrLatest() { assertThat(startCursor).isEqualTo(StartCursor.fromMessageId(new MessageIdImpl(0, 0, 100))); } + @Test + void publishTimeStartCursor() { + final Map options = createDefaultOptions(); + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345"); + StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); + assertThat(startCursor).isInstanceOf(PublishTimestampStartCursor.class); + + options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) + .withMessage("Could not parse value '12345L' for key 'source.start.publish-time'."); + } + + @Test + void canParseMessageIdNeverOrLatestStopCursor() { + String never = "never"; + StopCursor stopCursor = parseAtMessageIdStopCursor(never); + assertThat(stopCursor).isEqualTo(StopCursor.never()); + + String latest = "latest"; + stopCursor = parseAtMessageIdStopCursor(latest); + assertThat(stopCursor).isEqualTo(StopCursor.latest()); + + String precise = "0:0:100"; + stopCursor = parseAtMessageIdStopCursor(precise); + assertThat(stopCursor).isEqualTo(StopCursor.atMessageId(new MessageIdImpl(0, 0, 100))); + + stopCursor = parseAfterMessageIdStopCursor(precise); + assertThat(stopCursor).isEqualTo(StopCursor.afterMessageId(new MessageIdImpl(0, 0, 100))); + } + + @Test + void publishTimeStopCursor() { + final Map options = createDefaultOptions(); + options.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "12345"); + StopCursor stopCursor = getStopCursor(Configuration.fromMap(options)); + assertThat(stopCursor).isInstanceOf(PublishTimestampStopCursor.class); + + options.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "12345L"); + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> getStopCursor(Configuration.fromMap(options))) + .withMessage( + "Could not parse value '12345L' for key 'source.stop.at-publish-time'."); + } + @Test void canParseMessageIdUsingMessageIdImpl() { final String invalidFormatMessage = @@ -222,65 +275,52 @@ void canParseMessageIdUsingMessageIdImpl() { final String invalidNumberMessage = "MessageId format must be ledgerId:entryId:partitionId. Each id should be able to parsed to long type."; String precise = "0:0:100"; - StartCursor startCursor = parseMessageIdStartCursor(precise); + assertThatNoException().isThrownBy(() -> parseMessageIdString(precise)); String empty = ""; assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> parseMessageIdStartCursor(empty)) + .isThrownBy(() -> parseMessageIdString(empty)) .withMessage(invalidFormatMessage); String noSemicolon = "0"; assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> parseMessageIdStartCursor(noSemicolon)) + .isThrownBy(() -> parseMessageIdString(noSemicolon)) .withMessage(invalidFormatMessage); String oneSemiColon = "0:"; assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> parseMessageIdStartCursor(oneSemiColon)) + .isThrownBy(() -> parseMessageIdString(oneSemiColon)) .withMessage(invalidFormatMessage); String oneSemiColonComplete = "0:0"; assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> parseMessageIdStartCursor(oneSemiColonComplete)) + .isThrownBy(() -> parseMessageIdString(oneSemiColonComplete)) .withMessage(invalidFormatMessage); String twoSemiColon = "0:0:"; assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> parseMessageIdStartCursor(twoSemiColon)) + .isThrownBy(() -> parseMessageIdString(twoSemiColon)) .withMessage(invalidNumberMessage); String twoSemiColonComplete = "0:0:0"; - startCursor = parseMessageIdStartCursor(twoSemiColonComplete); + assertThatNoException().isThrownBy(() -> parseMessageIdString(twoSemiColonComplete)); String threeSemicolon = "0:0:0:"; assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> parseMessageIdStartCursor(threeSemicolon)) + .isThrownBy(() -> parseMessageIdString(threeSemicolon)) .withMessage(invalidNumberMessage); String threeSemicolonComplete = "0:0:0:0"; assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> parseMessageIdStartCursor(threeSemicolonComplete)) + .isThrownBy(() -> parseMessageIdString(threeSemicolonComplete)) .withMessage(invalidNumberMessage); String invalidNumber = "0:0:adf"; assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> parseMessageIdStartCursor(invalidNumber)) + .isThrownBy(() -> parseMessageIdString(invalidNumber)) .withMessage(invalidNumberMessage); } - @Test - void publishTimeStartCursor() { - final Map options = createDefaultOptions(); - options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345"); - StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); - assertThat(startCursor).isInstanceOf(PublishTimestampStartCursor.class); - - options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345L"); - assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> getStartCursor(Configuration.fromMap(options))) - .withMessage("Could not parse value '12345L' for key 'source.start.publish-time'."); - } - // -------------------------------------------------------------------------------------------- // Table Sink Option Utils Test // -------------------------------------------------------------------------------------------- diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java index 413acad203b7d..f48f070364a00 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionsTest.java @@ -37,6 +37,8 @@ import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SINK_TOPIC_ROUTING_MODE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_MESSAGE_ID; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_START_FROM_PUBLISH_TIME; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_MESSAGE_ID; +import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_STOP_AT_PUBLISH_TIME; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.SOURCE_SUBSCRIPTION_TYPE; import static org.apache.flink.connector.pulsar.table.PulsarTableOptions.TOPICS; import static org.apache.flink.table.factories.TestDynamicTableFactory.VALUE_FORMAT; @@ -253,6 +255,54 @@ void timestampStartCursor() { runSourceAndExpectSucceed(topicName); } + @Test + void messageIdStopCursorNever() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "never"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStopCursorLatest() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "latest"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void messageIdStopCursorExact() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_MESSAGE_ID.key(), "0:0:-1"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + + @Test + void timestampStopCursor() { + final String topicName = randomTopicName(); + Map testConfigs = testConfigWithTopicAndFormat(topicName); + + testConfigs.put(SOURCE_STOP_AT_PUBLISH_TIME.key(), "233010230"); + + runSql(topicName, createTestConfig(testConfigs)); + runSinkAndExpectSucceed(topicName); + runSourceAndExpectSucceed(topicName); + } + @Test void topicRoutingMode() { final String topicName = randomTopicName(); From 79703ebca65c766002a16b8b6f858d29a116d30b Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Fri, 19 Aug 2022 17:10:43 +0800 Subject: [PATCH 246/258] 176: add functional testing for StopCursor --- .../pulsar/table/PulsarTableITCase.java | 89 +++++++++++++++++++ 1 file changed, 89 insertions(+) diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java index 3bdde02f8fe9c..3f009d7c94ca7 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -18,7 +18,9 @@ package org.apache.flink.connector.pulsar.table; +import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.connector.pulsar.table.testutils.TestingUser; +import org.apache.flink.core.execution.JobClient; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.table.api.DataTypes; @@ -29,6 +31,7 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.Schema; import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -40,6 +43,8 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.stream.Stream; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; @@ -49,6 +54,8 @@ import static org.apache.flink.util.CollectionUtil.entry; import static org.apache.flink.util.CollectionUtil.map; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; +import static org.assertj.core.api.Assertions.assertThatNoException; /** * IT cases for the Pulsar table source and sink. It aims to verify runtime behaviour and certain @@ -441,6 +448,88 @@ void selectIntoTableUsingAvroBasedSchema( assertThat(sinkResult.getValue()).isEqualTo(value); } + @Test + void sendMessageToTopicAndReadUntilBoundedStopCursor() throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, Schema.STRING, randomAlphabetic(5)); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'pulsar.source.partitionDiscoveryIntervalMs' = '-1',\n" + + " 'source.stop.at-message-id' = 'latest',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + RAW_FORMAT); + + tableEnv.executeSql(createSourceTable); + JobClient jobClient = + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)) + .execute() + .getJobClient() + .get(); + assertThatNoException() + .isThrownBy( + () -> { + JobExecutionResult result = + jobClient.getJobExecutionResult().get(1, TimeUnit.MINUTES); + }); + } + + @Test + void sendMessageToTopicAndReadUntilBoundedStopCursorButHasPartitionDiscovery() + throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + pulsar.operator().sendMessage(sourceTopic, Schema.STRING, randomAlphabetic(5)); + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'source.stop.at-message-id' = 'latest',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + RAW_FORMAT); + + tableEnv.executeSql(createSourceTable); + JobClient jobClient = + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)) + .execute() + .getJobClient() + .get(); + assertThatExceptionOfType(TimeoutException.class) + .isThrownBy( + () -> { + JobExecutionResult result = + jobClient.getJobExecutionResult().get(1, TimeUnit.MINUTES); + }); + } + private static final class TestingSinkFunction implements SinkFunction { private static final long serialVersionUID = 455430015321124493L; From 3b8a31adeb21bedb9112a60fd9a241810d4ee932 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 16 Aug 2022 09:45:55 +0800 Subject: [PATCH 247/258] flink-183: add pulsar client config guides in the doc --- docs/content/docs/connectors/table/pulsar.md | 24 +++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/docs/content/docs/connectors/table/pulsar.md b/docs/content/docs/connectors/table/pulsar.md index 42190c8c1cab0..0f11eb896af3a 100644 --- a/docs/content/docs/connectors/table/pulsar.md +++ b/docs/content/docs/connectors/table/pulsar.md @@ -56,6 +56,8 @@ This table outlines options for the Pulsar SQL connector. | `source.stop.after-message-id` | (none) | String | (Optional) message id used to specify a stop position but include the given message in the consuming result for the unbounded sql source. Pass in a message id representation in `ledgerId:entryId:partitionId`, such as `12:2:-1`. | | `topics` | (none) | List<String> | Topic name(s) the table reads data from. It can be a single topic name or a list of topic names separated by a semicolon symbol (`;`) like `topic-1;topic-2`. | +However, besides these Pulsar SQL connector options, you can configure the underlying Pulsar DataStream Connector using the connector options defined in [Apache Pulsar Source Connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#source-configurable-options) and [Apache Pulsar Sink Connector]([subscription type]([Apache Pulsar Source Connector](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/#sink-configurable-options) using `WITH` clause. + ## Features This section describes some significant features for the Pulsar SQL connector. @@ -413,5 +415,25 @@ CREATE TABLE users 'key.json.ignore-parse-errors' = 'true', 'key.fields' = 'uid;item_id', 'value.format' = 'json', - 'value.json.fail-on-missing-field' = 'false', + 'value.json.fail-on-missing-field' = 'false' +``` + +### Configure auth parameters + +This example shows how to specify and configure the auth parameters used by the underlying Pulsar admin client. T + +```sql +CREATE TABLE users +( + `uid` BIGINT, + `item_id` BIGINT, + `description` STRING +) WITH ( + 'connector' = 'pulsar', + 'topics' = 'persistent://public/default/users', + 'service-url' = 'pulsar://localhost:6650', + 'admin-url' = 'http://localhost:8080', + 'pulsar.client.tlsAllowInsecureConnection' = 'true', + 'pulsar.client.authPluginClassName' = 'org.apache.pulsar.client.impl.auth.AuthenticationToken', + 'pulsar.client.authParams' = 'token:eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ1c2VyMSJ9.2AgtxHe8-2QBV529B5DrRtpuqP6RJjrk21Mhn' ``` From 8e24c7db611bab3e094597e6a350715956af346e Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 16 Aug 2022 09:46:30 +0800 Subject: [PATCH 248/258] flink-173: add releationship between apache/flink and streamnative/flink drop extra empty line Apply suggestions from code review Co-authored-by: Huanli Meng <48120384+Huanli-Meng@users.noreply.github.com> --- docs/content/docs/connectors/table/pulsar.md | 42 +++++++++++++++++++- 1 file changed, 40 insertions(+), 2 deletions(-) diff --git a/docs/content/docs/connectors/table/pulsar.md b/docs/content/docs/connectors/table/pulsar.md index 0f11eb896af3a..53917535370d0 100644 --- a/docs/content/docs/connectors/table/pulsar.md +++ b/docs/content/docs/connectors/table/pulsar.md @@ -418,9 +418,9 @@ CREATE TABLE users 'value.json.fail-on-missing-field' = 'false' ``` -### Configure auth parameters +### Configure authentication parameters -This example shows how to specify and configure the auth parameters used by the underlying Pulsar admin client. T +This example shows how to specify and configure the authentication parameters used by the underlying Pulsar admin client. T ```sql CREATE TABLE users @@ -437,3 +437,41 @@ CREATE TABLE users 'pulsar.client.authPluginClassName' = 'org.apache.pulsar.client.impl.auth.AuthenticationToken', 'pulsar.client.authParams' = 'token:eyJhbGciOiJIUzI1NiJ9.eyJzdWIiOiJ1c2VyMSJ9.2AgtxHe8-2QBV529B5DrRtpuqP6RJjrk21Mhn' ``` + +## Relationship between the `apache/flink` repository and `streamnative/flink` repository + +StreamNative forks the [apache/flink](https://github.com/apache/flink) repository and maintains the project in [streamnative/flink](https://github.com/streamnative/flink). Compared with the official Flink repository, the StreamNative forked repository maintains a different Pulsar DataStream Connector and Pulsar SQL Connector. + +Currently, the `apache/flink` repository only contains the Pulsar DataStream connector while the `streamnative/flink` repository contains both the Pulsar DataStream connector and the Pulsar SQL connector. + +- Pulsar DataStream connector: the code and distribution in the `streamnative/flink` repository differ slightly from those in the `apache/flink` repository. Usually, new features and bug fixes will first go into the `streamnative/flink` repository, and then related PRs are submitted to the `apache/flink` repository. +- Pulsar SQL connector: StreamNative will merge the Pulsar SQL connector back to the Flink community in the coming months. + +For detailed differences and recommendations, see the following sections. + +### Components + +| Repository | Pulsar DataStream source | Pulsar DataStream sink | Pulsar SQL source | Pulsar SQL sink | +|-------------------------|--------------------------|------------------------|-------------------|-----------------| +| apache/flink 1.14 | Yes | | | | +| apache/flink 1.15 | Yes | Yes | | | +| streamnative/flink 1.15 | Yes | Yes | Yes | Yes | + +### Release process, version, and documentation + +| Repository | Release | Versioning | DataStream connector documentation | SQL connector documentation | +|-------------------|----------------------------------------------------------|------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------|-----------------------------| +| apache/flink | Managed by Flink community, following Flink release process | Standard Semantic Versioning, like 1.15.0 | Maintained on [Official Flink Website](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/) | Not available yet | +| streamnative/flink | Managed by StreamNative, following a monthly release process. | Use an extra digit on top of the Flink version, such as 1.15.0.1 | Not available yet, but similar to the [Official Flink Website](https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/pulsar/) | Maintained in current documentation | + +### Issue and Feature tracking + +- For the Pulsar DataStream connector, use [Flink JIRA](https://issues.apache.org/jira/projects/FLINK/issues) to submit bug reports and feature requests. +- For the Pulsar SQL connector, since it's not merged back to the official Flink repository yet, create an [issue](https://github.com/streamnative/flink/issues) in the `streamnative/flink` repository to submit bug reports and feature requests. Once the SQL Connector is merged back to the official Flink repository, use the Flink JIRA as well. + +### Artifact locations + +| Repository | Pulsar DataStream connector | Pulsar SQL connector | +|----------------------|-------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------| +| apache/flink | https://search.maven.org/artifact/org.apache.flink/flink-connector-pulsar | | +| streamnative/flink | https://search.maven.org/artifact/io.streamnative.connectors/flink-connector-pulsar | https://search.maven.org/artifact/io.streamnative.connectors/flink-sql-connector-pulsar | From 47a73b645199cf51e8c06e918b255bbd833c3f06 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Tue, 23 Aug 2022 13:54:03 +0800 Subject: [PATCH 249/258] 170: add properties metadata testing and documentation --- docs/content/docs/connectors/table/pulsar.md | 22 ++++---- .../pulsar/table/PulsarTableITCase.java | 52 +++++++++++++++++++ .../runtime/PulsarRuntimeOperator.java | 52 +++++++++++-------- 3 files changed, 92 insertions(+), 34 deletions(-) diff --git a/docs/content/docs/connectors/table/pulsar.md b/docs/content/docs/connectors/table/pulsar.md index 53917535370d0..0f6bef921a4e2 100644 --- a/docs/content/docs/connectors/table/pulsar.md +++ b/docs/content/docs/connectors/table/pulsar.md @@ -73,19 +73,17 @@ This table outlines the Pulsar topic metadata that can be mapped by the Pulsar S > - The `R/W` column defines whether a metadata is readable from the Pulsar topics (`R`) and/or writable to the Pulsar topics (`W`). > - The `R` column defines that a metadata is read-only. The read-only metadata must be declared `VIRTUAL` to exclude them during an `INSERT INTO` operation. -| Key | Data Type | Description | R/W | -|:--------------|:--------------------------|:------------|-----| -| topic | STRING NOT NULL | | R | -| message_size | INT NOT NULL | | R | -| producer_name | STRING NOT NULL | | R | -| message_id | BYTES NOT NULL | | R | -| sequenceId | BIGINT NOT NULL | | R | -| publish_time | TIMESTAMP_LTZ(3) NOT NULL | | R | -| event_time | TIMESTAMP_LTZ(3) NOT NULL | | R/W | +| Key | Data Type | R/W | +|:--------------|:-----------------------------|-----| +| topic | STRING NOT NULL | R | +| message_size | INT NOT NULL | R | +| producer_name | STRING NOT NULL | R | +| message_id | BYTES NOT NULL | R | +| sequenceId | BIGINT NOT NULL | R | +| publish_time | TIMESTAMP_LTZ(3) NOT NULL | R | +| event_time | TIMESTAMP_LTZ(3) NOT NULL | R/W | +| properties | MAP NOT NULL | R/W | -> **Note** -> -> Currently, the `properties` metadata field is not supported yet. The extended `CREATE TABLE` example demonstrates the syntax for exposing `publish_time`, `producer_name`, and `topic` metadata fields. diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java index 3f009d7c94ca7..18992a17fbc60 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -29,6 +29,8 @@ import org.apache.flink.types.Row; import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; @@ -42,13 +44,16 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Stream; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; +import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyThrow; import static org.apache.flink.connector.pulsar.table.testutils.PulsarTableTestUtils.collectRows; import static org.apache.flink.connector.pulsar.table.testutils.TestingUser.createRandomUser; import static org.apache.flink.util.CollectionUtil.entry; @@ -285,6 +290,53 @@ void pulsarSourceSinkWithMetadata(String format) throws Exception { true)); } + @Test + void sendMessageWithPropertiesAndReadPropertiesMetadata() throws Exception { + final String sourceTopic = "source_topic_" + randomAlphanumeric(3); + createTestTopic(sourceTopic, 1); + + // create producer and send one message + String value = randomAlphabetic(5); + Map properties = new HashMap<>(); + properties.put("key1", "value1"); + properties.put("key2", "value2"); + try { + Producer producer = + pulsar.operator().createProducer(sourceTopic, Schema.STRING); + producer.newMessage().value(value).properties(properties).send(); + } catch (PulsarClientException e) { + sneakyThrow(e); + } + + String sourceTableName = randomAlphabetic(5); + final String createSourceTable = + String.format( + "create table %s (\n" + + " name STRING\n," + + " `properties` MAP METADATA\n" + + ") with (\n" + + " 'connector' = '%s',\n" + + " 'topics' = '%s',\n" + + " 'service-url' = '%s',\n" + + " 'admin-url' = '%s',\n" + + " 'format' = '%s'\n" + + ")", + sourceTableName, + PulsarTableFactory.IDENTIFIER, + sourceTopic, + pulsar.operator().serviceUrl(), + pulsar.operator().adminUrl(), + RAW_FORMAT); + + tableEnv.executeSql(createSourceTable); + final List result = + collectRows( + tableEnv.sqlQuery(String.format("SELECT * FROM %s", sourceTableName)), 1); + assertThat(result) + .containsExactlyInAnyOrder( + Row.of(value, map(entry("key1", "value1"), entry("key2", "value2")))); + } + // TODO split this into two tests @ParameterizedTest @MethodSource("provideSchemaData") diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java index 60b5707d9f304..8587c1f34553d 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java @@ -277,6 +277,36 @@ public Map> topicsInfo(Collection topics) { return topics.stream().collect(toMap(identity(), this::topicInfo)); } + /** + * Create a producer for a given topic and schema. + * + * @param topic The name of the topic. + * @param schema The schema for serialization. + * @param The type of the record. + * @return producer instance. + */ + @SuppressWarnings("unchecked") + public Producer createProducer(String topic, Schema schema) + throws PulsarClientException { + TopicName topicName = TopicName.get(topic); + String name = topicName.getPartitionedTopicName(); + int index = topicName.getPartitionIndex(); + ConcurrentHashMap> topicProducers = + producers.computeIfAbsent(name, d -> new ConcurrentHashMap<>()); + + return (Producer) + topicProducers.computeIfAbsent( + index, + i -> { + try { + return client().newProducer(schema).topic(topic).create(); + } catch (PulsarClientException e) { + sneakyThrow(e); + return null; + } + }); + } + /** * Send a single message to Pulsar, return the message id after the ack from Pulsar. * @@ -507,28 +537,6 @@ public void close() throws IOException { // --------------------------- Private Methods ----------------------------- - @SuppressWarnings("unchecked") - private Producer createProducer(String topic, Schema schema) - throws PulsarClientException { - TopicName topicName = TopicName.get(topic); - String name = topicName.getPartitionedTopicName(); - int index = topicName.getPartitionIndex(); - ConcurrentHashMap> topicProducers = - producers.computeIfAbsent(name, d -> new ConcurrentHashMap<>()); - - return (Producer) - topicProducers.computeIfAbsent( - index, - i -> { - try { - return client().newProducer(schema).topic(topic).create(); - } catch (PulsarClientException e) { - sneakyThrow(e); - return null; - } - }); - } - @SuppressWarnings("unchecked") private Consumer createConsumer(String topic, Schema schema) throws PulsarClientException { From 3d3b6adf253ac5621057bf6324f61d3f26bec817 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Fri, 26 Aug 2022 21:02:57 +0800 Subject: [PATCH 250/258] Backport FLINK-27399 and support new start cursor and stop cursor. --- .../pulsar/source/PulsarSourceOptions.java | 6 +- .../config/PulsarSourceConfigUtils.java | 3 - .../enumerator/PulsarSourceEnumState.java | 13 +- ...igner.java => NonSharedSplitAssigner.java} | 35 ++--- .../assigner/SharedSplitAssigner.java | 33 ++--- .../assigner/SplitAssignerFactory.java | 21 +-- .../enumerator/cursor/CursorPosition.java | 16 ++- .../enumerator/cursor/MessageIdUtils.java | 71 ++++++++++ .../source/enumerator/cursor/StartCursor.java | 17 ++- .../source/enumerator/cursor/StopCursor.java | 72 ++++++++-- .../cursor/start/MessageIdStartCursor.java | 39 ++---- ...tCursor.java => TimestampStartCursor.java} | 10 +- .../cursor/stop/EventTimestampStopCursor.java | 13 +- .../cursor/stop/LatestMessageStopCursor.java | 32 ++--- .../cursor/stop/MessageIdStopCursor.java | 40 +++--- .../cursor/stop/NeverStopCursor.java | 10 +- .../stop/PublishTimestampStopCursor.java | 13 +- .../reader/PulsarSourceReaderFactory.java | 2 +- .../PulsarOrderedPartitionSplitReader.java | 34 +++-- .../split/PulsarPartitionSplitReaderBase.java | 53 ++++---- .../PulsarUnorderedPartitionSplitReader.java | 2 +- .../source/split/PulsarPartitionSplit.java | 5 +- .../pulsar/source/PulsarSourceITCase.java | 20 --- .../assigner/NonSharedSplitAssignerTest.java | 95 ++++++++++++++ .../assigner/SharedSplitAssignerTest.java | 98 ++++++++++++++ .../assigner/SplitAssignerTestBase.java | 113 ++++++++++++++++ .../enumerator/cursor/StopCursorTest.java | 9 +- .../PulsarPartitionSplitReaderTestBase.java | 7 +- .../table/PulsarTableOptionUtilsTest.java | 6 +- .../runtime/PulsarRuntimeOperator.java | 124 +++++++----------- 30 files changed, 677 insertions(+), 335 deletions(-) rename flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/{NormalSplitAssigner.java => NonSharedSplitAssigner.java} (80%) create mode 100644 flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java rename flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/{PublishTimestampStartCursor.java => TimestampStartCursor.java} (82%) create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java create mode 100644 flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java index 74e39ac005c54..6e66abbc01b5e 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/PulsarSourceOptions.java @@ -523,7 +523,11 @@ private PulsarSourceOptions() { code("PulsarClientException")) .build()); - /** @deprecated This option would be reset by {@link StartCursor}, no need to use it anymore. */ + /** + * @deprecated This option would be reset by {@link StartCursor}, no need to use it anymore. + * Pulsar didn't support this config option before 1.10.1, so we have to remove this config + * option. + */ @Deprecated public static final ConfigOption PULSAR_SUBSCRIPTION_INITIAL_POSITION = diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java index 602a1577938e1..0a4dc31e8d320 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/config/PulsarSourceConfigUtils.java @@ -59,7 +59,6 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_REPLICATE_SUBSCRIPTION_STATE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_RETRY_ENABLE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_RETRY_LETTER_TOPIC; -import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_INITIAL_POSITION; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_MODE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_TYPE; @@ -113,8 +112,6 @@ public static ConsumerBuilder createConsumerBuilder( builder::consumerName); configuration.useOption(PULSAR_READ_COMPACTED, builder::readCompacted); configuration.useOption(PULSAR_PRIORITY_LEVEL, builder::priorityLevel); - configuration.useOption( - PULSAR_SUBSCRIPTION_INITIAL_POSITION, builder::subscriptionInitialPosition); createDeadLetterPolicy(configuration).ifPresent(builder::deadLetterPolicy); configuration.useOption( PULSAR_AUTO_UPDATE_PARTITIONS_INTERVAL_SECONDS, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java index 0f3c66d40e8d0..56bbbd20a32a2 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/PulsarSourceEnumState.java @@ -22,6 +22,8 @@ import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Set; @@ -47,11 +49,12 @@ public class PulsarSourceEnumState { private final Map> sharedPendingPartitionSplits; /** - * It is used for Shared subscription. A {@link PulsarPartitionSplit} should be assigned for all - * flink readers. Using this map for recording assign status. + * It is used for Shared subscription. Every {@link PulsarPartitionSplit} should be assigned for + * all flink readers. Using this map for recording assign status. */ private final Map> readerAssignedSplits; + /** The pipeline has been triggered and topic partitions have been assigned to readers. */ private final boolean initialized; public PulsarSourceEnumState( @@ -86,4 +89,10 @@ public Map> getReaderAssignedSplits() { public boolean isInitialized() { return initialized; } + + /** The initial assignment state for Pulsar. */ + public static PulsarSourceEnumState initialState() { + return new PulsarSourceEnumState( + new HashSet<>(), new HashSet<>(), new HashMap<>(), new HashMap<>(), false); + } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NormalSplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java similarity index 80% rename from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NormalSplitAssigner.java rename to flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java index a03e3e64e6214..087e96157d655 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NormalSplitAssigner.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssigner.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.pulsar.source.enumerator.assigner; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.connector.source.SplitsAssignment; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; @@ -29,42 +30,34 @@ import java.util.ArrayList; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; /** - * This assigner is used for {@link SubscriptionType#Failover} and {@link - * SubscriptionType#Exclusive} subscriptions. + * This assigner is used for {@link SubscriptionType#Failover}, {@link SubscriptionType#Exclusive} + * and {@link SubscriptionType#Key_Shared} subscriptions. */ -public class NormalSplitAssigner implements SplitAssigner { +@Internal +public class NonSharedSplitAssigner implements SplitAssigner { private static final long serialVersionUID = 8412586087991597092L; private final StopCursor stopCursor; - private final SourceConfiguration sourceConfiguration; + private final boolean enablePartitionDiscovery; - // These states would be saved into checkpoint. + // These fields would be saved into checkpoint. private final Set appendedPartitions; private final Set pendingPartitionSplits; private boolean initialized; - public NormalSplitAssigner(StopCursor stopCursor, SourceConfiguration sourceConfiguration) { - this.stopCursor = stopCursor; - this.sourceConfiguration = sourceConfiguration; - this.appendedPartitions = new HashSet<>(); - this.pendingPartitionSplits = new HashSet<>(); - this.initialized = false; - } - - public NormalSplitAssigner( + public NonSharedSplitAssigner( StopCursor stopCursor, SourceConfiguration sourceConfiguration, PulsarSourceEnumState sourceEnumState) { this.stopCursor = stopCursor; - this.sourceConfiguration = sourceConfiguration; + this.enablePartitionDiscovery = sourceConfiguration.isEnablePartitionDiscovery(); this.appendedPartitions = sourceEnumState.getAppendedPartitions(); this.pendingPartitionSplits = sourceEnumState.getPendingPartitionSplits(); this.initialized = sourceEnumState.isInitialized(); @@ -72,7 +65,7 @@ public NormalSplitAssigner( @Override public List registerTopicPartitions(Set fetchedPartitions) { - List newPartitions = new ArrayList<>(fetchedPartitions.size()); + List newPartitions = new ArrayList<>(); for (TopicPartition partition : fetchedPartitions) { if (!appendedPartitions.contains(partition)) { @@ -102,25 +95,23 @@ public Optional> createAssignment( } Map> assignMap = new HashMap<>(); + List partitionSplits = new ArrayList<>(pendingPartitionSplits); - pendingPartitionSplits.clear(); int readerCount = readers.size(); - for (int i = 0; i < partitionSplits.size(); i++) { int index = i % readerCount; Integer readerId = readers.get(index); PulsarPartitionSplit split = partitionSplits.get(i); assignMap.computeIfAbsent(readerId, id -> new ArrayList<>()).add(split); } + pendingPartitionSplits.clear(); return Optional.of(new SplitsAssignment<>(assignMap)); } @Override public boolean noMoreSplits(Integer reader) { - return !sourceConfiguration.isEnablePartitionDiscovery() - && initialized - && pendingPartitionSplits.isEmpty(); + return !enablePartitionDiscovery && initialized && pendingPartitionSplits.isEmpty(); } @Override diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java index 70d1287f71197..48d75c8dee30d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssigner.java @@ -18,6 +18,7 @@ package org.apache.flink.connector.pulsar.source.enumerator.assigner; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.connector.source.SplitsAssignment; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; @@ -36,11 +37,12 @@ import java.util.Set; /** This assigner is used for {@link SubscriptionType#Shared} subscriptions. */ +@Internal public class SharedSplitAssigner implements SplitAssigner { private static final long serialVersionUID = 8468503133499402491L; private final StopCursor stopCursor; - private final SourceConfiguration sourceConfiguration; + private final boolean enablePartitionDiscovery; // These fields would be saved into checkpoint. @@ -49,23 +51,12 @@ public class SharedSplitAssigner implements SplitAssigner { private final Map> readerAssignedSplits; private boolean initialized; - // These fields are used as the dynamic initializing record. - - public SharedSplitAssigner(StopCursor stopCursor, SourceConfiguration sourceConfiguration) { - this.stopCursor = stopCursor; - this.sourceConfiguration = sourceConfiguration; - this.appendedPartitions = new HashSet<>(); - this.sharedPendingPartitionSplits = new HashMap<>(); - this.readerAssignedSplits = new HashMap<>(); - this.initialized = false; - } - public SharedSplitAssigner( StopCursor stopCursor, SourceConfiguration sourceConfiguration, PulsarSourceEnumState sourceEnumState) { this.stopCursor = stopCursor; - this.sourceConfiguration = sourceConfiguration; + this.enablePartitionDiscovery = sourceConfiguration.isEnablePartitionDiscovery(); this.appendedPartitions = sourceEnumState.getAppendedPartitions(); this.sharedPendingPartitionSplits = sourceEnumState.getSharedPendingPartitionSplits(); this.readerAssignedSplits = sourceEnumState.getReaderAssignedSplits(); @@ -74,12 +65,12 @@ public SharedSplitAssigner( @Override public List registerTopicPartitions(Set fetchedPartitions) { - List newPartitions = new ArrayList<>(fetchedPartitions.size()); + List newPartitions = new ArrayList<>(); - for (TopicPartition fetchedPartition : fetchedPartitions) { - if (!appendedPartitions.contains(fetchedPartition)) { - newPartitions.add(fetchedPartition); - appendedPartitions.add(fetchedPartition); + for (TopicPartition partition : fetchedPartitions) { + if (!appendedPartitions.contains(partition)) { + appendedPartitions.add(partition); + newPartitions.add(partition); } } @@ -92,9 +83,9 @@ public List registerTopicPartitions(Set fetchedP @Override public void addSplitsBack(List splits, int subtaskId) { - Set pending = + Set pendingPartitionSplits = sharedPendingPartitionSplits.computeIfAbsent(subtaskId, id -> new HashSet<>()); - pending.addAll(splits); + pendingPartitionSplits.addAll(splits); } @Override @@ -139,7 +130,7 @@ public boolean noMoreSplits(Integer reader) { Set pendingSplits = sharedPendingPartitionSplits.get(reader); Set assignedSplits = readerAssignedSplits.get(reader); - return !sourceConfiguration.isEnablePartitionDiscovery() + return !enablePartitionDiscovery && initialized && (pendingSplits == null || pendingSplits.isEmpty()) && (assignedSplits != null && assignedSplits.size() == appendedPartitions.size()); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java index 0dc8d23525d90..3e6ebccb49b4c 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerFactory.java @@ -18,20 +18,21 @@ package org.apache.flink.connector.pulsar.source.enumerator.assigner; +import org.apache.flink.annotation.Internal; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.pulsar.client.api.SubscriptionType; -import javax.annotation.Nullable; - +import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState.initialState; import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; import static org.apache.pulsar.client.api.SubscriptionType.Failover; import static org.apache.pulsar.client.api.SubscriptionType.Key_Shared; import static org.apache.pulsar.client.api.SubscriptionType.Shared; /** The factory for creating split assigner. */ +@Internal public final class SplitAssignerFactory { private SplitAssignerFactory() { @@ -41,29 +42,21 @@ private SplitAssignerFactory() { /** Create blank assigner. */ public static SplitAssigner create( StopCursor stopCursor, SourceConfiguration sourceConfiguration) { - return create(stopCursor, sourceConfiguration, null); + return create(stopCursor, sourceConfiguration, initialState()); } /** Create assigner from checkpoint state. */ public static SplitAssigner create( StopCursor stopCursor, SourceConfiguration sourceConfiguration, - @Nullable PulsarSourceEnumState sourceEnumState) { + PulsarSourceEnumState sourceEnumState) { SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType(); if (subscriptionType == Exclusive || subscriptionType == Failover || subscriptionType == Key_Shared) { - if (sourceEnumState == null) { - return new NormalSplitAssigner(stopCursor, sourceConfiguration); - } else { - return new NormalSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); - } + return new NonSharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); } else if (subscriptionType == Shared) { - if (sourceEnumState == null) { - return new SharedSplitAssigner(stopCursor, sourceConfiguration); - } else { - return new SharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); - } + return new SharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); } else { throw new IllegalArgumentException( "We don't support this subscription type: " + subscriptionType); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java index 769585182e254..c965ff962f807 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/CursorPosition.java @@ -18,15 +18,16 @@ package org.apache.flink.connector.pulsar.source.enumerator.cursor; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; import org.apache.pulsar.client.api.MessageId; -import javax.annotation.Nullable; - import java.io.Serializable; -/** The class for defining the start or stop position. */ +/** + * The class for defining the start or stop position. We only expose the constructor for end user. + */ @PublicEvolving public final class CursorPosition implements Serializable { private static final long serialVersionUID = -802405183307684549L; @@ -37,26 +38,29 @@ public final class CursorPosition implements Serializable { private final Long timestamp; - public CursorPosition(@Nullable MessageId messageId) { + public CursorPosition(MessageId messageId) { this.type = Type.MESSAGE_ID; this.messageId = messageId; this.timestamp = null; } - public CursorPosition(@Nullable Long timestamp) { + public CursorPosition(Long timestamp) { this.type = Type.TIMESTAMP; this.messageId = null; this.timestamp = timestamp; } + @Internal public Type getType() { return type; } + @Internal public MessageId getMessageId() { return messageId; } + @Internal public Long getTimestamp() { return timestamp; } @@ -73,7 +77,7 @@ public String toString() { /** * The position type for reader to choose whether timestamp or message id as the start position. */ - @PublicEvolving + @Internal public enum Type { TIMESTAMP, diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java new file mode 100644 index 0000000000000..a8c3a6b2ef284 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/MessageIdUtils.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.cursor; + +import org.apache.flink.annotation.Internal; + +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.impl.BatchMessageIdImpl; +import org.apache.pulsar.client.impl.MessageIdImpl; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** The helper class for Pulsar's message id. */ +@Internal +public final class MessageIdUtils { + + private MessageIdUtils() { + // No public constructor. + } + + /** + * The implementation from this + * code snippet to get next message id. + */ + public static MessageId nextMessageId(MessageId messageId) { + MessageIdImpl idImpl = unwrapMessageId(messageId); + + if (idImpl.getEntryId() < 0) { + return newMessageId(idImpl.getLedgerId(), 0, idImpl.getPartitionIndex()); + } else { + return newMessageId( + idImpl.getLedgerId(), idImpl.getEntryId() + 1, idImpl.getPartitionIndex()); + } + } + + /** + * Convert the message id interface to its backend implementation. And check if it's a batch + * message id. We don't support the batch message for its low performance now. + */ + public static MessageIdImpl unwrapMessageId(MessageId messageId) { + MessageIdImpl idImpl = MessageIdImpl.convertToMessageIdImpl(messageId); + if (idImpl instanceof BatchMessageIdImpl) { + int batchSize = ((BatchMessageIdImpl) idImpl).getBatchSize(); + checkArgument(batchSize == 1, "We only support normal message id currently."); + } + + return idImpl; + } + + /** Hide the message id implementation. */ + public static MessageId newMessageId(long ledgerId, long entryId, int partitionIndex) { + return new MessageIdImpl(ledgerId, entryId, partitionIndex); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java index da0c4ec739b1f..9c1d699a269f7 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StartCursor.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.PublishTimestampStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.SubscriptionType; @@ -73,7 +73,20 @@ static StartCursor fromMessageId(MessageId messageId, boolean inclusive) { return new MessageIdStartCursor(messageId, inclusive); } + /** + * This method is designed for seeking message from event time. But Pulsar didn't support + * seeking from message time, instead, it would seek the position from publish time. We only + * keep this method for backward compatible. + * + * @deprecated Use {@link #fromPublishTime(long)} instead. + */ + @Deprecated + static StartCursor fromMessageTime(long timestamp) { + return new TimestampStartCursor(timestamp, true); + } + + /** Seek the start position by using message publish time. */ static StartCursor fromPublishTime(long timestamp) { - return new PublishTimestampStartCursor(timestamp); + return new TimestampStartCursor(timestamp, true); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java index 3a55c4fbad73d..d44c78fcf1a44 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursor.java @@ -44,11 +44,55 @@ public interface StopCursor extends Serializable { /** The open method for the cursor initializer. This method could be executed multiple times. */ default void open(PulsarAdmin admin, TopicPartition partition) {} - /** - * Determine whether to pause consumption on the current message by the returned boolean value. - * The message presented in method argument wouldn't be consumed if the return result is true. - */ - boolean shouldStop(Message message); + /** Determine whether to pause consumption on the current message by the returned enum. */ + StopCondition shouldStop(Message message); + + /** The conditional for control the stop behavior of the pulsar source. */ + @PublicEvolving + enum StopCondition { + + /** This message should be included in the result. */ + CONTINUE, + /** This message should be included in the result and stop consuming. */ + EXACTLY, + /** Stop consuming, the given message wouldn't be included in the result. */ + TERMINATE; + + /** + * Common methods for comparing the message id. + * + * @param desired The stop goal of the message id. + * @param current The upcoming message id. + * @param inclusive Should the desired message be included in the consuming result. + */ + public static StopCondition compare( + MessageId desired, MessageId current, boolean inclusive) { + if (current.compareTo(desired) < 0) { + return StopCondition.CONTINUE; + } else if (current.compareTo(desired) == 0) { + return inclusive ? StopCondition.EXACTLY : StopCondition.TERMINATE; + } else { + return StopCondition.TERMINATE; + } + } + + /** + * Common methods for comparing the message time. + * + * @param desired The stop goal of the message time. + * @param current The upcoming message time. + * @param inclusive Should the desired message be included in the consuming result. + */ + public static StopCondition compare(long desired, long current, boolean inclusive) { + if (current < desired) { + return StopCondition.CONTINUE; + } else if (current == desired) { + return inclusive ? StopCondition.EXACTLY : StopCondition.TERMINATE; + } else { + return StopCondition.TERMINATE; + } + } + } // --------------------------- Static Factory Methods ----------------------------- @@ -61,7 +105,7 @@ static StopCursor never() { } static StopCursor latest() { - return new LatestMessageStopCursor(); + return new LatestMessageStopCursor(true); } /** @@ -70,9 +114,9 @@ static StopCursor latest() { */ static StopCursor atMessageId(MessageId messageId) { if (MessageId.latest.equals(messageId)) { - return new LatestMessageStopCursor(true); + return new LatestMessageStopCursor(false); } else { - return new MessageIdStopCursor(messageId); + return new MessageIdStopCursor(messageId, false); } } @@ -82,31 +126,31 @@ static StopCursor atMessageId(MessageId messageId) { */ static StopCursor afterMessageId(MessageId messageId) { if (MessageId.latest.equals(messageId)) { - return new LatestMessageStopCursor(false); + return new LatestMessageStopCursor(true); } else { - return new MessageIdStopCursor(messageId, false); + return new MessageIdStopCursor(messageId, true); } } /** Stop consuming when message eventTime is greater than or equals the specified timestamp. */ static StopCursor atEventTime(long timestamp) { - return new EventTimestampStopCursor(timestamp); + return new EventTimestampStopCursor(timestamp, false); } /** Stop consuming when message eventTime is greater than the specified timestamp. */ static StopCursor afterEventTime(long timestamp) { - return new EventTimestampStopCursor(timestamp + 1); + return new EventTimestampStopCursor(timestamp, true); } /** * Stop consuming when message publishTime is greater than or equals the specified timestamp. */ static StopCursor atPublishTime(long timestamp) { - return new PublishTimestampStopCursor(timestamp); + return new PublishTimestampStopCursor(timestamp, false); } /** Stop consuming when message publishTime is greater than the specified timestamp. */ static StopCursor afterPublishTime(long timestamp) { - return new PublishTimestampStopCursor(timestamp + 1); + return new PublishTimestampStopCursor(timestamp, true); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java index f4156d4e7acfc..1f26e834749fa 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/MessageIdStartCursor.java @@ -23,13 +23,12 @@ import org.apache.pulsar.client.api.ConsumerBuilder; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; -import org.apache.pulsar.client.internal.DefaultImplementation; import java.util.Objects; -import static org.apache.flink.util.Preconditions.checkState; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.nextMessageId; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId; /** This cursor would leave pulsar start consuming from a specific message id. */ public class MessageIdStartCursor implements StartCursor { @@ -47,38 +46,16 @@ public class MessageIdStartCursor implements StartCursor { * code for understanding pulsar internal logic. * * @param messageId The message id for start position. - * @param inclusive Should we include the start message id in consuming result. This is works - * only if we provide a specified message id instead of {@link MessageId#earliest} or {@link + * @param inclusive Whether we include the start message id in consuming result. This works only + * if we provide a specified message id instead of {@link MessageId#earliest} or {@link * MessageId#latest}. */ public MessageIdStartCursor(MessageId messageId, boolean inclusive) { - MessageIdImpl id = MessageIdImpl.convertToMessageIdImpl(messageId); - checkState( - !(id instanceof BatchMessageIdImpl), - "We only support normal message id currently."); - - if (MessageId.earliest.equals(id) || MessageId.latest.equals(id) || inclusive) { - this.messageId = id; - } else { - this.messageId = getNext(id); - } - } - - /** - * The implementation from the this - * code to get the next message id. - */ - public static MessageId getNext(MessageIdImpl messageId) { - if (messageId.getEntryId() < 0) { - return DefaultImplementation.getDefaultImplementation() - .newMessageId(messageId.getLedgerId(), 0, messageId.getPartitionIndex()); + MessageIdImpl idImpl = unwrapMessageId(messageId); + if (MessageId.earliest.equals(idImpl) || MessageId.latest.equals(idImpl) || inclusive) { + this.messageId = idImpl; } else { - return DefaultImplementation.getDefaultImplementation() - .newMessageId( - messageId.getLedgerId(), - messageId.getEntryId() + 1, - messageId.getPartitionIndex()); + this.messageId = nextMessageId(idImpl); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/PublishTimestampStartCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java similarity index 82% rename from flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/PublishTimestampStartCursor.java rename to flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java index d0d7428825c39..065de66b39fed 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/PublishTimestampStartCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/start/TimestampStartCursor.java @@ -23,14 +23,14 @@ import java.util.Objects; -/** This cursor make pulsar start consuming from a specific publish timestamp. */ -public class PublishTimestampStartCursor implements StartCursor { +/** This cursor would left pulsar start consuming from a specific publish timestamp. */ +public class TimestampStartCursor implements StartCursor { private static final long serialVersionUID = 5170578885838095320L; private final long timestamp; - public PublishTimestampStartCursor(long timestamp) { - this.timestamp = timestamp; + public TimestampStartCursor(long timestamp, boolean inclusive) { + this.timestamp = inclusive ? timestamp : timestamp + 1; } @Override @@ -46,7 +46,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - PublishTimestampStartCursor that = (PublishTimestampStartCursor) o; + TimestampStartCursor that = (TimestampStartCursor) o; return timestamp == that.timestamp; } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java index c569fe1e3b317..dfaadbed0a4b3 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/EventTimestampStopCursor.java @@ -29,14 +29,17 @@ public class EventTimestampStopCursor implements StopCursor { private static final long serialVersionUID = 2391576769339369027L; private final long timestamp; + private final boolean inclusive; - public EventTimestampStopCursor(long timestamp) { + public EventTimestampStopCursor(long timestamp, boolean inclusive) { this.timestamp = timestamp; + this.inclusive = inclusive; } @Override - public boolean shouldStop(Message message) { - return message.getEventTime() >= timestamp; + public StopCondition shouldStop(Message message) { + long eventTime = message.getEventTime(); + return StopCondition.compare(timestamp, eventTime, inclusive); } @Override @@ -48,11 +51,11 @@ public boolean equals(Object o) { return false; } EventTimestampStopCursor that = (EventTimestampStopCursor) o; - return timestamp == that.timestamp; + return timestamp == that.timestamp && inclusive == that.inclusive; } @Override public int hashCode() { - return Objects.hash(timestamp); + return Objects.hash(timestamp, inclusive); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java index bdd054a3c442d..0311e4a8fb362 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/LatestMessageStopCursor.java @@ -38,35 +38,23 @@ public class LatestMessageStopCursor implements StopCursor { private static final long serialVersionUID = 1702059838323965723L; private MessageId messageId; + private final boolean inclusive; - /** - * Set this to false would include the latest available message when the flink pipeline start. - */ - private final boolean exclusive; - - public LatestMessageStopCursor() { - this.exclusive = false; + public LatestMessageStopCursor(boolean inclusive) { + this.inclusive = inclusive; } - public LatestMessageStopCursor(boolean exclusive) { - this.exclusive = exclusive; + @Override + public StopCondition shouldStop(Message message) { + MessageId current = message.getMessageId(); + return StopCondition.compare(messageId, current, inclusive); } @Override public void open(PulsarAdmin admin, TopicPartition partition) { if (messageId == null) { String topic = partition.getFullTopicName(); - messageId = sneakyAdmin(() -> admin.topics().getLastMessageId(topic)); - } - } - - @Override - public boolean shouldStop(Message message) { - MessageId id = message.getMessageId(); - if (exclusive) { - return id.compareTo(messageId) > 0; - } else { - return id.compareTo(messageId) >= 0; + this.messageId = sneakyAdmin(() -> admin.topics().getLastMessageId(topic)); } } @@ -79,11 +67,11 @@ public boolean equals(Object o) { return false; } LatestMessageStopCursor that = (LatestMessageStopCursor) o; - return exclusive == that.exclusive && Objects.equals(messageId, that.messageId); + return inclusive == that.inclusive && Objects.equals(messageId, that.messageId); } @Override public int hashCode() { - return Objects.hash(messageId, exclusive); + return Objects.hash(messageId, inclusive); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java index 7ea6c50c6dbc9..f6ff913199fd1 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/MessageIdStopCursor.java @@ -22,13 +22,14 @@ import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.impl.BatchMessageIdImpl; import org.apache.pulsar.client.impl.MessageIdImpl; import java.util.Objects; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.unwrapMessageId; import static org.apache.flink.util.Preconditions.checkArgument; -import static org.apache.flink.util.Preconditions.checkState; +import static org.apache.pulsar.client.api.MessageId.earliest; +import static org.apache.pulsar.client.api.MessageId.latest; /** * Stop consuming message at a given message id. We use the {@link MessageId#compareTo(Object)} for @@ -39,34 +40,23 @@ public class MessageIdStopCursor implements StopCursor { private final MessageId messageId; - private final boolean exclusive; + private final boolean inclusive; - public MessageIdStopCursor(MessageId messageId) { - this(messageId, true); - } - - public MessageIdStopCursor(MessageId messageId, boolean exclusive) { - MessageIdImpl id = MessageIdImpl.convertToMessageIdImpl(messageId); - checkState( - !(id instanceof BatchMessageIdImpl), - "We only support normal message id currently."); - checkArgument(!MessageId.earliest.equals(id), "MessageId.earliest is not supported."); + public MessageIdStopCursor(MessageId messageId, boolean inclusive) { + MessageIdImpl idImpl = unwrapMessageId(messageId); + checkArgument(!earliest.equals(idImpl), "MessageId.earliest is not supported."); checkArgument( - !MessageId.latest.equals(id), + !latest.equals(idImpl), "MessageId.latest is not supported, use LatestMessageStopCursor instead."); - this.messageId = id; - this.exclusive = exclusive; + this.messageId = idImpl; + this.inclusive = inclusive; } @Override - public boolean shouldStop(Message message) { - MessageId id = message.getMessageId(); - if (exclusive) { - return id.compareTo(messageId) > 0; - } else { - return id.compareTo(messageId) >= 0; - } + public StopCondition shouldStop(Message message) { + MessageId current = message.getMessageId(); + return StopCondition.compare(messageId, current, inclusive); } @Override @@ -78,11 +68,11 @@ public boolean equals(Object o) { return false; } MessageIdStopCursor that = (MessageIdStopCursor) o; - return exclusive == that.exclusive && messageId.equals(that.messageId); + return inclusive == that.inclusive && Objects.equals(messageId, that.messageId); } @Override public int hashCode() { - return Objects.hash(messageId, exclusive); + return Objects.hash(messageId, inclusive); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java index 8d646c572c77f..99a42976d5b92 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/NeverStopCursor.java @@ -27,8 +27,8 @@ public class NeverStopCursor implements StopCursor { private static final long serialVersionUID = -3113601090292771786L; @Override - public boolean shouldStop(Message message) { - return false; + public StopCondition shouldStop(Message message) { + return StopCondition.CONTINUE; } @Override @@ -36,9 +36,7 @@ public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { - return false; - } - return true; + + return o != null && getClass() == o.getClass(); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java index 9dc914de8cb35..c374f90c6a7af 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/stop/PublishTimestampStopCursor.java @@ -29,14 +29,17 @@ public class PublishTimestampStopCursor implements StopCursor { private static final long serialVersionUID = 4386276745339324527L; private final long timestamp; + private final boolean inclusive; - public PublishTimestampStopCursor(long timestamp) { + public PublishTimestampStopCursor(long timestamp, boolean inclusive) { this.timestamp = timestamp; + this.inclusive = inclusive; } @Override - public boolean shouldStop(Message message) { - return message.getPublishTime() >= timestamp; + public StopCondition shouldStop(Message message) { + long publishTime = message.getPublishTime(); + return StopCondition.compare(timestamp, publishTime, inclusive); } @Override @@ -48,11 +51,11 @@ public boolean equals(Object o) { return false; } PublishTimestampStopCursor that = (PublishTimestampStopCursor) o; - return timestamp == that.timestamp; + return timestamp == that.timestamp && inclusive == that.inclusive; } @Override public int hashCode() { - return Objects.hash(timestamp); + return Objects.hash(timestamp, inclusive); } } diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java index d86e6a95166dd..ab384fba061b5 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/PulsarSourceReaderFactory.java @@ -98,7 +98,7 @@ public static SourceReader create( SubscriptionType subscriptionType = sourceConfiguration.getSubscriptionType(); if (subscriptionType == SubscriptionType.Failover || subscriptionType == SubscriptionType.Exclusive) { - // Create a ordered split reader supplier. + // Create an ordered split reader supplier. Supplier splitReaderSupplier = () -> new PulsarOrderedPartitionSplitReader( diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java index 3131c00cb8524..200f5b356abc8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarOrderedPartitionSplitReader.java @@ -20,30 +20,30 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.reader.source.PulsarOrderedSourceReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.pulsar.client.admin.PulsarAdmin; -import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.api.CryptoKeyReader; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.impl.MessageIdImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; import java.time.Duration; +import java.util.List; import java.util.concurrent.TimeUnit; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; import static org.apache.flink.connector.pulsar.source.config.CursorVerification.FAIL_ON_MISMATCH; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.MessageIdUtils.nextMessageId; /** * The split reader a given {@link PulsarPartitionSplit}, it would be closed once the {@link @@ -78,25 +78,39 @@ protected void finishedPollMessage(Message message) { } @Override - protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + protected void beforeCreatingConsumer(PulsarPartitionSplit split) { MessageId latestConsumedId = split.getLatestConsumedId(); // Reset the start position for ordered pulsar consumer. if (latestConsumedId != null) { - LOG.debug("Start seeking from the checkpoint {}", latestConsumedId); + LOG.info("Reset subscription position by the checkpoint {}", latestConsumedId); try { MessageId initialPosition; if (latestConsumedId == MessageId.latest || latestConsumedId == MessageId.earliest) { + // for compatibility initialPosition = latestConsumedId; } else { - MessageIdImpl messageId = - MessageIdImpl.convertToMessageIdImpl(latestConsumedId); - initialPosition = MessageIdStartCursor.getNext(messageId); + initialPosition = nextMessageId(latestConsumedId); } - consumer.seek(initialPosition); - } catch (PulsarClientException e) { + // Remove Consumer.seek() here for waiting for pulsar-client-all 2.12.0 + // See https://github.com/apache/pulsar/issues/16757 for more details. + + String topicName = split.getPartition().getFullTopicName(); + List subscriptions = pulsarAdmin.topics().getSubscriptions(topicName); + String subscriptionName = sourceConfiguration.getSubscriptionName(); + + if (!subscriptions.contains(subscriptionName)) { + // If this subscription is not available. Just create it. + pulsarAdmin + .topics() + .createSubscription(topicName, subscriptionName, initialPosition); + } else { + // Reset the subscription if this is existed. + pulsarAdmin.topics().resetCursor(topicName, subscriptionName, initialPosition); + } + } catch (PulsarAdminException e) { if (sourceConfiguration.getVerifyInitialOffsets() == FAIL_ON_MISMATCH) { throw new IllegalArgumentException(e); } else { diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java index abba214763f41..2e7b972d4472a 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderBase.java @@ -26,6 +26,7 @@ import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.StopCondition; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; import org.apache.flink.util.Preconditions; @@ -49,7 +50,6 @@ import java.time.Duration; import java.util.List; import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyClient; import static org.apache.flink.connector.pulsar.source.config.PulsarSourceConfigUtils.createConsumerBuilder; @@ -65,8 +65,6 @@ abstract class PulsarPartitionSplitReaderBase protected final Schema schema; @Nullable protected final CryptoKeyReader cryptoKeyReader; - protected final AtomicBoolean wakeup; - protected Consumer pulsarConsumer; protected PulsarPartitionSplit registeredSplit; @@ -81,7 +79,6 @@ protected PulsarPartitionSplitReaderBase( this.sourceConfiguration = sourceConfiguration; this.schema = schema; this.cryptoKeyReader = cryptoKeyReader; - this.wakeup = new AtomicBoolean(false); } @Override @@ -93,18 +90,13 @@ public RecordsWithSplitIds> fetch() throws IOException { return builder.build(); } - // Set wakeup to false for start consuming. - wakeup.compareAndSet(true, false); - StopCursor stopCursor = registeredSplit.getStopCursor(); String splitId = registeredSplit.splitId(); Deadline deadline = Deadline.fromNow(sourceConfiguration.getMaxFetchTime()); // Consume messages from pulsar until it was waked up by flink reader. for (int messageNum = 0; - messageNum < sourceConfiguration.getMaxFetchRecords() - && deadline.hasTimeLeft() - && isNotWakeup(); + messageNum < sourceConfiguration.getMaxFetchRecords() && deadline.hasTimeLeft(); messageNum++) { try { Message message = pollMessage(sourceConfiguration.getDefaultFetchTime()); @@ -112,12 +104,17 @@ public RecordsWithSplitIds> fetch() throws IOException { break; } - builder.add(splitId, message); + StopCondition condition = stopCursor.shouldStop(message); - // Acknowledge message if needed. - finishedPollMessage(message); + if (condition == StopCondition.CONTINUE || condition == StopCondition.EXACTLY) { + // Deserialize message. + builder.add(splitId, message); + + // Acknowledge message if needed. + finishedPollMessage(message); + } - if (stopCursor.shouldStop(message)) { + if (condition == StopCondition.EXACTLY || condition == StopCondition.TERMINATE) { builder.addFinishedSplit(splitId); break; } @@ -156,23 +153,27 @@ public void handleSplitsChanges(SplitsChange splitsChanges newSplits.size() == 1, "This pulsar split reader only support one split."); PulsarPartitionSplit newSplit = newSplits.get(0); + // Open stop cursor. + newSplit.open(pulsarAdmin); + + // Before creating the consumer. + beforeCreatingConsumer(newSplit); + // Create pulsar consumer. Consumer consumer = createPulsarConsumer(newSplit); - // Open start & stop cursor. - newSplit.open(pulsarAdmin); - - // Start Consumer. - startConsumer(newSplit, consumer); + // After creating the consumer. + afterCreatingConsumer(newSplit, consumer); LOG.info("Register split {} consumer for current reader.", newSplit); + this.registeredSplit = newSplit; this.pulsarConsumer = consumer; } @Override public void wakeUp() { - wakeup.compareAndSet(false, true); + // Nothing to do on this method. } @Override @@ -188,14 +189,16 @@ protected abstract Message pollMessage(Duration timeout) protected abstract void finishedPollMessage(Message message); - protected abstract void startConsumer(PulsarPartitionSplit split, Consumer consumer); - - // --------------------------- Helper Methods ----------------------------- + protected void beforeCreatingConsumer(PulsarPartitionSplit split) { + // Nothing to do by default. + } - protected boolean isNotWakeup() { - return !wakeup.get(); + protected void afterCreatingConsumer(PulsarPartitionSplit split, Consumer consumer) { + // Nothing to do by default. } + // --------------------------- Helper Methods ----------------------------- + /** Create a specified {@link Consumer} by the given split information. */ protected Consumer createPulsarConsumer(PulsarPartitionSplit split) { return createPulsarConsumer(split.getPartition()); diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java index eb7064872db1b..b076557bb3a84 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarUnorderedPartitionSplitReader.java @@ -116,7 +116,7 @@ protected void finishedPollMessage(Message message) { } @Override - protected void startConsumer(PulsarPartitionSplit split, Consumer consumer) { + protected void afterCreatingConsumer(PulsarPartitionSplit split, Consumer consumer) { TxnID uncommittedTransactionId = split.getUncommittedTransactionId(); // Abort the uncommitted pulsar transaction. diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java index 90e29ca47120f..2b2a7cda80fab 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/source/split/PulsarPartitionSplit.java @@ -30,14 +30,15 @@ import javax.annotation.Nullable; +import java.io.Serializable; import java.util.Objects; import static org.apache.flink.util.Preconditions.checkNotNull; /** A {@link SourceSplit} implementation for a Pulsar's partition. */ @Internal -public class PulsarPartitionSplit implements SourceSplit { - +public class PulsarPartitionSplit implements SourceSplit, Serializable { + private static final long serialVersionUID = -6857317360756062625L; private final TopicPartition partition; private final StopCursor stopCursor; diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java index 497297830767d..847c3dd5a9ea1 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/PulsarSourceITCase.java @@ -24,8 +24,6 @@ import org.apache.flink.connector.pulsar.testutils.source.cases.MultipleTopicConsumingContext; import org.apache.flink.connector.pulsar.testutils.source.cases.SingleTopicConsumingContext; import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; -import org.apache.flink.connector.testframe.environment.TestEnvironment; -import org.apache.flink.connector.testframe.external.source.DataStreamSourceExternalContext; import org.apache.flink.connector.testframe.junit.annotations.TestContext; import org.apache.flink.connector.testframe.junit.annotations.TestEnv; import org.apache.flink.connector.testframe.junit.annotations.TestExternalSystem; @@ -41,24 +39,6 @@ @ExtendWith(MiniClusterExtension.class) class PulsarSourceITCase extends SourceTestSuiteBase { - @Override - public void testScaleUp( - TestEnvironment testEnv, - DataStreamSourceExternalContext externalContext, - CheckpointingMode semantic) - throws Exception { - super.testScaleUp(testEnv, externalContext, semantic); - } - - @Override - public void testScaleDown( - TestEnvironment testEnv, - DataStreamSourceExternalContext externalContext, - CheckpointingMode semantic) - throws Exception { - super.testScaleDown(testEnv, externalContext, semantic); - } - // Defines test environment on Flink MiniCluster @TestEnv MiniClusterTestEnvironment flink = new MiniClusterTestEnvironment(); diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java new file mode 100644 index 0000000000000..2e9ada3b74135 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/NonSharedSplitAssignerTest.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static java.util.Collections.singletonList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Unit tests for {@link NonSharedSplitAssigner}. */ +class NonSharedSplitAssignerTest extends SplitAssignerTestBase { + + @Test + void noMoreSplits() { + NonSharedSplitAssigner assigner = splitAssigner(true); + assertFalse(assigner.noMoreSplits(3)); + + assigner = splitAssigner(false); + assertFalse(assigner.noMoreSplits(3)); + + assigner.registerTopicPartitions(createPartitions("f", 8)); + assertFalse(assigner.noMoreSplits(3)); + + assigner.createAssignment(singletonList(1)); + assertTrue(assigner.noMoreSplits(1)); + assertTrue(assigner.noMoreSplits(3)); + } + + @Test + void partitionsAssignment() { + NonSharedSplitAssigner assigner = splitAssigner(true); + assigner.registerTopicPartitions(createPartitions("d", 4)); + List readers = Arrays.asList(1, 3, 5, 7); + + // Assignment with initial states. + Optional> assignment = + assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(1); + + // Reassignment with same readers. + assignment = assigner.createAssignment(readers); + assertThat(assignment).isNotPresent(); + + // Register new partition and assign. + assigner.registerTopicPartitions(createPartitions("e", 5)); + assigner.registerTopicPartitions(createPartitions("f", 1)); + assigner.registerTopicPartitions(createPartitions("g", 3)); + assigner.registerTopicPartitions(createPartitions("h", 4)); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(4); + + // Assign to new readers. + readers = Arrays.asList(2, 4, 6, 8); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isNotPresent(); + } + + @Override + protected NonSharedSplitAssigner createAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + return new NonSharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java new file mode 100644 index 0000000000000..91584b8768821 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SharedSplitAssignerTest.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +import static java.util.Collections.singletonList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** Unit tests for {@link SharedSplitAssigner}. */ +class SharedSplitAssignerTest extends SplitAssignerTestBase { + + @Test + void noMoreSplits() { + SharedSplitAssigner assigner = splitAssigner(true); + assertFalse(assigner.noMoreSplits(3)); + + assigner = splitAssigner(false); + assertFalse(assigner.noMoreSplits(3)); + + assigner.registerTopicPartitions(createPartitions("f", 8)); + assertFalse(assigner.noMoreSplits(3)); + + assigner.createAssignment(singletonList(1)); + assertTrue(assigner.noMoreSplits(1)); + assertFalse(assigner.noMoreSplits(3)); + + assigner.createAssignment(singletonList(3)); + assertTrue(assigner.noMoreSplits(3)); + } + + @Test + void partitionsAssignment() { + SharedSplitAssigner assigner = splitAssigner(true); + assigner.registerTopicPartitions(createPartitions("d", 4)); + List readers = Arrays.asList(1, 3, 5, 7); + + // Assignment with initial states. + Optional> assignment = + assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(4); + + // Reassignment with same readers. + assignment = assigner.createAssignment(readers); + assertThat(assignment).isNotPresent(); + + // Register new partition and assign. + assigner.registerTopicPartitions(createPartitions("e", 5)); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()).hasSize(4); + + // Assign to new readers. + readers = Arrays.asList(2, 4, 6, 8); + assignment = assigner.createAssignment(readers); + assertThat(assignment).isPresent(); + assertThat(assignment.get().assignment()) + .hasSize(4) + .allSatisfy((k, v) -> assertThat(v).hasSize(2)); + } + + @Override + protected SharedSplitAssigner createAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState) { + return new SharedSplitAssigner(stopCursor, sourceConfiguration, sourceEnumState); + } +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java new file mode 100644 index 0000000000000..6509401472068 --- /dev/null +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/assigner/SplitAssignerTestBase.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.connector.pulsar.source.enumerator.assigner; + +import org.apache.flink.api.connector.source.SplitsAssignment; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; +import org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; +import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; +import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; +import org.apache.flink.util.TestLogger; + +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static java.util.Collections.emptyList; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonList; +import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_PARTITION_DISCOVERY_INTERVAL_MS; +import static org.apache.flink.connector.pulsar.source.enumerator.PulsarSourceEnumState.initialState; +import static org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor.defaultStopCursor; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test utils for split assigners. */ +abstract class SplitAssignerTestBase extends TestLogger { + + @Test + void registerTopicPartitionsWillOnlyReturnNewPartitions() { + T assigner = splitAssigner(true); + + Set partitions = createPartitions("persistent://public/default/a", 1); + List newPartitions = assigner.registerTopicPartitions(partitions); + assertThat(newPartitions) + .hasSize(1) + .first() + .hasFieldOrPropertyWithValue("topic", "persistent://public/default/a") + .hasFieldOrPropertyWithValue("partitionId", 1); + + newPartitions = assigner.registerTopicPartitions(partitions); + assertThat(newPartitions).isEmpty(); + + partitions = createPartitions("persistent://public/default/b", 2); + newPartitions = assigner.registerTopicPartitions(partitions); + assertThat(newPartitions) + .hasSize(1) + .hasSize(1) + .first() + .hasFieldOrPropertyWithValue("topic", "persistent://public/default/b") + .hasFieldOrPropertyWithValue("partitionId", 2); + } + + @Test + void noReadersProvideForAssignment() { + T assigner = splitAssigner(false); + assigner.registerTopicPartitions(createPartitions("c", 5)); + + Optional> assignment = + assigner.createAssignment(emptyList()); + assertThat(assignment).isNotPresent(); + } + + @Test + void noPartitionsProvideForAssignment() { + T assigner = splitAssigner(true); + Optional> assignment = + assigner.createAssignment(singletonList(4)); + assertThat(assignment).isNotPresent(); + } + + protected Set createPartitions(String topic, int partitionId) { + TopicPartition p1 = new TopicPartition(topic, partitionId, createFullRange()); + return singleton(p1); + } + + protected T splitAssigner(boolean discovery) { + Configuration configuration = new Configuration(); + + if (discovery) { + configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, 1000L); + } else { + configuration.set(PULSAR_PARTITION_DISCOVERY_INTERVAL_MS, -1L); + } + + SourceConfiguration sourceConfiguration = new SourceConfiguration(configuration); + return createAssigner(defaultStopCursor(), sourceConfiguration, initialState()); + } + + protected abstract T createAssigner( + StopCursor stopCursor, + SourceConfiguration sourceConfiguration, + PulsarSourceEnumState sourceEnumState); +} diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java index 02ef370da9a99..0632fa8f40294 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/enumerator/cursor/StopCursorTest.java @@ -22,7 +22,6 @@ import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; import org.apache.flink.connector.pulsar.source.config.SourceConfiguration; -import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils; import org.apache.flink.connector.pulsar.source.enumerator.topic.TopicPartition; import org.apache.flink.connector.pulsar.source.reader.split.PulsarOrderedPartitionSplitReader; import org.apache.flink.connector.pulsar.source.split.PulsarPartitionSplit; @@ -43,6 +42,7 @@ import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_SUBSCRIPTION_NAME; +import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicRange.createFullRange; import static org.assertj.core.api.Assertions.assertThat; @@ -64,7 +64,7 @@ void publishTimeStopCursor() throws IOException { // send the first message and set the stopCursor to filter any late stopCursor operator() .sendMessage( - TopicNameUtils.topicNameWithPartition(topicName, 0), + topicNameWithPartition(topicName, 0), Schema.STRING, randomAlphanumeric(10)); long currentTimeStamp = System.currentTimeMillis(); @@ -85,12 +85,11 @@ void publishTimeStopCursor() throws IOException { // send the second message and expect it will not be received operator() .sendMessage( - TopicNameUtils.topicNameWithPartition(topicName, 0), + topicNameWithPartition(topicName, 0), Schema.STRING, randomAlphanumeric(10)); RecordsWithSplitIds> secondResult = splitReader.fetch(); - assertThat(secondResult.nextSplit()).isNotNull(); - assertThat(firstResult.nextRecordFromSplit()).isNull(); + assertThat(secondResult.nextSplit()).isNull(); assertThat(secondResult.finishedSplits()).isNotEmpty(); } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java index f5c89fc2b6939..e5d2bf9276e40 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java @@ -85,7 +85,7 @@ TestOrderlinessExtension.class, TestLoggerExtension.class, }) -public abstract class PulsarPartitionSplitReaderTestBase extends PulsarTestSuiteBase { +abstract class PulsarPartitionSplitReaderTestBase extends PulsarTestSuiteBase { @RegisterExtension PulsarSplitReaderInvocationContextProvider provider = @@ -136,8 +136,7 @@ private void seekStartPositionAndHandleSplit( SplitsAddition addition = new SplitsAddition<>(singletonList(split)); // create consumer and seek before split changes - try (Consumer consumer = - (Consumer) reader.createPulsarConsumer(partition)) { + try (Consumer consumer = reader.createPulsarConsumer(partition)) { // inclusive messageId consumer.seek(startPosition); } catch (PulsarClientException e) { @@ -185,7 +184,7 @@ private List> fetchedMessages( if (verify) { assertThat(messages).as("We should fetch the expected size").hasSize(expectedCount); if (boundedness == Boundedness.CONTINUOUS_UNBOUNDED) { - assertThat(finishedSplits).as("Split should not be marked as finished").hasSize(0); + assertThat(finishedSplits).as("Split should not be marked as finished").isEmpty(); } else { assertThat(finishedSplits).as("Split should be marked as finished").hasSize(1); } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java index 4e1e42241d02c..53ce6bd02a2a5 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableOptionUtilsTest.java @@ -24,7 +24,7 @@ import org.apache.flink.connector.pulsar.source.enumerator.cursor.StartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.StopCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.MessageIdStartCursor; -import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.PublishTimestampStartCursor; +import org.apache.flink.connector.pulsar.source.enumerator.cursor.start.TimestampStartCursor; import org.apache.flink.connector.pulsar.source.enumerator.cursor.stop.PublishTimestampStopCursor; import org.apache.flink.connector.pulsar.table.testutils.MockTopicRouter; import org.apache.flink.table.api.DataTypes; @@ -187,7 +187,7 @@ void startCursor() { options.remove(SOURCE_START_FROM_MESSAGE_ID.key()); options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545"); startCursor = getStartCursor(Configuration.fromMap(options)); - assertThat(startCursor).isInstanceOf(PublishTimestampStartCursor.class); + assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "123545L"); assertThatExceptionOfType(IllegalArgumentException.class) @@ -228,7 +228,7 @@ void publishTimeStartCursor() { final Map options = createDefaultOptions(); options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345"); StartCursor startCursor = getStartCursor(Configuration.fromMap(options)); - assertThat(startCursor).isInstanceOf(PublishTimestampStartCursor.class); + assertThat(startCursor).isInstanceOf(TimestampStartCursor.class); options.put(SOURCE_START_FROM_PUBLISH_TIME.key(), "12345L"); assertThatExceptionOfType(IllegalArgumentException.class) diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java index 8587c1f34553d..618e879eeb577 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/testutils/runtime/PulsarRuntimeOperator.java @@ -32,13 +32,14 @@ import org.apache.pulsar.client.admin.PulsarAdminException; import org.apache.pulsar.client.admin.PulsarAdminException.NotFoundException; import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.ConsumerBuilder; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; -import org.apache.pulsar.client.api.SubscriptionInitialPosition; import org.apache.pulsar.client.api.TypedMessageBuilder; import org.apache.pulsar.client.api.transaction.TransactionCoordinatorClient; import org.apache.pulsar.client.api.transaction.TxnID; @@ -50,10 +51,8 @@ import java.io.IOException; import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Map; import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; @@ -63,9 +62,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.singletonList; import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static java.util.function.Function.identity; import static java.util.stream.Collectors.toList; -import static java.util.stream.Collectors.toMap; import static org.apache.commons.lang3.RandomStringUtils.randomAlphanumeric; import static org.apache.flink.connector.base.DeliveryGuarantee.EXACTLY_ONCE; import static org.apache.flink.connector.pulsar.common.config.PulsarOptions.PULSAR_ADMIN_URL; @@ -80,9 +77,11 @@ import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicName; import static org.apache.flink.connector.pulsar.source.enumerator.topic.TopicNameUtils.topicNameWithPartition; import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.pulsar.client.api.SubscriptionInitialPosition.Earliest; import static org.apache.pulsar.client.api.SubscriptionMode.Durable; import static org.apache.pulsar.client.api.SubscriptionType.Exclusive; import static org.apache.pulsar.common.naming.TopicDomain.persistent; +import static org.apache.pulsar.common.partition.PartitionedTopicMetadata.NON_PARTITIONED; /** * A pulsar cluster operator used for operating pulsar instance. It's serializable for using in @@ -209,12 +208,10 @@ public void increaseTopicPartitions(String topic, int newPartitionsNum) { PartitionedTopicMetadata metadata = sneakyAdmin(() -> admin().topics().getPartitionedTopicMetadata(topic)); checkArgument( - metadata.partitions <= newPartitionsNum, - "The new partition size which should exceed previous size."); + metadata.partitions < newPartitionsNum, + "The new partition size which should greater than previous size."); - if (metadata.partitions < newPartitionsNum) { - sneakyAdmin(() -> admin().topics().updatePartitionedTopic(topic, newPartitionsNum)); - } + sneakyAdmin(() -> admin().topics().updatePartitionedTopic(topic, newPartitionsNum)); } /** @@ -236,9 +233,11 @@ public void deleteTopic(String topic) { return; } + // Close all the available consumers and producers. removeConsumers(topic); removeProducers(topic); - if (metadata.partitions <= 0) { + + if (metadata.partitions == NON_PARTITIONED) { sneakyAdmin(() -> admin().topics().delete(topicName)); } else { sneakyAdmin(() -> admin().topics().deletePartitionedTopic(topicName)); @@ -261,52 +260,6 @@ public List topicInfo(String topic) { } } - /** - * Query a list of topics. Convert the topic metadata into a list of topic partitions. Return a - * mapping for topic and its partitions. - */ - public Map> topicsInfo(String... topics) { - return topicsInfo(Arrays.asList(topics)); - } - - /** - * Query a list of topics. Convert the topic metadata into a list of topic partitions. Return a - * mapping for topic and its partitions. - */ - public Map> topicsInfo(Collection topics) { - return topics.stream().collect(toMap(identity(), this::topicInfo)); - } - - /** - * Create a producer for a given topic and schema. - * - * @param topic The name of the topic. - * @param schema The schema for serialization. - * @param The type of the record. - * @return producer instance. - */ - @SuppressWarnings("unchecked") - public Producer createProducer(String topic, Schema schema) - throws PulsarClientException { - TopicName topicName = TopicName.get(topic); - String name = topicName.getPartitionedTopicName(); - int index = topicName.getPartitionIndex(); - ConcurrentHashMap> topicProducers = - producers.computeIfAbsent(name, d -> new ConcurrentHashMap<>()); - - return (Producer) - topicProducers.computeIfAbsent( - index, - i -> { - try { - return client().newProducer(schema).topic(topic).create(); - } catch (PulsarClientException e) { - sneakyThrow(e); - return null; - } - }); - } - /** * Send a single message to Pulsar, return the message id after the ack from Pulsar. * @@ -537,6 +490,29 @@ public void close() throws IOException { // --------------------------- Private Methods ----------------------------- + @SuppressWarnings("unchecked") + public Producer createProducer(String topic, Schema schema) + throws PulsarClientException { + TopicName topicName = TopicName.get(topic); + String name = topicName.getPartitionedTopicName(); + int index = topicName.getPartitionIndex(); + ConcurrentHashMap> topicProducers = + producers.computeIfAbsent(name, d -> new ConcurrentHashMap<>()); + + return (Producer) + topicProducers.computeIfAbsent( + index, + i -> { + ProducerBuilder builder = + client().newProducer(schema) + .topic(topic) + .enableBatching(false) + .enableMultiSchema(true); + + return sneakyClient(builder::create); + }); + } + @SuppressWarnings("unchecked") private Consumer createConsumer(String topic, Schema schema) throws PulsarClientException { @@ -550,19 +526,15 @@ private Consumer createConsumer(String topic, Schema schema) topicConsumers.computeIfAbsent( index, i -> { - try { - return client().newConsumer(schema) - .topic(topic) - .subscriptionName(SUBSCRIPTION_NAME) - .subscriptionMode(Durable) - .subscriptionType(Exclusive) - .subscriptionInitialPosition( - SubscriptionInitialPosition.Earliest) - .subscribe(); - } catch (PulsarClientException e) { - sneakyThrow(e); - return null; - } + ConsumerBuilder builder = + client().newConsumer(schema) + .topic(topic) + .subscriptionName(SUBSCRIPTION_NAME) + .subscriptionMode(Durable) + .subscriptionType(Exclusive) + .subscriptionInitialPosition(Earliest); + + return sneakyClient(builder::subscribe); }); } @@ -571,11 +543,7 @@ private void removeProducers(String topic) { ConcurrentHashMap> integerProducers = producers.remove(topicName); if (integerProducers != null) { for (Producer producer : integerProducers.values()) { - try { - producer.close(); - } catch (PulsarClientException e) { - sneakyThrow(e); - } + sneakyClient(producer::close); } } } @@ -585,11 +553,7 @@ private void removeConsumers(String topic) { ConcurrentHashMap> integerConsumers = consumers.remove(topicName); if (integerConsumers != null) { for (Consumer consumer : integerConsumers.values()) { - try { - consumer.close(); - } catch (PulsarClientException e) { - sneakyThrow(e); - } + sneakyClient(consumer::close); } } } From 64136610fe6482066ed68e1ab2ff5bd4e91e0cb6 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Fri, 26 Aug 2022 21:47:33 +0800 Subject: [PATCH 251/258] Backport FLINK-27917 and drop consumer seek method. (#204) --- .../PulsarPartitionSplitReaderTestBase.java | 37 +++++++++++++------ 1 file changed, 26 insertions(+), 11 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java index e5d2bf9276e40..96055cff72e2d 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/source/reader/split/PulsarPartitionSplitReaderTestBase.java @@ -30,10 +30,9 @@ import org.apache.flink.connector.pulsar.testutils.extension.TestOrderlinessExtension; import org.apache.flink.util.TestLoggerExtension; -import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.admin.PulsarAdmin; import org.apache.pulsar.client.api.Message; import org.apache.pulsar.client.api.MessageId; -import org.apache.pulsar.client.api.PulsarClientException; import org.apache.pulsar.client.api.Schema; import org.apache.pulsar.client.api.SubscriptionType; import org.apache.pulsar.client.impl.MessageIdImpl; @@ -61,7 +60,6 @@ import static java.util.Collections.singletonList; import static org.apache.commons.lang3.RandomStringUtils.randomAlphabetic; import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyAdmin; -import static org.apache.flink.connector.pulsar.common.utils.PulsarExceptionUtils.sneakyThrow; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_DEFAULT_FETCH_TIME; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_ENABLE_AUTO_ACKNOWLEDGE_MESSAGE; import static org.apache.flink.connector.pulsar.source.PulsarSourceOptions.PULSAR_MAX_FETCH_RECORDS; @@ -135,14 +133,31 @@ private void seekStartPositionAndHandleSplit( new PulsarPartitionSplit(partition, StopCursor.never(), null, null); SplitsAddition addition = new SplitsAddition<>(singletonList(split)); - // create consumer and seek before split changes - try (Consumer consumer = reader.createPulsarConsumer(partition)) { - // inclusive messageId - consumer.seek(startPosition); - } catch (PulsarClientException e) { - sneakyThrow(e); + // Create the subscription and set the start position for this reader. + // Remember not to use Consumer.seek(startPosition) + SourceConfiguration sourceConfiguration = reader.sourceConfiguration; + PulsarAdmin pulsarAdmin = reader.pulsarAdmin; + String subscriptionName = sourceConfiguration.getSubscriptionName(); + List subscriptions = + sneakyAdmin(() -> pulsarAdmin.topics().getSubscriptions(topicName)); + if (!subscriptions.contains(subscriptionName)) { + // If this subscription is not available. Just create it. + sneakyAdmin( + () -> + pulsarAdmin + .topics() + .createSubscription( + topicName, subscriptionName, startPosition)); + } else { + // Reset the subscription if this is existed. + sneakyAdmin( + () -> + pulsarAdmin + .topics() + .resetCursor(topicName, subscriptionName, startPosition)); } + // Accept the split and start consuming. reader.handleSplitsChanges(addition); } @@ -199,7 +214,7 @@ void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) String topicName = randomAlphabetic(10); // Add a split - seekStartPositionAndHandleSplit(splitReader, topicName, 0); + handleSplit(splitReader, topicName, 0); // Poll once with a null message Message message1 = fetchedMessage(splitReader); @@ -223,7 +238,7 @@ void pollMessageAfterTimeout(PulsarPartitionSplitReaderBase splitReader) void consumeMessageCreatedAfterHandleSplitChangesAndFetch( PulsarPartitionSplitReaderBase splitReader) { String topicName = randomAlphabetic(10); - seekStartPositionAndHandleSplit(splitReader, topicName, 0); + handleSplit(splitReader, topicName, 0); operator().sendMessage(topicNameWithPartition(topicName, 0), STRING, randomAlphabetic(10)); fetchedMessages(splitReader, 1, true); } From 370be48e4c21fb280b444417c716d8df507c8e40 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Fri, 26 Aug 2022 21:54:39 +0800 Subject: [PATCH 252/258] [FLINK-28960][Connector/Pulsar] Add jaxb-api back to flink-connector-pulsar (#205) --- flink-connectors/flink-connector-pulsar/pom.xml | 4 ---- 1 file changed, 4 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/pom.xml b/flink-connectors/flink-connector-pulsar/pom.xml index b452d0f7c3459..d308c95451c6b 100644 --- a/flink-connectors/flink-connector-pulsar/pom.xml +++ b/flink-connectors/flink-connector-pulsar/pom.xml @@ -267,10 +267,6 @@ under the License. javax.validation validation-api - - javax.xml.bind - jaxb-api - net.jcip jcip-annotations From 629e7ac00e951648915a15168aee6ef3b8795692 Mon Sep 17 00:00:00 2001 From: Yufan Sheng Date: Thu, 1 Sep 2022 15:00:24 +0800 Subject: [PATCH 253/258] PulsarSchema: fix the byte array serialization issues. (#207) --- .../pulsar/common/schema/PulsarSchema.java | 99 ++++++++++++------- .../pulsar/common/utils/PulsarSerdeUtils.java | 5 +- 2 files changed, 67 insertions(+), 37 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java index 5fd176e73267b..0d0d11497b793 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/schema/PulsarSchema.java @@ -27,8 +27,12 @@ import org.apache.pulsar.common.schema.SchemaInfo; import org.apache.pulsar.common.schema.SchemaType; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Arrays; +import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -57,11 +61,6 @@ public final class PulsarSchema implements Serializable { private transient Schema schema; private transient SchemaInfo schemaInfo; - private String schemaName; - private byte[] schemaBytes; - private SchemaType schemaType; - private Map schemaProperties; - /** Create serializable pulsar schema for primitive types. */ public PulsarSchema(Schema schema) { SchemaInfo info = schema.getSchemaInfo(); @@ -78,7 +77,8 @@ public PulsarSchema(Schema schema) { // Primitive type information could be reflected from the schema class. Class typeClass = getTemplateType1(schema.getClass()); - setSchemaInfo(encodeClassInfo(info, typeClass)); + this.schemaInfo = encodeClassInfo(info, typeClass); + this.schema = createSchema(schemaInfo); } /** @@ -94,7 +94,8 @@ public PulsarSchema(Schema schema, Class typeClass) { "Key Value Schema should provide the type classes of key and value"); validateSchemaInfo(info); - setSchemaInfo(encodeClassInfo(info, typeClass)); + this.schemaInfo = encodeClassInfo(info, typeClass); + this.schema = createSchema(schemaInfo); } /** Create serializable pulsar schema for key value type. */ @@ -117,37 +118,66 @@ public PulsarSchema( SchemaInfo encodedInfo = encodeKeyValueSchemaInfo(info.getName(), infoKey, infoValue, encodingType); - setSchemaInfo(encodeClassInfo(encodedInfo, KeyValue.class)); - } - - /** Validate the schema for having the required class info. */ - private void setSchemaInfo(SchemaInfo schemaInfo) { - this.schema = createSchema(schemaInfo); - this.schemaInfo = schemaInfo; - - this.schemaName = schemaInfo.getName(); - this.schemaBytes = schemaInfo.getSchema(); - this.schemaType = schemaInfo.getType(); - this.schemaProperties = schemaInfo.getProperties(); + this.schemaInfo = encodeClassInfo(encodedInfo, KeyValue.class); + this.schema = createSchema(this.schemaInfo); } public Schema getPulsarSchema() { - if (schema == null) { - this.schema = createSchema(getSchemaInfo()); - } return schema; } public SchemaInfo getSchemaInfo() { - if (schemaInfo == null) { - this.schemaInfo = - new SchemaInfoImpl(schemaName, schemaBytes, schemaType, schemaProperties); - } return schemaInfo; } public Class getRecordClass() { - return decodeClassInfo(getSchemaInfo()); + return decodeClassInfo(schemaInfo); + } + + private void writeObject(ObjectOutputStream oos) throws IOException { + // Name + oos.writeUTF(schemaInfo.getName()); + + // Schema + byte[] schemaBytes = schemaInfo.getSchema(); + oos.writeInt(schemaBytes.length); + oos.write(schemaBytes); + + // Type + SchemaType type = schemaInfo.getType(); + oos.writeInt(type.getValue()); + + // Properties + Map properties = schemaInfo.getProperties(); + oos.writeInt(properties.size()); + for (Map.Entry entry : properties.entrySet()) { + oos.writeUTF(entry.getKey()); + oos.writeUTF(entry.getValue()); + } + } + + private void readObject(ObjectInputStream ois) throws ClassNotFoundException, IOException { + // Name + String name = ois.readUTF(); + + // Schema + int byteLen = ois.readInt(); + byte[] schemaBytes = new byte[byteLen]; + ois.readFully(schemaBytes); + + // Type + int typeIdx = ois.readInt(); + SchemaType type = SchemaType.valueOf(typeIdx); + + // Properties + int propSize = ois.readInt(); + Map properties = new HashMap<>(propSize); + for (int i = 0; i < propSize; i++) { + properties.put(ois.readUTF(), ois.readUTF()); + } + + this.schemaInfo = new SchemaInfoImpl(name, schemaBytes, type, properties); + this.schema = createSchema(schemaInfo); } @Override @@ -158,21 +188,24 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - PulsarSchema that = ((PulsarSchema) o); + SchemaInfo that = ((PulsarSchema) o).getPulsarSchema().getSchemaInfo(); - return Objects.equals(schemaType, that.schemaType) - && Arrays.equals(schemaBytes, that.schemaBytes) - && Objects.equals(schemaProperties, that.schemaProperties); + return Objects.equals(schemaInfo.getType(), that.getType()) + && Arrays.equals(schemaInfo.getSchema(), that.getSchema()) + && Objects.equals(schemaInfo.getProperties(), that.getProperties()); } @Override public int hashCode() { - return Objects.hash(schemaType, Arrays.hashCode(schemaBytes), schemaProperties); + return Objects.hash( + schemaInfo.getType(), + Arrays.hashCode(schemaInfo.getSchema()), + schemaInfo.getProperties()); } @Override public String toString() { - return getSchemaInfo().toString(); + return schemaInfo.toString(); } /** diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java index 9f64172a504ef..93609bc720d69 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/common/utils/PulsarSerdeUtils.java @@ -50,10 +50,7 @@ public static void serializeBytes(DataOutputStream out, byte[] bytes) throws IOE public static byte[] deserializeBytes(DataInputStream in) throws IOException { int size = in.readInt(); byte[] bytes = new byte[size]; - int result = in.read(bytes); - if (result < 0) { - throw new IOException("Couldn't deserialize the object, wrong byte buffer."); - } + in.readFully(bytes); return bytes; } From 6306d40112233556d071d997ad0627cecb12ed8e Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Wed, 7 Sep 2022 12:46:02 +0800 Subject: [PATCH 254/258] use native_avro schema and retrieve avro format from Flink AvroSerializationSchema --- .../sink/PulsarTableSerializationSchema.java | 21 ++++++++++++++++++- .../pulsar/table/PulsarTableITCase.java | 7 +++---- .../avro/AvroRowDataSerializationSchema.java | 5 +++++ 3 files changed, 28 insertions(+), 5 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java index 0f996193f160f..56126d68cf00d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java @@ -14,12 +14,16 @@ package org.apache.flink.connector.pulsar.table.sink; +import org.apache.avro.generic.GenericRecord; + import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessage; import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; +import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; +import org.apache.flink.formats.avro.AvroSerializationSchema; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.types.RowKind; @@ -51,6 +55,8 @@ public class PulsarTableSerializationSchema implements PulsarSerializationSchema private final PulsarWritableMetadata writableMetadata; +// private final Schema pulsarSchema; + public PulsarTableSerializationSchema( @Nullable SerializationSchema keySerialization, RowData.FieldGetter[] keyFieldGetters, @@ -62,6 +68,7 @@ public PulsarTableSerializationSchema( this.valueSerialization = checkNotNull(valueSerialization); this.valueFieldGetters = checkNotNull(valueFieldGetters); this.writableMetadata = checkNotNull(writableMetadata); +// this.pulsarSchema = getPulsarSchemaFromSerialization(valueSerialization); } @Override @@ -89,7 +96,8 @@ public PulsarMessage serialize(RowData consumedRow, PulsarSinkContext sinkCon } byte[] serializedData = valueSerialization.serialize(valueRow); - messageBuilder.value(Schema.BYTES, serializedData); +// messageBuilder.value(Schema.BYTES, serializedData); + messageBuilder.value(getPulsarSchemaFromSerialization(valueSerialization), serializedData); return messageBuilder.build(); } @@ -107,4 +115,15 @@ private static RowData createProjectedRow( public interface MetadataConverter extends Serializable { Object read(RowData consumedRow, int pos); } + + // TODO need to make the schemas serializable + private static Schema getPulsarSchemaFromSerialization(SerializationSchema serializationSchema) { + if (serializationSchema instanceof AvroRowDataSerializationSchema) { + SerializationSchema nestedSchema = ((AvroRowDataSerializationSchema) serializationSchema).getNestedSchema(); + org.apache.avro.Schema avroSchema = ((AvroSerializationSchema) nestedSchema).getSchema(); + return Schema.NATIVE_AVRO(avroSchema); + } else { + return Schema.BYTES; + } + } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java index 18992a17fbc60..d5d166a31e898 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -638,9 +638,8 @@ private static Stream provideSchemaData() { private static Stream provideAvroBasedSchemaData() { return Stream.of( - Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class), createRandomUser()) - // Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class), - // createRandomUser()) - ); +// Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class), createRandomUser()) + Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class), createRandomUser()) + ); } } diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java index a48c8c8c012fe..385533ec730ee 100644 --- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java @@ -103,6 +103,11 @@ public boolean equals(Object o) { return nestedSchema.equals(that.nestedSchema) && rowType.equals(that.rowType); } + // TODO protect this schema to make sure it's not modified + public SerializationSchema getNestedSchema() { + return nestedSchema; + } + @Override public int hashCode() { return Objects.hash(nestedSchema, rowType); From d4559e043dbf53ad2a85fc8b12a2ad3ad1ef75b8 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Wed, 7 Sep 2022 12:55:47 +0800 Subject: [PATCH 255/258] fix 126 as well --- .../sink/PulsarTableSerializationSchema.java | 24 +++++++++++-------- .../pulsar/table/PulsarTableITCase.java | 5 ++-- .../table/catalog/PulsarCatalogITTest.java | 4 +--- .../avro/AvroRowDataSerializationSchema.java | 9 ++++--- 4 files changed, 21 insertions(+), 21 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java index 56126d68cf00d..d1346e3e2e3a8 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java @@ -14,8 +14,6 @@ package org.apache.flink.connector.pulsar.table.sink; -import org.apache.avro.generic.GenericRecord; - import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; @@ -28,12 +26,14 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.RowKind; +import org.apache.avro.generic.GenericRecord; import org.apache.pulsar.client.api.Schema; import javax.annotation.Nullable; import java.io.Serializable; +import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -55,8 +55,6 @@ public class PulsarTableSerializationSchema implements PulsarSerializationSchema private final PulsarWritableMetadata writableMetadata; -// private final Schema pulsarSchema; - public PulsarTableSerializationSchema( @Nullable SerializationSchema keySerialization, RowData.FieldGetter[] keyFieldGetters, @@ -68,7 +66,6 @@ public PulsarTableSerializationSchema( this.valueSerialization = checkNotNull(valueSerialization); this.valueFieldGetters = checkNotNull(valueFieldGetters); this.writableMetadata = checkNotNull(writableMetadata); -// this.pulsarSchema = getPulsarSchemaFromSerialization(valueSerialization); } @Override @@ -96,7 +93,9 @@ public PulsarMessage serialize(RowData consumedRow, PulsarSinkContext sinkCon } byte[] serializedData = valueSerialization.serialize(valueRow); -// messageBuilder.value(Schema.BYTES, serializedData); + + // TODO, getting the schema everytime send message introduce an overhead, + // create a transient Schema field and set the Schema value in open(). messageBuilder.value(getPulsarSchemaFromSerialization(valueSerialization), serializedData); return messageBuilder.build(); } @@ -116,11 +115,16 @@ public interface MetadataConverter extends Serializable { Object read(RowData consumedRow, int pos); } - // TODO need to make the schemas serializable - private static Schema getPulsarSchemaFromSerialization(SerializationSchema serializationSchema) { + private static Schema getPulsarSchemaFromSerialization( + SerializationSchema serializationSchema) { if (serializationSchema instanceof AvroRowDataSerializationSchema) { - SerializationSchema nestedSchema = ((AvroRowDataSerializationSchema) serializationSchema).getNestedSchema(); - org.apache.avro.Schema avroSchema = ((AvroSerializationSchema) nestedSchema).getSchema(); + SerializationSchema nestedSchema = + ((AvroRowDataSerializationSchema) serializationSchema).getNestedSchema(); + checkArgument( + nestedSchema instanceof AvroSerializationSchema, + "Expect to see the nested schema is AvroSerializationSchema"); + org.apache.avro.Schema avroSchema = + ((AvroSerializationSchema) nestedSchema).getSchema(); return Schema.NATIVE_AVRO(avroSchema); } else { return Schema.BYTES; diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java index d5d166a31e898..5dfe450195ab6 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/PulsarTableITCase.java @@ -638,8 +638,7 @@ private static Stream provideSchemaData() { private static Stream provideAvroBasedSchemaData() { return Stream.of( -// Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class), createRandomUser()) - Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class), createRandomUser()) - ); + Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class), createRandomUser()), + Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class), createRandomUser())); } } diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java index e1be82c5ef75d..6e70ee939c1ba 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java @@ -32,7 +32,6 @@ import org.apache.pulsar.common.policies.data.TenantInfo; import org.apache.pulsar.shade.org.apache.commons.lang3.RandomStringUtils; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -719,7 +718,6 @@ void writeToExplicitTableAndReadWithAvroBasedSchema(String format, Schema schema) throws Exception { String databaseName = newDatabaseName(); @@ -753,7 +751,7 @@ void writeToExplicitTableAndReadWithAvroBasedSchemaUsingPulsarConsumer( + " ('oid1', 10),\n" + " ('oid2', 20),\n" + " ('oid3', 30),\n" - + " ('oid4', 10)", + + " ('oid4', 40)", tableSinkName); tableEnv.executeSql(insertQ).await(); diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java index 385533ec730ee..09c3505636de6 100644 --- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java @@ -74,6 +74,10 @@ public AvroRowDataSerializationSchema( this.runtimeConverter = runtimeConverter; } + public SerializationSchema getNestedSchema() { + return nestedSchema; + } + @Override public void open(InitializationContext context) throws Exception { this.schema = AvroSchemaConverter.convertToSchema(rowType); @@ -103,11 +107,6 @@ public boolean equals(Object o) { return nestedSchema.equals(that.nestedSchema) && rowType.equals(that.rowType); } - // TODO protect this schema to make sure it's not modified - public SerializationSchema getNestedSchema() { - return nestedSchema; - } - @Override public int hashCode() { return Objects.hash(nestedSchema, rowType); From c60ab0fe2f5de6cd303d941d29fb57ab7e96e61c Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Wed, 7 Sep 2022 21:40:07 +0800 Subject: [PATCH 256/258] newest update --- .../sink/PulsarTableSerializationSchema.java | 20 +++++++++---------- ...PulsarTableSerializationSchemaFactory.java | 5 ++++- .../table/catalog/PulsarCatalogITTest.java | 5 ++--- .../avro/AvroRowDataSerializationSchema.java | 4 ---- 4 files changed, 16 insertions(+), 18 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java index d1346e3e2e3a8..7a478b2b37bab 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java @@ -22,8 +22,10 @@ import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; import org.apache.flink.formats.avro.AvroSerializationSchema; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.RowKind; import org.apache.avro.generic.GenericRecord; @@ -55,17 +57,21 @@ public class PulsarTableSerializationSchema implements PulsarSerializationSchema private final PulsarWritableMetadata writableMetadata; + private final RowType rowType; + public PulsarTableSerializationSchema( @Nullable SerializationSchema keySerialization, RowData.FieldGetter[] keyFieldGetters, SerializationSchema valueSerialization, RowData.FieldGetter[] valueFieldGetters, - PulsarWritableMetadata writableMetadata) { + PulsarWritableMetadata writableMetadata, + RowType rowType) { this.keySerialization = keySerialization; this.keyFieldGetters = checkNotNull(keyFieldGetters); this.valueSerialization = checkNotNull(valueSerialization); this.valueFieldGetters = checkNotNull(valueFieldGetters); this.writableMetadata = checkNotNull(writableMetadata); + this.rowType = checkNotNull(rowType); } @Override @@ -96,7 +102,7 @@ public PulsarMessage serialize(RowData consumedRow, PulsarSinkContext sinkCon // TODO, getting the schema everytime send message introduce an overhead, // create a transient Schema field and set the Schema value in open(). - messageBuilder.value(getPulsarSchemaFromSerialization(valueSerialization), serializedData); + messageBuilder.value(Schema.BYTES, serializedData); return messageBuilder.build(); } @@ -115,16 +121,10 @@ public interface MetadataConverter extends Serializable { Object read(RowData consumedRow, int pos); } - private static Schema getPulsarSchemaFromSerialization( + private Schema getPulsarSchemaFromSerialization( SerializationSchema serializationSchema) { if (serializationSchema instanceof AvroRowDataSerializationSchema) { - SerializationSchema nestedSchema = - ((AvroRowDataSerializationSchema) serializationSchema).getNestedSchema(); - checkArgument( - nestedSchema instanceof AvroSerializationSchema, - "Expect to see the nested schema is AvroSerializationSchema"); - org.apache.avro.Schema avroSchema = - ((AvroSerializationSchema) nestedSchema).getSchema(); + org.apache.avro.Schema avroSchema = AvroSchemaConverter.convertToSchema(rowType); return Schema.NATIVE_AVRO(avroSchema); } else { return Schema.BYTES; diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java index fd3f7c252489f..a2db75424020d 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchemaFactory.java @@ -22,6 +22,7 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.utils.DataTypeUtils; import javax.annotation.Nullable; @@ -84,12 +85,14 @@ public PulsarSerializationSchema createPulsarSerializationSchema( final PulsarWritableMetadata writableMetadata = new PulsarWritableMetadata(writableMetadataKeys, physicalChildren.size()); + final RowType rowType = (RowType) physicalDataType.getLogicalType(); return new PulsarTableSerializationSchema( keySerialization, keyFieldGetters, valueSerialization, valueFieldGetters, - writableMetadata); + writableMetadata, + rowType); } private @Nullable SerializationSchema createSerialization( diff --git a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java index 6e70ee939c1ba..8579f87886580 100644 --- a/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java +++ b/flink-connectors/flink-connector-pulsar/src/test/java/org/apache/flink/connector/pulsar/table/catalog/PulsarCatalogITTest.java @@ -922,8 +922,7 @@ private String newTopicName() { private static Stream provideAvroBasedSchemaData() { return Stream.of( - Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class)) - // Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class)) - ); + Arguments.of(JSON_FORMAT, Schema.JSON(TestingUser.class)), + Arguments.of(AVRO_FORMAT, Schema.AVRO(TestingUser.class))); } } diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java index 09c3505636de6..a48c8c8c012fe 100644 --- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java +++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/AvroRowDataSerializationSchema.java @@ -74,10 +74,6 @@ public AvroRowDataSerializationSchema( this.runtimeConverter = runtimeConverter; } - public SerializationSchema getNestedSchema() { - return nestedSchema; - } - @Override public void open(InitializationContext context) throws Exception { this.schema = AvroSchemaConverter.convertToSchema(rowType); From febab4499ec6b29805834dbcf5cb5867461ea5e9 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Wed, 7 Sep 2022 22:09:20 +0800 Subject: [PATCH 257/258] WIP --- .../sink/PulsarTableSerializationSchema.java | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java index 7a478b2b37bab..00bafd3cf39af 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java @@ -15,27 +15,28 @@ package org.apache.flink.connector.pulsar.table.sink; import org.apache.flink.api.common.serialization.SerializationSchema; +import org.apache.flink.connector.pulsar.common.schema.BytesSchema; +import org.apache.flink.connector.pulsar.common.schema.PulsarSchema; import org.apache.flink.connector.pulsar.sink.config.SinkConfiguration; import org.apache.flink.connector.pulsar.sink.writer.context.PulsarSinkContext; import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessage; import org.apache.flink.connector.pulsar.sink.writer.message.PulsarMessageBuilder; import org.apache.flink.connector.pulsar.sink.writer.serializer.PulsarSerializationSchema; import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; -import org.apache.flink.formats.avro.AvroSerializationSchema; import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.json.JsonRowDataSerializationSchema; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.RowKind; -import org.apache.avro.generic.GenericRecord; import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.api.schema.SchemaDefinition; import javax.annotation.Nullable; import java.io.Serializable; -import static org.apache.flink.util.Preconditions.checkArgument; import static org.apache.flink.util.Preconditions.checkNotNull; /** @@ -102,7 +103,7 @@ public PulsarMessage serialize(RowData consumedRow, PulsarSinkContext sinkCon // TODO, getting the schema everytime send message introduce an overhead, // create a transient Schema field and set the Schema value in open(). - messageBuilder.value(Schema.BYTES, serializedData); + messageBuilder.value(getPulsarSchemaFromSerialization(), serializedData); return messageBuilder.build(); } @@ -121,11 +122,13 @@ public interface MetadataConverter extends Serializable { Object read(RowData consumedRow, int pos); } - private Schema getPulsarSchemaFromSerialization( - SerializationSchema serializationSchema) { - if (serializationSchema instanceof AvroRowDataSerializationSchema) { + private Schema getPulsarSchemaFromSerialization() { + if (valueSerialization instanceof AvroRowDataSerializationSchema) { org.apache.avro.Schema avroSchema = AvroSchemaConverter.convertToSchema(rowType); return Schema.NATIVE_AVRO(avroSchema); + } else if (valueSerialization instanceof JsonRowDataSerializationSchema) { + return new BytesSchema( + new PulsarSchema(Schema.JSON(SchemaDefinition.builder().build()))); } else { return Schema.BYTES; } From 39280fbd790fcb51d8ca03d044d9884ff60daf06 Mon Sep 17 00:00:00 2001 From: Yufei Zhang Date: Wed, 7 Sep 2022 22:42:41 +0800 Subject: [PATCH 258/258] add support for RAW formats --- .../sink/PulsarTableSerializationSchema.java | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java index 00bafd3cf39af..cc54cc2773454 100644 --- a/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java +++ b/flink-connectors/flink-connector-pulsar/src/main/java/org/apache/flink/connector/pulsar/table/sink/PulsarTableSerializationSchema.java @@ -25,8 +25,11 @@ import org.apache.flink.formats.avro.AvroRowDataSerializationSchema; import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; import org.apache.flink.formats.json.JsonRowDataSerializationSchema; +import org.apache.flink.formats.raw.RawFormatSerializationSchema; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RawType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.types.RowKind; @@ -129,8 +132,40 @@ private Schema getPulsarSchemaFromSerialization() { } else if (valueSerialization instanceof JsonRowDataSerializationSchema) { return new BytesSchema( new PulsarSchema(Schema.JSON(SchemaDefinition.builder().build()))); + } else if (valueSerialization instanceof RawFormatSerializationSchema) { + return mapFromFlinkTypeToPulsarSchema(rowType); } else { return Schema.BYTES; } } + + private Schema mapFromFlinkTypeToPulsarSchema( + LogicalType type) { + switch (type.getTypeRoot()) { + case CHAR: + case VARCHAR: + return Schema.STRING; + case VARBINARY: + case BINARY: + case RAW: + return Schema.BYTES; + case BOOLEAN: + return Schema.BOOL; + case TINYINT: + return Schema.INT8; + case SMALLINT: + return Schema.INT16; + case INTEGER: + return Schema.INT32; + case BIGINT: + return Schema.INT64; + case FLOAT: + return Schema.FLOAT; + case DOUBLE: + return Schema.DOUBLE; + default: + throw new UnsupportedOperationException( + String.format("cannot map %s type to a pulsar primitive schema ", type)); + } + } }