From 5b27418ad54b0f3f38431c6410806bc9542a46a3 Mon Sep 17 00:00:00 2001 From: Wink <32723967+aiwenmo@users.noreply.github.com> Date: Sun, 17 Nov 2024 17:58:48 +0800 Subject: [PATCH] [Optimization-3917][core] Optimize explain and add test (#3918) --- .gitignore | 1 + .../main/java/org/dinky/data/dto/TaskDTO.java | 2 +- .../service/impl/PrintTableServiceImpl.java | 2 +- .../dinky/service/impl/StudioServiceImpl.java | 2 +- .../dinky/service/impl/TaskServiceImpl.java | 2 +- .../org/dinky/app/flinksql/Submitter.java | 2 +- .../org/dinky/app/model/StatementParam.java | 2 +- .../executor/CustomTableEnvironmentImpl.java | 203 +++++++--- .../executor/CustomTableEnvironmentImpl.java | 189 ++++++++-- .../executor/CustomTableEnvironmentImpl.java | 182 +++++++-- .../AbstractCustomTableEnvironment.java | 38 -- .../executor/CustomTableEnvironmentImpl.java | 179 ++++++++- .../AbstractCustomTableEnvironment.java | 38 -- .../executor/CustomTableEnvironmentImpl.java | 275 +++++++++++++- .../AbstractCustomTableEnvironment.java | 38 -- .../executor/CustomTableEnvironmentImpl.java | 263 ++++++++++++- .../AbstractCustomTableEnvironment.java | 38 -- .../executor/CustomTableEnvironmentImpl.java | 263 ++++++++++++- .../executor/CustomTableEnvironment.java | 12 +- .../java/org/dinky/data/job/JobStatement.java | 92 +++++ .../org/dinky/data/job/JobStatementType.java | 27 ++ .../java/org/dinky/data/job}/SqlCategory.java | 2 +- .../java/org/dinky/data/job}/SqlType.java | 40 +- .../dinky/data/model/SystemConfiguration.java | 3 +- .../flink/FlinkPipelineExecution.java | 62 ---- .../org/dinky/data/result/ResultBuilder.java | 2 +- .../java/org/dinky/executor/Executor.java | 19 +- .../java/org/dinky/explainer/Explainer.java | 186 ++++------ .../explainer/lineage/LineageBuilder.java | 22 +- .../mock/MockStatementExplainer.java | 89 +++-- .../print_table/PrintStatementExplainer.java | 9 + .../main/java/org/dinky/job/JobConfig.java | 8 +- .../main/java/org/dinky/job/JobManager.java | 59 ++- .../main/java/org/dinky/job/JobRunner.java | 37 ++ .../java/org/dinky/job/JobRunnerFactory.java | 59 +++ .../java/org/dinky/job/JobStatementPlan.java | 85 +++++ .../java/org/dinky/job/StatementParam.java | 2 +- .../org/dinky/job/builder/JobDDLBuilder.java | 38 +- .../dinky/job/builder/JobExecuteBuilder.java | 6 +- .../job/builder/JobJarStreamGraphBuilder.java | 2 +- .../dinky/job/builder/JobTransBuilder.java | 15 +- .../dinky/job/runner/AbstractJobRunner.java | 81 ++++ .../org/dinky/job/runner/JobDDLRunner.java | 297 +++++++++++++++ .../org/dinky/job/runner/JobJarRunner.java | 231 ++++++++++++ .../dinky/job/runner/JobPipelineRunner.java | 210 +++++++++++ .../org/dinky/job/runner/JobSetRunner.java | 86 +++++ .../org/dinky/job/runner/JobSqlRunner.java | 346 ++++++++++++++++++ .../main/java/org/dinky/trans/Operations.java | 2 +- .../java/org/dinky/core/JobManagerTest.java | 78 ---- .../java/org/dinky/job/JobManagerTest.java | 214 +++++++++++ .../java/org/dinky/job/JobTestHandler.java | 67 ++++ .../java/org/dinky/parse/SqlTypeTest.java | 2 +- .../services/org.dinky.job.JobHandler | 1 + .../resources/flink/sql/single-insert.sql | 26 ++ .../flink/sql/statement-set-batch.sql | 98 +++++ .../flink/sql/statement-set-stream.sql | 98 +++++ .../src/test/resources/flink/sql/variable.sql | 27 ++ dinky-flink/dinky-flink-1.20/pom.xml | 31 +- .../SqlTask/TaskConfig/BasicConfig.tsx | 3 + .../TaskConfig/ProFormFlinkUdfConfig.tsx | 17 +- .../SqlTask/TaskConfig/index.tsx | 2 + .../CenterTabContent/SqlTask/index.tsx | 21 +- .../Toolbar/Service/Result/index.tsx | 3 +- 63 files changed, 3818 insertions(+), 718 deletions(-) create mode 100644 dinky-common/src/main/java/org/dinky/data/job/JobStatement.java create mode 100644 dinky-common/src/main/java/org/dinky/data/job/JobStatementType.java rename {dinky-client/dinky-client-base/src/main/java/org/dinky/parser => dinky-common/src/main/java/org/dinky/data/job}/SqlCategory.java (97%) rename {dinky-client/dinky-client-base/src/main/java/org/dinky/parser => dinky-common/src/main/java/org/dinky/data/job}/SqlType.java (73%) delete mode 100644 dinky-core/src/main/java/com/ververica/cdc/composer/flink/FlinkPipelineExecution.java create mode 100644 dinky-core/src/main/java/org/dinky/job/JobRunner.java create mode 100644 dinky-core/src/main/java/org/dinky/job/JobRunnerFactory.java create mode 100644 dinky-core/src/main/java/org/dinky/job/JobStatementPlan.java create mode 100644 dinky-core/src/main/java/org/dinky/job/runner/AbstractJobRunner.java create mode 100644 dinky-core/src/main/java/org/dinky/job/runner/JobDDLRunner.java create mode 100644 dinky-core/src/main/java/org/dinky/job/runner/JobJarRunner.java create mode 100644 dinky-core/src/main/java/org/dinky/job/runner/JobPipelineRunner.java create mode 100644 dinky-core/src/main/java/org/dinky/job/runner/JobSetRunner.java create mode 100644 dinky-core/src/main/java/org/dinky/job/runner/JobSqlRunner.java delete mode 100644 dinky-core/src/test/java/org/dinky/core/JobManagerTest.java create mode 100644 dinky-core/src/test/java/org/dinky/job/JobManagerTest.java create mode 100644 dinky-core/src/test/java/org/dinky/job/JobTestHandler.java create mode 100644 dinky-core/src/test/resources/META-INF/services/org.dinky.job.JobHandler create mode 100644 dinky-core/src/test/resources/flink/sql/single-insert.sql create mode 100644 dinky-core/src/test/resources/flink/sql/statement-set-batch.sql create mode 100644 dinky-core/src/test/resources/flink/sql/statement-set-stream.sql create mode 100644 dinky-core/src/test/resources/flink/sql/variable.sql diff --git a/.gitignore b/.gitignore index 3bb0e62122..f642fdb5b1 100644 --- a/.gitignore +++ b/.gitignore @@ -49,6 +49,7 @@ dinky-web/pnpm-lock.yaml dinky-admin/src/main/resources/static/* dinky-admin/tmp/* dependency-reduced-pom.xml +dinky-core/tmp/* # Dinky Docs gitignore docs/.docusaurus diff --git a/dinky-admin/src/main/java/org/dinky/data/dto/TaskDTO.java b/dinky-admin/src/main/java/org/dinky/data/dto/TaskDTO.java index 213af7fd5d..50495bf612 100644 --- a/dinky-admin/src/main/java/org/dinky/data/dto/TaskDTO.java +++ b/dinky-admin/src/main/java/org/dinky/data/dto/TaskDTO.java @@ -213,7 +213,7 @@ public class TaskDTO extends AbstractStatementDTO { dataType = "boolean", example = "true", notes = "Flag indicating whether to mock sink function") - private boolean mockSinkFunction = true; + private boolean mockSinkFunction = false; @ApiModelProperty(value = "Session", dataType = "String", example = "session_id", notes = "The session identifier") private String session; diff --git a/dinky-admin/src/main/java/org/dinky/service/impl/PrintTableServiceImpl.java b/dinky-admin/src/main/java/org/dinky/service/impl/PrintTableServiceImpl.java index e759ce621c..6df5ef8f84 100644 --- a/dinky-admin/src/main/java/org/dinky/service/impl/PrintTableServiceImpl.java +++ b/dinky-admin/src/main/java/org/dinky/service/impl/PrintTableServiceImpl.java @@ -19,9 +19,9 @@ package org.dinky.service.impl; +import org.dinky.data.job.SqlType; import org.dinky.data.vo.PrintTableVo; import org.dinky.explainer.print_table.PrintStatementExplainer; -import org.dinky.parser.SqlType; import org.dinky.service.PrintTableService; import org.dinky.trans.Operations; import org.dinky.utils.SqlUtil; diff --git a/dinky-admin/src/main/java/org/dinky/service/impl/StudioServiceImpl.java b/dinky-admin/src/main/java/org/dinky/service/impl/StudioServiceImpl.java index f98ee33148..547f722242 100644 --- a/dinky-admin/src/main/java/org/dinky/service/impl/StudioServiceImpl.java +++ b/dinky-admin/src/main/java/org/dinky/service/impl/StudioServiceImpl.java @@ -122,7 +122,7 @@ public LineageResult getLineage(StudioLineageDTO studioCADTO) { TaskDTO taskDTO = taskService.getTaskInfoById(studioCADTO.getTaskId()); taskDTO.setStatement(taskService.buildEnvSql(taskDTO) + studioCADTO.getStatement()); JobConfig jobConfig = taskDTO.getJobConfig(); - return LineageBuilder.getColumnLineageByLogicalPlan(taskDTO.getStatement(), jobConfig.getExecutorSetting()); + return LineageBuilder.getColumnLineageByLogicalPlan(taskDTO.getStatement(), jobConfig); } } diff --git a/dinky-admin/src/main/java/org/dinky/service/impl/TaskServiceImpl.java b/dinky-admin/src/main/java/org/dinky/service/impl/TaskServiceImpl.java index 583de12f7f..a87d3a1d69 100644 --- a/dinky-admin/src/main/java/org/dinky/service/impl/TaskServiceImpl.java +++ b/dinky-admin/src/main/java/org/dinky/service/impl/TaskServiceImpl.java @@ -999,7 +999,7 @@ public LineageResult getTaskLineage(Integer id) { } else { task.setStatement(buildEnvSql(task) + task.getStatement()); JobConfig jobConfig = task.getJobConfig(); - return LineageBuilder.getColumnLineageByLogicalPlan(task.getStatement(), jobConfig.getExecutorSetting()); + return LineageBuilder.getColumnLineageByLogicalPlan(task.getStatement(), jobConfig); } } diff --git a/dinky-app/dinky-app-base/src/main/java/org/dinky/app/flinksql/Submitter.java b/dinky-app/dinky-app-base/src/main/java/org/dinky/app/flinksql/Submitter.java index 24f4e7c351..79c22d7c78 100644 --- a/dinky-app/dinky-app-base/src/main/java/org/dinky/app/flinksql/Submitter.java +++ b/dinky-app/dinky-app-base/src/main/java/org/dinky/app/flinksql/Submitter.java @@ -32,11 +32,11 @@ import org.dinky.data.app.AppTask; import org.dinky.data.constant.DirConstant; import org.dinky.data.enums.GatewayType; +import org.dinky.data.job.SqlType; import org.dinky.data.model.SystemConfiguration; import org.dinky.executor.Executor; import org.dinky.executor.ExecutorConfig; import org.dinky.executor.ExecutorFactory; -import org.dinky.parser.SqlType; import org.dinky.resource.BaseResourceManager; import org.dinky.trans.Operations; import org.dinky.trans.dml.ExecuteJarOperation; diff --git a/dinky-app/dinky-app-base/src/main/java/org/dinky/app/model/StatementParam.java b/dinky-app/dinky-app-base/src/main/java/org/dinky/app/model/StatementParam.java index d42c84da69..7e715ef19c 100644 --- a/dinky-app/dinky-app-base/src/main/java/org/dinky/app/model/StatementParam.java +++ b/dinky-app/dinky-app-base/src/main/java/org/dinky/app/model/StatementParam.java @@ -19,7 +19,7 @@ package org.dinky.app.model; -import org.dinky.parser.SqlType; +import org.dinky.data.job.SqlType; import lombok.AllArgsConstructor; import lombok.Data; diff --git a/dinky-client/dinky-client-1.14/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.14/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index 72b8ba8b48..1fa904f11a 100644 --- a/dinky-client/dinky-client-1.14/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.14/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -19,11 +19,15 @@ package org.dinky.executor; +import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; import org.dinky.data.model.LineageRel; import org.dinky.data.result.SqlExplainResult; import org.dinky.parser.CustomParserImpl; import org.dinky.utils.JsonUtils; import org.dinky.utils.LineageContext; +import org.dinky.utils.SqlUtil; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.common.RuntimeExecutionMode; @@ -49,6 +53,8 @@ import org.apache.flink.table.catalog.CatalogManager; import org.apache.flink.table.catalog.FunctionCatalog; import org.apache.flink.table.catalog.GenericInMemoryCatalog; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.UnresolvedIdentifier; import org.apache.flink.table.delegation.Executor; import org.apache.flink.table.delegation.ExecutorFactory; import org.apache.flink.table.delegation.Planner; @@ -57,7 +63,7 @@ import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.factories.PlannerFactoryUtil; import org.apache.flink.table.module.ModuleManager; -import org.apache.flink.table.operations.ExplainOperation; +import org.apache.flink.table.operations.CollectModifyOperation; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.QueryOperation; @@ -66,23 +72,31 @@ import java.lang.reflect.Field; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; +import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.fasterxml.jackson.databind.node.ObjectNode; +import cn.hutool.core.text.StrFormatter; import cn.hutool.core.util.ReflectUtil; -import lombok.extern.slf4j.Slf4j; /** * 定制TableEnvironmentImpl * * @since 2021/10/22 10:02 */ -@Slf4j public class CustomTableEnvironmentImpl extends AbstractCustomTableEnvironment { + + private static final Logger log = LoggerFactory.getLogger(CustomTableEnvironmentImpl.class); + private final CustomExtendedOperationExecutorImpl extendedExecutor = new CustomExtendedOperationExecutorImpl(this); private static final String UNSUPPORTED_QUERY_IN_EXECUTE_SQL_MSG = "Unsupported SQL query! executeSql() only accepts a single SQL statement of type " @@ -253,66 +267,77 @@ private Map getFlinkConfigurationMap() { } @Override - public JobPlanInfo getJobPlanInfo(List statements) { + public JobPlanInfo getJobPlanInfo(List statements) { return new JobPlanInfo(JsonPlanGenerator.generatePlan(getJobGraphFromInserts(statements))); } @Override - public StreamGraph getStreamGraphFromInserts(List statements) { - List modifyOperations = new ArrayList(); - for (String statement : statements) { - List operations = getParser().parse(statement); - if (operations.size() != 1) { - throw new TableException("Only single statement is supported."); - } else { - Operation operation = operations.get(0); - if (operation instanceof ModifyOperation) { - modifyOperations.add((ModifyOperation) operation); - } else { - throw new TableException("Only insert statement is supported now."); - } - } + public StreamGraph getStreamGraphFromInserts(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.CTAS)); + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); + List modifyOperations = new ArrayList<>(); + statements.stream() + .map(statement -> getParser().parse(statement.getStatement())) + .forEach(operations -> { + if (operations.size() != 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + modifyOperations.add((ModifyOperation) operation); + } else if (operation instanceof QueryOperation) { + final UnresolvedIdentifier unresolvedIdentifier = UnresolvedIdentifier.of( + "Unregistered_Collect_Sink_" + CollectModifyOperation.getUniqueId()); + final ObjectIdentifier objectIdentifier = + getCatalogManager().qualifyIdentifier(unresolvedIdentifier); + modifyOperations.add(new CollectModifyOperation(objectIdentifier, (QueryOperation) operation)); + } else { + log.info("Only insert statement is supported now. The statement is skipped: " + + operation.asSummaryString()); + } + }); + if (modifyOperations.isEmpty()) { + throw new TableException("Only insert statement is supported now. None operation to execute."); } + return transOperatoinsToStreamGraph(modifyOperations); + } + + private StreamGraph transOperatoinsToStreamGraph(List modifyOperations) { List> trans = getPlanner().translate(modifyOperations); - if (executor instanceof DefaultExecutor) { - StreamGraph streamGraph = - ((DefaultExecutor) executor).getExecutionEnvironment().generateStreamGraph(trans); - if (getConfig().getConfiguration().containsKey(PipelineOptions.NAME.key())) { - streamGraph.setJobName(getConfig().getConfiguration().getString(PipelineOptions.NAME)); - } - return streamGraph; - } else { - throw new TableException("Unsupported SQL query! ExecEnv need a ExecutorBase."); + final StreamExecutionEnvironment environment = getStreamExecutionEnvironment(); + trans.forEach(environment::addOperator); + + StreamGraph streamGraph = environment.getStreamGraph(); + final Configuration configuration = getConfig().getConfiguration(); + if (configuration.containsKey(PipelineOptions.NAME.key())) { + streamGraph.setJobName(configuration.getString(PipelineOptions.NAME)); } + return streamGraph; } @Override public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { - SqlExplainResult record = new SqlExplainResult(); List operations = getParser().parse(statement); - record.setParseTrue(true); if (operations.size() != 1) { throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); } Operation operation = operations.get(0); + SqlExplainResult data = new SqlExplainResult(); + data.setParseTrue(true); + data.setExplainTrue(true); if (operation instanceof ModifyOperation) { - record.setType("Modify DML"); - } else if (operation instanceof ExplainOperation) { - record.setType("Explain DML"); + data.setType("DML"); } else if (operation instanceof QueryOperation) { - record.setType("Query DML"); + data.setType("DQL"); } else { - record.setExplain(operation.asSummaryString()); - record.setType("DDL"); - } - record.setExplainTrue(true); - if ("DDL".equals(record.getType())) { - // record.setExplain("DDL语句不进行解释。"); - return record; + data.setExplain(operation.asSummaryString()); + data.setType("DDL"); + return data; } - record.setExplain(getPlanner().explain(operations, extraDetails)); - return record; + + data.setExplain(getPlanner().explain(Collections.singletonList(operation), extraDetails)); + return data; } @Override @@ -356,4 +381,98 @@ public TableResult executeInternal(Operation operation) { } return super.executeInternal(operation); } + + @Override + public SqlExplainResult explainStatementSet(List statements, ExplainDetail... extraDetails) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + List operations = new ArrayList<>(); + for (JobStatement statement : statements) { + if (statement.getSqlType().equals(SqlType.CTAS)) { + resultBuilder + .sql(statement.getStatement()) + .type(statement.getSqlType().getType()) + .error("CTAS is not supported in Apache Flink 1.15.") + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + return resultBuilder.build(); + } + if (statement.getSqlType().equals(SqlType.RTAS)) { + resultBuilder + .sql(statement.getStatement()) + .type(statement.getSqlType().getType()) + .error("RTAS is not supported in Apache Flink 1.14.") + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + return resultBuilder.build(); + } + try { + List itemOperations = getParser().parse(statement.getStatement()); + if (!itemOperations.isEmpty()) { + for (Operation operation : itemOperations) { + operations.add(operation); + } + } + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(statement.getStatement()), + e.getMessage()); + resultBuilder + .sql(statement.getStatement()) + .type(SqlType.INSERT.getType()) + .error(error) + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + log.error(error); + return resultBuilder.build(); + } + } + if (operations.isEmpty()) { + throw new DinkyException("None of the job in the statement set."); + } + resultBuilder.parseTrue(true); + resultBuilder.explain(getPlanner().explain(operations, extraDetails)); + return resultBuilder + .explainTrue(true) + .explainTime(LocalDateTime.now()) + .type(SqlType.INSERT.getType()) + .build(); + } + + @Override + public TableResult executeStatementSet(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.CTAS)); + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); + statements.removeIf(statement -> !statement.getSqlType().isSinkyModify()); + List modifyOperations = statements.stream() + .map(statement -> getModifyOperationFromInsert(statement.getStatement())) + .collect(Collectors.toList()); + return executeInternal(modifyOperations); + } + + public ModifyOperation getModifyOperationFromInsert(String statement) { + List operations = getParser().parse(statement); + if (operations.isEmpty()) { + throw new TableException("None of the statement is parsed."); + } + if (operations.size() > 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + return (ModifyOperation) operation; + } else if (operation instanceof QueryOperation) { + final UnresolvedIdentifier unresolvedIdentifier = + UnresolvedIdentifier.of("Unregistered_Collect_Sink_" + CollectModifyOperation.getUniqueId()); + final ObjectIdentifier objectIdentifier = getCatalogManager().qualifyIdentifier(unresolvedIdentifier); + return new CollectModifyOperation(objectIdentifier, (QueryOperation) operation); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + return null; + } + } } diff --git a/dinky-client/dinky-client-1.15/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.15/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index 2d790d2667..01a486a3c2 100644 --- a/dinky-client/dinky-client-1.15/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.15/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -21,10 +21,14 @@ import static org.apache.flink.table.api.bridge.internal.AbstractStreamTableEnvironmentImpl.lookupExecutor; +import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; import org.dinky.data.model.LineageRel; import org.dinky.data.result.SqlExplainResult; import org.dinky.parser.CustomParserImpl; import org.dinky.utils.LineageContext; +import org.dinky.utils.SqlUtil; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.common.RuntimeExecutionMode; @@ -58,22 +62,29 @@ import org.apache.flink.table.expressions.Expression; import org.apache.flink.table.factories.PlannerFactoryUtil; import org.apache.flink.table.module.ModuleManager; -import org.apache.flink.table.operations.ExplainOperation; +import org.apache.flink.table.operations.CollectModifyOperation; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.QueryOperation; import java.lang.reflect.Field; +import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; import cn.hutool.core.collection.CollUtil; +import cn.hutool.core.text.StrFormatter; /** * CustomTableEnvironmentImpl @@ -81,6 +92,9 @@ * @since 2022/05/08 */ public class CustomTableEnvironmentImpl extends AbstractCustomTableEnvironment { + + private static final Logger log = LoggerFactory.getLogger(CustomTableEnvironmentImpl.class); + private final CustomExtendedOperationExecutorImpl extendedExecutor = new CustomExtendedOperationExecutorImpl(this); private static final String UNSUPPORTED_QUERY_IN_EXECUTE_SQL_MSG = "Unsupported SQL query! executeSql() only accepts a single SQL statement of type " @@ -226,63 +240,73 @@ private Map getFlinkConfigurationMap() { } @Override - public JobPlanInfo getJobPlanInfo(List statements) { + public JobPlanInfo getJobPlanInfo(List statements) { return new JobPlanInfo(JsonPlanGenerator.generatePlan(getJobGraphFromInserts(statements))); } @Override - public StreamGraph getStreamGraphFromInserts(List statements) { - List modifyOperations = new ArrayList(); - for (String statement : statements) { - List operations = getParser().parse(statement); - if (operations.size() != 1) { - throw new TableException("Only single statement is supported."); - } else { - Operation operation = operations.get(0); - if (operation instanceof ModifyOperation) { - modifyOperations.add((ModifyOperation) operation); - } else { - throw new TableException("Only insert statement is supported now."); - } - } + public StreamGraph getStreamGraphFromInserts(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.CTAS)); + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); + List modifyOperations = new ArrayList<>(); + statements.stream() + .map(statement -> getParser().parse(statement.getStatement())) + .forEach(operations -> { + if (operations.size() != 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + modifyOperations.add((ModifyOperation) operation); + } else if (operation instanceof QueryOperation) { + modifyOperations.add(new CollectModifyOperation((QueryOperation) operation)); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + } + }); + if (modifyOperations.isEmpty()) { + throw new TableException("Only insert statement is supported now. None operation to execute."); } + return transOperatoinsToStreamGraph(modifyOperations); + } + + private StreamGraph transOperatoinsToStreamGraph(List modifyOperations) { List> trans = getPlanner().translate(modifyOperations); - for (Transformation transformation : trans) { - getStreamExecutionEnvironment().addOperator(transformation); - } - StreamGraph streamGraph = getStreamExecutionEnvironment().getStreamGraph(); - if (getConfig().getConfiguration().containsKey(PipelineOptions.NAME.key())) { - streamGraph.setJobName(getConfig().getConfiguration().getString(PipelineOptions.NAME)); + final StreamExecutionEnvironment environment = getStreamExecutionEnvironment(); + trans.forEach(environment::addOperator); + + StreamGraph streamGraph = environment.getStreamGraph(); + final Configuration configuration = getConfig().getConfiguration(); + if (configuration.containsKey(PipelineOptions.NAME.key())) { + streamGraph.setJobName(configuration.getString(PipelineOptions.NAME)); } return streamGraph; } @Override public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { - SqlExplainResult record = new SqlExplainResult(); List operations = getParser().parse(statement); - record.setParseTrue(true); if (operations.size() != 1) { throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); } + Operation operation = operations.get(0); + SqlExplainResult data = new SqlExplainResult(); + data.setParseTrue(true); + data.setExplainTrue(true); if (operation instanceof ModifyOperation) { - record.setType("Modify DML"); - } else if (operation instanceof ExplainOperation) { - record.setType("Explain DML"); + data.setType("DML"); } else if (operation instanceof QueryOperation) { - record.setType("Query DML"); + data.setType("DQL"); } else { - record.setExplain(operation.asSummaryString()); - record.setType("DDL"); - } - record.setExplainTrue(true); - if ("DDL".equals(record.getType())) { - // record.setExplain("DDL语句不进行解释。"); - return record; + data.setExplain(operation.asSummaryString()); + data.setType("DDL"); + return data; } - record.setExplain(getPlanner().explain(operations, extraDetails)); - return record; + + data.setExplain(getPlanner().explain(Collections.singletonList(operation), extraDetails)); + return data; } @Override @@ -336,4 +360,95 @@ public TableResultInternal executeInternal(Operation operation) { } return super.executeInternal(operation); } + + @Override + public SqlExplainResult explainStatementSet(List statements, ExplainDetail... extraDetails) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + List operations = new ArrayList<>(); + for (JobStatement statement : statements) { + if (statement.getSqlType().equals(SqlType.CTAS)) { + resultBuilder + .sql(statement.getStatement()) + .type(statement.getSqlType().getType()) + .error("CTAS is not supported in Apache Flink 1.15.") + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + return resultBuilder.build(); + } + if (statement.getSqlType().equals(SqlType.RTAS)) { + resultBuilder + .sql(statement.getStatement()) + .type(statement.getSqlType().getType()) + .error("RTAS is not supported in Apache Flink 1.15.") + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + return resultBuilder.build(); + } + try { + List itemOperations = getParser().parse(statement.getStatement()); + if (!itemOperations.isEmpty()) { + for (Operation operation : itemOperations) { + operations.add(operation); + } + } + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(statement.getStatement()), + e.getMessage()); + resultBuilder + .sql(statement.getStatement()) + .type(SqlType.INSERT.getType()) + .error(error) + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + log.error(error); + return resultBuilder.build(); + } + } + if (operations.isEmpty()) { + throw new DinkyException("None of the job in the statement set."); + } + resultBuilder.parseTrue(true); + resultBuilder.explain(getPlanner().explain(operations, extraDetails)); + return resultBuilder + .explainTrue(true) + .explainTime(LocalDateTime.now()) + .type(SqlType.INSERT.getType()) + .build(); + } + + @Override + public TableResult executeStatementSet(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.CTAS)); + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); + statements.removeIf(statement -> !statement.getSqlType().isSinkyModify()); + List modifyOperations = statements.stream() + .map(statement -> getModifyOperationFromInsert(statement.getStatement())) + .collect(Collectors.toList()); + return executeInternal(modifyOperations); + } + + public ModifyOperation getModifyOperationFromInsert(String statement) { + List operations = getParser().parse(statement); + if (operations.isEmpty()) { + throw new TableException("None of the statement is parsed."); + } + if (operations.size() > 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + return (ModifyOperation) operation; + } else if (operation instanceof QueryOperation) { + return new CollectModifyOperation((QueryOperation) operation); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + return null; + } + } } diff --git a/dinky-client/dinky-client-1.16/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.16/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index ca3b6e9f31..eb6d5adb33 100644 --- a/dinky-client/dinky-client-1.16/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.16/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -19,9 +19,13 @@ package org.dinky.executor; +import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; import org.dinky.data.result.SqlExplainResult; import org.dinky.parser.CustomParserImpl; import org.dinky.utils.JsonUtils; +import org.dinky.utils.SqlUtil; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.dag.Transformation; @@ -35,13 +39,17 @@ import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.ExplainDetail; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.operations.ExplainOperation; +import org.apache.flink.table.operations.CollectModifyOperation; +import org.apache.flink.table.operations.CreateTableASOperation; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; import org.apache.flink.table.operations.QueryOperation; +import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -50,6 +58,8 @@ import com.fasterxml.jackson.databind.node.ObjectNode; +import cn.hutool.core.text.StrFormatter; + /** * CustomTableEnvironmentImpl * @@ -119,27 +129,48 @@ private StreamGraph transOperatoinsToStreamGraph(List modifyOpe } @Override - public JobPlanInfo getJobPlanInfo(List statements) { + public JobPlanInfo getJobPlanInfo(List statements) { return new JobPlanInfo(JsonPlanGenerator.generatePlan(getJobGraphFromInserts(statements))); } - public StreamGraph getStreamGraphFromInserts(List statements) { + @Override + public StreamGraph getStreamGraphFromInserts(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); List modifyOperations = new ArrayList<>(); - statements.stream().map(statement -> getParser().parse(statement)).forEach(operations -> { - if (operations.size() != 1) { - throw new TableException("Only single statement is supported."); - } - Operation operation = operations.get(0); - if (operation instanceof ModifyOperation) { - modifyOperations.add((ModifyOperation) operation); - } else { - throw new TableException("Only insert statement is supported now."); - } - }); - + statements.stream() + .map(statement -> getParser().parse(statement.getStatement())) + .forEach(operations -> { + if (operations.size() != 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + CreateTableASOperation createTableASOperation = (CreateTableASOperation) operation; + executeInternal(createTableASOperation.getCreateTableOperation()); + modifyOperations.add(createTableASOperation.toSinkModifyOperation(getCatalogManager())); + } else { + modifyOperations.add((ModifyOperation) operation); + } + } else if (operation instanceof QueryOperation) { + modifyOperations.add(new CollectModifyOperation((QueryOperation) operation)); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + } + }); + if (modifyOperations.isEmpty()) { + throw new TableException("Only insert statement is supported now. None operation to execute."); + } return transOperatoinsToStreamGraph(modifyOperations); } + @Override + public SqlNode parseSql(String sql) { + return ((ExtendedParser) getParser()).parseSql(sql); + } + + @Override public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { List operations = getParser().parse(statement); if (operations.size() != 1) { @@ -147,30 +178,117 @@ public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extr } Operation operation = operations.get(0); - SqlExplainResult record = new SqlExplainResult(); - record.setParseTrue(true); - record.setExplainTrue(true); - + SqlExplainResult data = new SqlExplainResult(); + data.setParseTrue(true); + data.setExplainTrue(true); if (operation instanceof ModifyOperation) { - record.setType("Modify DML"); - } else if (operation instanceof ExplainOperation) { - record.setType("Explain DML"); + if (operation instanceof CreateTableASOperation) { + CreateTableASOperation createTableASOperation = (CreateTableASOperation) operation; + executeInternal(createTableASOperation.getCreateTableOperation()); + operation = createTableASOperation.toSinkModifyOperation(getCatalogManager()); + } + data.setType("DML"); } else if (operation instanceof QueryOperation) { - record.setType("Query DML"); + data.setType("DQL"); } else { - record.setExplain(operation.asSummaryString()); - record.setType("DDL"); - - // record.setExplain("DDL statement needn't comment。"); - return record; + data.setExplain(operation.asSummaryString()); + data.setType("DDL"); + return data; } - record.setExplain(getPlanner().explain(operations, extraDetails)); - return record; + data.setExplain(getPlanner().explain(Collections.singletonList(operation), extraDetails)); + return data; } @Override - public SqlNode parseSql(String sql) { - return ((ExtendedParser) getParser()).parseSql(sql); + public SqlExplainResult explainStatementSet(List statements, ExplainDetail... extraDetails) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + List operations = new ArrayList<>(); + for (JobStatement statement : statements) { + if (statement.getSqlType().equals(SqlType.RTAS)) { + resultBuilder + .sql(statement.getStatement()) + .type(statement.getSqlType().getType()) + .error("RTAS is not supported in Apache Flink 1.16.") + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + return resultBuilder.build(); + } + try { + List itemOperations = getParser().parse(statement.getStatement()); + if (!itemOperations.isEmpty()) { + for (Operation operation : itemOperations) { + if (operation instanceof CreateTableASOperation) { + CreateTableASOperation createTableASOperation = (CreateTableASOperation) operation; + executeInternal(createTableASOperation.getCreateTableOperation()); + operations.add(createTableASOperation.toSinkModifyOperation(getCatalogManager())); + } else { + operations.add(operation); + } + } + } + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(statement.getStatement()), + e.getMessage()); + resultBuilder + .sql(statement.getStatement()) + .type(SqlType.INSERT.getType()) + .error(error) + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + log.error(error); + return resultBuilder.build(); + } + } + if (operations.isEmpty()) { + throw new DinkyException("None of the job in the statement set."); + } + resultBuilder.parseTrue(true); + resultBuilder.explain(getPlanner().explain(operations, extraDetails)); + return resultBuilder + .explainTrue(true) + .explainTime(LocalDateTime.now()) + .type(SqlType.INSERT.getType()) + .build(); + } + + @Override + public TableResult executeStatementSet(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); + statements.removeIf(statement -> !statement.getSqlType().isSinkyModify()); + List modifyOperations = statements.stream() + .map(statement -> getModifyOperationFromInsert(statement.getStatement())) + .collect(Collectors.toList()); + return executeInternal(modifyOperations); + } + + public ModifyOperation getModifyOperationFromInsert(String statement) { + List operations = getParser().parse(statement); + if (operations.isEmpty()) { + throw new TableException("None of the statement is parsed."); + } + if (operations.size() > 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + CreateTableASOperation createTableASOperation = (CreateTableASOperation) operation; + executeInternal(createTableASOperation.getCreateTableOperation()); + return createTableASOperation.toSinkModifyOperation(getCatalogManager()); + } else { + return (ModifyOperation) operation; + } + } else if (operation instanceof QueryOperation) { + return new CollectModifyOperation((QueryOperation) operation); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + return null; + } } } diff --git a/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java b/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java index d4a02dea0f..8a7adfc800 100644 --- a/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java +++ b/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java @@ -20,23 +20,15 @@ package org.dinky.executor; import org.dinky.data.model.LineageRel; -import org.dinky.data.result.SqlExplainResult; import org.dinky.utils.LineageContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.ExplainDetail; -import org.apache.flink.table.api.ExplainFormat; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl; import org.apache.flink.table.delegation.ExtendedOperationExecutor; import org.apache.flink.table.delegation.Planner; -import org.apache.flink.table.operations.ExplainOperation; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.operations.QueryOperation; import org.apache.flink.table.planner.delegation.PlannerBase; import java.util.List; @@ -93,36 +85,6 @@ public Configuration getRootConfiguration() { return (Configuration) this.getConfig().getRootConfiguration(); } - @Override - public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { - List operations = getParser().parse(statement); - if (operations.size() != 1) { - throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); - } - - Operation operation = operations.get(0); - SqlExplainResult data = new SqlExplainResult(); - data.setParseTrue(true); - data.setExplainTrue(true); - - if (operation instanceof ModifyOperation) { - data.setType("Modify DML"); - } else if (operation instanceof ExplainOperation) { - data.setType("Explain DML"); - } else if (operation instanceof QueryOperation) { - data.setType("Query DML"); - } else { - data.setExplain(operation.asSummaryString()); - data.setType("DDL"); - - // data.setExplain("DDL statement needn't comment。"); - return data; - } - - data.setExplain(getPlanner().explain(operations, ExplainFormat.TEXT, extraDetails)); - return data; - } - @Override public List getLineage(String statement) { LineageContext lineageContext = new LineageContext(this); diff --git a/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index 3b6e4b1a9e..988884ada8 100644 --- a/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.17/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -19,7 +19,12 @@ package org.dinky.executor; +import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; +import org.dinky.data.result.SqlExplainResult; import org.dinky.parser.CustomParserImpl; +import org.dinky.utils.SqlUtil; import org.apache.calcite.sql.SqlNode; import org.apache.flink.api.dag.Transformation; @@ -31,12 +36,20 @@ import org.apache.flink.streaming.api.graph.JSONGenerator; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.ExplainDetail; +import org.apache.flink.table.api.ExplainFormat; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.operations.CollectModifyOperation; +import org.apache.flink.table.operations.CreateTableASOperation; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.QueryOperation; +import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; @@ -47,6 +60,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; +import cn.hutool.core.text.StrFormatter; + /** * CustomTableEnvironmentImpl * @@ -124,25 +139,39 @@ private StreamGraph transOperatoinsToStreamGraph(List modifyOpe } @Override - public JobPlanInfo getJobPlanInfo(List statements) { + public JobPlanInfo getJobPlanInfo(List statements) { return new JobPlanInfo(JsonPlanGenerator.generatePlan(getJobGraphFromInserts(statements))); } @Override - public StreamGraph getStreamGraphFromInserts(List statements) { + public StreamGraph getStreamGraphFromInserts(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); List modifyOperations = new ArrayList<>(); - statements.stream().map(statement -> getParser().parse(statement)).forEach(operations -> { - if (operations.size() != 1) { - throw new TableException("Only single statement is supported."); - } - Operation operation = operations.get(0); - if (operation instanceof ModifyOperation) { - modifyOperations.add((ModifyOperation) operation); - } else { - throw new TableException("Only insert statement is supported now."); - } - }); - + statements.stream() + .map(statement -> getParser().parse(statement.getStatement())) + .forEach(operations -> { + if (operations.size() != 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + CreateTableASOperation createTableASOperation = (CreateTableASOperation) operation; + executeInternal(createTableASOperation.getCreateTableOperation()); + modifyOperations.add(createTableASOperation.toSinkModifyOperation(getCatalogManager())); + } else { + modifyOperations.add((ModifyOperation) operation); + } + } else if (operation instanceof QueryOperation) { + modifyOperations.add(new CollectModifyOperation((QueryOperation) operation)); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + } + }); + if (modifyOperations.isEmpty()) { + throw new TableException("Only insert statement is supported now. None operation to execute."); + } return transOperatoinsToStreamGraph(modifyOperations); } @@ -150,4 +179,126 @@ public StreamGraph getStreamGraphFromInserts(List statements) { public SqlNode parseSql(String sql) { return ((ExtendedParser) getParser()).parseSql(sql); } + + @Override + public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { + List operations = getParser().parse(statement); + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); + } + + Operation operation = operations.get(0); + SqlExplainResult data = new SqlExplainResult(); + data.setParseTrue(true); + data.setExplainTrue(true); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + CreateTableASOperation createTableASOperation = (CreateTableASOperation) operation; + executeInternal(createTableASOperation.getCreateTableOperation()); + operation = createTableASOperation.toSinkModifyOperation(getCatalogManager()); + } + data.setType("DML"); + } else if (operation instanceof QueryOperation) { + data.setType("DQL"); + } else { + data.setExplain(operation.asSummaryString()); + data.setType("DDL"); + return data; + } + + data.setExplain(getPlanner().explain(Collections.singletonList(operation), ExplainFormat.TEXT, extraDetails)); + return data; + } + + @Override + public SqlExplainResult explainStatementSet(List statements, ExplainDetail... extraDetails) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + List operations = new ArrayList<>(); + for (JobStatement statement : statements) { + if (statement.getSqlType().equals(SqlType.RTAS)) { + resultBuilder + .sql(statement.getStatement()) + .type(statement.getSqlType().getType()) + .error("RTAS is not supported in Apache Flink 1.17.") + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + return resultBuilder.build(); + } + try { + List itemOperations = getParser().parse(statement.getStatement()); + if (!itemOperations.isEmpty()) { + for (Operation operation : itemOperations) { + if (operation instanceof CreateTableASOperation) { + CreateTableASOperation createTableASOperation = (CreateTableASOperation) operation; + executeInternal(createTableASOperation.getCreateTableOperation()); + operations.add(createTableASOperation.toSinkModifyOperation(getCatalogManager())); + } else { + operations.add(operation); + } + } + } + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(statement.getStatement()), + e.getMessage()); + resultBuilder + .sql(statement.getStatement()) + .type(SqlType.INSERT.getType()) + .error(error) + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + log.error(error); + return resultBuilder.build(); + } + } + if (operations.isEmpty()) { + throw new DinkyException("None of the job in the statement set."); + } + resultBuilder.parseTrue(true); + resultBuilder.explain(getPlanner().explain(operations, ExplainFormat.TEXT, extraDetails)); + return resultBuilder + .explainTrue(true) + .explainTime(LocalDateTime.now()) + .type(SqlType.INSERT.getType()) + .build(); + } + + @Override + public TableResult executeStatementSet(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); + statements.removeIf(statement -> !statement.getSqlType().isSinkyModify()); + List modifyOperations = statements.stream() + .map(statement -> getModifyOperationFromInsert(statement.getStatement())) + .collect(Collectors.toList()); + return executeInternal(modifyOperations); + } + + public ModifyOperation getModifyOperationFromInsert(String statement) { + List operations = getParser().parse(statement); + if (operations.isEmpty()) { + throw new TableException("None of the statement is parsed."); + } + if (operations.size() > 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + CreateTableASOperation createTableASOperation = (CreateTableASOperation) operation; + executeInternal(createTableASOperation.getCreateTableOperation()); + return createTableASOperation.toSinkModifyOperation(getCatalogManager()); + } else { + return (ModifyOperation) operation; + } + } else if (operation instanceof QueryOperation) { + return new CollectModifyOperation((QueryOperation) operation); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + return null; + } + } } diff --git a/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java index 6414271715..cf1724512f 100644 --- a/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java +++ b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java @@ -20,22 +20,14 @@ package org.dinky.executor; import org.dinky.data.model.LineageRel; -import org.dinky.data.result.SqlExplainResult; import org.dinky.utils.LineageContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.ExplainDetail; -import org.apache.flink.table.api.ExplainFormat; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl; import org.apache.flink.table.delegation.Planner; -import org.apache.flink.table.operations.ExplainOperation; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.operations.QueryOperation; import java.util.List; @@ -85,36 +77,6 @@ public Configuration getRootConfiguration() { return (Configuration) this.getConfig().getRootConfiguration(); } - @Override - public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { - List operations = getParser().parse(statement); - if (operations.size() != 1) { - throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); - } - - Operation operation = operations.get(0); - SqlExplainResult data = new SqlExplainResult(); - data.setParseTrue(true); - data.setExplainTrue(true); - - if (operation instanceof ModifyOperation) { - data.setType("Modify DML"); - } else if (operation instanceof ExplainOperation) { - data.setType("Explain DML"); - } else if (operation instanceof QueryOperation) { - data.setType("Query DML"); - } else { - data.setExplain(operation.asSummaryString()); - data.setType("DDL"); - - // data.setExplain("DDL statement needn't comment。"); - return data; - } - - data.setExplain(getPlanner().explain(operations, ExplainFormat.TEXT, extraDetails)); - return data; - } - @Override public List getLineage(String statement) { LineageContext lineageContext = new LineageContext(this); diff --git a/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index 4a63497546..5066302ecb 100644 --- a/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.18/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -19,11 +19,17 @@ package org.dinky.executor; +import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; +import org.dinky.data.result.SqlExplainResult; import org.dinky.operations.CustomNewParserImpl; import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.dag.Transformation; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator; import org.apache.flink.runtime.rest.messages.JobPlanInfo; @@ -31,14 +37,40 @@ import org.apache.flink.streaming.api.graph.JSONGenerator; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.ExplainDetail; +import org.apache.flink.table.api.ExplainFormat; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogDescriptor; +import org.apache.flink.table.catalog.ContextResolvedTable; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.StagedTable; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkStagingContext; +import org.apache.flink.table.connector.sink.abilities.SupportsStaging; +import org.apache.flink.table.execution.StagingSinkJobStatusHook; +import org.apache.flink.table.factories.TableFactoryUtil; +import org.apache.flink.table.module.Module; +import org.apache.flink.table.module.ModuleManager; +import org.apache.flink.table.operations.CollectModifyOperation; +import org.apache.flink.table.operations.CreateTableASOperation; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.QueryOperation; +import org.apache.flink.table.operations.ReplaceTableAsOperation; +import org.apache.flink.table.operations.ddl.CreateTableOperation; +import org.apache.flink.table.operations.utils.ExecutableOperationUtils; +import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; import org.slf4j.Logger; @@ -124,25 +156,38 @@ private StreamGraph transOperatoinsToStreamGraph(List modifyOpe } @Override - public JobPlanInfo getJobPlanInfo(List statements) { + public JobPlanInfo getJobPlanInfo(List statements) { return new JobPlanInfo(JsonPlanGenerator.generatePlan(getJobGraphFromInserts(statements))); } @Override - public StreamGraph getStreamGraphFromInserts(List statements) { + public StreamGraph getStreamGraphFromInserts(List statements) { List modifyOperations = new ArrayList<>(); - statements.stream().map(statement -> getParser().parse(statement)).forEach(operations -> { - if (operations.size() != 1) { - throw new TableException("Only single statement is supported."); - } - Operation operation = operations.get(0); - if (operation instanceof ModifyOperation) { - modifyOperations.add((ModifyOperation) operation); - } else { - throw new TableException("Only insert statement is supported now."); - } - }); - + statements.stream() + .map(statement -> getParser().parse(statement.getStatement())) + .forEach(operations -> { + if (operations.size() != 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + modifyOperations.add(getModifyOperation((CreateTableASOperation) operation)); + } else if (operation instanceof ReplaceTableAsOperation) { + modifyOperations.add(getModifyOperation((ReplaceTableAsOperation) operation)); + } else { + modifyOperations.add((ModifyOperation) operation); + } + } else if (operation instanceof QueryOperation) { + modifyOperations.add(new CollectModifyOperation((QueryOperation) operation)); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + } + }); + if (modifyOperations.isEmpty()) { + throw new TableException("Only insert statement is supported now. None operation to execute."); + } return transOperatoinsToStreamGraph(modifyOperations); } @@ -155,4 +200,206 @@ public void createCatalog(String catalogName, CatalogDescriptor catalogDescripto public SqlNode parseSql(String sql) { return ((ExtendedParser) getParser()).getCustomParser().parseSql(sql); } + + @Override + public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { + List operations = getParser().parse(statement); + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); + } + + Operation operation = operations.get(0); + SqlExplainResult data = new SqlExplainResult(); + data.setParseTrue(true); + data.setExplainTrue(true); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + operation = getModifyOperation((CreateTableASOperation) operation); + } else if (operation instanceof ReplaceTableAsOperation) { + operation = getModifyOperation((ReplaceTableAsOperation) operation); + } + data.setType("DML"); + } else if (operation instanceof QueryOperation) { + data.setType("DQL"); + } else { + data.setExplain(operation.asSummaryString()); + data.setType("DDL"); + return data; + } + + data.setExplain(getPlanner().explain(Collections.singletonList(operation), ExplainFormat.TEXT, extraDetails)); + return data; + } + + @Override + public SqlExplainResult explainStatementSet(List statements, ExplainDetail... extraDetails) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + List operations = new ArrayList<>(); + for (JobStatement statement : statements) { + List itemOperations = getParser().parse(statement.getStatement()); + if (!itemOperations.isEmpty()) { + for (Operation operation : itemOperations) { + if (operation instanceof CreateTableASOperation) { + operations.add(getModifyOperation((CreateTableASOperation) operation)); + } else if (operation instanceof ReplaceTableAsOperation) { + operations.add(getModifyOperation((ReplaceTableAsOperation) operation)); + } else { + operations.add(operation); + } + } + } + } + if (operations.isEmpty()) { + throw new DinkyException("None of the job in the statement set."); + } + resultBuilder.parseTrue(true); + resultBuilder.explain(getPlanner().explain(operations, ExplainFormat.TEXT, extraDetails)); + return resultBuilder + .explainTrue(true) + .explainTime(LocalDateTime.now()) + .type(SqlType.INSERT.getType()) + .build(); + } + + @Override + public TableResult executeStatementSet(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); + statements.removeIf(statement -> !statement.getSqlType().isSinkyModify()); + List modifyOperations = statements.stream() + .map(statement -> getModifyOperationFromInsert(statement.getStatement())) + .collect(Collectors.toList()); + return executeInternal(modifyOperations); + } + + public ModifyOperation getModifyOperationFromInsert(String statement) { + List operations = getParser().parse(statement); + if (operations.isEmpty()) { + throw new TableException("None of the statement is parsed."); + } + if (operations.size() > 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + return getModifyOperation((CreateTableASOperation) operation); + } else if (operation instanceof ReplaceTableAsOperation) { + return getModifyOperation((ReplaceTableAsOperation) operation); + } else { + return (ModifyOperation) operation; + } + } else if (operation instanceof QueryOperation) { + return new CollectModifyOperation((QueryOperation) operation); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + return null; + } + } + + private ModifyOperation getModifyOperation(CreateTableASOperation ctasOperation) { + CreateTableOperation createTableOperation = ctasOperation.getCreateTableOperation(); + ObjectIdentifier tableIdentifier = createTableOperation.getTableIdentifier(); + Catalog catalog = getCatalogManager().getCatalogOrThrowException(tableIdentifier.getCatalogName()); + ResolvedCatalogTable catalogTable = + getCatalogManager().resolveCatalogTable(createTableOperation.getCatalogTable()); + Optional stagingDynamicTableSink = + getSupportsStagingDynamicTableSink(createTableOperation, catalog, catalogTable); + if (stagingDynamicTableSink.isPresent()) { + // use atomic ctas + DynamicTableSink dynamicTableSink = stagingDynamicTableSink.get(); + SupportsStaging.StagingPurpose stagingPurpose = createTableOperation.isIgnoreIfExists() + ? SupportsStaging.StagingPurpose.CREATE_TABLE_AS_IF_NOT_EXISTS + : SupportsStaging.StagingPurpose.CREATE_TABLE_AS; + StagedTable stagedTable = + ((SupportsStaging) dynamicTableSink).applyStaging(new SinkStagingContext(stagingPurpose)); + StagingSinkJobStatusHook stagingSinkJobStatusHook = new StagingSinkJobStatusHook(stagedTable); + return ctasOperation.toStagedSinkModifyOperation(tableIdentifier, catalogTable, catalog, dynamicTableSink); + } + // use non-atomic ctas, create table first + executeInternal(createTableOperation); + return ctasOperation.toSinkModifyOperation(getCatalogManager()); + } + + private ModifyOperation getModifyOperation(ReplaceTableAsOperation rtasOperation) { + CreateTableOperation createTableOperation = rtasOperation.getCreateTableOperation(); + ObjectIdentifier tableIdentifier = createTableOperation.getTableIdentifier(); + // First check if the replacedTable exists + Optional replacedTable = getCatalogManager().getTable(tableIdentifier); + if (!rtasOperation.isCreateOrReplace() && !replacedTable.isPresent()) { + throw new TableException(String.format( + "The table %s to be replaced doesn't exist. " + + "You can try to use CREATE TABLE AS statement or " + + "CREATE OR REPLACE TABLE AS statement.", + tableIdentifier)); + } + Catalog catalog = getCatalogManager().getCatalogOrThrowException(tableIdentifier.getCatalogName()); + ResolvedCatalogTable catalogTable = + getCatalogManager().resolveCatalogTable(createTableOperation.getCatalogTable()); + Optional stagingDynamicTableSink = + getSupportsStagingDynamicTableSink(createTableOperation, catalog, catalogTable); + if (stagingDynamicTableSink.isPresent()) { + // use atomic rtas + DynamicTableSink dynamicTableSink = stagingDynamicTableSink.get(); + SupportsStaging.StagingPurpose stagingPurpose = rtasOperation.isCreateOrReplace() + ? SupportsStaging.StagingPurpose.CREATE_OR_REPLACE_TABLE_AS + : SupportsStaging.StagingPurpose.REPLACE_TABLE_AS; + + StagedTable stagedTable = + ((SupportsStaging) dynamicTableSink).applyStaging(new SinkStagingContext(stagingPurpose)); + StagingSinkJobStatusHook stagingSinkJobStatusHook = new StagingSinkJobStatusHook(stagedTable); + return rtasOperation.toStagedSinkModifyOperation(tableIdentifier, catalogTable, catalog, dynamicTableSink); + } + // non-atomic rtas drop table first if exists, then create + if (replacedTable.isPresent()) { + getCatalogManager().dropTable(tableIdentifier, false); + } + executeInternal(createTableOperation); + return rtasOperation.toSinkModifyOperation(getCatalogManager()); + } + + private Optional getSupportsStagingDynamicTableSink( + CreateTableOperation createTableOperation, Catalog catalog, ResolvedCatalogTable catalogTable) { + TableConfig tableConfig = getTableEnvironment().getConfig(); + boolean isStreamingMode = true; + RuntimeExecutionMode runtimeExecutionMode = + getStreamExecutionEnvironment().getConfiguration().get(ExecutionOptions.RUNTIME_MODE); + if (RuntimeExecutionMode.BATCH.equals(runtimeExecutionMode)) { + isStreamingMode = false; + } + if (tableConfig.get(TableConfigOptions.TABLE_RTAS_CTAS_ATOMICITY_ENABLED)) { + if (!TableFactoryUtil.isLegacyConnectorOptions( + catalog, + tableConfig, + isStreamingMode, + createTableOperation.getTableIdentifier(), + catalogTable, + createTableOperation.isTemporary())) { + try { + DynamicTableSink dynamicTableSink = ExecutableOperationUtils.createDynamicTableSink( + catalog, + () -> (new ModuleManager()).getFactory((Module::getTableSinkFactory)), + createTableOperation.getTableIdentifier(), + catalogTable, + Collections.emptyMap(), + tableConfig, + getUserClassLoader(), + createTableOperation.isTemporary()); + if (dynamicTableSink instanceof SupportsStaging) { + return Optional.of(dynamicTableSink); + } + } catch (Exception e) { + throw new TableException( + String.format( + "Fail to create DynamicTableSink for the table %s, " + + "maybe the table does not support atomicity of CTAS/RTAS, " + + "please set %s to false and try again.", + createTableOperation.getTableIdentifier(), + TableConfigOptions.TABLE_RTAS_CTAS_ATOMICITY_ENABLED.key()), + e); + } + } + } + return Optional.empty(); + } } diff --git a/dinky-client/dinky-client-1.19/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java b/dinky-client/dinky-client-1.19/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java index 6414271715..cf1724512f 100644 --- a/dinky-client/dinky-client-1.19/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java +++ b/dinky-client/dinky-client-1.19/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java @@ -20,22 +20,14 @@ package org.dinky.executor; import org.dinky.data.model.LineageRel; -import org.dinky.data.result.SqlExplainResult; import org.dinky.utils.LineageContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.ExplainDetail; -import org.apache.flink.table.api.ExplainFormat; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl; import org.apache.flink.table.delegation.Planner; -import org.apache.flink.table.operations.ExplainOperation; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.operations.QueryOperation; import java.util.List; @@ -85,36 +77,6 @@ public Configuration getRootConfiguration() { return (Configuration) this.getConfig().getRootConfiguration(); } - @Override - public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { - List operations = getParser().parse(statement); - if (operations.size() != 1) { - throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); - } - - Operation operation = operations.get(0); - SqlExplainResult data = new SqlExplainResult(); - data.setParseTrue(true); - data.setExplainTrue(true); - - if (operation instanceof ModifyOperation) { - data.setType("Modify DML"); - } else if (operation instanceof ExplainOperation) { - data.setType("Explain DML"); - } else if (operation instanceof QueryOperation) { - data.setType("Query DML"); - } else { - data.setExplain(operation.asSummaryString()); - data.setType("DDL"); - - // data.setExplain("DDL statement needn't comment。"); - return data; - } - - data.setExplain(getPlanner().explain(operations, ExplainFormat.TEXT, extraDetails)); - return data; - } - @Override public List getLineage(String statement) { LineageContext lineageContext = new LineageContext(this); diff --git a/dinky-client/dinky-client-1.19/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.19/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index a36e38fcc3..9f858cfcfe 100644 --- a/dinky-client/dinky-client-1.19/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.19/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -19,11 +19,17 @@ package org.dinky.executor; +import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; +import org.dinky.data.result.SqlExplainResult; import org.dinky.operations.CustomNewParserImpl; import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.dag.Transformation; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator; import org.apache.flink.runtime.rest.messages.JobPlanInfo; @@ -31,16 +37,39 @@ import org.apache.flink.streaming.api.graph.JSONGenerator; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.ExplainDetail; +import org.apache.flink.table.api.ExplainFormat; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.internal.CachedPlan; import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogDescriptor; +import org.apache.flink.table.catalog.ContextResolvedTable; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsStaging; +import org.apache.flink.table.factories.TableFactoryUtil; +import org.apache.flink.table.module.Module; +import org.apache.flink.table.module.ModuleManager; +import org.apache.flink.table.operations.CollectModifyOperation; +import org.apache.flink.table.operations.CreateTableASOperation; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.QueryOperation; +import org.apache.flink.table.operations.ReplaceTableAsOperation; +import org.apache.flink.table.operations.ddl.CreateTableOperation; +import org.apache.flink.table.operations.utils.ExecutableOperationUtils; +import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; import org.slf4j.Logger; @@ -126,25 +155,38 @@ private StreamGraph transOperatoinsToStreamGraph(List modifyOpe } @Override - public JobPlanInfo getJobPlanInfo(List statements) { + public JobPlanInfo getJobPlanInfo(List statements) { return new JobPlanInfo(JsonPlanGenerator.generatePlan(getJobGraphFromInserts(statements))); } @Override - public StreamGraph getStreamGraphFromInserts(List statements) { + public StreamGraph getStreamGraphFromInserts(List statements) { List modifyOperations = new ArrayList<>(); - statements.stream().map(statement -> getParser().parse(statement)).forEach(operations -> { - if (operations.size() != 1) { - throw new TableException("Only single statement is supported."); - } - Operation operation = operations.get(0); - if (operation instanceof ModifyOperation) { - modifyOperations.add((ModifyOperation) operation); - } else { - throw new TableException("Only insert statement is supported now."); - } - }); - + statements.stream() + .map(statement -> getParser().parse(statement.getStatement())) + .forEach(operations -> { + if (operations.size() != 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + modifyOperations.add(getModifyOperation((CreateTableASOperation) operation)); + } else if (operation instanceof ReplaceTableAsOperation) { + modifyOperations.add(getModifyOperation((ReplaceTableAsOperation) operation)); + } else { + modifyOperations.add((ModifyOperation) operation); + } + } else if (operation instanceof QueryOperation) { + modifyOperations.add(new CollectModifyOperation((QueryOperation) operation)); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + } + }); + if (modifyOperations.isEmpty()) { + throw new TableException("Only insert statement is supported now. None operation to execute."); + } return transOperatoinsToStreamGraph(modifyOperations); } @@ -155,7 +197,6 @@ public void createCatalog(String catalogName, CatalogDescriptor catalogDescripto @Override public TableResultInternal executeCachedPlanInternal(CachedPlan cachedPlan) { - return null; } @@ -163,4 +204,196 @@ public TableResultInternal executeCachedPlanInternal(CachedPlan cachedPlan) { public SqlNode parseSql(String sql) { return ((ExtendedParser) getParser()).getCustomParser().parseSql(sql); } + + @Override + public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { + List operations = getParser().parse(statement); + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); + } + + Operation operation = operations.get(0); + SqlExplainResult data = new SqlExplainResult(); + data.setParseTrue(true); + data.setExplainTrue(true); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + operation = getModifyOperation((CreateTableASOperation) operation); + } else if (operation instanceof ReplaceTableAsOperation) { + operation = getModifyOperation((ReplaceTableAsOperation) operation); + } + data.setType("DML"); + } else if (operation instanceof QueryOperation) { + data.setType("DQL"); + } else { + data.setExplain(operation.asSummaryString()); + data.setType("DDL"); + return data; + } + + data.setExplain(getPlanner().explain(Collections.singletonList(operation), ExplainFormat.TEXT, extraDetails)); + return data; + } + + @Override + public SqlExplainResult explainStatementSet(List statements, ExplainDetail... extraDetails) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + List operations = new ArrayList<>(); + for (JobStatement statement : statements) { + List itemOperations = getParser().parse(statement.getStatement()); + if (!itemOperations.isEmpty()) { + for (Operation operation : itemOperations) { + if (operation instanceof CreateTableASOperation) { + operations.add(getModifyOperation((CreateTableASOperation) operation)); + } else if (operation instanceof ReplaceTableAsOperation) { + operations.add(getModifyOperation((ReplaceTableAsOperation) operation)); + } else { + operations.add(operation); + } + } + } + } + if (operations.isEmpty()) { + throw new DinkyException("None of the job in the statement set."); + } + resultBuilder.parseTrue(true); + resultBuilder.explain(getPlanner().explain(operations, ExplainFormat.TEXT, extraDetails)); + return resultBuilder + .explainTrue(true) + .explainTime(LocalDateTime.now()) + .type(SqlType.INSERT.getType()) + .build(); + } + + @Override + public TableResult executeStatementSet(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); + statements.removeIf(statement -> !statement.getSqlType().isSinkyModify()); + List modifyOperations = statements.stream() + .map(statement -> getModifyOperationFromInsert(statement.getStatement())) + .collect(Collectors.toList()); + return executeInternal(modifyOperations); + } + + public ModifyOperation getModifyOperationFromInsert(String statement) { + List operations = getParser().parse(statement); + if (operations.isEmpty()) { + throw new TableException("None of the statement is parsed."); + } + if (operations.size() > 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + return getModifyOperation((CreateTableASOperation) operation); + } else if (operation instanceof ReplaceTableAsOperation) { + return getModifyOperation((ReplaceTableAsOperation) operation); + } else { + return (ModifyOperation) operation; + } + } else if (operation instanceof QueryOperation) { + return new CollectModifyOperation((QueryOperation) operation); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + return null; + } + } + + private ModifyOperation getModifyOperation(CreateTableASOperation ctasOperation) { + CreateTableOperation createTableOperation = ctasOperation.getCreateTableOperation(); + ObjectIdentifier tableIdentifier = createTableOperation.getTableIdentifier(); + Catalog catalog = getCatalogManager().getCatalogOrThrowException(tableIdentifier.getCatalogName()); + ResolvedCatalogTable catalogTable = + getCatalogManager().resolveCatalogTable(createTableOperation.getCatalogTable()); + Optional stagingDynamicTableSink = + getSupportsStagingDynamicTableSink(createTableOperation, catalog, catalogTable); + if (stagingDynamicTableSink.isPresent()) { + // use atomic ctas + DynamicTableSink dynamicTableSink = stagingDynamicTableSink.get(); + return ctasOperation.toStagedSinkModifyOperation(tableIdentifier, catalogTable, catalog, dynamicTableSink); + } + // use non-atomic ctas, create table first + executeInternal(createTableOperation); + return ctasOperation.toSinkModifyOperation(getCatalogManager()); + } + + private ModifyOperation getModifyOperation(ReplaceTableAsOperation rtasOperation) { + CreateTableOperation createTableOperation = rtasOperation.getCreateTableOperation(); + ObjectIdentifier tableIdentifier = createTableOperation.getTableIdentifier(); + // First check if the replacedTable exists + Optional replacedTable = getCatalogManager().getTable(tableIdentifier); + if (!rtasOperation.isCreateOrReplace() && !replacedTable.isPresent()) { + throw new TableException(String.format( + "The table %s to be replaced doesn't exist. " + + "You can try to use CREATE TABLE AS statement or " + + "CREATE OR REPLACE TABLE AS statement.", + tableIdentifier)); + } + Catalog catalog = getCatalogManager().getCatalogOrThrowException(tableIdentifier.getCatalogName()); + ResolvedCatalogTable catalogTable = + getCatalogManager().resolveCatalogTable(createTableOperation.getCatalogTable()); + Optional stagingDynamicTableSink = + getSupportsStagingDynamicTableSink(createTableOperation, catalog, catalogTable); + if (stagingDynamicTableSink.isPresent()) { + // use atomic rtas + DynamicTableSink dynamicTableSink = stagingDynamicTableSink.get(); + SupportsStaging.StagingPurpose stagingPurpose = rtasOperation.isCreateOrReplace() + ? SupportsStaging.StagingPurpose.CREATE_OR_REPLACE_TABLE_AS + : SupportsStaging.StagingPurpose.REPLACE_TABLE_AS; + return rtasOperation.toStagedSinkModifyOperation(tableIdentifier, catalogTable, catalog, dynamicTableSink); + } + // non-atomic rtas drop table first if exists, then create + if (replacedTable.isPresent()) { + getCatalogManager().dropTable(tableIdentifier, false); + } + executeInternal(createTableOperation); + return rtasOperation.toSinkModifyOperation(getCatalogManager()); + } + + private Optional getSupportsStagingDynamicTableSink( + CreateTableOperation createTableOperation, Catalog catalog, ResolvedCatalogTable catalogTable) { + TableConfig tableConfig = getTableEnvironment().getConfig(); + boolean isStreamingMode = true; + RuntimeExecutionMode runtimeExecutionMode = + getStreamExecutionEnvironment().getConfiguration().get(ExecutionOptions.RUNTIME_MODE); + if (RuntimeExecutionMode.BATCH.equals(runtimeExecutionMode)) { + isStreamingMode = false; + } + if (tableConfig.get(TableConfigOptions.TABLE_RTAS_CTAS_ATOMICITY_ENABLED)) { + if (!TableFactoryUtil.isLegacyConnectorOptions( + catalog, + tableConfig, + isStreamingMode, + createTableOperation.getTableIdentifier(), + catalogTable, + createTableOperation.isTemporary())) { + try { + DynamicTableSink dynamicTableSink = ExecutableOperationUtils.createDynamicTableSink( + catalog, + () -> (new ModuleManager()).getFactory((Module::getTableSinkFactory)), + createTableOperation.getTableIdentifier(), + catalogTable, + Collections.emptyMap(), + tableConfig, + getUserClassLoader(), + createTableOperation.isTemporary()); + if (dynamicTableSink instanceof SupportsStaging) { + return Optional.of(dynamicTableSink); + } + } catch (Exception e) { + throw new TableException( + String.format( + "Fail to create DynamicTableSink for the table %s, " + + "maybe the table does not support atomicity of CTAS/RTAS, " + + "please set %s to false and try again.", + createTableOperation.getTableIdentifier(), + TableConfigOptions.TABLE_RTAS_CTAS_ATOMICITY_ENABLED.key()), + e); + } + } + } + return Optional.empty(); + } } diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java index 6414271715..cf1724512f 100644 --- a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java @@ -20,22 +20,14 @@ package org.dinky.executor; import org.dinky.data.model.LineageRel; -import org.dinky.data.result.SqlExplainResult; import org.dinky.utils.LineageContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.ExplainDetail; -import org.apache.flink.table.api.ExplainFormat; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl; import org.apache.flink.table.delegation.Planner; -import org.apache.flink.table.operations.ExplainOperation; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.operations.Operation; -import org.apache.flink.table.operations.QueryOperation; import java.util.List; @@ -85,36 +77,6 @@ public Configuration getRootConfiguration() { return (Configuration) this.getConfig().getRootConfiguration(); } - @Override - public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { - List operations = getParser().parse(statement); - if (operations.size() != 1) { - throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); - } - - Operation operation = operations.get(0); - SqlExplainResult data = new SqlExplainResult(); - data.setParseTrue(true); - data.setExplainTrue(true); - - if (operation instanceof ModifyOperation) { - data.setType("Modify DML"); - } else if (operation instanceof ExplainOperation) { - data.setType("Explain DML"); - } else if (operation instanceof QueryOperation) { - data.setType("Query DML"); - } else { - data.setExplain(operation.asSummaryString()); - data.setType("DDL"); - - // data.setExplain("DDL statement needn't comment。"); - return data; - } - - data.setExplain(getPlanner().explain(operations, ExplainFormat.TEXT, extraDetails)); - return data; - } - @Override public List getLineage(String statement) { LineageContext lineageContext = new LineageContext(this); diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java index a36e38fcc3..9f858cfcfe 100644 --- a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -19,11 +19,17 @@ package org.dinky.executor; +import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; +import org.dinky.data.result.SqlExplainResult; import org.dinky.operations.CustomNewParserImpl; import org.apache.calcite.sql.SqlNode; +import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.dag.Transformation; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ExecutionOptions; import org.apache.flink.configuration.PipelineOptions; import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator; import org.apache.flink.runtime.rest.messages.JobPlanInfo; @@ -31,16 +37,39 @@ import org.apache.flink.streaming.api.graph.JSONGenerator; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.ExplainDetail; +import org.apache.flink.table.api.ExplainFormat; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.internal.CachedPlan; import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.catalog.Catalog; import org.apache.flink.table.catalog.CatalogDescriptor; +import org.apache.flink.table.catalog.ContextResolvedTable; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsStaging; +import org.apache.flink.table.factories.TableFactoryUtil; +import org.apache.flink.table.module.Module; +import org.apache.flink.table.module.ModuleManager; +import org.apache.flink.table.operations.CollectModifyOperation; +import org.apache.flink.table.operations.CreateTableASOperation; import org.apache.flink.table.operations.ModifyOperation; import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.QueryOperation; +import org.apache.flink.table.operations.ReplaceTableAsOperation; +import org.apache.flink.table.operations.ddl.CreateTableOperation; +import org.apache.flink.table.operations.utils.ExecutableOperationUtils; +import java.time.LocalDateTime; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; import org.slf4j.Logger; @@ -126,25 +155,38 @@ private StreamGraph transOperatoinsToStreamGraph(List modifyOpe } @Override - public JobPlanInfo getJobPlanInfo(List statements) { + public JobPlanInfo getJobPlanInfo(List statements) { return new JobPlanInfo(JsonPlanGenerator.generatePlan(getJobGraphFromInserts(statements))); } @Override - public StreamGraph getStreamGraphFromInserts(List statements) { + public StreamGraph getStreamGraphFromInserts(List statements) { List modifyOperations = new ArrayList<>(); - statements.stream().map(statement -> getParser().parse(statement)).forEach(operations -> { - if (operations.size() != 1) { - throw new TableException("Only single statement is supported."); - } - Operation operation = operations.get(0); - if (operation instanceof ModifyOperation) { - modifyOperations.add((ModifyOperation) operation); - } else { - throw new TableException("Only insert statement is supported now."); - } - }); - + statements.stream() + .map(statement -> getParser().parse(statement.getStatement())) + .forEach(operations -> { + if (operations.size() != 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + modifyOperations.add(getModifyOperation((CreateTableASOperation) operation)); + } else if (operation instanceof ReplaceTableAsOperation) { + modifyOperations.add(getModifyOperation((ReplaceTableAsOperation) operation)); + } else { + modifyOperations.add((ModifyOperation) operation); + } + } else if (operation instanceof QueryOperation) { + modifyOperations.add(new CollectModifyOperation((QueryOperation) operation)); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + } + }); + if (modifyOperations.isEmpty()) { + throw new TableException("Only insert statement is supported now. None operation to execute."); + } return transOperatoinsToStreamGraph(modifyOperations); } @@ -155,7 +197,6 @@ public void createCatalog(String catalogName, CatalogDescriptor catalogDescripto @Override public TableResultInternal executeCachedPlanInternal(CachedPlan cachedPlan) { - return null; } @@ -163,4 +204,196 @@ public TableResultInternal executeCachedPlanInternal(CachedPlan cachedPlan) { public SqlNode parseSql(String sql) { return ((ExtendedParser) getParser()).getCustomParser().parseSql(sql); } + + @Override + public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { + List operations = getParser().parse(statement); + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); + } + + Operation operation = operations.get(0); + SqlExplainResult data = new SqlExplainResult(); + data.setParseTrue(true); + data.setExplainTrue(true); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + operation = getModifyOperation((CreateTableASOperation) operation); + } else if (operation instanceof ReplaceTableAsOperation) { + operation = getModifyOperation((ReplaceTableAsOperation) operation); + } + data.setType("DML"); + } else if (operation instanceof QueryOperation) { + data.setType("DQL"); + } else { + data.setExplain(operation.asSummaryString()); + data.setType("DDL"); + return data; + } + + data.setExplain(getPlanner().explain(Collections.singletonList(operation), ExplainFormat.TEXT, extraDetails)); + return data; + } + + @Override + public SqlExplainResult explainStatementSet(List statements, ExplainDetail... extraDetails) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + List operations = new ArrayList<>(); + for (JobStatement statement : statements) { + List itemOperations = getParser().parse(statement.getStatement()); + if (!itemOperations.isEmpty()) { + for (Operation operation : itemOperations) { + if (operation instanceof CreateTableASOperation) { + operations.add(getModifyOperation((CreateTableASOperation) operation)); + } else if (operation instanceof ReplaceTableAsOperation) { + operations.add(getModifyOperation((ReplaceTableAsOperation) operation)); + } else { + operations.add(operation); + } + } + } + } + if (operations.isEmpty()) { + throw new DinkyException("None of the job in the statement set."); + } + resultBuilder.parseTrue(true); + resultBuilder.explain(getPlanner().explain(operations, ExplainFormat.TEXT, extraDetails)); + return resultBuilder + .explainTrue(true) + .explainTime(LocalDateTime.now()) + .type(SqlType.INSERT.getType()) + .build(); + } + + @Override + public TableResult executeStatementSet(List statements) { + statements.removeIf(statement -> statement.getSqlType().equals(SqlType.RTAS)); + statements.removeIf(statement -> !statement.getSqlType().isSinkyModify()); + List modifyOperations = statements.stream() + .map(statement -> getModifyOperationFromInsert(statement.getStatement())) + .collect(Collectors.toList()); + return executeInternal(modifyOperations); + } + + public ModifyOperation getModifyOperationFromInsert(String statement) { + List operations = getParser().parse(statement); + if (operations.isEmpty()) { + throw new TableException("None of the statement is parsed."); + } + if (operations.size() > 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + if (operation instanceof CreateTableASOperation) { + return getModifyOperation((CreateTableASOperation) operation); + } else if (operation instanceof ReplaceTableAsOperation) { + return getModifyOperation((ReplaceTableAsOperation) operation); + } else { + return (ModifyOperation) operation; + } + } else if (operation instanceof QueryOperation) { + return new CollectModifyOperation((QueryOperation) operation); + } else { + log.info("Only insert statement or select is supported now. The statement is skipped: " + + operation.asSummaryString()); + return null; + } + } + + private ModifyOperation getModifyOperation(CreateTableASOperation ctasOperation) { + CreateTableOperation createTableOperation = ctasOperation.getCreateTableOperation(); + ObjectIdentifier tableIdentifier = createTableOperation.getTableIdentifier(); + Catalog catalog = getCatalogManager().getCatalogOrThrowException(tableIdentifier.getCatalogName()); + ResolvedCatalogTable catalogTable = + getCatalogManager().resolveCatalogTable(createTableOperation.getCatalogTable()); + Optional stagingDynamicTableSink = + getSupportsStagingDynamicTableSink(createTableOperation, catalog, catalogTable); + if (stagingDynamicTableSink.isPresent()) { + // use atomic ctas + DynamicTableSink dynamicTableSink = stagingDynamicTableSink.get(); + return ctasOperation.toStagedSinkModifyOperation(tableIdentifier, catalogTable, catalog, dynamicTableSink); + } + // use non-atomic ctas, create table first + executeInternal(createTableOperation); + return ctasOperation.toSinkModifyOperation(getCatalogManager()); + } + + private ModifyOperation getModifyOperation(ReplaceTableAsOperation rtasOperation) { + CreateTableOperation createTableOperation = rtasOperation.getCreateTableOperation(); + ObjectIdentifier tableIdentifier = createTableOperation.getTableIdentifier(); + // First check if the replacedTable exists + Optional replacedTable = getCatalogManager().getTable(tableIdentifier); + if (!rtasOperation.isCreateOrReplace() && !replacedTable.isPresent()) { + throw new TableException(String.format( + "The table %s to be replaced doesn't exist. " + + "You can try to use CREATE TABLE AS statement or " + + "CREATE OR REPLACE TABLE AS statement.", + tableIdentifier)); + } + Catalog catalog = getCatalogManager().getCatalogOrThrowException(tableIdentifier.getCatalogName()); + ResolvedCatalogTable catalogTable = + getCatalogManager().resolveCatalogTable(createTableOperation.getCatalogTable()); + Optional stagingDynamicTableSink = + getSupportsStagingDynamicTableSink(createTableOperation, catalog, catalogTable); + if (stagingDynamicTableSink.isPresent()) { + // use atomic rtas + DynamicTableSink dynamicTableSink = stagingDynamicTableSink.get(); + SupportsStaging.StagingPurpose stagingPurpose = rtasOperation.isCreateOrReplace() + ? SupportsStaging.StagingPurpose.CREATE_OR_REPLACE_TABLE_AS + : SupportsStaging.StagingPurpose.REPLACE_TABLE_AS; + return rtasOperation.toStagedSinkModifyOperation(tableIdentifier, catalogTable, catalog, dynamicTableSink); + } + // non-atomic rtas drop table first if exists, then create + if (replacedTable.isPresent()) { + getCatalogManager().dropTable(tableIdentifier, false); + } + executeInternal(createTableOperation); + return rtasOperation.toSinkModifyOperation(getCatalogManager()); + } + + private Optional getSupportsStagingDynamicTableSink( + CreateTableOperation createTableOperation, Catalog catalog, ResolvedCatalogTable catalogTable) { + TableConfig tableConfig = getTableEnvironment().getConfig(); + boolean isStreamingMode = true; + RuntimeExecutionMode runtimeExecutionMode = + getStreamExecutionEnvironment().getConfiguration().get(ExecutionOptions.RUNTIME_MODE); + if (RuntimeExecutionMode.BATCH.equals(runtimeExecutionMode)) { + isStreamingMode = false; + } + if (tableConfig.get(TableConfigOptions.TABLE_RTAS_CTAS_ATOMICITY_ENABLED)) { + if (!TableFactoryUtil.isLegacyConnectorOptions( + catalog, + tableConfig, + isStreamingMode, + createTableOperation.getTableIdentifier(), + catalogTable, + createTableOperation.isTemporary())) { + try { + DynamicTableSink dynamicTableSink = ExecutableOperationUtils.createDynamicTableSink( + catalog, + () -> (new ModuleManager()).getFactory((Module::getTableSinkFactory)), + createTableOperation.getTableIdentifier(), + catalogTable, + Collections.emptyMap(), + tableConfig, + getUserClassLoader(), + createTableOperation.isTemporary()); + if (dynamicTableSink instanceof SupportsStaging) { + return Optional.of(dynamicTableSink); + } + } catch (Exception e) { + throw new TableException( + String.format( + "Fail to create DynamicTableSink for the table %s, " + + "maybe the table does not support atomicity of CTAS/RTAS, " + + "please set %s to false and try again.", + createTableOperation.getTableIdentifier(), + TableConfigOptions.TABLE_RTAS_CTAS_ATOMICITY_ENABLED.key()), + e); + } + } + } + return Optional.empty(); + } } diff --git a/dinky-client/dinky-client-base/src/main/java/org/dinky/executor/CustomTableEnvironment.java b/dinky-client/dinky-client-base/src/main/java/org/dinky/executor/CustomTableEnvironment.java index 13bc6b49ad..ea803ca137 100644 --- a/dinky-client/dinky-client-base/src/main/java/org/dinky/executor/CustomTableEnvironment.java +++ b/dinky-client/dinky-client-base/src/main/java/org/dinky/executor/CustomTableEnvironment.java @@ -19,6 +19,7 @@ package org.dinky.executor; +import org.dinky.data.job.JobStatement; import org.dinky.data.model.LineageRel; import org.dinky.data.result.SqlExplainResult; @@ -31,6 +32,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.graph.StreamGraph; import org.apache.flink.table.api.ExplainDetail; +import org.apache.flink.table.api.TableResult; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.table.api.internal.TableEnvironmentInternal; import org.apache.flink.table.delegation.Planner; @@ -57,12 +59,16 @@ public interface CustomTableEnvironment ObjectNode getStreamGraph(String statement); - JobPlanInfo getJobPlanInfo(List statements); + JobPlanInfo getJobPlanInfo(List statements); - StreamGraph getStreamGraphFromInserts(List statements); + StreamGraph getStreamGraphFromInserts(List statements); SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails); + SqlExplainResult explainStatementSet(List statements, ExplainDetail... extraDetails); + + TableResult executeStatementSet(List statements); + StreamExecutionEnvironment getStreamExecutionEnvironment(); Planner getPlanner(); @@ -73,7 +79,7 @@ public interface CustomTableEnvironment SqlNode parseSql(String sql); - default JobGraph getJobGraphFromInserts(List statements) { + default JobGraph getJobGraphFromInserts(List statements) { return getStreamGraphFromInserts(statements).getJobGraph(); } diff --git a/dinky-common/src/main/java/org/dinky/data/job/JobStatement.java b/dinky-common/src/main/java/org/dinky/data/job/JobStatement.java new file mode 100644 index 0000000000..e9492dc08b --- /dev/null +++ b/dinky-common/src/main/java/org/dinky/data/job/JobStatement.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.dinky.data.job; + +public class JobStatement { + + private int index; + private String statement; + private JobStatementType statementType; + private SqlType sqlType; + private boolean isGenerated; + private boolean isFinalExecutableStatement; + private boolean isFinalCreateFunctionStatement; + + public JobStatement( + int index, String statement, JobStatementType statementType, SqlType sqlType, boolean isGenerated) { + this.index = index; + this.statement = statement; + this.statementType = statementType; + this.sqlType = sqlType; + this.isGenerated = isGenerated; + this.isFinalExecutableStatement = false; + this.isFinalCreateFunctionStatement = false; + } + + public JobStatement(int index, String statement, JobStatementType statementType, boolean isGenerated) { + this.index = index; + this.statement = statement; + this.statementType = statementType; + this.isGenerated = isGenerated; + this.isFinalExecutableStatement = false; + this.isFinalCreateFunctionStatement = false; + } + + public int getIndex() { + return index; + } + + public String getStatement() { + return statement; + } + + public JobStatementType getStatementType() { + return statementType; + } + + public SqlType getSqlType() { + return sqlType; + } + + public boolean isGenerated() { + return isGenerated; + } + + public boolean isFinalExecutableStatement() { + return isFinalExecutableStatement; + } + + public void asFinalExecutableStatement() { + isFinalExecutableStatement = true; + } + + public boolean isFinalCreateFunctionStatement() { + return isFinalCreateFunctionStatement; + } + + public void asFinalCreateFunctionStatement() { + isFinalCreateFunctionStatement = true; + } + + public static JobStatement generateJobStatement( + int index, String statement, JobStatementType statementType, SqlType sqlType) { + return new JobStatement(index, statement, statementType, sqlType, true); + } +} diff --git a/dinky-common/src/main/java/org/dinky/data/job/JobStatementType.java b/dinky-common/src/main/java/org/dinky/data/job/JobStatementType.java new file mode 100644 index 0000000000..1ca6c37e68 --- /dev/null +++ b/dinky-common/src/main/java/org/dinky/data/job/JobStatementType.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.dinky.data.job; + +public enum JobStatementType { + SET, + DDL, + SQL, + PIPELINE, +} diff --git a/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/SqlCategory.java b/dinky-common/src/main/java/org/dinky/data/job/SqlCategory.java similarity index 97% rename from dinky-client/dinky-client-base/src/main/java/org/dinky/parser/SqlCategory.java rename to dinky-common/src/main/java/org/dinky/data/job/SqlCategory.java index 4d546f5cb9..9a6aa3ceb8 100644 --- a/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/SqlCategory.java +++ b/dinky-common/src/main/java/org/dinky/data/job/SqlCategory.java @@ -17,7 +17,7 @@ * */ -package org.dinky.parser; +package org.dinky.data.job; /** * SqlCategory diff --git a/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/SqlType.java b/dinky-common/src/main/java/org/dinky/data/job/SqlType.java similarity index 73% rename from dinky-client/dinky-client-base/src/main/java/org/dinky/parser/SqlType.java rename to dinky-common/src/main/java/org/dinky/data/job/SqlType.java index c5e55d43c8..61836ce8cb 100644 --- a/dinky-client/dinky-client-base/src/main/java/org/dinky/parser/SqlType.java +++ b/dinky-common/src/main/java/org/dinky/data/job/SqlType.java @@ -17,7 +17,7 @@ * */ -package org.dinky.parser; +package org.dinky.data.job; import java.util.List; import java.util.regex.Pattern; @@ -30,16 +30,22 @@ * @since 2021/7/3 11:11 */ public enum SqlType { - SELECT("SELECT", "^SELECT.*", SqlCategory.DQL), + SELECT("SELECT", "^\\(*\\s*SELECT.*", SqlCategory.DQL), CREATE("CREATE", "^CREATE(?!\\s+TABLE.*AS SELECT).*$", SqlCategory.DDL), DROP("DROP", "^DROP.*", SqlCategory.DDL), + TRUNCATE("TRUNCATE ", "^TRUNCATE .*", SqlCategory.DDL), + ALTER("ALTER", "^ALTER.*", SqlCategory.DDL), INSERT("INSERT", "^INSERT.*", SqlCategory.DML), + UPDATE("UPDATE", "^UPDATE.*", SqlCategory.DML), + + DELETE("DELETE", "^DELETE.*", SqlCategory.DML), + DESC("DESC", "^DESC.*", SqlCategory.DDL), DESCRIBE("DESCRIBE", "^DESCRIBE.*", SqlCategory.DDL), @@ -50,6 +56,8 @@ public enum SqlType { SHOW("SHOW", "^SHOW.*", SqlCategory.DDL), + ANALYZE("ANALYZE ", "^ANALYZE.*", SqlCategory.DDL), + LOAD("LOAD", "^LOAD.*", SqlCategory.DDL), UNLOAD("UNLOAD", "^UNLOAD.*", SqlCategory.DDL), @@ -58,15 +66,23 @@ public enum SqlType { RESET("RESET", "^RESET.*", SqlCategory.DDL), - EXECUTE("EXECUTE", "^EXECUTE.*", SqlCategory.DQL), + EXECUTE("EXECUTE", "^EXECUTE.*", SqlCategory.DML), ADD_JAR("ADD_JAR", "^ADD\\s+JAR\\s+\\S+", SqlCategory.DDL), + ADD("ADD", "^ADD\\s+CUSTOMJAR\\s+\\S+", SqlCategory.DDL), + ADD_FILE("ADD_FILE", "^ADD\\s+FILE\\s+\\S+", SqlCategory.DDL), PRINT("PRINT", "^PRINT.*", SqlCategory.DQL), - CTAS("CTAS", "^CREATE\\s.*AS\\sSELECT.*$", SqlCategory.DDL), + REMOVE("REMOVE", "^REMOVE.*", SqlCategory.DDL), + + STOP("STOP", "^STOP.*", SqlCategory.DDL), + + CTAS("CTAS", "^CREATE\\s+TABLE.*AS\\sSELECT.*$", SqlCategory.DDL), + + RTAS("RTAS", "^REPLACE\\s.*AS\\sSELECT.*$", SqlCategory.DML), WITH("WITH", "^WITH.*", SqlCategory.DQL), @@ -77,9 +93,13 @@ public enum SqlType { private SqlCategory category; private static final List TRANS_SQL_TYPES = - Lists.newArrayList(INSERT, SELECT, WITH, SHOW, DESCRIBE, DESC, CTAS); + Lists.newArrayList(INSERT, SELECT, WITH, SHOW, DESCRIBE, DESC, CTAS, RTAS, UPDATE, DELETE); + + private static final List CTAS_TYPES = Lists.newArrayList(CTAS, RTAS, PRINT); - private static final List PIPELINE_SQL_TYPES = Lists.newArrayList(INSERT, SELECT, WITH, CTAS); + private static final List PIPELINE_SQL_TYPES = Lists.newArrayList(INSERT, SELECT, WITH, CTAS, RTAS, PRINT); + + private static final List SINK_MODIFY_SQL_TYPES = Lists.newArrayList(INSERT, CTAS, RTAS, PRINT); SqlType(String type, String regrex, SqlCategory category) { this.type = type; @@ -110,4 +130,12 @@ public static List getTransSqlTypes() { public boolean isPipeline() { return PIPELINE_SQL_TYPES.contains(this); } + + public boolean isSinkyModify() { + return SINK_MODIFY_SQL_TYPES.contains(this); + } + + public boolean isCTAS() { + return CTAS_TYPES.contains(this); + } } diff --git a/dinky-common/src/main/java/org/dinky/data/model/SystemConfiguration.java b/dinky-common/src/main/java/org/dinky/data/model/SystemConfiguration.java index a82b233e36..539ca363a0 100644 --- a/dinky-common/src/main/java/org/dinky/data/model/SystemConfiguration.java +++ b/dinky-common/src/main/java/org/dinky/data/model/SystemConfiguration.java @@ -19,6 +19,7 @@ package org.dinky.data.model; +import org.dinky.assertion.Asserts; import org.dinky.context.EngineContextHolder; import org.dinky.data.constant.CommonConstant; import org.dinky.data.constant.DirConstant; @@ -387,7 +388,7 @@ public Map>> getAllConfiguration() { } public boolean isUseRestAPI() { - return useRestAPI.getValue(); + return Asserts.isNull(useRestAPI.getValue()) ? useRestAPI.getDefaultValue() : useRestAPI.getValue(); } public int getJobIdWait() { diff --git a/dinky-core/src/main/java/com/ververica/cdc/composer/flink/FlinkPipelineExecution.java b/dinky-core/src/main/java/com/ververica/cdc/composer/flink/FlinkPipelineExecution.java deleted file mode 100644 index 52e5f04e4f..0000000000 --- a/dinky-core/src/main/java/com/ververica/cdc/composer/flink/FlinkPipelineExecution.java +++ /dev/null @@ -1,62 +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.cdc.composer.flink; - -import org.apache.flink.cdc.composer.PipelineExecution; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; - -/** - * A pipeline execution that run the defined pipeline via Flink's {@link - * StreamExecutionEnvironment}. - */ -public class FlinkPipelineExecution implements PipelineExecution { - - private final StreamExecutionEnvironment env; - private final String jobName; - private final boolean isBlocking; - - public FlinkPipelineExecution(StreamExecutionEnvironment env, String jobName, boolean isBlocking) { - this.env = env; - this.jobName = jobName; - this.isBlocking = isBlocking; - } - - public StreamExecutionEnvironment getEnv() { - return env; - } - - public String getJobName() { - return jobName; - } - - public boolean isBlocking() { - return isBlocking; - } - - @Override - public ExecutionInfo execute() throws Exception { - JobClient jobClient = env.executeAsync(jobName); - if (isBlocking) { - jobClient.getJobExecutionResult().get(); - } - return new ExecutionInfo(jobClient.getJobID().toString(), jobName); - } -} diff --git a/dinky-core/src/main/java/org/dinky/data/result/ResultBuilder.java b/dinky-core/src/main/java/org/dinky/data/result/ResultBuilder.java index faae72a2ac..49973e1678 100644 --- a/dinky-core/src/main/java/org/dinky/data/result/ResultBuilder.java +++ b/dinky-core/src/main/java/org/dinky/data/result/ResultBuilder.java @@ -19,8 +19,8 @@ package org.dinky.data.result; +import org.dinky.data.job.SqlType; import org.dinky.job.JobHandler; -import org.dinky.parser.SqlType; import org.apache.flink.table.api.TableResult; diff --git a/dinky-core/src/main/java/org/dinky/executor/Executor.java b/dinky-core/src/main/java/org/dinky/executor/Executor.java index a055b0dc2a..d1c42d9959 100644 --- a/dinky-core/src/main/java/org/dinky/executor/Executor.java +++ b/dinky-core/src/main/java/org/dinky/executor/Executor.java @@ -22,6 +22,7 @@ import org.dinky.assertion.Asserts; import org.dinky.classloader.DinkyClassLoader; import org.dinky.context.CustomTableEnvironmentContext; +import org.dinky.data.job.JobStatement; import org.dinky.data.model.LineageRel; import org.dinky.data.result.SqlExplainResult; import org.dinky.interceptor.FlinkInterceptor; @@ -246,11 +247,11 @@ public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extr return SqlExplainResult.INVALID_EXPLAIN_RESULT; } - public StreamGraph getStreamGraphFromStatement(List statements) { + public StreamGraph getStreamGraphFromStatement(List statements) { return tableEnvironment.getStreamGraphFromInserts(statements); } - public ObjectNode getStreamGraph(List statements) { + public ObjectNode getStreamGraph(List statements) { StreamGraph streamGraph = tableEnvironment.getStreamGraphFromInserts(statements); return getStreamGraphJsonNode(streamGraph); } @@ -282,7 +283,7 @@ public ObjectNode getStreamGraphFromDataStream(List statements) { return getStreamGraphJsonNode(getStreamGraph()); } - public JobPlanInfo getJobPlanInfo(List statements) { + public JobPlanInfo getJobPlanInfo(List statements) { return tableEnvironment.getJobPlanInfo(statements); } @@ -292,7 +293,7 @@ public JobPlanInfo getJobPlanInfoFromDataStream(List statements) { return new JobPlanInfo(JsonPlanGenerator.generatePlan(streamGraph.getJobGraph())); } - public JobGraph getJobGraphFromInserts(List statements) { + public JobGraph getJobGraphFromInserts(List statements) { return tableEnvironment.getJobGraphFromInserts(statements); } @@ -302,10 +303,12 @@ public TableResult executeStatementSet(List statements) { return statementSet.execute(); } - public String explainStatementSet(List statements) { - StatementSet statementSet = tableEnvironment.createStatementSet(); - statements.forEach(statementSet::addInsertSql); - return statementSet.explain(); + public TableResult executeStatements(List statements) { + return tableEnvironment.executeStatementSet(statements); + } + + public SqlExplainResult explainStatementSet(List statements) { + return tableEnvironment.explainStatementSet(statements); } public List getLineage(String statement) { diff --git a/dinky-core/src/main/java/org/dinky/explainer/Explainer.java b/dinky-core/src/main/java/org/dinky/explainer/Explainer.java index 75e94d53b3..3203b9ac1c 100644 --- a/dinky-core/src/main/java/org/dinky/explainer/Explainer.java +++ b/dinky-core/src/main/java/org/dinky/explainer/Explainer.java @@ -22,6 +22,9 @@ import org.dinky.assertion.Asserts; import org.dinky.data.enums.GatewayType; import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.JobStatementType; +import org.dinky.data.job.SqlType; import org.dinky.data.model.LineageRel; import org.dinky.data.result.ExplainResult; import org.dinky.data.result.SqlExplainResult; @@ -34,33 +37,24 @@ import org.dinky.job.JobConfig; import org.dinky.job.JobManager; import org.dinky.job.JobParam; -import org.dinky.job.StatementParam; -import org.dinky.job.builder.JobDDLBuilder; -import org.dinky.job.builder.JobExecuteBuilder; -import org.dinky.job.builder.JobTransBuilder; +import org.dinky.job.JobRunnerFactory; +import org.dinky.job.JobStatementPlan; import org.dinky.job.builder.JobUDFBuilder; -import org.dinky.parser.SqlType; import org.dinky.trans.Operations; import org.dinky.utils.DinkyClassLoaderUtil; -import org.dinky.utils.IpUtil; import org.dinky.utils.SqlUtil; import org.apache.flink.runtime.rest.messages.JobPlanInfo; +import org.apache.flink.streaming.api.graph.JSONGenerator; +import org.apache.flink.streaming.api.graph.StreamGraph; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import com.google.common.collect.Sets; -import cn.hutool.core.collection.CollUtil; -import cn.hutool.core.text.StrBuilder; import cn.hutool.core.util.StrUtil; import lombok.extern.slf4j.Slf4j; @@ -74,7 +68,6 @@ public class Explainer { private Executor executor; private boolean useStatementSet; - private ObjectMapper mapper = new ObjectMapper(); private JobManager jobManager; public Explainer(Executor executor, boolean useStatementSet, JobManager jobManager) { @@ -83,6 +76,10 @@ public Explainer(Executor executor, boolean useStatementSet, JobManager jobManag this.jobManager = jobManager; } + public static Explainer build(JobManager jobManager) { + return new Explainer(jobManager.getExecutor(), true, jobManager); + } + public static Explainer build(Executor executor, boolean useStatementSet, JobManager jobManager) { return new Explainer(executor, useStatementSet, jobManager); } @@ -100,57 +97,45 @@ public Explainer initialize(JobConfig config, String statement) { return this; } - public JobParam pretreatStatements(String[] statements) { - List ddl = new ArrayList<>(); - List trans = new ArrayList<>(); - List execute = new ArrayList<>(); - List statementList = new ArrayList<>(); - List udfList = new ArrayList<>(); - StrBuilder parsedSql = new StrBuilder(); - - List statementsWithUdf = Arrays.stream(statements).collect(Collectors.toList()); + public JobStatementPlan parseStatements(String[] statements) { + JobStatementPlan jobStatementPlan = new JobStatementPlan(); + List udfStatements = new ArrayList<>(); Optional.ofNullable(jobManager.getConfig().getUdfRefer()) .ifPresent(t -> t.forEach((key, value) -> { String sql = String.format("create temporary function %s as '%s'", value, key); - statementsWithUdf.add(0, sql); + udfStatements.add(sql); })); - - List transSqlTypes = SqlType.getTransSqlTypes(); - Set transSqlTypeSet = Sets.newHashSet(transSqlTypes); - for (String item : statementsWithUdf) { + for (String udfStatement : udfStatements) { + jobStatementPlan.addJobStatementGenerated(udfStatement, JobStatementType.DDL, SqlType.CREATE); + } + for (String item : statements) { String statement = executor.pretreatStatement(item); - parsedSql.append(statement).append(";\n"); if (statement.isEmpty()) { continue; } SqlType operationType = Operations.getOperationType(statement); - if (transSqlTypeSet.contains(operationType)) { - trans.add(new StatementParam(statement, operationType)); - statementList.add(statement); + if (operationType.equals(SqlType.SET) || operationType.equals(SqlType.RESET)) { + jobStatementPlan.addJobStatement(statement, JobStatementType.SET, operationType); } else if (operationType.equals(SqlType.EXECUTE)) { - execute.add(new StatementParam(statement, operationType)); + jobStatementPlan.addJobStatement(statement, JobStatementType.PIPELINE, operationType); } else if (operationType.equals(SqlType.PRINT)) { - Map config = this.executor.getExecutorConfig().getConfig(); - String host = config.getOrDefault("dinky.dinkyHost", IpUtil.getHostIp()); - int port = Integer.parseInt(config.getOrDefault("dinky.dinkyPrintPort", "7125")); - String[] tableNames = PrintStatementExplainer.getTableNames(statement); - for (String tableName : tableNames) { - trans.add(new StatementParam( - PrintStatementExplainer.getCreateStatement(tableName, host, port), SqlType.CTAS)); + for (String tableName : PrintStatementExplainer.getTableNames(statement)) { + jobStatementPlan.addJobStatement( + PrintStatementExplainer.getCreateStatement( + tableName, this.executor.getExecutorConfig().getConfig()), + JobStatementType.SQL, + operationType); } + } else if (SqlType.getTransSqlTypes().contains(operationType)) { + jobStatementPlan.addJobStatement(statement, JobStatementType.SQL, operationType); } else { - ddl.add(new StatementParam(statement, operationType)); - statementList.add(statement); + jobStatementPlan.addJobStatement(statement, JobStatementType.DDL, operationType); } } - JobParam jobParam = - new JobParam(statementList, ddl, trans, execute, CollUtil.removeNull(udfList), parsedSql.toString()); - - MockStatementExplainer.build(executor.getCustomTableEnvironment()) - .isMockSink(jobManager.getConfig().isMockSinkFunction()) - .jobParamMock(jobParam); - - return jobParam; + if (jobManager.getConfig().isMockSinkFunction()) { + MockStatementExplainer.build(executor.getCustomTableEnvironment()).jobStatementPlanMock(jobStatementPlan); + } + return jobStatementPlan; } public List parseUDFFromStatements(String[] statements) { @@ -169,39 +154,27 @@ public List parseUDFFromStatements(String[] statements) { public ExplainResult explainSql(String statement) { log.info("Start explain FlinkSQL..."); - JobParam jobParam; + JobStatementPlan jobStatementPlan; List sqlExplainRecords = new ArrayList<>(); boolean correct = true; try { - jobParam = pretreatStatements(SqlUtil.getStatements(statement)); - jobManager.setJobParam(jobParam); + jobStatementPlan = parseStatements(SqlUtil.getStatements(statement)); + jobStatementPlan.buildFinalStatement(); + jobManager.setJobStatementPlan(jobStatementPlan); } catch (Exception e) { SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); resultBuilder.error(e.getMessage()).parseTrue(false); sqlExplainRecords.add(resultBuilder.build()); - log.error("Failed to pretreat statements:", e); + log.error("Failed parseStatements:", e); return new ExplainResult(false, sqlExplainRecords.size(), sqlExplainRecords); } - // step 1: explain and execute ddl - List ddlSqlExplainResults = - JobDDLBuilder.build(jobManager).explain(); - sqlExplainRecords.addAll(ddlSqlExplainResults); - for (SqlExplainResult item : ddlSqlExplainResults) { - if (!item.isParseTrue() || !item.isExplainTrue()) { - correct = false; - } - } - if (correct && !jobParam.getTrans().isEmpty()) { - // step 2: explain modifyOptions - sqlExplainRecords.addAll(JobTransBuilder.build(jobManager).explain()); - // step 3: explain pipeline - sqlExplainRecords.addAll(JobExecuteBuilder.build(jobManager).explain()); - } - int index = 1; - for (SqlExplainResult item : sqlExplainRecords) { - item.setIndex(index++); - if (!item.isParseTrue() || !item.isExplainTrue()) { - correct = false; + JobRunnerFactory jobRunnerFactory = JobRunnerFactory.create(jobManager); + for (JobStatement jobStatement : jobStatementPlan.getJobStatementList()) { + SqlExplainResult sqlExplainResult = jobRunnerFactory + .getJobRunner(jobStatement.getStatementType()) + .explain(jobStatement); + if (!sqlExplainResult.isInvalid()) { + sqlExplainRecords.add(sqlExplainResult); } } log.info(StrUtil.format("A total of {} FlinkSQL have been Explained.", sqlExplainRecords.size())); @@ -209,45 +182,46 @@ public ExplainResult explainSql(String statement) { } public ObjectNode getStreamGraph(String statement) { - try { - JobParam jobParam = pretreatStatements(SqlUtil.getStatements(statement)); - jobManager.setJobParam(jobParam); - // step 1: execute ddl - JobDDLBuilder.build(jobManager).run(); - // step 2: get the stream graph of trans - if (!jobParam.getTrans().isEmpty()) { - return executor.getStreamGraphJsonNode( - JobTransBuilder.build(jobManager).getStreamGraph()); - } - // step 3: get the stream graph of pipeline - if (!jobParam.getExecute().isEmpty()) { - return executor.getStreamGraphJsonNode( - JobExecuteBuilder.build(jobManager).getStreamGraph()); + log.info("Start explain FlinkSQL..."); + JobStatementPlan jobStatementPlan = parseStatements(SqlUtil.getStatements(statement)); + jobStatementPlan.buildFinalStatement(); + log.info("Explain FlinkSQL successful"); + JobRunnerFactory jobRunnerFactory = JobRunnerFactory.create(jobManager); + for (JobStatement jobStatement : jobStatementPlan.getJobStatementList()) { + StreamGraph streamGraph = jobRunnerFactory + .getJobRunner(jobStatement.getStatementType()) + .getStreamGraph(jobStatement); + if (Asserts.isNotNull(streamGraph)) { + JSONGenerator jsonGenerator = new JSONGenerator(streamGraph); + String json = jsonGenerator.getJSON(); + ObjectMapper mapper = new ObjectMapper(); + ObjectNode objectNode = mapper.createObjectNode(); + try { + objectNode = (ObjectNode) mapper.readTree(json); + } catch (Exception e) { + log.error("Get stream graph json node error.", e); + } + return objectNode; } - } catch (Exception e) { - throw new RuntimeException(e); } - return mapper.createObjectNode(); + throw new DinkyException("None of the StreamGraph were found."); } public JobPlanInfo getJobPlanInfo(String statement) { - try { - JobParam jobParam = pretreatStatements(SqlUtil.getStatements(statement)); - jobManager.setJobParam(jobParam); - // step 1: execute ddl - JobDDLBuilder.build(jobManager).run(); - // step 2: get the job plan info of trans - if (!jobParam.getTrans().isEmpty()) { - return JobTransBuilder.build(jobManager).getJobPlanInfo(); - } - // step 3: get the job plan info of pipeline - if (!jobParam.getExecute().isEmpty()) { - return JobExecuteBuilder.build(jobManager).getJobPlanInfo(); + log.info("Start explain FlinkSQL..."); + JobStatementPlan jobStatementPlan = parseStatements(SqlUtil.getStatements(statement)); + jobStatementPlan.buildFinalStatement(); + log.info("Explain FlinkSQL successful"); + JobRunnerFactory jobRunnerFactory = JobRunnerFactory.create(jobManager); + for (JobStatement jobStatement : jobStatementPlan.getJobStatementList()) { + JobPlanInfo jobPlanInfo = jobRunnerFactory + .getJobRunner(jobStatement.getStatementType()) + .getJobPlanInfo(jobStatement); + if (Asserts.isNotNull(jobPlanInfo)) { + return jobPlanInfo; } - } catch (Exception e) { - throw new RuntimeException(e); } - throw new RuntimeException("Creating job plan fails because this job doesn't contain an insert statement."); + throw new DinkyException("None of the JobPlanInfo were found."); } public List getLineage(String statement) { diff --git a/dinky-core/src/main/java/org/dinky/explainer/lineage/LineageBuilder.java b/dinky-core/src/main/java/org/dinky/explainer/lineage/LineageBuilder.java index 903671c3f7..8dcb03e069 100644 --- a/dinky-core/src/main/java/org/dinky/explainer/lineage/LineageBuilder.java +++ b/dinky-core/src/main/java/org/dinky/explainer/lineage/LineageBuilder.java @@ -19,14 +19,14 @@ package org.dinky.explainer.lineage; -import org.dinky.classloader.DinkyClassLoader; import org.dinky.data.model.LineageRel; +import org.dinky.executor.Executor; import org.dinky.executor.ExecutorConfig; import org.dinky.executor.ExecutorFactory; import org.dinky.explainer.Explainer; +import org.dinky.job.JobConfig; import org.dinky.job.JobManager; -import java.lang.ref.WeakReference; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -39,11 +39,21 @@ */ public class LineageBuilder { + public static LineageResult getColumnLineageByLogicalPlan(String statement, JobConfig jobConfig) { + JobManager jobManager = JobManager.buildPlanMode(jobConfig); + Explainer explainer = new Explainer(jobManager.getExecutor(), false, jobManager); + return getColumnLineageByLogicalPlan(statement, explainer); + } + public static LineageResult getColumnLineageByLogicalPlan(String statement, ExecutorConfig executorConfig) { - Explainer explainer = new Explainer( - ExecutorFactory.buildExecutor(executorConfig, new WeakReference<>(DinkyClassLoader.build()).get()), - false, - new JobManager()); + JobManager jobManager = JobManager.buildPlanMode(JobConfig.buildPlanConfig()); + Executor executor = ExecutorFactory.buildExecutor(executorConfig, jobManager.getDinkyClassLoader()); + jobManager.setExecutor(executor); + Explainer explainer = new Explainer(executor, false, jobManager); + return getColumnLineageByLogicalPlan(statement, explainer); + } + + public static LineageResult getColumnLineageByLogicalPlan(String statement, Explainer explainer) { List lineageRelList = explainer.getLineage(statement); List relations = new ArrayList<>(); Map tableMap = new HashMap<>(); diff --git a/dinky-core/src/main/java/org/dinky/explainer/mock/MockStatementExplainer.java b/dinky-core/src/main/java/org/dinky/explainer/mock/MockStatementExplainer.java index c6ad11ce17..ad9a316fae 100644 --- a/dinky-core/src/main/java/org/dinky/explainer/mock/MockStatementExplainer.java +++ b/dinky-core/src/main/java/org/dinky/explainer/mock/MockStatementExplainer.java @@ -20,10 +20,11 @@ package org.dinky.explainer.mock; import org.dinky.connector.mock.sink.MockDynamicTableSinkFactory; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.JobStatementType; +import org.dinky.data.job.SqlType; import org.dinky.executor.CustomTableEnvironment; -import org.dinky.job.JobParam; -import org.dinky.job.StatementParam; -import org.dinky.parser.SqlType; +import org.dinky.job.JobStatementPlan; import org.dinky.utils.JsonUtils; import org.apache.calcite.config.Lex; @@ -50,7 +51,6 @@ public class MockStatementExplainer { // Because calcite cannot parse flink sql ddl, a table environment is designed here for flink sql ddl pars private final CustomTableEnvironment tableEnv; - private boolean isMockSink = false; private final SqlParser.Config calciteConfig; private final String DROP_TABLE_SQL_TEMPLATE = "DROP TABLE IF EXISTS {0}"; private final String MOCK_SQL_TEMPLATE = "CREATE TABLE {0} ({1}) WITH ({2})"; @@ -64,65 +64,57 @@ public MockStatementExplainer(CustomTableEnvironment tableEnv) { this.calciteConfig = SqlParser.config().withLex(Lex.JAVA); } - public MockStatementExplainer isMockSink(boolean isMockSink) { - this.isMockSink = isMockSink; - return this; - } - - public void jobParamMock(JobParam jobParam) { - if (isMockSink) { - mockSink(jobParam); - } + public void jobStatementPlanMock(JobStatementPlan jobStatementPlan) { + mockSink(jobStatementPlan); } /** * The connector of insert tables will be changed to {@link MockDynamicTableSinkFactory} * - * @param jobParam job param + * @param jobStatementPlan JobStatementPlan */ - private void mockSink(JobParam jobParam) { + private void mockSink(JobStatementPlan jobStatementPlan) { // Get table names that need to be mocked, and modify insert statement. - Set tablesNeedMock = getTableNamesNeedMockAndModifyTrans(jobParam); + Set tablesNeedMock = getTableNamesNeedMockAndModifyTrans(jobStatementPlan); // mock insert table ddl - List mockedDdl = new ArrayList<>(); - for (StatementParam ddl : jobParam.getDdl()) { - SqlNode sqlNode = tableEnv.parseSql(ddl.getValue()); - boolean isDdlMocked = false; + List jobStatementList = jobStatementPlan.getJobStatementList(); + for (int i = 0; i < jobStatementList.size(); i++) { + SqlNode sqlNode = tableEnv.parseSql(jobStatementList.get(i).getStatement()); if (sqlNode instanceof SqlCreateTable) { SqlCreateTable sqlCreateTable = (SqlCreateTable) sqlNode; String tableName = sqlCreateTable.getTableName().toString(); if (tablesNeedMock.contains(tableName)) { // generate mock statement - mockedDdl.add(new StatementParam( - getSinkMockDdlStatement( - tableName, sqlCreateTable.getColumnList().toString()), - SqlType.CREATE)); - isDdlMocked = true; + jobStatementList.set( + i, + JobStatement.generateJobStatement( + jobStatementList.get(i).getIndex(), + getSinkMockDdlStatement( + tableName, + sqlCreateTable.getColumnList().toString()), + JobStatementType.DDL, + jobStatementList.get(i).getSqlType())); } } - if (!isDdlMocked) { - mockedDdl.add(ddl); - } } - jobParam.setDdl(mockedDdl); - log.debug("Mock sink succeed: {}", JsonUtils.toJsonString(jobParam)); + log.debug("Mock sink succeed: {}", JsonUtils.toJsonString(jobStatementPlan)); } /** * get tables names of insert statements, these tables will be mocked * - * @param jobParam jobParam + * @param jobStatementPlan JobStatementPlan * @return a hash set, which contains all insert table names */ - private Set getTableNamesNeedMockAndModifyTrans(JobParam jobParam) { - List transStatements = jobParam.getTrans(); - List mockedTransStatements = new ArrayList<>(); + private Set getTableNamesNeedMockAndModifyTrans(JobStatementPlan jobStatementPlan) { Set insertTables = new HashSet<>(); - for (StatementParam statement : transStatements) { - if (statement.getType().equals(SqlType.INSERT)) { + List jobStatementList = jobStatementPlan.getJobStatementList(); + for (int i = 0; i < jobStatementList.size(); i++) { + if (jobStatementList.get(i).getSqlType().equals(SqlType.INSERT)) { try { - SqlInsert sqlInsert = (SqlInsert) SqlParser.create(statement.getValue(), calciteConfig) - .parseQuery(); + SqlInsert sqlInsert = + (SqlInsert) SqlParser.create(jobStatementList.get(i).getStatement(), calciteConfig) + .parseQuery(); insertTables.add(sqlInsert.getTargetTable().toString()); SqlInsert mockedInsertTrans = new SqlInsert( sqlInsert.getParserPosition(), @@ -133,19 +125,22 @@ private Set getTableNamesNeedMockAndModifyTrans(JobParam jobParam) { SqlParserPos.ZERO), sqlInsert.getSource(), sqlInsert.getTargetColumnList()); - mockedTransStatements.add(new StatementParam( - mockedInsertTrans - .toSqlString(AnsiSqlDialect.DEFAULT) - .toString(), - SqlType.INSERT)); + jobStatementList.set( + i, + JobStatement.generateJobStatement( + jobStatementList.get(i).getIndex(), + mockedInsertTrans + .toSqlString(AnsiSqlDialect.DEFAULT) + .toString(), + JobStatementType.SQL, + jobStatementList.get(i).getSqlType())); } catch (Exception e) { - log.error("Statement parse error, statement: {}", statement.getValue()); + log.error( + "Statement parse error, statement: {}", + jobStatementList.get(i).getStatement()); } - } else { - mockedTransStatements.add(statement); } } - jobParam.setTrans(mockedTransStatements); return insertTables; } diff --git a/dinky-core/src/main/java/org/dinky/explainer/print_table/PrintStatementExplainer.java b/dinky-core/src/main/java/org/dinky/explainer/print_table/PrintStatementExplainer.java index ee3000b55e..6e4ffa317e 100644 --- a/dinky-core/src/main/java/org/dinky/explainer/print_table/PrintStatementExplainer.java +++ b/dinky-core/src/main/java/org/dinky/explainer/print_table/PrintStatementExplainer.java @@ -19,9 +19,12 @@ package org.dinky.explainer.print_table; +import org.dinky.utils.IpUtil; + import java.net.InetAddress; import java.net.UnknownHostException; import java.text.MessageFormat; +import java.util.Map; import java.util.Optional; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -63,6 +66,12 @@ public static String getCreateStatement(String tableName, String localIp, Intege return MessageFormat.format(CREATE_SQL_TEMPLATE, tableName, ip, port); } + public static String getCreateStatement(String tableName, Map config) { + String host = config.getOrDefault("dinky.dinkyHost", IpUtil.getHostIp()); + int port = Integer.parseInt(config.getOrDefault("dinky.dinkyPrintPort", "7125")); + return getCreateStatement(tableName, host, port); + } + private static Optional getSystemLocalIp() { try { return Optional.of(InetAddress.getLocalHost()); diff --git a/dinky-core/src/main/java/org/dinky/job/JobConfig.java b/dinky-core/src/main/java/org/dinky/job/JobConfig.java index 8718e524aa..1e377f6e9f 100644 --- a/dinky-core/src/main/java/org/dinky/job/JobConfig.java +++ b/dinky-core/src/main/java/org/dinky/job/JobConfig.java @@ -225,7 +225,7 @@ public ExecutorConfig getExecutorSetting() { Map config = new HashMap<>(32); if (GatewayType.isDeployCluster(type) && gatewayConfig != null && gatewayConfig.getFlinkConfig() != null) { config.putAll(gatewayConfig.getFlinkConfig().getConfiguration()); - } else { + } else if (Asserts.isNotNull(configJson)) { config.putAll(configJson); } return ExecutorConfig.build( @@ -283,4 +283,10 @@ public boolean isUseRemote() { public void buildLocal() { type = GatewayType.LOCAL.getLongValue(); } + + public static JobConfig buildPlanConfig() { + JobConfig jobConfig = new JobConfig(); + jobConfig.setType(GatewayType.LOCAL.getLongValue()); + return jobConfig; + } } diff --git a/dinky-core/src/main/java/org/dinky/job/JobManager.java b/dinky-core/src/main/java/org/dinky/job/JobManager.java index 9dd9e57947..2e6cfd7749 100644 --- a/dinky-core/src/main/java/org/dinky/job/JobManager.java +++ b/dinky-core/src/main/java/org/dinky/job/JobManager.java @@ -30,6 +30,8 @@ import org.dinky.data.enums.ProcessStepType; import org.dinky.data.enums.Status; import org.dinky.data.exception.BusException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; import org.dinky.data.model.SystemConfiguration; import org.dinky.data.result.ErrorResult; import org.dinky.data.result.ExplainResult; @@ -50,11 +52,8 @@ import org.dinky.gateway.result.GatewayResult; import org.dinky.gateway.result.SavePointResult; import org.dinky.gateway.result.TestResult; -import org.dinky.job.builder.JobDDLBuilder; -import org.dinky.job.builder.JobExecuteBuilder; import org.dinky.job.builder.JobJarStreamGraphBuilder; -import org.dinky.job.builder.JobTransBuilder; -import org.dinky.parser.SqlType; +import org.dinky.job.runner.JobJarRunner; import org.dinky.trans.Operations; import org.dinky.trans.parse.AddFileSqlParseStrategy; import org.dinky.trans.parse.AddJarSqlParseStrategy; @@ -111,6 +110,7 @@ public class JobManager { private boolean useRestAPI = false; private GatewayType runMode = GatewayType.LOCAL; private JobParam jobParam = null; + private JobStatementPlan jobStatementPlan; private String currentSql = ""; private final WeakReference dinkyClassLoader = new WeakReference<>(DinkyClassLoader.build()); private Job job; @@ -169,9 +169,13 @@ public boolean isUseGateway() { return useGateway; } + public JobStatementPlan getJobStatementPlan() { + return jobStatementPlan; + } + // return dinkyclassloader public DinkyClassLoader getDinkyClassLoader() { - return dinkyClassLoader.get(); + return Asserts.isNotNull(dinkyClassLoader.get()) ? dinkyClassLoader.get() : DinkyClassLoader.build(); } // return udfPathContextHolder @@ -219,18 +223,19 @@ public void init() { executorConfig = config.getExecutorSetting(); executorConfig.setPlan(isPlanMode); executor = ExecutorFactory.buildExecutor(executorConfig, getDinkyClassLoader()); + DinkyClassLoaderUtil.initClassLoader(config, getDinkyClassLoader()); } private boolean ready() { - return handler.init(job); + return isPlanMode || handler.init(job); } private boolean success() { - return handler.success(); + return isPlanMode || handler.success(); } private boolean failed() { - return handler.failed(); + return isPlanMode || handler.failed(); } public boolean close() { @@ -257,12 +262,16 @@ public JobResult executeJarSql(String statement) throws Exception { .map(t -> executor.pretreatStatement(t)) .collect(Collectors.toList()); statement = String.join(";\n", statements); - jobParam = - Explainer.build(executor, useStatementSet, this).pretreatStatements(SqlUtil.getStatements(statement)); + jobStatementPlan = Explainer.build(this).parseStatements(SqlUtil.getStatements(statement)); job = Job.build(runMode, config, executorConfig, executor, statement, useGateway); ready(); try { - JobJarStreamGraphBuilder.build(this).run(); + // Only one is executed. + for (JobStatement jobStatement : jobStatementPlan.getJobStatementList()) { + JobJarRunner jobJarRunner = new JobJarRunner(this); + jobJarRunner.run(jobStatement); + break; + } if (job.isFailed()) { failed(); } else { @@ -287,18 +296,13 @@ public JobResult executeJarSql(String statement) throws Exception { public JobResult executeSql(String statement) throws Exception { job = Job.build(runMode, config, executorConfig, executor, statement, useGateway); ready(); - - DinkyClassLoaderUtil.initClassLoader(config, getDinkyClassLoader()); - jobParam = - Explainer.build(executor, useStatementSet, this).pretreatStatements(SqlUtil.getStatements(statement)); try { - // step 1: execute ddl - JobDDLBuilder.build(this).run(); - // step 2: execute insert/select/show/desc/CTAS... - JobTransBuilder.build(this).run(); - // step 3: execute custom data stream task - JobExecuteBuilder.build(this).run(); - // finished + jobStatementPlan = Explainer.build(this).parseStatements(SqlUtil.getStatements(statement)); + jobStatementPlan.buildFinalStatement(); + JobRunnerFactory jobRunnerFactory = JobRunnerFactory.create(this); + for (JobStatement jobStatement : jobStatementPlan.getJobStatementList()) { + jobRunnerFactory.getJobRunner(jobStatement.getStatementType()).run(jobStatement); + } job.setEndTime(LocalDateTime.now()); if (job.isFailed()) { failed(); @@ -367,20 +371,15 @@ public static SelectResult getJobData(String jobId) { } public ExplainResult explainSql(String statement) { - return Explainer.build(executor, useStatementSet, this).explainSql(statement); + return Explainer.build(this).explainSql(statement); } public ObjectNode getStreamGraph(String statement) { - return Explainer.build(executor, useStatementSet, this) - .initialize(config, statement) - .getStreamGraph(statement); + return Explainer.build(this).getStreamGraph(statement); } public String getJobPlanJson(String statement) { - return Explainer.build(executor, useStatementSet, this) - .initialize(config, statement) - .getJobPlanInfo(statement) - .getJsonPlan(); + return Explainer.build(this).getJobPlanInfo(statement).getJsonPlan(); } public boolean cancelNormal(String jobId) { diff --git a/dinky-core/src/main/java/org/dinky/job/JobRunner.java b/dinky-core/src/main/java/org/dinky/job/JobRunner.java new file mode 100644 index 0000000000..6adf173f9b --- /dev/null +++ b/dinky-core/src/main/java/org/dinky/job/JobRunner.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.dinky.job; + +import org.dinky.data.job.JobStatement; +import org.dinky.data.result.SqlExplainResult; + +import org.apache.flink.runtime.rest.messages.JobPlanInfo; +import org.apache.flink.streaming.api.graph.StreamGraph; + +public interface JobRunner { + + void run(JobStatement jobStatement) throws Exception; + + SqlExplainResult explain(JobStatement jobStatement); + + StreamGraph getStreamGraph(JobStatement jobStatement); + + JobPlanInfo getJobPlanInfo(JobStatement jobStatement); +} diff --git a/dinky-core/src/main/java/org/dinky/job/JobRunnerFactory.java b/dinky-core/src/main/java/org/dinky/job/JobRunnerFactory.java new file mode 100644 index 0000000000..909d1b9fd9 --- /dev/null +++ b/dinky-core/src/main/java/org/dinky/job/JobRunnerFactory.java @@ -0,0 +1,59 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.job; + +import org.dinky.data.job.JobStatementType; +import org.dinky.job.runner.JobDDLRunner; +import org.dinky.job.runner.JobPipelineRunner; +import org.dinky.job.runner.JobSetRunner; +import org.dinky.job.runner.JobSqlRunner; + +public class JobRunnerFactory { + + private JobSetRunner jobSetRunner; + private JobSqlRunner jobSqlRunner; + private JobPipelineRunner jobPipelineRunner; + private JobDDLRunner jobDDLRunner; + + public JobRunnerFactory(JobManager jobManager) { + this.jobSetRunner = new JobSetRunner(jobManager); + this.jobSqlRunner = new JobSqlRunner(jobManager); + this.jobPipelineRunner = new JobPipelineRunner(jobManager); + this.jobDDLRunner = new JobDDLRunner(jobManager); + } + + public JobRunner getJobRunner(JobStatementType jobStatementType) { + switch (jobStatementType) { + case SET: + return jobSetRunner; + case SQL: + return jobSqlRunner; + case PIPELINE: + return jobPipelineRunner; + case DDL: + default: + return jobDDLRunner; + } + } + + public static JobRunnerFactory create(JobManager jobManager) { + return new JobRunnerFactory(jobManager); + } +} diff --git a/dinky-core/src/main/java/org/dinky/job/JobStatementPlan.java b/dinky-core/src/main/java/org/dinky/job/JobStatementPlan.java new file mode 100644 index 0000000000..fbf2dc1b9d --- /dev/null +++ b/dinky-core/src/main/java/org/dinky/job/JobStatementPlan.java @@ -0,0 +1,85 @@ +/* + * + * 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.dinky.job; + +import org.dinky.constant.FlinkSQLConstant; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.JobStatementType; +import org.dinky.data.job.SqlType; +import org.dinky.function.util.UDFUtil; + +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +public class JobStatementPlan { + + private List jobStatementList = new ArrayList<>(); + + public JobStatementPlan() {} + + public List getJobStatementList() { + return jobStatementList; + } + + public void addJobStatement(String statement, JobStatementType statementType, SqlType sqlType) { + jobStatementList.add(new JobStatement(jobStatementList.size() + 1, statement, statementType, sqlType, false)); + } + + public void addJobStatementGenerated(String statement, JobStatementType statementType, SqlType sqlType) { + jobStatementList.add(new JobStatement(jobStatementList.size() + 1, statement, statementType, sqlType, true)); + } + + public String getStatements() { + return StringUtils.join( + jobStatementList.stream().map(JobStatement::getStatement).collect(Collectors.toList()), + FlinkSQLConstant.SEPARATOR); + } + + public void buildFinalStatement() { + if (jobStatementList.size() == 0) { + return; + } + int executableIndex = -1; + int createFunctionIndex = -1; + for (int i = 0; i < jobStatementList.size(); i++) { + if (jobStatementList.get(i).getSqlType().isPipeline()) { + executableIndex = i; + } else if (UDFUtil.isUdfStatement(jobStatementList.get(i).getStatement())) { + createFunctionIndex = i; + } + } + if (executableIndex >= 0) { + jobStatementList.get(executableIndex).asFinalExecutableStatement(); + } else { + for (int i = 0; i < jobStatementList.size(); i++) { + if (jobStatementList.get(i).getStatementType().equals(JobStatementType.SQL)) { + jobStatementList.get(i).asFinalExecutableStatement(); + break; + } + } + } + if (createFunctionIndex >= 0) { + jobStatementList.get(createFunctionIndex).asFinalCreateFunctionStatement(); + } + } +} diff --git a/dinky-core/src/main/java/org/dinky/job/StatementParam.java b/dinky-core/src/main/java/org/dinky/job/StatementParam.java index 6ff7ec5bd8..ffbecfda2c 100644 --- a/dinky-core/src/main/java/org/dinky/job/StatementParam.java +++ b/dinky-core/src/main/java/org/dinky/job/StatementParam.java @@ -19,7 +19,7 @@ package org.dinky.job; -import org.dinky.parser.SqlType; +import org.dinky.data.job.SqlType; /** * StatementParam diff --git a/dinky-core/src/main/java/org/dinky/job/builder/JobDDLBuilder.java b/dinky-core/src/main/java/org/dinky/job/builder/JobDDLBuilder.java index 739dd537a5..b973065cb8 100644 --- a/dinky-core/src/main/java/org/dinky/job/builder/JobDDLBuilder.java +++ b/dinky-core/src/main/java/org/dinky/job/builder/JobDDLBuilder.java @@ -22,6 +22,7 @@ import static org.dinky.function.util.UDFUtil.*; import org.dinky.assertion.Asserts; +import org.dinky.data.job.SqlType; import org.dinky.data.model.SystemConfiguration; import org.dinky.data.result.SqlExplainResult; import org.dinky.executor.CustomTableEnvironment; @@ -30,7 +31,6 @@ import org.dinky.job.JobBuilder; import org.dinky.job.JobManager; import org.dinky.job.StatementParam; -import org.dinky.parser.SqlType; import org.dinky.trans.ddl.CustomSetOperation; import org.dinky.trans.parse.AddFileSqlParseStrategy; import org.dinky.trans.parse.AddJarSqlParseStrategy; @@ -75,6 +75,7 @@ public static JobDDLBuilder build(JobManager jobManager) { @Override public void run() throws Exception { List udfList = new ArrayList<>(); + List udfStatements = new ArrayList<>(); for (StatementParam item : jobParam.getDdl()) { jobManager.setCurrentSql(item.getValue()); switch (item.getType()) { @@ -92,7 +93,11 @@ public void run() throws Exception { break; case CREATE: if (UDFUtil.isUdfStatement(item.getValue())) { - udfList.add(UDFUtil.toUDF(item.getValue(), executor.getDinkyClassLoader())); + UDF udf = toUDF(item.getValue(), executor.getDinkyClassLoader()); + if (Asserts.isNotNull(udf)) { + udfList.add(UDFUtil.toUDF(item.getValue(), executor.getDinkyClassLoader())); + } + udfStatements.add(item.getValue()); } else { executor.executeSql(item.getValue()); } @@ -102,7 +107,10 @@ public void run() throws Exception { } } if (!udfList.isEmpty()) { - executeCreateFunction(udfList); + compileUDF(udfList); + } + if (!udfStatements.isEmpty()) { + executeCreateFunction(udfStatements); } } @@ -173,7 +181,9 @@ public List explain() { SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); String udfStatement = StringUtils.join(udfStatements, ";\n"); try { - explainCreateFunction(udfList); + SqlExplainResult recordResult = null; + recordResult = explainCreateFunction(udfList, udfStatements); + resultBuilder = SqlExplainResult.newBuilder(recordResult); } catch (Exception e) { String error = StrFormatter.format( "Exception in executing CreateFunction:\n{}\n{}", @@ -223,7 +233,13 @@ private void executeAddJar(String statement) { executor.executeSql(statement); } - private void executeCreateFunction(List udfList) { + private void executeCreateFunction(List udfStatements) { + for (String statement : udfStatements) { + executor.executeSql(statement); + } + } + + private void compileUDF(List udfList) { Integer taskId = config.getTaskId(); if (taskId == null) { taskId = -RandomUtil.randomInt(0, 1000); @@ -328,11 +344,17 @@ private SqlExplainResult explainAddJar(String statement) { return sqlExplainResult; } - private SqlExplainResult explainCreateFunction(List udfList) { + private SqlExplainResult explainCreateFunction(List udfList, List udfStatements) { SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); - executeCreateFunction(udfList); + compileUDF(udfList); + executeCreateFunction(udfStatements); String explain = udfList.toString(); - return resultBuilder.parseTrue(true).explainTrue(true).explain(explain).build(); + return resultBuilder + .type(SqlType.CREATE.getType()) + .parseTrue(true) + .explainTrue(true) + .explain(explain) + .build(); } private SqlExplainResult explainOtherDDL(String statement) { diff --git a/dinky-core/src/main/java/org/dinky/job/builder/JobExecuteBuilder.java b/dinky-core/src/main/java/org/dinky/job/builder/JobExecuteBuilder.java index b31844b9c0..835cf2285d 100644 --- a/dinky-core/src/main/java/org/dinky/job/builder/JobExecuteBuilder.java +++ b/dinky-core/src/main/java/org/dinky/job/builder/JobExecuteBuilder.java @@ -20,6 +20,7 @@ package org.dinky.job.builder; import org.dinky.assertion.Asserts; +import org.dinky.data.job.SqlType; import org.dinky.data.result.IResult; import org.dinky.data.result.InsertResult; import org.dinky.data.result.ResultBuilder; @@ -30,7 +31,6 @@ import org.dinky.job.JobBuilder; import org.dinky.job.JobManager; import org.dinky.job.StatementParam; -import org.dinky.parser.SqlType; import org.dinky.trans.dml.ExecuteJarOperation; import org.dinky.trans.parse.ExecuteJarParseStrategy; import org.dinky.utils.FlinkStreamEnvironmentUtil; @@ -149,7 +149,7 @@ public List explain() { SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); try { SqlExplainResult sqlExplainResult = executor.explainSqlRecord(item.getValue()); - if (Asserts.isNull(sqlExplainResult)) { + if (!sqlExplainResult.isInvalid()) { sqlExplainResult = new SqlExplainResult(); } else if (ExecuteJarParseStrategy.INSTANCE.match(item.getValue())) { List allFileByAdd = jobManager.getAllFileSet(); @@ -193,6 +193,6 @@ public StreamGraph getStreamGraph() { @Override public JobPlanInfo getJobPlanInfo() { - return executor.getJobPlanInfo(jobParam.getExecuteStatement()); + return executor.getJobPlanInfo(null); } } diff --git a/dinky-core/src/main/java/org/dinky/job/builder/JobJarStreamGraphBuilder.java b/dinky-core/src/main/java/org/dinky/job/builder/JobJarStreamGraphBuilder.java index 182177b92b..eadfe071c5 100644 --- a/dinky-core/src/main/java/org/dinky/job/builder/JobJarStreamGraphBuilder.java +++ b/dinky-core/src/main/java/org/dinky/job/builder/JobJarStreamGraphBuilder.java @@ -22,6 +22,7 @@ import org.dinky.assertion.Asserts; import org.dinky.classloader.DinkyClassLoader; import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.SqlType; import org.dinky.data.result.InsertResult; import org.dinky.data.result.SqlExplainResult; import org.dinky.gateway.Gateway; @@ -30,7 +31,6 @@ import org.dinky.job.Job; import org.dinky.job.JobBuilder; import org.dinky.job.JobManager; -import org.dinky.parser.SqlType; import org.dinky.trans.Operations; import org.dinky.trans.ddl.CustomSetOperation; import org.dinky.trans.dml.ExecuteJarOperation; diff --git a/dinky-core/src/main/java/org/dinky/job/builder/JobTransBuilder.java b/dinky-core/src/main/java/org/dinky/job/builder/JobTransBuilder.java index 8b678c1803..7207151dfe 100644 --- a/dinky-core/src/main/java/org/dinky/job/builder/JobTransBuilder.java +++ b/dinky-core/src/main/java/org/dinky/job/builder/JobTransBuilder.java @@ -22,6 +22,7 @@ import org.dinky.assertion.Asserts; import org.dinky.constant.FlinkSQLConstant; import org.dinky.data.enums.GatewayType; +import org.dinky.data.job.SqlType; import org.dinky.data.result.IResult; import org.dinky.data.result.InsertResult; import org.dinky.data.result.ResultBuilder; @@ -36,7 +37,6 @@ import org.dinky.job.JobConfig; import org.dinky.job.JobManager; import org.dinky.job.StatementParam; -import org.dinky.parser.SqlType; import org.dinky.utils.LogUtil; import org.dinky.utils.SqlUtil; import org.dinky.utils.URLUtils; @@ -91,7 +91,7 @@ public List explain() { if (Asserts.isNullCollection(jobParam.getTrans())) { return sqlExplainResults; } - if (useStatementSet) { + if (inferStatementSet()) { List inserts = new ArrayList<>(); for (StatementParam item : jobParam.getTrans()) { if (item.getType().equals(SqlType.INSERT) || item.getType().equals(SqlType.CTAS)) { @@ -102,10 +102,7 @@ public List explain() { SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); String sqlSet = StringUtils.join(inserts, ";\r"); try { - resultBuilder - .explain(executor.explainStatementSet(inserts)) - .parseTrue(true) - .explainTrue(true); + resultBuilder.explain(null).parseTrue(true).explainTrue(true); } catch (Exception e) { String error = LogUtil.getError(e); resultBuilder @@ -153,12 +150,12 @@ public List explain() { @Override public StreamGraph getStreamGraph() { - return executor.getStreamGraphFromStatement(jobParam.getTransStatement()); + return executor.getStreamGraphFromStatement(null); } @Override public JobPlanInfo getJobPlanInfo() { - return executor.getJobPlanInfo(jobParam.getTransStatement()); + return executor.getJobPlanInfo(null); } private boolean inferStatementSet() { @@ -285,7 +282,7 @@ private GatewayResult submitByGateway(List inserts) { gatewayResult = Gateway.build(config.getGatewayConfig()) .submitJar(executor.getDinkyClassLoader().getUdfPathContextHolder()); } else { - JobGraph jobGraph = executor.getJobGraphFromInserts(inserts); + JobGraph jobGraph = executor.getJobGraphFromInserts(null); // Perjob mode need to set savepoint restore path, when recovery from savepoint. if (Asserts.isNotNullString(config.getSavePointPath())) { jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(config.getSavePointPath(), true)); diff --git a/dinky-core/src/main/java/org/dinky/job/runner/AbstractJobRunner.java b/dinky-core/src/main/java/org/dinky/job/runner/AbstractJobRunner.java new file mode 100644 index 0000000000..1348d18e1a --- /dev/null +++ b/dinky-core/src/main/java/org/dinky/job/runner/AbstractJobRunner.java @@ -0,0 +1,81 @@ +/* + * + * 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.dinky.job.runner; + +import org.dinky.data.job.JobStatement; +import org.dinky.data.result.SqlExplainResult; +import org.dinky.job.JobManager; +import org.dinky.job.JobRunner; +import org.dinky.utils.LogUtil; +import org.dinky.utils.SqlUtil; + +import org.apache.flink.runtime.rest.messages.JobPlanInfo; +import org.apache.flink.streaming.api.graph.StreamGraph; + +import java.time.LocalDateTime; + +import cn.hutool.core.text.StrFormatter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public abstract class AbstractJobRunner implements JobRunner { + + protected JobManager jobManager; + + public SqlExplainResult explain(JobStatement jobStatement) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + try { + run(jobStatement); + resultBuilder + .parseTrue(true) + .explainTrue(true) + .type(jobStatement.getSqlType().getType()) + .sql(jobStatement.getStatement()) + .explainTime(LocalDateTime.now()) + .index(jobStatement.getIndex()); + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(jobStatement.getStatement()), + LogUtil.getError(e)); + resultBuilder + .parseTrue(false) + .error(error) + .explainTrue(false) + .type(jobStatement.getSqlType().getType()) + .sql(jobStatement.getStatement()) + .explainTime(LocalDateTime.now()) + .index(jobStatement.getIndex()); + log.error(error); + return resultBuilder.build(); + } + return resultBuilder.build(); + } + + public StreamGraph getStreamGraph(JobStatement jobStatement) { + explain(jobStatement); + return null; + } + + public JobPlanInfo getJobPlanInfo(JobStatement jobStatement) { + explain(jobStatement); + return null; + } +} diff --git a/dinky-core/src/main/java/org/dinky/job/runner/JobDDLRunner.java b/dinky-core/src/main/java/org/dinky/job/runner/JobDDLRunner.java new file mode 100644 index 0000000000..e7e1ebe615 --- /dev/null +++ b/dinky-core/src/main/java/org/dinky/job/runner/JobDDLRunner.java @@ -0,0 +1,297 @@ +/* + * + * 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.dinky.job.runner; + +import static org.dinky.function.util.UDFUtil.*; + +import org.dinky.assertion.Asserts; +import org.dinky.data.job.JobStatement; +import org.dinky.data.model.SystemConfiguration; +import org.dinky.data.result.SqlExplainResult; +import org.dinky.executor.CustomTableEnvironment; +import org.dinky.function.data.model.UDF; +import org.dinky.function.util.UDFUtil; +import org.dinky.job.JobManager; +import org.dinky.trans.parse.AddFileSqlParseStrategy; +import org.dinky.trans.parse.AddJarSqlParseStrategy; +import org.dinky.utils.LogUtil; +import org.dinky.utils.SqlUtil; +import org.dinky.utils.URLUtils; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileSystem; + +import java.io.File; +import java.net.URL; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +import cn.hutool.core.collection.CollUtil; +import cn.hutool.core.text.StrFormatter; +import cn.hutool.core.util.ArrayUtil; +import cn.hutool.core.util.RandomUtil; +import cn.hutool.core.util.StrUtil; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class JobDDLRunner extends AbstractJobRunner { + + private List udfStatements = new ArrayList<>(); + + public JobDDLRunner(JobManager jobManager) { + this.jobManager = jobManager; + } + + @Override + public void run(JobStatement jobStatement) throws Exception { + jobManager.setCurrentSql(jobStatement.getStatement()); + switch (jobStatement.getSqlType()) { + case ADD: + executeAdd(jobStatement.getStatement()); + break; + case ADD_FILE: + executeAddFile(jobStatement.getStatement()); + break; + case ADD_JAR: + executeAddJar(jobStatement.getStatement()); + break; + case CREATE: + if (UDFUtil.isUdfStatement(jobStatement.getStatement())) { + udfStatements.add(jobStatement.getStatement()); + break; + } + default: + jobManager.getExecutor().executeSql(jobStatement.getStatement()); + } + if (jobStatement.isFinalCreateFunctionStatement() && !udfStatements.isEmpty()) { + executeCreateFunction(udfStatements); + } + } + + @Override + public SqlExplainResult explain(JobStatement jobStatement) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + try { + SqlExplainResult recordResult = null; + switch (jobStatement.getSqlType()) { + case ADD: + recordResult = explainAdd(jobStatement.getStatement()); + break; + case ADD_FILE: + recordResult = explainAddFile(jobStatement.getStatement()); + break; + case ADD_JAR: + recordResult = explainAddJar(jobStatement.getStatement()); + break; + case CREATE: + if (UDFUtil.isUdfStatement(jobStatement.getStatement())) { + udfStatements.add(jobStatement.getStatement()); + recordResult = jobManager.getExecutor().explainSqlRecord(jobStatement.getStatement()); + break; + } + default: + recordResult = explainOtherDDL(jobStatement.getStatement()); + } + if (jobStatement.isFinalCreateFunctionStatement() && !udfStatements.isEmpty()) { + explainCreateFunction(jobStatement); + } + if (Asserts.isNull(recordResult)) { + return resultBuilder.invalid().build(); + } + resultBuilder = SqlExplainResult.newBuilder(recordResult); + resultBuilder + .explainTrue(true) + .type(jobStatement.getSqlType().getType()) + .sql(jobStatement.getStatement()) + .index(jobStatement.getIndex()); + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(jobStatement.getStatement()), + LogUtil.getError(e)); + resultBuilder + .error(error) + .explainTrue(false) + .type(jobStatement.getSqlType().getType()) + .sql(jobStatement.getStatement()) + .index(jobStatement.getIndex()); + log.error(error); + } finally { + resultBuilder.explainTime(LocalDateTime.now()); + return resultBuilder.build(); + } + } + + private void executeAdd(String statement) { + AddJarSqlParseStrategy.getAllFilePath(statement) + .forEach(t -> jobManager.getUdfPathContextHolder().addOtherPlugins(t)); + (jobManager.getExecutor().getDinkyClassLoader()) + .addURLs(URLUtils.getURLs(jobManager.getUdfPathContextHolder().getOtherPluginsFiles())); + } + + private void executeAddFile(String statement) { + AddFileSqlParseStrategy.getAllFilePath(statement) + .forEach(t -> jobManager.getUdfPathContextHolder().addFile(t)); + (jobManager.getExecutor().getDinkyClassLoader()) + .addURLs(URLUtils.getURLs(jobManager.getUdfPathContextHolder().getFiles())); + } + + private void executeAddJar(String statement) { + Configuration combinationConfig = getCombinationConfig(); + FileSystem.initialize(combinationConfig, null); + jobManager.getExecutor().executeSql(statement); + } + + private void executeCreateFunction(List udfStatements) { + List udfList = new ArrayList<>(); + for (String statement : udfStatements) { + UDF udf = toUDF(statement, jobManager.getExecutor().getDinkyClassLoader()); + if (Asserts.isNotNull(udf)) { + udfList.add(UDFUtil.toUDF(statement, jobManager.getExecutor().getDinkyClassLoader())); + } + } + if (!udfList.isEmpty()) { + compileUDF(udfList); + } + for (String statement : udfStatements) { + jobManager.getExecutor().executeSql(statement); + } + } + + private SqlExplainResult explainAdd(String statement) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + executeAdd(statement); + String explain = Arrays.toString( + URLUtils.getURLs(jobManager.getUdfPathContextHolder().getOtherPluginsFiles())); + return resultBuilder.parseTrue(true).explainTrue(true).explain(explain).build(); + } + + private SqlExplainResult explainAddFile(String statement) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + executeAddFile(statement); + String explain = Arrays.toString( + URLUtils.getURLs(jobManager.getUdfPathContextHolder().getFiles())); + return resultBuilder.parseTrue(true).explainTrue(true).explain(explain).build(); + } + + private SqlExplainResult explainAddJar(String statement) { + SqlExplainResult sqlExplainResult = jobManager.getExecutor().explainSqlRecord(statement); + executeAddJar(statement); + return sqlExplainResult; + } + + private SqlExplainResult explainCreateFunction(JobStatement jobStatement) { + udfStatements.add(jobStatement.getStatement()); + SqlExplainResult sqlExplainResult = explainOtherDDL(jobStatement.getStatement()); + if (jobStatement.isFinalCreateFunctionStatement()) { + executeCreateFunction(udfStatements); + } + return sqlExplainResult; + } + + private SqlExplainResult explainOtherDDL(String statement) { + SqlExplainResult sqlExplainResult = jobManager.getExecutor().explainSqlRecord(statement); + jobManager.getExecutor().executeSql(statement); + return sqlExplainResult; + } + + private Configuration getCombinationConfig() { + CustomTableEnvironment cte = jobManager.getExecutor().getCustomTableEnvironment(); + Configuration rootConfig = cte.getRootConfiguration(); + Configuration config = cte.getConfig().getConfiguration(); + Configuration combinationConfig = new Configuration(); + combinationConfig.addAll(rootConfig); + combinationConfig.addAll(config); + return combinationConfig; + } + + private void compileUDF(List udfList) { + Integer taskId = jobManager.getConfig().getTaskId(); + if (taskId == null) { + taskId = -RandomUtil.randomInt(0, 1000); + } + // 1. Obtain the path of the jar package and inject it into the remote environment + List jarFiles = + new ArrayList<>(jobManager.getUdfPathContextHolder().getAllFileSet()); + + String[] userCustomUdfJarPath = UDFUtil.initJavaUDF(udfList, taskId); + String[] jarPaths = CollUtil.removeNull(jarFiles).stream() + .map(File::getAbsolutePath) + .toArray(String[]::new); + if (GATEWAY_TYPE_MAP.get(SESSION).contains(jobManager.getRunMode())) { + jobManager.getConfig().setJarFiles(jarPaths); + } + + // 2.Compile Python + String[] pyPaths = UDFUtil.initPythonUDF( + udfList, + jobManager.getRunMode(), + jobManager.getConfig().getTaskId(), + jobManager.getExecutor().getTableConfig().getConfiguration()); + + jobManager.getExecutor().initUDF(userCustomUdfJarPath); + jobManager.getExecutor().initUDF(jarPaths); + + if (ArrayUtil.isNotEmpty(pyPaths)) { + for (String pyPath : pyPaths) { + if (StrUtil.isNotBlank(pyPath)) { + jarFiles.add(new File(pyPath)); + jobManager.getUdfPathContextHolder().addPyUdfPath(new File(pyPath)); + } + } + } + if (ArrayUtil.isNotEmpty(userCustomUdfJarPath)) { + for (String jarPath : userCustomUdfJarPath) { + if (StrUtil.isNotBlank(jarPath)) { + jarFiles.add(new File(jarPath)); + jobManager.getUdfPathContextHolder().addUdfPath(new File(jarPath)); + } + } + } + + Set pyUdfFile = jobManager.getUdfPathContextHolder().getPyUdfFile(); + jobManager + .getExecutor() + .initPyUDF( + SystemConfiguration.getInstances().getPythonHome(), + pyUdfFile.stream().map(File::getAbsolutePath).toArray(String[]::new)); + if (GATEWAY_TYPE_MAP.get(YARN).contains(jobManager.getRunMode())) { + jobManager.getConfig().getGatewayConfig().setJarPaths(ArrayUtil.append(jarPaths, pyPaths)); + } + + try { + List jarList = CollUtil.newArrayList(URLUtils.getURLs(jarFiles)); + // 3.Write the required files for UDF + UDFUtil.writeManifest(taskId, jarList, jobManager.getUdfPathContextHolder()); + UDFUtil.addConfigurationClsAndJars( + jobManager.getExecutor().getCustomTableEnvironment(), + jarList, + CollUtil.newArrayList(URLUtils.getURLs(jarFiles))); + } catch (Exception e) { + throw new RuntimeException("add configuration failed: ", e); + } + + log.info(StrUtil.format("A total of {} UDF have been Init.", udfList.size() + pyUdfFile.size())); + log.info("Initializing Flink UDF...Finish"); + } +} diff --git a/dinky-core/src/main/java/org/dinky/job/runner/JobJarRunner.java b/dinky-core/src/main/java/org/dinky/job/runner/JobJarRunner.java new file mode 100644 index 0000000000..1202b49181 --- /dev/null +++ b/dinky-core/src/main/java/org/dinky/job/runner/JobJarRunner.java @@ -0,0 +1,231 @@ +/* + * + * 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.dinky.job.runner; + +import org.dinky.assertion.Asserts; +import org.dinky.classloader.DinkyClassLoader; +import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; +import org.dinky.data.result.InsertResult; +import org.dinky.data.result.SqlExplainResult; +import org.dinky.gateway.Gateway; +import org.dinky.gateway.config.GatewayConfig; +import org.dinky.gateway.result.GatewayResult; +import org.dinky.job.Job; +import org.dinky.job.JobManager; +import org.dinky.trans.Operations; +import org.dinky.trans.ddl.CustomSetOperation; +import org.dinky.trans.dml.ExecuteJarOperation; +import org.dinky.trans.parse.AddFileSqlParseStrategy; +import org.dinky.trans.parse.AddJarSqlParseStrategy; +import org.dinky.trans.parse.ExecuteJarParseStrategy; +import org.dinky.trans.parse.SetSqlParseStrategy; +import org.dinky.utils.DinkyClassLoaderUtil; +import org.dinky.utils.FlinkStreamEnvironmentUtil; +import org.dinky.utils.LogUtil; +import org.dinky.utils.SqlUtil; +import org.dinky.utils.URLUtils; + +import org.apache.flink.api.common.Plan; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.streaming.api.graph.StreamGraph; + +import java.io.File; +import java.net.URL; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +import cn.hutool.core.lang.Assert; +import cn.hutool.core.text.StrFormatter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class JobJarRunner extends AbstractJobRunner { + private final Configuration configuration; + + public JobJarRunner(JobManager jobManager) { + this.jobManager = jobManager; + configuration = + jobManager.getExecutor().getCustomTableEnvironment().getConfig().getConfiguration(); + } + + @Override + public void run(JobStatement jobStatement) throws Exception { + if (!jobManager.isUseGateway()) { + submitNormal(jobStatement); + } else { + GatewayResult gatewayResult; + if (jobManager.getRunMode().isApplicationMode()) { + gatewayResult = submitGateway(jobStatement); + } else { + gatewayResult = submitNormalWithGateway(jobStatement); + } + jobManager.getJob().setResult(InsertResult.success(gatewayResult.getId())); + jobManager.getJob().setJobId(gatewayResult.getId()); + jobManager.getJob().setJids(gatewayResult.getJids()); + jobManager.getJob().setJobManagerAddress(URLUtils.formatAddress(gatewayResult.getWebURL())); + + if (gatewayResult.isSuccess()) { + jobManager.getJob().setStatus(Job.JobStatus.SUCCESS); + } else { + jobManager.getJob().setStatus(Job.JobStatus.FAILED); + jobManager.getJob().setError(gatewayResult.getError()); + log.error(gatewayResult.getError()); + } + } + } + + @Override + public SqlExplainResult explain(JobStatement jobStatement) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + + try { + // Execute task does not support statement set. + Pipeline pipeline = getPipeline(jobStatement); + resultBuilder + .explain(FlinkStreamEnvironmentUtil.getStreamingPlanAsJSON(pipeline)) + .type(jobStatement.getSqlType().getType()) + .parseTrue(true) + .explainTrue(true) + .sql(jobStatement.getStatement()) + .index(jobStatement.getIndex()); + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(jobStatement.getStatement()), + LogUtil.getError(e)); + resultBuilder + .error(error) + .explainTrue(false) + .type(jobStatement.getSqlType().getType()) + .sql(jobStatement.getStatement()) + .index(jobStatement.getIndex()); + log.error(error); + } finally { + resultBuilder.explainTime(LocalDateTime.now()); + return resultBuilder.build(); + } + } + + private GatewayResult submitGateway(JobStatement jobStatement) throws Exception { + configuration.set(PipelineOptions.JARS, getUris(jobStatement.getStatement())); + jobManager.getConfig().addGatewayConfig(configuration); + jobManager.getConfig().getGatewayConfig().setSql(jobStatement.getStatement()); + return Gateway.build(jobManager.getConfig().getGatewayConfig()).submitJar(jobManager.getUdfPathContextHolder()); + } + + private GatewayResult submitNormalWithGateway(JobStatement jobStatement) { + Pipeline pipeline = getPipeline(jobStatement); + if (pipeline instanceof StreamGraph) { + ((StreamGraph) pipeline).setJobName(jobManager.getConfig().getJobName()); + } else if (pipeline instanceof Plan) { + ((Plan) pipeline).setJobName(jobManager.getConfig().getJobName()); + } + JobGraph jobGraph = FlinkStreamEnvironmentUtil.getJobGraph(pipeline, configuration); + GatewayConfig gatewayConfig = jobManager.getConfig().getGatewayConfig(); + List uriList = getUris(jobStatement.getStatement()); + String[] jarPaths = uriList.stream() + .map(URLUtils::toFile) + .map(File::getAbsolutePath) + .toArray(String[]::new); + gatewayConfig.setJarPaths(jarPaths); + return Gateway.build(gatewayConfig).submitJobGraph(jobGraph); + } + + private Pipeline getPipeline(JobStatement jobStatement) { + Pipeline pipeline = getJarStreamGraph(jobStatement.getStatement(), jobManager.getDinkyClassLoader()); + if (pipeline instanceof StreamGraph) { + if (Asserts.isNotNullString(jobManager.getConfig().getSavePointPath())) { + ((StreamGraph) pipeline) + .setSavepointRestoreSettings(SavepointRestoreSettings.forPath( + jobManager.getConfig().getSavePointPath(), + configuration.get(SavepointConfigOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE))); + } + } + return pipeline; + } + + private void submitNormal(JobStatement jobStatement) throws Exception { + JobClient jobClient = FlinkStreamEnvironmentUtil.executeAsync( + getPipeline(jobStatement), jobManager.getExecutor().getStreamExecutionEnvironment()); + if (Asserts.isNotNull(jobClient)) { + jobManager.getJob().setJobId(jobClient.getJobID().toHexString()); + jobManager.getJob().setJids(new ArrayList() { + { + add(jobManager.getJob().getJobId()); + } + }); + jobManager.getJob().setStatus(Job.JobStatus.SUCCESS); + } else { + jobManager.getJob().setStatus(Job.JobStatus.FAILED); + } + } + + public Pipeline getJarStreamGraph(String statement, DinkyClassLoader dinkyClassLoader) { + DinkyClassLoaderUtil.initClassLoader(jobManager.getConfig(), dinkyClassLoader); + String[] statements = SqlUtil.getStatements(statement); + ExecuteJarOperation executeJarOperation = null; + for (String sql : statements) { + String sqlStatement = jobManager.getExecutor().pretreatStatement(sql); + if (ExecuteJarParseStrategy.INSTANCE.match(sqlStatement)) { + executeJarOperation = new ExecuteJarOperation(sqlStatement); + break; + } + SqlType operationType = Operations.getOperationType(sqlStatement); + if (operationType.equals(SqlType.SET) && SetSqlParseStrategy.INSTANCE.match(sqlStatement)) { + CustomSetOperation customSetOperation = new CustomSetOperation(sqlStatement); + customSetOperation.execute(jobManager.getExecutor().getCustomTableEnvironment()); + } else if (operationType.equals(SqlType.ADD)) { + Set files = AddJarSqlParseStrategy.getAllFilePath(sqlStatement); + files.forEach(jobManager.getExecutor()::addJar); + files.forEach(jobManager.getUdfPathContextHolder()::addOtherPlugins); + } else if (operationType.equals(SqlType.ADD_FILE)) { + Set files = AddFileSqlParseStrategy.getAllFilePath(sqlStatement); + files.forEach(jobManager.getExecutor()::addJar); + files.forEach(jobManager.getUdfPathContextHolder()::addFile); + } + } + Assert.notNull(executeJarOperation, () -> new DinkyException("Not found execute jar operation.")); + List urLs = jobManager.getAllFileSet(); + return executeJarOperation.explain(jobManager.getExecutor().getCustomTableEnvironment(), urLs); + } + + public List getUris(String statement) { + String[] statements = SqlUtil.getStatements(statement); + List uriList = new ArrayList<>(); + for (String sql : statements) { + String sqlStatement = jobManager.getExecutor().pretreatStatement(sql); + if (ExecuteJarParseStrategy.INSTANCE.match(sqlStatement)) { + uriList.add(ExecuteJarParseStrategy.getInfo(statement).getUri()); + break; + } + } + return uriList; + } +} diff --git a/dinky-core/src/main/java/org/dinky/job/runner/JobPipelineRunner.java b/dinky-core/src/main/java/org/dinky/job/runner/JobPipelineRunner.java new file mode 100644 index 0000000000..bc3640c3b8 --- /dev/null +++ b/dinky-core/src/main/java/org/dinky/job/runner/JobPipelineRunner.java @@ -0,0 +1,210 @@ +/* + * + * 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.dinky.job.runner; + +import org.dinky.assertion.Asserts; +import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; +import org.dinky.data.result.IResult; +import org.dinky.data.result.InsertResult; +import org.dinky.data.result.ResultBuilder; +import org.dinky.data.result.SqlExplainResult; +import org.dinky.executor.Executor; +import org.dinky.gateway.Gateway; +import org.dinky.gateway.result.GatewayResult; +import org.dinky.job.Job; +import org.dinky.job.JobConfig; +import org.dinky.job.JobManager; +import org.dinky.utils.FlinkStreamEnvironmentUtil; +import org.dinky.utils.LogUtil; +import org.dinky.utils.SqlUtil; +import org.dinky.utils.URLUtils; + +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.rest.messages.JobPlanInfo; +import org.apache.flink.streaming.api.graph.StreamGraph; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.List; + +import cn.hutool.core.text.StrFormatter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class JobPipelineRunner extends AbstractJobRunner { + + private List statements; + + public JobPipelineRunner(JobManager jobManager) { + this.jobManager = jobManager; + this.statements = new ArrayList<>(); + } + + @Override + public void run(JobStatement jobStatement) throws Exception { + statements.add(jobStatement); + jobManager.getExecutor().executeSql(jobStatement.getStatement()); + if (statements.size() == 1) { + if (jobManager.isUseGateway()) { + processWithGateway(); + } else { + processWithoutGateway(); + } + } else { + log.error( + "Only one pipeline job is executed. The statement has be skipped: " + jobStatement.getStatement()); + return; + } + } + + @Override + public SqlExplainResult explain(JobStatement jobStatement) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + statements.add(jobStatement); + // pipeline job execute to generate stream graph. + jobManager.getExecutor().executeSql(jobStatement.getStatement()); + if (statements.size() == 1) { + try { + resultBuilder + .explain(FlinkStreamEnvironmentUtil.getStreamingPlanAsJSON( + jobManager.getExecutor().getStreamGraph())) + .type(jobStatement.getSqlType().getType()) + .parseTrue(true) + .explainTrue(true) + .sql(jobStatement.getStatement()) + .explainTime(LocalDateTime.now()) + .index(jobStatement.getIndex()); + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(jobStatement.getStatement()), + LogUtil.getError(e)); + resultBuilder + .parseTrue(false) + .error(error) + .explainTrue(false) + .type(jobStatement.getSqlType().getType()) + .sql(jobStatement.getStatement()) + .explainTime(LocalDateTime.now()) + .index(jobStatement.getIndex()); + log.error(error); + return resultBuilder.build(); + } + return resultBuilder.build(); + } else { + String error = + "Only one pipeline job is explained. The statement has be skipped: " + jobStatement.getStatement(); + log.error(error); + resultBuilder + .parseTrue(false) + .error(error) + .explainTrue(false) + .type(jobStatement.getSqlType().getType()) + .sql(jobStatement.getStatement()) + .explainTime(LocalDateTime.now()) + .index(jobStatement.getIndex()); + return resultBuilder.build(); + } + } + + @Override + public StreamGraph getStreamGraph(JobStatement jobStatement) { + explain(jobStatement); + if (statements.size() == 1) { + return jobManager.getExecutor().getStreamGraph(); + } else { + throw new DinkyException( + "Only one pipeline job is explained. The statement has be skipped: " + jobStatement.getStatement()); + } + } + + @Override + public JobPlanInfo getJobPlanInfo(JobStatement jobStatement) { + explain(jobStatement); + if (statements.size() == 1) { + return jobManager.getExecutor().getJobPlanInfo(statements); + } else { + throw new DinkyException( + "Only one pipeline job is explained. The statement has be skipped: " + jobStatement.getStatement()); + } + } + + private void processWithGateway() throws Exception { + Executor executor = jobManager.getExecutor(); + JobConfig config = jobManager.getConfig(); + Job job = jobManager.getJob(); + config.addGatewayConfig(executor.getSetConfig()); + config.addGatewayConfig(executor.getCustomTableEnvironment().getConfig().getConfiguration()); + GatewayResult gatewayResult = null; + if (jobManager.getRunMode().isApplicationMode()) { + config.getGatewayConfig().setSql(jobManager.getJobStatementPlan().getStatements()); + gatewayResult = Gateway.build(config.getGatewayConfig()) + .submitJar(executor.getDinkyClassLoader().getUdfPathContextHolder()); + } else { + StreamGraph streamGraph = executor.getStreamGraph(); + streamGraph.setJobName(config.getJobName()); + JobGraph jobGraph = streamGraph.getJobGraph(); + if (Asserts.isNotNullString(config.getSavePointPath())) { + jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(config.getSavePointPath(), true)); + } + gatewayResult = Gateway.build(config.getGatewayConfig()).submitJobGraph(jobGraph); + } + job.setResult(InsertResult.success(gatewayResult.getId())); + job.setJobId(gatewayResult.getId()); + job.setJids(gatewayResult.getJids()); + job.setJobManagerAddress(URLUtils.formatAddress(gatewayResult.getWebURL())); + if (gatewayResult.isSuccess()) { + job.setStatus(Job.JobStatus.SUCCESS); + } else { + job.setStatus(Job.JobStatus.FAILED); + job.setError(gatewayResult.getError()); + } + } + + private void processWithoutGateway() throws Exception { + Executor executor = jobManager.getExecutor(); + JobConfig config = jobManager.getConfig(); + Job job = jobManager.getJob(); + JobClient jobClient = executor.executeAsync(config.getJobName()); + if (Asserts.isNotNull(jobClient)) { + job.setJobId(jobClient.getJobID().toHexString()); + job.setJids(new ArrayList() { + { + add(job.getJobId()); + } + }); + } + if (config.isUseResult()) { + IResult result = ResultBuilder.build( + SqlType.EXECUTE, + job.getId().toString(), + config.getMaxRowNum(), + config.isUseChangeLog(), + config.isUseAutoCancel(), + executor.getTimeZone()) + .getResult(null); + job.setResult(result); + } + } +} diff --git a/dinky-core/src/main/java/org/dinky/job/runner/JobSetRunner.java b/dinky-core/src/main/java/org/dinky/job/runner/JobSetRunner.java new file mode 100644 index 0000000000..277738cdc9 --- /dev/null +++ b/dinky-core/src/main/java/org/dinky/job/runner/JobSetRunner.java @@ -0,0 +1,86 @@ +/* + * + * 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.dinky.job.runner; + +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; +import org.dinky.data.result.SqlExplainResult; +import org.dinky.job.JobManager; +import org.dinky.trans.ddl.CustomSetOperation; +import org.dinky.utils.LogUtil; +import org.dinky.utils.SqlUtil; + +import java.time.LocalDateTime; + +import cn.hutool.core.text.StrFormatter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class JobSetRunner extends AbstractJobRunner { + + public JobSetRunner(JobManager jobManager) { + this.jobManager = jobManager; + } + + @Override + public void run(JobStatement jobStatement) throws Exception { + if (SqlType.SET.equals(jobStatement.getSqlType())) { + CustomSetOperation customSetOperation = new CustomSetOperation(jobStatement.getStatement()); + customSetOperation.execute(jobManager.getExecutor().getCustomTableEnvironment()); + } else if (SqlType.RESET.equals(jobStatement.getSqlType())) { + // todo: reset + throw new RuntimeException("Not support reset operation."); + } + } + + @Override + public SqlExplainResult explain(JobStatement jobStatement) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + try { + CustomSetOperation customSetOperation = new CustomSetOperation(jobStatement.getStatement()); + String explain = customSetOperation.explain(jobManager.getExecutor().getCustomTableEnvironment()); + customSetOperation.execute(jobManager.getExecutor().getCustomTableEnvironment()); + resultBuilder + .explain(explain) + .parseTrue(true) + .explainTrue(true) + .type(jobStatement.getSqlType().getType()) + .sql(jobStatement.getStatement()) + .explainTime(LocalDateTime.now()) + .index(jobStatement.getIndex()); + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(jobStatement.getStatement()), + LogUtil.getError(e)); + resultBuilder + .parseTrue(false) + .error(error) + .explainTrue(false) + .type(jobStatement.getSqlType().getType()) + .sql(jobStatement.getStatement()) + .explainTime(LocalDateTime.now()) + .index(jobStatement.getIndex()); + log.error(error); + return resultBuilder.build(); + } + return resultBuilder.build(); + } +} diff --git a/dinky-core/src/main/java/org/dinky/job/runner/JobSqlRunner.java b/dinky-core/src/main/java/org/dinky/job/runner/JobSqlRunner.java new file mode 100644 index 0000000000..619027f1b6 --- /dev/null +++ b/dinky-core/src/main/java/org/dinky/job/runner/JobSqlRunner.java @@ -0,0 +1,346 @@ +/* + * + * 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.dinky.job.runner; + +import org.dinky.assertion.Asserts; +import org.dinky.constant.FlinkSQLConstant; +import org.dinky.data.enums.GatewayType; +import org.dinky.data.exception.DinkyException; +import org.dinky.data.job.JobStatement; +import org.dinky.data.job.SqlType; +import org.dinky.data.result.IResult; +import org.dinky.data.result.InsertResult; +import org.dinky.data.result.ResultBuilder; +import org.dinky.data.result.SqlExplainResult; +import org.dinky.executor.Executor; +import org.dinky.gateway.Gateway; +import org.dinky.gateway.result.GatewayResult; +import org.dinky.interceptor.FlinkInterceptor; +import org.dinky.interceptor.FlinkInterceptorResult; +import org.dinky.job.Job; +import org.dinky.job.JobConfig; +import org.dinky.job.JobManager; +import org.dinky.utils.SqlUtil; +import org.dinky.utils.URLUtils; + +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; +import org.apache.flink.runtime.rest.messages.JobPlanInfo; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.table.api.TableResult; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; + +import cn.hutool.core.text.StrFormatter; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class JobSqlRunner extends AbstractJobRunner { + + private List statements; + + public JobSqlRunner(JobManager jobManager) { + this.jobManager = jobManager; + this.statements = new ArrayList<>(); + } + + @Override + public void run(JobStatement jobStatement) throws Exception { + statements.add(jobStatement); + if (jobStatement.isFinalExecutableStatement()) { + if (inferStatementSet()) { + handleStatementSet(); + } else { + handleNonStatementSet(); + } + } + } + + @Override + public SqlExplainResult explain(JobStatement jobStatement) { + SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); + // show and desc + if (!jobStatement.getSqlType().isPipeline()) { + try { + resultBuilder = SqlExplainResult.newBuilder( + jobManager.getExecutor().explainSqlRecord(jobStatement.getStatement())); + resultBuilder.parseTrue(true).explainTrue(true); + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(jobStatement.getStatement()), + e.getMessage()); + resultBuilder + .type(jobStatement.getSqlType().getType()) + .index(jobStatement.getIndex()) + .error(error) + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + log.error(error); + return resultBuilder.build(); + } + resultBuilder + .index(jobStatement.getIndex()) + .type(jobStatement.getSqlType().getType()) + .explainTime(LocalDateTime.now()) + .sql(jobStatement.getStatement()); + return resultBuilder.build(); + } + statements.add(jobStatement); + if (!jobStatement.isFinalExecutableStatement()) { + return resultBuilder.invalid().build(); + } + if (inferStatementSet()) { + List inserts = + statements.stream().map(JobStatement::getStatement).collect(Collectors.toList()); + if (!inserts.isEmpty()) { + String sqlSet = StringUtils.join(inserts, FlinkSQLConstant.SEPARATOR); + try { + resultBuilder = + SqlExplainResult.newBuilder(jobManager.getExecutor().explainStatementSet(statements)); + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(jobStatement.getStatement()), + e.getMessage()); + resultBuilder + .sql(sqlSet) + .index(jobStatement.getIndex()) + .type(SqlType.INSERT.getType()) + .error(error) + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + log.error(error); + return resultBuilder.build(); + } + resultBuilder + .type(SqlType.INSERT.getType()) + .index(jobStatement.getIndex()) + .explainTime(LocalDateTime.now()) + .sql(sqlSet); + return resultBuilder.build(); + } + return resultBuilder.invalid().build(); + } else { + try { + resultBuilder = SqlExplainResult.newBuilder( + jobManager.getExecutor().explainSqlRecord(jobStatement.getStatement())); + resultBuilder.parseTrue(true).explainTrue(true); + } catch (Exception e) { + String error = StrFormatter.format( + "Exception in explaining FlinkSQL:\n{}\n{}", + SqlUtil.addLineNumber(jobStatement.getStatement()), + e.getMessage()); + resultBuilder + .type(jobStatement.getSqlType().getType()) + .index(jobStatement.getIndex()) + .error(error) + .parseTrue(false) + .explainTrue(false) + .explainTime(LocalDateTime.now()); + log.error(error); + return resultBuilder.build(); + } + resultBuilder + .type(jobStatement.getSqlType().getType()) + .index(jobStatement.getIndex()) + .explainTime(LocalDateTime.now()) + .sql(jobStatement.getStatement()); + return resultBuilder.build(); + } + } + + @Override + public StreamGraph getStreamGraph(JobStatement jobStatement) { + statements.add(jobStatement); + if (!jobStatement.isFinalExecutableStatement()) { + return null; + } + if (!statements.isEmpty()) { + return jobManager.getExecutor().getStreamGraphFromStatement(statements); + } + throw new DinkyException("None jobs in statement."); + } + + @Override + public JobPlanInfo getJobPlanInfo(JobStatement jobStatement) { + statements.add(jobStatement); + if (!jobStatement.isFinalExecutableStatement()) { + return null; + } + if (!statements.isEmpty()) { + return jobManager.getExecutor().getJobPlanInfo(statements); + } + throw new DinkyException("None jobs in statement."); + } + + private boolean inferStatementSet() { + boolean hasInsert = false; + for (JobStatement item : statements) { + if (item.getSqlType().equals(SqlType.INSERT)) { + hasInsert = true; + break; + } + } + return hasInsert; + } + + private void handleStatementSet() throws Exception { + if (jobManager.isUseGateway()) { + processWithGateway(); + return; + } + processWithoutGateway(); + } + + private void handleNonStatementSet() throws Exception { + if (jobManager.isUseGateway()) { + processSingleInsertWithGateway(); + return; + } + processFirstStatement(); + } + + private void processWithGateway() { + List inserts = + statements.stream().map(JobStatement::getStatement).collect(Collectors.toList()); + jobManager.setCurrentSql(StringUtils.join(inserts, FlinkSQLConstant.SEPARATOR)); + GatewayResult gatewayResult = submitByGateway(statements); + setJobResultFromGatewayResult(gatewayResult); + } + + private void processWithoutGateway() { + if (!statements.isEmpty()) { + List inserts = + statements.stream().map(JobStatement::getStatement).collect(Collectors.toList()); + jobManager.setCurrentSql(StringUtils.join(inserts, FlinkSQLConstant.SEPARATOR)); + TableResult tableResult = jobManager.getExecutor().executeStatements(statements); + updateJobWithTableResult(tableResult); + } + } + + private void processSingleInsertWithGateway() { + List singleInsert = Collections.singletonList(statements.get(0)); + jobManager.getJob().setPipeline(statements.get(0).getSqlType().isPipeline()); + jobManager.setCurrentSql(statements.get(0).getStatement()); + GatewayResult gatewayResult = submitByGateway(singleInsert); + setJobResultFromGatewayResult(gatewayResult); + } + + private void processFirstStatement() throws Exception { + if (statements.isEmpty()) { + return; + } + // Only process the first statement when not using statement set + JobStatement item = statements.get(0); + jobManager.getJob().setPipeline(item.getSqlType().isPipeline()); + jobManager.setCurrentSql(item.getStatement()); + processSingleStatement(item); + } + + private void processSingleStatement(JobStatement item) { + FlinkInterceptorResult flinkInterceptorResult = + FlinkInterceptor.build(jobManager.getExecutor(), item.getStatement()); + if (Asserts.isNotNull(flinkInterceptorResult.getTableResult())) { + updateJobWithTableResult(flinkInterceptorResult.getTableResult(), item.getSqlType()); + } else if (!flinkInterceptorResult.isNoExecute()) { + TableResult tableResult = jobManager.getExecutor().executeSql(item.getStatement()); + updateJobWithTableResult(tableResult, item.getSqlType()); + } + } + + private void setJobResultFromGatewayResult(GatewayResult gatewayResult) { + jobManager.getJob().setResult(InsertResult.success(gatewayResult.getId())); + jobManager.getJob().setJobId(gatewayResult.getId()); + jobManager.getJob().setJids(gatewayResult.getJids()); + jobManager.getJob().setJobManagerAddress(URLUtils.formatAddress(gatewayResult.getWebURL())); + jobManager.getJob().setStatus(gatewayResult.isSuccess() ? Job.JobStatus.SUCCESS : Job.JobStatus.FAILED); + if (!gatewayResult.isSuccess()) { + jobManager.getJob().setError(gatewayResult.getError()); + } + } + + private void updateJobWithTableResult(TableResult tableResult) { + updateJobWithTableResult(tableResult, SqlType.INSERT); + } + + private void updateJobWithTableResult(TableResult tableResult, SqlType sqlType) { + if (tableResult.getJobClient().isPresent()) { + jobManager + .getJob() + .setJobId(tableResult.getJobClient().get().getJobID().toHexString()); + jobManager + .getJob() + .setJids(Collections.singletonList(jobManager.getJob().getJobId())); + } else if (!sqlType.getCategory().getHasJobClient()) { + jobManager.getJob().setJobId(UUID.randomUUID().toString().replace("-", "")); + jobManager + .getJob() + .setJids(Collections.singletonList(jobManager.getJob().getJobId())); + } + + if (jobManager.getConfig().isUseResult()) { + IResult result = ResultBuilder.build( + sqlType, + jobManager.getJob().getId().toString(), + jobManager.getConfig().getMaxRowNum(), + jobManager.getConfig().isUseChangeLog(), + jobManager.getConfig().isUseAutoCancel(), + jobManager.getExecutor().getTimeZone(), + jobManager.getConfig().isMockSinkFunction()) + .getResultWithPersistence(tableResult, jobManager.getHandler()); + jobManager.getJob().setResult(result); + } + } + + private GatewayResult submitByGateway(List inserts) { + JobConfig config = jobManager.getConfig(); + GatewayType runMode = jobManager.getRunMode(); + Executor executor = jobManager.getExecutor(); + + GatewayResult gatewayResult = null; + + // Use gateway need to build gateway config, include flink configuration. + config.addGatewayConfig(executor.getCustomTableEnvironment().getConfig().getConfiguration()); + if (runMode.isApplicationMode()) { + // Application mode need to submit dinky-app.jar that in the hdfs or image. + config.getGatewayConfig().setSql(jobManager.getJobStatementPlan().getStatements()); + gatewayResult = Gateway.build(config.getGatewayConfig()) + .submitJar(executor.getDinkyClassLoader().getUdfPathContextHolder()); + } else { + JobGraph jobGraph = executor.getJobGraphFromInserts(inserts); + // Perjob mode need to set savepoint restore path, when recovery from savepoint. + if (Asserts.isNotNullString(config.getSavePointPath())) { + jobGraph.setSavepointRestoreSettings(SavepointRestoreSettings.forPath(config.getSavePointPath(), true)); + } + // Perjob mode need to submit job graph. + gatewayResult = Gateway.build(config.getGatewayConfig()).submitJobGraph(jobGraph); + } + return gatewayResult; + } +} diff --git a/dinky-core/src/main/java/org/dinky/trans/Operations.java b/dinky-core/src/main/java/org/dinky/trans/Operations.java index f4984f416e..76705eda2e 100644 --- a/dinky-core/src/main/java/org/dinky/trans/Operations.java +++ b/dinky-core/src/main/java/org/dinky/trans/Operations.java @@ -19,8 +19,8 @@ package org.dinky.trans; +import org.dinky.data.job.SqlType; import org.dinky.function.data.model.UDF; -import org.dinky.parser.SqlType; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.table.catalog.FunctionLanguage; diff --git a/dinky-core/src/test/java/org/dinky/core/JobManagerTest.java b/dinky-core/src/test/java/org/dinky/core/JobManagerTest.java deleted file mode 100644 index 978a92024b..0000000000 --- a/dinky-core/src/test/java/org/dinky/core/JobManagerTest.java +++ /dev/null @@ -1,78 +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.dinky.core; - -import org.dinky.data.enums.GatewayType; -import org.dinky.data.result.ResultPool; -import org.dinky.data.result.SelectResult; -import org.dinky.job.JobConfig; -import org.dinky.job.JobManager; -import org.dinky.job.JobResult; - -import org.junit.Ignore; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * JobManagerTest - * - * @since 2021/6/3 - */ -@Ignore -public class JobManagerTest { - - private static final Logger LOGGER = LoggerFactory.getLogger(JobManagerTest.class); - - @Ignore - @Test - public void cancelJobSelect() throws Exception { - JobConfig config = JobConfig.builder() - .type(GatewayType.YARN_SESSION.getLongValue()) - .useResult(true) - .useChangeLog(true) - .useAutoCancel(true) - .clusterId(2) - .jobName("Test") - .fragment(false) - .statementSet(false) - .batchModel(false) - .maxRowNum(100) - .parallelism(1) - .build(); - if (config.isUseRemote()) { - config.setAddress("192.168.123.157:8081"); - } - JobManager jobManager = JobManager.build(config); - String sql1 = "CREATE TABLE Orders (\n" - + " order_number BIGINT,\n" - + " price DECIMAL(32,2),\n" - + " order_time TIMESTAMP(3)\n" - + ") WITH (\n" - + " 'connector' = 'datagen',\n" - + " 'rows-per-second' = '1'\n" - + ");"; - String sql3 = "select order_number,price,order_time from Orders"; - String sql = sql1 + sql3; - JobResult result = jobManager.executeSql(sql); - SelectResult selectResult = ResultPool.get(result.getJobId()); - LOGGER.info("sql:{}, execute result:{}", sql, result.isSuccess()); - } -} diff --git a/dinky-core/src/test/java/org/dinky/job/JobManagerTest.java b/dinky-core/src/test/java/org/dinky/job/JobManagerTest.java new file mode 100644 index 0000000000..ace3b28d94 --- /dev/null +++ b/dinky-core/src/test/java/org/dinky/job/JobManagerTest.java @@ -0,0 +1,214 @@ +/* + * + * 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.dinky.job; + +import static org.junit.jupiter.api.Assertions.*; + +import org.dinky.data.enums.GatewayType; +import org.dinky.data.result.ExplainResult; +import org.dinky.executor.ExecutorConfig; +import org.dinky.explainer.lineage.LineageBuilder; +import org.dinky.explainer.lineage.LineageResult; + +import org.apache.commons.io.IOUtils; +import org.apache.flink.shaded.guava31.com.google.common.io.Resources; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +import org.junit.jupiter.api.Test; + +import com.fasterxml.jackson.databind.node.ObjectNode; + +class JobManagerTest { + + private JobConfig config; + + private JobManager jobManager; + + void initLocalStreamPlanEnvironment() { + config = JobConfig.builder() + .fragment(true) + .statementSet(true) + .type(GatewayType.LOCAL.getLongValue()) + .parallelism(1) + .maxRowNum(100) + .useAutoCancel(true) + .useChangeLog(false) + .useRemote(false) + .useResult(false) + .batchModel(false) + .jobName("Test") + .checkpoint(1000) + .build(); + jobManager = JobManager.buildPlanMode(config); + } + + void initLocalBatchPlanEnvironment() { + config = JobConfig.builder() + .fragment(true) + .statementSet(true) + .type(GatewayType.LOCAL.getLongValue()) + .parallelism(1) + .maxRowNum(100) + .useAutoCancel(true) + .useChangeLog(false) + .useRemote(false) + .useResult(false) + .batchModel(true) + .jobName("Test") + .checkpoint(1000) + .build(); + jobManager = JobManager.buildPlanMode(config); + } + + @Test + void testExplainSql() throws Exception { + checkExplainStreamSqlFromFile("flink/sql/statement-set-stream.sql", 16); + checkExplainStreamSqlFromFile("flink/sql/variable.sql", 3); + checkExplainBatchSqlFromFile("flink/sql/statement-set-batch.sql", 16); + } + + @Test + void testGetStreamGraph() throws Exception { + checkGetStreamGraphFromFile("flink/sql/statement-set-stream.sql"); + checkGetStreamGraphFromFile("flink/sql/variable.sql"); + checkGetBatchStreamGraphFromFile("flink/sql/statement-set-batch.sql"); + } + + @Test + void testGetJobPlanJson() throws Exception { + checkGetStreamJobPlanJsonFromFile("flink/sql/statement-set-stream.sql"); + checkGetStreamJobPlanJsonFromFile("flink/sql/variable.sql"); + checkGetBatchJobPlanJsonFromFile("flink/sql/statement-set-batch.sql"); + } + + /*@Ignore + @Test + void testExecuteSql() throws Exception { + checkStreamExecuteSqlFromFile("flink/sql/statement-set-stream.sql"); + checkBatchExecuteSqlFromFile("flink/sql/statement-set-batch.sql"); + }*/ + + @Test + void testLineageSqlSingle() throws Exception { + String statement = + IOUtils.toString(Resources.getResource("flink/sql/single-insert.sql"), StandardCharsets.UTF_8); + LineageResult result = LineageBuilder.getColumnLineageByLogicalPlan(statement, ExecutorConfig.DEFAULT); + assertNotNull(result); + assertEquals(2, result.getTables().size()); + assertEquals(4, result.getRelations().size()); + } + + private void checkExplainStreamSqlFromFile(String path, int total) throws IOException { + String statement = IOUtils.toString(Resources.getResource(path), StandardCharsets.UTF_8); + initLocalStreamPlanEnvironment(); + checkExplainSql(statement, total); + jobManager.close(); + } + + private void checkExplainBatchSqlFromFile(String path, int total) throws IOException { + String statement = IOUtils.toString(Resources.getResource(path), StandardCharsets.UTF_8); + initLocalBatchPlanEnvironment(); + checkExplainSql(statement, total); + jobManager.close(); + } + + private void checkExplainSql(String statement, int total) { + ExplainResult explainResult = jobManager.explainSql(statement); + assertNotNull(explainResult); + explainResult.getSqlExplainResults().forEach(sqlExplainResult -> { + if (!sqlExplainResult.isParseTrue() || !sqlExplainResult.isExplainTrue()) { + // Flink 1.14,1.15,1.16,1.17 not support RTAS + if (sqlExplainResult.getError().contains("not support") + // Flink 1.14 not support show create view + || sqlExplainResult.getError().contains("SQL parse failed. Encountered \"VIEW\"")) { + sqlExplainResult.setParseTrue(true); + sqlExplainResult.setExplainTrue(true); + } else { + throw new RuntimeException(sqlExplainResult.getError()); + } + } + assertTrue(sqlExplainResult.isParseTrue()); + assertTrue(sqlExplainResult.isExplainTrue()); + }); + assertEquals(total, explainResult.getTotal()); + assertTrue(explainResult.isCorrect()); + } + + private void checkGetStreamGraphFromFile(String path) throws IOException { + String statement = IOUtils.toString(Resources.getResource(path), StandardCharsets.UTF_8); + initLocalStreamPlanEnvironment(); + checkGetStreamGraph(statement); + jobManager.close(); + } + + private void checkGetBatchStreamGraphFromFile(String path) throws IOException { + String statement = IOUtils.toString(Resources.getResource(path), StandardCharsets.UTF_8); + initLocalBatchPlanEnvironment(); + checkGetStreamGraph(statement); + jobManager.close(); + } + + private void checkGetStreamGraph(String statement) { + ObjectNode streamGraph = jobManager.getStreamGraph(statement); + assertNotNull(streamGraph); + assertNotNull(streamGraph.get("nodes")); + } + + private void checkGetStreamJobPlanJsonFromFile(String path) throws IOException { + String statement = IOUtils.toString(Resources.getResource(path), StandardCharsets.UTF_8); + initLocalStreamPlanEnvironment(); + checkGetJobPlanJson(statement); + jobManager.close(); + } + + private void checkGetBatchJobPlanJsonFromFile(String path) throws IOException { + String statement = IOUtils.toString(Resources.getResource(path), StandardCharsets.UTF_8); + initLocalBatchPlanEnvironment(); + checkGetJobPlanJson(statement); + jobManager.close(); + } + + private void checkGetJobPlanJson(String statement) { + String jobPlanJson = jobManager.getJobPlanJson(statement); + assertNotNull(jobPlanJson); + } + + private void checkStreamExecuteSqlFromFile(String path) throws Exception { + String statement = IOUtils.toString(Resources.getResource(path), StandardCharsets.UTF_8); + initLocalStreamPlanEnvironment(); + checkExecuteSql(statement); + jobManager.close(); + } + + private void checkBatchExecuteSqlFromFile(String path) throws Exception { + String statement = IOUtils.toString(Resources.getResource(path), StandardCharsets.UTF_8); + initLocalBatchPlanEnvironment(); + checkExecuteSql(statement); + jobManager.close(); + } + + private void checkExecuteSql(String statement) throws Exception { + JobResult jobResult = jobManager.executeSql(statement); + assertNotNull(jobResult); + assertTrue(jobResult.isSuccess()); + } +} diff --git a/dinky-core/src/test/java/org/dinky/job/JobTestHandler.java b/dinky-core/src/test/java/org/dinky/job/JobTestHandler.java new file mode 100644 index 0000000000..6e36d88b73 --- /dev/null +++ b/dinky-core/src/test/java/org/dinky/job/JobTestHandler.java @@ -0,0 +1,67 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.job; + +import java.util.List; + +public class JobTestHandler implements JobHandler { + @Override + public boolean init(Job job) { + return true; + } + + @Override + public boolean ready() { + return true; + } + + @Override + public boolean running() { + return true; + } + + @Override + public boolean success() { + return true; + } + + @Override + public boolean failed() { + return true; + } + + @Override + public boolean callback() { + return true; + } + + @Override + public boolean close() { + return true; + } + + @Override + public void persistResultData(List jobIds) {} + + @Override + public JobReadHandler getReadHandler() { + return null; + } +} diff --git a/dinky-core/src/test/java/org/dinky/parse/SqlTypeTest.java b/dinky-core/src/test/java/org/dinky/parse/SqlTypeTest.java index 0c0dc2ee4e..77a5d61edb 100644 --- a/dinky-core/src/test/java/org/dinky/parse/SqlTypeTest.java +++ b/dinky-core/src/test/java/org/dinky/parse/SqlTypeTest.java @@ -19,7 +19,7 @@ package org.dinky.parse; -import org.dinky.parser.SqlType; +import org.dinky.data.job.SqlType; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; diff --git a/dinky-core/src/test/resources/META-INF/services/org.dinky.job.JobHandler b/dinky-core/src/test/resources/META-INF/services/org.dinky.job.JobHandler new file mode 100644 index 0000000000..bab0c103f1 --- /dev/null +++ b/dinky-core/src/test/resources/META-INF/services/org.dinky.job.JobHandler @@ -0,0 +1 @@ +org.dinky.job.JobTestHandler \ No newline at end of file diff --git a/dinky-core/src/test/resources/flink/sql/single-insert.sql b/dinky-core/src/test/resources/flink/sql/single-insert.sql new file mode 100644 index 0000000000..3e3044e7e9 --- /dev/null +++ b/dinky-core/src/test/resources/flink/sql/single-insert.sql @@ -0,0 +1,26 @@ +CREATE TABLE datagen_source +( + id BIGINT, + name STRING, + sex INT, + age INT +) WITH ( + 'connector' = 'datagen' + ); + +CREATE TABLE print_sink +( + id BIGINT, + name STRING, + sex INT, + age INT +) WITH ( + 'connector' = 'print' + ); + +INSERT INTO print_sink +SELECT id, + name, + sex, + age +from datagen_source; \ No newline at end of file diff --git a/dinky-core/src/test/resources/flink/sql/statement-set-batch.sql b/dinky-core/src/test/resources/flink/sql/statement-set-batch.sql new file mode 100644 index 0000000000..26955409c6 --- /dev/null +++ b/dinky-core/src/test/resources/flink/sql/statement-set-batch.sql @@ -0,0 +1,98 @@ +CREATE TABLE datagen_source +( + id BIGINT, + name STRING, + sex INT, + age INT +) WITH ( + 'connector' = 'datagen', + 'number-of-rows' = '10' + ); + +CREATE TABLE print_sink +( + id BIGINT, + name STRING, + sex INT, + age INT +) WITH ( + 'connector' = 'print' + ); + +CREATE TABLE print_sink2 +( + sex BIGINT, + total BIGINT +) WITH ( + 'connector' = 'print' + ); + +CREATE TABLE print_sink3 +( + id BIGINT, + name STRING, + sex INT, + age INT +) WITH ( + 'connector' = 'print' + ); + +INSERT INTO print_sink +SELECT id, + name, + sex, + age +from datagen_source /*+ OPTIONS('rows-per-second'='1') */ ; + +SELECT id as select_id, + name as select_name +from datagen_source; + +SHOW TABLES; + +WITH sex_with AS ( + SELECT id, sex + FROM datagen_source +) +SELECT sex, 1 as cnt +FROM sex_with +GROUP BY sex; + +INSERT INTO print_sink2 +SELECT sex, 2 as total +FROM datagen_source +GROUP BY sex; + +CREATE TABLE print_sink4 + WITH ( + 'connector' = 'print' + ) +AS SELECT id, name, sex, age FROM datagen_source WHERE mod(id, 10) = 4; + +REPLACE TABLE print_sink3 +WITH ( + 'connector' = 'print' +) +AS SELECT id, name, sex, age FROM datagen_source WHERE mod(id, 10) = 0; + +CREATE VIEW t1(s) AS VALUES ('c'), ('a'), ('b'), ('b'), ('c'); + +CREATE VIEW t2(s) AS VALUES ('d'), ('e'), ('a'), ('b'), ('b'); + +(SELECT s FROM t1) UNION (SELECT s FROM t2); + +ALTER TABLE print_sink3 RENAME TO print_sink5; + +DESCRIBE print_sink5; + +USE MODULES core; + +SHOW TABLES; + +SHOW CREATE VIEW t1; + +UNLOAD MODULE core; + +SET 'table.local-time-zone' = 'Europe/Berlin'; + +DROP TABLE print_sink5; \ No newline at end of file diff --git a/dinky-core/src/test/resources/flink/sql/statement-set-stream.sql b/dinky-core/src/test/resources/flink/sql/statement-set-stream.sql new file mode 100644 index 0000000000..60ad4d9227 --- /dev/null +++ b/dinky-core/src/test/resources/flink/sql/statement-set-stream.sql @@ -0,0 +1,98 @@ +CREATE TABLE datagen_source +( + id BIGINT, + name STRING, + sex INT, + age INT +) WITH ( + 'connector' = 'datagen', + 'number-of-rows' = '10' + ); + +CREATE TABLE print_sink +( + id BIGINT, + name STRING, + sex INT, + age INT +) WITH ( + 'connector' = 'print' + ); + +CREATE TABLE print_sink2 +( + sex BIGINT, + total BIGINT +) WITH ( + 'connector' = 'print' + ); + +CREATE TABLE print_sink3 +( + id BIGINT, + name STRING, + sex INT, + age INT +) WITH ( + 'connector' = 'print' + ); + +INSERT INTO print_sink +SELECT id, + name, + sex, + age +from datagen_source /*+ OPTIONS('rows-per-second'='1') */ ; + +SELECT id as select_id, + name as select_name +from datagen_source; + +WITH sex_with AS ( + SELECT id, sex + FROM datagen_source +) +SELECT sex, 1 as cnt +FROM sex_with +GROUP BY sex; + +INSERT INTO print_sink2 +SELECT sex, 2 as total +FROM datagen_source +GROUP BY sex; + +CREATE TABLE print_sink4 + WITH ( + 'connector' = 'print' + ) +AS SELECT id, name, sex, age FROM datagen_source WHERE mod(id, 10) = 4; + +REPLACE TABLE print_sink3 +WITH ( + 'connector' = 'print' +) +AS SELECT id, name, sex, age FROM datagen_source WHERE mod(id, 10) = 0; + +CREATE VIEW t1(s) AS VALUES ('c'), ('a'), ('b'), ('b'), ('c'); + +CREATE VIEW t2(s) AS VALUES ('d'), ('e'), ('a'), ('b'), ('b'); + +(SELECT s FROM t1) UNION (SELECT s FROM t2); + +ALTER TABLE print_sink3 RENAME TO print_sink5; + +DESCRIBE print_sink5; + +EXPLAIN SELECT id, name, sex, age FROM datagen_source; + +USE MODULES core; + +SHOW TABLES; + +SHOW CREATE VIEW t1; + +UNLOAD MODULE core; + +SET 'table.local-time-zone' = 'Europe/Berlin'; + +DROP TABLE print_sink5; \ No newline at end of file diff --git a/dinky-core/src/test/resources/flink/sql/variable.sql b/dinky-core/src/test/resources/flink/sql/variable.sql new file mode 100644 index 0000000000..cd4b7256ec --- /dev/null +++ b/dinky-core/src/test/resources/flink/sql/variable.sql @@ -0,0 +1,27 @@ +tb:=datagen_source; +CREATE TABLE datagen_source +( + id BIGINT, + name STRING, + sex INT, + age INT +) WITH ( + 'connector' = 'datagen' + ); + +CREATE TABLE print_sink +( + id BIGINT, + name STRING, + sex INT, + age INT +) WITH ( + 'connector' = 'print' + ); + +INSERT INTO print_sink +SELECT id, + name, + sex, + age +from ${tb}; \ No newline at end of file diff --git a/dinky-flink/dinky-flink-1.20/pom.xml b/dinky-flink/dinky-flink-1.20/pom.xml index e10f28ea54..bfc0f9adc2 100644 --- a/dinky-flink/dinky-flink-1.20/pom.xml +++ b/dinky-flink/dinky-flink-1.20/pom.xml @@ -22,9 +22,33 @@ + org.apache.flink - flink-python + flink-clients + ${flink.version} + + + + + org.apache.flink + flink-table-api-java-bridge + ${flink.version} + + + + org.apache.flink + flink-table-runtime ${flink.version} @@ -38,6 +62,11 @@ + + org.apache.flink + flink-python + ${flink.version} + org.apache.flink flink-connector-base diff --git a/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/TaskConfig/BasicConfig.tsx b/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/TaskConfig/BasicConfig.tsx index 65f9524207..253e928d0d 100644 --- a/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/TaskConfig/BasicConfig.tsx +++ b/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/TaskConfig/BasicConfig.tsx @@ -42,6 +42,7 @@ export const BasicConfig = (props: { tempData: TempData; data: TaskState; onValuesChange?: (changedValues: any, values: TaskState) => void; + setCurrentState?: (values: TaskState) => void; isLockTask: boolean; }) => { const { alertGroup, flinkConfigOptions, flinkUdfOptions } = props.tempData; @@ -171,8 +172,10 @@ export const BasicConfig = (props: { /> formRef.current!!} + setCurrentState={props.setCurrentState} defaultValue={[]} /> diff --git a/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/TaskConfig/ProFormFlinkUdfConfig.tsx b/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/TaskConfig/ProFormFlinkUdfConfig.tsx index 572bdf84f2..a4d62ce161 100644 --- a/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/TaskConfig/ProFormFlinkUdfConfig.tsx +++ b/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/TaskConfig/ProFormFlinkUdfConfig.tsx @@ -30,14 +30,17 @@ import React, { useState } from 'react'; import { DefaultOptionType } from 'antd/es/select'; import { TaskUdfRefer } from '@/types/Studio/data'; import { calculatorWidth } from '@/pages/DataStudioNew/CenterTabContent/SqlTask/TaskConfig/function'; +import {TaskState} from "@/pages/DataStudioNew/type"; export const ProFormFlinkUdfConfig = (props: { containerWidth: number; + data: TaskState; flinkUdfOptions: DefaultOptionType[]; proFormInstance: () => ProFormInstance; + setCurrentState?: (values: TaskState) => void; defaultValue: { className: string; name: string }[]; }) => { - const { flinkUdfOptions, containerWidth, proFormInstance } = props; + const { data, flinkUdfOptions, containerWidth, proFormInstance, setCurrentState } = props; const [currentSelectUdfIndexMap, setCurrentSelectUdfIndexMap] = useState< Map @@ -86,14 +89,14 @@ export const ProFormFlinkUdfConfig = (props: { }; })} onChange={(value: string) => { + const simpleClassName = value?.split('.')?.pop() ?? ''; + const lowerName = + simpleClassName.charAt(0).toLowerCase() + simpleClassName.slice(1); setCurrentSelectUdfIndexMap((prevState) => { const newState = new Map(prevState); - newState.set(index, { name: '', className: value }); + newState.set(index, { name: lowerName, className: value }); return newState; }); - const simpleClassName = value?.split('.')?.pop() ?? ''; - const lowerName = - simpleClassName.charAt(0).toLowerCase() + simpleClassName.slice(1); proFormInstance().setFieldsValue({ configJson: { udfRefer: { @@ -104,6 +107,10 @@ export const ProFormFlinkUdfConfig = (props: { } } }); + let newCurrentState = data; + newCurrentState.configJson.udfRefer[index].className = value; + newCurrentState.configJson.udfRefer[index].name = lowerName; + setCurrentState?.(newCurrentState); }} /> void; + setCurrentState?: (values: TaskState) => void; isLockTask: boolean; }) => { const { data, tempData } = props; @@ -53,6 +54,7 @@ export default (props: { tempData={props.tempData} data={props.data} onValuesChange={props.onValuesChange} + setCurrentState={props.setCurrentState} isLockTask={props.isLockTask} /> ) diff --git a/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/index.tsx b/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/index.tsx index f56c006fd7..d8461a084f 100644 --- a/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/index.tsx +++ b/dinky-web/src/pages/DataStudioNew/CenterTabContent/SqlTask/index.tsx @@ -291,6 +291,7 @@ export const SqlTask = memo((props: FlinkSqlProps & any) => { tempData={tempData} data={currentState} onValuesChange={debounce(onValuesChange, 500)} + setCurrentState={debounce(setCurrentState, 500)} isLockTask={isLockTask} /> ) @@ -415,13 +416,15 @@ export const SqlTask = memo((props: FlinkSqlProps & any) => { updateAction({ actionType: DataStudioActionType.TASK_RUN_DEBUG, params: { - taskId: params.taskId + taskId: params.taskId, + columns: res.data?.result?.columns ?? [], + rowData: res.data?.result?.rowData ?? [], } }); setCurrentState((prevState) => { return { ...prevState, - status: res.data.status === 'SUCCESS' ? 'RUNNING' : res.data.status + status: res.data.status === 'SUCCESS' ? (res.data.pipeline?'RUNNING':'SUCCESS') : res.data.status }; }); } @@ -429,14 +432,12 @@ export const SqlTask = memo((props: FlinkSqlProps & any) => { const handleStop = useCallback(async () => { const result = await cancelTask('', currentState.taskId, false); - if (result.success) { - setCurrentState((prevState) => { - return { - ...prevState, - status: 'CANCEL' - }; - }); - } + setCurrentState((prevState) => { + return { + ...prevState, + status: 'CANCEL' + }; + }); }, [currentState.taskId]); const handleGotoDevOps = useCallback(async () => { diff --git a/dinky-web/src/pages/DataStudioNew/Toolbar/Service/Result/index.tsx b/dinky-web/src/pages/DataStudioNew/Toolbar/Service/Result/index.tsx index a6ecaf1200..f0bed3cad5 100644 --- a/dinky-web/src/pages/DataStudioNew/Toolbar/Service/Result/index.tsx +++ b/dinky-web/src/pages/DataStudioNew/Toolbar/Service/Result/index.tsx @@ -64,7 +64,8 @@ export default (props: { taskId: number; action: any; dialect: string }) => { const [searchedColumn, setSearchedColumn] = useState(''); const searchInput = useRef(null); useEffect(() => { - if (actionType === DataStudioActionType.TASK_PREVIEW_RESULT) { + if (actionType === DataStudioActionType.TASK_PREVIEW_RESULT + || actionType === DataStudioActionType.TASK_RUN_DEBUG) { if (data.mockSinkResult == true) { setDataList(convertMockResultToList({ columns: params.columns, rowData: params.rowData })); } else {