From ad5add31757ce130840d4119890f46dcda87c8ac Mon Sep 17 00:00:00 2001 From: Davidhua1996 Date: Fri, 28 May 2021 15:57:38 +0800 Subject: [PATCH 1/3] Add flink-engineconn plugin; Optimize the architecture of engine plugin; Avoid the useless log in refreshing plugin cache. fixes #745 close #745 --- .../engineconn-plugins/flink/pom.xml | 454 ++++++++++++ .../flink/src/main/assembly/distribution.xml | 72 ++ .../flink/client/config/Environment.java | 288 ++++++++ .../client/config/entries/ConfigEntry.java | 68 ++ .../client/config/entries/ExecutionEntry.java | 372 ++++++++++ .../client/context/ExecutionContext.java | 655 ++++++++++++++++++ .../deployment/ClusterDescriptorAdapter.java | 166 +++++ .../ClusterDescriptorAdapterFactory.java | 53 ++ .../client/deployment/ProgramDeployer.java | 74 ++ ...rnApplicationClusterDescriptorAdapter.java | 59 ++ .../YarnPerJobClusterDescriptorAdapter.java | 84 +++ .../LinkisYarnClusterClientFactory.java | 70 ++ .../flink/client/result/AbstractResult.java | 51 ++ .../flink/client/result/BatchResult.java | 141 ++++ .../flink/client/result/ChangelogResult.java | 221 ++++++ .../flink/client/result/Result.java | 55 ++ .../flink/client/result/ResultUtil.java | 122 ++++ .../flink/client/result/TypedResult.java | 89 +++ .../sql/operation/AbstractJobOperation.java | 170 +++++ .../flink/client/sql/operation/JobInfo.java | 14 + .../client/sql/operation/JobOperation.java | 40 ++ .../client/sql/operation/NonJobOperation.java | 23 + .../flink/client/sql/operation/Operation.java | 31 + .../sql/operation/OperationFactory.java | 129 ++++ .../client/sql/operation/OperationUtil.java | 78 +++ .../operation/impl/CreateViewOperation.java | 69 ++ .../sql/operation/impl/DDLOperation.java | 92 +++ .../impl/DescribeTableOperation.java | 109 +++ .../sql/operation/impl/DropViewOperation.java | 79 +++ .../sql/operation/impl/ExplainOperation.java | 78 +++ .../sql/operation/impl/InsertOperation.java | 140 ++++ .../sql/operation/impl/ResetOperation.java | 52 ++ .../sql/operation/impl/SelectOperation.java | 260 +++++++ .../sql/operation/impl/SetOperation.java | 111 +++ .../operation/impl/ShowCatalogsOperation.java | 47 ++ .../impl/ShowCurrentCatalogOperation.java | 45 ++ .../impl/ShowCurrentDatabaseOperation.java | 45 ++ .../impl/ShowDatabasesOperation.java | 47 ++ .../impl/ShowFunctionsOperation.java | 47 ++ .../operation/impl/ShowModulesOperation.java | 47 ++ .../operation/impl/ShowTablesOperation.java | 63 ++ .../operation/impl/ShowViewsOperation.java | 66 ++ .../operation/impl/UseCatalogOperation.java | 58 ++ .../operation/impl/UseDatabaseOperation.java | 56 ++ .../sql/operation/result/ColumnInfo.java | 101 +++ .../sql/operation/result/ConstantNames.java | 59 ++ .../sql/operation/result/ResultKind.java | 30 + .../sql/operation/result/ResultSet.java | 189 +++++ .../result/ResultSetJsonDeserializer.java | 200 ++++++ .../result/ResultSetJsonSerializer.java | 114 +++ .../flink/client/utils/SqlCommandParser.java | 403 +++++++++++ .../flink/client/utils/YarnConfLoader.java | 51 ++ .../exception/ExecutorInitException.java | 47 ++ .../exception/FlinkInitFailedException.java | 38 + .../exception/JobExecutionException.java | 38 + .../exception/SqlExecutionException.java | 38 + .../flink/exception/SqlParseException.java | 38 + .../flink/util/RetryUtil.java | 54 ++ .../resources/linkis-engineconn.properties | 27 + .../src/main/resources/log4j2-engineconn.xml | 64 ++ .../flink/FlinkEngineConnPlugin.scala | 77 ++ .../flink/config/FlinkEnvConfiguration.scala | 58 ++ .../config/FlinkResourceConfiguration.scala | 41 ++ .../flink/context/EnvironmentContext.scala | 106 +++ .../context/FlinkEngineConnContext.scala | 40 ++ .../FlinkApplicationJobExecutor.scala | 88 +++ .../flink/executor/FlinkJobExecutor.scala | 133 ++++ .../flink/executor/FlinkSQLJobExecutor.scala | 226 ++++++ .../FlinkApplicationExecutorFactory.scala | 44 ++ .../factory/FlinkEngineConnFactory.scala | 186 +++++ .../factory/FlinkSQLExecutorFactory.scala | 63 ++ .../launch/FlinkEngineConnLaunchBuilder.scala | 42 ++ .../flink/listener/FlinkListener.scala | 47 ++ .../flink/listener/FlinkListenerGroup.scala | 62 ++ .../FlinkEngineConnResourceFactory.scala | 47 ++ .../flink/util/FlinkValueFormatUtil.scala | 42 ++ .../resources/linkis-engineconn.properties | 4 +- .../src/main/resources/log4j2-engineconn.xml | 21 +- .../hive/conf/HiveEngineConfiguration.scala | 3 + .../hive/creation/HiveEngineConnFactory.scala | 55 +- .../executor/HiveEngineConnExecutor.scala | 4 +- .../hive/hook/HiveAddJarsEngineHook.scala | 10 +- .../hive/hook/UseDatabaseEngineHook.scala | 9 +- .../HiveProcessEngineConnLaunchBuilder.scala | 7 + .../engineconn-plugins/io_file/pom.xml | 2 +- .../src/main/assembly/distribution.xml | 1 + .../io/utils/ReflectionUtils.java | 15 + .../resources/linkis-engineconn.properties | 3 +- .../src/main/resources/log4j2-engineconn.xml | 15 +- .../engineplugin/io/IoEngineConnPlugin.scala | 2 - .../io/executor/IoEngineConnExecutor.scala | 13 +- .../io/factory/IoEngineConnFactory.scala | 39 +- .../io/service/FsProxyService.scala | 2 +- .../engineconn-plugins/jdbc/pom.xml | 1 + .../resources/linkis-engineconn.properties | 2 +- .../src/main/resources/log4j2-engineconn.xml | 21 +- .../jdbc/JDBCEngineConnPlugin.scala | 2 +- .../jdbc/factory/JDBCEngineConnFactory.scala | 38 +- .../resources/linkis-engineconn.properties | 2 +- .../pipeline/PipelineEngineConnPlugin.scala | 4 +- .../pipeline/conversions/FsConvertions.scala | 4 +- .../pipeline/executor/CSVExecutor.scala | 3 +- .../pipeline/executor/CopyExecutor.scala | 1 - .../pipeline/executor/ExcelExecutor.scala | 1 + .../executor/PipelineEngineConnExecutor.scala | 15 +- .../executor/PipelineExecutorSelector.scala | 47 ++ .../factory/PipelineEngineConnFactory.scala | 36 +- .../src/main/resources/log4j2-engineconn.xml | 21 +- .../executor/PythonEngineConnExecutor.scala | 10 +- .../python/executor/PythonSession.scala | 5 +- .../factory/PythonEngineConnFactory.scala | 35 +- .../python/hook/PythonVersionEngineHook.scala | 6 +- .../resources/linkis-engineconn.properties | 2 +- .../src/main/resources/log4j2-engineconn.xml | 24 +- .../executor/ShellEngineConnExecutor.scala | 2 +- .../factory/ShellEngineConnFactory.scala | 37 +- .../engineconn-plugins/spark/pom.xml | 13 + .../resources/linkis-engineconn.properties | 3 +- .../src/main/resources/log4j2-engineconn.xml | 29 +- .../src/main/resources/python/mix_pyspark.py | 12 +- .../spark/config/SparkConfiguration.scala | 7 +- ...SparkEngineServerSpringConfiguration.scala | 5 - .../spark/executor/SQLSession.scala | 4 +- .../executor/SparkEngineConnExecutor.scala | 14 +- .../spark/executor/SparkPythonExecutor.scala | 47 +- .../spark/executor/SparkScalaExecutor.scala | 30 +- .../spark/executor/SparkSqlExecutor.scala | 2 +- .../parser/SparkCombinedCodeParser.scala | 5 +- .../factory/SparkEngineConnFactory.scala | 29 +- .../factory/SparkPythonExecutorFactory.scala | 43 +- .../factory/SparkScalaExecutorFactory.scala | 57 +- .../factory/SparkSqlExecutorFactory.scala | 44 +- .../spark/imexport/ExportData.scala | 7 +- .../spark/imexport/LoadData.scala | 7 +- ...SubmitProcessEngineConnLaunchBuilder.scala | 12 +- .../spark/mdq/MDQPostExecutionHook.scala | 7 +- .../spark/mdq/MDQPreExecutionHook.scala | 28 +- .../spark/metadata/HPOBuilder.scala | 77 ++ .../spark/metadata/LineageResolveLogic.scala | 184 +++++ .../spark/metadata/MetaDataInfoTool.scala | 47 ++ .../spark/metadata/SparkHiveObject.scala | 31 + .../metadata/SparkSQLHistoryParser.scala | 342 +++++++++ .../spark/utils/EngineUtils.scala | 4 +- .../spark/sql/MethodInvokeHelper.scala} | 24 +- .../spark/sql/SparkLogicalPlanHelper.scala | 34 + .../cache/GuavaEngineConnPluginCache.java | 5 +- .../common/conf/EngineConnPluginConf.scala | 2 + .../common/creation/EngineConnFactory.scala | 77 +- .../common/creation/ExecutorFactory.scala | 56 +- ...ctory.scala => LabelExecutorFactory.scala} | 28 +- .../JavaProcessEngineConnLaunchBuilder.scala | 10 +- .../ProcessEngineConnLaunchBuilder.scala | 1 + .../AbstractEngineResourceFactory.scala | 16 +- .../resource/UserTimeoutNodeResource.scala | 30 + .../DefaultEngineConnPluginLoader.java | 4 +- .../config/EngineConnPluginLoaderConf.scala | 3 +- .../server/LinkisEngineConnPluginServer.java | 8 +- .../dao/impl/EngineConnBmlResourceMapper.xml | 6 +- .../conf/EngineConnPluginConfiguration.scala | 4 +- ...stractEngineConnBmlResourceGenerator.scala | 6 +- ...aultEngineConnResourceFactoryService.scala | 4 +- 161 files changed, 9250 insertions(+), 565 deletions(-) create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/pom.xml create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/assembly/distribution.xml create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/Environment.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ConfigEntry.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ExecutionEntry.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/context/ExecutionContext.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapter.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapterFactory.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ProgramDeployer.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnApplicationClusterDescriptorAdapter.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnPerJobClusterDescriptorAdapter.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/factory/LinkisYarnClusterClientFactory.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/AbstractResult.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/BatchResult.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ChangelogResult.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/Result.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ResultUtil.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/TypedResult.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/AbstractJobOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobInfo.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/NonJobOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/Operation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationFactory.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationUtil.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/CreateViewOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DDLOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DescribeTableOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DropViewOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ExplainOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/InsertOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ResetOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SelectOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SetOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCatalogsOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentCatalogOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentDatabaseOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowDatabasesOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowFunctionsOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowModulesOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowTablesOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowViewsOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseCatalogOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseDatabaseOperation.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ColumnInfo.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ConstantNames.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultKind.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSet.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonDeserializer.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonSerializer.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/SqlCommandParser.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/YarnConfLoader.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/ExecutorInitException.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/FlinkInitFailedException.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/JobExecutionException.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlExecutionException.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlParseException.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/RetryUtil.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/linkis-engineconn.properties create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/log4j2-engineconn.xml create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/FlinkEngineConnPlugin.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkEnvConfiguration.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkResourceConfiguration.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/EnvironmentContext.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/FlinkEngineConnContext.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkApplicationJobExecutor.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkJobExecutor.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkSQLJobExecutor.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkApplicationExecutorFactory.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkEngineConnFactory.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkSQLExecutorFactory.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/launch/FlinkEngineConnLaunchBuilder.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListener.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListenerGroup.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/resource/FlinkEngineConnResourceFactory.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/FlinkValueFormatUtil.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/io/utils/ReflectionUtils.java create mode 100644 linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/PipelineExecutorSelector.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/HPOBuilder.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/LineageResolveLogic.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/MetaDataInfoTool.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkHiveObject.scala create mode 100644 linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkSQLHistoryParser.scala rename linkis-engineconn-plugins/engineconn-plugins/{io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/utils/ReflectionUtils.scala => spark/src/main/scala/org/apache/spark/sql/MethodInvokeHelper.scala} (56%) create mode 100644 linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/org/apache/spark/sql/SparkLogicalPlanHelper.scala rename linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/{JavaProcessEngineConnFactory.scala => LabelExecutorFactory.scala} (53%) create mode 100644 linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/resource/UserTimeoutNodeResource.scala diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/pom.xml b/linkis-engineconn-plugins/engineconn-plugins/flink/pom.xml new file mode 100644 index 0000000000..0ff898c8c6 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/pom.xml @@ -0,0 +1,454 @@ + + + + linkis + com.webank.wedatasphere.linkis + 1.0.0-RC1 + ../../pom.xml + + 4.0.0 + + linkis-engineconn-plugin-flink + + 1.11.1 + 1.2.1 + + + + + + + org.apache.flink + flink-core + ${flink.version} + + + + org.apache.flink + flink-runtime_${scala.binary.version} + ${flink.version} + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + + + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + + + + org.apache.flink + flink-sql-client_${scala.binary.version} + ${flink.version} + + + + commons-cli + commons-cli + 1.3.1 + + + org.apache.flink + flink-table-common + ${flink.version} + + + + org.apache.flink + flink-table-api-java + ${flink.version} + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${flink.version} + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + + + + + + + + + + + + + + + + org.apache.flink + flink-sql-parser + ${flink.version} + + + + org.apache.flink + flink-shaded-jackson + 2.10.1-9.0 + + + + org.apache.flink + flink-yarn_${scala.binary.version} + ${flink.version} + + + org.apache.flink + flink-shaded-hadoop2 + + + org.eclipse.jetty + * + + + io.netty + netty + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + httpclient + org.apache.httpcomponents + + + httpcore + org.apache.httpcomponents + + + hadoop-common + org.apache.hadoop + + + hadoop-hdfs + org.apache.hadoop + + + hadoop-yarn-client + org.apache.hadoop + + + hadoop-mapreduce-client-core + org.apache.hadoop + + + hadoop-yarn-common + org.apache.hadoop + + + + + + junit + junit + 4.12 + test + + + org.apache.flink + flink-connector-hive_2.11 + ${flink.version} + + + + + org.apache.hive + hive-exec + ${hive.version} + + + httpclient + org.apache.httpcomponents + + + hadoop-yarn-server-resourcemanager + org.apache.hadoop + + + + + + + com.webank.wedatasphere.linkis + linkis-engineconn-plugin-core + ${linkis.version} + + + + com.webank.wedatasphere.linkis + linkis-computation-engineconn + ${linkis.version} + + + httpclient + org.apache.httpcomponents + + + + + + + com.webank.wedatasphere.linkis + linkis-message-scheduler + ${linkis.version} + provided + + + + com.webank.wedatasphere.linkis + linkis-once-engineconn + ${linkis.version} + + + + + com.webank.wedatasphere.linkis + linkis-hadoop-common + ${linkis.version} + provided + + + + org.apache.hadoop + hadoop-yarn-api + ${hadoop.version} + + + + + + + + + org.apache.hadoop + hadoop-yarn-client + ${hadoop.version} + + + servlet-api + javax.servlet + + + + + + + com.webank.wedatasphere.linkis + linkis-storage + ${linkis.version} + + + commons-httpclient + commons-httpclient + + + commons-configuration + commons-configuration + + + commons-logging + commons-logging + + + netty + io.netty + + + xlsx-streamer + com.monitorjbl + + + netty-all + io.netty + + + + hadoop-common + org.apache.hadoop + + + + hadoop-hdfs + org.apache.hadoop + + + + hadoop-auth + org.apache.hadoop + + + + org.json4s + json4s-jackson_${scala.binary.version} + + + provided + + + + org.apache.commons + commons-exec + 1.3 + + + + + com.google.guava + guava + ${guava.version} + provided + + + + org.scala-lang + scala-library + provided + + + org.scala-lang + scala-compiler + provided + + + org.scala-lang + scala-reflect + provided + + + + org.json4s + json4s-jackson_${scala.binary.version} + ${json4s.version} + provided + + + + io.netty + netty-all + ${netty.version} + provided + + + + com.fasterxml.jackson.core + jackson-databind + ${fasterxml.jackson.version} + + + + com.webank.wedatasphere.linkis + linkis-bml-engine-hook + ${linkis.version} + + + commons-logging + commons-logging + + + + + com.github.rholder + guava-retrying + 2.0.0 + + + + + + + + org.apache.maven.plugins + maven-deploy-plugin + + + + net.alchim31.maven + scala-maven-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-assembly-plugin + 2.3 + false + + + make-assembly + package + + single + + + + src/main/assembly/distribution.xml + + + + + + false + flink-engineconn + false + false + + src/main/assembly/distribution.xml + + + + + + + src/main/java + + **/*.xml + + + + src/main/resources + + **/*.properties + **/application.yml + **/bootstrap.yml + **/log4j2.xml + + + + + \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/assembly/distribution.xml b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/assembly/distribution.xml new file mode 100644 index 0000000000..40a5a00027 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/assembly/distribution.xml @@ -0,0 +1,72 @@ + + + + linkis-engineconn-plugin-flink + + dir + zip + + true + flink + + + + + + /dist/v${flink.version}/lib + true + true + false + false + true + + + + + + + ${basedir}/src/main/resources + + * + + 0777 + 0755 + /dist/v${flink.version}/conf + unix + + + + ${basedir}/target + + *.jar + + + *doc.jar + + 0777 + /plugin/${flink.version} + + + + + + + diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/Environment.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/Environment.java new file mode 100644 index 0000000000..d02cbf4d3d --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/Environment.java @@ -0,0 +1,288 @@ +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.FlinkInitFailedException; +import java.io.IOException; +import java.net.URL; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonMappingException; +import org.apache.flink.table.client.config.ConfigUtil; +import org.apache.flink.table.client.config.entries.CatalogEntry; +import org.apache.flink.table.client.config.entries.ConfigurationEntry; +import org.apache.flink.table.client.config.entries.DeploymentEntry; +import org.apache.flink.table.client.config.entries.ExecutionEntry; +import org.apache.flink.table.client.config.entries.FunctionEntry; +import org.apache.flink.table.client.config.entries.ModuleEntry; +import org.apache.flink.table.client.config.entries.TableEntry; +import org.apache.flink.table.client.config.entries.ViewEntry; + + +public class Environment { + + public static final String SESSION_ENTRY = "session"; + + public static final String EXECUTION_ENTRY = "execution"; + + public static final String CONFIGURATION_ENTRY = "table"; + + public static final String DEPLOYMENT_ENTRY = "deployment"; + + private ExecutionEntry execution; + + private Map modules; + + private Map catalogs; + + private Map tables; + + private Map functions; + + private ConfigurationEntry configuration; + + private DeploymentEntry deployment; + + public Environment() { + this.modules = new LinkedHashMap<>(); + this.catalogs = Collections.emptyMap(); + this.tables = Collections.emptyMap(); + this.functions = Collections.emptyMap(); + this.execution = ExecutionEntry.DEFAULT_INSTANCE; + this.configuration = ConfigurationEntry.DEFAULT_INSTANCE; + this.deployment = DeploymentEntry.DEFAULT_INSTANCE; + } + + public Map getModules() { + return modules; + } + + public void setModules(List> modules) throws FlinkInitFailedException { + this.modules = new LinkedHashMap<>(modules.size()); + + for(Map config : modules) { + final ModuleEntry entry = ModuleEntry.create(config); + if (this.modules.containsKey(entry.getName())) { + throw new FlinkInitFailedException( + String.format("Cannot register module '%s' because a module with this name is already registered.", + entry.getName())); + } + this.modules.put(entry.getName(), entry); + } + } + + public Map getCatalogs() { + return catalogs; + } + + public void setCatalogs(List> catalogs) throws FlinkInitFailedException { + this.catalogs = new HashMap<>(catalogs.size()); + + for(Map config : catalogs) { + final CatalogEntry catalog = CatalogEntry.create(config); + if (this.catalogs.containsKey(catalog.getName())) { + throw new FlinkInitFailedException( + String.format("Cannot create catalog '%s' because a catalog with this name is already registered.", + catalog.getName())); + } + this.catalogs.put(catalog.getName(), catalog); + } + } + + public Map getTables() { + return tables; + } + + public void setTables(List> tables) throws FlinkInitFailedException { + this.tables = new LinkedHashMap<>(tables.size()); + + for(Map config : tables) { + final TableEntry table = TableEntry.create(config); + if (this.tables.containsKey(table.getName())) { + throw new FlinkInitFailedException( + "Cannot create table '" + table + .getName() + "' because a table with this name is already registered."); + } + this.tables.put(table.getName(), table); + } + } + + public Map getFunctions() { + return functions; + } + + public void setFunctions(List> functions) throws FlinkInitFailedException { + this.functions = new HashMap<>(functions.size()); + + for(Map config : functions) { + final FunctionEntry function = FunctionEntry.create(config); + if (this.functions.containsKey(function.getName())) { + throw new FlinkInitFailedException( + "Cannot create function '" + function + .getName() + "' because a function with this name is already registered."); + } + this.functions.put(function.getName(), function); + } + } + + public void setExecution(Map config) { + this.execution = ExecutionEntry.create(config); + } + + public ExecutionEntry getExecution() { + return execution; + } + + public void setConfiguration(Map config) { + this.configuration = ConfigurationEntry.create(config); + } + + public ConfigurationEntry getConfiguration() { + return configuration; + } + + public void setDeployment(Map config) { + this.deployment = DeploymentEntry.create(config); + } + + public DeploymentEntry getDeployment() { + return deployment; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(); + sb.append("==================== Session =====================\n"); + sb.append("===================== Modules =====================\n"); + modules.forEach((name, module) -> { + sb.append("- ").append(ModuleEntry.MODULE_NAME).append(": ").append(name).append("\n"); + module.asMap().forEach((k, v) -> sb.append(" ").append(k).append(": ").append(v).append('\n')); + }); + sb.append("===================== Catalogs =====================\n"); + catalogs.forEach((name, catalog) -> { + sb.append("- ").append(CatalogEntry.CATALOG_NAME).append(": ").append(name).append("\n"); + catalog.asMap().forEach((k, v) -> sb.append(" ").append(k).append(": ").append(v).append('\n')); + }); + sb.append("===================== Tables =====================\n"); + tables.forEach((name, table) -> { + sb.append("- ").append(TableEntry.TABLES_NAME).append(": ").append(name).append("\n"); + table.asMap().forEach((k, v) -> sb.append(" ").append(k).append(": ").append(v).append('\n')); + }); + sb.append("=================== Functions ====================\n"); + functions.forEach((name, function) -> { + sb.append("- ").append(FunctionEntry.FUNCTIONS_NAME).append(": ").append(name).append("\n"); + function.asMap().forEach((k, v) -> sb.append(" ").append(k).append(": ").append(v).append('\n')); + }); + sb.append("=================== Execution ====================\n"); + execution.asTopLevelMap().forEach((k, v) -> sb.append(k).append(": ").append(v).append('\n')); + sb.append("================== Configuration =================\n"); + configuration.asMap().forEach((k, v) -> sb.append(k).append(": ").append(v).append('\n')); + sb.append("=================== Deployment ===================\n"); + deployment.asTopLevelMap().forEach((k, v) -> sb.append(k).append(": ").append(v).append('\n')); + return sb.toString(); + } + + // -------------------------------------------------------------------------------------------- + + /** + * Parses an environment file from an URL. + */ + public static Environment parse(URL url) throws IOException, FlinkInitFailedException { + try { + return new ConfigUtil.LowerCaseYamlMapper().readValue(url, Environment.class); + } catch (JsonMappingException e) { + throw new FlinkInitFailedException("Could not parse environment file. Cause: " + e.getMessage(), e); + } + } + + /** + * Parses an environment file from an String. + */ + public static Environment parse(String content) throws IOException, FlinkInitFailedException { + try { + return new ConfigUtil.LowerCaseYamlMapper().readValue(content, Environment.class); + } catch (JsonMappingException e) { + throw new FlinkInitFailedException("Could not parse environment file. Cause: " + e.getMessage(), e); + } + } + + /** + * Merges two environments. The properties of the first environment might be overwritten by the second one. + */ + public static Environment merge(Environment env1, Environment env2) { + if(null==env2){ + return env1; + } + final Environment mergedEnv = new Environment(); + + // merge modules + final Map modules = new LinkedHashMap<>(env1.getModules()); + modules.putAll(env2.getModules()); + mergedEnv.modules = modules; + + // merge catalogs + final Map catalogs = new HashMap<>(env1.getCatalogs()); + catalogs.putAll(env2.getCatalogs()); + mergedEnv.catalogs = catalogs; + + // merge tables + final Map tables = new LinkedHashMap<>(env1.getTables()); + tables.putAll(env2.getTables()); + mergedEnv.tables = tables; + + // merge functions + final Map functions = new HashMap<>(env1.getFunctions()); + functions.putAll(env2.getFunctions()); + mergedEnv.functions = functions; + + // merge execution properties + mergedEnv.execution = ExecutionEntry.merge(env1.getExecution(), env2.getExecution()); + + // merge configuration properties + mergedEnv.configuration = ConfigurationEntry.merge(env1.getConfiguration(), env2.getConfiguration()); + + // merge deployment properties + mergedEnv.deployment = DeploymentEntry.merge(env1.getDeployment(), env2.getDeployment()); + + return mergedEnv; + } + + @Override + public Environment clone() { + return enrich(this, Collections.emptyMap(), Collections.emptyMap()); + } + + /** + * Enriches an environment with new/modified properties or views and returns the new instance. + */ + public static Environment enrich( + Environment env, + Map properties, + Map views) { + final Environment enrichedEnv = new Environment(); + + enrichedEnv.modules = new LinkedHashMap<>(env.getModules()); + + // merge catalogs + enrichedEnv.catalogs = new LinkedHashMap<>(env.getCatalogs()); + + // merge tables + enrichedEnv.tables = new LinkedHashMap<>(env.getTables()); + enrichedEnv.tables.putAll(views); + + // merge functions + enrichedEnv.functions = new HashMap<>(env.getFunctions()); + + // enrich execution properties + enrichedEnv.execution = ExecutionEntry.enrich(env.execution, properties); + + // enrich configuration properties + enrichedEnv.configuration = ConfigurationEntry.enrich(env.configuration, properties); + + // enrich deployment properties + enrichedEnv.deployment = DeploymentEntry.enrich(env.deployment, properties); + + return enrichedEnv; + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ConfigEntry.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ConfigEntry.java new file mode 100644 index 0000000000..b628bc167d --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ConfigEntry.java @@ -0,0 +1,68 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.entries; + + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.FlinkInitFailedException; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.descriptors.DescriptorProperties; + + +public abstract class ConfigEntry { + + protected final DescriptorProperties properties; + + protected ConfigEntry(DescriptorProperties properties) throws FlinkInitFailedException { + try { + validate(properties); + } catch (ValidationException e) { + throw new FlinkInitFailedException("Invalid configuration entry.", e); + } + + this.properties = properties; + } + + /** + * Performs syntactic validation. + */ + protected abstract void validate(DescriptorProperties properties); + + public Map asMap() { + return properties.asMap(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ConfigEntry that = (ConfigEntry) o; + return Objects.equals(properties, that.properties); + } + + @Override + public int hashCode() { + return Objects.hash(properties); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ExecutionEntry.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ExecutionEntry.java new file mode 100644 index 0000000000..6425c2901e --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ExecutionEntry.java @@ -0,0 +1,372 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.entries; + + +import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment.EXECUTION_ENTRY; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.FlinkInitFailedException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.time.Time; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.client.config.ConfigUtil; +import org.apache.flink.table.descriptors.DescriptorProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class ExecutionEntry extends ConfigEntry { + + private static final Logger LOG = LoggerFactory.getLogger(ExecutionEntry.class); + + public static final List AVAILABLE_PLANNERS = Arrays.asList( + ExecutionEntry.EXECUTION_PLANNER_VALUE_OLD, + ExecutionEntry.EXECUTION_PLANNER_VALUE_BLINK); + + public static final List AVAILABLE_EXECUTION_TYPES = Arrays.asList( + ExecutionEntry.EXECUTION_TYPE_VALUE_BATCH, + ExecutionEntry.EXECUTION_TYPE_VALUE_STREAMING); + + public static final String EXECUTION_PLANNER = "planner"; + + public static final String EXECUTION_PLANNER_VALUE_OLD = "old"; + + public static final String EXECUTION_PLANNER_VALUE_BLINK = "blink"; + + public static final String EXECUTION_TYPE = "type"; + + public static final String EXECUTION_TYPE_VALUE_STREAMING = "streaming"; + + public static final String EXECUTION_TYPE_VALUE_BATCH = "batch"; + + private static final String EXECUTION_TIME_CHARACTERISTIC = "time-characteristic"; + + private static final String EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME = "event-time"; + + private static final String EXECUTION_TIME_CHARACTERISTIC_VALUE_PROCESSING_TIME = "processing-time"; + + private static final String EXECUTION_PERIODIC_WATERMARKS_INTERVAL = "periodic-watermarks-interval"; + + private static final String EXECUTION_MIN_STATE_RETENTION = "min-idle-state-retention"; + + private static final String EXECUTION_MAX_STATE_RETENTION = "max-idle-state-retention"; + + private static final String EXECUTION_PARALLELISM = "parallelism"; + + private static final String EXECUTION_MAX_PARALLELISM = "max-parallelism"; + + public static final String EXECUTION_RESULT_MODE = "result-mode"; + + public static final String EXECUTION_RESULT_MODE_VALUE_CHANGELOG = "changelog"; + + public static final String EXECUTION_RESULT_MODE_VALUE_TABLE = "table"; + + public static final String EXECUTION_MAX_BUFFER_SIZE = "max_buffer_size"; + + private static final String EXECUTION_RESTART_STRATEGY_TYPE = "restart-strategy.type"; + + private static final String EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FALLBACK = "fallback"; + + private static final String EXECUTION_RESTART_STRATEGY_TYPE_VALUE_NONE = "none"; + + private static final String EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FIXED_DELAY = "fixed-delay"; + + private static final String EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FAILURE_RATE = "failure-rate"; + + private static final String EXECUTION_RESTART_STRATEGY_ATTEMPTS = "restart-strategy.attempts"; + + private static final String EXECUTION_RESTART_STRATEGY_DELAY = "restart-strategy.delay"; + + private static final String EXECUTION_RESTART_STRATEGY_FAILURE_RATE_INTERVAL = "restart-strategy.failure-rate-interval"; + + private static final String EXECUTION_RESTART_STRATEGY_MAX_FAILURES_PER_INTERVAL = "restart-strategy.max-failures-per-interval"; + + public static final String EXECUTION_CURRENT_CATALOG = "current-catalog"; + + public static final String EXECUTION_CURRENT_DATABASE = "current-database"; + + private ExecutionEntry(DescriptorProperties properties) throws FlinkInitFailedException { + super(properties); + } + + @Override + protected void validate(DescriptorProperties properties) { + properties.validateEnumValues( + EXECUTION_PLANNER, + true, + Arrays.asList( + EXECUTION_PLANNER_VALUE_OLD, + EXECUTION_PLANNER_VALUE_BLINK)); + properties.validateEnumValues( + EXECUTION_TYPE, + true, + Arrays.asList( + EXECUTION_TYPE_VALUE_BATCH, + EXECUTION_TYPE_VALUE_STREAMING)); + properties.validateEnumValues( + EXECUTION_TIME_CHARACTERISTIC, + true, + Arrays.asList( + EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME, + EXECUTION_TIME_CHARACTERISTIC_VALUE_PROCESSING_TIME)); + properties.validateLong(EXECUTION_PERIODIC_WATERMARKS_INTERVAL, true, 1); + properties.validateLong(EXECUTION_MIN_STATE_RETENTION, true, 0); + properties.validateLong(EXECUTION_MAX_STATE_RETENTION, true, 0); + properties.validateInt(EXECUTION_PARALLELISM, true, 1); + properties.validateInt(EXECUTION_MAX_PARALLELISM, true, 1); + properties.validateEnumValues( + EXECUTION_RESTART_STRATEGY_TYPE, + true, + Arrays.asList( + EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FALLBACK, + EXECUTION_RESTART_STRATEGY_TYPE_VALUE_NONE, + EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FIXED_DELAY, + EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FAILURE_RATE)); + properties.validateInt(EXECUTION_RESTART_STRATEGY_ATTEMPTS, true, 1); + properties.validateLong(EXECUTION_RESTART_STRATEGY_DELAY, true, 0); + properties.validateLong(EXECUTION_RESTART_STRATEGY_FAILURE_RATE_INTERVAL, true, 1); + properties.validateInt(EXECUTION_RESTART_STRATEGY_MAX_FAILURES_PER_INTERVAL, true, 1); + properties.validateString(EXECUTION_CURRENT_CATALOG, true, 1); + properties.validateString(EXECUTION_CURRENT_DATABASE, true, 1); + properties.validateInt(EXECUTION_MAX_BUFFER_SIZE, true, 1); + } + + public EnvironmentSettings getEnvironmentSettings() { + final EnvironmentSettings.Builder builder = EnvironmentSettings.newInstance(); + + if (inStreamingMode()) { + builder.inStreamingMode(); + } else if (inBatchMode()) { + builder.inBatchMode(); + } + + final String planner = properties.getOptionalString(EXECUTION_PLANNER) + .orElse(EXECUTION_PLANNER_VALUE_OLD); + + if (planner.equals(EXECUTION_PLANNER_VALUE_OLD)) { + builder.useOldPlanner(); + } else if (planner.equals(EXECUTION_PLANNER_VALUE_BLINK)) { + builder.useBlinkPlanner(); + } + + return builder.build(); + } + + public boolean inStreamingMode() { + return properties.getOptionalString(EXECUTION_TYPE) + .map((v) -> v.equals(EXECUTION_TYPE_VALUE_STREAMING)) + .orElse(false); + } + + public boolean inBatchMode() { + return properties.getOptionalString(EXECUTION_TYPE) + .map((v) -> v.equals(EXECUTION_TYPE_VALUE_BATCH)) + .orElse(false); + } + + public boolean isStreamingPlanner() { + final String planner = properties.getOptionalString(EXECUTION_PLANNER) + .orElse(EXECUTION_PLANNER_VALUE_OLD); + + // Blink planner is a streaming planner + if (planner.equals(EXECUTION_PLANNER_VALUE_BLINK)) { + return true; + } + // Old planner can be a streaming or batch planner + else if (planner.equals(EXECUTION_PLANNER_VALUE_OLD)) { + return inStreamingMode(); + } + + return false; + } + + public String getPlanner() { + return properties.getOptionalString(EXECUTION_PLANNER).orElse(EXECUTION_PLANNER_VALUE_OLD); + } + + public boolean isBatchPlanner() { + final String planner = properties.getOptionalString(EXECUTION_PLANNER) + .orElse(EXECUTION_PLANNER_VALUE_OLD); + + // Blink planner is not a batch planner + if (planner.equals(EXECUTION_PLANNER_VALUE_BLINK)) { + return false; + } + // Old planner can be a streaming or batch planner + else if (planner.equals(EXECUTION_PLANNER_VALUE_OLD)) { + return inBatchMode(); + } + + return false; + } + + public TimeCharacteristic getTimeCharacteristic() { + return properties.getOptionalString(EXECUTION_TIME_CHARACTERISTIC) + .flatMap((v) -> { + switch (v) { + case EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME: + return Optional.of(TimeCharacteristic.EventTime); + case EXECUTION_TIME_CHARACTERISTIC_VALUE_PROCESSING_TIME: + return Optional.of(TimeCharacteristic.ProcessingTime); + default: + return Optional.empty(); + } + }) + .orElseGet(() -> + useDefaultValue( + EXECUTION_TIME_CHARACTERISTIC, + TimeCharacteristic.EventTime, + EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME)); + } + + public long getPeriodicWatermarksInterval() { + return properties.getOptionalLong(EXECUTION_PERIODIC_WATERMARKS_INTERVAL) + .orElseGet(() -> useDefaultValue(EXECUTION_PERIODIC_WATERMARKS_INTERVAL, 200L)); + } + + public long getMinStateRetention() { + return properties.getOptionalLong(EXECUTION_MIN_STATE_RETENTION) + .orElseGet(() -> useDefaultValue(EXECUTION_MIN_STATE_RETENTION, 0L)); + } + + public long getMaxStateRetention() { + return properties.getOptionalLong(EXECUTION_MAX_STATE_RETENTION) + .orElseGet(() -> useDefaultValue(EXECUTION_MAX_STATE_RETENTION, 0L)); + } + + public int getParallelism() { + return properties.getOptionalInt(EXECUTION_PARALLELISM) + .orElseGet(() -> useDefaultValue(EXECUTION_PARALLELISM, 1)); + } + + public int getMaxParallelism() { + return properties.getOptionalInt(EXECUTION_MAX_PARALLELISM) + .orElseGet(() -> useDefaultValue(EXECUTION_MAX_PARALLELISM, 128)); + } + + public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() { + return properties.getOptionalString(EXECUTION_RESTART_STRATEGY_TYPE) + .flatMap((v) -> { + switch (v) { + case EXECUTION_RESTART_STRATEGY_TYPE_VALUE_NONE: + return Optional.of(RestartStrategies.noRestart()); + case EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FIXED_DELAY: + final int attempts = properties.getOptionalInt(EXECUTION_RESTART_STRATEGY_ATTEMPTS) + .orElseGet(() -> useDefaultValue(EXECUTION_RESTART_STRATEGY_ATTEMPTS, Integer.MAX_VALUE)); + final long fixedDelay = properties.getOptionalLong(EXECUTION_RESTART_STRATEGY_DELAY) + .orElseGet(() -> useDefaultValue(EXECUTION_RESTART_STRATEGY_DELAY, 10_000L)); + return Optional.of(RestartStrategies.fixedDelayRestart(attempts, fixedDelay)); + case EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FAILURE_RATE: + final int failureRate = properties + .getOptionalInt(EXECUTION_RESTART_STRATEGY_MAX_FAILURES_PER_INTERVAL) + .orElseGet(() -> useDefaultValue(EXECUTION_RESTART_STRATEGY_MAX_FAILURES_PER_INTERVAL, 1)); + final long failureInterval = properties + .getOptionalLong(EXECUTION_RESTART_STRATEGY_FAILURE_RATE_INTERVAL) + .orElseGet( + () -> useDefaultValue(EXECUTION_RESTART_STRATEGY_FAILURE_RATE_INTERVAL, 60_000L)); + final long attemptDelay = properties.getOptionalLong(EXECUTION_RESTART_STRATEGY_DELAY) + .orElseGet(() -> useDefaultValue(EXECUTION_RESTART_STRATEGY_DELAY, 10_000L)); + return Optional.of(RestartStrategies.failureRateRestart( + failureRate, + Time.milliseconds(failureInterval), + Time.milliseconds(attemptDelay))); + default: + return Optional.empty(); + } + }) + .orElseGet(() -> + useDefaultValue( + EXECUTION_RESTART_STRATEGY_TYPE, + RestartStrategies.fallBackRestart(), + EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FALLBACK)); + } + + public Optional getCurrentCatalog() { + return properties.getOptionalString(EXECUTION_CURRENT_CATALOG); + } + + public Optional getCurrentDatabase() { + return properties.getOptionalString(EXECUTION_CURRENT_DATABASE); + } + + public Map asTopLevelMap() { + return properties.asPrefixedMap(EXECUTION_ENTRY + '.'); + } + + public int getMaxBufferSize() { + return properties.getOptionalInt(EXECUTION_MAX_BUFFER_SIZE) + .orElseGet(() -> useDefaultValue(EXECUTION_MAX_BUFFER_SIZE, 5000)); + } + + private V useDefaultValue(String key, V defaultValue) { + return useDefaultValue(key, defaultValue, defaultValue.toString()); + } + + private V useDefaultValue(String key, V defaultValue, String defaultString) { + LOG.info("Property '{}.{}' not specified. Using default value: {}", EXECUTION_ENTRY, key, defaultString); + return defaultValue; + } + + // -------------------------------------------------------------------------------------------- + + public static ExecutionEntry create(Map config) throws FlinkInitFailedException { + return new ExecutionEntry(ConfigUtil.normalizeYaml(config)); + } + + /** + * Merges two execution entries. The properties of the first execution entry might be + * overwritten by the second one. + */ + public static ExecutionEntry merge(ExecutionEntry execution1, ExecutionEntry execution2) throws FlinkInitFailedException { + final Map mergedProperties = new HashMap<>(execution1.asMap()); + mergedProperties.putAll(execution2.asMap()); + + final DescriptorProperties properties = new DescriptorProperties(true); + properties.putProperties(mergedProperties); + + return new ExecutionEntry(properties); + } + + /** + * Creates a new execution entry enriched with additional properties that are prefixed with + * {@link Environment#EXECUTION_ENTRY}. + */ + public static ExecutionEntry enrich(ExecutionEntry execution, Map prefixedProperties) throws FlinkInitFailedException { + final Map enrichedProperties = new HashMap<>(execution.asMap()); + + prefixedProperties.forEach((k, v) -> { + final String normalizedKey = k.toLowerCase(); + if (k.startsWith(EXECUTION_ENTRY + '.')) { + enrichedProperties.put(normalizedKey.substring(EXECUTION_ENTRY.length() + 1), v); + } + }); + + final DescriptorProperties properties = new DescriptorProperties(true); + properties.putProperties(enrichedProperties); + + return new ExecutionEntry(properties); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/context/ExecutionContext.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/context/ExecutionContext.java new file mode 100644 index 0000000000..b1f9ba1bc6 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/context/ExecutionContext.java @@ -0,0 +1,655 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.factory.LinkisYarnClusterClientFactory; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import java.lang.reflect.Method; +import java.net.URL; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.function.Supplier; +import javax.annotation.Nullable; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.api.java.ExecutionEnvironment; +import org.apache.flink.client.ClientUtils; +import org.apache.flink.client.deployment.ClusterClientServiceLoader; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.bridge.java.BatchTableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.bridge.java.internal.BatchTableEnvironmentImpl; +import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl; +import org.apache.flink.table.catalog.Catalog; +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.client.config.entries.ExecutionEntry; +import org.apache.flink.table.client.config.entries.SinkTableEntry; +import org.apache.flink.table.client.config.entries.SourceSinkTableEntry; +import org.apache.flink.table.client.config.entries.SourceTableEntry; +import org.apache.flink.table.client.config.entries.TableEntry; +import org.apache.flink.table.client.config.entries.TemporalTableEntry; +import org.apache.flink.table.client.config.entries.ViewEntry; +import org.apache.flink.table.delegation.Executor; +import org.apache.flink.table.delegation.ExecutorFactory; +import org.apache.flink.table.delegation.Planner; +import org.apache.flink.table.delegation.PlannerFactory; +import org.apache.flink.table.descriptors.CoreModuleDescriptorValidator; +import org.apache.flink.table.factories.BatchTableSinkFactory; +import org.apache.flink.table.factories.BatchTableSourceFactory; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.flink.table.factories.ComponentFactoryService; +import org.apache.flink.table.factories.ModuleFactory; +import org.apache.flink.table.factories.TableFactoryService; +import org.apache.flink.table.factories.TableSinkFactory; +import org.apache.flink.table.factories.TableSourceFactory; +import org.apache.flink.table.functions.AggregateFunction; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.FunctionService; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.table.functions.UserDefinedFunction; +import org.apache.flink.table.module.Module; +import org.apache.flink.table.module.ModuleManager; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.sources.TableSource; +import org.apache.flink.util.TemporaryClassLoaderContext; +import org.apache.flink.yarn.YarnClusterDescriptor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 运行上下文 + */ +public class ExecutionContext { + + private static final Logger LOG = LoggerFactory.getLogger(ExecutionContext.class); + + private final Environment environment; + private final ClassLoader classLoader; + + private final Configuration flinkConfig; + private final LinkisYarnClusterClientFactory clusterClientFactory; + + private TableEnvironment tableEnv; + private ExecutionEnvironment execEnv; + private StreamExecutionEnvironment streamExecEnv; + private Executor executor; + + // Members that should be reused in the same session. + private SessionState sessionState; + + private ExecutionContext( + Environment environment, + @Nullable SessionState sessionState, + List dependencies, + Configuration flinkConfig, + ClusterClientServiceLoader clusterClientServiceLoader) throws SqlExecutionException { + this.environment = environment; + this.flinkConfig = flinkConfig; + // create class loader + classLoader = ClientUtils.buildUserCodeClassLoader( + Collections.emptyList(), + Collections.emptyList(), + this.getClass().getClassLoader(), + flinkConfig); + + // Initialize the TableEnvironment. + initializeTableEnvironment(sessionState); + + LOG.debug("Deployment descriptor: {}", environment.getDeployment()); + LOG.info("flinkConfig config: {}", flinkConfig); + clusterClientFactory = new LinkisYarnClusterClientFactory(); + } + + + public void setString(String key, String value) { + this.flinkConfig.setString(key, value); + } + + public void setBoolean(String key, boolean value) { + this.flinkConfig.setBoolean(key, value); + } + + public Configuration getFlinkConfig() { + return flinkConfig; + } + + public ClassLoader getClassLoader() { + return classLoader; + } + + public Environment getEnvironment() { + return environment; + } + + public YarnClusterDescriptor createClusterDescriptor() { + return clusterClientFactory.createClusterDescriptor(this.flinkConfig); + } + + public Map getCatalogs() { + Map catalogs = new HashMap<>(); + for (String name : tableEnv.listCatalogs()) { + tableEnv.getCatalog(name).ifPresent(c -> catalogs.put(name, c)); + } + return catalogs; + } + + public SessionState getSessionState() { + return this.sessionState; + } + + /** + * Executes the given supplier using the execution context's classloader as thread classloader. + */ + public R wrapClassLoader(Supplier supplier) { + try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(classLoader)) { + return supplier.get(); + } + } + + /** + * Executes the given Runnable using the execution context's classloader as thread classloader. + */ + void wrapClassLoader(Runnable runnable) { + try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(classLoader)) { + runnable.run(); + } + } + + public TableEnvironment getTableEnvironment() { + return tableEnv; + } + + public ExecutionConfig getExecutionConfig() { + if (streamExecEnv != null) { + return streamExecEnv.getConfig(); + } else { + return execEnv.getConfig(); + } + } + + public LinkisYarnClusterClientFactory getClusterClientFactory() { + return clusterClientFactory; + } + + public Pipeline createPipeline(String name) { + return wrapClassLoader(() -> { + if (streamExecEnv != null) { + StreamTableEnvironmentImpl streamTableEnv = (StreamTableEnvironmentImpl) tableEnv; + return streamTableEnv.getPipeline(name); + } else { + BatchTableEnvironmentImpl batchTableEnv = (BatchTableEnvironmentImpl) tableEnv; + return batchTableEnv.getPipeline(name); + } + }); + } + + + /** Returns a builder for this {@link ExecutionContext}. */ + public static Builder builder( + Environment defaultEnv, + Environment sessionEnv, + List dependencies, + Configuration configuration, + ClusterClientServiceLoader serviceLoader) { + return new Builder(defaultEnv, sessionEnv, dependencies, configuration, + serviceLoader); + } + + private Module createModule(Map moduleProperties, ClassLoader classLoader) { + final ModuleFactory factory = + TableFactoryService.find(ModuleFactory.class, moduleProperties, classLoader); + return factory.createModule(moduleProperties); + } + + private Catalog createCatalog(String name, Map catalogProperties, ClassLoader classLoader) { + final CatalogFactory factory = + TableFactoryService.find(CatalogFactory.class, catalogProperties, classLoader); + return factory.createCatalog(name, catalogProperties); + } + + private static TableSource createTableSource(ExecutionEntry execution, Map sourceProperties, + ClassLoader classLoader) throws SqlExecutionException { + if (execution.isStreamingPlanner()) { + final TableSourceFactory factory = (TableSourceFactory) + TableFactoryService.find(TableSourceFactory.class, sourceProperties, classLoader); + return factory.createTableSource(sourceProperties); + } else if (execution.isBatchPlanner()) { + final BatchTableSourceFactory factory = (BatchTableSourceFactory) + TableFactoryService.find(BatchTableSourceFactory.class, sourceProperties, classLoader); + return factory.createBatchTableSource(sourceProperties); + } + throw new SqlExecutionException("Unsupported execution type for sources."); + } + + private static TableSink createTableSink(ExecutionEntry execution, Map sinkProperties, + ClassLoader classLoader) throws SqlExecutionException { + if (execution.isStreamingPlanner()) { + final TableSinkFactory factory = (TableSinkFactory) + TableFactoryService.find(TableSinkFactory.class, sinkProperties, classLoader); + return factory.createTableSink(sinkProperties); + } else if (execution.isBatchPlanner()) { + final BatchTableSinkFactory factory = (BatchTableSinkFactory) + TableFactoryService.find(BatchTableSinkFactory.class, sinkProperties, classLoader); + return factory.createBatchTableSink(sinkProperties); + } + throw new SqlExecutionException("Unsupported execution type for sinks."); + } + + private TableEnvironment createStreamTableEnvironment( + StreamExecutionEnvironment env, + EnvironmentSettings settings, + TableConfig config, + Executor executor, + CatalogManager catalogManager, + ModuleManager moduleManager, + FunctionCatalog functionCatalog) { + final Map plannerProperties = settings.toPlannerProperties(); + final Planner planner = ComponentFactoryService.find(PlannerFactory.class, plannerProperties) + .create(plannerProperties, executor, config, functionCatalog, catalogManager); + + return new StreamTableEnvironmentImpl( + catalogManager, + moduleManager, + functionCatalog, + config, + env, + planner, + executor, + settings.isStreamingMode(), + classLoader); + } + + private static Executor lookupExecutor( + Map executorProperties, + StreamExecutionEnvironment executionEnvironment) { + try { + ExecutorFactory executorFactory = ComponentFactoryService.find(ExecutorFactory.class, executorProperties); + Method createMethod = executorFactory.getClass() + .getMethod("create", Map.class, StreamExecutionEnvironment.class); + + return (Executor) createMethod.invoke( + executorFactory, + executorProperties, + executionEnvironment); + } catch (Exception e) { + throw new TableException( + "Could not instantiate the executor. Make sure a planner module is on the classpath", + e); + } + } + + private void initializeTableEnvironment(@Nullable SessionState sessionState) throws SqlExecutionException { + final EnvironmentSettings settings = environment.getExecution().getEnvironmentSettings(); + // Step 0.0 Initialize the table configuration. + final TableConfig config = new TableConfig(); + environment.getConfiguration().asMap().forEach((k, v) -> + config.getConfiguration().setString(k, v)); + final boolean noInheritedState = sessionState == null; + if (noInheritedState) { + //-------------------------------------------------------------------------------------------------------------- + // Step.1 Create environments + //-------------------------------------------------------------------------------------------------------------- + // Step 1.0 Initialize the ModuleManager if required. + final ModuleManager moduleManager = new ModuleManager(); + // Step 1.1 Initialize the CatalogManager if required. + final CatalogManager catalogManager = CatalogManager.newBuilder() + .classLoader(classLoader) + .config(config.getConfiguration()) + .defaultCatalog( + settings.getBuiltInCatalogName(), + new GenericInMemoryCatalog( + settings.getBuiltInCatalogName(), + settings.getBuiltInDatabaseName())) + .build(); + // Step 1.2 Initialize the FunctionCatalog if required. + final FunctionCatalog functionCatalog = new FunctionCatalog(config, catalogManager, moduleManager); + // Step 1.4 Set up session state. + this.sessionState = SessionState.of(catalogManager, moduleManager, functionCatalog); + + // Must initialize the table environment before actually the + createTableEnvironment(settings, config, catalogManager, moduleManager, functionCatalog); + + //-------------------------------------------------------------------------------------------------------------- + // Step.2 Create modules and load them into the TableEnvironment. + //-------------------------------------------------------------------------------------------------------------- + // No need to register the modules info if already inherit from the same session. + Map modules = new LinkedHashMap<>(); + environment.getModules().forEach((name, entry) -> + modules.put(name, createModule(entry.asMap(), classLoader)) + ); + if (!modules.isEmpty()) { + // unload core module first to respect whatever users configure + tableEnv.unloadModule(CoreModuleDescriptorValidator.MODULE_TYPE_CORE); + modules.forEach(tableEnv::loadModule); + } + + //-------------------------------------------------------------------------------------------------------------- + // Step.3 create user-defined functions and temporal tables then register them. + //-------------------------------------------------------------------------------------------------------------- + // No need to register the functions if already inherit from the same session. + registerFunctions(); + + //-------------------------------------------------------------------------------------------------------------- + // Step.4 Create catalogs and register them. + //-------------------------------------------------------------------------------------------------------------- + // No need to register the catalogs if already inherit from the same session. + initializeCatalogs(); + } else { + // Set up session state. + this.sessionState = sessionState; + createTableEnvironment( + settings, + config, + sessionState.catalogManager, + sessionState.moduleManager, + sessionState.functionCatalog); + } + } + + private void createTableEnvironment( + EnvironmentSettings settings, + TableConfig config, + CatalogManager catalogManager, + ModuleManager moduleManager, + FunctionCatalog functionCatalog) { + + //流式 + if (environment.getExecution().isStreamingPlanner()) { + streamExecEnv = createStreamExecutionEnvironment(); + execEnv = null; + final Map executorProperties = settings.toExecutorProperties(); + executor = lookupExecutor(executorProperties, streamExecEnv); + tableEnv = createStreamTableEnvironment( + streamExecEnv, + settings, + config, + executor, + catalogManager, + moduleManager, + functionCatalog); + return ; + } + //默认批 + streamExecEnv = null; + execEnv = createExecutionEnvironment(); + executor = null; + tableEnv = new BatchTableEnvironmentImpl( + execEnv, + config, + catalogManager, + moduleManager); + + } + + private void initializeCatalogs() throws SqlExecutionException { + //-------------------------------------------------------------------------------------------------------------- + // Step.1 Create catalogs and register them. + //-------------------------------------------------------------------------------------------------------------- + wrapClassLoader(() -> environment.getCatalogs().forEach((name, entry) -> { + Catalog catalog = createCatalog(name, entry.asMap(), classLoader); + tableEnv.registerCatalog(name, catalog); + }) + ); + + //-------------------------------------------------------------------------------------------------------------- + // Step.2 create table sources & sinks, and register them. + //-------------------------------------------------------------------------------------------------------------- + Map> tableSources = new HashMap<>(); + Map> tableSinks = new HashMap<>(); + for(Entry keyValue: environment.getTables().entrySet()) { + String name = keyValue.getKey(); + TableEntry entry = keyValue.getValue(); + if (entry instanceof SourceTableEntry || entry instanceof SourceSinkTableEntry) { + tableSources.put(name, createTableSource(environment.getExecution(), entry.asMap(), classLoader)); + } + if (entry instanceof SinkTableEntry || entry instanceof SourceSinkTableEntry) { + tableSinks.put(name, createTableSink(environment.getExecution(), entry.asMap(), classLoader)); + } + } + // register table sources + tableSources.forEach(tableEnv::registerTableSource); + // register table sinks + tableSinks.forEach(tableEnv::registerTableSink); + + //-------------------------------------------------------------------------------------------------------------- + // Step.4 Register temporal tables. + //-------------------------------------------------------------------------------------------------------------- + for(Entry keyValue : environment.getTables().entrySet()) { + TableEntry entry = keyValue.getValue(); + if (entry instanceof TemporalTableEntry) { + final TemporalTableEntry temporalTableEntry = (TemporalTableEntry) entry; + registerTemporalTable(temporalTableEntry); + } + } + + //-------------------------------------------------------------------------------------------------------------- + // Step.5 Register views in specified order. + //-------------------------------------------------------------------------------------------------------------- + for(Entry keyValue : environment.getTables().entrySet()) { + // if registering a view fails at this point, + // it means that it accesses tables that are not available anymore + TableEntry entry = keyValue.getValue(); + if (entry instanceof ViewEntry) { + final ViewEntry viewEntry = (ViewEntry) entry; + registerView(viewEntry); + } + } + + //-------------------------------------------------------------------------------------------------------------- + // Step.6 Set current catalog and database. + //-------------------------------------------------------------------------------------------------------------- + // Switch to the current catalog. + Optional catalog = environment.getExecution().getCurrentCatalog(); + catalog.ifPresent(tableEnv::useCatalog); + + // Switch to the current database. + Optional database = environment.getExecution().getCurrentDatabase(); + database.ifPresent(tableEnv::useDatabase); + } + + private ExecutionEnvironment createExecutionEnvironment() { + final ExecutionEnvironment execEnv = ExecutionEnvironment.getExecutionEnvironment(); + execEnv.setRestartStrategy(environment.getExecution().getRestartStrategy()); + execEnv.setParallelism(environment.getExecution().getParallelism()); + return execEnv; + } + + private StreamExecutionEnvironment createStreamExecutionEnvironment() { + final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRestartStrategy(environment.getExecution().getRestartStrategy()); + env.setParallelism(environment.getExecution().getParallelism()); + env.setMaxParallelism(environment.getExecution().getMaxParallelism()); + env.setStreamTimeCharacteristic(environment.getExecution().getTimeCharacteristic()); + if (env.getStreamTimeCharacteristic() == TimeCharacteristic.EventTime) { + env.getConfig().setAutoWatermarkInterval(environment.getExecution().getPeriodicWatermarksInterval()); + } + return env; + } + + private void registerFunctions() throws SqlExecutionException { + Map functions = new LinkedHashMap<>(); + environment.getFunctions().forEach((name, entry) -> { + final UserDefinedFunction function = FunctionService + .createFunction(entry.getDescriptor(), classLoader, false); + functions.put(name, function); + }); + registerFunctions(functions); + } + + private void registerFunctions(Map functions) throws SqlExecutionException { + if (tableEnv instanceof StreamTableEnvironment) { + StreamTableEnvironment streamTableEnvironment = (StreamTableEnvironment) tableEnv; + for(Entry keyValue: functions.entrySet()) { + String k = keyValue.getKey(); + FunctionDefinition v = keyValue.getValue(); + if (v instanceof ScalarFunction) { + streamTableEnvironment.registerFunction(k, (ScalarFunction) v); + } else if (v instanceof AggregateFunction) { + streamTableEnvironment.registerFunction(k, (AggregateFunction) v); + } else if (v instanceof TableFunction) { + streamTableEnvironment.registerFunction(k, (TableFunction) v); + } else { + throw new SqlExecutionException("Unsupported function type: " + v.getClass().getName()); + } + } + } else { + BatchTableEnvironment batchTableEnvironment = (BatchTableEnvironment) tableEnv; + for(Entry keyValue: functions.entrySet()) { + String k = keyValue.getKey(); + FunctionDefinition v = keyValue.getValue(); + if (v instanceof ScalarFunction) { + batchTableEnvironment.registerFunction(k, (ScalarFunction) v); + } else if (v instanceof AggregateFunction) { + batchTableEnvironment.registerFunction(k, (AggregateFunction) v); + } else if (v instanceof TableFunction) { + batchTableEnvironment.registerFunction(k, (TableFunction) v); + } else { + throw new SqlExecutionException("Unsupported function type: " + v.getClass().getName()); + } + } + } + } + + private void registerView(ViewEntry viewEntry) throws SqlExecutionException { + try { + tableEnv.registerTable(viewEntry.getName(), tableEnv.sqlQuery(viewEntry.getQuery())); + } catch (Exception e) { + throw new SqlExecutionException( + "Invalid view '" + viewEntry.getName() + "' with query:\n" + viewEntry.getQuery() + + "\nCause: " + e.getMessage()); + } + } + + private void registerTemporalTable(TemporalTableEntry temporalTableEntry) throws SqlExecutionException { + try { + final Table table = tableEnv.scan(temporalTableEntry.getHistoryTable()); + final TableFunction function = table.createTemporalTableFunction( + temporalTableEntry.getTimeAttribute(), + String.join(",", temporalTableEntry.getPrimaryKeyFields())); + if (tableEnv instanceof StreamTableEnvironment) { + StreamTableEnvironment streamTableEnvironment = (StreamTableEnvironment) tableEnv; + streamTableEnvironment.registerFunction(temporalTableEntry.getName(), function); + } else { + BatchTableEnvironment batchTableEnvironment = (BatchTableEnvironment) tableEnv; + batchTableEnvironment.registerFunction(temporalTableEntry.getName(), function); + } + } catch (Exception e) { + throw new SqlExecutionException( + "Invalid temporal table '" + temporalTableEntry.getName() + "' over table '" + + temporalTableEntry.getHistoryTable() + ".\nCause: " + e.getMessage()); + } + } + + //~ Inner Class ------------------------------------------------------------------------------- + + /** Builder for {@link ExecutionContext}. */ + public static class Builder { + // Required members. + private final Environment sessionEnv; + private final List dependencies; + private final Configuration configuration; + private final ClusterClientServiceLoader serviceLoader; + + private Environment defaultEnv; + private Environment currentEnv; + + // Optional members. + @Nullable + private SessionState sessionState; + + private Builder( + Environment defaultEnv, + @Nullable Environment sessionEnv, + List dependencies, + Configuration configuration, + ClusterClientServiceLoader serviceLoader) { + this.defaultEnv = defaultEnv; + this.sessionEnv = sessionEnv; + this.dependencies = dependencies; + this.configuration = configuration; + this.serviceLoader = serviceLoader; + } + + public Builder env(Environment environment) { + this.currentEnv = environment; + return this; + } + + public Builder sessionState(SessionState sessionState) { + this.sessionState = sessionState; + return this; + } + + public ExecutionContext build() throws SqlExecutionException { + if(sessionEnv == null){ + this.currentEnv = defaultEnv; + } + try { + return new ExecutionContext( + this.currentEnv == null ? Environment.merge(defaultEnv, sessionEnv) : this.currentEnv, + this.sessionState, + this.dependencies, + this.configuration, + this.serviceLoader); + } catch (Throwable t) { + // catch everything such that a configuration does not crash the executor + throw new SqlExecutionException("Could not create execution context.", t); + } + } + } + + /** Represents the state that should be reused in one session. **/ + public static class SessionState { + public final CatalogManager catalogManager; + public final ModuleManager moduleManager; + public final FunctionCatalog functionCatalog; + + private SessionState( + CatalogManager catalogManager, + ModuleManager moduleManager, + FunctionCatalog functionCatalog) { + this.catalogManager = catalogManager; + this.moduleManager = moduleManager; + this.functionCatalog = functionCatalog; + } + + public static SessionState of( + CatalogManager catalogManager, + ModuleManager moduleManager, + FunctionCatalog functionCatalog) { + return new SessionState(catalogManager, moduleManager, functionCatalog); + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapter.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapter.java new file mode 100644 index 0000000000..ba13a0103b --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapter.java @@ -0,0 +1,166 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import java.io.Closeable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.client.deployment.ClusterRetrieveException; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @program: linkis + * @description: 集群交互适配器,适合datastream、sql方式作业 + */ +public abstract class ClusterDescriptorAdapter implements Closeable { + public static final Logger LOG = LoggerFactory.getLogger(ClusterDescriptorAdapter.class); + + public static final long CLIENT_REQUEST_TIMEOUT = FlinkEnvConfiguration.FLINK_CLIENT_REQUEST_TIMEOUT().getValue().toLong(); + + protected final ExecutionContext executionContext; + // jobId is not null only after job is submitted + protected JobID jobId; + protected ApplicationId clusterID; + protected ClusterClient clusterClient; + private YarnClusterDescriptor clusterDescriptor; + + public JobID getJobId() { + return jobId; + } + + public ApplicationId getClusterID() { + return clusterID; + } + + public String getWebInterfaceUrl() { + return webInterfaceUrl; + } + + protected String webInterfaceUrl; + + public ClusterDescriptorAdapter( + ExecutionContext executionContext, + JobID jobId) { + this.executionContext = executionContext; + this.jobId = jobId; + this.clusterID = executionContext.getClusterClientFactory().getClusterId(executionContext.getFlinkConfig()); + } + + /** + * Returns the status of the flink job. + */ + public JobStatus getJobStatus() throws JobExecutionException { + if (jobId == null) { + throw new JobExecutionException("No job has been submitted. This is a bug."); + } + return bridgeClientRequest(this.executionContext, jobId, () -> clusterClient.getJobStatus(jobId), false); + } + + /** + * Cancel the flink job. + */ + public void cancelJob() throws JobExecutionException { + if (jobId == null) { + throw new JobExecutionException("No job has been submitted. This is a bug."); + } + LOG.info("Start to cancel job {}.", jobId); + bridgeClientRequest(this.executionContext, jobId, () -> clusterClient.cancel(jobId), true); + } + + public abstract void deployCluster(String[] programArguments, String applicationClassName) throws JobExecutionException; + + /** + * The reason of using ClusterClient instead of JobClient to retrieve a cluster is + * the JobClient can't know whether the job is finished on yarn-per-job mode. + * + *

If a job is finished, JobClient always get java.util.concurrent.TimeoutException + * when getting job status and canceling a job after job is finished. + * This method will throw org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException + * when creating a ClusterClient if the job is finished. This is more user-friendly. + */ + protected R bridgeClientRequest( + ExecutionContext executionContext, + JobID jobId, + Supplier> function, boolean ignoreError) throws JobExecutionException { + if(clusterClient == null) { + if (this.clusterID == null) { + LOG.error("Cluster information don't exist."); + throw new JobExecutionException("Cluster information don't exist."); + } + clusterDescriptor = executionContext.createClusterDescriptor(); + try { + clusterClient = clusterDescriptor.retrieve(this.clusterID).getClusterClient(); + } catch (ClusterRetrieveException e) { + LOG.error(String.format("Job: %s could not retrieve or create a cluster.", jobId), e); + throw new JobExecutionException(String.format("Job: %s could not retrieve or create a cluster.", jobId), e); + } + } + try { + return function.get().get(CLIENT_REQUEST_TIMEOUT, TimeUnit.MILLISECONDS); + } catch (Exception e) { + if(ignoreError) { + return null; + } else { + LOG.error(String.format("Job: %s operation failed!", jobId), e); + throw new JobExecutionException(String.format("Job: %s operation failed!", jobId), e); + } + } + } + + @Override + public String toString() { + return "ClusterDescriptorAdapter{" + + "jobId=" + jobId + + ", clusterID=" + clusterID + + '}'; + } + + @Override + public void close() { + if(clusterClient != null) { + clusterClient.shutDownCluster(); + clusterClient.close(); + } + if(clusterDescriptor != null) { + clusterDescriptor.close(); + } + } + + /** + * Checks whether this job state is globally terminal. + * A globally terminal job is complete and cannot fail any more + * and will not be restarted or recovered by another standby master node. + * + *

When a globally terminal state has been reached, + * all recovery data for the job is dropped from the high-availability services. + * + * @return True, if this job status is globally terminal, false otherwise. + */ + public abstract boolean isGloballyTerminalState() throws JobExecutionException; +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapterFactory.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapterFactory.java new file mode 100644 index 0000000000..28f92674c9 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapterFactory.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import org.apache.flink.api.common.JobID; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.yarn.configuration.YarnDeploymentTarget; + +/** + * 集群交互适配器工厂 + */ +public class ClusterDescriptorAdapterFactory { + + public static ClusterDescriptorAdapter create( + ExecutionContext executionContext, + JobID jobId) { + String yarnDeploymentTarget = executionContext.getFlinkConfig().get(DeploymentOptions.TARGET); + ClusterDescriptorAdapter clusterDescriptorAdapter = null; + if (YarnDeploymentTarget.PER_JOB.getName().equals(yarnDeploymentTarget)) { + clusterDescriptorAdapter = new YarnPerJobClusterDescriptorAdapter( + executionContext, + jobId); + } else if (YarnDeploymentTarget.APPLICATION.getName().equals(yarnDeploymentTarget)) { + clusterDescriptorAdapter = new YarnApplicationClusterDescriptorAdapter( + executionContext, + jobId); + } else if(YarnDeploymentTarget.SESSION.getName().equals(yarnDeploymentTarget)) { + // Just the same with per_job. + clusterDescriptorAdapter = new YarnPerJobClusterDescriptorAdapter( + executionContext, + jobId); + } + return clusterDescriptorAdapter; + } + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ProgramDeployer.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ProgramDeployer.java new file mode 100644 index 0000000000..e4812e661c --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ProgramDeployer.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import java.util.concurrent.CompletableFuture; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.client.deployment.executors.AbstractJobClusterExecutor; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.execution.PipelineExecutor; +import org.apache.flink.yarn.YarnClusterClientFactory; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @program: linkis + * @description: 集群部署 Table program + */ +public class ProgramDeployer { + private static final Logger LOG = LoggerFactory.getLogger(ProgramDeployer.class); + + private final Configuration configuration; + private final Pipeline pipeline; + private final String jobName; + + /** + * Deploys a table program on the cluster. + * + * @param configuration the {@link Configuration} that is used for deployment + * @param jobName job name of the Flink job to be submitted + * @param pipeline Flink {@link Pipeline} to execute + */ + public ProgramDeployer( + Configuration configuration, + String jobName, + Pipeline pipeline) { + this.configuration = configuration; + this.pipeline = pipeline; + this.jobName = jobName; + } + + public CompletableFuture deploy(ExecutionContext context) throws Exception { + LOG.info("Submitting job {} for query {}`", pipeline, jobName); + if (LOG.isDebugEnabled()) { + LOG.debug("Submitting job {} with configuration: \n{}", pipeline, configuration); + } + if (configuration.get(DeploymentOptions.TARGET) == null) { + throw new JobExecutionException("No execution. Target specified in your configuration file."); + } + final PipelineExecutor executor = new AbstractJobClusterExecutor(context.getClusterClientFactory()); + return executor.execute(pipeline, configuration); + } +} + diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnApplicationClusterDescriptorAdapter.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnApplicationClusterDescriptorAdapter.java new file mode 100644 index 0000000000..01ce1074ea --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnApplicationClusterDescriptorAdapter.java @@ -0,0 +1,59 @@ +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.ExecutorInitException; +import java.util.Objects; +import java.util.concurrent.TimeUnit; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import org.apache.flink.api.common.JobID; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.deployment.application.ApplicationConfiguration; +import org.apache.flink.client.program.ClusterClientProvider; +import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.yarn.api.records.ApplicationId; + +/** + * @program: flink-parent + * @description: + */ +public class YarnApplicationClusterDescriptorAdapter extends ClusterDescriptorAdapter { + + public YarnApplicationClusterDescriptorAdapter(ExecutionContext executionContext, JobID jobId) { + super(executionContext, jobId); + } + + @Override + public void deployCluster(String[] programArguments, String applicationClassName) throws JobExecutionException { + ApplicationConfiguration applicationConfiguration = new ApplicationConfiguration(programArguments,applicationClassName); + ClusterSpecification clusterSpecification = this.executionContext.getClusterClientFactory().getClusterSpecification(this.executionContext.getFlinkConfig()); + YarnClusterDescriptor clusterDescriptor = this.executionContext.createClusterDescriptor(); + try { + ClusterClientProvider clusterClientProvider = clusterDescriptor.deployApplicationCluster( + clusterSpecification, + applicationConfiguration); + clusterClient = clusterClientProvider.getClusterClient(); + super.clusterID = clusterClient.getClusterId(); + super.webInterfaceUrl = clusterClient.getWebInterfaceURL(); + } catch (Exception e) { + throw new JobExecutionException(e.getMessage()); + } + + } + + public boolean initJobId() throws Exception { + clusterClient.listJobs().thenAccept(list -> list.forEach(jobStatusMessage -> { + if (Objects.nonNull(jobStatusMessage.getJobId())) { + jobId = jobStatusMessage.getJobId(); + } + })).get(CLIENT_REQUEST_TIMEOUT, TimeUnit.MILLISECONDS); + return null != jobId; + } + + + @Override + public boolean isGloballyTerminalState() { + return false; + } + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnPerJobClusterDescriptorAdapter.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnPerJobClusterDescriptorAdapter.java new file mode 100644 index 0000000000..f402877eae --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnPerJobClusterDescriptorAdapter.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import java.util.concurrent.CompletableFuture; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @program: flink-parent + * @description: + */ +public class YarnPerJobClusterDescriptorAdapter extends ClusterDescriptorAdapter { + private static final Logger LOG = LoggerFactory.getLogger(YarnPerJobClusterDescriptorAdapter.class); + + public YarnPerJobClusterDescriptorAdapter( + ExecutionContext executionContext, + JobID jobId) { + super(executionContext, jobId); + } + + @Override + public boolean isGloballyTerminalState() throws JobExecutionException { + boolean isGloballyTerminalState; + try { + JobStatus jobStatus = getJobStatus(); + isGloballyTerminalState = jobStatus.isGloballyTerminalState(); + } catch (JobExecutionException e) { + if (isYarnApplicationStopped(e)) { + isGloballyTerminalState = true; + } else { + throw e; + } + } + + return isGloballyTerminalState; + } + + /** + * The yarn application is not running when its final status is not UNDEFINED. + * + *

In this case, it will throw + * RuntimeException("The Yarn application " + applicationId + " doesn't run anymore.") + * from retrieve method in YarnClusterDescriptor.java + */ + private boolean isYarnApplicationStopped(Throwable e) { + do { + String exceptionMessage = e.getMessage(); + if (StringUtils.equals(exceptionMessage, "The Yarn application " + clusterID + " doesn't run anymore.")) { + LOG.info("{} is stopped.", clusterID); + return true; + } + e = e.getCause(); + } while (e != null); + return false; + } + + @Override + public void deployCluster(String[] programArguments, String applicationClassName) throws JobExecutionException { + webInterfaceUrl = this.bridgeClientRequest(executionContext, jobId, () -> CompletableFuture.completedFuture(clusterClient.getWebInterfaceURL()), false); + } + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/factory/LinkisYarnClusterClientFactory.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/factory/LinkisYarnClusterClientFactory.java new file mode 100644 index 0000000000..1c88ad80df --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/factory/LinkisYarnClusterClientFactory.java @@ -0,0 +1,70 @@ +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.factory; + +import static org.apache.flink.configuration.ConfigOptions.key; +import static org.apache.flink.util.Preconditions.checkNotNull; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils.YarnConfLoader; +import java.io.Closeable; +import java.io.IOException; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.DeploymentOptionsInternal; +import org.apache.flink.yarn.YarnClientYarnClusterInformationRetriever; +import org.apache.flink.yarn.YarnClusterClientFactory; +import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.flink.yarn.configuration.YarnLogConfigUtil; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @program: linkis + */ +public class LinkisYarnClusterClientFactory extends YarnClusterClientFactory implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger(LinkisYarnClusterClientFactory.class); + + public static final ConfigOption YARN_CONFIG_DIR = + key("$internal.yarn.config-dir") + .stringType() + .noDefaultValue() + .withDescription("**DO NOT USE** The location of the log config file, e.g. the path to your log4j.properties for log4j."); + + private YarnConfiguration yarnConfiguration; + private YarnClient yarnClient; + + private void initYarnClient(Configuration configuration) { + String yarnConfDir = configuration.getString(YARN_CONFIG_DIR); + yarnConfiguration = YarnConfLoader.getYarnConf(yarnConfDir); + yarnClient = YarnClient.createYarnClient(); + yarnClient.init(yarnConfiguration); + yarnClient.start(); + } + + @Override + public YarnClusterDescriptor createClusterDescriptor(Configuration configuration) { + checkNotNull(configuration); + final String configurationDirectory = + configuration.get(DeploymentOptionsInternal.CONF_DIR); + YarnLogConfigUtil.setLogConfigFileInConfig(configuration, configurationDirectory); + if(yarnClient == null) { + synchronized (this) { + if(yarnClient == null) { + initYarnClient(configuration); + } + } + } + return new YarnClusterDescriptor( + configuration, + yarnConfiguration, + yarnClient, + YarnClientYarnClusterInformationRetriever.create(yarnClient), + true); + } + + @Override + public void close() throws IOException { + yarnClient.close(); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/AbstractResult.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/AbstractResult.java new file mode 100644 index 0000000000..98c11e99e4 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/AbstractResult.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; + + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.FlinkListenerGroupImpl; + +public abstract class AbstractResult extends FlinkListenerGroupImpl implements Result { + + protected C clusterId; + protected String webInterfaceUrl; + + @Override + public void setClusterInformation(C clusterId, String webInterfaceUrl) { + if (this.clusterId != null || this.webInterfaceUrl != null) { + throw new IllegalStateException("Cluster information is already present."); + } + this.clusterId = clusterId; + this.webInterfaceUrl = webInterfaceUrl; + } + + public C getClusterId() { + if (this.clusterId == null) { + throw new IllegalStateException("Cluster ID has not been set."); + } + return clusterId; + } + + public String getWebInterfaceUrl() { + if (this.webInterfaceUrl == null) { + throw new IllegalStateException("Cluster web interface URL has not been set."); + } + return webInterfaceUrl; + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/BatchResult.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/BatchResult.java new file mode 100644 index 0000000000..69af532024 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/BatchResult.java @@ -0,0 +1,141 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.RowsType; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.accumulators.SerializedListAccumulator; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.client.gateway.local.CollectBatchTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; +import org.apache.flink.util.AbstractID; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class BatchResult extends AbstractResult { + + private Logger LOG = LoggerFactory.getLogger(getClass()); + + private final String accumulatorName; + private final CollectBatchTableSink tableSink; + private final Object resultLock; + private final ClassLoader classLoader; + + private AtomicReference executionException = new AtomicReference<>(); + private List resultTable; + + private boolean allResultRetrieved = false; + + public BatchResult( + TableSchema tableSchema, + RowTypeInfo outputType, + ExecutionConfig config, + ClassLoader classLoader) { + // TODO supports large result set + accumulatorName = new AbstractID().toString(); + tableSink = new CollectBatchTableSink(accumulatorName, outputType.createSerializer(config), tableSchema); + resultLock = new Object(); + this.classLoader = checkNotNull(classLoader); + } + + @Override + public void startRetrieval(JobClient jobClient) { + CompletableFuture.completedFuture(jobClient) + .thenCompose(client -> client.getJobExecutionResult(classLoader)) + .thenAccept(new ResultRetrievalHandler()) + .whenComplete((unused, throwable) -> { + if (throwable != null) { + executionException.compareAndSet( + null, + new SqlExecutionException("Error while submitting job.", throwable)); + } + }); + } + + @Override + public TypedResult> retrieveChanges() throws SqlExecutionException { + synchronized (resultLock) { + // the job finished with an exception + SqlExecutionException e = executionException.get(); + if (e != null) { + throw e; + } + // wait for a result + if (null == resultTable) { + return TypedResult.empty(); + } + if (allResultRetrieved) { + return TypedResult.endOfStream(); + } else { + allResultRetrieved = true; + return TypedResult.payload(resultTable); + } + } + } + + @Override + public TableSink getTableSink() { + return tableSink; + } + + @Override + public void close() { + } + + // -------------------------------------------------------------------------------------------- + + private class ResultRetrievalHandler implements Consumer { + + @Override + public void accept(JobExecutionResult jobExecutionResult) { + try { + final ArrayList accResult = jobExecutionResult.getAccumulatorResult(accumulatorName); + if (accResult == null) { + throw new JobExecutionException("The accumulator could not retrieve the result."); + } + final List resultTable = SerializedListAccumulator + .deserializeList(accResult, tableSink.getSerializer()); + // sets the result table all at once + synchronized (resultLock) { + BatchResult.this.resultTable = resultTable; + } + LOG.info("Accept the result, row is "+ resultTable.size()); + getFlinkStatusListeners().forEach(listener -> listener.onSuccess(resultTable.size(), RowsType.Fetched())); + } catch (ClassNotFoundException | IOException | JobExecutionException e) { + getFlinkStatusListeners().forEach(listener -> listener.onFailed("Serialization error while deserialize collected data.", e)); + throw new RuntimeException("Serialization error while deserialize collected data.", e); + } + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ChangelogResult.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ChangelogResult.java new file mode 100644 index 0000000000..4456c6f606 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ChangelogResult.java @@ -0,0 +1,221 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.RowsType; +import java.io.IOException; +import java.net.InetAddress; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobExecutionResult; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.experimental.SocketStreamIterator; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.client.gateway.local.CollectStreamTableSink; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class ChangelogResult extends AbstractResult> { + + private Logger LOG = LoggerFactory.getLogger(getClass()); + + private final SocketStreamIterator> iterator; + private final CollectStreamTableSink collectTableSink; + private final ResultRetrievalThread retrievalThread; + private final ClassLoader classLoader; + private CompletableFuture jobExecutionResultFuture; + + private final Object resultLock; + private AtomicReference executionException = new AtomicReference<>(); + private final List> changeRecordBuffer; + private final int maxBufferSize; + + public ChangelogResult( + RowTypeInfo outputType, + TableSchema tableSchema, + ExecutionConfig config, + InetAddress gatewayAddress, + int gatewayPort, + ClassLoader classLoader, + int maxBufferSize) throws SqlExecutionException { + resultLock = new Object(); + + // create socket stream iterator + final TypeInformation> socketType = Types.TUPLE(Types.BOOLEAN, outputType); + final TypeSerializer> serializer = socketType.createSerializer(config); + try { + // pass gateway port and address such that iterator knows where to bind to + iterator = new SocketStreamIterator<>(gatewayPort, gatewayAddress, serializer); + } catch (IOException e) { + throw new SqlExecutionException("Could not start socket for result retrieval.", e); + } + + // create table sink + // pass binding address and port such that sink knows where to send to + collectTableSink = new CollectStreamTableSink( + iterator.getBindAddress(), iterator.getPort(), serializer, tableSchema); + retrievalThread = new ResultRetrievalThread(); + + this.classLoader = checkNotNull(classLoader); + + // prepare for changelog + changeRecordBuffer = new ArrayList<>(); + this.maxBufferSize = maxBufferSize; + } + + @Override + public void startRetrieval(JobClient jobClient) { + // start listener thread + retrievalThread.setName(jobClient.getJobID().toHexString() + "-JobResult-Fetch-Thread"); + retrievalThread.start(); + + jobExecutionResultFuture = CompletableFuture.completedFuture(jobClient) + .thenCompose(client -> client.getJobExecutionResult(classLoader)) + .whenComplete((unused, throwable) -> { + if (throwable != null) { + executionException.compareAndSet( + null, + new JobExecutionException("Error while submitting job.", throwable)); + } + }); + } + + @Override + public TypedResult>> retrieveChanges() throws JobExecutionException { + synchronized (resultLock) { + // retrieval thread is alive return a record if available + // but the program must not have failed + if (isRetrieving() && executionException.get() == null) { + if (changeRecordBuffer.isEmpty()) { + return TypedResult.empty(); + } else { + final List> change = new ArrayList<>(changeRecordBuffer); + changeRecordBuffer.clear(); + resultLock.notify(); + return TypedResult.payload(change); + } + } + // retrieval thread is dead but there is still a record to be delivered + else if (!isRetrieving() && !changeRecordBuffer.isEmpty()) { + final List> change = new ArrayList<>(changeRecordBuffer); + changeRecordBuffer.clear(); + return TypedResult.payload(change); + } + // no results can be returned anymore + else { + return handleMissingResult(); + } + } + } + + @Override + public TableSink getTableSink() { + return collectTableSink; + } + + @Override + public void close() { + retrievalThread.isRunning = false; + retrievalThread.interrupt(); + iterator.close(); + } + + // -------------------------------------------------------------------------------------------- + + private TypedResult handleMissingResult() throws JobExecutionException { + + // check if the monitoring thread is still there + // we need to wait until we know what is going on + if (!jobExecutionResultFuture.isDone()) { + return TypedResult.empty(); + } + + if (executionException.get() != null) { + throw executionException.get(); + } + + // we assume that a bounded job finished + return TypedResult.endOfStream(); + } + + private boolean isRetrieving() { + return retrievalThread.isRunning; + } + + private void processRecord(Tuple2 change) { + synchronized (resultLock) { + // wait if the buffer is full + while (changeRecordBuffer.size() >= maxBufferSize) { + try { + getFlinkStreamingResultSetListeners().forEach(listener -> listener.onResultSetPulled(changeRecordBuffer.size())); + resultLock.wait(); + } catch (InterruptedException e) { + // ignore + } + } + changeRecordBuffer.add(change); + } + LOG.debug("Accept the streaming result, row is "+ change.f1.toString()); + } + + // -------------------------------------------------------------------------------------------- + + private class ResultRetrievalThread extends Thread { + + public volatile boolean isRunning = true; + + @Override + public void run() { + int rows = 0; + try { + while (isRunning && iterator.hasNext()) { + final Tuple2 change = iterator.next(); + processRecord(change); + rows ++; + } + } catch (Exception e) { + // ignore socket exceptions + LOG.warn(getName() + " has finished with an error.", e); + } + int totalRows = rows; + if(!changeRecordBuffer.isEmpty()) { + getFlinkStreamingResultSetListeners().forEach(listener -> listener.onResultSetPulled(changeRecordBuffer.size())); + } + getFlinkStatusListeners().forEach(listener -> listener.onSuccess(totalRows, RowsType.Fetched())); + // no result anymore + // either the job is done or an error occurred + isRunning = false; + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/Result.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/Result.java new file mode 100644 index 0000000000..cc05cfe593 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/Result.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import java.util.List; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.table.sinks.TableSink; + + +public interface Result { + + /** + * Sets the cluster information of the cluster this result comes from. This method should only be called once. + */ + void setClusterInformation(C clusterId, String webInterfaceUrl); + + /** + * Starts the table program using the given deployer and monitors it's execution. + */ + void startRetrieval(JobClient jobClient); + + /** + * Retrieves the available result records. + */ + TypedResult> retrieveChanges() throws JobExecutionException, SqlExecutionException; + + /** + * Returns the table sink required by this result type. + */ + TableSink getTableSink(); + + /** + * Closes the retrieval and all involved threads. + */ + void close(); + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ResultUtil.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ResultUtil.java new file mode 100644 index 0000000000..10db00ca13 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ResultUtil.java @@ -0,0 +1,122 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.util.stream.Stream; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.runtime.net.ConnectionUtils; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.client.config.entries.DeploymentEntry; +import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; + + +public class ResultUtil { + + public static BatchResult createBatchResult( + TableSchema schema, + ExecutionConfig config, + ClassLoader classLoader) { + final TypeInformation[] schemaTypeInfos = Stream.of(schema.getFieldDataTypes()) + .map(TypeInfoDataTypeConverter::fromDataTypeToTypeInfo) + .toArray(TypeInformation[]::new); + final RowTypeInfo outputType = new RowTypeInfo(schemaTypeInfos, schema.getFieldNames()); + + return new BatchResult<>(schema, outputType, config, classLoader); + } + + public static ChangelogResult createChangelogResult( + Configuration flinkConfig, + Environment env, + TableSchema schema, + ExecutionConfig config, + ClassLoader classLoader) throws JobExecutionException, SqlExecutionException { + final TypeInformation[] schemaTypeInfos = Stream.of(schema.getFieldDataTypes()) + .map(TypeInfoDataTypeConverter::fromDataTypeToTypeInfo) + .toArray(TypeInformation[]::new); + final RowTypeInfo outputType = new RowTypeInfo(schemaTypeInfos, schema.getFieldNames()); + + // determine gateway address (and port if possible) + final InetAddress gatewayAddress = getGatewayAddress(env.getDeployment(), flinkConfig); + final int gatewayPort = getGatewayPort(env.getDeployment()); + final int maxBufferSize = env.getExecution().getMaxTableResultRows(); + + return new ChangelogResult<>( + outputType, + schema, + config, + gatewayAddress, + gatewayPort, + classLoader, + maxBufferSize); + } + + // -------------------------------------------------------------------------------------------- + + private static int getGatewayPort(DeploymentEntry deploy) { + // try to get address from deployment configuration + return deploy.getGatewayPort(); + } + + private static InetAddress getGatewayAddress(DeploymentEntry deploy, Configuration flinkConfig) throws JobExecutionException { + // try to get address from deployment configuration + final String address = deploy.getGatewayAddress(); + + // use manually defined address + if (!address.isEmpty()) { + try { + return InetAddress.getByName(address); + } catch (UnknownHostException e) { + throw new JobExecutionException("Invalid gateway address '" + address + "' for result retrieval.", e); + } + } else { + // TODO cache this + // try to get the address by communicating to JobManager + final String jobManagerAddress = flinkConfig.getString(JobManagerOptions.ADDRESS); + final int jobManagerPort = flinkConfig.getInteger(JobManagerOptions.PORT); + if (jobManagerAddress != null && !jobManagerAddress.isEmpty()) { + try { + return ConnectionUtils.findConnectingAddress( + new InetSocketAddress(jobManagerAddress, jobManagerPort), + deploy.getResponseTimeout(), + 400); + } catch (Exception e) { + throw new JobExecutionException("Could not determine address of the gateway for result retrieval " + + "by connecting to the job manager. Please specify the gateway address manually.", e); + } + } else { + try { + return InetAddress.getLocalHost(); + } catch (UnknownHostException e) { + throw new JobExecutionException("Could not determine address of the gateway for result retrieval. " + + "Please specify the gateway address manually.", e); + } + } + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/TypedResult.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/TypedResult.java new file mode 100644 index 0000000000..3fec1c6f32 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/TypedResult.java @@ -0,0 +1,89 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; + +import java.util.Objects; + + +public class TypedResult

{ + + private ResultType type; + + private P payload; + + private TypedResult(ResultType type, P payload) { + this.type = type; + this.payload = payload; + } + + public ResultType getType() { + return type; + } + + public P getPayload() { + return payload; + } + + @Override + public String toString() { + return "TypedResult<" + type + ">"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TypedResult that = (TypedResult) o; + return type == that.type && Objects.equals(payload, that.payload); + } + + @Override + public int hashCode() { + return Objects.hash(type, payload); + } + + // -------------------------------------------------------------------------------------------- + + public static TypedResult empty() { + return new TypedResult<>(ResultType.EMPTY, null); + } + + public static TypedResult payload(T payload) { + return new TypedResult<>(ResultType.PAYLOAD, payload); + } + + public static TypedResult endOfStream() { + return new TypedResult<>(ResultType.EOS, null); + } + + // -------------------------------------------------------------------------------------------- + + /** + * Result types. + */ + public enum ResultType { + PAYLOAD, + EMPTY, + EOS + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/AbstractJobOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/AbstractJobOperation.java new file mode 100644 index 0000000000..032d354e5b --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/AbstractJobOperation.java @@ -0,0 +1,170 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment.ClusterDescriptorAdapter; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.FlinkListenerGroupImpl; +import java.util.List; +import java.util.Optional; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.Row; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A default implementation of JobOperation. + */ +public abstract class AbstractJobOperation extends FlinkListenerGroupImpl implements JobOperation { + + private static final Logger LOG = LoggerFactory.getLogger(AbstractJobOperation.class); + + protected final FlinkEngineConnContext context; + // clusterDescriptorAdapter is not null only after job is submitted + protected ClusterDescriptorAdapter clusterDescriptorAdapter; + private volatile JobID jobId; + protected boolean noMoreResult; + + private volatile boolean isJobCanceled; + + protected final Object lock = new Object(); + + public AbstractJobOperation(FlinkEngineConnContext context) { + this.context = context; + this.isJobCanceled = false; + this.noMoreResult = false; + } + + @Override + public ResultSet execute() throws SqlExecutionException, JobExecutionException { + JobInfo jobInfo = submitJob(); + jobId = jobInfo.getJobId(); + String strJobId = jobId.toString(); + return ResultSet.builder() + .resultKind(ResultKind.SUCCESS_WITH_CONTENT) + .columns( + ColumnInfo.create("jobId", new VarCharType(false, strJobId.length())), + ColumnInfo.create("applicationId", new VarCharType(false, jobInfo.getApplicationId().length())), + ColumnInfo.create("webInterfaceUrl", new VarCharType(false, jobInfo.getWebInterfaceUrl().length())) + ) + .data(Row.of(strJobId, jobInfo.getApplicationId(), jobInfo.getWebInterfaceUrl())) + .build(); + } + + public JobInfo transformToJobInfo(ResultSet resultSet) throws JobExecutionException { + if(resultSet.getColumns().size() != 3) { + throw new JobExecutionException("Not support to transform this resultSet to JobInfo."); + } + Row row = resultSet.getData().get(0); + return new JobInfoImpl(JobID.fromHexString(row.getField(0).toString()), + row.getField(1).toString(), + row.getField(2).toString()); + } + + @Override + public JobStatus getJobStatus() throws JobExecutionException { + synchronized (lock) { + return clusterDescriptorAdapter.getJobStatus(); + } + } + + @Override + public void cancelJob() throws JobExecutionException { + if (isJobCanceled) { + // just for fast failure + return; + } + synchronized (lock) { + if (jobId == null) { + LOG.error("No job has been submitted. This is a bug."); + throw new IllegalStateException("No job has been submitted. This is a bug."); + } + if (isJobCanceled) { + return; + } + + cancelJobInternal(); + isJobCanceled = true; + } + } + + protected abstract JobInfo submitJob() throws JobExecutionException, SqlExecutionException; + + protected abstract void cancelJobInternal() throws JobExecutionException; + + @Override + public JobID getJobId() { + if (jobId == null) { + throw new IllegalStateException("No job has been submitted. This is a bug."); + } + return jobId; + } + + @Override + public synchronized Optional getJobResult() throws JobExecutionException, SqlExecutionException { + Optional, List>> newResults = fetchJobResults(); + return newResults.map(r -> ResultSet.builder() + .resultKind(ResultKind.SUCCESS_WITH_CONTENT) + .columns(getColumnInfos()) + .data(r.f0) + .changeFlags(newResults.get().f1) + .build()); + } + + protected abstract Optional, List>> fetchJobResults() throws JobExecutionException, SqlExecutionException; + + protected abstract List getColumnInfos(); + + + protected class JobInfoImpl implements JobInfo { + + private JobID jobId; + private String applicationId, webInterfaceUrl; + + public JobInfoImpl(JobID jobId, String applicationId, String webInterfaceUrl) { + this.jobId = jobId; + this.applicationId = applicationId; + this.webInterfaceUrl = webInterfaceUrl; + } + + @Override + public JobID getJobId() { + return jobId; + } + + @Override + public String getApplicationId() { + return applicationId; + } + + @Override + public String getWebInterfaceUrl() { + return webInterfaceUrl; + } + } + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobInfo.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobInfo.java new file mode 100644 index 0000000000..0561c0519e --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobInfo.java @@ -0,0 +1,14 @@ +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; + +import org.apache.flink.api.common.JobID; + +/** + * Description: + */ +public interface JobInfo { + JobID getJobId(); + + String getApplicationId(); + + String getWebInterfaceUrl(); +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobOperation.java new file mode 100644 index 0000000000..eca9a795f8 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobOperation.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.FlinkListenerGroup; +import java.util.Optional; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; + + +public interface JobOperation extends Operation, FlinkListenerGroup { + + JobID getJobId(); + + Optional getJobResult() throws JobExecutionException, SqlExecutionException; + + JobStatus getJobStatus() throws JobExecutionException; + + void cancelJob() throws JobExecutionException; + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/NonJobOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/NonJobOperation.java new file mode 100644 index 0000000000..11dc76424a --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/NonJobOperation.java @@ -0,0 +1,23 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; + + +public interface NonJobOperation extends Operation { +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/Operation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/Operation.java new file mode 100644 index 0000000000..0d9a015496 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/Operation.java @@ -0,0 +1,31 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; + +public interface Operation { + + /** + * Execute the command and return the result. + */ + ResultSet execute() throws SqlExecutionException, JobExecutionException; +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationFactory.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationFactory.java new file mode 100644 index 0000000000..ca362204a7 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationFactory.java @@ -0,0 +1,129 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; + + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.CreateViewOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.DDLOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.DescribeTableOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.DropViewOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ExplainOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.InsertOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ResetOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.SelectOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.SetOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowCatalogsOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowCurrentCatalogOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowCurrentDatabaseOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowDatabasesOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowFunctionsOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowModulesOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowTablesOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowViewsOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.UseCatalogOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.UseDatabaseOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils.SqlCommandParser.SqlCommandCall; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlParseException; + +public class OperationFactory { + + public static Operation createOperation(SqlCommandCall call, FlinkEngineConnContext context) throws SqlParseException { + + Operation operation; + switch (call.command) { + case SELECT: + operation = new SelectOperation(context, call.operands[0]); + break; + case CREATE_VIEW: + operation = new CreateViewOperation(context, call.operands[0], call.operands[1]); + break; + case DROP_VIEW: + operation = new DropViewOperation(context, call.operands[0], Boolean.parseBoolean(call.operands[1])); + break; + case CREATE_TABLE: + case DROP_TABLE: + case ALTER_TABLE: + case CREATE_DATABASE: + case DROP_DATABASE: + case ALTER_DATABASE: + operation = new DDLOperation(context, call.operands[0], call.command); + break; + case SET: + // list all properties + if (call.operands.length == 0) { + operation = new SetOperation(context); + } else { + // set a property + operation = new SetOperation(context, call.operands[0], call.operands[1]); + } + break; + case RESET: + if (call.operands.length > 0) { + throw new SqlParseException("Only RESET ALL is supported now"); + } + operation = new ResetOperation(context); + break; + case USE_CATALOG: + operation = new UseCatalogOperation(context, call.operands[0]); + break; + case USE: + operation = new UseDatabaseOperation(context, call.operands[0]); + break; + case INSERT_INTO: + case INSERT_OVERWRITE: + operation = new InsertOperation(context, call.operands[0], call.operands[1]); + break; + case SHOW_MODULES: + operation = new ShowModulesOperation(context); + break; + case SHOW_CATALOGS: + operation = new ShowCatalogsOperation(context); + break; + case SHOW_CURRENT_CATALOG: + operation = new ShowCurrentCatalogOperation(context); + break; + case SHOW_DATABASES: + operation = new ShowDatabasesOperation(context); + break; + case SHOW_CURRENT_DATABASE: + operation = new ShowCurrentDatabaseOperation(context); + break; + case SHOW_TABLES: + operation = new ShowTablesOperation(context); + break; + case SHOW_VIEWS: + operation = new ShowViewsOperation(context); + break; + case SHOW_FUNCTIONS: + operation = new ShowFunctionsOperation(context); + break; + case DESCRIBE_TABLE: + operation = new DescribeTableOperation(context, call.operands[0]); + break; + case EXPLAIN: + operation = new ExplainOperation(context, call.operands[0]); + break; + default: + throw new SqlParseException("Unsupported command call " + call + ". This is a bug."); + } + + return operation; + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationUtil.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationUtil.java new file mode 100644 index 0000000000..fd8e428f0f --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationUtil.java @@ -0,0 +1,78 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import java.util.ArrayList; +import java.util.List; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.Row; + + +public class OperationUtil { + + public static final ResultSet OK = ResultSet.builder() + .resultKind(ResultKind.SUCCESS) + .columns(ColumnInfo.create(ConstantNames.RESULT, new VarCharType(2))) + .data(Row.of(ConstantNames.OK)) + .build(); + + public static ResultSet singleStringToResultSet(String str, String columnName) { + boolean isNullable; + int length; + + if (str == null) { + isNullable = true; + length = VarCharType.DEFAULT_LENGTH; + } else { + isNullable = false; + length = str.length(); + } + + return ResultSet.builder() + .resultKind(ResultKind.SUCCESS_WITH_CONTENT) + .columns(ColumnInfo.create(columnName, new VarCharType(isNullable, length))) + .data(Row.of(str)) + .build(); + } + + public static ResultSet stringListToResultSet(List strings, String columnName) { + List data = new ArrayList<>(); + boolean isNullable = false; + int maxLength = VarCharType.DEFAULT_LENGTH; + + for (String str : strings) { + if (str == null) { + isNullable = true; + } else { + maxLength = Math.max(str.length(), maxLength); + data.add(Row.of(str)); + } + } + + return ResultSet.builder() + .resultKind(ResultKind.SUCCESS_WITH_CONTENT) + .columns(ColumnInfo.create(columnName, new VarCharType(isNullable, maxLength))) + .data(data) + .build(); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/CreateViewOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/CreateViewOperation.java new file mode 100644 index 0000000000..b3686fd388 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/CreateViewOperation.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.client.config.entries.TableEntry; +import org.apache.flink.table.client.config.entries.ViewEntry; + +/** + * Operation for CREATE VIEW command. + */ +public class CreateViewOperation implements NonJobOperation { + private final ExecutionContext context; + private final String viewName; + private final String query; + + public CreateViewOperation(FlinkEngineConnContext context, String viewName, String query) { + this.context = context.getExecutionContext(); + this.viewName = viewName; + this.query = query; + } + + @Override + public ResultSet execute() throws SqlExecutionException { + Environment env = context.getEnvironment(); + TableEntry tableEntry = env.getTables().get(viewName); + if (tableEntry instanceof ViewEntry) { + throw new SqlExecutionException("'" + viewName + "' has already been defined in the current session."); + } + + // TODO check the logic + TableEnvironment tableEnv = context.getTableEnvironment(); + try { + context.wrapClassLoader(() -> { + tableEnv.createTemporaryView(viewName, tableEnv.sqlQuery(query)); + return null; + }); + } catch (Throwable t) { + // catch everything such that the query does not crash the executor + throw new SqlExecutionException("Invalid SQL statement.", t); + } + // Also attach the view to ExecutionContext#environment. + env.getTables().put(viewName, ViewEntry.create(viewName, query)); + return OperationUtil.OK; + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DDLOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DDLOperation.java new file mode 100644 index 0000000000..4449a1e96b --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DDLOperation.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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils.SqlCommandParser.SqlCommand; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import org.apache.flink.table.api.TableEnvironment; + +/** + * Operation for CREATE/DROP/ALTER TABLE/DATABASE command. + */ +public class DDLOperation implements NonJobOperation { + private final ExecutionContext context; + private final String ddl; + private final SqlCommand command; + + public DDLOperation(FlinkEngineConnContext context, String ddl, SqlCommand command) { + this.context = context.getExecutionContext(); + this.ddl = ddl; + this.command = command; + } + + @Override + public ResultSet execute() throws SqlExecutionException { + final TableEnvironment tEnv = context.getTableEnvironment(); + // parse and validate statement + try { + context.wrapClassLoader(() -> { + tEnv.sqlUpdate(ddl); + return null; + }); + } catch (Exception t) { + // catch everything such that the statement does not crash the executor + throw new SqlExecutionException(getExceptionMsg(), t); + } + + return OperationUtil.OK; + } + + private String getExceptionMsg() { + final String actionMsg; + switch (command) { + case CREATE_TABLE: + actionMsg = "create a table"; + break; + case CREATE_DATABASE: + actionMsg = "create a database"; + break; + case DROP_TABLE: + actionMsg = "drop a table"; + break; + case DROP_DATABASE: + actionMsg = "drop a database"; + break; + case ALTER_TABLE: + actionMsg = "alter a table"; + break; + case ALTER_DATABASE: + actionMsg = "alter a database"; + break; + default: + actionMsg = null; + } + + if (actionMsg != null) { + return String.format("Could not %s from statement: %s.", actionMsg, ddl); + } else { + return "Invalid DDL statement."; + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DescribeTableOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DescribeTableOperation.java new file mode 100644 index 0000000000..3ece9261d0 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DescribeTableOperation.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.WatermarkSpec; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.types.Row; + +/** + * Operation for DESCRIBE TABLE command. + */ +public class DescribeTableOperation implements NonJobOperation { + private final ExecutionContext context; + private final String tableName; + + public DescribeTableOperation(FlinkEngineConnContext context, String tableName) { + this.context = context.getExecutionContext(); + this.tableName = tableName; + } + + @Override + @SuppressWarnings("unchecked") + public ResultSet execute() throws SqlExecutionException { + // the implementation should be in sync with Flink, see FLINK-17112 + final TableEnvironment tableEnv = context.getTableEnvironment(); + TableSchema schema; + try { + schema = context.wrapClassLoader(() -> tableEnv.from(tableName).getSchema()); + } catch (Throwable t) { + // catch everything such that the query does not crash the executor + throw new SqlExecutionException("No table with this name could be found.", t); + } + + Map fieldToWatermark = new HashMap<>(); + for (WatermarkSpec spec : schema.getWatermarkSpecs()) { + fieldToWatermark.put(spec.getRowtimeAttribute(), spec.getWatermarkExpr()); + } + + Map fieldToPrimaryKey = new HashMap<>(); + if (schema.getPrimaryKey().isPresent()) { + List columns = schema.getPrimaryKey().get().getColumns(); + String primaryKey = "PRI(" + String.join(", ", columns) + ")"; + for (String column : columns) { + fieldToPrimaryKey.put(column, primaryKey); + } + } + + List columns = schema.getTableColumns(); + List data = new ArrayList<>(); + for (TableColumn column : columns) { + LogicalType logicalType = column.getType().getLogicalType(); + + String name = column.getName(); + String type = StringUtils.removeEnd(logicalType.toString(), " NOT NULL"); + boolean isNullable = logicalType.isNullable(); + String key = fieldToPrimaryKey.getOrDefault(column.getName(), null); + String computedColumn = column.getExpr().orElse(null); + String watermark = fieldToWatermark.getOrDefault(column.getName(), null); + + data.add(Row.of(name, type, isNullable, key, computedColumn, watermark)); + } + + return ResultSet.builder() + .resultKind(ResultKind.SUCCESS_WITH_CONTENT) + .columns( + ColumnInfo.create(ConstantNames.DESCRIBE_NAME, DataTypes.STRING().getLogicalType()), + ColumnInfo.create(ConstantNames.DESCRIBE_TYPE, DataTypes.STRING().getLogicalType()), + ColumnInfo.create(ConstantNames.DESCRIBE_NULL, new BooleanType()), + ColumnInfo.create(ConstantNames.DESCRIBE_KEY, DataTypes.STRING().getLogicalType()), + ColumnInfo.create(ConstantNames.DESCRIBE_COMPUTED_COLUMN, DataTypes.STRING().getLogicalType()), + ColumnInfo.create(ConstantNames.DESCRIBE_WATERMARK, DataTypes.STRING().getLogicalType())) + .data(data) + .build(); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DropViewOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DropViewOperation.java new file mode 100644 index 0000000000..a8ed9590f0 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DropViewOperation.java @@ -0,0 +1,79 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import org.apache.flink.table.client.config.entries.TableEntry; +import org.apache.flink.table.client.config.entries.ViewEntry; + +/** + * Operation for DROP VIEW command. + */ +public class DropViewOperation implements NonJobOperation { + private final FlinkEngineConnContext context; + private final String viewName; + private final boolean ifExists; + + public DropViewOperation(FlinkEngineConnContext context, String viewName, boolean ifExists) { + this.context = context; + this.viewName = viewName; + this.ifExists = ifExists; + } + + @Override + public ResultSet execute() throws SqlExecutionException { + Environment env = context.getExecutionContext().getEnvironment(); + TableEntry tableEntry = env.getTables().get(viewName); + if (!(tableEntry instanceof ViewEntry)) { + if (!ifExists) { + throw new SqlExecutionException("'" + viewName + "' does not exist in the current session."); + } + } + + // Here we rebuild the ExecutionContext because we want to ensure that all the remaining views can work fine. + // Assume the case: + // view1=select 1; + // view2=select * from view1; + // If we delete view1 successfully, then query view2 will throw exception because view1 does not exist. we want + // all the remaining views are OK, so do the ExecutionContext rebuilding to avoid breaking the view dependency. + Environment newEnv = env.clone(); + if (newEnv.getTables().remove(viewName) != null) { + ExecutionContext oldExecutionContext = context.getExecutionContext(); + oldExecutionContext.wrapClassLoader(() -> { + oldExecutionContext.getTableEnvironment().dropTemporaryView(viewName); + return null; + }); + // Renew the ExecutionContext. + ExecutionContext newExecutionContext = context + .newExecutionContextBuilder(context.getEnvironmentContext().getDefaultEnv()) + .env(newEnv) + .sessionState(context.getExecutionContext().getSessionState()) + .build(); + context.setExecutionContext(newExecutionContext); + } + + return OperationUtil.OK; + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ExplainOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ExplainOperation.java new file mode 100644 index 0000000000..644b2ac726 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ExplainOperation.java @@ -0,0 +1,78 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.Row; + +/** + * Operation for EXPLAIN command. + */ +public class ExplainOperation implements NonJobOperation { + private final ExecutionContext context; + private final String statement; + + public ExplainOperation(FlinkEngineConnContext context, String statement) { + this.context = context.getExecutionContext(); + this.statement = statement; + } + + @Override + public ResultSet execute() throws SqlExecutionException { + final TableEnvironment tableEnv = context.getTableEnvironment(); + // translate + try { + final Table table = createTable(context, tableEnv, statement); + String explanation = context.wrapClassLoader(() -> tableEnv.explain(table)); + return ResultSet.builder() + .resultKind(ResultKind.SUCCESS_WITH_CONTENT) + .columns(ColumnInfo.create(ConstantNames.EXPLAIN_RESULT, new VarCharType(false, explanation.length()))) + .data(Row.of(explanation)) + .build(); + } catch (SqlExecutionException t) { + throw t; + } catch (Exception t) { + // catch everything such that the query does not crash the executor + throw new SqlExecutionException("Invalid SQL statement.", t); + } + } + + /** + * Creates a table using the given query in the given table environment. + */ + private Table createTable(ExecutionContext context, TableEnvironment tableEnv, String selectQuery) throws SqlExecutionException { + // parse and validate query + try { + return context.wrapClassLoader(() -> tableEnv.sqlQuery(selectQuery)); + } catch (Exception t) { + // catch everything such that the query does not crash the executor + throw new SqlExecutionException("Invalid SQL statement.", t); + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/InsertOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/InsertOperation.java new file mode 100644 index 0000000000..68b593b983 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/InsertOperation.java @@ -0,0 +1,140 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment.ClusterDescriptorAdapterFactory; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.AbstractJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.JobInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.RowsType; +import java.sql.Statement; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Operation for INSERT command. + */ +public class InsertOperation extends AbstractJobOperation { + private static final Logger LOG = LoggerFactory.getLogger(InsertOperation.class); + + private final String statement; + private final List columnInfos; + + private boolean fetched = false; + + public InsertOperation(FlinkEngineConnContext context, String statement, String tableIdentifier) { + super(context); + this.statement = statement; + this.columnInfos = Collections.singletonList( + ColumnInfo.create(tableIdentifier, new BigIntType(false))); + } + + @Override + protected JobInfo submitJob() throws SqlExecutionException, JobExecutionException { + JobID jobId = executeUpdateInternal(context.getExecutionContext()); + String applicationId = this.clusterDescriptorAdapter.getClusterID().toString(); + String webInterfaceUrl = this.clusterDescriptorAdapter.getWebInterfaceUrl(); + return new JobInfoImpl(jobId, applicationId, webInterfaceUrl); + } + + @Override + protected Optional, List>> fetchJobResults() throws JobExecutionException { + if (fetched) { + return Optional.empty(); + } else { + // for session mode, we can get job status from JM, because JM is a long life service. + // while for per-job mode, JM will be also destroy after the job is finished. + // so it's hard to say whether the job is finished/canceled + // or the job status is just inaccessible at that moment. + // currently only yarn-per-job is supported, + // and if the exception (thrown when getting job status) contains ApplicationNotFoundException, + // we can say the job is finished. + boolean isGloballyTerminalState = clusterDescriptorAdapter.isGloballyTerminalState(); + if (isGloballyTerminalState) { + // TODO get affected_row_count for batch job + fetched = true; + return Optional.of(Tuple2.of(Collections.singletonList( + Row.of((long) Statement.SUCCESS_NO_INFO)), null)); + } else { + // TODO throws exception if the job fails + return Optional.of(Tuple2.of(Collections.emptyList(), null)); + } + } + } + + @Override + protected List getColumnInfos() { + return columnInfos; + } + + @Override + protected void cancelJobInternal() throws JobExecutionException { + clusterDescriptorAdapter.cancelJob(); + } + + private JobID executeUpdateInternal(ExecutionContext executionContext) throws SqlExecutionException, JobExecutionException { + TableEnvironment tableEnv = executionContext.getTableEnvironment(); + // parse and validate statement + TableResult tableResult; + try { + tableResult = executionContext.wrapClassLoader(() -> tableEnv.executeSql(statement)); + } catch (Exception t) { + LOG.error(String.format("Invalid SQL query, sql is: %s.", statement), t); + // catch everything such that the statement does not crash the executor + throw new SqlExecutionException("Invalid SQL statement.", t); + } + tableResult.collect(); + asyncNotify(tableResult); + JobID jobId = tableResult.getJobClient().get().getJobID(); + this.clusterDescriptorAdapter = + ClusterDescriptorAdapterFactory.create(context.getExecutionContext(), jobId); + clusterDescriptorAdapter.deployCluster(null, null); + return jobId; + } + + protected void asyncNotify(TableResult tableResult) { + CompletableFuture.completedFuture(tableResult) + .thenAccept(result -> { + CloseableIterator iterator = result.collect(); + int affected = 0; + while(iterator.hasNext()) { + Row row = iterator.next(); + affected = (int) row.getField(0); + } + int finalAffected = affected; + getFlinkStatusListeners().forEach(listener -> listener.onSuccess(finalAffected, RowsType.Affected())); + }).whenComplete((unused, throwable) -> getFlinkStatusListeners().forEach(listener -> listener.onFailed("Error while submitting job.", throwable))); + } + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ResetOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ResetOperation.java new file mode 100644 index 0000000000..c34e3b36d0 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ResetOperation.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; + +/** + * Operation for RESET command. + */ +public class ResetOperation implements NonJobOperation { + private final FlinkEngineConnContext context; + + public ResetOperation(FlinkEngineConnContext context) { + this.context = context; + } + + @Override + public ResultSet execute() throws SqlExecutionException { + ExecutionContext executionContext = context.getExecutionContext(); + // Renew the ExecutionContext by merging the default environment with original session context. + // Book keep all the session states of current ExecutionContext then + // re-register them into the new one. + ExecutionContext newExecutionContext = context + .newExecutionContextBuilder(context.getEnvironmentContext().getDefaultEnv()) + .sessionState(executionContext.getSessionState()) + .build(); + context.setExecutionContext(newExecutionContext); + + return OperationUtil.OK; + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SelectOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SelectOperation.java new file mode 100644 index 0000000000..845f8dde34 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SelectOperation.java @@ -0,0 +1,260 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment.ClusterDescriptorAdapterFactory; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment.ProgramDeployer; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result.AbstractResult; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result.BatchResult; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result.ChangelogResult; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result.ResultUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result.TypedResult; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.AbstractJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.JobInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.UUID; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.DeploymentOptions; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.utils.LogicalTypeUtils; +import org.apache.flink.table.types.utils.DataTypeUtils; +import org.apache.flink.types.Row; +import org.mortbay.util.ajax.JSON; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Operation for SELECT command. + */ +public class SelectOperation extends AbstractJobOperation { + + private static final Logger LOG = LoggerFactory.getLogger(SelectOperation.class); + + private final String query; + + private AbstractResult result; + + private TableSchema resultSchema; + + private List columnInfos; + + private volatile boolean noMoreResult; + + public SelectOperation(FlinkEngineConnContext context, String query) { + super(context); + this.query = query; + this.noMoreResult = false; + } + + @Override + protected JobInfo submitJob() throws JobExecutionException, SqlExecutionException { + JobID jobId = executeQueryInternal(context.getExecutionContext(), query); + List resultSchemaColumns = resultSchema.getTableColumns(); + columnInfos = new ArrayList<>(); + for (TableColumn column : resultSchemaColumns) { + columnInfos.add(ColumnInfo.create(column.getName(), column.getType().getLogicalType())); + } + String applicationId = this.clusterDescriptorAdapter.getClusterID().toString(); + String webInterfaceUrl = this.clusterDescriptorAdapter.getWebInterfaceUrl(); + return new JobInfoImpl(jobId, applicationId, webInterfaceUrl); + } + + @Override + protected void cancelJobInternal() throws JobExecutionException { + LOG.info("Start to cancel job {} and result retrieval.", getJobId()); + result.close(); + // ignore if there is no more result + // the job might has finished earlier. it's hard to say whether it need to be canceled, + // so the clients should be care for the exceptions ??? + if (noMoreResult) { + return; + } + + clusterDescriptorAdapter.cancelJob(); + } + + @Override + protected Optional, List>> fetchJobResults() throws SqlExecutionException, JobExecutionException { + Optional, List>> ret; + synchronized (lock) { + if (result == null) { + LOG.error("The job for this query has been canceled."); + throw new SqlExecutionException("The job for this query has been canceled."); + } + + if (this.result instanceof ChangelogResult) { + ret = fetchStreamingResult(); + } else { + ret = fetchBatchResult(); + } + } + return ret; + } + + @Override + protected List getColumnInfos() { + return columnInfos; + } + + private Optional, List>> fetchBatchResult() throws SqlExecutionException { + BatchResult batchResult = (BatchResult) this.result; + TypedResult> typedResult = batchResult.retrieveChanges(); + if (typedResult.getType() == TypedResult.ResultType.PAYLOAD) { + List payload = typedResult.getPayload(); + return Optional.of(Tuple2.of(payload, null)); + } else { + return Optional.of(Tuple2.of(Collections.emptyList(), null)); + } + } + + private Optional, List>> fetchStreamingResult() throws JobExecutionException { + ChangelogResult changLogResult = (ChangelogResult) this.result; + TypedResult>> typedResult = changLogResult.retrieveChanges(); + if (typedResult.getType() == TypedResult.ResultType.EOS) { + return Optional.of(Tuple2.of(Collections.emptyList(), Collections.emptyList())); + } else if (typedResult.getType() == TypedResult.ResultType.PAYLOAD) { + List> payload = typedResult.getPayload(); + List data = new ArrayList<>(); + List changeFlags = new ArrayList<>(); + for (Tuple2 tuple : payload) { + data.add(tuple.f1); + changeFlags.add(tuple.f0); + } + return Optional.of(Tuple2.of(data, changeFlags)); + } else { + return Optional.of(Tuple2.of(Collections.emptyList(), Collections.emptyList())); + } + } + + private JobID executeQueryInternal(ExecutionContext executionContext, String query) + throws JobExecutionException, SqlExecutionException { + // create table + final Table table = createTable(executionContext, executionContext.getTableEnvironment(), query); + boolean isChangelogResult = executionContext.getEnvironment().getExecution().inStreamingMode(); + // initialize result + resultSchema = removeTimeAttributes(table.getSchema()); + if (isChangelogResult) { + result = ResultUtil.createChangelogResult( + executionContext.getFlinkConfig(), + executionContext.getEnvironment(), + resultSchema, + executionContext.getExecutionConfig(), + executionContext.getClassLoader()); + } else { + result = ResultUtil.createBatchResult( + resultSchema, + executionContext.getExecutionConfig(), + executionContext.getClassLoader()); + } + result.setFlinkListeners(getFlinkListeners()); + final String tableName = String.format("_tmp_table_%s", UUID.randomUUID().toString().replace("-", "")); + final Pipeline pipeline; + try { + // writing to a sink requires an optimization step that might reference UDFs during code compilation + executionContext.wrapClassLoader(() -> { + executionContext.getTableEnvironment().registerTableSink(tableName, result.getTableSink()); + table.insertInto(tableName); + return null; + }); + pipeline = executionContext.createPipeline(query); + } catch (Exception t) { + // the result needs to be closed as long as + // it not stored in the result store + result.close(); + LOG.error(String.format("Invalid SQL query, sql is %s.", query), t); + // catch everything such that the query does not crash the executor + throw new SqlExecutionException("Invalid SQL query.", t); + } finally { + // Remove the temporal table object. + executionContext.wrapClassLoader(() -> { + executionContext.getTableEnvironment().dropTemporaryTable(tableName); + return null; + }); + } + + context.getExecutionContext().getFlinkConfig().setBoolean(DeploymentOptions.ATTACHED, true); + context.getExecutionContext().getFlinkConfig().setBoolean(DeploymentOptions.SHUTDOWN_IF_ATTACHED, true); + LOG.info("Deployer flink config {}.", JSON.toString(context.getExecutionContext().getFlinkConfig().toString())); + final ProgramDeployer deployer = new ProgramDeployer(context.getExecutionContext().getFlinkConfig(), query, pipeline); + // blocking deployment + JobClient jobClient; + try { + jobClient = deployer.deploy(context.getExecutionContext()).get(); + } catch (Exception e) { + LOG.error(String.format("Error running SQL job, sql is %s.", query), e); + throw new SqlExecutionException("Error running SQL job.", e); + } + + JobID jobId = jobClient.getJobID(); + this.clusterDescriptorAdapter = + ClusterDescriptorAdapterFactory.create(context.getExecutionContext(), jobId); + this.clusterDescriptorAdapter.deployCluster(null, null); + if (LOG.isDebugEnabled()) { + LOG.debug("Cluster Descriptor Adapter: {}", clusterDescriptorAdapter); + } + clusterDescriptorAdapter.deployCluster(null, null); + LOG.info("Submit flink job: {} successfully, query: {}.", jobId, query); + // start result retrieval + result.startRetrieval(jobClient); + return jobId; + } + + + /** + * Creates a table using the given query in the given table environment. + */ + private Table createTable(ExecutionContext context, TableEnvironment tableEnv, String selectQuery) throws SqlExecutionException { + // parse and validate query + try { + return context.wrapClassLoader(() -> tableEnv.sqlQuery(selectQuery)); + } catch (Exception t) { + // catch everything such that the query does not crash the executor + throw new SqlExecutionException("Invalid SQL statement.", t); + } + } + + private TableSchema removeTimeAttributes(TableSchema schema) { + final TableSchema.Builder builder = TableSchema.builder(); + for (int i = 0; i < schema.getFieldCount(); i++) { + final DataType dataType = schema.getFieldDataTypes()[i]; + final DataType convertedType = DataTypeUtils.replaceLogicalType( + dataType, + LogicalTypeUtils.removeTimeAttributes(dataType.getLogicalType())); + builder.field(schema.getFieldNames()[i], convertedType); + } + return builder.build(); + } + + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SetOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SetOperation.java new file mode 100644 index 0000000000..b09f7b290e --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SetOperation.java @@ -0,0 +1,111 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableMap; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.Row; + +/** + * Operation for SET command. + */ +public class SetOperation implements NonJobOperation { + private final FlinkEngineConnContext context; + private final String key; + private final String value; + + public SetOperation(FlinkEngineConnContext context, String key, String value) { + this.context = context; + this.key = key; + this.value = value; + } + + public SetOperation(FlinkEngineConnContext context) { + this(context, null, null); + } + + @Override + public ResultSet execute() throws SqlExecutionException { + ExecutionContext executionContext = context.getExecutionContext(); + Environment env = executionContext.getEnvironment(); + + // list all properties + if (key == null) { + List data = new ArrayList<>(); + Tuple2 maxKeyLenAndMaxValueLen = new Tuple2<>(1, 1); + buildResult(env.getExecution().asTopLevelMap(), data, maxKeyLenAndMaxValueLen); + buildResult(env.getDeployment().asTopLevelMap(), data, maxKeyLenAndMaxValueLen); + buildResult(env.getConfiguration().asMap(), data, maxKeyLenAndMaxValueLen); + + return ResultSet.builder() + .resultKind(ResultKind.SUCCESS_WITH_CONTENT) + .columns( + ColumnInfo.create(ConstantNames.SET_KEY, new VarCharType(true, maxKeyLenAndMaxValueLen.f0)), + ColumnInfo.create(ConstantNames.SET_VALUE, new VarCharType(true, maxKeyLenAndMaxValueLen.f1))) + .data(data) + .build(); + } else { + // TODO avoid to build a new Environment for some cases + // set a property + Environment newEnv = Environment.enrich(env, ImmutableMap.of(key.trim(), value.trim()), ImmutableMap.of()); + ExecutionContext.SessionState sessionState = executionContext.getSessionState(); + + // Renew the ExecutionContext by new environment. + // Book keep all the session states of current ExecutionContext then + // re-register them into the new one. + ExecutionContext newExecutionContext = context + .newExecutionContextBuilder(context.getEnvironmentContext().getDefaultEnv()) + .env(newEnv) + .sessionState(sessionState) + .build(); + context.setExecutionContext(newExecutionContext); + + return OperationUtil.OK; + } + } + + private void buildResult( + Map properties, + List data, + Tuple2 maxKeyLenAndMaxValueLen) { + for (Map.Entry entry : properties.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + data.add(Row.of(key, value)); + // update max key length + maxKeyLenAndMaxValueLen.f0 = Math.max(maxKeyLenAndMaxValueLen.f0, key.length()); + // update max value length + maxKeyLenAndMaxValueLen.f1 = Math.max(maxKeyLenAndMaxValueLen.f1, value.length()); + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCatalogsOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCatalogsOperation.java new file mode 100644 index 0000000000..b898b97838 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCatalogsOperation.java @@ -0,0 +1,47 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import java.util.Arrays; +import java.util.List; +import org.apache.flink.table.api.TableEnvironment; + +/** + * Operation for SHOW CATALOGS command. + */ +public class ShowCatalogsOperation implements NonJobOperation { + private final ExecutionContext context; + + public ShowCatalogsOperation(FlinkEngineConnContext context) { + this.context = context.getExecutionContext(); + } + + @Override + public ResultSet execute() { + final TableEnvironment tableEnv = context.getTableEnvironment(); + final List catalogs = context.wrapClassLoader(() -> Arrays.asList(tableEnv.listCatalogs())); + return OperationUtil.stringListToResultSet(catalogs, ConstantNames.SHOW_CATALOGS_RESULT); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentCatalogOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentCatalogOperation.java new file mode 100644 index 0000000000..9da10a51e7 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentCatalogOperation.java @@ -0,0 +1,45 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import org.apache.flink.table.api.TableEnvironment; + +/** + * Operation for SHOW CURRENT CATALOG command. + */ +public class ShowCurrentCatalogOperation implements NonJobOperation { + private final ExecutionContext context; + + public ShowCurrentCatalogOperation(FlinkEngineConnContext context) { + this.context = context.getExecutionContext(); + } + + @Override + public ResultSet execute() { + final TableEnvironment tableEnv = context.getTableEnvironment(); + return OperationUtil.singleStringToResultSet( + context.wrapClassLoader(tableEnv::getCurrentCatalog), ConstantNames.SHOW_CURRENT_CATALOG_RESULT); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentDatabaseOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentDatabaseOperation.java new file mode 100644 index 0000000000..3d36041fa7 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentDatabaseOperation.java @@ -0,0 +1,45 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import org.apache.flink.table.api.TableEnvironment; + +/** + * Operation for SHOW CURRENT DATABASE command. + */ +public class ShowCurrentDatabaseOperation implements NonJobOperation { + private final ExecutionContext context; + + public ShowCurrentDatabaseOperation(FlinkEngineConnContext context) { + this.context = context.getExecutionContext(); + } + + @Override + public ResultSet execute() { + final TableEnvironment tableEnv = context.getTableEnvironment(); + return OperationUtil.singleStringToResultSet( + context.wrapClassLoader(tableEnv::getCurrentDatabase), ConstantNames.SHOW_CURRENT_DATABASE_RESULT); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowDatabasesOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowDatabasesOperation.java new file mode 100644 index 0000000000..257fbdcbe2 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowDatabasesOperation.java @@ -0,0 +1,47 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import java.util.Arrays; +import java.util.List; +import org.apache.flink.table.api.TableEnvironment; + +/** + * Operation for SHOW DATABASES command. + */ +public class ShowDatabasesOperation implements NonJobOperation { + private final ExecutionContext context; + + public ShowDatabasesOperation(FlinkEngineConnContext context) { + this.context = context.getExecutionContext(); + } + + @Override + public ResultSet execute() { + final TableEnvironment tableEnv = context.getTableEnvironment(); + final List databases = context.wrapClassLoader(() -> Arrays.asList(tableEnv.listDatabases())); + return OperationUtil.stringListToResultSet(databases, ConstantNames.SHOW_DATABASES_RESULT); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowFunctionsOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowFunctionsOperation.java new file mode 100644 index 0000000000..64378702d9 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowFunctionsOperation.java @@ -0,0 +1,47 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import java.util.Arrays; +import java.util.List; +import org.apache.flink.table.api.TableEnvironment; + +/** + * Operation for SHOW FUNCTIONS command. + */ +public class ShowFunctionsOperation implements NonJobOperation { + private final ExecutionContext context; + + public ShowFunctionsOperation(FlinkEngineConnContext context) { + this.context = context.getExecutionContext(); + } + + @Override + public ResultSet execute() { + final TableEnvironment tableEnv = context.getTableEnvironment(); + final List functions = context.wrapClassLoader(() -> Arrays.asList(tableEnv.listFunctions())); + return OperationUtil.stringListToResultSet(functions, ConstantNames.SHOW_FUNCTIONS_RESULT); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowModulesOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowModulesOperation.java new file mode 100644 index 0000000000..c305e36b72 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowModulesOperation.java @@ -0,0 +1,47 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import java.util.Arrays; +import java.util.List; +import org.apache.flink.table.api.TableEnvironment; + +/** + * Operation for SHOW MODULES command. + */ +public class ShowModulesOperation implements NonJobOperation { + private final ExecutionContext context; + + public ShowModulesOperation(FlinkEngineConnContext context) { + this.context = context.getExecutionContext(); + } + + @Override + public ResultSet execute() { + final TableEnvironment tableEnv = context.getTableEnvironment(); + final List modules = context.wrapClassLoader(() -> Arrays.asList(tableEnv.listModules())); + return OperationUtil.stringListToResultSet(modules, ConstantNames.SHOW_MODULES_RESULT); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowTablesOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowTablesOperation.java new file mode 100644 index 0000000000..9e85e7708d --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowTablesOperation.java @@ -0,0 +1,63 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.Row; + +/** + * Operation for SHOW TABLES command. + */ +public class ShowTablesOperation implements NonJobOperation { + private final ExecutionContext context; + + public ShowTablesOperation(FlinkEngineConnContext context) { + this.context = context.getExecutionContext(); + } + + @Override + public ResultSet execute() { + List rows = new ArrayList<>(); + int maxNameLength = 1; + + final TableEnvironment tableEnv = context.getTableEnvironment(); + // listTables will return all tables and views + for (String table : context.wrapClassLoader(() -> Arrays.asList(tableEnv.listTables()))) { + rows.add(Row.of(table)); + maxNameLength = Math.max(maxNameLength, table.length()); + } + + return ResultSet.builder() + .resultKind(ResultKind.SUCCESS_WITH_CONTENT) + .columns(ColumnInfo.create(ConstantNames.SHOW_TABLES_RESULT, new VarCharType(false, maxNameLength))) + .data(rows) + .build(); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowViewsOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowViewsOperation.java new file mode 100644 index 0000000000..fae722a5d5 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowViewsOperation.java @@ -0,0 +1,66 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.client.config.entries.TableEntry; +import org.apache.flink.table.client.config.entries.ViewEntry; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.types.Row; + +/** + * Operation for SHOW VIEWS command. + */ +public class ShowViewsOperation implements NonJobOperation { + + private final ExecutionContext context; + + public ShowViewsOperation(FlinkEngineConnContext context) { + this.context = context.getExecutionContext(); + } + + @Override + public ResultSet execute() { + List rows = new ArrayList<>(); + int maxNameLength = 1; + + for (Map.Entry entry : context.getEnvironment().getTables().entrySet()) { + if (entry.getValue() instanceof ViewEntry) { + String name = entry.getKey(); + rows.add(Row.of(name)); + maxNameLength = Math.max(maxNameLength, name.length()); + } + } + + return ResultSet.builder() + .resultKind(ResultKind.SUCCESS_WITH_CONTENT) + .columns(ColumnInfo.create(ConstantNames.SHOW_VIEWS_RESULT, new VarCharType(false, maxNameLength))) + .data(rows) + .build(); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseCatalogOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseCatalogOperation.java new file mode 100644 index 0000000000..2c31182212 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseCatalogOperation.java @@ -0,0 +1,58 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.exceptions.CatalogException; + +/** + * Operation for USE CATALOG command. + */ +public class UseCatalogOperation implements NonJobOperation { + private final ExecutionContext context; + private final String catalogName; + + public UseCatalogOperation(FlinkEngineConnContext context, String catalogName) { + this.context = context.getExecutionContext(); + this.catalogName = catalogName; + } + + @Override + public ResultSet execute() throws SqlExecutionException { + final TableEnvironment tableEnv = context.getTableEnvironment(); + + try { + context.wrapClassLoader(() -> { + // Rely on TableEnvironment/CatalogManager to validate input + tableEnv.useCatalog(catalogName); + return null; + }); + } catch (CatalogException e) { + throw new SqlExecutionException("Failed to switch to catalog " + catalogName, e); + } + + return OperationUtil.OK; + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseDatabaseOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseDatabaseOperation.java new file mode 100644 index 0000000000..a450e29a2d --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseDatabaseOperation.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.catalog.exceptions.CatalogException; + +/** + * Operation for USE DATABASE command. + */ +public class UseDatabaseOperation implements NonJobOperation { + private final ExecutionContext context; + private final String databaseName; + + public UseDatabaseOperation(FlinkEngineConnContext context, String databaseName) { + this.context = context.getExecutionContext(); + this.databaseName = databaseName; + } + + @Override + public ResultSet execute() throws SqlExecutionException { + final TableEnvironment tableEnv = context.getTableEnvironment(); + try { + context.wrapClassLoader(() -> { + // Rely on TableEnvironment/CatalogManager to validate input + tableEnv.useDatabase(databaseName); + return null; + }); + } catch (CatalogException e) { + throw new SqlExecutionException("Failed to switch to database " + databaseName, e); + } + return OperationUtil.OK; + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ColumnInfo.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ColumnInfo.java new file mode 100644 index 0000000000..8540154f75 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ColumnInfo.java @@ -0,0 +1,101 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; + +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.utils.LogicalTypeParser; +import org.apache.flink.util.Preconditions; + +/** + * A column info represents a table column's structure with column name, column type. + */ +public class ColumnInfo { + + private static final String FIELD_NAME_NAME = "name"; + private static final String FIELD_NAME_TYPE = "type"; + + @JsonProperty(FIELD_NAME_NAME) + private String name; + + @JsonProperty(FIELD_NAME_TYPE) + private String type; + + @JsonIgnore + @Nullable + private LogicalType logicalType; + + @JsonCreator + public ColumnInfo( + @JsonProperty(FIELD_NAME_NAME) String name, + @JsonProperty(FIELD_NAME_TYPE) String type) { + this.name = Preconditions.checkNotNull(name, "name must not be null"); + this.type = Preconditions.checkNotNull(type, "type must not be null"); + } + + public static ColumnInfo create(String name, LogicalType type) { + return new ColumnInfo(name, type.toString()); + } + + public String getName() { + return name; + } + + public String getType() { + return type; + } + + @JsonIgnore + public LogicalType getLogicalType() { + if (logicalType == null) { + logicalType = LogicalTypeParser.parse(type); + } + return logicalType; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ColumnInfo that = (ColumnInfo) o; + return name.equals(that.name) && + type.equals(that.type); + } + + @Override + public int hashCode() { + return Objects.hash(name, type); + } + + @Override + public String toString() { + return "ColumnInfo{" + + "name='" + name + '\'' + + ", type='" + type + '\'' + + '}'; + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ConstantNames.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ConstantNames.java new file mode 100644 index 0000000000..e29297b2fe --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ConstantNames.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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; + +/** + * Constant column names. + */ +public class ConstantNames { + + // for statement execution + public static final String JOB_ID = "job_id"; + // for results with SUCCESS result kind + public static final String RESULT = "result"; + public static final String OK = "OK"; + + public static final String SHOW_MODULES_RESULT = "modules"; + + public static final String SHOW_CURRENT_CATALOG_RESULT = "catalog"; + + public static final String SHOW_CATALOGS_RESULT = "catalogs"; + + public static final String SHOW_CURRENT_DATABASE_RESULT = "database"; + + public static final String SHOW_DATABASES_RESULT = "databases"; + + public static final String SHOW_FUNCTIONS_RESULT = "functions"; + + public static final String EXPLAIN_RESULT = "explanation"; + + public static final String DESCRIBE_NAME = "name"; + public static final String DESCRIBE_TYPE = "type"; + public static final String DESCRIBE_NULL = "null"; + public static final String DESCRIBE_KEY = "key"; + public static final String DESCRIBE_COMPUTED_COLUMN = "computed_column"; + public static final String DESCRIBE_WATERMARK = "watermark"; + + public static final String SHOW_TABLES_RESULT = "tables"; + + public static final String SHOW_VIEWS_RESULT = "views"; + + public static final String SET_KEY = "key"; + public static final String SET_VALUE = "value"; +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultKind.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultKind.java new file mode 100644 index 0000000000..27cb198d8d --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultKind.java @@ -0,0 +1,30 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; + +/** + * ResultKind defines the types of the result. + */ +public enum ResultKind { + // for DDL, DCL and statements with a simple "OK" + SUCCESS, + + // rows with important content are available (DML, DQL) + SUCCESS_WITH_CONTENT +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSet.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSet.java new file mode 100644 index 0000000000..54f54e5c42 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSet.java @@ -0,0 +1,189 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; + +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import javax.annotation.Nullable; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; + +/** + * A set of one statement execution result containing result kind, column infos, + * rows of data and change flags for streaming mode. + */ +@JsonSerialize(using = ResultSetJsonSerializer.class) +@JsonDeserialize(using = ResultSetJsonDeserializer.class) +public class ResultSet { + static final String FIELD_NAME_RESULT_KIND = "result_kind"; + static final String FIELD_NAME_COLUMNS = "columns"; + static final String FIELD_NAME_DATA = "data"; + static final String FIELD_NAME_CHANGE_FLAGS = "change_flags"; + + private final ResultKind resultKind; + private final List columns; + private final List data; + + // null in batch mode + // + // list of boolean in streaming mode, + // true if the corresponding row is an append row, false if its a retract row + private final List changeFlags; + + private ResultSet( + ResultKind resultKind, + List columns, + List data, + @Nullable List changeFlags) { + this.resultKind = Preconditions.checkNotNull(resultKind, "resultKind must not be null"); + this.columns = Preconditions.checkNotNull(columns, "columns must not be null"); + this.data = Preconditions.checkNotNull(data, "data must not be null"); + if (!data.isEmpty()) { + Preconditions.checkArgument(columns.size() == data.get(0).getArity(), + "the size of columns and the number of fields in the row should be equal"); + } + this.changeFlags = changeFlags; + if (changeFlags != null) { + Preconditions.checkArgument(data.size() == changeFlags.size(), + "the size of data and the size of changeFlags should be equal"); + } + } + + public ResultKind getResultKind() { + return resultKind; + } + + public List getColumns() { + return columns; + } + + public List getData() { + return data; + } + + public Optional> getChangeFlags() { + return Optional.ofNullable(changeFlags); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ResultSet resultSet = (ResultSet) o; + return resultKind.equals(resultSet.resultKind) && + columns.equals(resultSet.columns) && + data.equals(resultSet.data) && + Objects.equals(changeFlags, resultSet.changeFlags); + } + + @Override + public int hashCode() { + return Objects.hash(resultKind, columns, data, changeFlags); + } + + @Override + public String toString() { + return "ResultSet{" + + "resultKind=" + resultKind + + ", columns=" + columns + + ", data=" + data + + ", changeFlags=" + changeFlags + + '}'; + } + + public static Builder builder() { + return new Builder(); + } + + /** + * Builder for {@link ResultSet}. + */ + public static class Builder { + private ResultKind resultKind = null; + private List columns = null; + private List data = null; + private List changeFlags = null; + + private Builder() { + } + + /** + * Set {@link ResultKind}. + */ + public Builder resultKind(ResultKind resultKind) { + this.resultKind = resultKind; + return this; + } + + /** + * Set {@link ColumnInfo}s. + */ + public Builder columns(ColumnInfo... columns) { + this.columns = Arrays.asList(columns); + return this; + } + + /** + * Set {@link ColumnInfo}s. + */ + public Builder columns(List columns) { + this.columns = columns; + return this; + } + + /** + * Set data. + */ + public Builder data(List data) { + this.data = data; + return this; + } + + /** + * Set data. + */ + public Builder data(Row... data) { + this.data = Arrays.asList(data); + return this; + } + + /** + * Set change flags. + */ + public Builder changeFlags(List changeFlags) { + this.changeFlags = changeFlags; + return this; + } + + /** + * Returns a {@link ResultSet} instance. + */ + public ResultSet build() { + return new ResultSet(resultKind, columns, data, changeFlags); + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonDeserializer.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonDeserializer.java new file mode 100644 index 0000000000..2d91a8fdc8 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonDeserializer.java @@ -0,0 +1,200 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; + +import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_CHANGE_FLAGS; +import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_COLUMNS; +import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_DATA; +import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_RESULT_KIND; + +import java.io.IOException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParseException; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonToken; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.types.Row; + +/** + * Json deserializer for {@link ResultSet}. + */ +public class ResultSetJsonDeserializer extends StdDeserializer { + + protected ResultSetJsonDeserializer() { + super(ResultSet.class); + } + + @Override + public ResultSet deserialize( + JsonParser jsonParser, + DeserializationContext ctx) throws IOException { + JsonNode node = jsonParser.getCodec().readTree(jsonParser); + + ResultKind resultKind; + List columns; + List changeFlags = null; + List data; + + JsonNode resultKindNode = node.get(FIELD_NAME_RESULT_KIND); + if (resultKindNode != null) { + JsonParser resultKindParser = node.get(FIELD_NAME_RESULT_KIND).traverse(); + resultKindParser.nextToken(); + resultKind = ctx.readValue(resultKindParser, ResultKind.class); + } else { + throw new JsonParseException(jsonParser, "Field resultKind must be provided"); + } + + JsonNode columnNode = node.get(FIELD_NAME_COLUMNS); + if (columnNode != null) { + JsonParser columnParser = node.get(FIELD_NAME_COLUMNS).traverse(); + columnParser.nextToken(); + columns = Arrays.asList(ctx.readValue(columnParser, ColumnInfo[].class)); + } else { + throw new JsonParseException(jsonParser, "Field column must be provided"); + } + + JsonNode changeFlagNode = node.get(FIELD_NAME_CHANGE_FLAGS); + if (changeFlagNode != null) { + JsonParser changeFlagParser = changeFlagNode.traverse(); + changeFlagParser.nextToken(); + changeFlags = Arrays.asList(ctx.readValue(changeFlagParser, Boolean[].class)); + } + + JsonNode dataNode = node.get(FIELD_NAME_DATA); + if (dataNode != null) { + data = deserializeRows(columns, dataNode, ctx); + } else { + throw new JsonParseException(jsonParser, "Field data must be provided"); + } + + return ResultSet.builder() + .resultKind(resultKind) + .columns(columns) + .data(data) + .changeFlags(changeFlags) + .build(); + } + + private List deserializeRows( + List columns, + JsonNode dataNode, + DeserializationContext ctx) throws IOException { + if (!dataNode.isArray()) { + throw new JsonParseException(dataNode.traverse(), "Expecting data to be an array but it's not"); + } + + List fields = new ArrayList<>(); + for (ColumnInfo column : columns) { + fields.add(new RowType.RowField(column.getName(), column.getLogicalType())); + } + RowType rowType = new RowType(fields); + + List data = new ArrayList<>(); + for (JsonNode rowNode : dataNode) { + data.add(deserializeRow(rowType, rowNode, ctx)); + } + return data; + } + + private LocalDate deserializeLocalDate( + JsonParser parser, + DeserializationContext ctx) throws IOException { + return LocalDate.parse(ctx.readValue(parser, String.class)); + } + + private LocalTime deserializeLocalTime( + JsonParser parser, + DeserializationContext ctx) throws IOException { + return LocalTime.parse(ctx.readValue(parser, String.class)); + } + + private LocalDateTime deserializeLocalDateTime( + JsonParser parser, + DeserializationContext ctx) throws IOException { + return LocalDateTime.parse(ctx.readValue(parser, String.class)); + } + + private Row deserializeRow( + RowType type, + JsonNode node, + DeserializationContext ctx) throws IOException { + if (!node.isArray()) { + throw new JsonParseException(node.traverse(), "Expecting row to be an array but it's not"); + } + + int fieldCount = type.getFieldCount(); + List fields = type.getFields(); + Row row = new Row(fieldCount); + + int i = 0; + for (JsonNode fieldNode : node) { + if (i >= fieldCount) { + throw new JsonParseException( + node.traverse(), "Number of columns in the row is not consistent with column infos"); + } + row.setField(i, deserializeObject(fields.get(i).getType(), fieldNode, ctx)); + i++; + } + if (i != fieldCount) { + throw new JsonParseException( + node.traverse(), "Number of columns in the row is not consistent with column infos"); + } + + return row; + } + + private Object deserializeObject( + LogicalType type, + JsonNode node, + DeserializationContext ctx) throws IOException { + if (type instanceof RowType) { + return deserializeRow((RowType) type, node, ctx); + } + + JsonParser parser = node.traverse(); + parser.nextToken(); + if (parser.currentToken() == JsonToken.VALUE_NULL) { + // we have to manually parse null value + // as jackson refuses to deserialize null value to java objects + return null; + } + + if (type instanceof DateType) { + return deserializeLocalDate(parser, ctx); + } else if (type instanceof TimeType) { + return deserializeLocalTime(parser, ctx); + } else if (type instanceof TimestampType) { + return deserializeLocalDateTime(parser, ctx); + } else { + return ctx.readValue(parser, type.getDefaultConversion()); + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonSerializer.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonSerializer.java new file mode 100644 index 0000000000..b958a19de2 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonSerializer.java @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; + +import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_CHANGE_FLAGS; +import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_COLUMNS; +import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_DATA; +import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_RESULT_KIND; + +import java.io.IOException; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.format.DateTimeFormatter; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; +import org.apache.flink.types.Row; + +/** + * Json serializer for {@link ResultSet}. + */ +public class ResultSetJsonSerializer extends StdSerializer { + + protected ResultSetJsonSerializer() { + super(ResultSet.class); + } + + @Override + public void serialize( + ResultSet resultSet, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) throws IOException { + jsonGenerator.writeStartObject(); + + serializerProvider.defaultSerializeField(FIELD_NAME_RESULT_KIND, resultSet.getResultKind(), jsonGenerator); + serializerProvider.defaultSerializeField(FIELD_NAME_COLUMNS, resultSet.getColumns(), jsonGenerator); + + jsonGenerator.writeFieldName(FIELD_NAME_DATA); + jsonGenerator.writeStartArray(); + for (Row row : resultSet.getData()) { + serializeRow(row, jsonGenerator, serializerProvider); + } + jsonGenerator.writeEndArray(); + + if (resultSet.getChangeFlags().isPresent()) { + serializerProvider.defaultSerializeField(FIELD_NAME_CHANGE_FLAGS, resultSet.getChangeFlags().get(), jsonGenerator); + } + + jsonGenerator.writeEndObject(); + } + + private void serializeLocalDate( + LocalDate localDate, + JsonGenerator jsonGenerator) throws IOException { + jsonGenerator.writeString(localDate.format(DateTimeFormatter.ISO_LOCAL_DATE)); + } + + private void serializeLocalTime( + LocalTime localTime, + JsonGenerator jsonGenerator) throws IOException { + jsonGenerator.writeString(localTime.format(DateTimeFormatter.ISO_LOCAL_TIME)); + } + + private void serializeLocalDateTime( + LocalDateTime localDateTime, + JsonGenerator jsonGenerator) throws IOException { + jsonGenerator.writeString(localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME)); + } + + private void serializeRow( + Row row, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) throws IOException { + jsonGenerator.writeStartArray(); + for (int i = 0; i < row.getArity(); i++) { + serializeObject(row.getField(i), jsonGenerator, serializerProvider); + } + jsonGenerator.writeEndArray(); + } + + private void serializeObject( + Object o, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) throws IOException { + if (o instanceof LocalDate) { + serializeLocalDate((LocalDate) o, jsonGenerator); + } else if (o instanceof LocalTime) { + serializeLocalTime((LocalTime) o, jsonGenerator); + } else if (o instanceof LocalDateTime) { + serializeLocalDateTime((LocalDateTime) o, jsonGenerator); + } else if (o instanceof Row) { + serializeRow((Row) o, jsonGenerator, serializerProvider); + } else { + serializerProvider.defaultSerializeValue(o, jsonGenerator); + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/SqlCommandParser.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/SqlCommandParser.java new file mode 100644 index 0000000000..a260a4e87e --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/SqlCommandParser.java @@ -0,0 +1,403 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils; + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlParseException; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Objects; +import java.util.Optional; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.calcite.config.Lex; +import org.apache.calcite.sql.SqlDrop; +import org.apache.calcite.sql.SqlExplain; +import org.apache.calcite.sql.SqlKind; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlNodeList; +import org.apache.calcite.sql.SqlSetOption; +import org.apache.calcite.sql.parser.SqlParser; +import org.apache.flink.sql.parser.ddl.SqlAlterDatabase; +import org.apache.flink.sql.parser.ddl.SqlAlterTable; +import org.apache.flink.sql.parser.ddl.SqlCreateDatabase; +import org.apache.flink.sql.parser.ddl.SqlCreateTable; +import org.apache.flink.sql.parser.ddl.SqlCreateView; +import org.apache.flink.sql.parser.ddl.SqlDropDatabase; +import org.apache.flink.sql.parser.ddl.SqlDropTable; +import org.apache.flink.sql.parser.ddl.SqlDropView; +import org.apache.flink.sql.parser.ddl.SqlUseCatalog; +import org.apache.flink.sql.parser.ddl.SqlUseDatabase; +import org.apache.flink.sql.parser.dml.RichSqlInsert; +import org.apache.flink.sql.parser.dql.SqlRichDescribeTable; +import org.apache.flink.sql.parser.dql.SqlShowCatalogs; +import org.apache.flink.sql.parser.dql.SqlShowDatabases; +import org.apache.flink.sql.parser.dql.SqlShowFunctions; +import org.apache.flink.sql.parser.dql.SqlShowTables; +import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl; +import org.apache.flink.sql.parser.validate.FlinkSqlConformance; + +/** + * Simple parser for determining the type of command and its parameters. + */ +public final class SqlCommandParser { + + private SqlCommandParser() { + // private + } + + /** + * Parse the given statement and return corresponding SqlCommandCall. + * + *

only `set`, `show modules`, `show current catalog` and `show current database` + * are parsed through regex matching, other commands are parsed through sql parser. + * + *

throw {@link SqlParseException} if the statement contains multiple sub-statements separated by semicolon + * or there is a parse error. + * + *

NOTE: sql parser only parses the statement to get the corresponding SqlCommand, + * do not check whether the statement is valid here. + */ + public static Optional parse(String stmt, boolean isBlinkPlanner) throws SqlParseException { + // normalize + String stmtForRegexMatch = stmt.trim(); + // remove ';' at the end + if (stmtForRegexMatch.endsWith(";")) { + stmtForRegexMatch = stmtForRegexMatch.substring(0, stmtForRegexMatch.length() - 1).trim(); + } + + // only parse gateway specific statements + for (SqlCommand cmd : SqlCommand.values()) { + if (cmd.hasPattern()) { + final Matcher matcher = cmd.pattern.matcher(stmtForRegexMatch); + if (matcher.matches()) { + final String[] groups = new String[matcher.groupCount()]; + for (int i = 0; i < groups.length; i++) { + groups[i] = matcher.group(i + 1); + } + return cmd.operandConverter.apply(groups) + .map((operands) -> new SqlCommandCall(cmd, operands)); + } + } + } + + return parseStmt(stmt, isBlinkPlanner); + } + + /** + * Flink Parser only supports partial Operations, so we directly use Calcite Parser here. + * Once Flink Parser supports all Operations, we should use Flink Parser instead of Calcite Parser. + */ + private static Optional parseStmt(String stmt, boolean isBlinkPlanner) throws SqlParseException { + SqlParser.Config config = createSqlParserConfig(isBlinkPlanner); + SqlParser sqlParser = SqlParser.create(stmt, config); + SqlNodeList sqlNodes; + try { + sqlNodes = sqlParser.parseStmtList(); + // no need check the statement is valid here + } catch (org.apache.calcite.sql.parser.SqlParseException e) { + throw new SqlParseException("Failed to parse statement.", e); + } + if (sqlNodes.size() != 1) { + throw new SqlParseException("Only single statement is supported now"); + } + + final String[] operands; + final SqlCommand cmd; + SqlNode node = sqlNodes.get(0); + if (node.getKind().belongsTo(SqlKind.QUERY)) { + cmd = SqlCommand.SELECT; + operands = new String[] { stmt }; + } else if (node instanceof RichSqlInsert) { + RichSqlInsert insertNode = (RichSqlInsert) node; + cmd = insertNode.isOverwrite() ? SqlCommand.INSERT_OVERWRITE : SqlCommand.INSERT_INTO; + operands = new String[] { stmt, insertNode.getTargetTable().toString() }; + } else if (node instanceof SqlShowTables) { + cmd = SqlCommand.SHOW_TABLES; + operands = new String[0]; + } else if (node instanceof SqlCreateTable) { + cmd = SqlCommand.CREATE_TABLE; + operands = new String[] { stmt }; + } else if (node instanceof SqlDropTable) { + cmd = SqlCommand.DROP_TABLE; + operands = new String[] { stmt }; + } else if (node instanceof SqlAlterTable) { + cmd = SqlCommand.ALTER_TABLE; + operands = new String[] { stmt }; + } else if (node instanceof SqlCreateView) { + // TableEnvironment currently does not support creating view + // so we have to perform the modification here + SqlCreateView createViewNode = (SqlCreateView) node; + cmd = SqlCommand.CREATE_VIEW; + operands = new String[] { + createViewNode.getViewName().toString(), + createViewNode.getQuery().toString() + }; + } else if (node instanceof SqlDropView) { + // TableEnvironment currently does not support dropping view + // so we have to perform the modification here + SqlDropView dropViewNode = (SqlDropView) node; + + // TODO: we can't get this field from SqlDropView normally until FLIP-71 is implemented + Field ifExistsField; + try { + ifExistsField = SqlDrop.class.getDeclaredField("ifExists"); + } catch (NoSuchFieldException e) { + throw new SqlParseException("Failed to parse drop view statement.", e); + } + ifExistsField.setAccessible(true); + boolean ifExists; + try { + ifExists = ifExistsField.getBoolean(dropViewNode); + } catch (IllegalAccessException e) { + throw new SqlParseException("Failed to parse drop view statement.", e); + } + + cmd = SqlCommand.DROP_VIEW; + operands = new String[] { dropViewNode.getViewName().toString(), String.valueOf(ifExists) }; + } else if (node instanceof SqlShowDatabases) { + cmd = SqlCommand.SHOW_DATABASES; + operands = new String[0]; + } else if (node instanceof SqlCreateDatabase) { + cmd = SqlCommand.CREATE_DATABASE; + operands = new String[] { stmt }; + } else if (node instanceof SqlDropDatabase) { + cmd = SqlCommand.DROP_DATABASE; + operands = new String[] { stmt }; + } else if (node instanceof SqlAlterDatabase) { + cmd = SqlCommand.ALTER_DATABASE; + operands = new String[] { stmt }; + } else if (node instanceof SqlShowCatalogs) { + cmd = SqlCommand.SHOW_CATALOGS; + operands = new String[0]; + } else if (node instanceof SqlShowFunctions) { + cmd = SqlCommand.SHOW_FUNCTIONS; + operands = new String[0]; + } else if (node instanceof SqlUseCatalog) { + cmd = SqlCommand.USE_CATALOG; + operands = new String[] { ((SqlUseCatalog) node).getCatalogName() }; + } else if (node instanceof SqlUseDatabase) { + cmd = SqlCommand.USE; + operands = new String[] { ((SqlUseDatabase) node).getDatabaseName().toString() }; + } else if (node instanceof SqlRichDescribeTable) { + cmd = SqlCommand.DESCRIBE_TABLE; + // TODO support describe extended + String[] fullTableName = ((SqlRichDescribeTable) node).fullTableName(); + String escapedName = + Stream.of(fullTableName).map(s -> "`" + s + "`").collect(Collectors.joining(".")); + operands = new String[] { escapedName }; + } else if (node instanceof SqlExplain) { + cmd = SqlCommand.EXPLAIN; + // TODO support explain details + operands = new String[] { ((SqlExplain) node).getExplicandum().toString() }; + } else if (node instanceof SqlSetOption) { + SqlSetOption setNode = (SqlSetOption) node; + // refer to SqlSetOption#unparseAlterOperation + if (setNode.getValue() != null) { + cmd = SqlCommand.SET; + operands = new String[] { setNode.getName().toString(), setNode.getValue().toString() }; + } else { + cmd = SqlCommand.RESET; + if (setNode.getName().toString().toUpperCase().equals("ALL")) { + operands = new String[0]; + } else { + operands = new String[] { setNode.getName().toString() }; + } + } + } else { + cmd = null; + operands = new String[0]; + } + + if (cmd == null) { + return Optional.empty(); + } else { + // use the origin given statement to make sure + // users can find the correct line number when parsing failed + return Optional.of(new SqlCommandCall(cmd, operands)); + } + } + + /** + * A temporary solution. We can't get the default SqlParser config through table environment now. + */ + private static SqlParser.Config createSqlParserConfig(boolean isBlinkPlanner) { + if (isBlinkPlanner) { + return SqlParser + .configBuilder() + .setParserFactory(FlinkSqlParserImpl.FACTORY) + .setConformance(FlinkSqlConformance.DEFAULT) + .setLex(Lex.JAVA) + .setIdentifierMaxLength(256) + .build(); + } else { + return SqlParser + .configBuilder() + .setParserFactory(FlinkSqlParserImpl.FACTORY) + .setConformance(FlinkSqlConformance.DEFAULT) + .setLex(Lex.JAVA) + .build(); + } + } + + // -------------------------------------------------------------------------------------------- + + private static final Function> NO_OPERANDS = + (operands) -> Optional.of(new String[0]); + + private static final int DEFAULT_PATTERN_FLAGS = Pattern.CASE_INSENSITIVE | Pattern.DOTALL; + + /** + * Supported SQL commands. + */ + public enum SqlCommand { + SELECT, + + INSERT_INTO, + + INSERT_OVERWRITE, + + CREATE_TABLE, + + ALTER_TABLE, + + DROP_TABLE, + + CREATE_VIEW, + + DROP_VIEW, + + CREATE_DATABASE, + + ALTER_DATABASE, + + DROP_DATABASE, + + USE_CATALOG, + + USE, + + SHOW_CATALOGS, + + SHOW_DATABASES, + + SHOW_TABLES, + + SHOW_FUNCTIONS, + + EXPLAIN, + + DESCRIBE_TABLE, + + RESET, + + // the following commands are not supported by SQL parser but are needed by users + + SET( + "SET", + // `SET` with operands can be parsed by SQL parser + // we keep `SET` with no operands here to print all properties + NO_OPERANDS), + + // the following commands will be supported by SQL parser in the future + // remove them once they're supported + + // FLINK-17396 + SHOW_MODULES( + "SHOW\\s+MODULES", + NO_OPERANDS), + + // FLINK-17111 + SHOW_VIEWS( + "SHOW\\s+VIEWS", + NO_OPERANDS), + + // the following commands are not supported by SQL parser but are needed by JDBC driver + // these should not be exposed to the user and should be used internally + + SHOW_CURRENT_CATALOG( + "SHOW\\s+CURRENT\\s+CATALOG", + NO_OPERANDS), + + SHOW_CURRENT_DATABASE( + "SHOW\\s+CURRENT\\s+DATABASE", + NO_OPERANDS); + + public final Pattern pattern; + public final Function> operandConverter; + + SqlCommand(String matchingRegex, Function> operandConverter) { + this.pattern = Pattern.compile(matchingRegex, DEFAULT_PATTERN_FLAGS); + this.operandConverter = operandConverter; + } + + SqlCommand() { + this.pattern = null; + this.operandConverter = null; + } + + @Override + public String toString() { + return super.toString().replace('_', ' '); + } + + boolean hasPattern() { + return pattern != null; + } + } + + /** + * Call of SQL command with operands and command type. + */ + public static class SqlCommandCall { + public final SqlCommand command; + public final String[] operands; + + public SqlCommandCall(SqlCommand command, String[] operands) { + this.command = command; + this.operands = operands; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SqlCommandCall that = (SqlCommandCall) o; + return command == that.command && Arrays.equals(operands, that.operands); + } + + @Override + public int hashCode() { + int result = Objects.hash(command); + result = 31 * result + Arrays.hashCode(operands); + return result; + } + + @Override + public String toString() { + return command + "(" + Arrays.toString(operands) + ")"; + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/YarnConfLoader.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/YarnConfLoader.java new file mode 100644 index 0000000000..a8caef920e --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/YarnConfLoader.java @@ -0,0 +1,51 @@ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils; + +import java.io.File; +import java.util.Iterator; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; + +public class YarnConfLoader { + public static YarnConfiguration getYarnConf(String yarnConfDir) { + YarnConfiguration yarnConf = new YarnConfiguration(); + try { + File dir = new File(yarnConfDir); + if (dir.exists() && dir.isDirectory()) { + File[] xmlFileList = new File(yarnConfDir).listFiles((dir1, name) -> { + if (name.endsWith(".xml")) { + return true; + } + return false; + }); + if (xmlFileList != null) { + for (File xmlFile : xmlFileList) { + yarnConf.addResource(xmlFile.toURI().toURL()); + } + } + } + } catch (Exception e) { + throw new RuntimeException(e); + } + haYarnConf(yarnConf); + return yarnConf; + } + + private static Configuration haYarnConf(Configuration yarnConf) { + Iterator> iterator = yarnConf.iterator(); + while (iterator.hasNext()) { + Map.Entry entry = iterator.next(); + String key = entry.getKey(); + String value = entry.getValue(); + if (key.startsWith("yarn.resourcemanager.hostname.")) { + String rm = key.substring("yarn.resourcemanager.hostname.".length()); + String addressKey = "yarn.resourcemanager.address." + rm; + if (yarnConf.get(addressKey) == null) { + yarnConf.set(addressKey, value + ":" + YarnConfiguration.DEFAULT_RM_PORT); + } + } + } + return yarnConf; + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/ExecutorInitException.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/ExecutorInitException.java new file mode 100644 index 0000000000..1b2e71d5a9 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/ExecutorInitException.java @@ -0,0 +1,47 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.exception; + + +import com.webank.wedatasphere.linkis.common.exception.ErrorException; + + +public class ExecutorInitException extends ErrorException { + + public static final int ERROR_CODE = 16021; + + private static final long serialVersionUID = 1L; + + public ExecutorInitException(int errCode, String desc) { + super(errCode, desc); + } + + public ExecutorInitException(String desc) { + super(20001, desc); + } + + public ExecutorInitException(Exception e) { + super(20001, e.getMessage()); + } + + public ExecutorInitException() { + super(20001, "argument illegal"); + } + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/FlinkInitFailedException.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/FlinkInitFailedException.java new file mode 100644 index 0000000000..c8fe2b7395 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/FlinkInitFailedException.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.exception; + + +import com.webank.wedatasphere.linkis.common.exception.ErrorException; + +public class FlinkInitFailedException extends ErrorException { + + public static final int ERROR_CODE = 16020; + + private static final long serialVersionUID = 1L; + + public FlinkInitFailedException(String msg) { + super(ERROR_CODE, msg); + } + + public FlinkInitFailedException(String msg, Throwable cause) { + super(ERROR_CODE, msg); + initCause(cause); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/JobExecutionException.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/JobExecutionException.java new file mode 100644 index 0000000000..832ffab5f2 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/JobExecutionException.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.exception; + + +import com.webank.wedatasphere.linkis.common.exception.ErrorException; + +public class JobExecutionException extends ErrorException { + + private static final long serialVersionUID = 1L; + + public static final int ERROR_CODE = 16023; + + public JobExecutionException(String message) { + super(ERROR_CODE, message); + } + + public JobExecutionException(String message, Throwable e) { + super(ERROR_CODE, message); + this.initCause(e); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlExecutionException.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlExecutionException.java new file mode 100644 index 0000000000..7c386e5100 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlExecutionException.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.exception; + + +import com.webank.wedatasphere.linkis.common.exception.ErrorException; + +public class SqlExecutionException extends ErrorException { + + public static final int ERROR_CODE = 16022; + + private static final long serialVersionUID = 1L; + + public SqlExecutionException(String message) { + super(ERROR_CODE, message); + } + + public SqlExecutionException(String message, Throwable e) { + super(ERROR_CODE, message); + initCause(e); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlParseException.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlParseException.java new file mode 100644 index 0000000000..e0916e258c --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlParseException.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.engineconnplugin.flink.exception; + + +import com.webank.wedatasphere.linkis.common.exception.ErrorException; + +public class SqlParseException extends ErrorException { + + public static final int ERROR_CODE = 16021; + + private static final long serialVersionUID = 1L; + + public SqlParseException(String message) { + super(ERROR_CODE, message); + } + + public SqlParseException(String message, Throwable e) { + super(ERROR_CODE, message); + this.initCause(e); + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/RetryUtil.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/RetryUtil.java new file mode 100644 index 0000000000..51c1f0eb59 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/RetryUtil.java @@ -0,0 +1,54 @@ +package com.webank.wedatasphere.linkis.engineconnplugin.flink.util; + +import com.github.rholder.retry.RetryException; +import com.github.rholder.retry.Retryer; +import com.github.rholder.retry.RetryerBuilder; +import com.github.rholder.retry.StopStrategies; +import com.github.rholder.retry.WaitStrategies; +import com.google.common.base.Predicate; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 基于guava-retry的重试工具类 + */ +public class RetryUtil { + + private static final Logger logger = LoggerFactory.getLogger(RetryUtil.class); + + /** + * @param task 要重试执行得任务 + * @param predicate 符合预期结果需要重试 + * @param fixedWaitTime 本次重试与上次重试之间的固定间隔时长 + * @param maxEachExecuTime 一次重试的最大执行的时间 + * @param attemptNumber 重试次数 + */ + + public static T retry(Callable task, Predicate predicate, long fixedWaitTime, long maxEachExecuTime, + TimeUnit timeUnit, int attemptNumber) { + Retryer retryer = RetryerBuilder + .newBuilder() + // 抛出runtime异常、checked异常时都会重试,但是抛出error不会重试。 + .retryIfException() + // 对执行结果的预期。符合预期就重试 + .retryIfResult(predicate) + // 每次重试固定等待fixedWaitTime时间 + .withWaitStrategy(WaitStrategies.fixedWait(fixedWaitTime, timeUnit)) + // 尝试次数 + .withStopStrategy(StopStrategies.stopAfterAttempt(attemptNumber)) + .build(); + T t = null; + try { + t = retryer.call(task); + } catch (ExecutionException e) { + logger.error("", e); + } catch (RetryException e) { + logger.error("", e); + } + return t; + } + +} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/linkis-engineconn.properties b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/linkis-engineconn.properties new file mode 100644 index 0000000000..27d74c8c34 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/linkis-engineconn.properties @@ -0,0 +1,27 @@ +# +# Copyright 2019 WeBank +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + + +wds.linkis.server.version=v1 + + +wds.linkis.engineconn.debug.enable=true + +#wds.linkis.keytab.enable=true + +wds.linkis.engineconn.plugin.default.class=com.webank.wedatasphere.linkis.engineconnplugin.flink.FlinkEngineConnPlugin + +wds.linkis.engine.connector.hooks=com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.ComputationEngineConnHook,com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.JarUdfEngineHook diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/log4j2-engineconn.xml b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/log4j2-engineconn.xml new file mode 100644 index 0000000000..b78b215127 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/log4j2-engineconn.xml @@ -0,0 +1,64 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/FlinkEngineConnPlugin.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/FlinkEngineConnPlugin.scala new file mode 100644 index 0000000000..63517afd56 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/FlinkEngineConnPlugin.scala @@ -0,0 +1,77 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.factory.FlinkEngineConnFactory +import com.webank.wedatasphere.linkis.engineconnplugin.flink.launch.FlinkEngineConnLaunchBuilder +import com.webank.wedatasphere.linkis.engineconnplugin.flink.resource.FlinkEngineConnResourceFactory +import com.webank.wedatasphere.linkis.manager.engineplugin.common.EngineConnPlugin +import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.EngineConnFactory +import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.EngineConnLaunchBuilder +import com.webank.wedatasphere.linkis.manager.engineplugin.common.resource.EngineResourceFactory +import com.webank.wedatasphere.linkis.manager.label.entity.Label + +/** + * + */ +class FlinkEngineConnPlugin extends EngineConnPlugin { + + private var engineResourceFactory: EngineResourceFactory = _ + private var engineConnLaunchBuilder: EngineConnLaunchBuilder = _ + private var engineConnFactory: EngineConnFactory = _ + + private val EP_CONTEXT_CONSTRUCTOR_LOCK = new Object() + + + override def init(params: java.util.Map[String, Any]): Unit = { + //do noting +// engineResourceFactory = new FlinkEngineConnResourceFactory +// engineConnLaunchBuilder = new FlinkEngineConnLaunchBuilder +// engineConnFactory = new FlinkEngineConnFactory + } + + override def getEngineResourceFactory: EngineResourceFactory = { + EP_CONTEXT_CONSTRUCTOR_LOCK.synchronized{ + if(null == engineResourceFactory){ + engineResourceFactory = new FlinkEngineConnResourceFactory + } + engineResourceFactory + } + } + + override def getEngineConnLaunchBuilder: EngineConnLaunchBuilder = { + EP_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + // todo check + if (null == engineConnLaunchBuilder) { + engineConnLaunchBuilder = new FlinkEngineConnLaunchBuilder() + } + engineConnLaunchBuilder + } + } + + override def getEngineConnFactory: EngineConnFactory = { + EP_CONTEXT_CONSTRUCTOR_LOCK.synchronized { + if (null == engineConnFactory) { + engineConnFactory = new FlinkEngineConnFactory + } + engineConnFactory + } + } + + override def getDefaultLabels: java.util.List[Label[_]] = new java.util.ArrayList[Label[_]] +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkEnvConfiguration.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkEnvConfiguration.scala new file mode 100644 index 0000000000..488cfe0d76 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkEnvConfiguration.scala @@ -0,0 +1,58 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.config + +import com.webank.wedatasphere.linkis.common.conf.{CommonVars, TimeType} +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.entries.ExecutionEntry +import com.webank.wedatasphere.linkis.manager.label.entity.cluster.EnvLabel + +/** + * + */ +object FlinkEnvConfiguration { + + val FLINK_HOME_ENV = "FLINK_HOME" + val FLINK_CONF_DIR_ENV = "FLINK_CONF_DIR" + val FLINK_HOME = CommonVars("flink.home", CommonVars(FLINK_HOME_ENV, "/appcom/Install/flink").getValue) + val FLINK_CONF_DIR = CommonVars("flink.conf.dir", CommonVars(FLINK_CONF_DIR_ENV, "/appcom/config/flink-config").getValue) + val FLINK_DIST_JAR_PATH = CommonVars("flink.dist.jar.path", "/appcom/Install/flink/lib/flink-dist_2.11-1.11.1.jar") + val FLINK_LIB_REMOTE_PATH = CommonVars("flink.lib.path", "") + val FLINK_USER_LIB_REMOTE_PATH = CommonVars("flink.user.lib.path", "", "The hdfs lib path of each user in Flink EngineConn.") + val FLINK_LIB_LOCAL_PATH = CommonVars("flink.local.lib.path", "/appcom/Install/flink/lib", "The local lib path of Flink EngineConn.") + val FLINK_USER_LIB_LOCAL_PATH = CommonVars("flink.user.local.lib.path", "/appcom/Install/flink/lib", "The local lib path of each user in Flink EngineConn.") + val FLINK_SHIP_DIRECTORIES = CommonVars("flink.yarn.ship-directories", "") + + val FLINK_ENV_TYPE = CommonVars("flink.app.env.type", EnvLabel.DEV) + + val FLINK_SAVE_POINT_PATH = CommonVars("flink.app.savePointPath", "") + val FLINK_APP_ALLOW_NON_RESTORED_STATUS = CommonVars("flink.app.allowNonRestoredStatus", "false") + val FLINK_SQL_PLANNER = CommonVars("flink.sql.planner", ExecutionEntry.EXECUTION_PLANNER_VALUE_BLINK) + val FLINK_SQL_EXECUTION_TYPE = CommonVars("flink.sql.executionType", ExecutionEntry.EXECUTION_TYPE_VALUE_BATCH) + + val FLINK_SQL_DEV_SELECT_MAX_LINES = CommonVars("flink.dev.sql.select.lines.max", 500) + val FLINK_SQL_DEV_RESULT_MAX_WAIT_TIME = CommonVars("flink.dev.sql.result.wait.time.max", new TimeType("1m")) + + val FLINK_APPLICATION_ARGS = CommonVars("flink.app.args", "") + val FLINK_APPLICATION_MAIN_CLASS = CommonVars("flink.app.main.class", "") + val FLINK_APPLICATION_MAIN_CLASS_JAR = CommonVars("flink.app.main.class.jar", "") + + val FLINK_CLIENT_REQUEST_TIMEOUT = CommonVars("flink.client.request.timeout", new TimeType("30s")) + val FLINK_APPLICATION_STATUS_FETCH_INTERVAL = CommonVars("flink.app.fetch.status.interval", new TimeType("5s")) + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkResourceConfiguration.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkResourceConfiguration.scala new file mode 100644 index 0000000000..69cf53b672 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkResourceConfiguration.scala @@ -0,0 +1,41 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.config + +import com.webank.wedatasphere.linkis.common.conf.CommonVars + +/** + * + */ +object FlinkResourceConfiguration { + + val LINKIS_FLINK_CLIENT_MEMORY = CommonVars[Int]("flink.client.memory", 4) //单位为G + val LINKIS_FLINK_CLIENT_CORES = 1 //Fixed to 1(固定为1) CommonVars[Int]("wds.linkis.driver.cores", 1) + + + val LINKIS_FLINK_JOB_MANAGER_MEMORY = CommonVars[Int]("flink.jobmanager.memory", 2) //单位为G + val LINKIS_FLINK_TASK_MANAGER_MEMORY = CommonVars[Int]("flink.taskmanager.memory", 4) //单位为G + val LINKIS_FLINK_TASK_SLOTS = CommonVars[Int]("flink.taskmanager.numberOfTaskSlots", 2) + val LINKIS_FLINK_TASK_MANAGER_CPU_CORES = CommonVars[Int]("flink.taskmanager.cpu.cores", 2) + val LINKIS_FLINK_CONTAINERS = CommonVars[Int]("flink.container.num", 2) + val LINKIS_QUEUE_NAME = CommonVars[String]("wds.linkis.rm.yarnqueue", "default") + + + val FLINK_APP_DEFAULT_PARALLELISM = CommonVars("wds.linkis.engineconn.flink.app.parallelism", 4) + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/EnvironmentContext.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/EnvironmentContext.scala new file mode 100644 index 0000000000..fe70c30b4b --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/EnvironmentContext.scala @@ -0,0 +1,106 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.context + +import java.net.URL +import java.util +import java.util.Objects + +import com.google.common.collect.Lists +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.factory.LinkisYarnClusterClientFactory +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment +import org.apache.commons.lang.StringUtils +import org.apache.flink.client.deployment.{ClusterClientServiceLoader, DefaultClusterClientServiceLoader} +import org.apache.flink.configuration.{Configuration, DeploymentOptionsInternal, GlobalConfiguration} +import org.apache.flink.yarn.configuration.{YarnConfigOptions, YarnDeploymentTarget} + +/** + * + */ +class EnvironmentContext(defaultEnv: Environment, + yarnConfDir: String, + flinkConfDir: String, + flinkHome: String, + distJarPath: String, + flinkLibRemotePath: String, + dependencies: util.List[URL]) { + + private var providedLibDirs: util.List[String] = _ + private var shipDirs: util.List[String] = _ + + private var flinkConfig: Configuration = _ + private var clusterClientServiceLoader: ClusterClientServiceLoader = _ + + private var deploymentTarget: YarnDeploymentTarget = YarnDeploymentTarget.PER_JOB + + def this(defaultEnv: Environment, systemConfiguration: Configuration, yarnConfDir: String, flinkConfDir: String, + flinkHome: String, distJarPath: String, flinkLibRemotePath: String, providedLibDirsArray: Array[String], + shipDirsArray: Array[String], dependencies: util.List[URL]) { + this(defaultEnv, yarnConfDir, flinkConfDir, flinkHome, distJarPath, flinkLibRemotePath, dependencies) + //远程资源目录 + this.providedLibDirs = Lists.newArrayList(providedLibDirsArray.filter(StringUtils.isNotBlank): _*) + //本地资源目录 + this.shipDirs = Lists.newArrayList(shipDirsArray.filter(StringUtils.isNotBlank): _*) + //加载系统级别配置 + this.flinkConfig = GlobalConfiguration.loadConfiguration(this.flinkConfDir) + if (null != systemConfiguration) this.flinkConfig.addAll(systemConfiguration) + //设置 flink conf目录 + this.flinkConfig.set(DeploymentOptionsInternal.CONF_DIR, this.flinkConfDir) + //设置 yarn conf目录 + this.flinkConfig.set(LinkisYarnClusterClientFactory.YARN_CONFIG_DIR, this.yarnConfDir) + //设置 flink dist jar + this.flinkConfig.set(YarnConfigOptions.FLINK_DIST_JAR, distJarPath) + clusterClientServiceLoader = new DefaultClusterClientServiceLoader + } + + def setDeploymentTarget(deploymentTarget: YarnDeploymentTarget): Unit = this.deploymentTarget = deploymentTarget + + def getDeploymentTarget: YarnDeploymentTarget = deploymentTarget + + def getProvidedLibDirs: util.List[String] = providedLibDirs + + def getShipDirs: util.List[String] = shipDirs + + def getYarnConfDir: String = yarnConfDir + + def getFlinkConfDir: String = flinkConfDir + + def getFlinkHome: String = flinkHome + + def getFlinkLibRemotePath: String = flinkLibRemotePath + + def getFlinkConfig: Configuration = flinkConfig + + def getDefaultEnv: Environment = defaultEnv + + def getDependencies: util.List[URL] = dependencies + + def getClusterClientServiceLoader: ClusterClientServiceLoader = clusterClientServiceLoader + + override def equals(o: Any): Boolean = o match { + case context: EnvironmentContext => + if(this eq context) return true + Objects.equals(defaultEnv, context.getDefaultEnv) && + Objects.equals(dependencies, context.getDependencies) && + Objects.equals(flinkConfig, context.flinkConfig) && + Objects.equals(clusterClientServiceLoader, context.clusterClientServiceLoader) + case _ => false + } + + override def hashCode: Int = Objects.hash(defaultEnv, dependencies, flinkConfig, clusterClientServiceLoader) +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/FlinkEngineConnContext.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/FlinkEngineConnContext.scala new file mode 100644 index 0000000000..049c1fc14d --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/FlinkEngineConnContext.scala @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.webank.wedatasphere.linkis.engineconnplugin.flink.context + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext + +/** + * + */ +class FlinkEngineConnContext(environmentContext: EnvironmentContext) { + + private var executionContext: ExecutionContext = _ + + def getEnvironmentContext: EnvironmentContext = environmentContext + + def getExecutionContext: ExecutionContext = executionContext + + def setExecutionContext(executionContext: ExecutionContext): Unit = this.executionContext = executionContext + + def newExecutionContextBuilder(environment: Environment): ExecutionContext.Builder = + ExecutionContext.builder(environmentContext.getDefaultEnv, environment, environmentContext.getDependencies, + environmentContext.getFlinkConfig, environmentContext.getClusterClientServiceLoader) + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkApplicationJobExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkApplicationJobExecutor.scala new file mode 100644 index 0000000000..50ea10c19a --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkApplicationJobExecutor.scala @@ -0,0 +1,88 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.executor + +import java.util.concurrent.{Future, TimeUnit} + +import com.webank.wedatasphere.linkis.common.utils.Utils +import com.webank.wedatasphere.linkis.engineconn.once.executor.{ManageableOnceExecutor, OnceExecutorExecutionContext} +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment.{ClusterDescriptorAdapterFactory, YarnApplicationClusterDescriptorAdapter} +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration._ +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.ExecutorInitException + +import scala.collection.convert.WrapAsScala._ +import scala.concurrent.duration.Duration + +/** + * + */ +class FlinkApplicationJobExecutor(id: Long, + override protected val flinkEngineConnContext: FlinkEngineConnContext) + extends ManageableOnceExecutor with FlinkJobExecutor { + + private var clusterDescriptor: YarnApplicationClusterDescriptorAdapter = _ + private var daemonThread: Future[_] = _ + + protected def submit(onceExecutorExecutionContext: OnceExecutorExecutionContext): Unit = { + ClusterDescriptorAdapterFactory.create(flinkEngineConnContext.getExecutionContext, null) match { + case adapter: YarnApplicationClusterDescriptorAdapter => clusterDescriptor = adapter + case _ => throw new ExecutorInitException("Not support ClusterDescriptorAdapter for flink application.") + } + val options = onceExecutorExecutionContext.getOnceExecutorContent.getJobContent.map { + case (k, v: String) => k -> v + case (k, v) if v != null => k -> v.toString + case (k, _) => k -> null + }.toMap + val programArguments = FLINK_APPLICATION_ARGS.getValue(options).split(" ") + val mainClass = FLINK_APPLICATION_MAIN_CLASS.getValue(options) + clusterDescriptor.deployCluster(programArguments, mainClass) + if (null == clusterDescriptor.getJobId || null == clusterDescriptor.getClusterID) + throw new ExecutorInitException("The app " + mainClass + " start failed, no result was returned.") + setJobID(clusterDescriptor.getJobId.toHexString) + setApplicationId(clusterDescriptor.getClusterID.toString) + setApplicationURL(clusterDescriptor.getWebInterfaceUrl) + } + + override def getId: String = "flinkApp_"+ id + + override def close(): Unit = { + if(daemonThread != null) daemonThread.cancel(true) + if(clusterDescriptor != null) { + clusterDescriptor.cancelJob() + clusterDescriptor.close() + } + flinkEngineConnContext.getExecutionContext.getClusterClientFactory.close() + super.close() + } + + override protected def waitToRunning(): Unit = { + Utils.waitUntil(() => clusterDescriptor.initJobId(), Duration.Inf) + daemonThread = Utils.defaultScheduler.scheduleAtFixedRate(new Runnable { + override def run(): Unit = { + val jobStatus = clusterDescriptor.getJobStatus + info(s"The jobStatus of $getJobID is $jobStatus.") + if(jobStatus.isGloballyTerminalState) + tryFailed() + else if(jobStatus.isTerminalState) tryShutdown() + } + }, FLINK_APPLICATION_STATUS_FETCH_INTERVAL.getValue.toLong, FLINK_APPLICATION_STATUS_FETCH_INTERVAL.getValue.toLong, TimeUnit.MILLISECONDS) + } + + override def supportCallBackLogs(): Boolean = true +} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkJobExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkJobExecutor.scala new file mode 100644 index 0000000000..cbb28fea0c --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkJobExecutor.scala @@ -0,0 +1,133 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.executor + +import java.util + +import com.webank.wedatasphere.linkis.common.io.resultset.ResultSetWriter +import com.webank.wedatasphere.linkis.common.io.{MetaData, Record} +import com.webank.wedatasphere.linkis.common.utils.{OverloadUtils, Utils} +import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext +import com.webank.wedatasphere.linkis.engineconn.executor.entity.{LabelExecutor, ResourceExecutor, YarnExecutor} +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkResourceConfiguration +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkResourceConfiguration.LINKIS_FLINK_CLIENT_CORES +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException +import com.webank.wedatasphere.linkis.engineconnplugin.flink.util.FlinkValueFormatUtil +import com.webank.wedatasphere.linkis.manager.common.entity.resource._ +import com.webank.wedatasphere.linkis.manager.common.utils.ResourceUtils +import com.webank.wedatasphere.linkis.manager.label.entity.Label +import com.webank.wedatasphere.linkis.storage.domain.{Column, DataType} +import com.webank.wedatasphere.linkis.storage.resultset.ResultSetFactory +import com.webank.wedatasphere.linkis.storage.resultset.table.{TableMetaData, TableRecord} +import org.apache.commons.io.IOUtils +import org.apache.flink.configuration.{CoreOptions, JobManagerOptions, TaskManagerOptions} +import org.apache.flink.types.Row +import org.apache.flink.yarn.configuration.YarnConfigOptions + +/** + * + */ +trait FlinkJobExecutor extends YarnExecutor with LabelExecutor with ResourceExecutor { + + private var jobID: String = _ + private var applicationId: String = _ + private var applicationURL: String = _ + private var yarnMode: String = "Client" + private var queue: String = _ + + private var executorLabels: util.List[Label[_]] = new util.ArrayList[Label[_]] + + def getJobID: String = jobID + + protected def setJobID(jobID: String): Unit = { + this.jobID = jobID + } + + override def getApplicationId: String = applicationId + def setApplicationId(applicationId: String): Unit = this.applicationId = applicationId + + override def getApplicationURL: String = applicationURL + def setApplicationURL(applicationURL: String): Unit = this.applicationURL = applicationURL + + override def getYarnMode: String = yarnMode + def setYarnMode(yarnMode: String): Unit = this.yarnMode = yarnMode + + override def getQueue: String = queue + def setQueue(queue: String): Unit = this.queue = queue + + override def getExecutorLabels(): util.List[Label[_]] = executorLabels + + override def setExecutorLabels(labels: util.List[Label[_]]): Unit = this.executorLabels = labels + + override def requestExpectedResource(expectedResource: NodeResource): NodeResource = throw new JobExecutionException("Not support method for requestExpectedResource.") + + protected val flinkEngineConnContext: FlinkEngineConnContext + + queue = flinkEngineConnContext.getEnvironmentContext.getFlinkConfig.get(YarnConfigOptions.APPLICATION_QUEUE) + + override def getCurrentNodeResource(): NodeResource = { + val flinkConfig = flinkEngineConnContext.getEnvironmentContext.getFlinkConfig + val jobManagerMemory = flinkConfig.get(JobManagerOptions.TOTAL_PROCESS_MEMORY).getBytes + val taskManagerMemory = flinkConfig.get(TaskManagerOptions.TOTAL_PROCESS_MEMORY).getBytes + val parallelism = flinkConfig.get(CoreOptions.DEFAULT_PARALLELISM) + val numOfTaskSlots =flinkConfig.get(TaskManagerOptions.NUM_TASK_SLOTS) + val containers = Math.round(parallelism * 1.0f / numOfTaskSlots) + val yarnMemory = taskManagerMemory * containers + jobManagerMemory + val yarnCores = FlinkResourceConfiguration.LINKIS_FLINK_TASK_MANAGER_CPU_CORES.getValue * containers + 1 + val resource = new DriverAndYarnResource( + new LoadInstanceResource(OverloadUtils.getProcessMaxMemory, + LINKIS_FLINK_CLIENT_CORES, + 1), + new YarnResource(yarnMemory, yarnCores, 0, queue) + ) + val engineResource = new CommonNodeResource + engineResource.setUsedResource(resource) + engineResource.setResourceType(ResourceUtils.getResourceTypeByResource(resource)) + engineResource + } + + def supportCallBackLogs(): Boolean = true +} + +object FlinkJobExecutor { + + import scala.collection.JavaConversions._ + + def writeResultSet(resultSet: ResultSet, resultSetWriter: ResultSetWriter[_ <: MetaData, _ <: Record]): Unit = { + val columns = resultSet.getColumns.map(columnInfo => Column(columnInfo.getName, DataType.toDataType(columnInfo.getType), null)).toArray + resultSetWriter.addMetaData(new TableMetaData(columns)) + resultSet.getData match { + case data: util.List[Row] => data.foreach { row => + val record = (0 until row.getArity).map(row.getField).map(FlinkValueFormatUtil.formatValue).toArray + resultSetWriter.addRecord(new TableRecord(record)) + } + case _ => + } + } + + def writeAndSendResultSet(resultSet: ResultSet, engineExecutionContext: EngineExecutionContext): Unit = { + val resultSetWriter = engineExecutionContext.createResultSetWriter(ResultSetFactory.TABLE_TYPE) + Utils.tryFinally{ + writeResultSet(resultSet, resultSetWriter) + engineExecutionContext.sendResultSet(resultSetWriter) + } (IOUtils.closeQuietly(resultSetWriter)) + } + +} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkSQLJobExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkSQLJobExecutor.scala new file mode 100644 index 0000000000..1f87ff44a2 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkSQLJobExecutor.scala @@ -0,0 +1,226 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.executor + +import java.io.Closeable +import java.util +import java.util.concurrent.TimeUnit + +import com.webank.wedatasphere.linkis.common.utils.{ByteTimeUtils, Logging, Utils} +import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.{ComputationExecutor, EngineExecutionContext} +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.{AbstractJobOperation, JobOperation, OperationFactory} +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils.SqlCommandParser +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlParseException +import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.RowsType.RowsType +import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.{FlinkStatusListener, FlinkStreamingResultSetListener} +import com.webank.wedatasphere.linkis.manager.label.entity.cluster.EnvLabel +import com.webank.wedatasphere.linkis.protocol.engine.JobProgressInfo +import com.webank.wedatasphere.linkis.scheduler.executer.{ErrorExecuteResponse, ExecuteResponse, SuccessExecuteResponse} +import com.webank.wedatasphere.linkis.storage.resultset.ResultSetFactory +import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelMetadataQueryBase} +import org.apache.flink.api.common.JobStatus._ +import org.apache.flink.table.planner.plan.metadata.FlinkDefaultRelMetadataProvider + +import scala.collection.JavaConversions._ + +/** + * + */ +class FlinkSQLJobExecutor(id: Long, + override protected val flinkEngineConnContext: FlinkEngineConnContext) extends ComputationExecutor with FlinkJobExecutor { + + private var operation: JobOperation = _ + + override def executeLine(engineExecutionContext: EngineExecutionContext, code: String): ExecuteResponse = { + val callOpt = SqlCommandParser.parse(code.trim, true) + val callSQL = if (!callOpt.isPresent) throw new SqlParseException("Unknown statement: " + code) + else callOpt.get + RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) + val operation = OperationFactory.createOperation(callSQL, flinkEngineConnContext) + operation match { + case jobOperation: JobOperation => + this.operation = jobOperation + jobOperation.addFlinkListener(new FlinkSQLStatusListener(jobOperation, engineExecutionContext)) + if(getEnvLabel.getEnvType == EnvLabel.DEV && callSQL.command == SqlCommandParser.SqlCommand.SELECT) { + jobOperation.addFlinkListener(new FlinkSQLStreamingResultSetListener(jobOperation, engineExecutionContext)) + val properties: util.Map[String, String] = engineExecutionContext.getProperties.map { + case (k, v: String) => (k, v) + case (k, v) if v != null => (k, v.toString) + case (k, _) => (k, null) + } + jobOperation.addFlinkListener(new DevFlinkSQLStreamingListener(jobOperation, properties)) + } + case _ => + } + val resultSet = operation.execute + resultSet.getResultKind match { + case ResultKind.SUCCESS => + new SuccessExecuteResponse + case ResultKind.SUCCESS_WITH_CONTENT if !operation.isInstanceOf[JobOperation] => + FlinkJobExecutor.writeAndSendResultSet(resultSet, engineExecutionContext) + new SuccessExecuteResponse + case _ => + operation match { + case jobOperation: AbstractJobOperation => + val jobInfo = jobOperation.transformToJobInfo(resultSet) + setJobID(jobInfo.getJobId.toHexString) + setApplicationId(jobInfo.getApplicationId) + setApplicationURL(jobInfo.getWebInterfaceUrl) + setYarnMode("client") + jobOperation.getFlinkStatusListeners.get(0) match { + case listener: FlinkSQLStatusListener => listener.waitForCompleted() + case _ => + } + case jobOperation: JobOperation => + jobOperation.getFlinkListeners.find(_.isInstanceOf[FlinkSQLStatusListener]).foreach { case listener: FlinkSQLStatusListener => + listener.waitForCompleted() + } + } + } + new SuccessExecuteResponse + } + + override def executeCompletely(engineExecutorContext: EngineExecutionContext, code: String, completedLine: String): ExecuteResponse = { + val newcode = completedLine + code + info("newcode is " + newcode) + executeLine(engineExecutorContext, newcode) + } + + //TODO wait for completed. + override def progress(): Float = if(operation == null) 0 else operation.getJobStatus match { + case jobState if jobState.isGloballyTerminalState => 1 + case RUNNING => 0.5f + case _ => 0 + } + + override def getProgressInfo: Array[JobProgressInfo] = Array.empty + + private var envLabel: EnvLabel = _ + + def getEnvLabel: EnvLabel = { + if(envLabel == null) getExecutorLabels().foreach{ + case l: EnvLabel => envLabel = l + case _ => + } + envLabel + } + + override def getId: String = "flinkSQL_"+ id + + override def close(): Unit = { + if(operation != null) { + operation.cancelJob() + } + flinkEngineConnContext.getExecutionContext.createClusterDescriptor().close() + flinkEngineConnContext.getExecutionContext.getClusterClientFactory.close() + super.close() + } +} + +class FlinkSQLStatusListener(jobOperation: JobOperation, engineExecutionContext: EngineExecutionContext) extends FlinkStatusListener { + + private var resp: ExecuteResponse = _ + private val startTime = System.currentTimeMillis + + override def onSuccess(rows: Int, rowsType: RowsType): Unit = { + engineExecutionContext.appendStdout(s"Time taken: ${ByteTimeUtils.msDurationToString(System.currentTimeMillis - startTime)}, $rowsType $rows row(s).") + Utils.tryCatch{ + FlinkJobExecutor.writeAndSendResultSet(jobOperation.getJobResult.get(), engineExecutionContext) + resp = new SuccessExecuteResponse + }{ e => + resp = ErrorExecuteResponse("Fail to run statement",e) + } + synchronized(notify()) + } + + override def onFailed(message: String, t: Throwable): Unit = { + resp = ErrorExecuteResponse(message, t) + synchronized(notify()) + } + + def getResponse: ExecuteResponse = resp + + def waitForCompleted(maxWaitTime: Long): Unit = synchronized { + if(maxWaitTime < 0) wait() else wait(maxWaitTime) + } + + def waitForCompleted(): Unit = waitForCompleted(-1) +} + +class FlinkSQLStreamingResultSetListener(jobOperation: JobOperation, + engineExecutionContext: EngineExecutionContext) + extends FlinkStreamingResultSetListener with Closeable with Logging { + + private val resultSetWriter = engineExecutionContext.createResultSetWriter(ResultSetFactory.TABLE_TYPE) + + override def onResultSetPulled(rows: Int): Unit = { + info(s"$rows resultSets has pulled.") + FlinkJobExecutor.writeResultSet(jobOperation.getJobResult.get(), resultSetWriter) + } + + override def close(): Unit = engineExecutionContext.sendResultSet(resultSetWriter) +} + +class DevFlinkSQLStreamingListener(jobOperation: JobOperation, + maxWrittenLines: Int, + maxWaitForResultTime: Long) extends FlinkStreamingResultSetListener with Logging { + + def this(jobOperation: JobOperation) = + this(jobOperation, + FlinkEnvConfiguration.FLINK_SQL_DEV_SELECT_MAX_LINES.getValue, + FlinkEnvConfiguration.FLINK_SQL_DEV_RESULT_MAX_WAIT_TIME.getValue.toLong + ) + + def this(jobOperation: JobOperation, properties: util.Map[String, String]) = + this(jobOperation, + FlinkEnvConfiguration.FLINK_SQL_DEV_SELECT_MAX_LINES.getValue(properties), + FlinkEnvConfiguration.FLINK_SQL_DEV_RESULT_MAX_WAIT_TIME.getValue(properties).toLong + ) + + private var lastPulledTime = System.currentTimeMillis + private var writtenLines = 0 + + override def onResultSetPulled(rows: Int): Unit = { + lastPulledTime = System.currentTimeMillis + writtenLines += rows + if(writtenLines >= maxWrittenLines) { + warn(s"The returned resultSet reached max lines $writtenLines, now kill the job automatic. Notice: only the dev environment will touch off the automatic kill.") + stopJobOperation() + } + } + + private val future = Utils.defaultScheduler.scheduleAtFixedRate(new Runnable { + override def run(): Unit = if(System.currentTimeMillis - lastPulledTime > maxWaitForResultTime) { + warn(s"Job killed since reached the max time ${ByteTimeUtils.msDurationToString(maxWaitForResultTime)} of waiting for resultSet. Notice: only the dev environment will touch off the automatic kill.") + stopJobOperation() + } + }, maxWaitForResultTime, maxWaitForResultTime, TimeUnit.MILLISECONDS) + + def stopJobOperation(): Unit = { + Utils.tryFinally(jobOperation.cancelJob()) { + jobOperation.getFlinkListeners.foreach { + case listener: FlinkStreamingResultSetListener with Closeable => + listener.close() + } + future.cancel(false) + } + } +} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkApplicationExecutorFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkApplicationExecutorFactory.scala new file mode 100644 index 0000000000..ce8403bbd8 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkApplicationExecutorFactory.scala @@ -0,0 +1,44 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.factory + +import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext +import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn +import com.webank.wedatasphere.linkis.engineconn.once.executor.OnceExecutor +import com.webank.wedatasphere.linkis.engineconn.once.executor.creation.OnceExecutorFactory +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext +import com.webank.wedatasphere.linkis.engineconnplugin.flink.executor.FlinkApplicationJobExecutor +import com.webank.wedatasphere.linkis.manager.label.entity.Label +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType + +/** + * + */ +class FlinkApplicationExecutorFactory extends OnceExecutorFactory { + + override protected def newExecutor(id: Int, + engineCreationContext: EngineCreationContext, + engineConn: EngineConn, + labels: Array[Label[_]]): OnceExecutor = engineConn.getEngineConnSession match { + case context: FlinkEngineConnContext => + new FlinkApplicationJobExecutor(id, context) + } + + override protected def getRunType: RunType = RunType.JAR +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkEngineConnFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkEngineConnFactory.scala new file mode 100644 index 0000000000..712ddb79ff --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkEngineConnFactory.scala @@ -0,0 +1,186 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.factory + +import java.util +import java.util.Collections + +import com.google.common.collect.Lists +import com.webank.wedatasphere.linkis.common.utils.Logging +import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.entries.ExecutionEntry +import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration._ +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkResourceConfiguration._ +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.{EnvironmentContext, FlinkEngineConnContext} +import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.FlinkInitFailedException +import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.EnvConfiguration +import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.{ExecutorFactory, MultiExecutorEngineConnFactory} +import com.webank.wedatasphere.linkis.manager.label.entity.Label +import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType.EngineType +import com.webank.wedatasphere.linkis.manager.label.entity.engine._ +import org.apache.commons.lang.StringUtils +import org.apache.flink.configuration._ +import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings +import org.apache.flink.yarn.configuration.{YarnConfigOptions, YarnDeploymentTarget} + +import scala.collection.convert.decorateAsScala._ + +/** + * + */ +class FlinkEngineConnFactory extends MultiExecutorEngineConnFactory with Logging { + + override protected def createEngineConnSession(engineCreationContext: EngineCreationContext): Any = { + val options = engineCreationContext.getOptions + val environmentContext = createEnvironmentContext(engineCreationContext) + val flinkEngineConnContext = createFlinkEngineConnContext(environmentContext) + val executionContext = createExecutionContext(options, environmentContext) + flinkEngineConnContext.setExecutionContext(executionContext) + flinkEngineConnContext + } + + protected def createEnvironmentContext(engineCreationContext: EngineCreationContext): EnvironmentContext = { + val options = engineCreationContext.getOptions + val defaultEnv = Environment.parse(this.getClass.getClassLoader.getResource("flink-sql-defaults.yaml")) + val hadoopConfDir = EnvConfiguration.HADOOP_CONF_DIR.getValue(options) + val flinkHome = FLINK_HOME.getValue(options) + val flinkConfDir = FLINK_CONF_DIR.getValue(options) + val flinkLibRemotePath = FLINK_LIB_REMOTE_PATH.getValue(options) + val flinkDistJarPath = FLINK_DIST_JAR_PATH.getValue(options) + val providedLibDirsArray = FLINK_LIB_LOCAL_PATH.getValue(options).split(",") + val shipDirsArray = FLINK_SHIP_DIRECTORIES.getValue(options).split(",") + val context = new EnvironmentContext(defaultEnv, new Configuration, hadoopConfDir, flinkConfDir, flinkHome, + flinkDistJarPath, flinkLibRemotePath, providedLibDirsArray, shipDirsArray, null) + //第一步: 环境级别配置 + val jobName = options.getOrDefault("flink.app.name", "EngineConn-Flink") + val yarnQueue = LINKIS_QUEUE_NAME.getValue(options) + val parallelism = FLINK_APP_DEFAULT_PARALLELISM.getValue(options) + val jobManagerMemory = LINKIS_FLINK_JOB_MANAGER_MEMORY.getValue(options) + "G" + val taskManagerMemory = LINKIS_FLINK_TASK_MANAGER_MEMORY.getValue(options) + "G" + val numberOfTaskSlots = LINKIS_FLINK_TASK_SLOTS.getValue(options) + info(s"Use yarn queue $yarnQueue, and set parallelism = $parallelism, jobManagerMemory = $jobManagerMemory G, taskManagerMemory = $taskManagerMemory G, numberOfTaskSlots = $numberOfTaskSlots.") + //第二步: 应用级别配置 + //构建应用配置 + val flinkConfig = context.getFlinkConfig + //构建依赖jar包环境 + val flinkUserLibRemotePath = FLINK_USER_LIB_REMOTE_PATH.getValue(options).split(",") + val providedLibDirList = Lists.newArrayList(flinkUserLibRemotePath.filter(StringUtils.isNotBlank): _*) + val flinkUserRemotePathList = Lists.newArrayList(flinkLibRemotePath.split(",").filter(StringUtils.isNotBlank): _*) + if (flinkUserRemotePathList != null && flinkUserRemotePathList.size() > 0) providedLibDirList.addAll(flinkUserRemotePathList) + //if(StringUtils.isNotBlank(flinkLibRemotePath)) providedLibDirList.add(flinkLibRemotePath) + flinkConfig.set(YarnConfigOptions.PROVIDED_LIB_DIRS, providedLibDirList) + //构建依赖jar包环境 + flinkConfig.set(YarnConfigOptions.SHIP_DIRECTORIES, context.getShipDirs) + //yarn 作业名称 + flinkConfig.set(YarnConfigOptions.APPLICATION_NAME, jobName) + //yarn queue + flinkConfig.set(YarnConfigOptions.APPLICATION_QUEUE, yarnQueue) + //设置:资源/并发度 + flinkConfig.setInteger(CoreOptions.DEFAULT_PARALLELISM, parallelism) + flinkConfig.set(JobManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse(jobManagerMemory)) + flinkConfig.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse(taskManagerMemory)) + flinkConfig.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, numberOfTaskSlots) + //设置 savePoint + val savePointPath = FLINK_SAVE_POINT_PATH.getValue(options) + if (StringUtils.isNotBlank(savePointPath)) { + val allowNonRestoredState = FLINK_APP_ALLOW_NON_RESTORED_STATUS.getValue(options).toBoolean + val savepointRestoreSettings = SavepointRestoreSettings.forPath(savePointPath, allowNonRestoredState) + SavepointRestoreSettings.toConfiguration(savepointRestoreSettings, flinkConfig) + } + //设置:用户入口jar:可以远程,只能设置1个jar + val flinkMainClassJar = FLINK_APPLICATION_MAIN_CLASS_JAR.getValue(options) + if(StringUtils.isNotBlank(flinkMainClassJar)) { + info(s"Ready to use $flinkMainClassJar as main class jar to submit application to Yarn.") + flinkConfig.set(PipelineOptions.JARS, Collections.singletonList(flinkMainClassJar)) + flinkConfig.set(DeploymentOptions.TARGET, YarnDeploymentTarget.APPLICATION.getName) + context.setDeploymentTarget(YarnDeploymentTarget.APPLICATION) + addApplicationLabels(engineCreationContext) + } else if(isOnceEngineConn(engineCreationContext.getLabels())) { + flinkConfig.set(DeploymentOptions.TARGET, YarnDeploymentTarget.PER_JOB.getName) + } else { + flinkConfig.set(DeploymentOptions.TARGET, YarnDeploymentTarget.SESSION.getName) + } + context + } + + protected def isOnceEngineConn(labels: util.List[Label[_]]): Boolean = { + val engineConnModeLabel = getEngineConnModeLabel(labels) + engineConnModeLabel != null && (EngineConnMode.toEngineConnMode(engineConnModeLabel.getEngineConnMode) match { + case EngineConnMode.Once | EngineConnMode.Once_With_Cluster => true + case _ => false + }) + } + + private def addApplicationLabels(engineCreationContext: EngineCreationContext): Unit = { + val labels = engineCreationContext.getLabels().asScala + if(!labels.exists(_.isInstanceOf[CodeLanguageLabel])) { + val codeLanguageLabel = new CodeLanguageLabel + codeLanguageLabel.setCodeType(RunType.JAR.toString) + engineCreationContext.getLabels().add(codeLanguageLabel) + } + if(!labels.exists(_.isInstanceOf[EngineConnModeLabel])) { + val engineConnModeLabel = new EngineConnModeLabel + engineConnModeLabel.setEngineConnMode(EngineConnMode.Once.toString) + engineCreationContext.getLabels().add(engineConnModeLabel) + } + } + + def createExecutionContext(options: util.Map[String, String], environmentContext: EnvironmentContext): ExecutionContext = { + val environment = environmentContext.getDeploymentTarget match { + case YarnDeploymentTarget.PER_JOB | YarnDeploymentTarget.SESSION => + val planner = FlinkEnvConfiguration.FLINK_SQL_PLANNER.getValue(options) + if (!ExecutionEntry.AVAILABLE_PLANNERS.contains(planner.toLowerCase)) + throw new FlinkInitFailedException("Planner must be one of these: " + String.join(", ", ExecutionEntry.AVAILABLE_PLANNERS)) + val executionType = FlinkEnvConfiguration.FLINK_SQL_EXECUTION_TYPE.getValue(options) + if (!ExecutionEntry.AVAILABLE_EXECUTION_TYPES.contains(executionType.toLowerCase)) + throw new FlinkInitFailedException("Execution type must be one of these: " + String.join(", ", ExecutionEntry.AVAILABLE_EXECUTION_TYPES)) + val properties = new util.HashMap[String, String] + properties.put(Environment.EXECUTION_ENTRY + "." + ExecutionEntry.EXECUTION_PLANNER, planner) + properties.put(Environment.EXECUTION_ENTRY + "." + ExecutionEntry.EXECUTION_TYPE, executionType) + if (executionType.equalsIgnoreCase(ExecutionEntry.EXECUTION_TYPE_VALUE_BATCH)) { + // for batch mode we ensure that results are provided in materialized form + properties.put(Environment.EXECUTION_ENTRY + "." + ExecutionEntry.EXECUTION_RESULT_MODE, ExecutionEntry.EXECUTION_RESULT_MODE_VALUE_TABLE) + } else { + // for streaming mode we ensure that results are provided in changelog form + properties.put(Environment.EXECUTION_ENTRY + "." + ExecutionEntry.EXECUTION_RESULT_MODE, ExecutionEntry.EXECUTION_RESULT_MODE_VALUE_CHANGELOG) + } + Environment.enrich(environmentContext.getDefaultEnv, properties, Collections.emptyMap()) + case YarnDeploymentTarget.APPLICATION => null + case t => + error(s"Not supported YarnDeploymentTarget ${t.getName}.") + throw new FlinkInitFailedException(s"Not supported YarnDeploymentTarget ${t.getName}.") + } + ExecutionContext.builder(environmentContext.getDefaultEnv, environment, environmentContext.getDependencies, + environmentContext.getFlinkConfig, environmentContext.getClusterClientServiceLoader).build() + } + + protected def createFlinkEngineConnContext(environmentContext: EnvironmentContext): FlinkEngineConnContext = + new FlinkEngineConnContext(environmentContext) + + override protected def getDefaultExecutorFactoryClass: Class[_ <: ExecutorFactory] = classOf[FlinkSQLExecutorFactory] + + override protected def getEngineConnType: EngineType = EngineType.FLINK + + private val executorFactoryArray = Array[ExecutorFactory](new FlinkSQLExecutorFactory, new FlinkApplicationExecutorFactory) + + + override def getExecutorFactories: Array[ExecutorFactory] = executorFactoryArray +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkSQLExecutorFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkSQLExecutorFactory.scala new file mode 100644 index 0000000000..5f0ef8306d --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkSQLExecutorFactory.scala @@ -0,0 +1,63 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.factory + +import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext +import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationExecutorFactory +import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.ComputationExecutor +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration +import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext +import com.webank.wedatasphere.linkis.engineconnplugin.flink.executor.FlinkSQLJobExecutor +import com.webank.wedatasphere.linkis.manager.label.entity.Label +import com.webank.wedatasphere.linkis.manager.label.entity.cluster.EnvLabel +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType + +import scala.collection.JavaConversions._ + +/** + * + */ +class FlinkSQLExecutorFactory extends ComputationExecutorFactory { + + override protected def newExecutor(id: Int, + engineCreationContext: EngineCreationContext, + engineConn: EngineConn, + labels: Array[Label[_]]): ComputationExecutor = engineConn.getEngineConnSession match { + case context: FlinkEngineConnContext => + val executor = new FlinkSQLJobExecutor(id, context) + if(!labels.exists(_.isInstanceOf[EnvLabel])) { + executor.getExecutorLabels().add(getEnvLabel(engineCreationContext)) + } + if(executor.getEnvLabel.getEnvType == EnvLabel.DEV) { + context.getEnvironmentContext.getDefaultEnv + .setExecution(Map("max-table-result-rows" -> FlinkEnvConfiguration.FLINK_SQL_DEV_SELECT_MAX_LINES.getValue.asInstanceOf[Object])) + } + executor + } + + protected def getEnvLabel(engineCreationContext: EngineCreationContext): EnvLabel = { + val envType = FlinkEnvConfiguration.FLINK_ENV_TYPE.getValue(engineCreationContext.getOptions) + val envLabel = new EnvLabel + envLabel.setEnvType(envType) + envLabel + } + + override protected def getRunType: RunType = RunType.SQL +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/launch/FlinkEngineConnLaunchBuilder.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/launch/FlinkEngineConnLaunchBuilder.scala new file mode 100644 index 0000000000..d5dc3794cd --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/launch/FlinkEngineConnLaunchBuilder.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.webank.wedatasphere.linkis.engineconnplugin.flink.launch + +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration._ +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkResourceConfiguration +import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.EnvConfiguration +import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.entity.EngineConnBuildRequest +import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder + +/** + * + */ +class FlinkEngineConnLaunchBuilder extends JavaProcessEngineConnLaunchBuilder { + + override protected def getCommands(implicit engineConnBuildRequest: EngineConnBuildRequest): Array[String] = { + val properties = engineConnBuildRequest.engineConnCreationDesc.properties + properties.put(EnvConfiguration.ENGINE_CONN_MEMORY.key, FlinkResourceConfiguration.LINKIS_FLINK_CLIENT_MEMORY.getValue(properties) + "G") + super.getCommands + } + + override protected def getNecessaryEnvironment(implicit engineConnBuildRequest: EngineConnBuildRequest): Array[String] = + Array(FLINK_HOME_ENV, FLINK_CONF_DIR_ENV) ++: super.getNecessaryEnvironment + + override protected def ifAddHiveConfigPath: Boolean = true + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListener.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListener.scala new file mode 100644 index 0000000000..f17bf960b1 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListener.scala @@ -0,0 +1,47 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.listener + +import com.webank.wedatasphere.linkis.common.listener.{Event, EventListener} +import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.RowsType.RowsType + +/** + * + */ +trait FlinkListener extends EventListener { + override def onEventError(event: Event, t: Throwable): Unit = {} +} + +trait FlinkStatusListener extends FlinkListener { + + def onSuccess(rows: Int, rowsType: RowsType): Unit + + def onFailed(message: String, t: Throwable): Unit + +} + +object RowsType extends Enumeration { + type RowsType = Value + val Fetched, Affected = Value +} + +trait FlinkStreamingResultSetListener extends FlinkListener { + + def onResultSetPulled(rows: Int): Unit + +} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListenerGroup.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListenerGroup.scala new file mode 100644 index 0000000000..e7eb3c38df --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListenerGroup.scala @@ -0,0 +1,62 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.listener + +import java.util + +import scala.collection.JavaConversions._ + +/** + * + */ +trait FlinkListenerGroup { + + def addFlinkListener(flinkListener: FlinkListener): Unit + + def getFlinkListeners: util.List[FlinkListener] + + def setFlinkListeners(flinkListeners: util.List[FlinkListener]): Unit + +} + +abstract class FlinkListenerGroupImpl extends FlinkListenerGroup { + + private var flinkListeners: util.List[FlinkListener] = _ + + override def addFlinkListener(flinkListener: FlinkListener): Unit = { + if (flinkListeners == null) flinkListeners = new util.ArrayList[FlinkListener] + flinkListeners.add(flinkListener) + } + + override def getFlinkListeners: util.List[FlinkListener] = flinkListeners + + override def setFlinkListeners(flinkListeners: util.List[FlinkListener]): Unit = { + this.flinkListeners = flinkListeners + } + + private def getFlinkListeners[T <: FlinkListener](clazz: Class[T]): util.List[T] = flinkListeners match { + case listeners: util.List[FlinkListener] => + listeners.filter(_.isInstanceOf[T]).map(_.asInstanceOf[T]) + case _ => new util.ArrayList[T] + } + + def getFlinkStatusListeners: util.List[FlinkStatusListener] = getFlinkListeners(classOf[FlinkStatusListener]) + + def getFlinkStreamingResultSetListeners: util.List[FlinkStreamingResultSetListener] = getFlinkListeners(classOf[FlinkStreamingResultSetListener]) + +} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/resource/FlinkEngineConnResourceFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/resource/FlinkEngineConnResourceFactory.scala new file mode 100644 index 0000000000..d4ea352907 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/resource/FlinkEngineConnResourceFactory.scala @@ -0,0 +1,47 @@ +/* + * 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 com.webank.wedatasphere.linkis.engineconnplugin.flink.resource + +import com.webank.wedatasphere.linkis.common.utils.ByteTimeUtils +import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkResourceConfiguration._ +import com.webank.wedatasphere.linkis.manager.common.entity.resource.{DriverAndYarnResource, LoadInstanceResource, Resource, YarnResource} +import com.webank.wedatasphere.linkis.manager.engineplugin.common.resource.AbstractEngineResourceFactory + +/** + * + */ +class FlinkEngineConnResourceFactory extends AbstractEngineResourceFactory { + + override def getRequestResource(properties: java.util.Map[String, String]): Resource = { + val containers = if(properties.containsKey(LINKIS_FLINK_CONTAINERS)) { + val containers = LINKIS_FLINK_CONTAINERS.getValue(properties) + properties.put(FLINK_APP_DEFAULT_PARALLELISM.key, String.valueOf(containers * LINKIS_FLINK_TASK_SLOTS.getValue(properties))) + containers + } else math.round(FLINK_APP_DEFAULT_PARALLELISM.getValue(properties) * 1.0f / LINKIS_FLINK_TASK_SLOTS.getValue(properties)) + val yarnMemory = ByteTimeUtils.byteStringAsBytes(LINKIS_FLINK_TASK_MANAGER_MEMORY.getValue(properties) * containers + "G") + + ByteTimeUtils.byteStringAsBytes(LINKIS_FLINK_JOB_MANAGER_MEMORY.getValue(properties) + "G") + val yarnCores = LINKIS_FLINK_TASK_MANAGER_CPU_CORES.getValue(properties) * containers + 1 + new DriverAndYarnResource( + new LoadInstanceResource(ByteTimeUtils.byteStringAsBytes(LINKIS_FLINK_CLIENT_MEMORY.getValue(properties) + "G"), + LINKIS_FLINK_CLIENT_CORES, + 1), + new YarnResource(yarnMemory, yarnCores, 0, LINKIS_QUEUE_NAME.getValue(properties)) + ) + } + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/FlinkValueFormatUtil.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/FlinkValueFormatUtil.scala new file mode 100644 index 0000000000..91e5805c5a --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/FlinkValueFormatUtil.scala @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.webank.wedatasphere.linkis.engineconnplugin.flink.util + +import java.text.NumberFormat + +import com.webank.wedatasphere.linkis.common.conf.CommonVars + +/** + * + */ +object FlinkValueFormatUtil { + + val FLINK_NF_FRACTION_LENGTH = CommonVars("wds.linkis.engine.flink.fraction.length", 30) + + private val nf = NumberFormat.getInstance() + nf.setGroupingUsed(false) + nf.setMaximumFractionDigits(FLINK_NF_FRACTION_LENGTH.getValue) + + def formatValue(value: Any): Any = value match { + case value: String => value.replaceAll("\n|\t", " ") + case value: Double => nf.format(value) + case value: Any => value.toString + case _ => null + } + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/resources/linkis-engineconn.properties b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/resources/linkis-engineconn.properties index 398cf12c14..09d77edcf4 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/resources/linkis-engineconn.properties +++ b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/resources/linkis-engineconn.properties @@ -22,4 +22,6 @@ wds.linkis.engineconn.debug.enable=true #wds.linkis.keytab.enable=true -wds.linkis.engineconn.plugin.default.clazz=com.webank.wedatasphere.linkis.engineplugin.hive.HiveEngineConnPlugin +wds.linkis.engineconn.plugin.default.class=com.webank.wedatasphere.linkis.engineplugin.hive.HiveEngineConnPlugin + +wds.linkis.engine.connector.hooks=com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.ComputationEngineConnHook,com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.JarUdfEngineHook \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/resources/log4j2-engineconn.xml b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/resources/log4j2-engineconn.xml index 0f5dd45478..6343fa34d8 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/resources/log4j2-engineconn.xml +++ b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/resources/log4j2-engineconn.xml @@ -20,19 +20,12 @@ - - - - - - @@ -54,6 +47,18 @@ - + + + + + + + + + + + + + diff --git a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/conf/HiveEngineConfiguration.scala b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/conf/HiveEngineConfiguration.scala index 9b17dbbb80..644ccb4c1c 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/conf/HiveEngineConfiguration.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/conf/HiveEngineConfiguration.scala @@ -16,7 +16,10 @@ package com.webank.wedatasphere.linkis.engineplugin.hive.conf +import com.webank.wedatasphere.linkis.common.conf.CommonVars + object HiveEngineConfiguration { + val HIVE_LIB_HOME = CommonVars[String]("hive.lib", CommonVars[String]("HIVE_LIB", "/appcom/Install/hive/lib").getValue) } \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/creation/HiveEngineConnFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/creation/HiveEngineConnFactory.scala index 6291bded4f..7dec7e65cf 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/creation/HiveEngineConnFactory.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/creation/HiveEngineConnFactory.scala @@ -21,43 +21,39 @@ import java.security.PrivilegedExceptionAction import com.webank.wedatasphere.linkis.common.utils.{Logging, Utils} import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconn.common.engineconn.{DefaultEngineConn, EngineConn} -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor +import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationSingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.executor.entity.LabelExecutor import com.webank.wedatasphere.linkis.engineplugin.hive.common.HiveUtils +import com.webank.wedatasphere.linkis.engineplugin.hive.conf.HiveEngineConfiguration import com.webank.wedatasphere.linkis.engineplugin.hive.entity.HiveSession import com.webank.wedatasphere.linkis.engineplugin.hive.exception.HiveSessionStartFailedException import com.webank.wedatasphere.linkis.engineplugin.hive.executor.HiveEngineConnExecutor import com.webank.wedatasphere.linkis.hadoop.common.utils.HDFSUtils -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.SingleExecutorEngineConnFactory -import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineRunTypeLabel, EngineType, RunType} +import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType.EngineType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineType, RunType} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.session.SessionState import scala.collection.JavaConversions._ -class HiveEngineConnFactory extends SingleExecutorEngineConnFactory with Logging { +class HiveEngineConnFactory extends ComputationSingleExecutorEngineConnFactory with Logging { private val HIVE_QUEUE_NAME: String = "mapreduce.job.queuename" private val BDP_QUEUE_NAME: String = "wds.linkis.rm.yarnqueue" - private var engineCreationContext: EngineCreationContext = _ - override def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn): Executor = { - engineConn.getEngine() match { + override protected def newExecutor(id: Int, engineCreationContext: EngineCreationContext, engineConn: EngineConn): LabelExecutor = { + engineConn.getEngineConnSession match { case hiveSession: HiveSession => - this.engineCreationContext = engineCreationContext - val id = ExecutorManager.getInstance().generateId() - val executor = new HiveEngineConnExecutor(id, hiveSession.sessionState, hiveSession.ugi, hiveSession.hiveConf, hiveSession.baos) - executor.getExecutorLabels().add(getDefaultEngineRunTypeLabel()) - executor + new HiveEngineConnExecutor(id, hiveSession.sessionState, hiveSession.ugi, hiveSession.hiveConf, hiveSession.baos) case _ => throw HiveSessionStartFailedException(40012, "Failed to create hive executor") } } - - override def createEngineConn(engineCreationContext: EngineCreationContext): EngineConn = { + override protected def createEngineConnSession(engineCreationContext: EngineCreationContext): HiveSession = { val options = engineCreationContext.getOptions val hiveConf: HiveConf = HiveUtils.getHiveConf hiveConf.setVar(HiveConf.ConfVars.HIVEJAR, HiveUtils.jarOfClass(classOf[Driver]) @@ -67,7 +63,17 @@ class HiveEngineConnFactory extends SingleExecutorEngineConnFactory with Logging info(s"key is $k, value is $v") if (BDP_QUEUE_NAME.equals(k)) hiveConf.set(HIVE_QUEUE_NAME, v) else hiveConf.set(k, v) } - + hiveConf.setVar(HiveConf.ConfVars.HIVE_HADOOP_CLASSPATH, HiveEngineConfiguration.HIVE_LIB_HOME.getValue + "/*") + /* //Update by peaceWong add hook to HiveDriver + if (StringUtils.isNotBlank(EnvConfiguration.LINKIS_HIVE_POST_HOOKS)) { + val hooks = if (StringUtils.isNotBlank(hiveConf.get("hive.exec.post.hooks"))) { + hiveConf.get("hive.exec.post.hooks") + "," + EnvConfiguration.LINKIS_HIVE_POST_HOOKS + } else { + EnvConfiguration.LINKIS_HIVE_POST_HOOKS + } + hiveConf.set("hive.exec.post.hooks", hooks) + }*/ + //Update by peaceWong enable hive.stats.collect.scancols hiveConf.setBoolean("hive.stats.collect.scancols", true) val ugi = HDFSUtils.getUserGroupInformation(Utils.getJvmUser) val sessionState: SessionState = ugi.doAs(new PrivilegedExceptionAction[SessionState] { @@ -79,18 +85,11 @@ class HiveEngineConnFactory extends SingleExecutorEngineConnFactory with Logging sessionState.err = new PrintStream(System.out, true, "utf-8") SessionState.start(sessionState) - val hiveSession = HiveSession(sessionState, ugi, hiveConf, baos) - val engineConn = new DefaultEngineConn(engineCreationContext) - engineConn.setEngineType(EngineType.HIVE.toString) - engineConn.setEngine(hiveSession) - engineConn + HiveSession(sessionState, ugi, hiveConf, baos) } - def getEngineCreationContext: EngineCreationContext = engineCreationContext + override protected def getEngineConnType: EngineType = EngineType.HIVE + + override protected def getRunType: RunType = RunType.HIVE - override def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel = { - val runTypeLabel = new EngineRunTypeLabel - runTypeLabel.setRunType(RunType.HIVE.toString) - runTypeLabel - } } diff --git a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/executor/HiveEngineConnExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/executor/HiveEngineConnExecutor.scala index b694205dd7..4e124548d9 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/executor/HiveEngineConnExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/executor/HiveEngineConnExecutor.scala @@ -21,11 +21,11 @@ import java.util.concurrent.atomic.AtomicBoolean import com.webank.wedatasphere.linkis.common.exception.ErrorException import com.webank.wedatasphere.linkis.common.utils.{Logging, Utils} import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.{ComputationExecutor, EngineExecutionContext} -import com.webank.wedatasphere.linkis.engineconn.computation.executor.parser.SQLCodeParser import com.webank.wedatasphere.linkis.engineconn.core.EngineConnObject import com.webank.wedatasphere.linkis.engineplugin.hive.cs.CSHiveHelper import com.webank.wedatasphere.linkis.engineplugin.hive.exception.HiveQueryFailedException import com.webank.wedatasphere.linkis.engineplugin.hive.progress.HiveProgressHelper +import com.webank.wedatasphere.linkis.governance.common.paser.SQLCodeParser import com.webank.wedatasphere.linkis.manager.common.entity.resource.{CommonNodeResource, LoadInstanceResource, NodeResource} import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.EngineConnPluginConf import com.webank.wedatasphere.linkis.manager.label.entity.Label @@ -260,7 +260,7 @@ class HiveEngineConnExecutor(id: Int, singleSqlProgressMap foreach { case (jobId, progress) => arrayBuffer += JobProgressInfo(jobId, 200, 0, 0, 200) } - engineExecutorContext.sendProgress(1.0f, arrayBuffer.toArray[JobProgressInfo]) + engineExecutorContext.pushProgress(1.0f, arrayBuffer.toArray[JobProgressInfo]) } } diff --git a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/hook/HiveAddJarsEngineHook.scala b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/hook/HiveAddJarsEngineHook.scala index bd0c7de1d8..fa3a30dcc0 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/hook/HiveAddJarsEngineHook.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/hook/HiveAddJarsEngineHook.scala @@ -20,9 +20,11 @@ import com.webank.wedatasphere.linkis.common.utils.{Logging, Utils} import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn import com.webank.wedatasphere.linkis.engineconn.common.hook.EngineConnHook +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationExecutorManager import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager import com.webank.wedatasphere.linkis.engineplugin.hive.executor.HiveEngineConnExecutor +import com.webank.wedatasphere.linkis.manager.label.entity.Label +import com.webank.wedatasphere.linkis.manager.label.entity.engine.{CodeLanguageLabel, RunType} import org.apache.commons.lang.StringUtils import scala.collection.JavaConversions._ @@ -45,13 +47,17 @@ class HiveAddJarsEngineHook extends EngineConnHook with Logging { jars = value } } + val codeLanguageLabel = new CodeLanguageLabel + codeLanguageLabel.setCodeType(RunType.HIVE.toString) + val labels = Array[Label[_]](codeLanguageLabel) + if (StringUtils.isNotBlank(jars)) { jars.split(",") foreach { jar => try { val sql = addSql + jar logger.info("begin to run hive sql {}", sql) - ExecutorManager.getInstance().getDefaultExecutor match { + ComputationExecutorManager.getInstance.getExecutorByLabels(labels) match { case executor: HiveEngineConnExecutor => executor.executeLine(new EngineExecutionContext(executor), sql) case _ => diff --git a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/hook/UseDatabaseEngineHook.scala b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/hook/UseDatabaseEngineHook.scala index c3e985487b..07a0401649 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/hook/UseDatabaseEngineHook.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/hook/UseDatabaseEngineHook.scala @@ -20,9 +20,11 @@ import com.webank.wedatasphere.linkis.common.utils.{Logging, Utils} import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn import com.webank.wedatasphere.linkis.engineconn.common.hook.EngineConnHook +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationExecutorManager import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager import com.webank.wedatasphere.linkis.engineplugin.hive.executor.HiveEngineConnExecutor +import com.webank.wedatasphere.linkis.manager.label.entity.Label +import com.webank.wedatasphere.linkis.manager.label.entity.engine.{CodeLanguageLabel, RunType} import org.apache.commons.lang.StringUtils class UseDatabaseEngineHook extends EngineConnHook with Logging { @@ -45,7 +47,10 @@ class UseDatabaseEngineHook extends EngineConnHook with Logging { } val useDataBaseSql = "use " + database info(s"hive client begin to run init_code $useDataBaseSql") - ExecutorManager.getInstance().getDefaultExecutor match { + val codeLanguageLabel = new CodeLanguageLabel + codeLanguageLabel.setCodeType(RunType.HIVE.toString) + val labels = Array[Label[_]](codeLanguageLabel) + ComputationExecutorManager.getInstance.getExecutorByLabels(labels) match { case executor: HiveEngineConnExecutor => executor.executeLine(new EngineExecutionContext(executor), useDataBaseSql) case _ => diff --git a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/launch/HiveProcessEngineConnLaunchBuilder.scala b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/launch/HiveProcessEngineConnLaunchBuilder.scala index 342724c45b..ad5ee094fa 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/launch/HiveProcessEngineConnLaunchBuilder.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/hive/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/hive/launch/HiveProcessEngineConnLaunchBuilder.scala @@ -16,10 +16,17 @@ package com.webank.wedatasphere.linkis.engineplugin.hive.launch +import java.util + +import com.google.common.collect.Lists +import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.entity.EngineConnBuildRequest import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder class HiveProcessEngineConnLaunchBuilder extends JavaProcessEngineConnLaunchBuilder { override protected def ifAddHiveConfigPath: Boolean = true + override protected def getEngineConnManagerHooks(implicit engineConnBuildRequest: EngineConnBuildRequest): util.List[String] = { + Lists.newArrayList("JarUDFLoadECMHook") + } } diff --git a/linkis-engineconn-plugins/engineconn-plugins/io_file/pom.xml b/linkis-engineconn-plugins/engineconn-plugins/io_file/pom.xml index 8ae2ff0767..d0d7f6a9ee 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/io_file/pom.xml +++ b/linkis-engineconn-plugins/engineconn-plugins/io_file/pom.xml @@ -28,7 +28,7 @@ linkis-engineplugin-io_file - 1 + 1.0 diff --git a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/assembly/distribution.xml b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/assembly/distribution.xml index c9c31be9f3..2f143d5af3 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/assembly/distribution.xml +++ b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/assembly/distribution.xml @@ -22,6 +22,7 @@ linkis-engineplugin-io_file dir + zip true io_file diff --git a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/io/utils/ReflectionUtils.java b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/io/utils/ReflectionUtils.java new file mode 100644 index 0000000000..ee49a37efd --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/io/utils/ReflectionUtils.java @@ -0,0 +1,15 @@ +package com.webank.wedatasphere.linkis.manager.engineplugin.io.utils; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; + + +public class ReflectionUtils { + public static Object invoke(Object object, Method method, Object[] args) throws Throwable { + try { + return method.invoke(object, args); + } catch (InvocationTargetException e) { + throw e.getCause(); + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/resources/linkis-engineconn.properties b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/resources/linkis-engineconn.properties index efe0cf5c8c..b67de2683d 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/resources/linkis-engineconn.properties +++ b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/resources/linkis-engineconn.properties @@ -21,7 +21,7 @@ wds.linkis.engineconn.debug.enable=true #wds.linkis.keytab.enable=true -wds.linkis.engineconn.plugin.default.clazz=com.webank.wedatasphere.linkis.manager.engineplugin.io.IoEngineConnPlugin +wds.linkis.engineconn.plugin.default.class=com.webank.wedatasphere.linkis.manager.engineplugin.io.IoEngineConnPlugin wds.linkis.engineconn.io.version=1 @@ -29,3 +29,4 @@ wds.linkis.engineconn.io.version=1 wds.linkis.engineconn.support.parallelism=true +wds.linkis.engineconn.max.free.time=0 \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/resources/log4j2-engineconn.xml b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/resources/log4j2-engineconn.xml index 74af83d050..66388d7469 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/resources/log4j2-engineconn.xml +++ b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/resources/log4j2-engineconn.xml @@ -22,21 +22,14 @@ - - - - - - - - - + + + + diff --git a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/IoEngineConnPlugin.scala b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/IoEngineConnPlugin.scala index 95200ec414..a0f47bda76 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/IoEngineConnPlugin.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/IoEngineConnPlugin.scala @@ -23,10 +23,8 @@ import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.Engin import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.EngineConnLaunchBuilder import com.webank.wedatasphere.linkis.manager.engineplugin.common.resource.{EngineResourceFactory, GenericEngineResourceFactory} import com.webank.wedatasphere.linkis.manager.engineplugin.io.builder.IoProcessEngineConnLaunchBuilder -import com.webank.wedatasphere.linkis.manager.engineplugin.io.conf.IOEngineConnConfiguration import com.webank.wedatasphere.linkis.manager.engineplugin.io.factory.IoEngineConnFactory import com.webank.wedatasphere.linkis.manager.label.entity.Label -import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineRunTypeLabel, EngineType, EngineTypeLabel, RunType} class IoEngineConnPlugin extends EngineConnPlugin { diff --git a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/executor/IoEngineConnExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/executor/IoEngineConnExecutor.scala index 694b28070d..0e004067b4 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/executor/IoEngineConnExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/executor/IoEngineConnExecutor.scala @@ -43,7 +43,7 @@ import org.json4s.DefaultFormats import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer -class IoEngineConnExecutor(val id: Int, val outputLimit: Int) extends ConcurrentComputationExecutor(outputLimit) with Logging { +class IoEngineConnExecutor(val id: Int, val outputLimit: Int = 10) extends ConcurrentComputationExecutor(outputLimit) with Logging { implicit val formats = DefaultFormats @@ -62,12 +62,9 @@ class IoEngineConnExecutor(val id: Int, val outputLimit: Int) extends Concurrent private val namePrefix: String = "IoEngineConnExecutor_" override def init(): Unit = { - if (!initialized) { - super.init - info("Ready to start IoEngine!") - cleanupThread.start() - initialized = true - } + super.init + info("Ready to start IoEngine!") + cleanupThread.start() } /* @@ -88,7 +85,7 @@ class IoEngineConnExecutor(val id: Int, val outputLimit: Int) extends Concurrent clearCount = clearCount + 1 } } - info(s"Finished to clear userFs, clear count: $clearCount") + debug(s"Finished to clear userFs, clear count: $clearCount") Utils.tryQuietly(Thread.sleep(IOEngineConnConfiguration.IO_FS_CLEAR_TIME.getValue)) } } diff --git a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/factory/IoEngineConnFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/factory/IoEngineConnFactory.scala index fee89f7651..09035c29d4 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/factory/IoEngineConnFactory.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/factory/IoEngineConnFactory.scala @@ -18,35 +18,24 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.io.factory import com.webank.wedatasphere.linkis.common.utils.Logging import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconn.common.engineconn.{DefaultEngineConn, EngineConn} -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.SingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationSingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.executor.entity.LabelExecutor import com.webank.wedatasphere.linkis.manager.engineplugin.io.conf.IOEngineConnConfiguration import com.webank.wedatasphere.linkis.manager.engineplugin.io.executor.IoEngineConnExecutor -import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineRunTypeLabel, EngineType, EngineTypeLabel, RunType} +import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType.EngineType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineType, EngineTypeLabel, RunType} import org.apache.commons.lang.StringUtils -class IoEngineConnFactory extends SingleExecutorEngineConnFactory with Logging { +class IoEngineConnFactory extends ComputationSingleExecutorEngineConnFactory with Logging { - private var engineCreationContext: EngineCreationContext = _ + override protected def getEngineConnType: EngineType = EngineType.IO_ENGINE_FILE - override def createEngineConn(engineCreationContext: EngineCreationContext): EngineConn = { - val engineConn = new DefaultEngineConn(engineCreationContext) - engineConn.setEngineType(EngineType.IO_ENGINE_FILE.toString) - engineConn - } - - override def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn): Executor = { - this.engineCreationContext = engineCreationContext - - val id = ExecutorManager.getInstance().generateId() + override protected def getRunType: RunType = RunType.IO_FILE - val executor = new IoEngineConnExecutor(id, IOEngineConnConfiguration.OUTPUT_LIMIT.getValue) - - val runTypeLabel = getDefaultEngineRunTypeLabel() - executor.getExecutorLabels().add(runTypeLabel) - executor + override def newExecutor(id: Int, engineCreationContext: EngineCreationContext, engineConn: EngineConn): LabelExecutor = { + new IoEngineConnExecutor(id, IOEngineConnConfiguration.OUTPUT_LIMIT.getValue) } private def engineVersionMatch(initalLabel: EngineTypeLabel, engineCreationLabel: EngineTypeLabel): Boolean = { @@ -59,10 +48,4 @@ class IoEngineConnFactory extends SingleExecutorEngineConnFactory with Logging { } } - override def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel = { - val runTypeLabel = new EngineRunTypeLabel - runTypeLabel.setRunType(RunType.IO_FILE.toString) - runTypeLabel - } - } diff --git a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/service/FsProxyService.scala b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/service/FsProxyService.scala index ff84db2b4b..1d931b4781 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/service/FsProxyService.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/service/FsProxyService.scala @@ -25,7 +25,7 @@ class FsProxyService extends Logging{ case StorageConfiguration.STORAGE_ROOT_USER.getValue => true case StorageConfiguration.LOCAL_ROOT_USER.getValue => StorageUtils.FILE == fsType case StorageConfiguration.HDFS_ROOT_USER.getValue => StorageUtils.HDFS == fsType - case _ => creatorUser.equals(proxyUser) + case _ => true//creatorUser.equals(proxyUser) } /* if(creatorUser.equals(proxyUser)){ return true diff --git a/linkis-engineconn-plugins/engineconn-plugins/jdbc/pom.xml b/linkis-engineconn-plugins/engineconn-plugins/jdbc/pom.xml index 120cfa5049..a67d9df57e 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/jdbc/pom.xml +++ b/linkis-engineconn-plugins/engineconn-plugins/jdbc/pom.xml @@ -148,6 +148,7 @@ + ${project.artifactId}-${project.version} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/resources/linkis-engineconn.properties b/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/resources/linkis-engineconn.properties index 1f0f88f3f6..376d6ecd18 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/resources/linkis-engineconn.properties +++ b/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/resources/linkis-engineconn.properties @@ -21,7 +21,7 @@ wds.linkis.engineconn.debug.enable=true #wds.linkis.keytab.enable=true -wds.linkis.engineconn.plugin.default.clazz=com.webank.wedatasphere.linkis.manager.engineplugin.jdbc.JDBCEngineConnPlugin +wds.linkis.engineconn.plugin.default.class=com.webank.wedatasphere.linkis.manager.engineplugin.jdbc.JDBCEngineConnPlugin #wds.linkis.engine.io.opts=" -Dfile.encoding=UTF-8 -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=49100 " diff --git a/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/resources/log4j2-engineconn.xml b/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/resources/log4j2-engineconn.xml index 3c12a8a073..4d824574c1 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/resources/log4j2-engineconn.xml +++ b/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/resources/log4j2-engineconn.xml @@ -21,12 +21,6 @@ - - - - - @@ -49,12 +43,25 @@ - + + + + + + + + + + + + + + diff --git a/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/jdbc/JDBCEngineConnPlugin.scala b/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/jdbc/JDBCEngineConnPlugin.scala index a31735f6ed..169cb5bbde 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/jdbc/JDBCEngineConnPlugin.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/jdbc/JDBCEngineConnPlugin.scala @@ -43,7 +43,7 @@ class JDBCEngineConnPlugin extends EngineConnPlugin { engineTypeLabel.setEngineType(EngineType.IO_ENGINE.toString) engineTypeLabel.setVersion(IOEngineConnConfiguration.DEFAULT_VERSION.getValue) this.defaultLabels.add(engineTypeLabel) - val runTypeLabel = new EngineRunTypeLabel() + val runTypeLabel = new CodeLanguageLabel() runTypeLabel.setRunType(RunType.IO.toString)*/ } diff --git a/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/jdbc/factory/JDBCEngineConnFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/jdbc/factory/JDBCEngineConnFactory.scala index 5d3e60e3d6..1699d51759 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/jdbc/factory/JDBCEngineConnFactory.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/jdbc/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/jdbc/factory/JDBCEngineConnFactory.scala @@ -18,38 +18,22 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.jdbc.factory import com.webank.wedatasphere.linkis.common.utils.Logging import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconn.common.engineconn.{DefaultEngineConn, EngineConn} -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.SingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationSingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.executor.entity.LabelExecutor import com.webank.wedatasphere.linkis.manager.engineplugin.jdbc.conf.JDBCConfiguration import com.webank.wedatasphere.linkis.manager.engineplugin.jdbc.executer.JDBCEngineConnExecutor -import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineRunTypeLabel, EngineType, RunType} +import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineType, RunType} +import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType.EngineType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType -class JDBCEngineConnFactory extends SingleExecutorEngineConnFactory with Logging{ +class JDBCEngineConnFactory extends ComputationSingleExecutorEngineConnFactory with Logging{ - private var engineCreationContext: EngineCreationContext = _ - - override def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn): Executor = { - this.engineCreationContext = engineCreationContext - val id = ExecutorManager.getInstance().generateId() - val executor = new JDBCEngineConnExecutor(JDBCConfiguration.ENGINE_DEFAULT_LIMIT.getValue, id) - val runTypeLabel = getDefaultEngineRunTypeLabel() - executor.getExecutorLabels().add(runTypeLabel) - executor - } - - override def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel = { - val runTypeLabel = new EngineRunTypeLabel - runTypeLabel.setRunType(RunType.JDBC.toString) - runTypeLabel - } - - override def createEngineConn(engineCreationContext: EngineCreationContext): EngineConn = { - val engineConn = new DefaultEngineConn(engineCreationContext) - engineConn.setEngineType(EngineType.JDBC.toString) - engineConn + override def newExecutor(id: Int, engineCreationContext: EngineCreationContext, engineConn: EngineConn): LabelExecutor = { + new JDBCEngineConnExecutor(JDBCConfiguration.ENGINE_DEFAULT_LIMIT.getValue, id) } + override protected def getEngineConnType: EngineType = EngineType.JDBC + override protected def getRunType: RunType = RunType.JDBC } diff --git a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/resources/linkis-engineconn.properties b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/resources/linkis-engineconn.properties index e7078fc989..380724d838 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/resources/linkis-engineconn.properties +++ b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/resources/linkis-engineconn.properties @@ -23,7 +23,7 @@ wds.linkis.engineconn.debug.enable=true #wds.linkis.keytab.enable=true -wds.linkis.engineconn.plugin.default.clazz=com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.PipelineEngineConnPlugin +wds.linkis.engineconn.plugin.default.class=com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.PipelineEngineConnPlugin wds.linkis.engineconn.max.free.time=5m diff --git a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/PipelineEngineConnPlugin.scala b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/PipelineEngineConnPlugin.scala index 6757fe196d..531ffaa54d 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/PipelineEngineConnPlugin.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/PipelineEngineConnPlugin.scala @@ -16,6 +16,8 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.pipeline +import java.util + import com.webank.wedatasphere.linkis.manager.engineplugin.common.EngineConnPlugin import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.EngineConnFactory import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.EngineConnLaunchBuilder @@ -24,8 +26,6 @@ import com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.builder.Pipe import com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.factory.PipelineEngineConnFactory import com.webank.wedatasphere.linkis.manager.label.entity.Label -import java.util - class PipelineEngineConnPlugin extends EngineConnPlugin { private val EP_CONTEXT_CONSTRUCTOR_LOCK = new Object() diff --git a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/conversions/FsConvertions.scala b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/conversions/FsConvertions.scala index 489c2d047b..2d9535f6f9 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/conversions/FsConvertions.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/conversions/FsConvertions.scala @@ -16,10 +16,10 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.conversions -import com.webank.wedatasphere.linkis.common.io.Fs - import java.io.Closeable +import com.webank.wedatasphere.linkis.common.io.Fs + object FsConvertions { implicit def fsToFsClosable(fs: Fs): Closeable = { diff --git a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/CSVExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/CSVExecutor.scala index 5f95ce9e11..a0d285483d 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/CSVExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/CSVExecutor.scala @@ -17,6 +17,7 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.executor import java.io.OutputStream + import com.webank.wedatasphere.linkis.common.io.FsPath import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext import com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.conf.PipelineEngineConfiguration.{PIPELINE_FIELD_SPLIT_STR, PIPELINE_OUTPUT_CHARSET_STR, PIPELINE_OUTPUT_ISOVERWRITE_SWITCH} @@ -39,7 +40,7 @@ class CSVExecutor extends PipeLineExecutor { throw new PipeLineErrorException(70001, "Not a result set file(不是结果集文件)") } val sourceFsPath = new FsPath(sourcePath) - val destFsPath = new FsPath(s"$destPath.$Kind") + val destFsPath = new FsPath(destPath) val sourceFs = FSFactory.getFs(sourceFsPath) sourceFs.init(null) val destFs = FSFactory.getFs(destFsPath) diff --git a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/CopyExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/CopyExecutor.scala index 0c70eea92b..d7e9bec810 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/CopyExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/CopyExecutor.scala @@ -19,7 +19,6 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.executor import com.webank.wedatasphere.linkis.common.io.FsPath import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext import com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.conf.PipelineEngineConfiguration.PIPELINE_OUTPUT_ISOVERWRITE_SWITCH -import com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.constant.PipeLineConstant.PIPELINE_OUTPUT_ISOVERWRITE import com.webank.wedatasphere.linkis.scheduler.executer.ExecuteResponse import com.webank.wedatasphere.linkis.storage.FSFactory import org.apache.commons.io.IOUtils diff --git a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/ExcelExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/ExcelExecutor.scala index b0eb8ed142..3122606b0b 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/ExcelExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/ExcelExecutor.scala @@ -17,6 +17,7 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.executor import java.io.OutputStream + import com.webank.wedatasphere.linkis.common.io.FsPath import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext import com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.conf.PipelineEngineConfiguration.PIPELINE_OUTPUT_ISOVERWRITE_SWITCH diff --git a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/PipelineEngineConnExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/PipelineEngineConnExecutor.scala index bd9d44d3a6..029a680586 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/PipelineEngineConnExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/PipelineEngineConnExecutor.scala @@ -30,7 +30,6 @@ import com.webank.wedatasphere.linkis.rpc.Sender import com.webank.wedatasphere.linkis.scheduler.executer.ExecuteResponse import scala.collection.JavaConversions.mapAsScalaMap -import scala.collection.JavaConverters._ class PipelineEngineConnExecutor(val id: Int) extends ComputationExecutor with Logging { @@ -42,23 +41,19 @@ class PipelineEngineConnExecutor(val id: Int) extends ComputationExecutor with L private val executorLabels: util.List[Label[_]] = new util.ArrayList[Label[_]]() + + + override def executeLine(engineExecutorContext: EngineExecutionContext, code: String): ExecuteResponse = { index += 1 var failedTasks = 0 var succeedTasks = 1 val newOptions = engineExecutorContext.getProperties - newOptions.foreach({case (k, v) => info(s"key is $k, value is ${v.toString}")}) - PipelineEngineConnExecutor.listPipelineExecutors.foreach(e => e.init(newOptions.map({case (k,v) => (k, v.toString)}).asJava)) + newOptions.foreach({ case (k, v) => info(s"key is $k, value is $v") }) val regex = "(?i)\\s*from\\s+(\\S+)\\s+to\\s+(\\S+)\\s?".r try { code match { - case regex(sourcePath, destPath) => { - if (destPath.contains(".")) { - PipelineEngineConnExecutor.listPipelineExecutors.find(f => "cp".equals(f.Kind)).get.execute(sourcePath, destPath,engineExecutorContext) - } else { - PipelineEngineConnExecutor.listPipelineExecutors.find(f => newOptions.get("pipeline.output.mold").toString.equalsIgnoreCase(f.Kind)).map(_.execute(sourcePath, destPath,engineExecutorContext)).get - } - } + case regex(sourcePath, destPath) => PipelineExecutorSelector.select(sourcePath,destPath,newOptions.asInstanceOf[util.Map[String, String]]).execute(sourcePath,destPath,engineExecutorContext) case _ => throw new PipeLineErrorException(70007, "非法的out脚本语句(Illegal out script statement)") } } catch { diff --git a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/PipelineExecutorSelector.scala b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/PipelineExecutorSelector.scala new file mode 100644 index 0000000000..108c48ce1b --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/executor/PipelineExecutorSelector.scala @@ -0,0 +1,47 @@ +/* + * Copyright 2019 WeBank + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.executor + +import java.io.File +import java.util + +import com.webank.wedatasphere.linkis.common.utils.{Logging, Utils} +import com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.exception.PipeLineErrorException + +/** + */ +object PipelineExecutorSelector extends Logging { + + @throws[PipeLineErrorException] + def select(sourcePath: String, destPath: String, options: util.Map[String, String]): PipeLineExecutor = { + PipelineEngineConnExecutor.listPipelineExecutors.foreach(_.init(options)) + Utils.tryCatch { + if (new File(sourcePath).getName.equals(new File(destPath).getName)) return PipelineEngineConnExecutor.listPipelineExecutors()(0) + getSuffix(destPath) match { + case ".csv" => PipelineEngineConnExecutor.listPipelineExecutors()(1) + case ".xlsx" => PipelineEngineConnExecutor.listPipelineExecutors()(2) + case _ => throw new PipeLineErrorException(70008, "unsupport output type") + } + } { + case e: Exception => error("select executor failed", e); throw new PipeLineErrorException(70008, "unsupport output type") + } + + } + + + def getSuffix(str: String): String = str.substring(str.lastIndexOf(".")) + +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/factory/PipelineEngineConnFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/factory/PipelineEngineConnFactory.scala index 709c8fdb99..56030d837c 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/factory/PipelineEngineConnFactory.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/pipeline/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/pipeline/factory/PipelineEngineConnFactory.scala @@ -18,33 +18,23 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.factory import com.webank.wedatasphere.linkis.common.utils.Logging import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconn.common.engineconn.{DefaultEngineConn, EngineConn} -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.SingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationSingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.executor.entity.LabelExecutor import com.webank.wedatasphere.linkis.manager.engineplugin.pipeline.executor.PipelineEngineConnExecutor -import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineRunTypeLabel, EngineType, RunType} +import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType.EngineType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineType, RunType} -class PipelineEngineConnFactory extends SingleExecutorEngineConnFactory with Logging { +class PipelineEngineConnFactory extends ComputationSingleExecutorEngineConnFactory with Logging { - override def createEngineConn(engineCreationContext: EngineCreationContext): EngineConn = { - val engineConn = new DefaultEngineConn(engineCreationContext) - engineConn.setEngineType(EngineType.PIPELINE.toString) - engineConn - } - override def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn): Executor = { - - val id = ExecutorManager.getInstance().generateId() - val executor = new PipelineEngineConnExecutor(id) - val runTypeLabel = getDefaultEngineRunTypeLabel() - executor.getExecutorLabels().add(runTypeLabel) - executor - } + override protected def getEngineConnType: EngineType = EngineType.PIPELINE - override def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel = { - val runTypeLabel = new EngineRunTypeLabel - runTypeLabel.setRunType(RunType.PIPELINE.toString) - runTypeLabel + override def newExecutor(id: Int, + engineCreationContext: EngineCreationContext, + engineConn: EngineConn): LabelExecutor = { + new PipelineEngineConnExecutor(id) } + override protected def getRunType: RunType = RunType.PIPELINE } diff --git a/linkis-engineconn-plugins/engineconn-plugins/python/src/main/resources/log4j2-engineconn.xml b/linkis-engineconn-plugins/engineconn-plugins/python/src/main/resources/log4j2-engineconn.xml index b68da82635..ff89c4e7d2 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/python/src/main/resources/log4j2-engineconn.xml +++ b/linkis-engineconn-plugins/engineconn-plugins/python/src/main/resources/log4j2-engineconn.xml @@ -22,12 +22,7 @@ - - - - - + @@ -47,7 +42,7 @@ - + @@ -56,6 +51,18 @@ + + + + + + + + + + + + diff --git a/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/executor/PythonEngineConnExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/executor/PythonEngineConnExecutor.scala index 7410b2daf5..18d0b244aa 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/executor/PythonEngineConnExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/executor/PythonEngineConnExecutor.scala @@ -19,8 +19,9 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.python.executor import java.util import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.{ComputationExecutor, EngineExecutionContext} -import com.webank.wedatasphere.linkis.engineconn.computation.executor.parser.PythonCodeParser import com.webank.wedatasphere.linkis.engineconn.core.EngineConnObject +import com.webank.wedatasphere.linkis.engineconn.launch.EngineConnServer +import com.webank.wedatasphere.linkis.governance.common.paser.PythonCodeParser import com.webank.wedatasphere.linkis.manager.common.entity.resource.{CommonNodeResource, LoadInstanceResource, NodeResource} import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.EngineConnPluginConf import com.webank.wedatasphere.linkis.manager.label.entity.Label @@ -44,9 +45,12 @@ class PythonEngineConnExecutor(id: Int, pythonSession: PythonSession, outputPrin super.init } + private val pythonDefaultVersion: String = EngineConnServer.getEngineCreationContext.getOptions.getOrDefault("python.version", "python") + override def executeLine(engineExecutionContext: EngineExecutionContext, code: String): ExecuteResponse = { - info(s" EngineExecutionContext user python.version = > ${engineExecutionContext.getProperties.getOrDefault("python.version", "python").toString.toLowerCase()}") - System.getProperties.put("python.version", engineExecutionContext.getProperties.getOrDefault("python.version", "python").toString.toLowerCase()) + val pythonVersion = engineExecutionContext.getProperties.getOrDefault("python.version", pythonDefaultVersion).toString.toLowerCase() + info(s" EngineExecutionContext user python.version = > ${pythonVersion}") + System.getProperties.put("python.version", pythonVersion) info(s" System getProperties python.version = > ${System.getProperties.getProperty("python.version")}") System.getProperties.put("python.application.pyFiles", engineExecutionContext.getProperties.getOrDefault("python.application.pyFiles", "file:///mnt/bdap/test/test/test.zip").toString) if(engineExecutionContext != this.engineExecutionContext){ diff --git a/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/executor/PythonSession.scala b/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/executor/PythonSession.scala index a157950a2d..7c9eec5980 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/executor/PythonSession.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/executor/PythonSession.scala @@ -22,6 +22,7 @@ import java.util.{List => JList} import com.webank.wedatasphere.linkis.common.utils.{Logging, Utils} import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext import com.webank.wedatasphere.linkis.engineconn.computation.executor.rs.RsOutputStream +import com.webank.wedatasphere.linkis.engineconn.launch.EngineConnServer import com.webank.wedatasphere.linkis.manager.engineplugin.python.conf.PythonEngineConfiguration import com.webank.wedatasphere.linkis.manager.engineplugin.python.exception.{ExecuteException, PythonExecuteError} import com.webank.wedatasphere.linkis.manager.engineplugin.python.utils.Kind @@ -53,6 +54,8 @@ class PythonSession extends Logging { private var code: String = _ private var pid: Option[String] = None private var gatewayInited = false + private val pythonDefaultVersion: String = EngineConnServer.getEngineCreationContext.getOptions.getOrDefault("python.version", "python") + def init(): Unit = { initGateway @@ -67,7 +70,7 @@ class PythonSession extends Logging { } private def initGateway = { - val userDefinePythonVersion = System.getProperties.getProperty("python.version") + val userDefinePythonVersion = Some(pythonDefaultVersion).getOrElse("python") info(s"System userDefinePythonVersion => ${userDefinePythonVersion}") val pythonExec = if ("python3".equalsIgnoreCase(userDefinePythonVersion)) PythonEngineConfiguration.PYTHON_VERSION.getValue else "python" info(s"pythonExec => ${pythonExec}") diff --git a/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/factory/PythonEngineConnFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/factory/PythonEngineConnFactory.scala index dab42fa486..f8569108d3 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/factory/PythonEngineConnFactory.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/factory/PythonEngineConnFactory.scala @@ -19,27 +19,22 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.python.factory import com.webank.wedatasphere.linkis.common.utils.{Logging, Utils} import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext import com.webank.wedatasphere.linkis.engineconn.common.engineconn.{DefaultEngineConn, EngineConn} -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.SingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationSingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.executor.entity.LabelExecutor import com.webank.wedatasphere.linkis.manager.engineplugin.python.conf.PythonEngineConfiguration import com.webank.wedatasphere.linkis.manager.engineplugin.python.exception.PythonSessionStartFailedExeception import com.webank.wedatasphere.linkis.manager.engineplugin.python.executor.{PythonEngineConnExecutor, PythonSession} -import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineRunTypeLabel, EngineType, RunType} +import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType.EngineType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineType, RunType} -class PythonEngineConnFactory extends SingleExecutorEngineConnFactory with Logging { +class PythonEngineConnFactory extends ComputationSingleExecutorEngineConnFactory with Logging { - private var engineCreationContext: EngineCreationContext = _ - - override def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn): Executor = { - engineConn.getEngine match { + override def newExecutor(id: Int, engineCreationContext: EngineCreationContext, engineConn: EngineConn): LabelExecutor = { + engineConn.getEngineConnSession match { case pythonSession: PythonSession => - this.engineCreationContext = engineCreationContext - val id = ExecutorManager.getInstance().generateId() - val executor = new PythonEngineConnExecutor(id, pythonSession, PythonEngineConfiguration.PYTHON_CONSOLE_OUTPUT_LINE_LIMIT.getValue) - executor.getExecutorLabels().add(getDefaultEngineRunTypeLabel()) - executor + new PythonEngineConnExecutor(id, pythonSession, PythonEngineConfiguration.PYTHON_CONSOLE_OUTPUT_LINE_LIMIT.getValue) case _ => throw PythonSessionStartFailedExeception("Invalid python session.") } @@ -49,14 +44,12 @@ class PythonEngineConnFactory extends SingleExecutorEngineConnFactory with Loggi val pythonSession = new PythonSession Utils.tryAndWarn(pythonSession.init()) val engineConn = new DefaultEngineConn(engineCreationContext) - engineConn.setEngineType(EngineType.PYTHON.toString) - engineConn.setEngine(pythonSession) + engineConn.setEngineConnType(EngineType.PYTHON.toString) + engineConn.setEngineConnSession(pythonSession) engineConn } - override def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel = { - val runTypeLabel = new EngineRunTypeLabel - runTypeLabel.setRunType(RunType.PYTHON.toString) - runTypeLabel - } + override protected def getEngineConnType: EngineType = EngineType.PYTHON + + override protected def getRunType: RunType = RunType.PYTHON } diff --git a/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/hook/PythonVersionEngineHook.scala b/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/hook/PythonVersionEngineHook.scala index ab17b46b41..7ba18bb30f 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/hook/PythonVersionEngineHook.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/python/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/python/hook/PythonVersionEngineHook.scala @@ -20,10 +20,8 @@ import com.webank.wedatasphere.linkis.common.utils.Logging import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn import com.webank.wedatasphere.linkis.engineconn.common.hook.EngineConnHook -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor import com.webank.wedatasphere.linkis.manager.engineplugin.python.conf.PythonEngineConfiguration import com.webank.wedatasphere.linkis.manager.engineplugin.python.executor.PythonSession -import com.webank.wedatasphere.linkis.server.JMap class PythonVersionEngineHook extends EngineConnHook with Logging{ @@ -42,12 +40,12 @@ class PythonVersionEngineHook extends EngineConnHook with Logging{ override def beforeExecutionExecute(engineCreationContext: EngineCreationContext, engineConn: EngineConn): Unit = { info("use python execute print cmd hello") - engineConn.getEngine match { + engineConn.getEngineConnSession match { case pythonSession: PythonSession => pythonSession.execute("print(1/2)") logger.info(s"print python version => ${PythonEngineConfiguration.PYTHON_VERSION.getValue}") case _ => - logger.error(s"Invalid pythonSession : ${engineConn.getEngine().getClass.getName}") + logger.error(s"Invalid pythonSession : ${engineConn.getEngineConnSession.getClass.getName}") } } diff --git a/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/resources/linkis-engineconn.properties b/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/resources/linkis-engineconn.properties index 0588a003cc..4b8cdf1b0f 100755 --- a/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/resources/linkis-engineconn.properties +++ b/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/resources/linkis-engineconn.properties @@ -16,4 +16,4 @@ wds.linkis.server.version=v1 -wds.linkis.engineconn.plugin.default.clazz=com.webank.wedatasphere.linkis.manager.engineplugin.shell.ShellEngineConnPlugin +wds.linkis.engineconn.plugin.default.class=com.webank.wedatasphere.linkis.manager.engineplugin.shell.ShellEngineConnPlugin diff --git a/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/resources/log4j2-engineconn.xml b/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/resources/log4j2-engineconn.xml index 0f5dd45478..ffacbbeb7b 100755 --- a/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/resources/log4j2-engineconn.xml +++ b/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/resources/log4j2-engineconn.xml @@ -20,12 +20,7 @@ - - - - - + @@ -54,6 +49,19 @@ - - + + + + + + + + + + + + + + + diff --git a/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/shell/executor/ShellEngineConnExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/shell/executor/ShellEngineConnExecutor.scala index a2356f3feb..358af59022 100755 --- a/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/shell/executor/ShellEngineConnExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/shell/executor/ShellEngineConnExecutor.scala @@ -57,7 +57,7 @@ class ShellEngineConnExecutor(id: Int) extends ComputationExecutor with Logging if (engineExecutionContext != this.engineExecutionContext) { this.engineExecutionContext = engineExecutionContext - info("Python executor reset new engineExecutionContext!") + info("Shell executor reset new engineExecutionContext!") } var bufferedReader: BufferedReader = null diff --git a/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/shell/factory/ShellEngineConnFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/shell/factory/ShellEngineConnFactory.scala index 439990a7e6..ff3b0de64b 100755 --- a/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/shell/factory/ShellEngineConnFactory.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/shell/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/shell/factory/ShellEngineConnFactory.scala @@ -17,33 +17,22 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.shell.factory import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconn.common.engineconn.{DefaultEngineConn, EngineConn} -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.SingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationSingleExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.engineconn.executor.entity.LabelExecutor import com.webank.wedatasphere.linkis.manager.engineplugin.shell.executor.ShellEngineConnExecutor -import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineRunTypeLabel, EngineType, RunType} +import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType.EngineType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineType, RunType} -class ShellEngineConnFactory extends SingleExecutorEngineConnFactory { +class ShellEngineConnFactory extends ComputationSingleExecutorEngineConnFactory { + override protected def newExecutor(id: Int, + engineCreationContext: EngineCreationContext, + engineConn: EngineConn): LabelExecutor = + new ShellEngineConnExecutor(id) - override def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn): Executor = { - val id = ExecutorManager.getInstance().generateId() - val executor = new ShellEngineConnExecutor(id) - val runTypeLabel = getDefaultEngineRunTypeLabel() - executor.getExecutorLabels().add(runTypeLabel) - executor - } + override protected def getEngineConnType: EngineType = EngineType.SHELL - override def createEngineConn(engineCreationContext: EngineCreationContext): EngineConn = { - val engineConn = new DefaultEngineConn(engineCreationContext) - engineConn.setEngineType(EngineType.SHELL.toString) - engineConn - } - - override def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel = { - val runTypeLabel = new EngineRunTypeLabel - runTypeLabel.setRunType(RunType.SHELL.toString) - runTypeLabel - } + override protected def getRunType: RunType = RunType.SHELL } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/pom.xml b/linkis-engineconn-plugins/engineconn-plugins/spark/pom.xml index 718b810a4b..d026621a66 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/pom.xml +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/pom.xml @@ -39,6 +39,12 @@ ${linkis.version} + + com.webank.wedatasphere + spark-excel_2.11 + ${linkis.version} + + com.webank.wedatasphere.linkis linkis-computation-engineconn @@ -254,6 +260,13 @@ + + org.apache.spark + spark-hive_2.11 + ${spark.version} + provided + + diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/linkis-engineconn.properties b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/linkis-engineconn.properties index 35697c93b7..629bc8b3b8 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/linkis-engineconn.properties +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/linkis-engineconn.properties @@ -24,6 +24,7 @@ wds.linkis.engineconn.debug.enable=true #wds.linkis.keytab.enable=true -wds.linkis.engineconn.plugin.default.clazz=com.webank.wedatasphere.linkis.engineplugin.spark.SparkEngineConnPlugin +wds.linkis.engineconn.plugin.default.class=com.webank.wedatasphere.linkis.engineplugin.spark.SparkEngineConnPlugin +wds.linkis.engine.connector.hooks=com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.ComputationEngineConnHook,com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.PyUdfEngineHook,com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.ScalaUdfEngineHook,com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.JarUdfEngineHook,com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.PyFunctionEngineHook,com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.ScalaFunctionEngineHook diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/log4j2-engineconn.xml b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/log4j2-engineconn.xml index 457ed5a99f..0fe2828a6f 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/log4j2-engineconn.xml +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/log4j2-engineconn.xml @@ -20,13 +20,11 @@ - - - - - + + + + @@ -48,12 +46,25 @@ - + - - + + + + + + + + + + + + + + + diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/python/mix_pyspark.py b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/python/mix_pyspark.py index 7137f63a93..a8af121be0 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/python/mix_pyspark.py +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/resources/python/mix_pyspark.py @@ -1,5 +1,6 @@ import sys, getopt, traceback, json, re import os +from py4j.protocol import Py4JJavaError, Py4JNetworkError os.environ['PYSPARK_ALLOW_INSECURE_GATEWAY']='1' import matplotlib import os @@ -11,7 +12,6 @@ sys.path.insert(0, paths[i]) from py4j.java_gateway import java_import, JavaGateway, GatewayClient -from py4j.protocol import Py4JJavaError from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.sql.session import SparkSession @@ -109,6 +109,12 @@ def show(obj): intp.showDF(jobGroup, obj._jdf) else: print((str(obj))) +def printlog(obj): + try: + intp.printLog(obj) + except Exception as e: + print("send log failed") + def showAlias(obj,alias): from pyspark.sql import DataFrame @@ -216,6 +222,10 @@ def existsUDF(self, name): if innerErrorStart > -1: excInnerError = excInnerError[innerErrorStart:] intp.setStatementsFinished(excInnerError + str(sys.exc_info()), True) + except Py4JNetworkError: + # lost connection from gateway server. exit + intp.setStatementsFinished(msg, True) + sys.exit(1) except: msg = traceback.format_exc() intp.setStatementsFinished(msg, True) diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/config/SparkConfiguration.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/config/SparkConfiguration.scala index d154862317..3d059d6d82 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/config/SparkConfiguration.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/config/SparkConfiguration.scala @@ -57,7 +57,7 @@ object SparkConfiguration extends Logging { val ENGINE_JAR = CommonVars[String]("wds.linkis.enginemanager.core.jar", getMainJarName) - val DEFAULT_SPARK_JAR_NAME = CommonVars[String]("wds.linkis.ecp.spark.default.jar", "linkis-engineconn-launch-1.0.0-RC1.jar") + val DEFAULT_SPARK_JAR_NAME = CommonVars[String]("wds.linkis.ecp.spark.default.jar", "linkis-engineconn-core-1.0.0.jar") val SPARK_DRIVER_CLASSPATH = CommonVars[String]("wds.linkis.spark.driver.extra.class.path", "") @@ -71,9 +71,12 @@ object SparkConfiguration extends Logging { val SQL_EXTENSION_TIMEOUT = CommonVars("wds.linkis.dws.ujes.spark.extension.timeout", 3000L) val SPARK_NF_FRACTION_LENGTH = CommonVars[Int]("wds.linkis.engine.spark.fraction.length", 30) val SHOW_DF_MAX_RES = CommonVars("wds.linkis.show.df.max.res", Int.MaxValue) - val MDQ_APPLICATION_NAME = CommonVars("wds.linkis.mdq.application.name", "linkis-ps-metadata") + val MDQ_APPLICATION_NAME = CommonVars("wds.linkis.mdq.application.name", "linkis-ps-datasource") val DOLPHIN_LIMIT_LEN = CommonVars("wds.linkis.dolphin.limit.len", 5000) + val IS_VIEWFS_ENV = CommonVars("wds.linkis.spark.engine.is.viewfs.env", true) + + private def getMainJarName(): String = { val somePath = ClassUtils.jarOfClass(classOf[SparkEngineConnFactory]) if (somePath.isDefined) { diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/configuration/SparkEngineServerSpringConfiguration.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/configuration/SparkEngineServerSpringConfiguration.scala index 5f15309687..32927c3d62 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/configuration/SparkEngineServerSpringConfiguration.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/configuration/SparkEngineServerSpringConfiguration.scala @@ -16,8 +16,6 @@ package com.webank.wedatasphere.linkis.engineplugin.spark.configuration import com.webank.wedatasphere.linkis.engineconn.common.hook.EngineConnHook -import com.webank.wedatasphere.linkis.engineconn.computation.executor.parser.CodeParser -import com.webank.wedatasphere.linkis.engineplugin.spark.executor.parser.SparkCombinedCodeParser import com.webank.wedatasphere.linkis.engineplugin.spark.hook.{SparkPythonVersionEngineHook, UserDataBaseHook} import org.springframework.context.annotation.{Bean, Configuration} @@ -26,9 +24,6 @@ import org.springframework.context.annotation.{Bean, Configuration} */ @Configuration class SparkEngineServerSpringConfiguration { - @Bean(Array("codeParser")) - def createCodeParser(): CodeParser = new SparkCombinedCodeParser() - @Bean(Array("engineHooks")) // def createEngineHooks(): Array[EngineConnHook] = Array(new ReleaseEngineHook, new MaxExecuteNumEngineHook,new SparkPythonVersionEngineHook, new JarUdfEngineHook, new PyUdfEngineHook, new ScalaUdfEngineHook, new PyFunctionEngineHook, new ScalaFunctionEngineHook,new UserDataBaseHook) diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SQLSession.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SQLSession.scala index 7978614a3b..1f041353f7 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SQLSession.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SQLSession.scala @@ -23,7 +23,7 @@ import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.En import com.webank.wedatasphere.linkis.engineplugin.spark.config.SparkConfiguration import com.webank.wedatasphere.linkis.engineplugin.spark.exception.SparkEngineException import com.webank.wedatasphere.linkis.engineplugin.spark.utils.EngineUtils -import com.webank.wedatasphere.linkis.governance.common.exception.DWCJobRetryException +import com.webank.wedatasphere.linkis.governance.common.exception.LinkisJobRetryException import com.webank.wedatasphere.linkis.storage.domain.{Column, DataType} import com.webank.wedatasphere.linkis.storage.resultset.ResultSetFactory import com.webank.wedatasphere.linkis.storage.resultset.table.{TableMetaData, TableRecord} @@ -51,7 +51,7 @@ object SQLSession extends Logging { // if (sc.isStopped) { log.error("Spark application has already stopped in showDF, please restart it.") - throw new DWCJobRetryException("Spark application sc has already stopped, please restart it.") + throw new LinkisJobRetryException("Spark application sc has already stopped, please restart it.") } val startTime = System.currentTimeMillis() // sc.setJobGroup(jobGroup, "Get IDE-SQL Results.", false) diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkEngineConnExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkEngineConnExecutor.scala index 7bb6d174d4..a1374d93a1 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkEngineConnExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkEngineConnExecutor.scala @@ -24,11 +24,11 @@ import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.{C import com.webank.wedatasphere.linkis.engineplugin.spark.common.Kind import com.webank.wedatasphere.linkis.engineplugin.spark.extension.{SparkPostExecutionHook, SparkPreExecutionHook} import com.webank.wedatasphere.linkis.engineplugin.spark.utils.JobProgressUtil -import com.webank.wedatasphere.linkis.governance.common.exception.DWCJobRetryException +import com.webank.wedatasphere.linkis.governance.common.exception.LinkisJobRetryException import com.webank.wedatasphere.linkis.manager.common.entity.enumeration.NodeStatus import com.webank.wedatasphere.linkis.manager.common.entity.resource._ import com.webank.wedatasphere.linkis.manager.label.entity.Label -import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineRunTypeLabel +import com.webank.wedatasphere.linkis.manager.label.entity.engine.CodeLanguageLabel import com.webank.wedatasphere.linkis.protocol.engine.JobProgressInfo import com.webank.wedatasphere.linkis.scheduler.executer.ExecuteResponse import org.apache.spark.SparkContext @@ -73,7 +73,7 @@ abstract class SparkEngineConnExecutor(val sc: SparkContext, id: Long) extends C if (sc.isStopped) { error("Spark application has already stopped, please restart it.") transition(NodeStatus.Failed) - throw new DWCJobRetryException("Spark application sc has already stopped, please restart it.") + throw new LinkisJobRetryException("Spark application sc has already stopped, please restart it.") } this.engineExecutionContext = engineExecutorContext oldprogress = 0f @@ -153,8 +153,8 @@ abstract class SparkEngineConnExecutor(val sc: SparkContext, id: Long) extends C // if(driverMemList.size > 0) { // driverMem = driverMemList.reduce((x, y) => x + y) // } - val sparkExecutorCores = sc.getConf.get("spark.executor.cores").toInt * executorNum - val sparkDriverCores = sc.getConf.get("spark.driver.cores").toInt + val sparkExecutorCores = sc.getConf.get("spark.executor.cores", "2").toInt * executorNum + val sparkDriverCores = sc.getConf.get("spark.driver.cores", "1").toInt val queue = sc.getConf.get("spark.yarn.queue") info("Current actual used resources is driverMem:" + driverMem + ",driverCores:" + sparkDriverCores + ",executorMem:" + executorMem + ",executorCores:" + sparkExecutorCores + ",queue:" + queue) val uesdResource = new DriverAndYarnResource( @@ -201,8 +201,8 @@ abstract class SparkEngineConnExecutor(val sc: SparkContext, id: Long) extends C protected def killRunningTask(): Unit = { var runType : String = "" getExecutorLabels().asScala.foreach {l => l match { - case label: EngineRunTypeLabel => - runType = label.getRunType + case label: CodeLanguageLabel => + runType = label.getCodeType case _ => } } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkPythonExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkPythonExecutor.scala index ff561a3450..e28aa050e7 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkPythonExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkPythonExecutor.scala @@ -18,9 +18,10 @@ import java.util import com.webank.wedatasphere.linkis.common.conf.CommonVars import com.webank.wedatasphere.linkis.common.utils.Utils +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationExecutorManager import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext -import com.webank.wedatasphere.linkis.engineconn.computation.executor.parser.PythonCodeParser import com.webank.wedatasphere.linkis.engineconn.computation.executor.rs.RsOutputStream +import com.webank.wedatasphere.linkis.engineconn.launch.EngineConnServer import com.webank.wedatasphere.linkis.engineplugin.spark.Interpreter.PythonInterpreter._ import com.webank.wedatasphere.linkis.engineplugin.spark.common.{Kind, PySpark} import com.webank.wedatasphere.linkis.engineplugin.spark.config.SparkConfiguration @@ -28,6 +29,7 @@ import com.webank.wedatasphere.linkis.engineplugin.spark.entity.SparkEngineSessi import com.webank.wedatasphere.linkis.engineplugin.spark.exception.ExecuteError import com.webank.wedatasphere.linkis.engineplugin.spark.imexport.CsvRelation import com.webank.wedatasphere.linkis.engineplugin.spark.utils.EngineUtils +import com.webank.wedatasphere.linkis.governance.common.paser.PythonCodeParser import com.webank.wedatasphere.linkis.scheduler.executer.{ExecuteResponse, SuccessExecuteResponse} import com.webank.wedatasphere.linkis.storage.resultset.ResultSetWriter import org.apache.commons.exec.CommandLine @@ -60,6 +62,7 @@ class SparkPythonExecutor(val sparkEngineSession: SparkEngineSession, val id: In implicit var sparkSession: SparkSession = sparkEngineSession.sparkSession private[spark] var engineExecutionContext: EngineExecutionContext = _ + private val engineCreationContext = EngineConnServer.getEngineCreationContext private val lineOutputStream = new RsOutputStream val sqlContext = sparkEngineSession.sqlContext val SUCCESS = "success" @@ -80,6 +83,16 @@ class SparkPythonExecutor(val sparkEngineSession: SparkEngineSession, val id: In info("spark sql executor start") } + override def killTask(taskID: String): Unit = { + info(s"Start to kill python task $taskID") + super.killTask(taskID) + info(s"To close python cli task $taskID") + Utils.tryAndError(close) + info(s"To delete python executor task $taskID") + Utils.tryAndError(ComputationExecutorManager.getInstance.removeExecutor(getExecutorLabels().asScala.toArray)) + info(s"Finished to kill python task $taskID") + } + override def close = { if (process != null) { if (gatewayServer != null) { @@ -98,8 +111,8 @@ class SparkPythonExecutor(val sparkEngineSession: SparkEngineSession, val id: In private def initGateway = { // 如果从前端获取到用户所设置的Python版本为Python3 则取Python3的环境变量,否则默认为Python2 - logger.info(s"spark.python.version => ${engineExecutionContext.getProperties.get("spark.python.version")}") - val userDefinePythonVersion = engineExecutionContext.getProperties.getOrDefault("spark.python.version","python").toString.toLowerCase() + logger.info(s"spark.python.version => ${engineCreationContext.getOptions.get("spark.python.version")}") + val userDefinePythonVersion = engineCreationContext.getOptions.get("spark.python.version").toString.toLowerCase() val sparkPythonVersion = if(null != userDefinePythonVersion && userDefinePythonVersion.equals("python3")) "python3" else "python" val pythonExec = CommonVars("PYSPARK_DRIVER_PYTHON", sparkPythonVersion).getValue @@ -194,7 +207,7 @@ class SparkPythonExecutor(val sparkEngineSession: SparkEngineSession, val id: In } /*override protected def getInitLabels(): util.List[Label[_]] = { - val runTypeLabel = new EngineRunTypeLabel + val runTypeLabel = new CodeLanguageLabel runTypeLabel.setRunType(RunType.PYSPARK.toString) val engineTypeLabel = getEngineTypeLabel val labels = new util.ArrayList[Label[_]](2) @@ -212,16 +225,16 @@ class SparkPythonExecutor(val sparkEngineSession: SparkEngineSession, val id: In } promise = Promise[String]() this.code = code + engineExecutionContext.appendStdout(s"${EngineUtils.getName} >> $code") queryLock synchronized queryLock.notify() Await.result(promise.future, Duration.Inf) lineOutputStream.flush() - engineExecutionContext.appendStdout(s"${EngineUtils.getName} >> $code") val outStr = lineOutputStream.toString() if(outStr.length >0) { val output = Utils.tryQuietly(ResultSetWriter.getRecordByRes(outStr, SparkConfiguration.SPARK_CONSOLE_OUTPUT_NUM.getValue)) val res = if(output != null) output.map(x => x.toString).toList.mkString("\n") else "" if (res.length > 0) { - engineExecutionContext.appendStdout(res) + engineExecutionContext.appendStdout(s"result is $res") } } SuccessExecuteResponse() @@ -235,8 +248,8 @@ class SparkPythonExecutor(val sparkEngineSession: SparkEngineSession, val id: In def getStatements = { queryLock synchronized {while(code == null) queryLock.wait()} - info("Prepare to deal python code, code: " + code) -// lineOutputStream.reset(this.engineExecutorContext) + info("Prepare to deal python code, code: " + code.substring(0, if (code.indexOf("\n") > 0) code.indexOf("\n") else code.length)) + // lineOutputStream.reset(this.engineExecutorContext) val request = PythonInterpretRequest(code, jobGroup) code = null request @@ -248,7 +261,12 @@ class SparkPythonExecutor(val sparkEngineSession: SparkEngineSession, val id: In if(! error) { promise.success(SUCCESS) } else { - promise.failure(new ExecuteError(40003,out)) + if (promise.isCompleted) { + info("promise is completed and should start another python gateway") + close + }else{ + promise.failure(ExecuteError(40003,out)) + } } } def appendOutput(message: String) = { @@ -264,7 +282,7 @@ class SparkPythonExecutor(val sparkEngineSession: SparkEngineSession, val id: In info(message) } else { error(message) - engineExecutionContext.appendStdout(message) + engineExecutionContext.appendStdout(s"errorMessage is $message") } } @@ -292,6 +310,15 @@ class SparkPythonExecutor(val sparkEngineSession: SparkEngineSession, val id: In override protected def getExecutorIdPreFix: String = "SparkPythonExecutor_" + def printLog(log:Any):Unit = { + if(engineExecutionContext != null){ + engineExecutionContext.appendStdout("+++++++++++++++") + engineExecutionContext.appendStdout(log.toString) + engineExecutionContext.appendStdout("+++++++++++++++") + }else{ + logger.warn("engine context is null can not send log") + } + } } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkScalaExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkScalaExecutor.scala index 9d35c22983..207ae38d39 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkScalaExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkScalaExecutor.scala @@ -19,17 +19,20 @@ package com.webank.wedatasphere.linkis.engineplugin.spark.executor import java.io.{BufferedReader, File} import com.webank.wedatasphere.linkis.common.utils.Utils +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationExecutorManager import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext -import com.webank.wedatasphere.linkis.engineconn.computation.executor.parser.ScalaCodeParser import com.webank.wedatasphere.linkis.engineconn.computation.executor.rs.RsOutputStream import com.webank.wedatasphere.linkis.engineplugin.spark.common.{Kind, SparkScala} import com.webank.wedatasphere.linkis.engineplugin.spark.config.SparkConfiguration import com.webank.wedatasphere.linkis.engineplugin.spark.entity.SparkEngineSession import com.webank.wedatasphere.linkis.engineplugin.spark.exception.{ApplicationAlreadyStoppedException, ExecuteError, SparkSessionNullException} import com.webank.wedatasphere.linkis.engineplugin.spark.utils.EngineUtils +import com.webank.wedatasphere.linkis.governance.common.paser.ScalaCodeParser import com.webank.wedatasphere.linkis.scheduler.executer.{ErrorExecuteResponse, ExecuteResponse, IncompleteExecuteResponse, SuccessExecuteResponse} import com.webank.wedatasphere.linkis.storage.resultset.ResultSetWriter import org.apache.commons.io.IOUtils +import org.apache.commons.lang.StringUtils +import org.apache.commons.lang.exception.ExceptionUtils import org.apache.spark.repl.SparkILoop import org.apache.spark.sql.{SQLContext, SparkSession} import org.apache.spark.util.SparkUtils @@ -150,8 +153,18 @@ class SparkScalaExecutor(sparkEngineSession: SparkEngineSession, id: Long) exten throw new ApplicationAlreadyStoppedException(40004,"Spark application has already stopped, please restart it.") } executeCount += 1 + val originalOut = System.out val result = scala.Console.withOut(lineOutputStream) { - Utils.tryCatch(sparkILoop.interpret(code)){ t => Results.Error} match { + Utils.tryCatch(sparkILoop.interpret(code)){ t => + error("task error info:", t) + val msg = ExceptionUtils.getRootCauseMessage(t) + if (msg.contains("OutOfMemoryError")) { + error("engine oom now to set status to shutdown") + ComputationExecutorManager.getInstance.getReportExecutor.tryShutdown() + } + engineExecutionContext.appendStdout("task error info: " + msg) + Results.Error + } match { case Results.Success => lineOutputStream.flush() engineExecutionContext.appendStdout("scala> " + code) @@ -170,12 +183,19 @@ class SparkScalaExecutor(sparkEngineSession: SparkEngineSession, id: Long) exten case Results.Error => lineOutputStream.flush() val output = lineOutputStream.toString - val errorMsg = Utils.tryCatch(EngineUtils.getResultStrByDolphinContent(output))(t => t.getMessage) - error("Execute code error for "+ errorMsg) IOUtils.closeQuietly(lineOutputStream) - ErrorExecuteResponse("",new ExecuteError(40005, "execute sparkScala failed!")) + var errorMsg: String = null + if (StringUtils.isNotBlank(output)) { + errorMsg = Utils.tryCatch(EngineUtils.getResultStrByDolphinTextContent(output))(t => t.getMessage) + error("Execute code error for "+ errorMsg) + } else { + error("No error message is captured, please see the detailed log") + } + ErrorExecuteResponse(errorMsg, ExecuteError(40005, "execute sparkScala failed!")) } } + // reset the java stdout + System.setOut(originalOut) result } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkSqlExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkSqlExecutor.scala index a02ae8630d..b799c21b57 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkSqlExecutor.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/SparkSqlExecutor.scala @@ -20,12 +20,12 @@ import java.lang.reflect.InvocationTargetException import com.webank.wedatasphere.linkis.common.utils.Utils import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext -import com.webank.wedatasphere.linkis.engineconn.computation.executor.parser.SQLCodeParser import com.webank.wedatasphere.linkis.engineplugin.spark.common.{Kind, SparkSQL} import com.webank.wedatasphere.linkis.engineplugin.spark.config.SparkConfiguration import com.webank.wedatasphere.linkis.engineplugin.spark.entity.SparkEngineSession import com.webank.wedatasphere.linkis.engineplugin.spark.extension.SparkSqlExtension import com.webank.wedatasphere.linkis.engineplugin.spark.utils.EngineUtils +import com.webank.wedatasphere.linkis.governance.common.paser.SQLCodeParser import com.webank.wedatasphere.linkis.scheduler.executer.{ErrorExecuteResponse, ExecuteResponse, SuccessExecuteResponse} import org.apache.commons.lang.exception.ExceptionUtils diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/parser/SparkCombinedCodeParser.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/parser/SparkCombinedCodeParser.scala index d0b458789e..1b5978e43f 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/parser/SparkCombinedCodeParser.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/executor/parser/SparkCombinedCodeParser.scala @@ -17,7 +17,7 @@ package com.webank.wedatasphere.linkis.engineplugin.spark.executor.parser import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext -import com.webank.wedatasphere.linkis.engineconn.computation.executor.parser._ +import com.webank.wedatasphere.linkis.governance.common.paser.{CombinedEngineCodeParser, PythonCodeParser, SQLCodeParser, ScalaCodeParser, SingleCodeParser} /** * @@ -25,8 +25,9 @@ import com.webank.wedatasphere.linkis.engineconn.computation.executor.parser._ class SparkCombinedCodeParser extends CombinedEngineCodeParser { override val parsers: Array[SingleCodeParser] = Array(new PythonCodeParser, new SQLCodeParser, new ScalaCodeParser) - override def getCodeType(code: String, engineExecutionContext: EngineExecutionContext): String = { + override def getCodeType(code: String, ): String = { if (engineExecutionContext.getProperties.get("runType") != null) engineExecutionContext.getProperties.get("runType").asInstanceOf[String] else "sql" } } +*/ diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkEngineConnFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkEngineConnFactory.scala index a6a0651565..e2360ae02c 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkEngineConnFactory.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkEngineConnFactory.scala @@ -23,12 +23,12 @@ import java.util import com.webank.wedatasphere.linkis.common.conf.CommonVars import com.webank.wedatasphere.linkis.common.utils.{Logging, Utils} import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconn.common.engineconn.{DefaultEngineConn, EngineConn} import com.webank.wedatasphere.linkis.engineplugin.spark.config.SparkConfiguration import com.webank.wedatasphere.linkis.engineplugin.spark.entity.SparkEngineSession import com.webank.wedatasphere.linkis.engineplugin.spark.exception.{SparkCreateFileException, SparkSessionNullException} -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.MultiExecutorEngineConnFactory +import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.{ExecutorFactory, MultiExecutorEngineConnFactory} import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType.EngineType import com.webank.wedatasphere.linkis.server.JMap import org.apache.commons.lang.StringUtils import org.apache.spark.sql.{SQLContext, SparkSession} @@ -37,12 +37,10 @@ import org.apache.spark.{SparkConf, SparkContext} /** * - * @Date 2020/10/23 */ class SparkEngineConnFactory extends MultiExecutorEngineConnFactory with Logging { - - override def createEngineConn(engineCreationContext: EngineCreationContext): EngineConn = { + override protected def createEngineConnSession(engineCreationContext: EngineCreationContext): Any = { val options = engineCreationContext.getOptions val useSparkSubmit = true val sparkConf: SparkConf = new SparkConf(true).setAppName(options.getOrDefault("spark.app.name", "EngineConn-Spark")) @@ -83,11 +81,7 @@ class SparkEngineConnFactory extends MultiExecutorEngineConnFactory with Logging sc.hadoopConfiguration.set("mapred.output.compress", SparkConfiguration.MAPRED_OUTPUT_COMPRESS.getValue(options)) sc.hadoopConfiguration.set("mapred.output.compression.codec", SparkConfiguration.MAPRED_OUTPUT_COMPRESSION_CODEC.getValue(options)) println("Application report for " + sc.applicationId) - val sparkEngineSession = SparkEngineSession(sc, sqlContext, sparkSession, outputDir) - val engineConn = new DefaultEngineConn(engineCreationContext) - engineConn.setEngineType(EngineType.SPARK.toString) - engineConn.setEngine(sparkEngineSession) - engineConn + SparkEngineSession(sc, sqlContext, sparkSession, outputDir) } def createSparkSession(outputDir: File, conf: SparkConf, addPythonSupport: Boolean = false): SparkSession = { @@ -124,7 +118,7 @@ class SparkEngineConnFactory extends MultiExecutorEngineConnFactory with Logging builder.enableHiveSupport().getOrCreate() } - def createSQLContext(sc: SparkContext,options: JMap[String, String], sparkSession: SparkSession) = { + def createSQLContext(sc: SparkContext,options: JMap[String, String], sparkSession: SparkSession): SQLContext = { var sqlc : SQLContext = null if (SparkConfiguration.LINKIS_SPARK_USEHIVECONTEXT.getValue(options)) { val name = "org.apache.spark.sql.hive.HiveContext" @@ -133,10 +127,9 @@ class SparkEngineConnFactory extends MultiExecutorEngineConnFactory with Logging hc = getClass.getClassLoader.loadClass(name).getConstructor(classOf[SparkContext]) sqlc = hc.newInstance(sc).asInstanceOf[SQLContext] } catch { - case e: Throwable => { + case e: Throwable => logger.warn("Can't create HiveContext. Fallback to SQLContext", e) sqlc = sparkSession.sqlContext - } } } else sqlc = sparkSession.sqlContext @@ -158,4 +151,14 @@ class SparkEngineConnFactory extends MultiExecutorEngineConnFactory with Logging }) } + override protected def getDefaultExecutorFactoryClass: Class[_ <: ExecutorFactory] = + classOf[SparkSqlExecutorFactory] + + override protected def getEngineConnType: EngineType = EngineType.SPARK + + private val executorFactoryArray = Array[ExecutorFactory](new SparkSqlExecutorFactory, new SparkPythonExecutorFactory, new SparkScalaExecutorFactory) + + override def getExecutorFactories: Array[ExecutorFactory] = { + executorFactoryArray + } } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkPythonExecutorFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkPythonExecutorFactory.scala index 08923d5495..787be32698 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkPythonExecutorFactory.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkPythonExecutorFactory.scala @@ -18,49 +18,32 @@ package com.webank.wedatasphere.linkis.engineplugin.spark.factory import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationExecutorFactory +import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.ComputationExecutor import com.webank.wedatasphere.linkis.engineplugin.spark.entity.SparkEngineSession import com.webank.wedatasphere.linkis.engineplugin.spark.exception.NotSupportSparkPythonTypeException -import com.webank.wedatasphere.linkis.engineplugin.spark.executor.{SparkExecutorOrder, SparkPythonExecutor} -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.ExecutorFactory +import com.webank.wedatasphere.linkis.engineplugin.spark.executor.SparkPythonExecutor import com.webank.wedatasphere.linkis.manager.label.entity.Label -import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineRunTypeLabel, RunType} +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType /** * * @date 2020/11/2 */ -class SparkPythonExecutorFactory extends ExecutorFactory { - /** - * Order of executors, the smallest one is the default - * - * @return - */ - override def getOrder: Int = SparkExecutorOrder.PYSPARK.id +class SparkPythonExecutorFactory extends ComputationExecutorFactory { - /** - * - * @param engineCreationContext - * @param engineConn - * @param labels - * @return - */ - override def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn, labels: Array[Label[_]]): Executor = { - engineConn.getEngine match { + override protected def newExecutor(id: Int, + engineCreationContext: EngineCreationContext, + engineConn: EngineConn, + labels: Array[Label[_]]): ComputationExecutor = { + engineConn.getEngineConnSession match { case sparkEngineSession: SparkEngineSession => - val id = ExecutorManager.getInstance().generateId() - val executor = new SparkPythonExecutor(sparkEngineSession, id) - executor.getExecutorLabels().add(getDefaultEngineRunTypeLabel()) - executor + new SparkPythonExecutor(sparkEngineSession, id) case _ => throw NotSupportSparkPythonTypeException("Invalid EngineConn engine session obj, failed to create sparkPython executor") } } - override def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel = { - val runTypeLabel = new EngineRunTypeLabel - runTypeLabel.setRunType(RunType.PYSPARK.toString) - runTypeLabel - } + override protected def getRunType: RunType = RunType.PYSPARK } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkScalaExecutorFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkScalaExecutorFactory.scala index abfae71ed4..842bf1d0d8 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkScalaExecutorFactory.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkScalaExecutorFactory.scala @@ -18,57 +18,32 @@ package com.webank.wedatasphere.linkis.engineplugin.spark.factory import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationExecutorFactory +import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.ComputationExecutor +import com.webank.wedatasphere.linkis.engineplugin.spark.common.SparkKind import com.webank.wedatasphere.linkis.engineplugin.spark.entity.SparkEngineSession -import com.webank.wedatasphere.linkis.engineplugin.spark.executor.{SparkExecutorOrder, SparkScalaExecutor} -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.ExecutorFactory +import com.webank.wedatasphere.linkis.engineplugin.spark.executor.SparkScalaExecutor import com.webank.wedatasphere.linkis.manager.label.entity.Label -import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineRunTypeLabel, RunType} +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType /** * - * @date 2020/11/2 */ -class SparkScalaExecutorFactory extends ExecutorFactory { - /** - * Order of executors, the smallest one is the default - * - * @return - */ - override def getOrder: Int = SparkExecutorOrder.SCALA.id +class SparkScalaExecutorFactory extends ComputationExecutorFactory { - override def canCreate(labels: Array[Label[_]]): Boolean = { - labels.foreach(l => l match { - case label: EngineRunTypeLabel => - if (RunType.SCALA.toString.equalsIgnoreCase(label.getRunType)) { - return true - } - case _ => - }) - false - } + override protected def getSupportRunTypes: Array[String] = Array(RunType.SCALA.toString, + SparkKind.FUNCTION_MDQ_TYPE) - /** - * - * @param engineCreationContext - * @param engineConn - * @param labels - * @return - */ - override def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn, labels: Array[Label[_]]): Executor = { - engineConn.getEngine match { + override protected def newExecutor(id: Int, + engineCreationContext: EngineCreationContext, + engineConn: EngineConn, + labels: Array[Label[_]]): ComputationExecutor = { + engineConn.getEngineConnSession match { case sparkEngineSession: SparkEngineSession => - val id = ExecutorManager.getInstance().generateId() - val executor = new SparkScalaExecutor(sparkEngineSession, id) - executor.getExecutorLabels().add(getDefaultEngineRunTypeLabel()) - executor + new SparkScalaExecutor(sparkEngineSession, id) } } - override def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel = { - val runTypeLabel = new EngineRunTypeLabel - runTypeLabel.setRunType(RunType.SCALA.toString) - runTypeLabel - } + override protected def getRunType: RunType = RunType.SCALA } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkSqlExecutorFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkSqlExecutorFactory.scala index db2d7b7420..c977c0216e 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkSqlExecutorFactory.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/factory/SparkSqlExecutorFactory.scala @@ -18,49 +18,31 @@ package com.webank.wedatasphere.linkis.engineplugin.spark.factory import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn -import com.webank.wedatasphere.linkis.engineconn.core.executor.ExecutorManager -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor +import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationExecutorFactory +import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.ComputationExecutor import com.webank.wedatasphere.linkis.engineplugin.spark.entity.SparkEngineSession import com.webank.wedatasphere.linkis.engineplugin.spark.exception.NotSupportSparkSqlTypeException -import com.webank.wedatasphere.linkis.engineplugin.spark.executor.{SparkExecutorOrder, SparkSqlExecutor} -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.ExecutorFactory +import com.webank.wedatasphere.linkis.engineplugin.spark.executor.SparkSqlExecutor import com.webank.wedatasphere.linkis.manager.label.entity.Label -import com.webank.wedatasphere.linkis.manager.label.entity.engine.{EngineRunTypeLabel, RunType} +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType +import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType /** * - * @date 2020/11/2 */ -class SparkSqlExecutorFactory extends ExecutorFactory { - /** - * Order of executors, the smallest one is the default - * - * @return - */ - override def getOrder: Int = SparkExecutorOrder.SQL.id +class SparkSqlExecutorFactory extends ComputationExecutorFactory { - /** - * - * @param engineCreationContext - * @param engineConn - * @param labels - * @return - */ - override def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn, labels: Array[Label[_]]): Executor = { - engineConn.getEngine match { + override protected def newExecutor(id: Int, + engineCreationContext: EngineCreationContext, + engineConn: EngineConn, + label: Array[Label[_]]): ComputationExecutor = { + engineConn.getEngineConnSession match { case sparkEngineSession: SparkEngineSession => - val id = ExecutorManager.getInstance().generateId() - val executor = new SparkSqlExecutor(sparkEngineSession, id) - executor.getExecutorLabels().add(getDefaultEngineRunTypeLabel()) - executor + new SparkSqlExecutor(sparkEngineSession, id) case _ => throw NotSupportSparkSqlTypeException("Invalid EngineConn engine session obj, failed to create sparkSql executor") } } - override def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel = { - val runTypeLabel = new EngineRunTypeLabel - runTypeLabel.setRunType(RunType.SQL.toString) - runTypeLabel - } + override protected def getRunType: RunType = RunType.SQL } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/imexport/ExportData.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/imexport/ExportData.scala index adaa02d8ad..3a076a60e8 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/imexport/ExportData.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/imexport/ExportData.scala @@ -17,10 +17,11 @@ package com.webank.wedatasphere.linkis.engineplugin.spark.imexport import com.webank.wedatasphere.linkis.common.utils.Logging +import com.webank.wedatasphere.linkis.engineplugin.spark.config.SparkConfiguration import com.webank.wedatasphere.linkis.engineplugin.spark.imexport.util.BackGroundServiceUtils import org.apache.spark.sql.SparkSession -import org.json4s.{DefaultFormats, _} import org.json4s.jackson.JsonMethods._ +import org.json4s.{DefaultFormats, _} /** @@ -47,8 +48,8 @@ object ExportData extends Logging { val pathType = LoadData.getMapValue[String](dest, "pathType", "share") val path = if ("share".equals(pathType)) "file://" + LoadData.getMapValue[String](dest, "path") - else - "hdfs://" + LoadData.getMapValue[String](dest, "path") + else if (SparkConfiguration.IS_VIEWFS_ENV.getValue) LoadData.getMapValue[String](dest, "path") + else "hdfs://" + LoadData.getMapValue[String](dest, "path") val hasHeader = LoadData.getMapValue[Boolean](dest, "hasHeader", false) val isCsv = LoadData.getMapValue[Boolean](dest, "isCsv", true) diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/imexport/LoadData.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/imexport/LoadData.scala index 9d36e4c6fb..43ece57ee8 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/imexport/LoadData.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/imexport/LoadData.scala @@ -16,6 +16,7 @@ package com.webank.wedatasphere.linkis.engineplugin.spark.imexport import java.io.{BufferedInputStream, File, FileInputStream} import com.webank.wedatasphere.linkis.common.utils.Utils +import com.webank.wedatasphere.linkis.engineplugin.spark.config.SparkConfiguration import com.webank.wedatasphere.linkis.engineplugin.spark.imexport.util.{BackGroundServiceUtils, ImExportUtils} import com.webank.wedatasphere.linkis.hadoop.common.conf.HadoopConf import com.webank.wedatasphere.linkis.hadoop.common.utils.HDFSUtils @@ -90,7 +91,7 @@ object LoadData { path = XlsUtils.excelToCsv(fs.open(new Path(path)), fs, hasHeader, sheetNames) hasHeader = false } else { - path = "hdfs://" + path + path = if (SparkConfiguration.IS_VIEWFS_ENV.getValue) path else "hdfs://" + path } } else { if (".xlsx".equalsIgnoreCase(suffix)) { @@ -121,7 +122,7 @@ object LoadData { .schema(StructType(getFields(columns))) .load(path) } else { - CsvRelation.csvToDF(spark, StructType(getFields(columns)), hasHeader, path, source,columns) + CsvRelation.csvToDF(spark, StructType(getFields(columns)), hasHeader, path, source,columns) } // warn(s"Fetched ${df.columns.length} col(s) : ${df.count()} row(s).") df.createOrReplaceTempView("tempTable") @@ -188,7 +189,7 @@ object LoadData { case JNothing => default case value: JValue => if("JString()".equals(value.toString)) default - else try value.extract[T] catch { case t: Throwable => default} + else try value.extract[T] catch { case t: Throwable => default} } } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/launch/SparkSubmitProcessEngineConnLaunchBuilder.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/launch/SparkSubmitProcessEngineConnLaunchBuilder.scala index 7ded1561ba..c749630984 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/launch/SparkSubmitProcessEngineConnLaunchBuilder.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/launch/SparkSubmitProcessEngineConnLaunchBuilder.scala @@ -17,7 +17,9 @@ package com.webank.wedatasphere.linkis.engineplugin.spark.launch import java.lang.ProcessBuilder.Redirect +import java.util +import com.google.common.collect.Lists import com.webank.wedatasphere.linkis.engineplugin.spark.config.{SparkConfiguration, SparkResourceConfiguration} import com.webank.wedatasphere.linkis.engineplugin.spark.launch.SparkSubmitProcessEngineConnLaunchBuilder.{AbsolutePath, Path, RelativePath} import com.webank.wedatasphere.linkis.manager.common.entity.resource.{DriverAndYarnResource, NodeResource} @@ -174,10 +176,10 @@ class SparkSubmitProcessEngineConnLaunchBuilder private extends JavaProcessEngin case (key, value) => if (key.startsWith("spark.")) { // subcommand cannot be quoted by double quote, use single quote instead - addOpt("--conf", Some(key + "='" + value + "'")) + addOpt("--conf", Some(key + "=\"" + value + "\"")) } else if (key.startsWith("hive.")) { - addOpt("--hiveconf", Some(key + "='" + value + "'")) + addOpt("--hiveconf", Some(key + "=\"" + value + "\"")) } } addOpt("--driver-memory", _driverMemory) @@ -324,7 +326,7 @@ class SparkSubmitProcessEngineConnLaunchBuilder private extends JavaProcessEngin driverJavaSet.append(s" -agentlib:jdwp=transport=dt_socket,server=y,suspend=n,address=${variable(RANDOM_PORT)}") } this.conf(SparkConfiguration.SPARK_DRIVER_EXTRA_JAVA_OPTIONS.key, driverJavaSet.toString()) - this.conf("spark.sql.extensions", "com.webank.wedatasphere.linkis.hook.spark.extension.SparkHistoryExtension") + //this.conf("spark.sql.extensions", "com.webank.wedatasphere.linkis.hook.spark.extension.SparkHistoryExtension") this.name(properties.getOrDefault("appName", "linkis")) this.className(properties.getOrDefault("className", getMainClass)) properties.getOrDefault("archives", "").toString.split(",").map(RelativePath).foreach(this.archive) @@ -384,6 +386,10 @@ class SparkSubmitProcessEngineConnLaunchBuilder private extends JavaProcessEngin } } + override protected def getEngineConnManagerHooks(implicit engineConnBuildRequest: EngineConnBuildRequest): util.List[String] = { + Lists.newArrayList("JarUDFLoadECMHook") + } + } object SparkSubmitProcessEngineConnLaunchBuilder { diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/mdq/MDQPostExecutionHook.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/mdq/MDQPostExecutionHook.scala index e7f2b41f74..afc7803c6f 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/mdq/MDQPostExecutionHook.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/mdq/MDQPostExecutionHook.scala @@ -20,10 +20,12 @@ import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.En import com.webank.wedatasphere.linkis.engineplugin.spark.common.SparkKind import com.webank.wedatasphere.linkis.engineplugin.spark.config.SparkConfiguration import com.webank.wedatasphere.linkis.engineplugin.spark.extension.SparkPostExecutionHook +import com.webank.wedatasphere.linkis.manager.label.entity.engine.CodeLanguageLabel import com.webank.wedatasphere.linkis.protocol.mdq.{DDLCompleteResponse, DDLExecuteResponse} import com.webank.wedatasphere.linkis.rpc.Sender import com.webank.wedatasphere.linkis.scheduler.executer.{ExecuteResponse, SuccessExecuteResponse} import com.webank.wedatasphere.linkis.storage.utils.StorageUtils + import javax.annotation.PostConstruct import org.apache.commons.lang.StringUtils import org.springframework.stereotype.Component @@ -40,8 +42,9 @@ class MDQPostExecutionHook extends SparkPostExecutionHook with Logging{ override def hookName: String = "MDQPostHook" override def callPostExecutionHook(engineExecutionContext: EngineExecutionContext, executeResponse: ExecuteResponse, code: String): Unit = { - val runType: String = engineExecutionContext.getProperties.get("runType") match { - case value:String => value + val codeLanguageLabel = engineExecutionContext.getLabels.filter(l => null != l && l.isInstanceOf[CodeLanguageLabel]).head + val runType: String = codeLanguageLabel match { + case l: CodeLanguageLabel => l.getCodeType case _ => "" } if(StringUtils.isEmpty(runType) || ! SparkKind.FUNCTION_MDQ_TYPE.equalsIgnoreCase(runType)) return diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/mdq/MDQPreExecutionHook.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/mdq/MDQPreExecutionHook.scala index 765ad36d53..8ad07e7a5a 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/mdq/MDQPreExecutionHook.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/mdq/MDQPreExecutionHook.scala @@ -16,16 +16,18 @@ package com.webank.wedatasphere.linkis.engineplugin.spark.mdq import java.util - -import com.webank.wedatasphere.linkis.common.utils.Logging +import com.webank.wedatasphere.linkis.common.utils.{Logging, Utils} import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext import com.webank.wedatasphere.linkis.engineplugin.spark.common.SparkKind import com.webank.wedatasphere.linkis.engineplugin.spark.config.SparkConfiguration import com.webank.wedatasphere.linkis.engineplugin.spark.exception.MDQErrorException import com.webank.wedatasphere.linkis.engineplugin.spark.extension.SparkPreExecutionHook +import com.webank.wedatasphere.linkis.manager.label.entity.engine.CodeLanguageLabel +import com.webank.wedatasphere.linkis.manager.label.utils.LabelUtil import com.webank.wedatasphere.linkis.protocol.mdq.{DDLRequest, DDLResponse} import com.webank.wedatasphere.linkis.rpc.Sender import com.webank.wedatasphere.linkis.storage.utils.StorageUtils + import javax.annotation.PostConstruct import org.springframework.stereotype.Component import org.springframework.util.StringUtils @@ -43,18 +45,30 @@ class MDQPreExecutionHook extends SparkPreExecutionHook with Logging { override def callPreExecutionHook(engineExecutionContext: EngineExecutionContext, code: String): String = { - val runType: String = engineExecutionContext.getProperties.get("runType") match { - case value:String => value - case _ => "" + val codeLanguageLabel = engineExecutionContext.getLabels.filter(l => null != l && l.isInstanceOf[CodeLanguageLabel]).head + val runType: String = codeLanguageLabel match { + case l: CodeLanguageLabel => + l.getCodeType + case _ => + "" } if(StringUtils.isEmpty(runType) || ! SparkKind.FUNCTION_MDQ_TYPE.equalsIgnoreCase(runType)) return code val sender = Sender.getSender(SparkConfiguration.MDQ_APPLICATION_NAME.getValue) val params = new util.HashMap[String,Object]() params.put("user", StorageUtils.getJvmUser) params.put("code", code) - sender.ask(DDLRequest(params)) match { + var resp: Any = null + Utils.tryCatch { + resp = sender.ask(DDLRequest(params)) + } { + case e: Exception => + error(s"Call MDQ rpc failed, ${e.getMessage}", e) + throw new MDQErrorException(40010, s"向MDQ服务请求解析为可以执行的sql时失败, ${e.getMessage}") + } + resp match { case DDLResponse(postCode) => postCode - case _ => throw new MDQErrorException(40010, "向MDQ服务请求解析为可以执行的sql时失败") + case _ => + throw new MDQErrorException(40010, s"向MDQ服务请求解析为可以执行的sql时失败") } } } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/HPOBuilder.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/HPOBuilder.scala new file mode 100644 index 0000000000..85b87009bf --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/HPOBuilder.scala @@ -0,0 +1,77 @@ +/* + * Copyright 2019 WeBank + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.webank.wedatasphere.linkis.engineplugin.spark.metadata + +import java.util.{List => JList} + +import com.webank.wedatasphere.linkis.cs.common.entity.history.metadata.TableOperationType +import com.webank.wedatasphere.linkis.cs.common.entity.metadata.CSColumn +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType + +/** + * + */ +object HPOBuilder { + + def apply( + hivePrivilegeObjectType: HivePrivilegeObjectType, + dbname: String, + objectName: String, + partKeys: JList[String], + columns: JList[CSColumn], + commandParams: JList[String]): SparkHiveObject = { + apply( + hivePrivilegeObjectType, dbname, objectName, partKeys, columns, TableOperationType.ACCESS, commandParams) + } + + def apply( + hivePrivilegeObjectType: HivePrivilegeObjectType, + dbname: String, + objectName: String, + partKeys: JList[String], + columns: JList[CSColumn], + actionType: TableOperationType, + commandParams: JList[String]): SparkHiveObject = { + SparkHiveObject( + hivePrivilegeObjectType, dbname, objectName, partKeys, columns, actionType, commandParams) + } + + def apply( + hivePrivilegeObjectType: HivePrivilegeObjectType, + dbname: String, + objectName: String, + partKeys: JList[String], + columns: JList[CSColumn]): SparkHiveObject = { + apply( + hivePrivilegeObjectType, dbname, objectName, partKeys, columns, TableOperationType.ACCESS, null) + } + + def apply( + hivePrivilegeObjectType: HivePrivilegeObjectType, + dbname: String, + objectName: String): SparkHiveObject = { + apply(hivePrivilegeObjectType, dbname, objectName, TableOperationType.ACCESS) + } + + def apply( + hivePrivilegeObjectType: HivePrivilegeObjectType, + dbname: String, + objectName: String, + actionType: TableOperationType): SparkHiveObject = { + apply(hivePrivilegeObjectType, dbname, objectName, null, null, actionType, null) + } +} + diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/LineageResolveLogic.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/LineageResolveLogic.scala new file mode 100644 index 0000000000..b9712b7e1c --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/LineageResolveLogic.scala @@ -0,0 +1,184 @@ +package com.webank.wedatasphere.linkis.engineplugin.spark.metadata + +import java.util + +import com.webank.wedatasphere.linkis.common.utils.Logging +import org.apache.commons.lang.StringUtils +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.execution.command.CreateTableLikeCommand +import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation} +import org.apache.spark.sql.hive.execution.{CreateHiveTableAsSelectCommand, InsertIntoHiveTable} + +/** + * + * Description: + */ +/* + * Copyright 2019 WeBank + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +object LineageResolveLogic extends Logging{ + + + + def resolveLogicPlan(plan: LogicalPlan, currentDB:String): (util.Set[TableInfo], util.Set[TableInfo]) ={ + val inputTables = new util.HashSet[TableInfo]() + val outputTables = new util.HashSet[TableInfo]() + resolveLogic(plan, currentDB, inputTables, outputTables) + Tuple2(inputTables, outputTables) + } + + def resolveLogic(plan: LogicalPlan, currentDB:String, inputTables:util.Set[TableInfo], outputTables:util.Set[TableInfo]): Unit ={ + plan match { + + case plan: Project => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: Union => + for(child <- plan.children){ + resolveLogic(child, currentDB, inputTables, outputTables) + } + + case plan: Join => + resolveLogic(plan.left, currentDB, inputTables, outputTables) + resolveLogic(plan.right, currentDB, inputTables, outputTables) + + case plan: Aggregate => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: Filter => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: Generate => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: RepartitionByExpression => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: SerializeFromObject => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: MapPartitions => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: DeserializeToObject => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: Repartition => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: Deduplicate => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: Window => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: MapElements => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: TypedFilter => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: Distinct => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: GlobalLimit => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: LocalLimit => + resolveLogic(plan.child, currentDB, inputTables, outputTables) + + case plan: SubqueryAlias => + val childInputTables = new util.HashSet[TableInfo]() + val childOutputTables = new util.HashSet[TableInfo]() + + resolveLogic(plan.child, currentDB, childInputTables, childOutputTables) + if(childInputTables.size()>0){ + inputTables.addAll(childInputTables) + }else{ + inputTables.add(TableInfo(currentDB, plan.alias)) + } + + case plan: UnresolvedRelation => + val tableInfo = TableInfo(plan.tableIdentifier.database.getOrElse(currentDB), plan.tableIdentifier.table) + inputTables.add(tableInfo) + + case plan: InsertIntoTable => + resolveLogic(plan.table, currentDB, outputTables, inputTables) + resolveLogic(plan.query, currentDB, inputTables, outputTables) + + case plan: CreateTable => + if(plan.query.isDefined){ + resolveLogic(plan.query.get, currentDB, inputTables, outputTables) + } + val tableIdentifier = plan.tableDesc.identifier + val tableInfo = TableInfo(tableIdentifier.database.getOrElse(currentDB), tableIdentifier.table) + outputTables.add(tableInfo) + + case plan:CreateTableLikeCommand => + val outputTable = TableInfo(plan.targetTable.database.getOrElse(currentDB),plan.targetTable.table) + val inputTable = TableInfo(plan.sourceTable.database.getOrElse(currentDB),plan.sourceTable.table) + outputTables.add(outputTable) + inputTables.add(inputTable) + + /** + * Spark 2.4.3 + */ + case plan:CreateHiveTableAsSelectCommand => + if(plan.query != null){ + resolveLogic(plan.query, currentDB, inputTables, outputTables) + } + val tableIdentifier = plan.tableDesc.identifier + val tableInfo = TableInfo(tableIdentifier.database.getOrElse(currentDB), tableIdentifier.table) + outputTables.add(tableInfo) + + case plan:InsertIntoHiveTable => + if(plan.query != null){ + resolveLogic(plan.query, currentDB, inputTables, outputTables) + } + val tableIdentifier = plan.table.identifier + val tableInfo = TableInfo(tableIdentifier.database.getOrElse(currentDB), tableIdentifier.table) + outputTables.add(tableInfo) + /* spark2.1.0 + case plan: CatalogRelation => + val identifier = plan.tableMeta.identifier + val tableInfo = TableInfo(identifier.database.getOrElse(currentDB), identifier.table) + inputTables.add(tableInfo)*/ + + case plan: LogicalRelation => + val identifier = plan.catalogTable.get + info(s"one identifier is ${identifier.database}") + info(s"two identifier is ${identifier.identifier.database.get}") + val inputTable = TableInfo(identifier.database,identifier.identifier.table) + inputTables.add(inputTable) + + case `plan` => /*logger.info("******child plan******:\n"+plan)*/ + } + } + + +} + +case class TableInfo(db:String,table:String){ + override def toString: String = { + if(StringUtils.isNotEmpty(db)){ + db + "." + table + } else { + table + } + } +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/MetaDataInfoTool.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/MetaDataInfoTool.scala new file mode 100644 index 0000000000..9488c753bb --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/MetaDataInfoTool.scala @@ -0,0 +1,47 @@ +/* + * Copyright 2019 WeBank + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.webank.wedatasphere.linkis.engineplugin.spark.metadata + + +import com.webank.wedatasphere.linkis.common.utils.Logging +import org.apache.spark.sql.{DataFrame, Dataset, SQLContext, SparkLogicalPlanHelper} + +/** + * + * Description: + * 1.先执行sql,获取dataframe + * 2.通过dataframe的queryExecution对logicPlan进行分析 + */ +class MetaDataInfoTool extends Logging{ + def getMetaDataInfo(sqlContext:SQLContext, sql:String, dataFrame:DataFrame):String = { + info(s"begin to get sql metadata info: ${cutSql(sql)}") + val startTime = System.currentTimeMillis + val inputTables = SparkLogicalPlanHelper.extract(sqlContext, sql, dataFrame.queryExecution, startTime) + info(s"end to get sql metadata info: ${cutSql(sql)}, metadata is ${inputTables}") + if (inputTables != null) inputTables.toString else "" + } + + private def cutSql(sql:String):String = { + if (sql.length >= 1024) sql.substring(0, 1024) else sql + } +} + + +object MetaDataInfoTool{ + def getMetaDataInfo(sqlContext:SQLContext, sql:String, dataFrame:DataFrame):String = { + new MetaDataInfoTool().getMetaDataInfo(sqlContext, sql, dataFrame) + } +} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkHiveObject.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkHiveObject.scala new file mode 100644 index 0000000000..82aef90940 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkHiveObject.scala @@ -0,0 +1,31 @@ +/* + * Copyright 2019 WeBank + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.webank.wedatasphere.linkis.engineplugin.spark.metadata + +import java.util.{List => JList} +import com.webank.wedatasphere.linkis.cs.common.entity.history.metadata.TableOperationType +import com.webank.wedatasphere.linkis.cs.common.entity.metadata.CSColumn +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject + +/** + * created by cooperyang on 2020/8/26 + * Description: + */ +case class SparkHiveObject(objectType: HivePrivilegeObject.HivePrivilegeObjectType, dbName: String, objectName: String, + partKeys: JList[String], columns: JList[CSColumn], + actionType: TableOperationType, commandParams: JList[String]){ + override def toString: String = s"$dbName.$objectName" +} diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkSQLHistoryParser.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkSQLHistoryParser.scala new file mode 100644 index 0000000000..8c19f95775 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkSQLHistoryParser.scala @@ -0,0 +1,342 @@ +/* + * Copyright 2019 WeBank + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.webank.wedatasphere.linkis.engineplugin.spark.metadata + +import java.util.{ArrayList => JAList, List => JList} + +import com.webank.wedatasphere.linkis.common.utils.ClassUtils._ +import com.webank.wedatasphere.linkis.cs.common.entity.history.metadata.TableOperationType +import com.webank.wedatasphere.linkis.cs.common.entity.metadata.CSColumn +import com.webank.wedatasphere.linkis.engineplugin.spark.metadata.{SparkHiveObject => HPO} +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, NamedExpression} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.execution.command._ +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.hive.execution.CreateHiveTableAsSelectCommand +import org.apache.spark.sql.types.{StructField, StructType} + +import scala.collection.JavaConverters._ + +/** + * + */ +object SparkSQLHistoryParser { + + /** + * Parse input and output metadata from a Spark's [[LogicalPlan]] + * + * For [[ExplainCommand]]s, parse its child. + * For other queries, build inputs. + * + * @param plan A Spark [[LogicalPlan]] + */ + def parse(plan: LogicalPlan): (JList[HPO], JList[HPO]) = { + plan match { + case e: ExplainCommand => doParse(e.logicalPlan) + case p => doParse(p) + } + } + + /** + * parse outputs if it has an target to write, parse inputs for the + * inside query if exists.For other queries, only parse inputs. + * + * @param plan A Spark [[LogicalPlan]] + * @return (Inputs, OutPuts) + */ + def doParse(plan: LogicalPlan): (JList[HPO], JList[HPO]) = { + val inputMetas = new JAList[HPO] + val outputMetas = new JAList[HPO] + plan match { + case cmd: Command => parseRunnableCommand(cmd, inputMetas, outputMetas) + case _ => ParseQuery(plan, inputMetas) + } + (inputMetas, outputMetas) + } + + + def toCSColumns(schema: StructType): JList[CSColumn] = { + if (null == schema) { + return null + } + schema.map(toCSColumn).filter(null != _).asJava + } + + def toCSColumns(fields: Seq[StructField]): JList[CSColumn] = { + if (null == fields) { + return null + } + fields.map(toCSColumn).filter(null != _).asJava + } + + def toCSColumn(field: StructField): CSColumn = { + if (null == field) { + return null + } + val csColumn = new CSColumn + csColumn.setName(field.name) + csColumn.setComment(field.getComment().orNull) + csColumn.setType(field.dataType.typeName) + csColumn + } + + def toCSColumnsByNamed(projectionList: Seq[NamedExpression]): JList[CSColumn] = { + if (null == projectionList) { + return null + } + projectionList.map { namedExpression => + val csColumn = new CSColumn + csColumn.setName(namedExpression.name) + namedExpression match { + case attribute: AttributeReference => + csColumn.setType(attribute.dataType.typeName) + case _ => + } + csColumn + }.filter(null != _).asJava + } + + def toCSColumnsByColumnName(columnNames: Seq[String]): JList[CSColumn] = { + if (null == columnNames) { + return null + } + columnNames.map { name => + val csColumn = new CSColumn + csColumn.setName(name) + csColumn + }.filter(null != _).asJava + } + + + /** + * Parse LogicalPlan to build sparkHiveObjects + * + * @param plan A Spark [[LogicalPlan]] + * @param sparkHiveObjects input or output hive privilege object list + * @param projectionList Projection list after pruning + */ + private[this] def ParseQuery( + plan: LogicalPlan, + sparkHiveObjects: JList[HPO], + projectionList: Seq[NamedExpression] = Nil): Unit = { + + /** + * Columns in Projection take priority for column level privilege checking + * + * @param table catalogTable of a given relation + */ + def mergeProjection(table: CatalogTable): Unit = { + if (projectionList.isEmpty) { + addTableOrViewLevelObjs( + table.identifier, + sparkHiveObjects, + table.partitionColumnNames, + toCSColumns(table.schema)) + } else { + addTableOrViewLevelObjs( + table.identifier, + sparkHiveObjects, + table.partitionColumnNames.filter(projectionList.map(_.name).contains(_)), + toCSColumnsByNamed(projectionList)) + } + } + + plan match { + case p: Project => ParseQuery(p.child, sparkHiveObjects, p.projectList) + + case h if h.nodeName == "HiveTableRelation" => + mergeProjection(getFieldVal(h, "tableMeta").asInstanceOf[CatalogTable]) + + case m if m.nodeName == "MetastoreRelation" => + mergeProjection(getFieldVal(m, "catalogTable").asInstanceOf[CatalogTable]) + + case l: LogicalRelation if l.catalogTable.nonEmpty => mergeProjection(l.catalogTable.get) + + case u: UnresolvedRelation => + addTableOrViewLevelObjs(u.tableIdentifier, sparkHiveObjects) + + case p => + for (child <- p.children) { + ParseQuery(child, sparkHiveObjects, projectionList) + } + } + } + + /** + * Build sparkHiveObjects from Spark LogicalPlan + * + * @param plan a Spark LogicalPlan used to generate sparkHiveObjects + * @param inputObjects input hive privilege object list + * @param outputObjects output hive privilege object list + */ + private[this] def parseRunnableCommand(plan: LogicalPlan, + inputObjects: JList[HPO], + outputObjects: JList[HPO]): Unit = { + plan match { + + case c: CreateDataSourceTableAsSelectCommand => + val columnList = if (null == c.table.schema || c.table.schema.isEmpty) toCSColumnsByColumnName(c.outputColumnNames) else toCSColumns(c.table.schema) + addTableOrViewLevelObjs(c.table.identifier, outputObjects, columns = columnList, actionType = TableOperationType.CREATE) + ParseQuery(c.query, inputObjects) + + case c: CreateDataSourceTableCommand => + addTableOrViewLevelObjs(c.table.identifier, outputObjects, columns = toCSColumns(c.table.schema), actionType = TableOperationType.CREATE) + + case c: CreateHiveTableAsSelectCommand => + val columnList = if (null == c.tableDesc.schema || c.tableDesc.schema.isEmpty) toCSColumnsByColumnName(c.outputColumnNames) else toCSColumns(c.tableDesc.schema) + addTableOrViewLevelObjs(c.tableDesc.identifier, outputObjects, columns = columnList, actionType = TableOperationType.CREATE) + ParseQuery(c.query, inputObjects) + + case c: CreateTableCommand => + addTableOrViewLevelObjs(c.table.identifier, outputObjects, columns = toCSColumns(c.table.schema), actionType = TableOperationType.CREATE) + + case c: CreateTableLikeCommand => + addTableOrViewLevelObjs(c.targetTable, outputObjects, actionType = TableOperationType.CREATE) + addTableOrViewLevelObjs(c.sourceTable, inputObjects) + + case c: CreateViewCommand => + addTableOrViewLevelObjs(c.name, outputObjects, columns = toCSColumnsByNamed(c.output), actionType = TableOperationType.CREATE) + ParseQuery(c.child, inputObjects) + + case l: LoadDataCommand => addTableOrViewLevelObjs(l.table, outputObjects) + + case i if i.nodeName == "InsertIntoHiveTable" => + val table = getFieldVal(i, "table").asInstanceOf[CatalogTable] + addTableOrViewLevelObjs( + table.identifier, outputObjects, columns = toCSColumns(table.schema), actionType = TableOperationType.CREATE) + ParseQuery(getFieldVal(i, "query").asInstanceOf[LogicalPlan], inputObjects) + + case d: DropTableCommand => addTableOrViewLevelObjs(d.tableName, outputObjects, actionType = TableOperationType.DROP) + + case s: TruncateTableCommand => addTableOrViewLevelObjs(s.tableName, outputObjects, actionType = TableOperationType.DROP) + + case a: AlterTableAddPartitionCommand => + addTableOrViewLevelObjs(a.tableName, outputObjects, actionType = TableOperationType.ALTER) + + case a: AlterTableDropPartitionCommand => + addTableOrViewLevelObjs(a.tableName, outputObjects) + + case a: AlterTableRenameCommand if !a.isView || a.oldName.database.nonEmpty => + addTableOrViewLevelObjs(a.oldName, inputObjects) + addTableOrViewLevelObjs(a.newName, outputObjects) + + case a: AlterTableRenamePartitionCommand => + addTableOrViewLevelObjs(a.tableName, inputObjects) + addTableOrViewLevelObjs(a.tableName, outputObjects) + + case a: AlterViewAsCommand => + if (a.name.database.nonEmpty) { + // it's a permanent view + addTableOrViewLevelObjs(a.name, outputObjects, actionType = TableOperationType.ALTER) + } + ParseQuery(a.query, inputObjects) + + + case a if a.nodeName == "AlterTableAddColumnsCommand" => + addTableOrViewLevelObjs( + getFieldVal(a, "table").asInstanceOf[TableIdentifier], + inputObjects, + columns = toCSColumns(getFieldVal(a, "colsToAdd").asInstanceOf[Seq[StructField]]) + ) + addTableOrViewLevelObjs( + getFieldVal(a, "table").asInstanceOf[TableIdentifier], + outputObjects, + columns = toCSColumns(getFieldVal(a, "colsToAdd").asInstanceOf[Seq[StructField]]), + actionType = TableOperationType.ALTER + ) + + case a if a.nodeName == "AlterTableChangeColumnCommand" => + addTableOrViewLevelObjs( + getFieldVal(a, "tableName").asInstanceOf[TableIdentifier], + inputObjects, + columns = toCSColumns(Seq(getFieldVal(a, "newColumn").asInstanceOf[StructField])), + actionType = TableOperationType.ALTER) + + case _ => + } + } + + /** + * Add database level hive privilege objects to input or output list + * + * @param dbName database name as hive privilege object + * @param sparkHiveObjects input or output list + */ + private[this] def addDbLevelObjs( + dbName: String, + sparkHiveObjects: JList[HPO], + actionType: TableOperationType = TableOperationType.ACCESS): Unit = { + sparkHiveObjects.add( + HPOBuilder(HivePrivilegeObjectType.DATABASE, dbName, dbName, actionType)) + } + + + /** + * Add table level hive objects to input or output list + * + * @param tableIdentifier table identifier contains database name, and table name as hive + * privilege object + * @param sparkHiveObjects input or output list + * @param actionType OperationType + */ + private def addTableOrViewLevelObjs( + tableIdentifier: TableIdentifier, + sparkHiveObjects: JList[HPO], + partKeys: Seq[String] = Nil, + columns: JList[CSColumn] = null, + actionType: TableOperationType = TableOperationType.ACCESS, + cmdParams: Seq[String] = Nil): Unit = { + tableIdentifier.database match { + case Some(db) => + val tbName = tableIdentifier.table + sparkHiveObjects.add( + HPOBuilder( + HivePrivilegeObjectType.TABLE_OR_VIEW, + db, + tbName, + partKeys.asJava, + columns, + actionType, + cmdParams.asJava)) + case _ => + } + } + + /** + * Add function level hive privilege objects to input or output list + * + * @param databaseName database name + * @param functionName function name as hive privilege object + * @param sparkHiveObjects input or output list + */ + private def addFunctionLevelObjs( + databaseName: Option[String], + functionName: String, + sparkHiveObjects: JList[HPO], + actionType: TableOperationType = TableOperationType.ACCESS): Unit = { + databaseName match { + case Some(db) => + sparkHiveObjects.add( + HPOBuilder(HivePrivilegeObjectType.FUNCTION, db, functionName, actionType)) + case _ => + } + } +} + diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/utils/EngineUtils.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/utils/EngineUtils.scala index f628bd180e..1953dca5a8 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/utils/EngineUtils.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/utils/EngineUtils.scala @@ -104,10 +104,10 @@ object EngineUtils { inputStream } - def getResultStrByDolphinContent(dolphinContent:String):String = { + def getResultStrByDolphinTextContent(dolphinContent:String):String = { val resultSetReader = ResultSetReader.getResultSetReader(dolphinContent) val errorMsg = resultSetReader.getMetaData match { - case metadata:LineMetaData => + case metadata: LineMetaData => val sb = new StringBuilder while (resultSetReader.hasNext){ sb.append(resultSetReader.getRecord).append("\n") diff --git a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/utils/ReflectionUtils.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/org/apache/spark/sql/MethodInvokeHelper.scala similarity index 56% rename from linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/utils/ReflectionUtils.scala rename to linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/org/apache/spark/sql/MethodInvokeHelper.scala index 69c92f3b96..69b82e5eb0 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/io_file/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/io/utils/ReflectionUtils.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/org/apache/spark/sql/MethodInvokeHelper.scala @@ -2,7 +2,7 @@ * Copyright 2019 WeBank * * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with 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 @@ -13,21 +13,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.spark.sql -package com.webank.wedatasphere.linkis.manager.engineplugin.io.utils +import org.apache.spark.sql.internal.SQLConf -import java.lang.reflect.{Method} - -object ReflectionUtils { - - @throws[Throwable] - def invoke(any: Any, method: Method, args: Array[AnyRef]): Unit = { - try { - method.invoke(any, args) - } catch { - case t: Throwable => - throw t.getCause - } +/** + * Description: + */ +object MethodInvokeHelper { + def getSQLConf(context: SQLContext):SQLConf = { + context.conf } - } diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/org/apache/spark/sql/SparkLogicalPlanHelper.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/org/apache/spark/sql/SparkLogicalPlanHelper.scala new file mode 100644 index 0000000000..e8a097f637 --- /dev/null +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/org/apache/spark/sql/SparkLogicalPlanHelper.scala @@ -0,0 +1,34 @@ +/* + * Copyright 2019 WeBank + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +import com.webank.wedatasphere.linkis.common.utils.Logging +import com.webank.wedatasphere.linkis.engineplugin.spark.metadata.{SparkHiveObject, SparkSQLHistoryParser} +import org.apache.spark.sql.execution.QueryExecution + +/** + * Description: + */ +object SparkLogicalPlanHelper extends Logging{ + + def extract(context: SQLContext, command: String, queryExecution: QueryExecution, startTime: Long): java.util.List[SparkHiveObject] = { + if (queryExecution == null) return null + val logicPlan = queryExecution.analyzed + val (in, out) = SparkSQLHistoryParser.parse(logicPlan) + in + } + +} diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-cache/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/cache/GuavaEngineConnPluginCache.java b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-cache/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/cache/GuavaEngineConnPluginCache.java index bdd2b54933..8b0888b086 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-cache/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/cache/GuavaEngineConnPluginCache.java +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-cache/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/cache/GuavaEngineConnPluginCache.java @@ -25,7 +25,7 @@ import com.webank.wedatasphere.linkis.manager.engineplugin.cache.refresh.*; import com.webank.wedatasphere.linkis.manager.engineplugin.common.loader.entity.EngineConnPluginInfo; import com.webank.wedatasphere.linkis.manager.engineplugin.common.loader.entity.EngineConnPluginInstance; -import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineTypeLabel; +import com.webank.wedatasphere.linkis.manager.engineplugin.common.loader.exception.EngineConnPluginNotFoundException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.concurrent.*; @@ -113,6 +113,9 @@ public EngineConnPluginInstance get(EngineConnPluginInfo pluginInfo, PluginGette try { //Use the getter method of plugin return caller.call(info); + } catch( EngineConnPluginNotFoundException ne) { + LOG.trace("Not need to refresh the cache of plugin: [" + info.toString() + "], because the resource is not found"); + return null; } catch (Exception e) { LOG.error("Refresh cache of plugin: [" + info.toString() + "] failed, message: [" + e.getMessage() + "]", e); return null; diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/conf/EngineConnPluginConf.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/conf/EngineConnPluginConf.scala index 20c6ad56f1..82446e3f9e 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/conf/EngineConnPluginConf.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/conf/EngineConnPluginConf.scala @@ -29,4 +29,6 @@ object EngineConnPluginConf { val ENGINECONN_TYPE_NAME = CommonVars[String]("wds.linkis.engineconn.type.name", "python") + val ENGINECONN_MAIN_CLASS = CommonVars[String]("wds.linkis.engineconn.main.class", "com.webank.wedatasphere.linkis.engineconn.launch.EngineConnServer") + } diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/EngineConnFactory.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/EngineConnFactory.scala index 509efb6ae3..50689eb1af 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/EngineConnFactory.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/EngineConnFactory.scala @@ -17,15 +17,17 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.common.creation -import com.webank.wedatasphere.linkis.common.utils.{Logging, Utils} +import java.util + +import com.webank.wedatasphere.linkis.common.utils.Logging import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn -import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor -import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineRunTypeLabel -import org.reflections.Reflections +import com.webank.wedatasphere.linkis.engineconn.common.engineconn.{DefaultEngineConn, EngineConn} +import com.webank.wedatasphere.linkis.manager.engineplugin.common.exception.EngineConnBuildFailedException +import com.webank.wedatasphere.linkis.manager.label.entity.Label +import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineConnModeLabel +import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType.EngineType -import scala.collection.mutable.ArrayBuffer -import scala.collection.JavaConverters._ +import scala.collection.JavaConversions.asScalaBuffer trait EngineConnFactory { @@ -34,46 +36,43 @@ trait EngineConnFactory { } +trait AbstractEngineConnFactory extends EngineConnFactory { + + protected def getEngineConnType: EngineType + + protected def createEngineConnSession(engineCreationContext: EngineCreationContext): Any + + override def createEngineConn(engineCreationContext: EngineCreationContext): EngineConn = { + val engineConn = new DefaultEngineConn(engineCreationContext) + val engineConnSession = createEngineConnSession(engineCreationContext) + engineConn.setEngineConnType(getEngineConnType.toString) + engineConn.setEngineConnSession(engineConnSession) + engineConn + } +} + /** * For only one kind of executor, like hive, python ... */ -trait SingleExecutorEngineConnFactory extends EngineConnFactory { +trait SingleExecutorEngineConnFactory extends AbstractEngineConnFactory with ExecutorFactory - def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn): Executor - - def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel -} +trait SingleLabelExecutorEngineConnFactory extends SingleExecutorEngineConnFactory with LabelExecutorFactory /** * For many kinds of executor, such as spark with spark-sql and spark-shell and pyspark */ -trait MultiExecutorEngineConnFactory extends EngineConnFactory with Logging { - - def getExecutorFactories: Array[ExecutorFactory] = { - val executorFactories = new ArrayBuffer[ExecutorFactory] - Utils.tryCatch { - val reflections = new Reflections("com.webank.wedatasphere.linkis", classOf[ExecutorFactory]) - val allSubClass = reflections.getSubTypesOf(classOf[ExecutorFactory]) - allSubClass.asScala.foreach(l => { - executorFactories += l.newInstance - }) - } { - t: Throwable => - error(t.getMessage) - } - executorFactories.toArray - } +trait MultiExecutorEngineConnFactory extends AbstractEngineConnFactory with Logging { - def getDefaultExecutorFactory: ExecutorFactory = { - var defaultExecutorFactory: ExecutorFactory = null - getExecutorFactories.foreach(f => { - if (null == defaultExecutorFactory) { - defaultExecutorFactory = f - } else if (f.getOrder < defaultExecutorFactory.getOrder) { - defaultExecutorFactory = f - } - }) - defaultExecutorFactory - } + + def getExecutorFactories: Array[ExecutorFactory] + + def getDefaultExecutorFactory: ExecutorFactory = + getExecutorFactories.find(_.getClass == getDefaultExecutorFactoryClass) + .getOrElse(throw new EngineConnBuildFailedException(20000, "Cannot find default ExecutorFactory.")) + + protected def getDefaultExecutorFactoryClass: Class[_ <: ExecutorFactory] + + protected def getEngineConnModeLabel(labels: util.List[Label[_]]): EngineConnModeLabel = + labels.find(_.isInstanceOf[EngineConnModeLabel]).map(_.asInstanceOf[EngineConnModeLabel]).orNull } \ No newline at end of file diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/ExecutorFactory.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/ExecutorFactory.scala index 1342318c22..ad9d78349d 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/ExecutorFactory.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/ExecutorFactory.scala @@ -16,62 +16,14 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.common.creation -import com.webank.wedatasphere.linkis.common.utils.Logging import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn import com.webank.wedatasphere.linkis.engineconn.executor.entity.Executor -import com.webank.wedatasphere.linkis.manager.engineplugin.common.exception.{EngineConnPluginErrorCode, EngineConnPluginErrorException} -import com.webank.wedatasphere.linkis.manager.label.entity.Label -import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineRunTypeLabel -import com.webank.wedatasphere.linkis.protocol.UserWithCreator +/** + */ +trait ExecutorFactory { -trait ExecutorFactory extends Logging { + def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn): Executor - /** - * Order of executors, the smallest one is the default - * @return - */ - def getOrder: Int - - def canCreate(labels: Array[Label[_]]): Boolean = { - val runTypeLabel = getDefaultEngineRunTypeLabel() - if (null == runTypeLabel) { - error("DefaultEngineRunTypeLabel must not be null!") - throw new EngineConnPluginErrorException(EngineConnPluginErrorCode.INVALID_RUNTYPE, "DefaultEngineRunTypeLabel cannot be null.") - } - labels.find(_.isInstanceOf[EngineRunTypeLabel]).foreach { - case label: EngineRunTypeLabel => - info(s"executor runType is ${runTypeLabel.getRunType} input runType is ${label.getRunType}") - if (runTypeLabel.getRunType.equalsIgnoreCase(label.getRunType)) { - return true - } - case _ => - error(s"runType label not exists") - } - false - } - - /** - * - * @param engineCreationContext - * @param engineConn - * @param labels - * @return - */ - def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn, labels: Array[Label[_]]): Executor - - def getDefaultEngineRunTypeLabel(): EngineRunTypeLabel -} - -object ExecutorFactory extends Logging { - - def parseUserWithCreator(labels: Array[Label[_]]): UserWithCreator = { - labels.foreach(l => l match { - case label: UserWithCreator => - return UserWithCreator(label.user, label.creator) - case _ => - }) - null - } } \ No newline at end of file diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/JavaProcessEngineConnFactory.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/LabelExecutorFactory.scala similarity index 53% rename from linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/JavaProcessEngineConnFactory.scala rename to linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/LabelExecutorFactory.scala index 536fde33b7..4627502d22 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/JavaProcessEngineConnFactory.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/creation/LabelExecutorFactory.scala @@ -13,21 +13,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package com.webank.wedatasphere.linkis.manager.engineplugin.common.creation import com.webank.wedatasphere.linkis.common.utils.Logging import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconn.common.engineconn.{DefaultEngineConn, EngineConn} -import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.EngineConnPluginConf +import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn +import com.webank.wedatasphere.linkis.engineconn.executor.entity.LabelExecutor +import com.webank.wedatasphere.linkis.manager.label.entity.Label +import com.webank.wedatasphere.linkis.manager.label.entity.engine.CodeLanguageLabel + +/** + * + */ +trait LabelExecutorFactory extends ExecutorFactory with Logging { + def canCreate(labels: Array[Label[_]]): Boolean -abstract class JavaProcessEngineConnFactory extends EngineConnFactory with Logging { + def createExecutor(engineCreationContext: EngineCreationContext, + engineConn: EngineConn, labels: Array[Label[_]]): LabelExecutor - override def createEngineConn(engineCreationContext: EngineCreationContext): EngineConn = { - val engineConn = new DefaultEngineConn(engineCreationContext) - engineConn.setEngineType(EngineConnPluginConf.ENGINECONN_TYPE_NAME.getValue) - engineConn - } + override def createExecutor(engineCreationContext: EngineCreationContext, engineConn: EngineConn): LabelExecutor } + +trait CodeLanguageLabelExecutorFactory extends LabelExecutorFactory { + + def getDefaultCodeLanguageLabel: CodeLanguageLabel + +} \ No newline at end of file diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/launch/process/JavaProcessEngineConnLaunchBuilder.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/launch/process/JavaProcessEngineConnLaunchBuilder.scala index 979dbaca4a..e9b35415a3 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/launch/process/JavaProcessEngineConnLaunchBuilder.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/launch/process/JavaProcessEngineConnLaunchBuilder.scala @@ -20,9 +20,10 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.proces import java.io.File import java.nio.file.Paths import java.util + import com.webank.wedatasphere.linkis.common.utils.Logging import com.webank.wedatasphere.linkis.manager.common.protocol.bml.BmlResource -import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.EnvConfiguration +import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.{EngineConnPluginConf, EnvConfiguration} import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.EnvConfiguration.LINKIS_PUBLIC_MODULE_PATH import com.webank.wedatasphere.linkis.manager.engineplugin.common.exception.EngineConnBuildFailedException import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.entity.{EngineConnBuildRequest, RicherEngineConnBuildRequest} @@ -43,10 +44,9 @@ abstract class JavaProcessEngineConnLaunchBuilder extends ProcessEngineConnLaunc def setEngineConnResourceGenerator(engineConnResourceGenerator: EngineConnResourceGenerator): Unit = this.engineConnResourceGenerator = engineConnResourceGenerator - protected def getGcLogDir(engineConnBuildRequest: EngineConnBuildRequest): String = variable(LOG_DIRS) + "/" + - engineConnBuildRequest.ticketId + "/gc.log" + DateFormatUtils.format(System.currentTimeMillis, "yyyyMMdd-HH_mm") + protected def getGcLogDir(engineConnBuildRequest: EngineConnBuildRequest): String = variable(LOG_DIRS) + "/gc.log" - protected def getLogDir(engineConnBuildRequest: EngineConnBuildRequest): String = s" -Dlogging.file=${EnvConfiguration.LOG4J2_XML_FILE.getValue} -D$LOG_DIRS_KEY=${variable(LOG_DIRS)}" + + protected def getLogDir(engineConnBuildRequest: EngineConnBuildRequest): String = s" -Dlogging.file=${EnvConfiguration.LOG4J2_XML_FILE.getValue} " + s" -D$TICKET_ID_KEY=${engineConnBuildRequest.ticketId}" override protected def getCommands(implicit engineConnBuildRequest: EngineConnBuildRequest): Array[String] = { @@ -72,7 +72,7 @@ abstract class JavaProcessEngineConnLaunchBuilder extends ProcessEngineConnLaunc commandLine.toArray } - protected def getMainClass: String = "com.webank.wedatasphere.linkis.engineconn.launch.EngineConnServer" + protected def getMainClass: String = EngineConnPluginConf.ENGINECONN_MAIN_CLASS.getValue override protected def getEnvironment(implicit engineConnBuildRequest: EngineConnBuildRequest): util.Map[String, String] = { info("Setting up the launch environment for engineconn.") diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/launch/process/ProcessEngineConnLaunchBuilder.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/launch/process/ProcessEngineConnLaunchBuilder.scala index 9db0cdbd4b..ed8984b91e 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/launch/process/ProcessEngineConnLaunchBuilder.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/launch/process/ProcessEngineConnLaunchBuilder.scala @@ -18,6 +18,7 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.proces import java.util +import com.webank.wedatasphere.linkis.engineconn.common.conf.EngineConnConf import com.webank.wedatasphere.linkis.manager.common.protocol.bml.BmlResource import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.EnvConfiguration import com.webank.wedatasphere.linkis.manager.engineplugin.common.exception.EngineConnBuildFailedException diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/resource/AbstractEngineResourceFactory.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/resource/AbstractEngineResourceFactory.scala index 4d055ffb8e..97216b0c1b 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/resource/AbstractEngineResourceFactory.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/resource/AbstractEngineResourceFactory.scala @@ -18,20 +18,28 @@ package com.webank.wedatasphere.linkis.manager.engineplugin.common.resource import com.webank.wedatasphere.linkis.manager.common.entity.resource.{NodeResource, Resource} import com.webank.wedatasphere.linkis.manager.common.utils.ResourceUtils +import com.webank.wedatasphere.linkis.manager.engineplugin.common.exception.EngineConnPluginErrorException trait AbstractEngineResourceFactory extends EngineResourceFactory { protected def getRequestResource(properties: java.util.Map[String, String]): Resource + protected def getMinRequestResource(engineResourceRequest: EngineResourceRequest): Resource = getRequestResource(engineResourceRequest.properties) + + protected def getMaxRequestResource(engineResourceRequest: EngineResourceRequest): Resource = getRequestResource(engineResourceRequest.properties) + override def createEngineResource(engineResourceRequest: EngineResourceRequest): NodeResource = { val user = engineResourceRequest.user val engineResource = new UserNodeResource - val resource = getRequestResource(engineResourceRequest.properties) + val minResource = getMinRequestResource(engineResourceRequest) + val maxResource = getMaxRequestResource(engineResourceRequest) + if(minResource.getClass != maxResource.getClass) throw new EngineConnPluginErrorException(70103, + s"The minResource ${minResource.getClass.getSimpleName} is not the same with the maxResource${maxResource.getClass.getSimpleName}.") engineResource.setUser(user) - engineResource.setMinResource(resource) - engineResource.setResourceType(ResourceUtils.getResourceTypeByResource(resource)) - engineResource.setMaxResource(engineResource.getMinResource) + engineResource.setMinResource(minResource) + engineResource.setResourceType(ResourceUtils.getResourceTypeByResource(minResource)) + engineResource.setMaxResource(maxResource) engineResource } } diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/resource/UserTimeoutNodeResource.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/resource/UserTimeoutNodeResource.scala new file mode 100644 index 0000000000..66fd946ddb --- /dev/null +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-core/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/common/resource/UserTimeoutNodeResource.scala @@ -0,0 +1,30 @@ +/* + * Copyright 2019 WeBank + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.webank.wedatasphere.linkis.manager.engineplugin.common.resource + + + +/* +/** + */ +class UserTimeoutNodeResource extends UserNodeResource with TimeoutNodeResource { + private var timeout: Long = _ + + override def getTimeout: Long = timeout + + override def setTimeout(timeout: Long): Unit = this.timeout = timeout +}*/ diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-loader/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/manager/loaders/DefaultEngineConnPluginLoader.java b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-loader/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/manager/loaders/DefaultEngineConnPluginLoader.java index 64e513c04b..58173e1bb1 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-loader/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/manager/loaders/DefaultEngineConnPluginLoader.java +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-loader/src/main/java/com/webank/wedatasphere/linkis/manager/engineplugin/manager/loaders/DefaultEngineConnPluginLoader.java @@ -91,7 +91,7 @@ protected EngineConnPluginInstance loadEngineConnPluginInternal(EngineConnPlugin if (!savePath.endsWith(String.valueOf(IOUtils.DIR_SEPARATOR))) { savePath += IOUtils.DIR_SEPARATOR; } - savePath += IOUtils.DIR_SEPARATOR + typeLabel.getEngineType() + IOUtils.DIR_SEPARATOR + PLUGIN_DIR + IOUtils.DIR_SEPARATOR; + savePath += typeLabel.getEngineType() + IOUtils.DIR_SEPARATOR + PLUGIN_DIR + IOUtils.DIR_SEPARATOR; if (StringUtils.isNoneBlank(typeLabel.getVersion())) { savePath += typeLabel.getVersion() + IOUtils.DIR_SEPARATOR; } @@ -138,7 +138,7 @@ protected EngineConnPluginInstance loadEngineConnPluginInternal(EngineConnPlugin return new EngineConnPluginInstance(newPluginInfo, enginePlugin); } } - throw new EngineConnPluginNotFoundException("No plugin found, please check your configuration", null); + throw new EngineConnPluginNotFoundException("No plugin found" + enginePluginInfo.typeLabel().getStringValue() + "please check your configuration", null); } /** diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-loader/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/manager/config/EngineConnPluginLoaderConf.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-loader/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/manager/config/EngineConnPluginLoaderConf.scala index a67af8c537..0c11ac975b 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-loader/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/manager/config/EngineConnPluginLoaderConf.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-loader/src/main/scala/com/webank/wedatasphere/linkis/manager/engineplugin/manager/config/EngineConnPluginLoaderConf.scala @@ -28,7 +28,8 @@ object EngineConnPluginLoaderConf { val ENGINE_PLUGIN_LOADER_DEFAULT_USER: CommonVars[String] = CommonVars("wds.linkis.engineconn.plugin.loader.defaultUser", "hadoop") - val ENGINE_PLUGIN_STORE_PATH: CommonVars[String] = CommonVars("wds.linkis.engineconn.plugin.loader.store.path", "") + val ENGINE_PLUGIN_STORE_PATH: CommonVars[String] = CommonVars("wds.linkis.engineconn.plugin.loader.store.path", + CommonVars[String]("ENGINE_CONN_HOME", Configuration.getLinkisHome() + "/lib/linkis-engineconn-plugins").getValue) val ENGINE_PLUGIN_PROPERTIES_NAME: CommonVars[String] = CommonVars("wds.linkis.engineconn.plugin.loader.properties.name", "plugins.properties") diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/java/com/webank/wedatasphere/linkis/engineplugin/server/LinkisEngineConnPluginServer.java b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/java/com/webank/wedatasphere/linkis/engineplugin/server/LinkisEngineConnPluginServer.java index adc6490161..a56fabca58 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/java/com/webank/wedatasphere/linkis/engineplugin/server/LinkisEngineConnPluginServer.java +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/java/com/webank/wedatasphere/linkis/engineplugin/server/LinkisEngineConnPluginServer.java @@ -16,17 +16,13 @@ package com.webank.wedatasphere.linkis.engineplugin.server; -import com.webank.wedatasphere.linkis.DataWorkCloudApplication; -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; +import com.webank.wedatasphere.linkis.LinkisBaseServerApp; public class LinkisEngineConnPluginServer { - private static final Log logger = LogFactory.getLog(LinkisEngineConnPluginServer.class); public static void main(String[] args) throws ReflectiveOperationException { - logger.info("Start to running LinkisEngineConnPluginServer"); - DataWorkCloudApplication.main(args); + LinkisBaseServerApp.main(args); } } diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/java/com/webank/wedatasphere/linkis/engineplugin/server/dao/impl/EngineConnBmlResourceMapper.xml b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/java/com/webank/wedatasphere/linkis/engineplugin/server/dao/impl/EngineConnBmlResourceMapper.xml index 6f3a12774f..2408142101 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/java/com/webank/wedatasphere/linkis/engineplugin/server/dao/impl/EngineConnBmlResourceMapper.xml +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/java/com/webank/wedatasphere/linkis/engineplugin/server/dao/impl/EngineConnBmlResourceMapper.xml @@ -28,19 +28,19 @@ - delete from linkis_engine_conn_plugin_bml_resources where + delete from linkis_cg_engine_conn_plugin_bml_resources where engine_conn_type=#{engineConnBmlResource.engineConnType} and `version`=#{engineConnBmlResource.version} and file_name=#{engineConnBmlResource.fileName} - update linkis_engine_conn_plugin_bml_resources + update linkis_cg_engine_conn_plugin_bml_resources `last_modified` = #{engineConnBmlResource.lastModified}, diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/conf/EngineConnPluginConfiguration.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/conf/EngineConnPluginConfiguration.scala index 31fa54dd42..94860db7c5 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/conf/EngineConnPluginConfiguration.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/conf/EngineConnPluginConfiguration.scala @@ -16,13 +16,13 @@ package com.webank.wedatasphere.linkis.engineplugin.server.conf -import com.webank.wedatasphere.linkis.common.conf.CommonVars +import com.webank.wedatasphere.linkis.common.conf.{CommonVars, Configuration} object EngineConnPluginConfiguration { val ENGINE_CONN_HOME = CommonVars("wds.linkis.engineconn.home", - CommonVars[String]("ENGINE_CONN_HOME", "").getValue) + CommonVars[String]("ENGINE_CONN_HOME", Configuration.getLinkisHome() + "/lib/linkis-engineconn-plugins").getValue) val ENGINE_CONN_DIST_LOAD_ENABLE = CommonVars("wds.linkis.engineconn.dist.load.enable", true) } diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/localize/AbstractEngineConnBmlResourceGenerator.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/localize/AbstractEngineConnBmlResourceGenerator.scala index 7637509b2c..20d534f50a 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/localize/AbstractEngineConnBmlResourceGenerator.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/localize/AbstractEngineConnBmlResourceGenerator.scala @@ -30,12 +30,10 @@ import org.apache.commons.lang.StringUtils abstract class AbstractEngineConnBmlResourceGenerator extends EngineConnBmlResourceGenerator { if(!new File(getEngineConnsHome).exists) - throw new EngineConnPluginErrorException(20001, "Cannot find the home path of engineConn.") + throw new EngineConnPluginErrorException(20001, s"Cannot find the home path(${getEngineConnsHome}) of engineConn.") protected def getEngineConnsHome: String = { - if(StringUtils.isBlank(ENGINE_CONN_HOME.getValue)) - Paths.get(ServerConfiguration.BDP_SERVER_HOME.getValue, "engineconns").toFile.getPath - else ENGINE_CONN_HOME.getValue + ENGINE_CONN_HOME.getValue } protected def getEngineConnDistHome(engineConnTypeLabel: EngineTypeLabel): String = diff --git a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/service/DefaultEngineConnResourceFactoryService.scala b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/service/DefaultEngineConnResourceFactoryService.scala index a9c4e895cd..579070a03e 100644 --- a/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/service/DefaultEngineConnResourceFactoryService.scala +++ b/linkis-engineconn-plugins/linkis-engineconn-plugin-framework/linkis-engineconn-plugin-server/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/server/service/DefaultEngineConnResourceFactoryService.scala @@ -16,6 +16,7 @@ package com.webank.wedatasphere.linkis.engineplugin.server.service +import com.webank.wedatasphere.linkis.common.utils.Logging import com.webank.wedatasphere.linkis.engineplugin.server.loader.EngineConnPluginsLoader import com.webank.wedatasphere.linkis.manager.common.entity.resource.NodeResource import com.webank.wedatasphere.linkis.manager.engineplugin.common.exception.EngineConnPluginErrorException @@ -28,7 +29,7 @@ import scala.collection.JavaConversions._ @Component -class DefaultEngineConnResourceFactoryService extends EngineConnResourceFactoryService { +class DefaultEngineConnResourceFactoryService extends EngineConnResourceFactoryService with Logging { override def getResourceFactoryBy(engineType: EngineTypeLabel): EngineResourceFactory = { val engineConnPluginInstance = EngineConnPluginsLoader.getEngineConnPluginsLoader().getEngineConnPlugin(engineType) @@ -37,6 +38,7 @@ class DefaultEngineConnResourceFactoryService extends EngineConnResourceFactoryS @Receiver override def createEngineResource(engineResourceRequest: EngineResourceRequest): NodeResource = { + info(s"To invoke createEngineResource $engineResourceRequest") val engineTypeOption = engineResourceRequest.labels.find(_.isInstanceOf[EngineTypeLabel]) if (engineTypeOption.isDefined) { From 69e51fb46babc211d95584b0a44b35f784e82838 Mon Sep 17 00:00:00 2001 From: Davidhua1996 Date: Fri, 4 Jun 2021 19:15:49 +0800 Subject: [PATCH 2/3] Remove flink engineConn --- .../engineconn-plugins/flink/pom.xml | 454 ------------ .../flink/src/main/assembly/distribution.xml | 72 -- .../flink/client/config/Environment.java | 288 -------- .../client/config/entries/ConfigEntry.java | 68 -- .../client/config/entries/ExecutionEntry.java | 372 ---------- .../client/context/ExecutionContext.java | 655 ------------------ .../deployment/ClusterDescriptorAdapter.java | 166 ----- .../ClusterDescriptorAdapterFactory.java | 53 -- .../client/deployment/ProgramDeployer.java | 74 -- ...rnApplicationClusterDescriptorAdapter.java | 59 -- .../YarnPerJobClusterDescriptorAdapter.java | 84 --- .../LinkisYarnClusterClientFactory.java | 70 -- .../flink/client/result/AbstractResult.java | 51 -- .../flink/client/result/BatchResult.java | 141 ---- .../flink/client/result/ChangelogResult.java | 221 ------ .../flink/client/result/Result.java | 55 -- .../flink/client/result/ResultUtil.java | 122 ---- .../flink/client/result/TypedResult.java | 89 --- .../sql/operation/AbstractJobOperation.java | 170 ----- .../flink/client/sql/operation/JobInfo.java | 14 - .../client/sql/operation/JobOperation.java | 40 -- .../client/sql/operation/NonJobOperation.java | 23 - .../flink/client/sql/operation/Operation.java | 31 - .../sql/operation/OperationFactory.java | 129 ---- .../client/sql/operation/OperationUtil.java | 78 --- .../operation/impl/CreateViewOperation.java | 69 -- .../sql/operation/impl/DDLOperation.java | 92 --- .../impl/DescribeTableOperation.java | 109 --- .../sql/operation/impl/DropViewOperation.java | 79 --- .../sql/operation/impl/ExplainOperation.java | 78 --- .../sql/operation/impl/InsertOperation.java | 140 ---- .../sql/operation/impl/ResetOperation.java | 52 -- .../sql/operation/impl/SelectOperation.java | 260 ------- .../sql/operation/impl/SetOperation.java | 111 --- .../operation/impl/ShowCatalogsOperation.java | 47 -- .../impl/ShowCurrentCatalogOperation.java | 45 -- .../impl/ShowCurrentDatabaseOperation.java | 45 -- .../impl/ShowDatabasesOperation.java | 47 -- .../impl/ShowFunctionsOperation.java | 47 -- .../operation/impl/ShowModulesOperation.java | 47 -- .../operation/impl/ShowTablesOperation.java | 63 -- .../operation/impl/ShowViewsOperation.java | 66 -- .../operation/impl/UseCatalogOperation.java | 58 -- .../operation/impl/UseDatabaseOperation.java | 56 -- .../sql/operation/result/ColumnInfo.java | 101 --- .../sql/operation/result/ConstantNames.java | 59 -- .../sql/operation/result/ResultKind.java | 30 - .../sql/operation/result/ResultSet.java | 189 ----- .../result/ResultSetJsonDeserializer.java | 200 ------ .../result/ResultSetJsonSerializer.java | 114 --- .../flink/client/utils/SqlCommandParser.java | 403 ----------- .../flink/client/utils/YarnConfLoader.java | 51 -- .../exception/ExecutorInitException.java | 47 -- .../exception/FlinkInitFailedException.java | 38 - .../exception/JobExecutionException.java | 38 - .../exception/SqlExecutionException.java | 38 - .../flink/exception/SqlParseException.java | 38 - .../flink/util/RetryUtil.java | 54 -- .../resources/linkis-engineconn.properties | 27 - .../src/main/resources/log4j2-engineconn.xml | 64 -- .../flink/FlinkEngineConnPlugin.scala | 77 -- .../flink/config/FlinkEnvConfiguration.scala | 58 -- .../config/FlinkResourceConfiguration.scala | 41 -- .../flink/context/EnvironmentContext.scala | 106 --- .../context/FlinkEngineConnContext.scala | 40 -- .../FlinkApplicationJobExecutor.scala | 88 --- .../flink/executor/FlinkJobExecutor.scala | 133 ---- .../flink/executor/FlinkSQLJobExecutor.scala | 226 ------ .../FlinkApplicationExecutorFactory.scala | 44 -- .../factory/FlinkEngineConnFactory.scala | 186 ----- .../factory/FlinkSQLExecutorFactory.scala | 63 -- .../launch/FlinkEngineConnLaunchBuilder.scala | 42 -- .../flink/listener/FlinkListener.scala | 47 -- .../flink/listener/FlinkListenerGroup.scala | 62 -- .../FlinkEngineConnResourceFactory.scala | 47 -- .../flink/util/FlinkValueFormatUtil.scala | 42 -- .../springcloud/LinkisGatewayApplication.java | 2 +- 77 files changed, 1 insertion(+), 7884 deletions(-) delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/pom.xml delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/assembly/distribution.xml delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/Environment.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ConfigEntry.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ExecutionEntry.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/context/ExecutionContext.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapter.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapterFactory.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ProgramDeployer.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnApplicationClusterDescriptorAdapter.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnPerJobClusterDescriptorAdapter.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/factory/LinkisYarnClusterClientFactory.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/AbstractResult.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/BatchResult.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ChangelogResult.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/Result.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ResultUtil.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/TypedResult.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/AbstractJobOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobInfo.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/NonJobOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/Operation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationFactory.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationUtil.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/CreateViewOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DDLOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DescribeTableOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DropViewOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ExplainOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/InsertOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ResetOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SelectOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SetOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCatalogsOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentCatalogOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentDatabaseOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowDatabasesOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowFunctionsOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowModulesOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowTablesOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowViewsOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseCatalogOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseDatabaseOperation.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ColumnInfo.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ConstantNames.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultKind.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSet.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonDeserializer.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonSerializer.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/SqlCommandParser.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/YarnConfLoader.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/ExecutorInitException.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/FlinkInitFailedException.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/JobExecutionException.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlExecutionException.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlParseException.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/RetryUtil.java delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/linkis-engineconn.properties delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/log4j2-engineconn.xml delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/FlinkEngineConnPlugin.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkEnvConfiguration.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkResourceConfiguration.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/EnvironmentContext.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/FlinkEngineConnContext.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkApplicationJobExecutor.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkJobExecutor.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkSQLJobExecutor.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkApplicationExecutorFactory.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkEngineConnFactory.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkSQLExecutorFactory.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/launch/FlinkEngineConnLaunchBuilder.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListener.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListenerGroup.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/resource/FlinkEngineConnResourceFactory.scala delete mode 100644 linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/FlinkValueFormatUtil.scala diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/pom.xml b/linkis-engineconn-plugins/engineconn-plugins/flink/pom.xml deleted file mode 100644 index 0ff898c8c6..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/pom.xml +++ /dev/null @@ -1,454 +0,0 @@ - - - - linkis - com.webank.wedatasphere.linkis - 1.0.0-RC1 - ../../pom.xml - - 4.0.0 - - linkis-engineconn-plugin-flink - - 1.11.1 - 1.2.1 - - - - - - - org.apache.flink - flink-core - ${flink.version} - - - - org.apache.flink - flink-runtime_${scala.binary.version} - ${flink.version} - - - - org.apache.flink - flink-streaming-java_${scala.binary.version} - ${flink.version} - - - - org.apache.flink - flink-clients_${scala.binary.version} - ${flink.version} - - - - org.apache.flink - flink-sql-client_${scala.binary.version} - ${flink.version} - - - - commons-cli - commons-cli - 1.3.1 - - - org.apache.flink - flink-table-common - ${flink.version} - - - - org.apache.flink - flink-table-api-java - ${flink.version} - - - - org.apache.flink - flink-table-api-java-bridge_${scala.binary.version} - ${flink.version} - - - - org.apache.flink - flink-table-planner_${scala.binary.version} - ${flink.version} - - - - - - - - - - - - - - - - org.apache.flink - flink-sql-parser - ${flink.version} - - - - org.apache.flink - flink-shaded-jackson - 2.10.1-9.0 - - - - org.apache.flink - flink-yarn_${scala.binary.version} - ${flink.version} - - - org.apache.flink - flink-shaded-hadoop2 - - - org.eclipse.jetty - * - - - io.netty - netty - - - org.codehaus.jackson - jackson-core-asl - - - org.codehaus.jackson - jackson-mapper-asl - - - httpclient - org.apache.httpcomponents - - - httpcore - org.apache.httpcomponents - - - hadoop-common - org.apache.hadoop - - - hadoop-hdfs - org.apache.hadoop - - - hadoop-yarn-client - org.apache.hadoop - - - hadoop-mapreduce-client-core - org.apache.hadoop - - - hadoop-yarn-common - org.apache.hadoop - - - - - - junit - junit - 4.12 - test - - - org.apache.flink - flink-connector-hive_2.11 - ${flink.version} - - - - - org.apache.hive - hive-exec - ${hive.version} - - - httpclient - org.apache.httpcomponents - - - hadoop-yarn-server-resourcemanager - org.apache.hadoop - - - - - - - com.webank.wedatasphere.linkis - linkis-engineconn-plugin-core - ${linkis.version} - - - - com.webank.wedatasphere.linkis - linkis-computation-engineconn - ${linkis.version} - - - httpclient - org.apache.httpcomponents - - - - - - - com.webank.wedatasphere.linkis - linkis-message-scheduler - ${linkis.version} - provided - - - - com.webank.wedatasphere.linkis - linkis-once-engineconn - ${linkis.version} - - - - - com.webank.wedatasphere.linkis - linkis-hadoop-common - ${linkis.version} - provided - - - - org.apache.hadoop - hadoop-yarn-api - ${hadoop.version} - - - - - - - - - org.apache.hadoop - hadoop-yarn-client - ${hadoop.version} - - - servlet-api - javax.servlet - - - - - - - com.webank.wedatasphere.linkis - linkis-storage - ${linkis.version} - - - commons-httpclient - commons-httpclient - - - commons-configuration - commons-configuration - - - commons-logging - commons-logging - - - netty - io.netty - - - xlsx-streamer - com.monitorjbl - - - netty-all - io.netty - - - - hadoop-common - org.apache.hadoop - - - - hadoop-hdfs - org.apache.hadoop - - - - hadoop-auth - org.apache.hadoop - - - - org.json4s - json4s-jackson_${scala.binary.version} - - - provided - - - - org.apache.commons - commons-exec - 1.3 - - - - - com.google.guava - guava - ${guava.version} - provided - - - - org.scala-lang - scala-library - provided - - - org.scala-lang - scala-compiler - provided - - - org.scala-lang - scala-reflect - provided - - - - org.json4s - json4s-jackson_${scala.binary.version} - ${json4s.version} - provided - - - - io.netty - netty-all - ${netty.version} - provided - - - - com.fasterxml.jackson.core - jackson-databind - ${fasterxml.jackson.version} - - - - com.webank.wedatasphere.linkis - linkis-bml-engine-hook - ${linkis.version} - - - commons-logging - commons-logging - - - - - com.github.rholder - guava-retrying - 2.0.0 - - - - - - - - org.apache.maven.plugins - maven-deploy-plugin - - - - net.alchim31.maven - scala-maven-plugin - - - org.apache.maven.plugins - maven-jar-plugin - - - org.apache.maven.plugins - maven-assembly-plugin - 2.3 - false - - - make-assembly - package - - single - - - - src/main/assembly/distribution.xml - - - - - - false - flink-engineconn - false - false - - src/main/assembly/distribution.xml - - - - - - - src/main/java - - **/*.xml - - - - src/main/resources - - **/*.properties - **/application.yml - **/bootstrap.yml - **/log4j2.xml - - - - - \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/assembly/distribution.xml b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/assembly/distribution.xml deleted file mode 100644 index 40a5a00027..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/assembly/distribution.xml +++ /dev/null @@ -1,72 +0,0 @@ - - - - linkis-engineconn-plugin-flink - - dir - zip - - true - flink - - - - - - /dist/v${flink.version}/lib - true - true - false - false - true - - - - - - - ${basedir}/src/main/resources - - * - - 0777 - 0755 - /dist/v${flink.version}/conf - unix - - - - ${basedir}/target - - *.jar - - - *doc.jar - - 0777 - /plugin/${flink.version} - - - - - - - diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/Environment.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/Environment.java deleted file mode 100644 index d02cbf4d3d..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/Environment.java +++ /dev/null @@ -1,288 +0,0 @@ -package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.FlinkInitFailedException; -import java.io.IOException; -import java.net.URL; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonMappingException; -import org.apache.flink.table.client.config.ConfigUtil; -import org.apache.flink.table.client.config.entries.CatalogEntry; -import org.apache.flink.table.client.config.entries.ConfigurationEntry; -import org.apache.flink.table.client.config.entries.DeploymentEntry; -import org.apache.flink.table.client.config.entries.ExecutionEntry; -import org.apache.flink.table.client.config.entries.FunctionEntry; -import org.apache.flink.table.client.config.entries.ModuleEntry; -import org.apache.flink.table.client.config.entries.TableEntry; -import org.apache.flink.table.client.config.entries.ViewEntry; - - -public class Environment { - - public static final String SESSION_ENTRY = "session"; - - public static final String EXECUTION_ENTRY = "execution"; - - public static final String CONFIGURATION_ENTRY = "table"; - - public static final String DEPLOYMENT_ENTRY = "deployment"; - - private ExecutionEntry execution; - - private Map modules; - - private Map catalogs; - - private Map tables; - - private Map functions; - - private ConfigurationEntry configuration; - - private DeploymentEntry deployment; - - public Environment() { - this.modules = new LinkedHashMap<>(); - this.catalogs = Collections.emptyMap(); - this.tables = Collections.emptyMap(); - this.functions = Collections.emptyMap(); - this.execution = ExecutionEntry.DEFAULT_INSTANCE; - this.configuration = ConfigurationEntry.DEFAULT_INSTANCE; - this.deployment = DeploymentEntry.DEFAULT_INSTANCE; - } - - public Map getModules() { - return modules; - } - - public void setModules(List> modules) throws FlinkInitFailedException { - this.modules = new LinkedHashMap<>(modules.size()); - - for(Map config : modules) { - final ModuleEntry entry = ModuleEntry.create(config); - if (this.modules.containsKey(entry.getName())) { - throw new FlinkInitFailedException( - String.format("Cannot register module '%s' because a module with this name is already registered.", - entry.getName())); - } - this.modules.put(entry.getName(), entry); - } - } - - public Map getCatalogs() { - return catalogs; - } - - public void setCatalogs(List> catalogs) throws FlinkInitFailedException { - this.catalogs = new HashMap<>(catalogs.size()); - - for(Map config : catalogs) { - final CatalogEntry catalog = CatalogEntry.create(config); - if (this.catalogs.containsKey(catalog.getName())) { - throw new FlinkInitFailedException( - String.format("Cannot create catalog '%s' because a catalog with this name is already registered.", - catalog.getName())); - } - this.catalogs.put(catalog.getName(), catalog); - } - } - - public Map getTables() { - return tables; - } - - public void setTables(List> tables) throws FlinkInitFailedException { - this.tables = new LinkedHashMap<>(tables.size()); - - for(Map config : tables) { - final TableEntry table = TableEntry.create(config); - if (this.tables.containsKey(table.getName())) { - throw new FlinkInitFailedException( - "Cannot create table '" + table - .getName() + "' because a table with this name is already registered."); - } - this.tables.put(table.getName(), table); - } - } - - public Map getFunctions() { - return functions; - } - - public void setFunctions(List> functions) throws FlinkInitFailedException { - this.functions = new HashMap<>(functions.size()); - - for(Map config : functions) { - final FunctionEntry function = FunctionEntry.create(config); - if (this.functions.containsKey(function.getName())) { - throw new FlinkInitFailedException( - "Cannot create function '" + function - .getName() + "' because a function with this name is already registered."); - } - this.functions.put(function.getName(), function); - } - } - - public void setExecution(Map config) { - this.execution = ExecutionEntry.create(config); - } - - public ExecutionEntry getExecution() { - return execution; - } - - public void setConfiguration(Map config) { - this.configuration = ConfigurationEntry.create(config); - } - - public ConfigurationEntry getConfiguration() { - return configuration; - } - - public void setDeployment(Map config) { - this.deployment = DeploymentEntry.create(config); - } - - public DeploymentEntry getDeployment() { - return deployment; - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder(); - sb.append("==================== Session =====================\n"); - sb.append("===================== Modules =====================\n"); - modules.forEach((name, module) -> { - sb.append("- ").append(ModuleEntry.MODULE_NAME).append(": ").append(name).append("\n"); - module.asMap().forEach((k, v) -> sb.append(" ").append(k).append(": ").append(v).append('\n')); - }); - sb.append("===================== Catalogs =====================\n"); - catalogs.forEach((name, catalog) -> { - sb.append("- ").append(CatalogEntry.CATALOG_NAME).append(": ").append(name).append("\n"); - catalog.asMap().forEach((k, v) -> sb.append(" ").append(k).append(": ").append(v).append('\n')); - }); - sb.append("===================== Tables =====================\n"); - tables.forEach((name, table) -> { - sb.append("- ").append(TableEntry.TABLES_NAME).append(": ").append(name).append("\n"); - table.asMap().forEach((k, v) -> sb.append(" ").append(k).append(": ").append(v).append('\n')); - }); - sb.append("=================== Functions ====================\n"); - functions.forEach((name, function) -> { - sb.append("- ").append(FunctionEntry.FUNCTIONS_NAME).append(": ").append(name).append("\n"); - function.asMap().forEach((k, v) -> sb.append(" ").append(k).append(": ").append(v).append('\n')); - }); - sb.append("=================== Execution ====================\n"); - execution.asTopLevelMap().forEach((k, v) -> sb.append(k).append(": ").append(v).append('\n')); - sb.append("================== Configuration =================\n"); - configuration.asMap().forEach((k, v) -> sb.append(k).append(": ").append(v).append('\n')); - sb.append("=================== Deployment ===================\n"); - deployment.asTopLevelMap().forEach((k, v) -> sb.append(k).append(": ").append(v).append('\n')); - return sb.toString(); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Parses an environment file from an URL. - */ - public static Environment parse(URL url) throws IOException, FlinkInitFailedException { - try { - return new ConfigUtil.LowerCaseYamlMapper().readValue(url, Environment.class); - } catch (JsonMappingException e) { - throw new FlinkInitFailedException("Could not parse environment file. Cause: " + e.getMessage(), e); - } - } - - /** - * Parses an environment file from an String. - */ - public static Environment parse(String content) throws IOException, FlinkInitFailedException { - try { - return new ConfigUtil.LowerCaseYamlMapper().readValue(content, Environment.class); - } catch (JsonMappingException e) { - throw new FlinkInitFailedException("Could not parse environment file. Cause: " + e.getMessage(), e); - } - } - - /** - * Merges two environments. The properties of the first environment might be overwritten by the second one. - */ - public static Environment merge(Environment env1, Environment env2) { - if(null==env2){ - return env1; - } - final Environment mergedEnv = new Environment(); - - // merge modules - final Map modules = new LinkedHashMap<>(env1.getModules()); - modules.putAll(env2.getModules()); - mergedEnv.modules = modules; - - // merge catalogs - final Map catalogs = new HashMap<>(env1.getCatalogs()); - catalogs.putAll(env2.getCatalogs()); - mergedEnv.catalogs = catalogs; - - // merge tables - final Map tables = new LinkedHashMap<>(env1.getTables()); - tables.putAll(env2.getTables()); - mergedEnv.tables = tables; - - // merge functions - final Map functions = new HashMap<>(env1.getFunctions()); - functions.putAll(env2.getFunctions()); - mergedEnv.functions = functions; - - // merge execution properties - mergedEnv.execution = ExecutionEntry.merge(env1.getExecution(), env2.getExecution()); - - // merge configuration properties - mergedEnv.configuration = ConfigurationEntry.merge(env1.getConfiguration(), env2.getConfiguration()); - - // merge deployment properties - mergedEnv.deployment = DeploymentEntry.merge(env1.getDeployment(), env2.getDeployment()); - - return mergedEnv; - } - - @Override - public Environment clone() { - return enrich(this, Collections.emptyMap(), Collections.emptyMap()); - } - - /** - * Enriches an environment with new/modified properties or views and returns the new instance. - */ - public static Environment enrich( - Environment env, - Map properties, - Map views) { - final Environment enrichedEnv = new Environment(); - - enrichedEnv.modules = new LinkedHashMap<>(env.getModules()); - - // merge catalogs - enrichedEnv.catalogs = new LinkedHashMap<>(env.getCatalogs()); - - // merge tables - enrichedEnv.tables = new LinkedHashMap<>(env.getTables()); - enrichedEnv.tables.putAll(views); - - // merge functions - enrichedEnv.functions = new HashMap<>(env.getFunctions()); - - // enrich execution properties - enrichedEnv.execution = ExecutionEntry.enrich(env.execution, properties); - - // enrich configuration properties - enrichedEnv.configuration = ConfigurationEntry.enrich(env.configuration, properties); - - // enrich deployment properties - enrichedEnv.deployment = DeploymentEntry.enrich(env.deployment, properties); - - return enrichedEnv; - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ConfigEntry.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ConfigEntry.java deleted file mode 100644 index b628bc167d..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ConfigEntry.java +++ /dev/null @@ -1,68 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.entries; - - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.FlinkInitFailedException; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.descriptors.DescriptorProperties; - - -public abstract class ConfigEntry { - - protected final DescriptorProperties properties; - - protected ConfigEntry(DescriptorProperties properties) throws FlinkInitFailedException { - try { - validate(properties); - } catch (ValidationException e) { - throw new FlinkInitFailedException("Invalid configuration entry.", e); - } - - this.properties = properties; - } - - /** - * Performs syntactic validation. - */ - protected abstract void validate(DescriptorProperties properties); - - public Map asMap() { - return properties.asMap(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ConfigEntry that = (ConfigEntry) o; - return Objects.equals(properties, that.properties); - } - - @Override - public int hashCode() { - return Objects.hash(properties); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ExecutionEntry.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ExecutionEntry.java deleted file mode 100644 index 6425c2901e..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/config/entries/ExecutionEntry.java +++ /dev/null @@ -1,372 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.entries; - - -import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment.EXECUTION_ENTRY; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.FlinkInitFailedException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.time.Time; -import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.client.config.ConfigUtil; -import org.apache.flink.table.descriptors.DescriptorProperties; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -public class ExecutionEntry extends ConfigEntry { - - private static final Logger LOG = LoggerFactory.getLogger(ExecutionEntry.class); - - public static final List AVAILABLE_PLANNERS = Arrays.asList( - ExecutionEntry.EXECUTION_PLANNER_VALUE_OLD, - ExecutionEntry.EXECUTION_PLANNER_VALUE_BLINK); - - public static final List AVAILABLE_EXECUTION_TYPES = Arrays.asList( - ExecutionEntry.EXECUTION_TYPE_VALUE_BATCH, - ExecutionEntry.EXECUTION_TYPE_VALUE_STREAMING); - - public static final String EXECUTION_PLANNER = "planner"; - - public static final String EXECUTION_PLANNER_VALUE_OLD = "old"; - - public static final String EXECUTION_PLANNER_VALUE_BLINK = "blink"; - - public static final String EXECUTION_TYPE = "type"; - - public static final String EXECUTION_TYPE_VALUE_STREAMING = "streaming"; - - public static final String EXECUTION_TYPE_VALUE_BATCH = "batch"; - - private static final String EXECUTION_TIME_CHARACTERISTIC = "time-characteristic"; - - private static final String EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME = "event-time"; - - private static final String EXECUTION_TIME_CHARACTERISTIC_VALUE_PROCESSING_TIME = "processing-time"; - - private static final String EXECUTION_PERIODIC_WATERMARKS_INTERVAL = "periodic-watermarks-interval"; - - private static final String EXECUTION_MIN_STATE_RETENTION = "min-idle-state-retention"; - - private static final String EXECUTION_MAX_STATE_RETENTION = "max-idle-state-retention"; - - private static final String EXECUTION_PARALLELISM = "parallelism"; - - private static final String EXECUTION_MAX_PARALLELISM = "max-parallelism"; - - public static final String EXECUTION_RESULT_MODE = "result-mode"; - - public static final String EXECUTION_RESULT_MODE_VALUE_CHANGELOG = "changelog"; - - public static final String EXECUTION_RESULT_MODE_VALUE_TABLE = "table"; - - public static final String EXECUTION_MAX_BUFFER_SIZE = "max_buffer_size"; - - private static final String EXECUTION_RESTART_STRATEGY_TYPE = "restart-strategy.type"; - - private static final String EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FALLBACK = "fallback"; - - private static final String EXECUTION_RESTART_STRATEGY_TYPE_VALUE_NONE = "none"; - - private static final String EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FIXED_DELAY = "fixed-delay"; - - private static final String EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FAILURE_RATE = "failure-rate"; - - private static final String EXECUTION_RESTART_STRATEGY_ATTEMPTS = "restart-strategy.attempts"; - - private static final String EXECUTION_RESTART_STRATEGY_DELAY = "restart-strategy.delay"; - - private static final String EXECUTION_RESTART_STRATEGY_FAILURE_RATE_INTERVAL = "restart-strategy.failure-rate-interval"; - - private static final String EXECUTION_RESTART_STRATEGY_MAX_FAILURES_PER_INTERVAL = "restart-strategy.max-failures-per-interval"; - - public static final String EXECUTION_CURRENT_CATALOG = "current-catalog"; - - public static final String EXECUTION_CURRENT_DATABASE = "current-database"; - - private ExecutionEntry(DescriptorProperties properties) throws FlinkInitFailedException { - super(properties); - } - - @Override - protected void validate(DescriptorProperties properties) { - properties.validateEnumValues( - EXECUTION_PLANNER, - true, - Arrays.asList( - EXECUTION_PLANNER_VALUE_OLD, - EXECUTION_PLANNER_VALUE_BLINK)); - properties.validateEnumValues( - EXECUTION_TYPE, - true, - Arrays.asList( - EXECUTION_TYPE_VALUE_BATCH, - EXECUTION_TYPE_VALUE_STREAMING)); - properties.validateEnumValues( - EXECUTION_TIME_CHARACTERISTIC, - true, - Arrays.asList( - EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME, - EXECUTION_TIME_CHARACTERISTIC_VALUE_PROCESSING_TIME)); - properties.validateLong(EXECUTION_PERIODIC_WATERMARKS_INTERVAL, true, 1); - properties.validateLong(EXECUTION_MIN_STATE_RETENTION, true, 0); - properties.validateLong(EXECUTION_MAX_STATE_RETENTION, true, 0); - properties.validateInt(EXECUTION_PARALLELISM, true, 1); - properties.validateInt(EXECUTION_MAX_PARALLELISM, true, 1); - properties.validateEnumValues( - EXECUTION_RESTART_STRATEGY_TYPE, - true, - Arrays.asList( - EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FALLBACK, - EXECUTION_RESTART_STRATEGY_TYPE_VALUE_NONE, - EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FIXED_DELAY, - EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FAILURE_RATE)); - properties.validateInt(EXECUTION_RESTART_STRATEGY_ATTEMPTS, true, 1); - properties.validateLong(EXECUTION_RESTART_STRATEGY_DELAY, true, 0); - properties.validateLong(EXECUTION_RESTART_STRATEGY_FAILURE_RATE_INTERVAL, true, 1); - properties.validateInt(EXECUTION_RESTART_STRATEGY_MAX_FAILURES_PER_INTERVAL, true, 1); - properties.validateString(EXECUTION_CURRENT_CATALOG, true, 1); - properties.validateString(EXECUTION_CURRENT_DATABASE, true, 1); - properties.validateInt(EXECUTION_MAX_BUFFER_SIZE, true, 1); - } - - public EnvironmentSettings getEnvironmentSettings() { - final EnvironmentSettings.Builder builder = EnvironmentSettings.newInstance(); - - if (inStreamingMode()) { - builder.inStreamingMode(); - } else if (inBatchMode()) { - builder.inBatchMode(); - } - - final String planner = properties.getOptionalString(EXECUTION_PLANNER) - .orElse(EXECUTION_PLANNER_VALUE_OLD); - - if (planner.equals(EXECUTION_PLANNER_VALUE_OLD)) { - builder.useOldPlanner(); - } else if (planner.equals(EXECUTION_PLANNER_VALUE_BLINK)) { - builder.useBlinkPlanner(); - } - - return builder.build(); - } - - public boolean inStreamingMode() { - return properties.getOptionalString(EXECUTION_TYPE) - .map((v) -> v.equals(EXECUTION_TYPE_VALUE_STREAMING)) - .orElse(false); - } - - public boolean inBatchMode() { - return properties.getOptionalString(EXECUTION_TYPE) - .map((v) -> v.equals(EXECUTION_TYPE_VALUE_BATCH)) - .orElse(false); - } - - public boolean isStreamingPlanner() { - final String planner = properties.getOptionalString(EXECUTION_PLANNER) - .orElse(EXECUTION_PLANNER_VALUE_OLD); - - // Blink planner is a streaming planner - if (planner.equals(EXECUTION_PLANNER_VALUE_BLINK)) { - return true; - } - // Old planner can be a streaming or batch planner - else if (planner.equals(EXECUTION_PLANNER_VALUE_OLD)) { - return inStreamingMode(); - } - - return false; - } - - public String getPlanner() { - return properties.getOptionalString(EXECUTION_PLANNER).orElse(EXECUTION_PLANNER_VALUE_OLD); - } - - public boolean isBatchPlanner() { - final String planner = properties.getOptionalString(EXECUTION_PLANNER) - .orElse(EXECUTION_PLANNER_VALUE_OLD); - - // Blink planner is not a batch planner - if (planner.equals(EXECUTION_PLANNER_VALUE_BLINK)) { - return false; - } - // Old planner can be a streaming or batch planner - else if (planner.equals(EXECUTION_PLANNER_VALUE_OLD)) { - return inBatchMode(); - } - - return false; - } - - public TimeCharacteristic getTimeCharacteristic() { - return properties.getOptionalString(EXECUTION_TIME_CHARACTERISTIC) - .flatMap((v) -> { - switch (v) { - case EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME: - return Optional.of(TimeCharacteristic.EventTime); - case EXECUTION_TIME_CHARACTERISTIC_VALUE_PROCESSING_TIME: - return Optional.of(TimeCharacteristic.ProcessingTime); - default: - return Optional.empty(); - } - }) - .orElseGet(() -> - useDefaultValue( - EXECUTION_TIME_CHARACTERISTIC, - TimeCharacteristic.EventTime, - EXECUTION_TIME_CHARACTERISTIC_VALUE_EVENT_TIME)); - } - - public long getPeriodicWatermarksInterval() { - return properties.getOptionalLong(EXECUTION_PERIODIC_WATERMARKS_INTERVAL) - .orElseGet(() -> useDefaultValue(EXECUTION_PERIODIC_WATERMARKS_INTERVAL, 200L)); - } - - public long getMinStateRetention() { - return properties.getOptionalLong(EXECUTION_MIN_STATE_RETENTION) - .orElseGet(() -> useDefaultValue(EXECUTION_MIN_STATE_RETENTION, 0L)); - } - - public long getMaxStateRetention() { - return properties.getOptionalLong(EXECUTION_MAX_STATE_RETENTION) - .orElseGet(() -> useDefaultValue(EXECUTION_MAX_STATE_RETENTION, 0L)); - } - - public int getParallelism() { - return properties.getOptionalInt(EXECUTION_PARALLELISM) - .orElseGet(() -> useDefaultValue(EXECUTION_PARALLELISM, 1)); - } - - public int getMaxParallelism() { - return properties.getOptionalInt(EXECUTION_MAX_PARALLELISM) - .orElseGet(() -> useDefaultValue(EXECUTION_MAX_PARALLELISM, 128)); - } - - public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() { - return properties.getOptionalString(EXECUTION_RESTART_STRATEGY_TYPE) - .flatMap((v) -> { - switch (v) { - case EXECUTION_RESTART_STRATEGY_TYPE_VALUE_NONE: - return Optional.of(RestartStrategies.noRestart()); - case EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FIXED_DELAY: - final int attempts = properties.getOptionalInt(EXECUTION_RESTART_STRATEGY_ATTEMPTS) - .orElseGet(() -> useDefaultValue(EXECUTION_RESTART_STRATEGY_ATTEMPTS, Integer.MAX_VALUE)); - final long fixedDelay = properties.getOptionalLong(EXECUTION_RESTART_STRATEGY_DELAY) - .orElseGet(() -> useDefaultValue(EXECUTION_RESTART_STRATEGY_DELAY, 10_000L)); - return Optional.of(RestartStrategies.fixedDelayRestart(attempts, fixedDelay)); - case EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FAILURE_RATE: - final int failureRate = properties - .getOptionalInt(EXECUTION_RESTART_STRATEGY_MAX_FAILURES_PER_INTERVAL) - .orElseGet(() -> useDefaultValue(EXECUTION_RESTART_STRATEGY_MAX_FAILURES_PER_INTERVAL, 1)); - final long failureInterval = properties - .getOptionalLong(EXECUTION_RESTART_STRATEGY_FAILURE_RATE_INTERVAL) - .orElseGet( - () -> useDefaultValue(EXECUTION_RESTART_STRATEGY_FAILURE_RATE_INTERVAL, 60_000L)); - final long attemptDelay = properties.getOptionalLong(EXECUTION_RESTART_STRATEGY_DELAY) - .orElseGet(() -> useDefaultValue(EXECUTION_RESTART_STRATEGY_DELAY, 10_000L)); - return Optional.of(RestartStrategies.failureRateRestart( - failureRate, - Time.milliseconds(failureInterval), - Time.milliseconds(attemptDelay))); - default: - return Optional.empty(); - } - }) - .orElseGet(() -> - useDefaultValue( - EXECUTION_RESTART_STRATEGY_TYPE, - RestartStrategies.fallBackRestart(), - EXECUTION_RESTART_STRATEGY_TYPE_VALUE_FALLBACK)); - } - - public Optional getCurrentCatalog() { - return properties.getOptionalString(EXECUTION_CURRENT_CATALOG); - } - - public Optional getCurrentDatabase() { - return properties.getOptionalString(EXECUTION_CURRENT_DATABASE); - } - - public Map asTopLevelMap() { - return properties.asPrefixedMap(EXECUTION_ENTRY + '.'); - } - - public int getMaxBufferSize() { - return properties.getOptionalInt(EXECUTION_MAX_BUFFER_SIZE) - .orElseGet(() -> useDefaultValue(EXECUTION_MAX_BUFFER_SIZE, 5000)); - } - - private V useDefaultValue(String key, V defaultValue) { - return useDefaultValue(key, defaultValue, defaultValue.toString()); - } - - private V useDefaultValue(String key, V defaultValue, String defaultString) { - LOG.info("Property '{}.{}' not specified. Using default value: {}", EXECUTION_ENTRY, key, defaultString); - return defaultValue; - } - - // -------------------------------------------------------------------------------------------- - - public static ExecutionEntry create(Map config) throws FlinkInitFailedException { - return new ExecutionEntry(ConfigUtil.normalizeYaml(config)); - } - - /** - * Merges two execution entries. The properties of the first execution entry might be - * overwritten by the second one. - */ - public static ExecutionEntry merge(ExecutionEntry execution1, ExecutionEntry execution2) throws FlinkInitFailedException { - final Map mergedProperties = new HashMap<>(execution1.asMap()); - mergedProperties.putAll(execution2.asMap()); - - final DescriptorProperties properties = new DescriptorProperties(true); - properties.putProperties(mergedProperties); - - return new ExecutionEntry(properties); - } - - /** - * Creates a new execution entry enriched with additional properties that are prefixed with - * {@link Environment#EXECUTION_ENTRY}. - */ - public static ExecutionEntry enrich(ExecutionEntry execution, Map prefixedProperties) throws FlinkInitFailedException { - final Map enrichedProperties = new HashMap<>(execution.asMap()); - - prefixedProperties.forEach((k, v) -> { - final String normalizedKey = k.toLowerCase(); - if (k.startsWith(EXECUTION_ENTRY + '.')) { - enrichedProperties.put(normalizedKey.substring(EXECUTION_ENTRY.length() + 1), v); - } - }); - - final DescriptorProperties properties = new DescriptorProperties(true); - properties.putProperties(enrichedProperties); - - return new ExecutionEntry(properties); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/context/ExecutionContext.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/context/ExecutionContext.java deleted file mode 100644 index b1f9ba1bc6..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/context/ExecutionContext.java +++ /dev/null @@ -1,655 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.factory.LinkisYarnClusterClientFactory; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import java.lang.reflect.Method; -import java.net.URL; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Optional; -import java.util.function.Supplier; -import javax.annotation.Nullable; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.dag.Pipeline; -import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.client.ClientUtils; -import org.apache.flink.client.deployment.ClusterClientServiceLoader; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableConfig; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.bridge.java.BatchTableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.bridge.java.internal.BatchTableEnvironmentImpl; -import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl; -import org.apache.flink.table.catalog.Catalog; -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.client.config.entries.ExecutionEntry; -import org.apache.flink.table.client.config.entries.SinkTableEntry; -import org.apache.flink.table.client.config.entries.SourceSinkTableEntry; -import org.apache.flink.table.client.config.entries.SourceTableEntry; -import org.apache.flink.table.client.config.entries.TableEntry; -import org.apache.flink.table.client.config.entries.TemporalTableEntry; -import org.apache.flink.table.client.config.entries.ViewEntry; -import org.apache.flink.table.delegation.Executor; -import org.apache.flink.table.delegation.ExecutorFactory; -import org.apache.flink.table.delegation.Planner; -import org.apache.flink.table.delegation.PlannerFactory; -import org.apache.flink.table.descriptors.CoreModuleDescriptorValidator; -import org.apache.flink.table.factories.BatchTableSinkFactory; -import org.apache.flink.table.factories.BatchTableSourceFactory; -import org.apache.flink.table.factories.CatalogFactory; -import org.apache.flink.table.factories.ComponentFactoryService; -import org.apache.flink.table.factories.ModuleFactory; -import org.apache.flink.table.factories.TableFactoryService; -import org.apache.flink.table.factories.TableSinkFactory; -import org.apache.flink.table.factories.TableSourceFactory; -import org.apache.flink.table.functions.AggregateFunction; -import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.flink.table.functions.FunctionService; -import org.apache.flink.table.functions.ScalarFunction; -import org.apache.flink.table.functions.TableFunction; -import org.apache.flink.table.functions.UserDefinedFunction; -import org.apache.flink.table.module.Module; -import org.apache.flink.table.module.ModuleManager; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.table.sources.TableSource; -import org.apache.flink.util.TemporaryClassLoaderContext; -import org.apache.flink.yarn.YarnClusterDescriptor; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * 运行上下文 - */ -public class ExecutionContext { - - private static final Logger LOG = LoggerFactory.getLogger(ExecutionContext.class); - - private final Environment environment; - private final ClassLoader classLoader; - - private final Configuration flinkConfig; - private final LinkisYarnClusterClientFactory clusterClientFactory; - - private TableEnvironment tableEnv; - private ExecutionEnvironment execEnv; - private StreamExecutionEnvironment streamExecEnv; - private Executor executor; - - // Members that should be reused in the same session. - private SessionState sessionState; - - private ExecutionContext( - Environment environment, - @Nullable SessionState sessionState, - List dependencies, - Configuration flinkConfig, - ClusterClientServiceLoader clusterClientServiceLoader) throws SqlExecutionException { - this.environment = environment; - this.flinkConfig = flinkConfig; - // create class loader - classLoader = ClientUtils.buildUserCodeClassLoader( - Collections.emptyList(), - Collections.emptyList(), - this.getClass().getClassLoader(), - flinkConfig); - - // Initialize the TableEnvironment. - initializeTableEnvironment(sessionState); - - LOG.debug("Deployment descriptor: {}", environment.getDeployment()); - LOG.info("flinkConfig config: {}", flinkConfig); - clusterClientFactory = new LinkisYarnClusterClientFactory(); - } - - - public void setString(String key, String value) { - this.flinkConfig.setString(key, value); - } - - public void setBoolean(String key, boolean value) { - this.flinkConfig.setBoolean(key, value); - } - - public Configuration getFlinkConfig() { - return flinkConfig; - } - - public ClassLoader getClassLoader() { - return classLoader; - } - - public Environment getEnvironment() { - return environment; - } - - public YarnClusterDescriptor createClusterDescriptor() { - return clusterClientFactory.createClusterDescriptor(this.flinkConfig); - } - - public Map getCatalogs() { - Map catalogs = new HashMap<>(); - for (String name : tableEnv.listCatalogs()) { - tableEnv.getCatalog(name).ifPresent(c -> catalogs.put(name, c)); - } - return catalogs; - } - - public SessionState getSessionState() { - return this.sessionState; - } - - /** - * Executes the given supplier using the execution context's classloader as thread classloader. - */ - public R wrapClassLoader(Supplier supplier) { - try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(classLoader)) { - return supplier.get(); - } - } - - /** - * Executes the given Runnable using the execution context's classloader as thread classloader. - */ - void wrapClassLoader(Runnable runnable) { - try (TemporaryClassLoaderContext ignored = TemporaryClassLoaderContext.of(classLoader)) { - runnable.run(); - } - } - - public TableEnvironment getTableEnvironment() { - return tableEnv; - } - - public ExecutionConfig getExecutionConfig() { - if (streamExecEnv != null) { - return streamExecEnv.getConfig(); - } else { - return execEnv.getConfig(); - } - } - - public LinkisYarnClusterClientFactory getClusterClientFactory() { - return clusterClientFactory; - } - - public Pipeline createPipeline(String name) { - return wrapClassLoader(() -> { - if (streamExecEnv != null) { - StreamTableEnvironmentImpl streamTableEnv = (StreamTableEnvironmentImpl) tableEnv; - return streamTableEnv.getPipeline(name); - } else { - BatchTableEnvironmentImpl batchTableEnv = (BatchTableEnvironmentImpl) tableEnv; - return batchTableEnv.getPipeline(name); - } - }); - } - - - /** Returns a builder for this {@link ExecutionContext}. */ - public static Builder builder( - Environment defaultEnv, - Environment sessionEnv, - List dependencies, - Configuration configuration, - ClusterClientServiceLoader serviceLoader) { - return new Builder(defaultEnv, sessionEnv, dependencies, configuration, - serviceLoader); - } - - private Module createModule(Map moduleProperties, ClassLoader classLoader) { - final ModuleFactory factory = - TableFactoryService.find(ModuleFactory.class, moduleProperties, classLoader); - return factory.createModule(moduleProperties); - } - - private Catalog createCatalog(String name, Map catalogProperties, ClassLoader classLoader) { - final CatalogFactory factory = - TableFactoryService.find(CatalogFactory.class, catalogProperties, classLoader); - return factory.createCatalog(name, catalogProperties); - } - - private static TableSource createTableSource(ExecutionEntry execution, Map sourceProperties, - ClassLoader classLoader) throws SqlExecutionException { - if (execution.isStreamingPlanner()) { - final TableSourceFactory factory = (TableSourceFactory) - TableFactoryService.find(TableSourceFactory.class, sourceProperties, classLoader); - return factory.createTableSource(sourceProperties); - } else if (execution.isBatchPlanner()) { - final BatchTableSourceFactory factory = (BatchTableSourceFactory) - TableFactoryService.find(BatchTableSourceFactory.class, sourceProperties, classLoader); - return factory.createBatchTableSource(sourceProperties); - } - throw new SqlExecutionException("Unsupported execution type for sources."); - } - - private static TableSink createTableSink(ExecutionEntry execution, Map sinkProperties, - ClassLoader classLoader) throws SqlExecutionException { - if (execution.isStreamingPlanner()) { - final TableSinkFactory factory = (TableSinkFactory) - TableFactoryService.find(TableSinkFactory.class, sinkProperties, classLoader); - return factory.createTableSink(sinkProperties); - } else if (execution.isBatchPlanner()) { - final BatchTableSinkFactory factory = (BatchTableSinkFactory) - TableFactoryService.find(BatchTableSinkFactory.class, sinkProperties, classLoader); - return factory.createBatchTableSink(sinkProperties); - } - throw new SqlExecutionException("Unsupported execution type for sinks."); - } - - private TableEnvironment createStreamTableEnvironment( - StreamExecutionEnvironment env, - EnvironmentSettings settings, - TableConfig config, - Executor executor, - CatalogManager catalogManager, - ModuleManager moduleManager, - FunctionCatalog functionCatalog) { - final Map plannerProperties = settings.toPlannerProperties(); - final Planner planner = ComponentFactoryService.find(PlannerFactory.class, plannerProperties) - .create(plannerProperties, executor, config, functionCatalog, catalogManager); - - return new StreamTableEnvironmentImpl( - catalogManager, - moduleManager, - functionCatalog, - config, - env, - planner, - executor, - settings.isStreamingMode(), - classLoader); - } - - private static Executor lookupExecutor( - Map executorProperties, - StreamExecutionEnvironment executionEnvironment) { - try { - ExecutorFactory executorFactory = ComponentFactoryService.find(ExecutorFactory.class, executorProperties); - Method createMethod = executorFactory.getClass() - .getMethod("create", Map.class, StreamExecutionEnvironment.class); - - return (Executor) createMethod.invoke( - executorFactory, - executorProperties, - executionEnvironment); - } catch (Exception e) { - throw new TableException( - "Could not instantiate the executor. Make sure a planner module is on the classpath", - e); - } - } - - private void initializeTableEnvironment(@Nullable SessionState sessionState) throws SqlExecutionException { - final EnvironmentSettings settings = environment.getExecution().getEnvironmentSettings(); - // Step 0.0 Initialize the table configuration. - final TableConfig config = new TableConfig(); - environment.getConfiguration().asMap().forEach((k, v) -> - config.getConfiguration().setString(k, v)); - final boolean noInheritedState = sessionState == null; - if (noInheritedState) { - //-------------------------------------------------------------------------------------------------------------- - // Step.1 Create environments - //-------------------------------------------------------------------------------------------------------------- - // Step 1.0 Initialize the ModuleManager if required. - final ModuleManager moduleManager = new ModuleManager(); - // Step 1.1 Initialize the CatalogManager if required. - final CatalogManager catalogManager = CatalogManager.newBuilder() - .classLoader(classLoader) - .config(config.getConfiguration()) - .defaultCatalog( - settings.getBuiltInCatalogName(), - new GenericInMemoryCatalog( - settings.getBuiltInCatalogName(), - settings.getBuiltInDatabaseName())) - .build(); - // Step 1.2 Initialize the FunctionCatalog if required. - final FunctionCatalog functionCatalog = new FunctionCatalog(config, catalogManager, moduleManager); - // Step 1.4 Set up session state. - this.sessionState = SessionState.of(catalogManager, moduleManager, functionCatalog); - - // Must initialize the table environment before actually the - createTableEnvironment(settings, config, catalogManager, moduleManager, functionCatalog); - - //-------------------------------------------------------------------------------------------------------------- - // Step.2 Create modules and load them into the TableEnvironment. - //-------------------------------------------------------------------------------------------------------------- - // No need to register the modules info if already inherit from the same session. - Map modules = new LinkedHashMap<>(); - environment.getModules().forEach((name, entry) -> - modules.put(name, createModule(entry.asMap(), classLoader)) - ); - if (!modules.isEmpty()) { - // unload core module first to respect whatever users configure - tableEnv.unloadModule(CoreModuleDescriptorValidator.MODULE_TYPE_CORE); - modules.forEach(tableEnv::loadModule); - } - - //-------------------------------------------------------------------------------------------------------------- - // Step.3 create user-defined functions and temporal tables then register them. - //-------------------------------------------------------------------------------------------------------------- - // No need to register the functions if already inherit from the same session. - registerFunctions(); - - //-------------------------------------------------------------------------------------------------------------- - // Step.4 Create catalogs and register them. - //-------------------------------------------------------------------------------------------------------------- - // No need to register the catalogs if already inherit from the same session. - initializeCatalogs(); - } else { - // Set up session state. - this.sessionState = sessionState; - createTableEnvironment( - settings, - config, - sessionState.catalogManager, - sessionState.moduleManager, - sessionState.functionCatalog); - } - } - - private void createTableEnvironment( - EnvironmentSettings settings, - TableConfig config, - CatalogManager catalogManager, - ModuleManager moduleManager, - FunctionCatalog functionCatalog) { - - //流式 - if (environment.getExecution().isStreamingPlanner()) { - streamExecEnv = createStreamExecutionEnvironment(); - execEnv = null; - final Map executorProperties = settings.toExecutorProperties(); - executor = lookupExecutor(executorProperties, streamExecEnv); - tableEnv = createStreamTableEnvironment( - streamExecEnv, - settings, - config, - executor, - catalogManager, - moduleManager, - functionCatalog); - return ; - } - //默认批 - streamExecEnv = null; - execEnv = createExecutionEnvironment(); - executor = null; - tableEnv = new BatchTableEnvironmentImpl( - execEnv, - config, - catalogManager, - moduleManager); - - } - - private void initializeCatalogs() throws SqlExecutionException { - //-------------------------------------------------------------------------------------------------------------- - // Step.1 Create catalogs and register them. - //-------------------------------------------------------------------------------------------------------------- - wrapClassLoader(() -> environment.getCatalogs().forEach((name, entry) -> { - Catalog catalog = createCatalog(name, entry.asMap(), classLoader); - tableEnv.registerCatalog(name, catalog); - }) - ); - - //-------------------------------------------------------------------------------------------------------------- - // Step.2 create table sources & sinks, and register them. - //-------------------------------------------------------------------------------------------------------------- - Map> tableSources = new HashMap<>(); - Map> tableSinks = new HashMap<>(); - for(Entry keyValue: environment.getTables().entrySet()) { - String name = keyValue.getKey(); - TableEntry entry = keyValue.getValue(); - if (entry instanceof SourceTableEntry || entry instanceof SourceSinkTableEntry) { - tableSources.put(name, createTableSource(environment.getExecution(), entry.asMap(), classLoader)); - } - if (entry instanceof SinkTableEntry || entry instanceof SourceSinkTableEntry) { - tableSinks.put(name, createTableSink(environment.getExecution(), entry.asMap(), classLoader)); - } - } - // register table sources - tableSources.forEach(tableEnv::registerTableSource); - // register table sinks - tableSinks.forEach(tableEnv::registerTableSink); - - //-------------------------------------------------------------------------------------------------------------- - // Step.4 Register temporal tables. - //-------------------------------------------------------------------------------------------------------------- - for(Entry keyValue : environment.getTables().entrySet()) { - TableEntry entry = keyValue.getValue(); - if (entry instanceof TemporalTableEntry) { - final TemporalTableEntry temporalTableEntry = (TemporalTableEntry) entry; - registerTemporalTable(temporalTableEntry); - } - } - - //-------------------------------------------------------------------------------------------------------------- - // Step.5 Register views in specified order. - //-------------------------------------------------------------------------------------------------------------- - for(Entry keyValue : environment.getTables().entrySet()) { - // if registering a view fails at this point, - // it means that it accesses tables that are not available anymore - TableEntry entry = keyValue.getValue(); - if (entry instanceof ViewEntry) { - final ViewEntry viewEntry = (ViewEntry) entry; - registerView(viewEntry); - } - } - - //-------------------------------------------------------------------------------------------------------------- - // Step.6 Set current catalog and database. - //-------------------------------------------------------------------------------------------------------------- - // Switch to the current catalog. - Optional catalog = environment.getExecution().getCurrentCatalog(); - catalog.ifPresent(tableEnv::useCatalog); - - // Switch to the current database. - Optional database = environment.getExecution().getCurrentDatabase(); - database.ifPresent(tableEnv::useDatabase); - } - - private ExecutionEnvironment createExecutionEnvironment() { - final ExecutionEnvironment execEnv = ExecutionEnvironment.getExecutionEnvironment(); - execEnv.setRestartStrategy(environment.getExecution().getRestartStrategy()); - execEnv.setParallelism(environment.getExecution().getParallelism()); - return execEnv; - } - - private StreamExecutionEnvironment createStreamExecutionEnvironment() { - final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setRestartStrategy(environment.getExecution().getRestartStrategy()); - env.setParallelism(environment.getExecution().getParallelism()); - env.setMaxParallelism(environment.getExecution().getMaxParallelism()); - env.setStreamTimeCharacteristic(environment.getExecution().getTimeCharacteristic()); - if (env.getStreamTimeCharacteristic() == TimeCharacteristic.EventTime) { - env.getConfig().setAutoWatermarkInterval(environment.getExecution().getPeriodicWatermarksInterval()); - } - return env; - } - - private void registerFunctions() throws SqlExecutionException { - Map functions = new LinkedHashMap<>(); - environment.getFunctions().forEach((name, entry) -> { - final UserDefinedFunction function = FunctionService - .createFunction(entry.getDescriptor(), classLoader, false); - functions.put(name, function); - }); - registerFunctions(functions); - } - - private void registerFunctions(Map functions) throws SqlExecutionException { - if (tableEnv instanceof StreamTableEnvironment) { - StreamTableEnvironment streamTableEnvironment = (StreamTableEnvironment) tableEnv; - for(Entry keyValue: functions.entrySet()) { - String k = keyValue.getKey(); - FunctionDefinition v = keyValue.getValue(); - if (v instanceof ScalarFunction) { - streamTableEnvironment.registerFunction(k, (ScalarFunction) v); - } else if (v instanceof AggregateFunction) { - streamTableEnvironment.registerFunction(k, (AggregateFunction) v); - } else if (v instanceof TableFunction) { - streamTableEnvironment.registerFunction(k, (TableFunction) v); - } else { - throw new SqlExecutionException("Unsupported function type: " + v.getClass().getName()); - } - } - } else { - BatchTableEnvironment batchTableEnvironment = (BatchTableEnvironment) tableEnv; - for(Entry keyValue: functions.entrySet()) { - String k = keyValue.getKey(); - FunctionDefinition v = keyValue.getValue(); - if (v instanceof ScalarFunction) { - batchTableEnvironment.registerFunction(k, (ScalarFunction) v); - } else if (v instanceof AggregateFunction) { - batchTableEnvironment.registerFunction(k, (AggregateFunction) v); - } else if (v instanceof TableFunction) { - batchTableEnvironment.registerFunction(k, (TableFunction) v); - } else { - throw new SqlExecutionException("Unsupported function type: " + v.getClass().getName()); - } - } - } - } - - private void registerView(ViewEntry viewEntry) throws SqlExecutionException { - try { - tableEnv.registerTable(viewEntry.getName(), tableEnv.sqlQuery(viewEntry.getQuery())); - } catch (Exception e) { - throw new SqlExecutionException( - "Invalid view '" + viewEntry.getName() + "' with query:\n" + viewEntry.getQuery() - + "\nCause: " + e.getMessage()); - } - } - - private void registerTemporalTable(TemporalTableEntry temporalTableEntry) throws SqlExecutionException { - try { - final Table table = tableEnv.scan(temporalTableEntry.getHistoryTable()); - final TableFunction function = table.createTemporalTableFunction( - temporalTableEntry.getTimeAttribute(), - String.join(",", temporalTableEntry.getPrimaryKeyFields())); - if (tableEnv instanceof StreamTableEnvironment) { - StreamTableEnvironment streamTableEnvironment = (StreamTableEnvironment) tableEnv; - streamTableEnvironment.registerFunction(temporalTableEntry.getName(), function); - } else { - BatchTableEnvironment batchTableEnvironment = (BatchTableEnvironment) tableEnv; - batchTableEnvironment.registerFunction(temporalTableEntry.getName(), function); - } - } catch (Exception e) { - throw new SqlExecutionException( - "Invalid temporal table '" + temporalTableEntry.getName() + "' over table '" + - temporalTableEntry.getHistoryTable() + ".\nCause: " + e.getMessage()); - } - } - - //~ Inner Class ------------------------------------------------------------------------------- - - /** Builder for {@link ExecutionContext}. */ - public static class Builder { - // Required members. - private final Environment sessionEnv; - private final List dependencies; - private final Configuration configuration; - private final ClusterClientServiceLoader serviceLoader; - - private Environment defaultEnv; - private Environment currentEnv; - - // Optional members. - @Nullable - private SessionState sessionState; - - private Builder( - Environment defaultEnv, - @Nullable Environment sessionEnv, - List dependencies, - Configuration configuration, - ClusterClientServiceLoader serviceLoader) { - this.defaultEnv = defaultEnv; - this.sessionEnv = sessionEnv; - this.dependencies = dependencies; - this.configuration = configuration; - this.serviceLoader = serviceLoader; - } - - public Builder env(Environment environment) { - this.currentEnv = environment; - return this; - } - - public Builder sessionState(SessionState sessionState) { - this.sessionState = sessionState; - return this; - } - - public ExecutionContext build() throws SqlExecutionException { - if(sessionEnv == null){ - this.currentEnv = defaultEnv; - } - try { - return new ExecutionContext( - this.currentEnv == null ? Environment.merge(defaultEnv, sessionEnv) : this.currentEnv, - this.sessionState, - this.dependencies, - this.configuration, - this.serviceLoader); - } catch (Throwable t) { - // catch everything such that a configuration does not crash the executor - throw new SqlExecutionException("Could not create execution context.", t); - } - } - } - - /** Represents the state that should be reused in one session. **/ - public static class SessionState { - public final CatalogManager catalogManager; - public final ModuleManager moduleManager; - public final FunctionCatalog functionCatalog; - - private SessionState( - CatalogManager catalogManager, - ModuleManager moduleManager, - FunctionCatalog functionCatalog) { - this.catalogManager = catalogManager; - this.moduleManager = moduleManager; - this.functionCatalog = functionCatalog; - } - - public static SessionState of( - CatalogManager catalogManager, - ModuleManager moduleManager, - FunctionCatalog functionCatalog) { - return new SessionState(catalogManager, moduleManager, functionCatalog); - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapter.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapter.java deleted file mode 100644 index ba13a0103b..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapter.java +++ /dev/null @@ -1,166 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import java.io.Closeable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; -import java.util.function.Supplier; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.client.deployment.ClusterRetrieveException; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.yarn.YarnClusterDescriptor; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * @program: linkis - * @description: 集群交互适配器,适合datastream、sql方式作业 - */ -public abstract class ClusterDescriptorAdapter implements Closeable { - public static final Logger LOG = LoggerFactory.getLogger(ClusterDescriptorAdapter.class); - - public static final long CLIENT_REQUEST_TIMEOUT = FlinkEnvConfiguration.FLINK_CLIENT_REQUEST_TIMEOUT().getValue().toLong(); - - protected final ExecutionContext executionContext; - // jobId is not null only after job is submitted - protected JobID jobId; - protected ApplicationId clusterID; - protected ClusterClient clusterClient; - private YarnClusterDescriptor clusterDescriptor; - - public JobID getJobId() { - return jobId; - } - - public ApplicationId getClusterID() { - return clusterID; - } - - public String getWebInterfaceUrl() { - return webInterfaceUrl; - } - - protected String webInterfaceUrl; - - public ClusterDescriptorAdapter( - ExecutionContext executionContext, - JobID jobId) { - this.executionContext = executionContext; - this.jobId = jobId; - this.clusterID = executionContext.getClusterClientFactory().getClusterId(executionContext.getFlinkConfig()); - } - - /** - * Returns the status of the flink job. - */ - public JobStatus getJobStatus() throws JobExecutionException { - if (jobId == null) { - throw new JobExecutionException("No job has been submitted. This is a bug."); - } - return bridgeClientRequest(this.executionContext, jobId, () -> clusterClient.getJobStatus(jobId), false); - } - - /** - * Cancel the flink job. - */ - public void cancelJob() throws JobExecutionException { - if (jobId == null) { - throw new JobExecutionException("No job has been submitted. This is a bug."); - } - LOG.info("Start to cancel job {}.", jobId); - bridgeClientRequest(this.executionContext, jobId, () -> clusterClient.cancel(jobId), true); - } - - public abstract void deployCluster(String[] programArguments, String applicationClassName) throws JobExecutionException; - - /** - * The reason of using ClusterClient instead of JobClient to retrieve a cluster is - * the JobClient can't know whether the job is finished on yarn-per-job mode. - * - *

If a job is finished, JobClient always get java.util.concurrent.TimeoutException - * when getting job status and canceling a job after job is finished. - * This method will throw org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException - * when creating a ClusterClient if the job is finished. This is more user-friendly. - */ - protected R bridgeClientRequest( - ExecutionContext executionContext, - JobID jobId, - Supplier> function, boolean ignoreError) throws JobExecutionException { - if(clusterClient == null) { - if (this.clusterID == null) { - LOG.error("Cluster information don't exist."); - throw new JobExecutionException("Cluster information don't exist."); - } - clusterDescriptor = executionContext.createClusterDescriptor(); - try { - clusterClient = clusterDescriptor.retrieve(this.clusterID).getClusterClient(); - } catch (ClusterRetrieveException e) { - LOG.error(String.format("Job: %s could not retrieve or create a cluster.", jobId), e); - throw new JobExecutionException(String.format("Job: %s could not retrieve or create a cluster.", jobId), e); - } - } - try { - return function.get().get(CLIENT_REQUEST_TIMEOUT, TimeUnit.MILLISECONDS); - } catch (Exception e) { - if(ignoreError) { - return null; - } else { - LOG.error(String.format("Job: %s operation failed!", jobId), e); - throw new JobExecutionException(String.format("Job: %s operation failed!", jobId), e); - } - } - } - - @Override - public String toString() { - return "ClusterDescriptorAdapter{" + - "jobId=" + jobId + - ", clusterID=" + clusterID + - '}'; - } - - @Override - public void close() { - if(clusterClient != null) { - clusterClient.shutDownCluster(); - clusterClient.close(); - } - if(clusterDescriptor != null) { - clusterDescriptor.close(); - } - } - - /** - * Checks whether this job state is globally terminal. - * A globally terminal job is complete and cannot fail any more - * and will not be restarted or recovered by another standby master node. - * - *

When a globally terminal state has been reached, - * all recovery data for the job is dropped from the high-availability services. - * - * @return True, if this job status is globally terminal, false otherwise. - */ - public abstract boolean isGloballyTerminalState() throws JobExecutionException; -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapterFactory.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapterFactory.java deleted file mode 100644 index 28f92674c9..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ClusterDescriptorAdapterFactory.java +++ /dev/null @@ -1,53 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import org.apache.flink.api.common.JobID; -import org.apache.flink.configuration.DeploymentOptions; -import org.apache.flink.yarn.configuration.YarnDeploymentTarget; - -/** - * 集群交互适配器工厂 - */ -public class ClusterDescriptorAdapterFactory { - - public static ClusterDescriptorAdapter create( - ExecutionContext executionContext, - JobID jobId) { - String yarnDeploymentTarget = executionContext.getFlinkConfig().get(DeploymentOptions.TARGET); - ClusterDescriptorAdapter clusterDescriptorAdapter = null; - if (YarnDeploymentTarget.PER_JOB.getName().equals(yarnDeploymentTarget)) { - clusterDescriptorAdapter = new YarnPerJobClusterDescriptorAdapter( - executionContext, - jobId); - } else if (YarnDeploymentTarget.APPLICATION.getName().equals(yarnDeploymentTarget)) { - clusterDescriptorAdapter = new YarnApplicationClusterDescriptorAdapter( - executionContext, - jobId); - } else if(YarnDeploymentTarget.SESSION.getName().equals(yarnDeploymentTarget)) { - // Just the same with per_job. - clusterDescriptorAdapter = new YarnPerJobClusterDescriptorAdapter( - executionContext, - jobId); - } - return clusterDescriptorAdapter; - } - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ProgramDeployer.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ProgramDeployer.java deleted file mode 100644 index e4812e661c..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/ProgramDeployer.java +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import java.util.concurrent.CompletableFuture; -import org.apache.flink.api.dag.Pipeline; -import org.apache.flink.client.deployment.executors.AbstractJobClusterExecutor; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.DeploymentOptions; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.core.execution.PipelineExecutor; -import org.apache.flink.yarn.YarnClusterClientFactory; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * @program: linkis - * @description: 集群部署 Table program - */ -public class ProgramDeployer { - private static final Logger LOG = LoggerFactory.getLogger(ProgramDeployer.class); - - private final Configuration configuration; - private final Pipeline pipeline; - private final String jobName; - - /** - * Deploys a table program on the cluster. - * - * @param configuration the {@link Configuration} that is used for deployment - * @param jobName job name of the Flink job to be submitted - * @param pipeline Flink {@link Pipeline} to execute - */ - public ProgramDeployer( - Configuration configuration, - String jobName, - Pipeline pipeline) { - this.configuration = configuration; - this.pipeline = pipeline; - this.jobName = jobName; - } - - public CompletableFuture deploy(ExecutionContext context) throws Exception { - LOG.info("Submitting job {} for query {}`", pipeline, jobName); - if (LOG.isDebugEnabled()) { - LOG.debug("Submitting job {} with configuration: \n{}", pipeline, configuration); - } - if (configuration.get(DeploymentOptions.TARGET) == null) { - throw new JobExecutionException("No execution. Target specified in your configuration file."); - } - final PipelineExecutor executor = new AbstractJobClusterExecutor(context.getClusterClientFactory()); - return executor.execute(pipeline, configuration); - } -} - diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnApplicationClusterDescriptorAdapter.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnApplicationClusterDescriptorAdapter.java deleted file mode 100644 index 01ce1074ea..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnApplicationClusterDescriptorAdapter.java +++ /dev/null @@ -1,59 +0,0 @@ -package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.ExecutorInitException; -import java.util.Objects; -import java.util.concurrent.TimeUnit; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import org.apache.flink.api.common.JobID; -import org.apache.flink.client.deployment.ClusterSpecification; -import org.apache.flink.client.deployment.application.ApplicationConfiguration; -import org.apache.flink.client.program.ClusterClientProvider; -import org.apache.flink.yarn.YarnClusterDescriptor; -import org.apache.hadoop.yarn.api.records.ApplicationId; - -/** - * @program: flink-parent - * @description: - */ -public class YarnApplicationClusterDescriptorAdapter extends ClusterDescriptorAdapter { - - public YarnApplicationClusterDescriptorAdapter(ExecutionContext executionContext, JobID jobId) { - super(executionContext, jobId); - } - - @Override - public void deployCluster(String[] programArguments, String applicationClassName) throws JobExecutionException { - ApplicationConfiguration applicationConfiguration = new ApplicationConfiguration(programArguments,applicationClassName); - ClusterSpecification clusterSpecification = this.executionContext.getClusterClientFactory().getClusterSpecification(this.executionContext.getFlinkConfig()); - YarnClusterDescriptor clusterDescriptor = this.executionContext.createClusterDescriptor(); - try { - ClusterClientProvider clusterClientProvider = clusterDescriptor.deployApplicationCluster( - clusterSpecification, - applicationConfiguration); - clusterClient = clusterClientProvider.getClusterClient(); - super.clusterID = clusterClient.getClusterId(); - super.webInterfaceUrl = clusterClient.getWebInterfaceURL(); - } catch (Exception e) { - throw new JobExecutionException(e.getMessage()); - } - - } - - public boolean initJobId() throws Exception { - clusterClient.listJobs().thenAccept(list -> list.forEach(jobStatusMessage -> { - if (Objects.nonNull(jobStatusMessage.getJobId())) { - jobId = jobStatusMessage.getJobId(); - } - })).get(CLIENT_REQUEST_TIMEOUT, TimeUnit.MILLISECONDS); - return null != jobId; - } - - - @Override - public boolean isGloballyTerminalState() { - return false; - } - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnPerJobClusterDescriptorAdapter.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnPerJobClusterDescriptorAdapter.java deleted file mode 100644 index f402877eae..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/deployment/YarnPerJobClusterDescriptorAdapter.java +++ /dev/null @@ -1,84 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import java.util.concurrent.CompletableFuture; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * @program: flink-parent - * @description: - */ -public class YarnPerJobClusterDescriptorAdapter extends ClusterDescriptorAdapter { - private static final Logger LOG = LoggerFactory.getLogger(YarnPerJobClusterDescriptorAdapter.class); - - public YarnPerJobClusterDescriptorAdapter( - ExecutionContext executionContext, - JobID jobId) { - super(executionContext, jobId); - } - - @Override - public boolean isGloballyTerminalState() throws JobExecutionException { - boolean isGloballyTerminalState; - try { - JobStatus jobStatus = getJobStatus(); - isGloballyTerminalState = jobStatus.isGloballyTerminalState(); - } catch (JobExecutionException e) { - if (isYarnApplicationStopped(e)) { - isGloballyTerminalState = true; - } else { - throw e; - } - } - - return isGloballyTerminalState; - } - - /** - * The yarn application is not running when its final status is not UNDEFINED. - * - *

In this case, it will throw - * RuntimeException("The Yarn application " + applicationId + " doesn't run anymore.") - * from retrieve method in YarnClusterDescriptor.java - */ - private boolean isYarnApplicationStopped(Throwable e) { - do { - String exceptionMessage = e.getMessage(); - if (StringUtils.equals(exceptionMessage, "The Yarn application " + clusterID + " doesn't run anymore.")) { - LOG.info("{} is stopped.", clusterID); - return true; - } - e = e.getCause(); - } while (e != null); - return false; - } - - @Override - public void deployCluster(String[] programArguments, String applicationClassName) throws JobExecutionException { - webInterfaceUrl = this.bridgeClientRequest(executionContext, jobId, () -> CompletableFuture.completedFuture(clusterClient.getWebInterfaceURL()), false); - } - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/factory/LinkisYarnClusterClientFactory.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/factory/LinkisYarnClusterClientFactory.java deleted file mode 100644 index 1c88ad80df..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/factory/LinkisYarnClusterClientFactory.java +++ /dev/null @@ -1,70 +0,0 @@ -package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.factory; - -import static org.apache.flink.configuration.ConfigOptions.key; -import static org.apache.flink.util.Preconditions.checkNotNull; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils.YarnConfLoader; -import java.io.Closeable; -import java.io.IOException; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.DeploymentOptionsInternal; -import org.apache.flink.yarn.YarnClientYarnClusterInformationRetriever; -import org.apache.flink.yarn.YarnClusterClientFactory; -import org.apache.flink.yarn.YarnClusterDescriptor; -import org.apache.flink.yarn.configuration.YarnLogConfigUtil; -import org.apache.hadoop.yarn.client.api.YarnClient; -import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * @program: linkis - */ -public class LinkisYarnClusterClientFactory extends YarnClusterClientFactory implements Closeable { - - private static final Logger LOG = LoggerFactory.getLogger(LinkisYarnClusterClientFactory.class); - - public static final ConfigOption YARN_CONFIG_DIR = - key("$internal.yarn.config-dir") - .stringType() - .noDefaultValue() - .withDescription("**DO NOT USE** The location of the log config file, e.g. the path to your log4j.properties for log4j."); - - private YarnConfiguration yarnConfiguration; - private YarnClient yarnClient; - - private void initYarnClient(Configuration configuration) { - String yarnConfDir = configuration.getString(YARN_CONFIG_DIR); - yarnConfiguration = YarnConfLoader.getYarnConf(yarnConfDir); - yarnClient = YarnClient.createYarnClient(); - yarnClient.init(yarnConfiguration); - yarnClient.start(); - } - - @Override - public YarnClusterDescriptor createClusterDescriptor(Configuration configuration) { - checkNotNull(configuration); - final String configurationDirectory = - configuration.get(DeploymentOptionsInternal.CONF_DIR); - YarnLogConfigUtil.setLogConfigFileInConfig(configuration, configurationDirectory); - if(yarnClient == null) { - synchronized (this) { - if(yarnClient == null) { - initYarnClient(configuration); - } - } - } - return new YarnClusterDescriptor( - configuration, - yarnConfiguration, - yarnClient, - YarnClientYarnClusterInformationRetriever.create(yarnClient), - true); - } - - @Override - public void close() throws IOException { - yarnClient.close(); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/AbstractResult.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/AbstractResult.java deleted file mode 100644 index 98c11e99e4..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/AbstractResult.java +++ /dev/null @@ -1,51 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; - - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.FlinkListenerGroupImpl; - -public abstract class AbstractResult extends FlinkListenerGroupImpl implements Result { - - protected C clusterId; - protected String webInterfaceUrl; - - @Override - public void setClusterInformation(C clusterId, String webInterfaceUrl) { - if (this.clusterId != null || this.webInterfaceUrl != null) { - throw new IllegalStateException("Cluster information is already present."); - } - this.clusterId = clusterId; - this.webInterfaceUrl = webInterfaceUrl; - } - - public C getClusterId() { - if (this.clusterId == null) { - throw new IllegalStateException("Cluster ID has not been set."); - } - return clusterId; - } - - public String getWebInterfaceUrl() { - if (this.webInterfaceUrl == null) { - throw new IllegalStateException("Cluster web interface URL has not been set."); - } - return webInterfaceUrl; - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/BatchResult.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/BatchResult.java deleted file mode 100644 index 69af532024..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/BatchResult.java +++ /dev/null @@ -1,141 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.RowsType; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Consumer; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.accumulators.SerializedListAccumulator; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.client.gateway.local.CollectBatchTableSink; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.types.Row; -import org.apache.flink.util.AbstractID; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -public class BatchResult extends AbstractResult { - - private Logger LOG = LoggerFactory.getLogger(getClass()); - - private final String accumulatorName; - private final CollectBatchTableSink tableSink; - private final Object resultLock; - private final ClassLoader classLoader; - - private AtomicReference executionException = new AtomicReference<>(); - private List resultTable; - - private boolean allResultRetrieved = false; - - public BatchResult( - TableSchema tableSchema, - RowTypeInfo outputType, - ExecutionConfig config, - ClassLoader classLoader) { - // TODO supports large result set - accumulatorName = new AbstractID().toString(); - tableSink = new CollectBatchTableSink(accumulatorName, outputType.createSerializer(config), tableSchema); - resultLock = new Object(); - this.classLoader = checkNotNull(classLoader); - } - - @Override - public void startRetrieval(JobClient jobClient) { - CompletableFuture.completedFuture(jobClient) - .thenCompose(client -> client.getJobExecutionResult(classLoader)) - .thenAccept(new ResultRetrievalHandler()) - .whenComplete((unused, throwable) -> { - if (throwable != null) { - executionException.compareAndSet( - null, - new SqlExecutionException("Error while submitting job.", throwable)); - } - }); - } - - @Override - public TypedResult> retrieveChanges() throws SqlExecutionException { - synchronized (resultLock) { - // the job finished with an exception - SqlExecutionException e = executionException.get(); - if (e != null) { - throw e; - } - // wait for a result - if (null == resultTable) { - return TypedResult.empty(); - } - if (allResultRetrieved) { - return TypedResult.endOfStream(); - } else { - allResultRetrieved = true; - return TypedResult.payload(resultTable); - } - } - } - - @Override - public TableSink getTableSink() { - return tableSink; - } - - @Override - public void close() { - } - - // -------------------------------------------------------------------------------------------- - - private class ResultRetrievalHandler implements Consumer { - - @Override - public void accept(JobExecutionResult jobExecutionResult) { - try { - final ArrayList accResult = jobExecutionResult.getAccumulatorResult(accumulatorName); - if (accResult == null) { - throw new JobExecutionException("The accumulator could not retrieve the result."); - } - final List resultTable = SerializedListAccumulator - .deserializeList(accResult, tableSink.getSerializer()); - // sets the result table all at once - synchronized (resultLock) { - BatchResult.this.resultTable = resultTable; - } - LOG.info("Accept the result, row is "+ resultTable.size()); - getFlinkStatusListeners().forEach(listener -> listener.onSuccess(resultTable.size(), RowsType.Fetched())); - } catch (ClassNotFoundException | IOException | JobExecutionException e) { - getFlinkStatusListeners().forEach(listener -> listener.onFailed("Serialization error while deserialize collected data.", e)); - throw new RuntimeException("Serialization error while deserialize collected data.", e); - } - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ChangelogResult.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ChangelogResult.java deleted file mode 100644 index 4456c6f606..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ChangelogResult.java +++ /dev/null @@ -1,221 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.RowsType; -import java.io.IOException; -import java.net.InetAddress; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicReference; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobExecutionResult; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.streaming.experimental.SocketStreamIterator; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.client.gateway.local.CollectStreamTableSink; -import org.apache.flink.table.sinks.TableSink; -import org.apache.flink.types.Row; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -public class ChangelogResult extends AbstractResult> { - - private Logger LOG = LoggerFactory.getLogger(getClass()); - - private final SocketStreamIterator> iterator; - private final CollectStreamTableSink collectTableSink; - private final ResultRetrievalThread retrievalThread; - private final ClassLoader classLoader; - private CompletableFuture jobExecutionResultFuture; - - private final Object resultLock; - private AtomicReference executionException = new AtomicReference<>(); - private final List> changeRecordBuffer; - private final int maxBufferSize; - - public ChangelogResult( - RowTypeInfo outputType, - TableSchema tableSchema, - ExecutionConfig config, - InetAddress gatewayAddress, - int gatewayPort, - ClassLoader classLoader, - int maxBufferSize) throws SqlExecutionException { - resultLock = new Object(); - - // create socket stream iterator - final TypeInformation> socketType = Types.TUPLE(Types.BOOLEAN, outputType); - final TypeSerializer> serializer = socketType.createSerializer(config); - try { - // pass gateway port and address such that iterator knows where to bind to - iterator = new SocketStreamIterator<>(gatewayPort, gatewayAddress, serializer); - } catch (IOException e) { - throw new SqlExecutionException("Could not start socket for result retrieval.", e); - } - - // create table sink - // pass binding address and port such that sink knows where to send to - collectTableSink = new CollectStreamTableSink( - iterator.getBindAddress(), iterator.getPort(), serializer, tableSchema); - retrievalThread = new ResultRetrievalThread(); - - this.classLoader = checkNotNull(classLoader); - - // prepare for changelog - changeRecordBuffer = new ArrayList<>(); - this.maxBufferSize = maxBufferSize; - } - - @Override - public void startRetrieval(JobClient jobClient) { - // start listener thread - retrievalThread.setName(jobClient.getJobID().toHexString() + "-JobResult-Fetch-Thread"); - retrievalThread.start(); - - jobExecutionResultFuture = CompletableFuture.completedFuture(jobClient) - .thenCompose(client -> client.getJobExecutionResult(classLoader)) - .whenComplete((unused, throwable) -> { - if (throwable != null) { - executionException.compareAndSet( - null, - new JobExecutionException("Error while submitting job.", throwable)); - } - }); - } - - @Override - public TypedResult>> retrieveChanges() throws JobExecutionException { - synchronized (resultLock) { - // retrieval thread is alive return a record if available - // but the program must not have failed - if (isRetrieving() && executionException.get() == null) { - if (changeRecordBuffer.isEmpty()) { - return TypedResult.empty(); - } else { - final List> change = new ArrayList<>(changeRecordBuffer); - changeRecordBuffer.clear(); - resultLock.notify(); - return TypedResult.payload(change); - } - } - // retrieval thread is dead but there is still a record to be delivered - else if (!isRetrieving() && !changeRecordBuffer.isEmpty()) { - final List> change = new ArrayList<>(changeRecordBuffer); - changeRecordBuffer.clear(); - return TypedResult.payload(change); - } - // no results can be returned anymore - else { - return handleMissingResult(); - } - } - } - - @Override - public TableSink getTableSink() { - return collectTableSink; - } - - @Override - public void close() { - retrievalThread.isRunning = false; - retrievalThread.interrupt(); - iterator.close(); - } - - // -------------------------------------------------------------------------------------------- - - private TypedResult handleMissingResult() throws JobExecutionException { - - // check if the monitoring thread is still there - // we need to wait until we know what is going on - if (!jobExecutionResultFuture.isDone()) { - return TypedResult.empty(); - } - - if (executionException.get() != null) { - throw executionException.get(); - } - - // we assume that a bounded job finished - return TypedResult.endOfStream(); - } - - private boolean isRetrieving() { - return retrievalThread.isRunning; - } - - private void processRecord(Tuple2 change) { - synchronized (resultLock) { - // wait if the buffer is full - while (changeRecordBuffer.size() >= maxBufferSize) { - try { - getFlinkStreamingResultSetListeners().forEach(listener -> listener.onResultSetPulled(changeRecordBuffer.size())); - resultLock.wait(); - } catch (InterruptedException e) { - // ignore - } - } - changeRecordBuffer.add(change); - } - LOG.debug("Accept the streaming result, row is "+ change.f1.toString()); - } - - // -------------------------------------------------------------------------------------------- - - private class ResultRetrievalThread extends Thread { - - public volatile boolean isRunning = true; - - @Override - public void run() { - int rows = 0; - try { - while (isRunning && iterator.hasNext()) { - final Tuple2 change = iterator.next(); - processRecord(change); - rows ++; - } - } catch (Exception e) { - // ignore socket exceptions - LOG.warn(getName() + " has finished with an error.", e); - } - int totalRows = rows; - if(!changeRecordBuffer.isEmpty()) { - getFlinkStreamingResultSetListeners().forEach(listener -> listener.onResultSetPulled(changeRecordBuffer.size())); - } - getFlinkStatusListeners().forEach(listener -> listener.onSuccess(totalRows, RowsType.Fetched())); - // no result anymore - // either the job is done or an error occurred - isRunning = false; - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/Result.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/Result.java deleted file mode 100644 index cc05cfe593..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/Result.java +++ /dev/null @@ -1,55 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import java.util.List; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.table.sinks.TableSink; - - -public interface Result { - - /** - * Sets the cluster information of the cluster this result comes from. This method should only be called once. - */ - void setClusterInformation(C clusterId, String webInterfaceUrl); - - /** - * Starts the table program using the given deployer and monitors it's execution. - */ - void startRetrieval(JobClient jobClient); - - /** - * Retrieves the available result records. - */ - TypedResult> retrieveChanges() throws JobExecutionException, SqlExecutionException; - - /** - * Returns the table sink required by this result type. - */ - TableSink getTableSink(); - - /** - * Closes the retrieval and all involved threads. - */ - void close(); - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ResultUtil.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ResultUtil.java deleted file mode 100644 index 10db00ca13..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/ResultUtil.java +++ /dev/null @@ -1,122 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import java.net.InetAddress; -import java.net.InetSocketAddress; -import java.net.UnknownHostException; -import java.util.stream.Stream; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.runtime.net.ConnectionUtils; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.client.config.entries.DeploymentEntry; -import org.apache.flink.table.runtime.types.TypeInfoDataTypeConverter; - - -public class ResultUtil { - - public static BatchResult createBatchResult( - TableSchema schema, - ExecutionConfig config, - ClassLoader classLoader) { - final TypeInformation[] schemaTypeInfos = Stream.of(schema.getFieldDataTypes()) - .map(TypeInfoDataTypeConverter::fromDataTypeToTypeInfo) - .toArray(TypeInformation[]::new); - final RowTypeInfo outputType = new RowTypeInfo(schemaTypeInfos, schema.getFieldNames()); - - return new BatchResult<>(schema, outputType, config, classLoader); - } - - public static ChangelogResult createChangelogResult( - Configuration flinkConfig, - Environment env, - TableSchema schema, - ExecutionConfig config, - ClassLoader classLoader) throws JobExecutionException, SqlExecutionException { - final TypeInformation[] schemaTypeInfos = Stream.of(schema.getFieldDataTypes()) - .map(TypeInfoDataTypeConverter::fromDataTypeToTypeInfo) - .toArray(TypeInformation[]::new); - final RowTypeInfo outputType = new RowTypeInfo(schemaTypeInfos, schema.getFieldNames()); - - // determine gateway address (and port if possible) - final InetAddress gatewayAddress = getGatewayAddress(env.getDeployment(), flinkConfig); - final int gatewayPort = getGatewayPort(env.getDeployment()); - final int maxBufferSize = env.getExecution().getMaxTableResultRows(); - - return new ChangelogResult<>( - outputType, - schema, - config, - gatewayAddress, - gatewayPort, - classLoader, - maxBufferSize); - } - - // -------------------------------------------------------------------------------------------- - - private static int getGatewayPort(DeploymentEntry deploy) { - // try to get address from deployment configuration - return deploy.getGatewayPort(); - } - - private static InetAddress getGatewayAddress(DeploymentEntry deploy, Configuration flinkConfig) throws JobExecutionException { - // try to get address from deployment configuration - final String address = deploy.getGatewayAddress(); - - // use manually defined address - if (!address.isEmpty()) { - try { - return InetAddress.getByName(address); - } catch (UnknownHostException e) { - throw new JobExecutionException("Invalid gateway address '" + address + "' for result retrieval.", e); - } - } else { - // TODO cache this - // try to get the address by communicating to JobManager - final String jobManagerAddress = flinkConfig.getString(JobManagerOptions.ADDRESS); - final int jobManagerPort = flinkConfig.getInteger(JobManagerOptions.PORT); - if (jobManagerAddress != null && !jobManagerAddress.isEmpty()) { - try { - return ConnectionUtils.findConnectingAddress( - new InetSocketAddress(jobManagerAddress, jobManagerPort), - deploy.getResponseTimeout(), - 400); - } catch (Exception e) { - throw new JobExecutionException("Could not determine address of the gateway for result retrieval " + - "by connecting to the job manager. Please specify the gateway address manually.", e); - } - } else { - try { - return InetAddress.getLocalHost(); - } catch (UnknownHostException e) { - throw new JobExecutionException("Could not determine address of the gateway for result retrieval. " + - "Please specify the gateway address manually.", e); - } - } - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/TypedResult.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/TypedResult.java deleted file mode 100644 index 3fec1c6f32..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/result/TypedResult.java +++ /dev/null @@ -1,89 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result; - -import java.util.Objects; - - -public class TypedResult

{ - - private ResultType type; - - private P payload; - - private TypedResult(ResultType type, P payload) { - this.type = type; - this.payload = payload; - } - - public ResultType getType() { - return type; - } - - public P getPayload() { - return payload; - } - - @Override - public String toString() { - return "TypedResult<" + type + ">"; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - TypedResult that = (TypedResult) o; - return type == that.type && Objects.equals(payload, that.payload); - } - - @Override - public int hashCode() { - return Objects.hash(type, payload); - } - - // -------------------------------------------------------------------------------------------- - - public static TypedResult empty() { - return new TypedResult<>(ResultType.EMPTY, null); - } - - public static TypedResult payload(T payload) { - return new TypedResult<>(ResultType.PAYLOAD, payload); - } - - public static TypedResult endOfStream() { - return new TypedResult<>(ResultType.EOS, null); - } - - // -------------------------------------------------------------------------------------------- - - /** - * Result types. - */ - public enum ResultType { - PAYLOAD, - EMPTY, - EOS - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/AbstractJobOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/AbstractJobOperation.java deleted file mode 100644 index 032d354e5b..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/AbstractJobOperation.java +++ /dev/null @@ -1,170 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment.ClusterDescriptorAdapter; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.FlinkListenerGroupImpl; -import java.util.List; -import java.util.Optional; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.types.Row; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A default implementation of JobOperation. - */ -public abstract class AbstractJobOperation extends FlinkListenerGroupImpl implements JobOperation { - - private static final Logger LOG = LoggerFactory.getLogger(AbstractJobOperation.class); - - protected final FlinkEngineConnContext context; - // clusterDescriptorAdapter is not null only after job is submitted - protected ClusterDescriptorAdapter clusterDescriptorAdapter; - private volatile JobID jobId; - protected boolean noMoreResult; - - private volatile boolean isJobCanceled; - - protected final Object lock = new Object(); - - public AbstractJobOperation(FlinkEngineConnContext context) { - this.context = context; - this.isJobCanceled = false; - this.noMoreResult = false; - } - - @Override - public ResultSet execute() throws SqlExecutionException, JobExecutionException { - JobInfo jobInfo = submitJob(); - jobId = jobInfo.getJobId(); - String strJobId = jobId.toString(); - return ResultSet.builder() - .resultKind(ResultKind.SUCCESS_WITH_CONTENT) - .columns( - ColumnInfo.create("jobId", new VarCharType(false, strJobId.length())), - ColumnInfo.create("applicationId", new VarCharType(false, jobInfo.getApplicationId().length())), - ColumnInfo.create("webInterfaceUrl", new VarCharType(false, jobInfo.getWebInterfaceUrl().length())) - ) - .data(Row.of(strJobId, jobInfo.getApplicationId(), jobInfo.getWebInterfaceUrl())) - .build(); - } - - public JobInfo transformToJobInfo(ResultSet resultSet) throws JobExecutionException { - if(resultSet.getColumns().size() != 3) { - throw new JobExecutionException("Not support to transform this resultSet to JobInfo."); - } - Row row = resultSet.getData().get(0); - return new JobInfoImpl(JobID.fromHexString(row.getField(0).toString()), - row.getField(1).toString(), - row.getField(2).toString()); - } - - @Override - public JobStatus getJobStatus() throws JobExecutionException { - synchronized (lock) { - return clusterDescriptorAdapter.getJobStatus(); - } - } - - @Override - public void cancelJob() throws JobExecutionException { - if (isJobCanceled) { - // just for fast failure - return; - } - synchronized (lock) { - if (jobId == null) { - LOG.error("No job has been submitted. This is a bug."); - throw new IllegalStateException("No job has been submitted. This is a bug."); - } - if (isJobCanceled) { - return; - } - - cancelJobInternal(); - isJobCanceled = true; - } - } - - protected abstract JobInfo submitJob() throws JobExecutionException, SqlExecutionException; - - protected abstract void cancelJobInternal() throws JobExecutionException; - - @Override - public JobID getJobId() { - if (jobId == null) { - throw new IllegalStateException("No job has been submitted. This is a bug."); - } - return jobId; - } - - @Override - public synchronized Optional getJobResult() throws JobExecutionException, SqlExecutionException { - Optional, List>> newResults = fetchJobResults(); - return newResults.map(r -> ResultSet.builder() - .resultKind(ResultKind.SUCCESS_WITH_CONTENT) - .columns(getColumnInfos()) - .data(r.f0) - .changeFlags(newResults.get().f1) - .build()); - } - - protected abstract Optional, List>> fetchJobResults() throws JobExecutionException, SqlExecutionException; - - protected abstract List getColumnInfos(); - - - protected class JobInfoImpl implements JobInfo { - - private JobID jobId; - private String applicationId, webInterfaceUrl; - - public JobInfoImpl(JobID jobId, String applicationId, String webInterfaceUrl) { - this.jobId = jobId; - this.applicationId = applicationId; - this.webInterfaceUrl = webInterfaceUrl; - } - - @Override - public JobID getJobId() { - return jobId; - } - - @Override - public String getApplicationId() { - return applicationId; - } - - @Override - public String getWebInterfaceUrl() { - return webInterfaceUrl; - } - } - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobInfo.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobInfo.java deleted file mode 100644 index 0561c0519e..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobInfo.java +++ /dev/null @@ -1,14 +0,0 @@ -package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; - -import org.apache.flink.api.common.JobID; - -/** - * Description: - */ -public interface JobInfo { - JobID getJobId(); - - String getApplicationId(); - - String getWebInterfaceUrl(); -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobOperation.java deleted file mode 100644 index eca9a795f8..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/JobOperation.java +++ /dev/null @@ -1,40 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.FlinkListenerGroup; -import java.util.Optional; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; - - -public interface JobOperation extends Operation, FlinkListenerGroup { - - JobID getJobId(); - - Optional getJobResult() throws JobExecutionException, SqlExecutionException; - - JobStatus getJobStatus() throws JobExecutionException; - - void cancelJob() throws JobExecutionException; - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/NonJobOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/NonJobOperation.java deleted file mode 100644 index 11dc76424a..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/NonJobOperation.java +++ /dev/null @@ -1,23 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; - - -public interface NonJobOperation extends Operation { -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/Operation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/Operation.java deleted file mode 100644 index 0d9a015496..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/Operation.java +++ /dev/null @@ -1,31 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; - -public interface Operation { - - /** - * Execute the command and return the result. - */ - ResultSet execute() throws SqlExecutionException, JobExecutionException; -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationFactory.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationFactory.java deleted file mode 100644 index ca362204a7..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationFactory.java +++ /dev/null @@ -1,129 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; - - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.CreateViewOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.DDLOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.DescribeTableOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.DropViewOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ExplainOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.InsertOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ResetOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.SelectOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.SetOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowCatalogsOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowCurrentCatalogOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowCurrentDatabaseOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowDatabasesOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowFunctionsOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowModulesOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowTablesOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.ShowViewsOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.UseCatalogOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl.UseDatabaseOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils.SqlCommandParser.SqlCommandCall; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlParseException; - -public class OperationFactory { - - public static Operation createOperation(SqlCommandCall call, FlinkEngineConnContext context) throws SqlParseException { - - Operation operation; - switch (call.command) { - case SELECT: - operation = new SelectOperation(context, call.operands[0]); - break; - case CREATE_VIEW: - operation = new CreateViewOperation(context, call.operands[0], call.operands[1]); - break; - case DROP_VIEW: - operation = new DropViewOperation(context, call.operands[0], Boolean.parseBoolean(call.operands[1])); - break; - case CREATE_TABLE: - case DROP_TABLE: - case ALTER_TABLE: - case CREATE_DATABASE: - case DROP_DATABASE: - case ALTER_DATABASE: - operation = new DDLOperation(context, call.operands[0], call.command); - break; - case SET: - // list all properties - if (call.operands.length == 0) { - operation = new SetOperation(context); - } else { - // set a property - operation = new SetOperation(context, call.operands[0], call.operands[1]); - } - break; - case RESET: - if (call.operands.length > 0) { - throw new SqlParseException("Only RESET ALL is supported now"); - } - operation = new ResetOperation(context); - break; - case USE_CATALOG: - operation = new UseCatalogOperation(context, call.operands[0]); - break; - case USE: - operation = new UseDatabaseOperation(context, call.operands[0]); - break; - case INSERT_INTO: - case INSERT_OVERWRITE: - operation = new InsertOperation(context, call.operands[0], call.operands[1]); - break; - case SHOW_MODULES: - operation = new ShowModulesOperation(context); - break; - case SHOW_CATALOGS: - operation = new ShowCatalogsOperation(context); - break; - case SHOW_CURRENT_CATALOG: - operation = new ShowCurrentCatalogOperation(context); - break; - case SHOW_DATABASES: - operation = new ShowDatabasesOperation(context); - break; - case SHOW_CURRENT_DATABASE: - operation = new ShowCurrentDatabaseOperation(context); - break; - case SHOW_TABLES: - operation = new ShowTablesOperation(context); - break; - case SHOW_VIEWS: - operation = new ShowViewsOperation(context); - break; - case SHOW_FUNCTIONS: - operation = new ShowFunctionsOperation(context); - break; - case DESCRIBE_TABLE: - operation = new DescribeTableOperation(context, call.operands[0]); - break; - case EXPLAIN: - operation = new ExplainOperation(context, call.operands[0]); - break; - default: - throw new SqlParseException("Unsupported command call " + call + ". This is a bug."); - } - - return operation; - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationUtil.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationUtil.java deleted file mode 100644 index fd8e428f0f..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/OperationUtil.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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import java.util.ArrayList; -import java.util.List; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.types.Row; - - -public class OperationUtil { - - public static final ResultSet OK = ResultSet.builder() - .resultKind(ResultKind.SUCCESS) - .columns(ColumnInfo.create(ConstantNames.RESULT, new VarCharType(2))) - .data(Row.of(ConstantNames.OK)) - .build(); - - public static ResultSet singleStringToResultSet(String str, String columnName) { - boolean isNullable; - int length; - - if (str == null) { - isNullable = true; - length = VarCharType.DEFAULT_LENGTH; - } else { - isNullable = false; - length = str.length(); - } - - return ResultSet.builder() - .resultKind(ResultKind.SUCCESS_WITH_CONTENT) - .columns(ColumnInfo.create(columnName, new VarCharType(isNullable, length))) - .data(Row.of(str)) - .build(); - } - - public static ResultSet stringListToResultSet(List strings, String columnName) { - List data = new ArrayList<>(); - boolean isNullable = false; - int maxLength = VarCharType.DEFAULT_LENGTH; - - for (String str : strings) { - if (str == null) { - isNullable = true; - } else { - maxLength = Math.max(str.length(), maxLength); - data.add(Row.of(str)); - } - } - - return ResultSet.builder() - .resultKind(ResultKind.SUCCESS_WITH_CONTENT) - .columns(ColumnInfo.create(columnName, new VarCharType(isNullable, maxLength))) - .data(data) - .build(); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/CreateViewOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/CreateViewOperation.java deleted file mode 100644 index b3686fd388..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/CreateViewOperation.java +++ /dev/null @@ -1,69 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.client.config.entries.TableEntry; -import org.apache.flink.table.client.config.entries.ViewEntry; - -/** - * Operation for CREATE VIEW command. - */ -public class CreateViewOperation implements NonJobOperation { - private final ExecutionContext context; - private final String viewName; - private final String query; - - public CreateViewOperation(FlinkEngineConnContext context, String viewName, String query) { - this.context = context.getExecutionContext(); - this.viewName = viewName; - this.query = query; - } - - @Override - public ResultSet execute() throws SqlExecutionException { - Environment env = context.getEnvironment(); - TableEntry tableEntry = env.getTables().get(viewName); - if (tableEntry instanceof ViewEntry) { - throw new SqlExecutionException("'" + viewName + "' has already been defined in the current session."); - } - - // TODO check the logic - TableEnvironment tableEnv = context.getTableEnvironment(); - try { - context.wrapClassLoader(() -> { - tableEnv.createTemporaryView(viewName, tableEnv.sqlQuery(query)); - return null; - }); - } catch (Throwable t) { - // catch everything such that the query does not crash the executor - throw new SqlExecutionException("Invalid SQL statement.", t); - } - // Also attach the view to ExecutionContext#environment. - env.getTables().put(viewName, ViewEntry.create(viewName, query)); - return OperationUtil.OK; - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DDLOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DDLOperation.java deleted file mode 100644 index 4449a1e96b..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DDLOperation.java +++ /dev/null @@ -1,92 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils.SqlCommandParser.SqlCommand; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import org.apache.flink.table.api.TableEnvironment; - -/** - * Operation for CREATE/DROP/ALTER TABLE/DATABASE command. - */ -public class DDLOperation implements NonJobOperation { - private final ExecutionContext context; - private final String ddl; - private final SqlCommand command; - - public DDLOperation(FlinkEngineConnContext context, String ddl, SqlCommand command) { - this.context = context.getExecutionContext(); - this.ddl = ddl; - this.command = command; - } - - @Override - public ResultSet execute() throws SqlExecutionException { - final TableEnvironment tEnv = context.getTableEnvironment(); - // parse and validate statement - try { - context.wrapClassLoader(() -> { - tEnv.sqlUpdate(ddl); - return null; - }); - } catch (Exception t) { - // catch everything such that the statement does not crash the executor - throw new SqlExecutionException(getExceptionMsg(), t); - } - - return OperationUtil.OK; - } - - private String getExceptionMsg() { - final String actionMsg; - switch (command) { - case CREATE_TABLE: - actionMsg = "create a table"; - break; - case CREATE_DATABASE: - actionMsg = "create a database"; - break; - case DROP_TABLE: - actionMsg = "drop a table"; - break; - case DROP_DATABASE: - actionMsg = "drop a database"; - break; - case ALTER_TABLE: - actionMsg = "alter a table"; - break; - case ALTER_DATABASE: - actionMsg = "alter a database"; - break; - default: - actionMsg = null; - } - - if (actionMsg != null) { - return String.format("Could not %s from statement: %s.", actionMsg, ddl); - } else { - return "Invalid DDL statement."; - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DescribeTableOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DescribeTableOperation.java deleted file mode 100644 index 3ece9261d0..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DescribeTableOperation.java +++ /dev/null @@ -1,109 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.WatermarkSpec; -import org.apache.flink.table.types.logical.BooleanType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.types.Row; - -/** - * Operation for DESCRIBE TABLE command. - */ -public class DescribeTableOperation implements NonJobOperation { - private final ExecutionContext context; - private final String tableName; - - public DescribeTableOperation(FlinkEngineConnContext context, String tableName) { - this.context = context.getExecutionContext(); - this.tableName = tableName; - } - - @Override - @SuppressWarnings("unchecked") - public ResultSet execute() throws SqlExecutionException { - // the implementation should be in sync with Flink, see FLINK-17112 - final TableEnvironment tableEnv = context.getTableEnvironment(); - TableSchema schema; - try { - schema = context.wrapClassLoader(() -> tableEnv.from(tableName).getSchema()); - } catch (Throwable t) { - // catch everything such that the query does not crash the executor - throw new SqlExecutionException("No table with this name could be found.", t); - } - - Map fieldToWatermark = new HashMap<>(); - for (WatermarkSpec spec : schema.getWatermarkSpecs()) { - fieldToWatermark.put(spec.getRowtimeAttribute(), spec.getWatermarkExpr()); - } - - Map fieldToPrimaryKey = new HashMap<>(); - if (schema.getPrimaryKey().isPresent()) { - List columns = schema.getPrimaryKey().get().getColumns(); - String primaryKey = "PRI(" + String.join(", ", columns) + ")"; - for (String column : columns) { - fieldToPrimaryKey.put(column, primaryKey); - } - } - - List columns = schema.getTableColumns(); - List data = new ArrayList<>(); - for (TableColumn column : columns) { - LogicalType logicalType = column.getType().getLogicalType(); - - String name = column.getName(); - String type = StringUtils.removeEnd(logicalType.toString(), " NOT NULL"); - boolean isNullable = logicalType.isNullable(); - String key = fieldToPrimaryKey.getOrDefault(column.getName(), null); - String computedColumn = column.getExpr().orElse(null); - String watermark = fieldToWatermark.getOrDefault(column.getName(), null); - - data.add(Row.of(name, type, isNullable, key, computedColumn, watermark)); - } - - return ResultSet.builder() - .resultKind(ResultKind.SUCCESS_WITH_CONTENT) - .columns( - ColumnInfo.create(ConstantNames.DESCRIBE_NAME, DataTypes.STRING().getLogicalType()), - ColumnInfo.create(ConstantNames.DESCRIBE_TYPE, DataTypes.STRING().getLogicalType()), - ColumnInfo.create(ConstantNames.DESCRIBE_NULL, new BooleanType()), - ColumnInfo.create(ConstantNames.DESCRIBE_KEY, DataTypes.STRING().getLogicalType()), - ColumnInfo.create(ConstantNames.DESCRIBE_COMPUTED_COLUMN, DataTypes.STRING().getLogicalType()), - ColumnInfo.create(ConstantNames.DESCRIBE_WATERMARK, DataTypes.STRING().getLogicalType())) - .data(data) - .build(); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DropViewOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DropViewOperation.java deleted file mode 100644 index a8ed9590f0..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/DropViewOperation.java +++ /dev/null @@ -1,79 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import org.apache.flink.table.client.config.entries.TableEntry; -import org.apache.flink.table.client.config.entries.ViewEntry; - -/** - * Operation for DROP VIEW command. - */ -public class DropViewOperation implements NonJobOperation { - private final FlinkEngineConnContext context; - private final String viewName; - private final boolean ifExists; - - public DropViewOperation(FlinkEngineConnContext context, String viewName, boolean ifExists) { - this.context = context; - this.viewName = viewName; - this.ifExists = ifExists; - } - - @Override - public ResultSet execute() throws SqlExecutionException { - Environment env = context.getExecutionContext().getEnvironment(); - TableEntry tableEntry = env.getTables().get(viewName); - if (!(tableEntry instanceof ViewEntry)) { - if (!ifExists) { - throw new SqlExecutionException("'" + viewName + "' does not exist in the current session."); - } - } - - // Here we rebuild the ExecutionContext because we want to ensure that all the remaining views can work fine. - // Assume the case: - // view1=select 1; - // view2=select * from view1; - // If we delete view1 successfully, then query view2 will throw exception because view1 does not exist. we want - // all the remaining views are OK, so do the ExecutionContext rebuilding to avoid breaking the view dependency. - Environment newEnv = env.clone(); - if (newEnv.getTables().remove(viewName) != null) { - ExecutionContext oldExecutionContext = context.getExecutionContext(); - oldExecutionContext.wrapClassLoader(() -> { - oldExecutionContext.getTableEnvironment().dropTemporaryView(viewName); - return null; - }); - // Renew the ExecutionContext. - ExecutionContext newExecutionContext = context - .newExecutionContextBuilder(context.getEnvironmentContext().getDefaultEnv()) - .env(newEnv) - .sessionState(context.getExecutionContext().getSessionState()) - .build(); - context.setExecutionContext(newExecutionContext); - } - - return OperationUtil.OK; - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ExplainOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ExplainOperation.java deleted file mode 100644 index 644b2ac726..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ExplainOperation.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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.types.Row; - -/** - * Operation for EXPLAIN command. - */ -public class ExplainOperation implements NonJobOperation { - private final ExecutionContext context; - private final String statement; - - public ExplainOperation(FlinkEngineConnContext context, String statement) { - this.context = context.getExecutionContext(); - this.statement = statement; - } - - @Override - public ResultSet execute() throws SqlExecutionException { - final TableEnvironment tableEnv = context.getTableEnvironment(); - // translate - try { - final Table table = createTable(context, tableEnv, statement); - String explanation = context.wrapClassLoader(() -> tableEnv.explain(table)); - return ResultSet.builder() - .resultKind(ResultKind.SUCCESS_WITH_CONTENT) - .columns(ColumnInfo.create(ConstantNames.EXPLAIN_RESULT, new VarCharType(false, explanation.length()))) - .data(Row.of(explanation)) - .build(); - } catch (SqlExecutionException t) { - throw t; - } catch (Exception t) { - // catch everything such that the query does not crash the executor - throw new SqlExecutionException("Invalid SQL statement.", t); - } - } - - /** - * Creates a table using the given query in the given table environment. - */ - private Table createTable(ExecutionContext context, TableEnvironment tableEnv, String selectQuery) throws SqlExecutionException { - // parse and validate query - try { - return context.wrapClassLoader(() -> tableEnv.sqlQuery(selectQuery)); - } catch (Exception t) { - // catch everything such that the query does not crash the executor - throw new SqlExecutionException("Invalid SQL statement.", t); - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/InsertOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/InsertOperation.java deleted file mode 100644 index 68b593b983..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/InsertOperation.java +++ /dev/null @@ -1,140 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment.ClusterDescriptorAdapterFactory; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.AbstractJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.JobInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.RowsType; -import java.sql.Statement; -import java.util.Collections; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableResult; -import org.apache.flink.table.types.logical.BigIntType; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Operation for INSERT command. - */ -public class InsertOperation extends AbstractJobOperation { - private static final Logger LOG = LoggerFactory.getLogger(InsertOperation.class); - - private final String statement; - private final List columnInfos; - - private boolean fetched = false; - - public InsertOperation(FlinkEngineConnContext context, String statement, String tableIdentifier) { - super(context); - this.statement = statement; - this.columnInfos = Collections.singletonList( - ColumnInfo.create(tableIdentifier, new BigIntType(false))); - } - - @Override - protected JobInfo submitJob() throws SqlExecutionException, JobExecutionException { - JobID jobId = executeUpdateInternal(context.getExecutionContext()); - String applicationId = this.clusterDescriptorAdapter.getClusterID().toString(); - String webInterfaceUrl = this.clusterDescriptorAdapter.getWebInterfaceUrl(); - return new JobInfoImpl(jobId, applicationId, webInterfaceUrl); - } - - @Override - protected Optional, List>> fetchJobResults() throws JobExecutionException { - if (fetched) { - return Optional.empty(); - } else { - // for session mode, we can get job status from JM, because JM is a long life service. - // while for per-job mode, JM will be also destroy after the job is finished. - // so it's hard to say whether the job is finished/canceled - // or the job status is just inaccessible at that moment. - // currently only yarn-per-job is supported, - // and if the exception (thrown when getting job status) contains ApplicationNotFoundException, - // we can say the job is finished. - boolean isGloballyTerminalState = clusterDescriptorAdapter.isGloballyTerminalState(); - if (isGloballyTerminalState) { - // TODO get affected_row_count for batch job - fetched = true; - return Optional.of(Tuple2.of(Collections.singletonList( - Row.of((long) Statement.SUCCESS_NO_INFO)), null)); - } else { - // TODO throws exception if the job fails - return Optional.of(Tuple2.of(Collections.emptyList(), null)); - } - } - } - - @Override - protected List getColumnInfos() { - return columnInfos; - } - - @Override - protected void cancelJobInternal() throws JobExecutionException { - clusterDescriptorAdapter.cancelJob(); - } - - private JobID executeUpdateInternal(ExecutionContext executionContext) throws SqlExecutionException, JobExecutionException { - TableEnvironment tableEnv = executionContext.getTableEnvironment(); - // parse and validate statement - TableResult tableResult; - try { - tableResult = executionContext.wrapClassLoader(() -> tableEnv.executeSql(statement)); - } catch (Exception t) { - LOG.error(String.format("Invalid SQL query, sql is: %s.", statement), t); - // catch everything such that the statement does not crash the executor - throw new SqlExecutionException("Invalid SQL statement.", t); - } - tableResult.collect(); - asyncNotify(tableResult); - JobID jobId = tableResult.getJobClient().get().getJobID(); - this.clusterDescriptorAdapter = - ClusterDescriptorAdapterFactory.create(context.getExecutionContext(), jobId); - clusterDescriptorAdapter.deployCluster(null, null); - return jobId; - } - - protected void asyncNotify(TableResult tableResult) { - CompletableFuture.completedFuture(tableResult) - .thenAccept(result -> { - CloseableIterator iterator = result.collect(); - int affected = 0; - while(iterator.hasNext()) { - Row row = iterator.next(); - affected = (int) row.getField(0); - } - int finalAffected = affected; - getFlinkStatusListeners().forEach(listener -> listener.onSuccess(finalAffected, RowsType.Affected())); - }).whenComplete((unused, throwable) -> getFlinkStatusListeners().forEach(listener -> listener.onFailed("Error while submitting job.", throwable))); - } - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ResetOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ResetOperation.java deleted file mode 100644 index c34e3b36d0..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ResetOperation.java +++ /dev/null @@ -1,52 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; - -/** - * Operation for RESET command. - */ -public class ResetOperation implements NonJobOperation { - private final FlinkEngineConnContext context; - - public ResetOperation(FlinkEngineConnContext context) { - this.context = context; - } - - @Override - public ResultSet execute() throws SqlExecutionException { - ExecutionContext executionContext = context.getExecutionContext(); - // Renew the ExecutionContext by merging the default environment with original session context. - // Book keep all the session states of current ExecutionContext then - // re-register them into the new one. - ExecutionContext newExecutionContext = context - .newExecutionContextBuilder(context.getEnvironmentContext().getDefaultEnv()) - .sessionState(executionContext.getSessionState()) - .build(); - context.setExecutionContext(newExecutionContext); - - return OperationUtil.OK; - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SelectOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SelectOperation.java deleted file mode 100644 index 845f8dde34..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SelectOperation.java +++ /dev/null @@ -1,260 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment.ClusterDescriptorAdapterFactory; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment.ProgramDeployer; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result.AbstractResult; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result.BatchResult; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result.ChangelogResult; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result.ResultUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.result.TypedResult; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.AbstractJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.JobInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Optional; -import java.util.UUID; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.dag.Pipeline; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.DeploymentOptions; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.utils.LogicalTypeUtils; -import org.apache.flink.table.types.utils.DataTypeUtils; -import org.apache.flink.types.Row; -import org.mortbay.util.ajax.JSON; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Operation for SELECT command. - */ -public class SelectOperation extends AbstractJobOperation { - - private static final Logger LOG = LoggerFactory.getLogger(SelectOperation.class); - - private final String query; - - private AbstractResult result; - - private TableSchema resultSchema; - - private List columnInfos; - - private volatile boolean noMoreResult; - - public SelectOperation(FlinkEngineConnContext context, String query) { - super(context); - this.query = query; - this.noMoreResult = false; - } - - @Override - protected JobInfo submitJob() throws JobExecutionException, SqlExecutionException { - JobID jobId = executeQueryInternal(context.getExecutionContext(), query); - List resultSchemaColumns = resultSchema.getTableColumns(); - columnInfos = new ArrayList<>(); - for (TableColumn column : resultSchemaColumns) { - columnInfos.add(ColumnInfo.create(column.getName(), column.getType().getLogicalType())); - } - String applicationId = this.clusterDescriptorAdapter.getClusterID().toString(); - String webInterfaceUrl = this.clusterDescriptorAdapter.getWebInterfaceUrl(); - return new JobInfoImpl(jobId, applicationId, webInterfaceUrl); - } - - @Override - protected void cancelJobInternal() throws JobExecutionException { - LOG.info("Start to cancel job {} and result retrieval.", getJobId()); - result.close(); - // ignore if there is no more result - // the job might has finished earlier. it's hard to say whether it need to be canceled, - // so the clients should be care for the exceptions ??? - if (noMoreResult) { - return; - } - - clusterDescriptorAdapter.cancelJob(); - } - - @Override - protected Optional, List>> fetchJobResults() throws SqlExecutionException, JobExecutionException { - Optional, List>> ret; - synchronized (lock) { - if (result == null) { - LOG.error("The job for this query has been canceled."); - throw new SqlExecutionException("The job for this query has been canceled."); - } - - if (this.result instanceof ChangelogResult) { - ret = fetchStreamingResult(); - } else { - ret = fetchBatchResult(); - } - } - return ret; - } - - @Override - protected List getColumnInfos() { - return columnInfos; - } - - private Optional, List>> fetchBatchResult() throws SqlExecutionException { - BatchResult batchResult = (BatchResult) this.result; - TypedResult> typedResult = batchResult.retrieveChanges(); - if (typedResult.getType() == TypedResult.ResultType.PAYLOAD) { - List payload = typedResult.getPayload(); - return Optional.of(Tuple2.of(payload, null)); - } else { - return Optional.of(Tuple2.of(Collections.emptyList(), null)); - } - } - - private Optional, List>> fetchStreamingResult() throws JobExecutionException { - ChangelogResult changLogResult = (ChangelogResult) this.result; - TypedResult>> typedResult = changLogResult.retrieveChanges(); - if (typedResult.getType() == TypedResult.ResultType.EOS) { - return Optional.of(Tuple2.of(Collections.emptyList(), Collections.emptyList())); - } else if (typedResult.getType() == TypedResult.ResultType.PAYLOAD) { - List> payload = typedResult.getPayload(); - List data = new ArrayList<>(); - List changeFlags = new ArrayList<>(); - for (Tuple2 tuple : payload) { - data.add(tuple.f1); - changeFlags.add(tuple.f0); - } - return Optional.of(Tuple2.of(data, changeFlags)); - } else { - return Optional.of(Tuple2.of(Collections.emptyList(), Collections.emptyList())); - } - } - - private JobID executeQueryInternal(ExecutionContext executionContext, String query) - throws JobExecutionException, SqlExecutionException { - // create table - final Table table = createTable(executionContext, executionContext.getTableEnvironment(), query); - boolean isChangelogResult = executionContext.getEnvironment().getExecution().inStreamingMode(); - // initialize result - resultSchema = removeTimeAttributes(table.getSchema()); - if (isChangelogResult) { - result = ResultUtil.createChangelogResult( - executionContext.getFlinkConfig(), - executionContext.getEnvironment(), - resultSchema, - executionContext.getExecutionConfig(), - executionContext.getClassLoader()); - } else { - result = ResultUtil.createBatchResult( - resultSchema, - executionContext.getExecutionConfig(), - executionContext.getClassLoader()); - } - result.setFlinkListeners(getFlinkListeners()); - final String tableName = String.format("_tmp_table_%s", UUID.randomUUID().toString().replace("-", "")); - final Pipeline pipeline; - try { - // writing to a sink requires an optimization step that might reference UDFs during code compilation - executionContext.wrapClassLoader(() -> { - executionContext.getTableEnvironment().registerTableSink(tableName, result.getTableSink()); - table.insertInto(tableName); - return null; - }); - pipeline = executionContext.createPipeline(query); - } catch (Exception t) { - // the result needs to be closed as long as - // it not stored in the result store - result.close(); - LOG.error(String.format("Invalid SQL query, sql is %s.", query), t); - // catch everything such that the query does not crash the executor - throw new SqlExecutionException("Invalid SQL query.", t); - } finally { - // Remove the temporal table object. - executionContext.wrapClassLoader(() -> { - executionContext.getTableEnvironment().dropTemporaryTable(tableName); - return null; - }); - } - - context.getExecutionContext().getFlinkConfig().setBoolean(DeploymentOptions.ATTACHED, true); - context.getExecutionContext().getFlinkConfig().setBoolean(DeploymentOptions.SHUTDOWN_IF_ATTACHED, true); - LOG.info("Deployer flink config {}.", JSON.toString(context.getExecutionContext().getFlinkConfig().toString())); - final ProgramDeployer deployer = new ProgramDeployer(context.getExecutionContext().getFlinkConfig(), query, pipeline); - // blocking deployment - JobClient jobClient; - try { - jobClient = deployer.deploy(context.getExecutionContext()).get(); - } catch (Exception e) { - LOG.error(String.format("Error running SQL job, sql is %s.", query), e); - throw new SqlExecutionException("Error running SQL job.", e); - } - - JobID jobId = jobClient.getJobID(); - this.clusterDescriptorAdapter = - ClusterDescriptorAdapterFactory.create(context.getExecutionContext(), jobId); - this.clusterDescriptorAdapter.deployCluster(null, null); - if (LOG.isDebugEnabled()) { - LOG.debug("Cluster Descriptor Adapter: {}", clusterDescriptorAdapter); - } - clusterDescriptorAdapter.deployCluster(null, null); - LOG.info("Submit flink job: {} successfully, query: {}.", jobId, query); - // start result retrieval - result.startRetrieval(jobClient); - return jobId; - } - - - /** - * Creates a table using the given query in the given table environment. - */ - private Table createTable(ExecutionContext context, TableEnvironment tableEnv, String selectQuery) throws SqlExecutionException { - // parse and validate query - try { - return context.wrapClassLoader(() -> tableEnv.sqlQuery(selectQuery)); - } catch (Exception t) { - // catch everything such that the query does not crash the executor - throw new SqlExecutionException("Invalid SQL statement.", t); - } - } - - private TableSchema removeTimeAttributes(TableSchema schema) { - final TableSchema.Builder builder = TableSchema.builder(); - for (int i = 0; i < schema.getFieldCount(); i++) { - final DataType dataType = schema.getFieldDataTypes()[i]; - final DataType convertedType = DataTypeUtils.replaceLogicalType( - dataType, - LogicalTypeUtils.removeTimeAttributes(dataType.getLogicalType())); - builder.field(schema.getFieldNames()[i], convertedType); - } - return builder.build(); - } - - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SetOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SetOperation.java deleted file mode 100644 index b09f7b290e..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/SetOperation.java +++ /dev/null @@ -1,111 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.shaded.guava18.com.google.common.collect.ImmutableMap; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.types.Row; - -/** - * Operation for SET command. - */ -public class SetOperation implements NonJobOperation { - private final FlinkEngineConnContext context; - private final String key; - private final String value; - - public SetOperation(FlinkEngineConnContext context, String key, String value) { - this.context = context; - this.key = key; - this.value = value; - } - - public SetOperation(FlinkEngineConnContext context) { - this(context, null, null); - } - - @Override - public ResultSet execute() throws SqlExecutionException { - ExecutionContext executionContext = context.getExecutionContext(); - Environment env = executionContext.getEnvironment(); - - // list all properties - if (key == null) { - List data = new ArrayList<>(); - Tuple2 maxKeyLenAndMaxValueLen = new Tuple2<>(1, 1); - buildResult(env.getExecution().asTopLevelMap(), data, maxKeyLenAndMaxValueLen); - buildResult(env.getDeployment().asTopLevelMap(), data, maxKeyLenAndMaxValueLen); - buildResult(env.getConfiguration().asMap(), data, maxKeyLenAndMaxValueLen); - - return ResultSet.builder() - .resultKind(ResultKind.SUCCESS_WITH_CONTENT) - .columns( - ColumnInfo.create(ConstantNames.SET_KEY, new VarCharType(true, maxKeyLenAndMaxValueLen.f0)), - ColumnInfo.create(ConstantNames.SET_VALUE, new VarCharType(true, maxKeyLenAndMaxValueLen.f1))) - .data(data) - .build(); - } else { - // TODO avoid to build a new Environment for some cases - // set a property - Environment newEnv = Environment.enrich(env, ImmutableMap.of(key.trim(), value.trim()), ImmutableMap.of()); - ExecutionContext.SessionState sessionState = executionContext.getSessionState(); - - // Renew the ExecutionContext by new environment. - // Book keep all the session states of current ExecutionContext then - // re-register them into the new one. - ExecutionContext newExecutionContext = context - .newExecutionContextBuilder(context.getEnvironmentContext().getDefaultEnv()) - .env(newEnv) - .sessionState(sessionState) - .build(); - context.setExecutionContext(newExecutionContext); - - return OperationUtil.OK; - } - } - - private void buildResult( - Map properties, - List data, - Tuple2 maxKeyLenAndMaxValueLen) { - for (Map.Entry entry : properties.entrySet()) { - String key = entry.getKey(); - String value = entry.getValue(); - data.add(Row.of(key, value)); - // update max key length - maxKeyLenAndMaxValueLen.f0 = Math.max(maxKeyLenAndMaxValueLen.f0, key.length()); - // update max value length - maxKeyLenAndMaxValueLen.f1 = Math.max(maxKeyLenAndMaxValueLen.f1, value.length()); - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCatalogsOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCatalogsOperation.java deleted file mode 100644 index b898b97838..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCatalogsOperation.java +++ /dev/null @@ -1,47 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import java.util.Arrays; -import java.util.List; -import org.apache.flink.table.api.TableEnvironment; - -/** - * Operation for SHOW CATALOGS command. - */ -public class ShowCatalogsOperation implements NonJobOperation { - private final ExecutionContext context; - - public ShowCatalogsOperation(FlinkEngineConnContext context) { - this.context = context.getExecutionContext(); - } - - @Override - public ResultSet execute() { - final TableEnvironment tableEnv = context.getTableEnvironment(); - final List catalogs = context.wrapClassLoader(() -> Arrays.asList(tableEnv.listCatalogs())); - return OperationUtil.stringListToResultSet(catalogs, ConstantNames.SHOW_CATALOGS_RESULT); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentCatalogOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentCatalogOperation.java deleted file mode 100644 index 9da10a51e7..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentCatalogOperation.java +++ /dev/null @@ -1,45 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import org.apache.flink.table.api.TableEnvironment; - -/** - * Operation for SHOW CURRENT CATALOG command. - */ -public class ShowCurrentCatalogOperation implements NonJobOperation { - private final ExecutionContext context; - - public ShowCurrentCatalogOperation(FlinkEngineConnContext context) { - this.context = context.getExecutionContext(); - } - - @Override - public ResultSet execute() { - final TableEnvironment tableEnv = context.getTableEnvironment(); - return OperationUtil.singleStringToResultSet( - context.wrapClassLoader(tableEnv::getCurrentCatalog), ConstantNames.SHOW_CURRENT_CATALOG_RESULT); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentDatabaseOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentDatabaseOperation.java deleted file mode 100644 index 3d36041fa7..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowCurrentDatabaseOperation.java +++ /dev/null @@ -1,45 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import org.apache.flink.table.api.TableEnvironment; - -/** - * Operation for SHOW CURRENT DATABASE command. - */ -public class ShowCurrentDatabaseOperation implements NonJobOperation { - private final ExecutionContext context; - - public ShowCurrentDatabaseOperation(FlinkEngineConnContext context) { - this.context = context.getExecutionContext(); - } - - @Override - public ResultSet execute() { - final TableEnvironment tableEnv = context.getTableEnvironment(); - return OperationUtil.singleStringToResultSet( - context.wrapClassLoader(tableEnv::getCurrentDatabase), ConstantNames.SHOW_CURRENT_DATABASE_RESULT); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowDatabasesOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowDatabasesOperation.java deleted file mode 100644 index 257fbdcbe2..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowDatabasesOperation.java +++ /dev/null @@ -1,47 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import java.util.Arrays; -import java.util.List; -import org.apache.flink.table.api.TableEnvironment; - -/** - * Operation for SHOW DATABASES command. - */ -public class ShowDatabasesOperation implements NonJobOperation { - private final ExecutionContext context; - - public ShowDatabasesOperation(FlinkEngineConnContext context) { - this.context = context.getExecutionContext(); - } - - @Override - public ResultSet execute() { - final TableEnvironment tableEnv = context.getTableEnvironment(); - final List databases = context.wrapClassLoader(() -> Arrays.asList(tableEnv.listDatabases())); - return OperationUtil.stringListToResultSet(databases, ConstantNames.SHOW_DATABASES_RESULT); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowFunctionsOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowFunctionsOperation.java deleted file mode 100644 index 64378702d9..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowFunctionsOperation.java +++ /dev/null @@ -1,47 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import java.util.Arrays; -import java.util.List; -import org.apache.flink.table.api.TableEnvironment; - -/** - * Operation for SHOW FUNCTIONS command. - */ -public class ShowFunctionsOperation implements NonJobOperation { - private final ExecutionContext context; - - public ShowFunctionsOperation(FlinkEngineConnContext context) { - this.context = context.getExecutionContext(); - } - - @Override - public ResultSet execute() { - final TableEnvironment tableEnv = context.getTableEnvironment(); - final List functions = context.wrapClassLoader(() -> Arrays.asList(tableEnv.listFunctions())); - return OperationUtil.stringListToResultSet(functions, ConstantNames.SHOW_FUNCTIONS_RESULT); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowModulesOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowModulesOperation.java deleted file mode 100644 index c305e36b72..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowModulesOperation.java +++ /dev/null @@ -1,47 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import java.util.Arrays; -import java.util.List; -import org.apache.flink.table.api.TableEnvironment; - -/** - * Operation for SHOW MODULES command. - */ -public class ShowModulesOperation implements NonJobOperation { - private final ExecutionContext context; - - public ShowModulesOperation(FlinkEngineConnContext context) { - this.context = context.getExecutionContext(); - } - - @Override - public ResultSet execute() { - final TableEnvironment tableEnv = context.getTableEnvironment(); - final List modules = context.wrapClassLoader(() -> Arrays.asList(tableEnv.listModules())); - return OperationUtil.stringListToResultSet(modules, ConstantNames.SHOW_MODULES_RESULT); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowTablesOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowTablesOperation.java deleted file mode 100644 index 9e85e7708d..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowTablesOperation.java +++ /dev/null @@ -1,63 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.types.Row; - -/** - * Operation for SHOW TABLES command. - */ -public class ShowTablesOperation implements NonJobOperation { - private final ExecutionContext context; - - public ShowTablesOperation(FlinkEngineConnContext context) { - this.context = context.getExecutionContext(); - } - - @Override - public ResultSet execute() { - List rows = new ArrayList<>(); - int maxNameLength = 1; - - final TableEnvironment tableEnv = context.getTableEnvironment(); - // listTables will return all tables and views - for (String table : context.wrapClassLoader(() -> Arrays.asList(tableEnv.listTables()))) { - rows.add(Row.of(table)); - maxNameLength = Math.max(maxNameLength, table.length()); - } - - return ResultSet.builder() - .resultKind(ResultKind.SUCCESS_WITH_CONTENT) - .columns(ColumnInfo.create(ConstantNames.SHOW_TABLES_RESULT, new VarCharType(false, maxNameLength))) - .data(rows) - .build(); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowViewsOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowViewsOperation.java deleted file mode 100644 index fae722a5d5..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/ShowViewsOperation.java +++ /dev/null @@ -1,66 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ColumnInfo; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ConstantNames; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.client.config.entries.TableEntry; -import org.apache.flink.table.client.config.entries.ViewEntry; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.types.Row; - -/** - * Operation for SHOW VIEWS command. - */ -public class ShowViewsOperation implements NonJobOperation { - - private final ExecutionContext context; - - public ShowViewsOperation(FlinkEngineConnContext context) { - this.context = context.getExecutionContext(); - } - - @Override - public ResultSet execute() { - List rows = new ArrayList<>(); - int maxNameLength = 1; - - for (Map.Entry entry : context.getEnvironment().getTables().entrySet()) { - if (entry.getValue() instanceof ViewEntry) { - String name = entry.getKey(); - rows.add(Row.of(name)); - maxNameLength = Math.max(maxNameLength, name.length()); - } - } - - return ResultSet.builder() - .resultKind(ResultKind.SUCCESS_WITH_CONTENT) - .columns(ColumnInfo.create(ConstantNames.SHOW_VIEWS_RESULT, new VarCharType(false, maxNameLength))) - .data(rows) - .build(); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseCatalogOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseCatalogOperation.java deleted file mode 100644 index 2c31182212..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseCatalogOperation.java +++ /dev/null @@ -1,58 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.catalog.exceptions.CatalogException; - -/** - * Operation for USE CATALOG command. - */ -public class UseCatalogOperation implements NonJobOperation { - private final ExecutionContext context; - private final String catalogName; - - public UseCatalogOperation(FlinkEngineConnContext context, String catalogName) { - this.context = context.getExecutionContext(); - this.catalogName = catalogName; - } - - @Override - public ResultSet execute() throws SqlExecutionException { - final TableEnvironment tableEnv = context.getTableEnvironment(); - - try { - context.wrapClassLoader(() -> { - // Rely on TableEnvironment/CatalogManager to validate input - tableEnv.useCatalog(catalogName); - return null; - }); - } catch (CatalogException e) { - throw new SqlExecutionException("Failed to switch to catalog " + catalogName, e); - } - - return OperationUtil.OK; - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseDatabaseOperation.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseDatabaseOperation.java deleted file mode 100644 index a450e29a2d..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/impl/UseDatabaseOperation.java +++ /dev/null @@ -1,56 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.impl; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.NonJobOperation; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.OperationUtil; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext; -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlExecutionException; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.catalog.exceptions.CatalogException; - -/** - * Operation for USE DATABASE command. - */ -public class UseDatabaseOperation implements NonJobOperation { - private final ExecutionContext context; - private final String databaseName; - - public UseDatabaseOperation(FlinkEngineConnContext context, String databaseName) { - this.context = context.getExecutionContext(); - this.databaseName = databaseName; - } - - @Override - public ResultSet execute() throws SqlExecutionException { - final TableEnvironment tableEnv = context.getTableEnvironment(); - try { - context.wrapClassLoader(() -> { - // Rely on TableEnvironment/CatalogManager to validate input - tableEnv.useDatabase(databaseName); - return null; - }); - } catch (CatalogException e) { - throw new SqlExecutionException("Failed to switch to database " + databaseName, e); - } - return OperationUtil.OK; - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ColumnInfo.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ColumnInfo.java deleted file mode 100644 index 8540154f75..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ColumnInfo.java +++ /dev/null @@ -1,101 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; - -import java.util.Objects; -import javax.annotation.Nullable; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.utils.LogicalTypeParser; -import org.apache.flink.util.Preconditions; - -/** - * A column info represents a table column's structure with column name, column type. - */ -public class ColumnInfo { - - private static final String FIELD_NAME_NAME = "name"; - private static final String FIELD_NAME_TYPE = "type"; - - @JsonProperty(FIELD_NAME_NAME) - private String name; - - @JsonProperty(FIELD_NAME_TYPE) - private String type; - - @JsonIgnore - @Nullable - private LogicalType logicalType; - - @JsonCreator - public ColumnInfo( - @JsonProperty(FIELD_NAME_NAME) String name, - @JsonProperty(FIELD_NAME_TYPE) String type) { - this.name = Preconditions.checkNotNull(name, "name must not be null"); - this.type = Preconditions.checkNotNull(type, "type must not be null"); - } - - public static ColumnInfo create(String name, LogicalType type) { - return new ColumnInfo(name, type.toString()); - } - - public String getName() { - return name; - } - - public String getType() { - return type; - } - - @JsonIgnore - public LogicalType getLogicalType() { - if (logicalType == null) { - logicalType = LogicalTypeParser.parse(type); - } - return logicalType; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ColumnInfo that = (ColumnInfo) o; - return name.equals(that.name) && - type.equals(that.type); - } - - @Override - public int hashCode() { - return Objects.hash(name, type); - } - - @Override - public String toString() { - return "ColumnInfo{" + - "name='" + name + '\'' + - ", type='" + type + '\'' + - '}'; - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ConstantNames.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ConstantNames.java deleted file mode 100644 index e29297b2fe..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ConstantNames.java +++ /dev/null @@ -1,59 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; - -/** - * Constant column names. - */ -public class ConstantNames { - - // for statement execution - public static final String JOB_ID = "job_id"; - // for results with SUCCESS result kind - public static final String RESULT = "result"; - public static final String OK = "OK"; - - public static final String SHOW_MODULES_RESULT = "modules"; - - public static final String SHOW_CURRENT_CATALOG_RESULT = "catalog"; - - public static final String SHOW_CATALOGS_RESULT = "catalogs"; - - public static final String SHOW_CURRENT_DATABASE_RESULT = "database"; - - public static final String SHOW_DATABASES_RESULT = "databases"; - - public static final String SHOW_FUNCTIONS_RESULT = "functions"; - - public static final String EXPLAIN_RESULT = "explanation"; - - public static final String DESCRIBE_NAME = "name"; - public static final String DESCRIBE_TYPE = "type"; - public static final String DESCRIBE_NULL = "null"; - public static final String DESCRIBE_KEY = "key"; - public static final String DESCRIBE_COMPUTED_COLUMN = "computed_column"; - public static final String DESCRIBE_WATERMARK = "watermark"; - - public static final String SHOW_TABLES_RESULT = "tables"; - - public static final String SHOW_VIEWS_RESULT = "views"; - - public static final String SET_KEY = "key"; - public static final String SET_VALUE = "value"; -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultKind.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultKind.java deleted file mode 100644 index 27cb198d8d..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultKind.java +++ /dev/null @@ -1,30 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; - -/** - * ResultKind defines the types of the result. - */ -public enum ResultKind { - // for DDL, DCL and statements with a simple "OK" - SUCCESS, - - // rows with important content are available (DML, DQL) - SUCCESS_WITH_CONTENT -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSet.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSet.java deleted file mode 100644 index 54f54e5c42..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSet.java +++ /dev/null @@ -1,189 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; - -import java.util.Arrays; -import java.util.List; -import java.util.Objects; -import java.util.Optional; -import javax.annotation.Nullable; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize; -import org.apache.flink.types.Row; -import org.apache.flink.util.Preconditions; - -/** - * A set of one statement execution result containing result kind, column infos, - * rows of data and change flags for streaming mode. - */ -@JsonSerialize(using = ResultSetJsonSerializer.class) -@JsonDeserialize(using = ResultSetJsonDeserializer.class) -public class ResultSet { - static final String FIELD_NAME_RESULT_KIND = "result_kind"; - static final String FIELD_NAME_COLUMNS = "columns"; - static final String FIELD_NAME_DATA = "data"; - static final String FIELD_NAME_CHANGE_FLAGS = "change_flags"; - - private final ResultKind resultKind; - private final List columns; - private final List data; - - // null in batch mode - // - // list of boolean in streaming mode, - // true if the corresponding row is an append row, false if its a retract row - private final List changeFlags; - - private ResultSet( - ResultKind resultKind, - List columns, - List data, - @Nullable List changeFlags) { - this.resultKind = Preconditions.checkNotNull(resultKind, "resultKind must not be null"); - this.columns = Preconditions.checkNotNull(columns, "columns must not be null"); - this.data = Preconditions.checkNotNull(data, "data must not be null"); - if (!data.isEmpty()) { - Preconditions.checkArgument(columns.size() == data.get(0).getArity(), - "the size of columns and the number of fields in the row should be equal"); - } - this.changeFlags = changeFlags; - if (changeFlags != null) { - Preconditions.checkArgument(data.size() == changeFlags.size(), - "the size of data and the size of changeFlags should be equal"); - } - } - - public ResultKind getResultKind() { - return resultKind; - } - - public List getColumns() { - return columns; - } - - public List getData() { - return data; - } - - public Optional> getChangeFlags() { - return Optional.ofNullable(changeFlags); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ResultSet resultSet = (ResultSet) o; - return resultKind.equals(resultSet.resultKind) && - columns.equals(resultSet.columns) && - data.equals(resultSet.data) && - Objects.equals(changeFlags, resultSet.changeFlags); - } - - @Override - public int hashCode() { - return Objects.hash(resultKind, columns, data, changeFlags); - } - - @Override - public String toString() { - return "ResultSet{" + - "resultKind=" + resultKind + - ", columns=" + columns + - ", data=" + data + - ", changeFlags=" + changeFlags + - '}'; - } - - public static Builder builder() { - return new Builder(); - } - - /** - * Builder for {@link ResultSet}. - */ - public static class Builder { - private ResultKind resultKind = null; - private List columns = null; - private List data = null; - private List changeFlags = null; - - private Builder() { - } - - /** - * Set {@link ResultKind}. - */ - public Builder resultKind(ResultKind resultKind) { - this.resultKind = resultKind; - return this; - } - - /** - * Set {@link ColumnInfo}s. - */ - public Builder columns(ColumnInfo... columns) { - this.columns = Arrays.asList(columns); - return this; - } - - /** - * Set {@link ColumnInfo}s. - */ - public Builder columns(List columns) { - this.columns = columns; - return this; - } - - /** - * Set data. - */ - public Builder data(List data) { - this.data = data; - return this; - } - - /** - * Set data. - */ - public Builder data(Row... data) { - this.data = Arrays.asList(data); - return this; - } - - /** - * Set change flags. - */ - public Builder changeFlags(List changeFlags) { - this.changeFlags = changeFlags; - return this; - } - - /** - * Returns a {@link ResultSet} instance. - */ - public ResultSet build() { - return new ResultSet(resultKind, columns, data, changeFlags); - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonDeserializer.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonDeserializer.java deleted file mode 100644 index 2d91a8fdc8..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonDeserializer.java +++ /dev/null @@ -1,200 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; - -import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_CHANGE_FLAGS; -import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_COLUMNS; -import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_DATA; -import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_RESULT_KIND; - -import java.io.IOException; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParseException; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonToken; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer; -import org.apache.flink.table.types.logical.DateType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.types.Row; - -/** - * Json deserializer for {@link ResultSet}. - */ -public class ResultSetJsonDeserializer extends StdDeserializer { - - protected ResultSetJsonDeserializer() { - super(ResultSet.class); - } - - @Override - public ResultSet deserialize( - JsonParser jsonParser, - DeserializationContext ctx) throws IOException { - JsonNode node = jsonParser.getCodec().readTree(jsonParser); - - ResultKind resultKind; - List columns; - List changeFlags = null; - List data; - - JsonNode resultKindNode = node.get(FIELD_NAME_RESULT_KIND); - if (resultKindNode != null) { - JsonParser resultKindParser = node.get(FIELD_NAME_RESULT_KIND).traverse(); - resultKindParser.nextToken(); - resultKind = ctx.readValue(resultKindParser, ResultKind.class); - } else { - throw new JsonParseException(jsonParser, "Field resultKind must be provided"); - } - - JsonNode columnNode = node.get(FIELD_NAME_COLUMNS); - if (columnNode != null) { - JsonParser columnParser = node.get(FIELD_NAME_COLUMNS).traverse(); - columnParser.nextToken(); - columns = Arrays.asList(ctx.readValue(columnParser, ColumnInfo[].class)); - } else { - throw new JsonParseException(jsonParser, "Field column must be provided"); - } - - JsonNode changeFlagNode = node.get(FIELD_NAME_CHANGE_FLAGS); - if (changeFlagNode != null) { - JsonParser changeFlagParser = changeFlagNode.traverse(); - changeFlagParser.nextToken(); - changeFlags = Arrays.asList(ctx.readValue(changeFlagParser, Boolean[].class)); - } - - JsonNode dataNode = node.get(FIELD_NAME_DATA); - if (dataNode != null) { - data = deserializeRows(columns, dataNode, ctx); - } else { - throw new JsonParseException(jsonParser, "Field data must be provided"); - } - - return ResultSet.builder() - .resultKind(resultKind) - .columns(columns) - .data(data) - .changeFlags(changeFlags) - .build(); - } - - private List deserializeRows( - List columns, - JsonNode dataNode, - DeserializationContext ctx) throws IOException { - if (!dataNode.isArray()) { - throw new JsonParseException(dataNode.traverse(), "Expecting data to be an array but it's not"); - } - - List fields = new ArrayList<>(); - for (ColumnInfo column : columns) { - fields.add(new RowType.RowField(column.getName(), column.getLogicalType())); - } - RowType rowType = new RowType(fields); - - List data = new ArrayList<>(); - for (JsonNode rowNode : dataNode) { - data.add(deserializeRow(rowType, rowNode, ctx)); - } - return data; - } - - private LocalDate deserializeLocalDate( - JsonParser parser, - DeserializationContext ctx) throws IOException { - return LocalDate.parse(ctx.readValue(parser, String.class)); - } - - private LocalTime deserializeLocalTime( - JsonParser parser, - DeserializationContext ctx) throws IOException { - return LocalTime.parse(ctx.readValue(parser, String.class)); - } - - private LocalDateTime deserializeLocalDateTime( - JsonParser parser, - DeserializationContext ctx) throws IOException { - return LocalDateTime.parse(ctx.readValue(parser, String.class)); - } - - private Row deserializeRow( - RowType type, - JsonNode node, - DeserializationContext ctx) throws IOException { - if (!node.isArray()) { - throw new JsonParseException(node.traverse(), "Expecting row to be an array but it's not"); - } - - int fieldCount = type.getFieldCount(); - List fields = type.getFields(); - Row row = new Row(fieldCount); - - int i = 0; - for (JsonNode fieldNode : node) { - if (i >= fieldCount) { - throw new JsonParseException( - node.traverse(), "Number of columns in the row is not consistent with column infos"); - } - row.setField(i, deserializeObject(fields.get(i).getType(), fieldNode, ctx)); - i++; - } - if (i != fieldCount) { - throw new JsonParseException( - node.traverse(), "Number of columns in the row is not consistent with column infos"); - } - - return row; - } - - private Object deserializeObject( - LogicalType type, - JsonNode node, - DeserializationContext ctx) throws IOException { - if (type instanceof RowType) { - return deserializeRow((RowType) type, node, ctx); - } - - JsonParser parser = node.traverse(); - parser.nextToken(); - if (parser.currentToken() == JsonToken.VALUE_NULL) { - // we have to manually parse null value - // as jackson refuses to deserialize null value to java objects - return null; - } - - if (type instanceof DateType) { - return deserializeLocalDate(parser, ctx); - } else if (type instanceof TimeType) { - return deserializeLocalTime(parser, ctx); - } else if (type instanceof TimestampType) { - return deserializeLocalDateTime(parser, ctx); - } else { - return ctx.readValue(parser, type.getDefaultConversion()); - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonSerializer.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonSerializer.java deleted file mode 100644 index b958a19de2..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/sql/operation/result/ResultSetJsonSerializer.java +++ /dev/null @@ -1,114 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result; - -import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_CHANGE_FLAGS; -import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_COLUMNS; -import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_DATA; -import static com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet.FIELD_NAME_RESULT_KIND; - -import java.io.IOException; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.format.DateTimeFormatter; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; -import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; -import org.apache.flink.types.Row; - -/** - * Json serializer for {@link ResultSet}. - */ -public class ResultSetJsonSerializer extends StdSerializer { - - protected ResultSetJsonSerializer() { - super(ResultSet.class); - } - - @Override - public void serialize( - ResultSet resultSet, - JsonGenerator jsonGenerator, - SerializerProvider serializerProvider) throws IOException { - jsonGenerator.writeStartObject(); - - serializerProvider.defaultSerializeField(FIELD_NAME_RESULT_KIND, resultSet.getResultKind(), jsonGenerator); - serializerProvider.defaultSerializeField(FIELD_NAME_COLUMNS, resultSet.getColumns(), jsonGenerator); - - jsonGenerator.writeFieldName(FIELD_NAME_DATA); - jsonGenerator.writeStartArray(); - for (Row row : resultSet.getData()) { - serializeRow(row, jsonGenerator, serializerProvider); - } - jsonGenerator.writeEndArray(); - - if (resultSet.getChangeFlags().isPresent()) { - serializerProvider.defaultSerializeField(FIELD_NAME_CHANGE_FLAGS, resultSet.getChangeFlags().get(), jsonGenerator); - } - - jsonGenerator.writeEndObject(); - } - - private void serializeLocalDate( - LocalDate localDate, - JsonGenerator jsonGenerator) throws IOException { - jsonGenerator.writeString(localDate.format(DateTimeFormatter.ISO_LOCAL_DATE)); - } - - private void serializeLocalTime( - LocalTime localTime, - JsonGenerator jsonGenerator) throws IOException { - jsonGenerator.writeString(localTime.format(DateTimeFormatter.ISO_LOCAL_TIME)); - } - - private void serializeLocalDateTime( - LocalDateTime localDateTime, - JsonGenerator jsonGenerator) throws IOException { - jsonGenerator.writeString(localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME)); - } - - private void serializeRow( - Row row, - JsonGenerator jsonGenerator, - SerializerProvider serializerProvider) throws IOException { - jsonGenerator.writeStartArray(); - for (int i = 0; i < row.getArity(); i++) { - serializeObject(row.getField(i), jsonGenerator, serializerProvider); - } - jsonGenerator.writeEndArray(); - } - - private void serializeObject( - Object o, - JsonGenerator jsonGenerator, - SerializerProvider serializerProvider) throws IOException { - if (o instanceof LocalDate) { - serializeLocalDate((LocalDate) o, jsonGenerator); - } else if (o instanceof LocalTime) { - serializeLocalTime((LocalTime) o, jsonGenerator); - } else if (o instanceof LocalDateTime) { - serializeLocalDateTime((LocalDateTime) o, jsonGenerator); - } else if (o instanceof Row) { - serializeRow((Row) o, jsonGenerator, serializerProvider); - } else { - serializerProvider.defaultSerializeValue(o, jsonGenerator); - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/SqlCommandParser.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/SqlCommandParser.java deleted file mode 100644 index a260a4e87e..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/SqlCommandParser.java +++ /dev/null @@ -1,403 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils; - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlParseException; -import java.lang.reflect.Field; -import java.util.Arrays; -import java.util.Objects; -import java.util.Optional; -import java.util.function.Function; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.calcite.config.Lex; -import org.apache.calcite.sql.SqlDrop; -import org.apache.calcite.sql.SqlExplain; -import org.apache.calcite.sql.SqlKind; -import org.apache.calcite.sql.SqlNode; -import org.apache.calcite.sql.SqlNodeList; -import org.apache.calcite.sql.SqlSetOption; -import org.apache.calcite.sql.parser.SqlParser; -import org.apache.flink.sql.parser.ddl.SqlAlterDatabase; -import org.apache.flink.sql.parser.ddl.SqlAlterTable; -import org.apache.flink.sql.parser.ddl.SqlCreateDatabase; -import org.apache.flink.sql.parser.ddl.SqlCreateTable; -import org.apache.flink.sql.parser.ddl.SqlCreateView; -import org.apache.flink.sql.parser.ddl.SqlDropDatabase; -import org.apache.flink.sql.parser.ddl.SqlDropTable; -import org.apache.flink.sql.parser.ddl.SqlDropView; -import org.apache.flink.sql.parser.ddl.SqlUseCatalog; -import org.apache.flink.sql.parser.ddl.SqlUseDatabase; -import org.apache.flink.sql.parser.dml.RichSqlInsert; -import org.apache.flink.sql.parser.dql.SqlRichDescribeTable; -import org.apache.flink.sql.parser.dql.SqlShowCatalogs; -import org.apache.flink.sql.parser.dql.SqlShowDatabases; -import org.apache.flink.sql.parser.dql.SqlShowFunctions; -import org.apache.flink.sql.parser.dql.SqlShowTables; -import org.apache.flink.sql.parser.impl.FlinkSqlParserImpl; -import org.apache.flink.sql.parser.validate.FlinkSqlConformance; - -/** - * Simple parser for determining the type of command and its parameters. - */ -public final class SqlCommandParser { - - private SqlCommandParser() { - // private - } - - /** - * Parse the given statement and return corresponding SqlCommandCall. - * - *

only `set`, `show modules`, `show current catalog` and `show current database` - * are parsed through regex matching, other commands are parsed through sql parser. - * - *

throw {@link SqlParseException} if the statement contains multiple sub-statements separated by semicolon - * or there is a parse error. - * - *

NOTE: sql parser only parses the statement to get the corresponding SqlCommand, - * do not check whether the statement is valid here. - */ - public static Optional parse(String stmt, boolean isBlinkPlanner) throws SqlParseException { - // normalize - String stmtForRegexMatch = stmt.trim(); - // remove ';' at the end - if (stmtForRegexMatch.endsWith(";")) { - stmtForRegexMatch = stmtForRegexMatch.substring(0, stmtForRegexMatch.length() - 1).trim(); - } - - // only parse gateway specific statements - for (SqlCommand cmd : SqlCommand.values()) { - if (cmd.hasPattern()) { - final Matcher matcher = cmd.pattern.matcher(stmtForRegexMatch); - if (matcher.matches()) { - final String[] groups = new String[matcher.groupCount()]; - for (int i = 0; i < groups.length; i++) { - groups[i] = matcher.group(i + 1); - } - return cmd.operandConverter.apply(groups) - .map((operands) -> new SqlCommandCall(cmd, operands)); - } - } - } - - return parseStmt(stmt, isBlinkPlanner); - } - - /** - * Flink Parser only supports partial Operations, so we directly use Calcite Parser here. - * Once Flink Parser supports all Operations, we should use Flink Parser instead of Calcite Parser. - */ - private static Optional parseStmt(String stmt, boolean isBlinkPlanner) throws SqlParseException { - SqlParser.Config config = createSqlParserConfig(isBlinkPlanner); - SqlParser sqlParser = SqlParser.create(stmt, config); - SqlNodeList sqlNodes; - try { - sqlNodes = sqlParser.parseStmtList(); - // no need check the statement is valid here - } catch (org.apache.calcite.sql.parser.SqlParseException e) { - throw new SqlParseException("Failed to parse statement.", e); - } - if (sqlNodes.size() != 1) { - throw new SqlParseException("Only single statement is supported now"); - } - - final String[] operands; - final SqlCommand cmd; - SqlNode node = sqlNodes.get(0); - if (node.getKind().belongsTo(SqlKind.QUERY)) { - cmd = SqlCommand.SELECT; - operands = new String[] { stmt }; - } else if (node instanceof RichSqlInsert) { - RichSqlInsert insertNode = (RichSqlInsert) node; - cmd = insertNode.isOverwrite() ? SqlCommand.INSERT_OVERWRITE : SqlCommand.INSERT_INTO; - operands = new String[] { stmt, insertNode.getTargetTable().toString() }; - } else if (node instanceof SqlShowTables) { - cmd = SqlCommand.SHOW_TABLES; - operands = new String[0]; - } else if (node instanceof SqlCreateTable) { - cmd = SqlCommand.CREATE_TABLE; - operands = new String[] { stmt }; - } else if (node instanceof SqlDropTable) { - cmd = SqlCommand.DROP_TABLE; - operands = new String[] { stmt }; - } else if (node instanceof SqlAlterTable) { - cmd = SqlCommand.ALTER_TABLE; - operands = new String[] { stmt }; - } else if (node instanceof SqlCreateView) { - // TableEnvironment currently does not support creating view - // so we have to perform the modification here - SqlCreateView createViewNode = (SqlCreateView) node; - cmd = SqlCommand.CREATE_VIEW; - operands = new String[] { - createViewNode.getViewName().toString(), - createViewNode.getQuery().toString() - }; - } else if (node instanceof SqlDropView) { - // TableEnvironment currently does not support dropping view - // so we have to perform the modification here - SqlDropView dropViewNode = (SqlDropView) node; - - // TODO: we can't get this field from SqlDropView normally until FLIP-71 is implemented - Field ifExistsField; - try { - ifExistsField = SqlDrop.class.getDeclaredField("ifExists"); - } catch (NoSuchFieldException e) { - throw new SqlParseException("Failed to parse drop view statement.", e); - } - ifExistsField.setAccessible(true); - boolean ifExists; - try { - ifExists = ifExistsField.getBoolean(dropViewNode); - } catch (IllegalAccessException e) { - throw new SqlParseException("Failed to parse drop view statement.", e); - } - - cmd = SqlCommand.DROP_VIEW; - operands = new String[] { dropViewNode.getViewName().toString(), String.valueOf(ifExists) }; - } else if (node instanceof SqlShowDatabases) { - cmd = SqlCommand.SHOW_DATABASES; - operands = new String[0]; - } else if (node instanceof SqlCreateDatabase) { - cmd = SqlCommand.CREATE_DATABASE; - operands = new String[] { stmt }; - } else if (node instanceof SqlDropDatabase) { - cmd = SqlCommand.DROP_DATABASE; - operands = new String[] { stmt }; - } else if (node instanceof SqlAlterDatabase) { - cmd = SqlCommand.ALTER_DATABASE; - operands = new String[] { stmt }; - } else if (node instanceof SqlShowCatalogs) { - cmd = SqlCommand.SHOW_CATALOGS; - operands = new String[0]; - } else if (node instanceof SqlShowFunctions) { - cmd = SqlCommand.SHOW_FUNCTIONS; - operands = new String[0]; - } else if (node instanceof SqlUseCatalog) { - cmd = SqlCommand.USE_CATALOG; - operands = new String[] { ((SqlUseCatalog) node).getCatalogName() }; - } else if (node instanceof SqlUseDatabase) { - cmd = SqlCommand.USE; - operands = new String[] { ((SqlUseDatabase) node).getDatabaseName().toString() }; - } else if (node instanceof SqlRichDescribeTable) { - cmd = SqlCommand.DESCRIBE_TABLE; - // TODO support describe extended - String[] fullTableName = ((SqlRichDescribeTable) node).fullTableName(); - String escapedName = - Stream.of(fullTableName).map(s -> "`" + s + "`").collect(Collectors.joining(".")); - operands = new String[] { escapedName }; - } else if (node instanceof SqlExplain) { - cmd = SqlCommand.EXPLAIN; - // TODO support explain details - operands = new String[] { ((SqlExplain) node).getExplicandum().toString() }; - } else if (node instanceof SqlSetOption) { - SqlSetOption setNode = (SqlSetOption) node; - // refer to SqlSetOption#unparseAlterOperation - if (setNode.getValue() != null) { - cmd = SqlCommand.SET; - operands = new String[] { setNode.getName().toString(), setNode.getValue().toString() }; - } else { - cmd = SqlCommand.RESET; - if (setNode.getName().toString().toUpperCase().equals("ALL")) { - operands = new String[0]; - } else { - operands = new String[] { setNode.getName().toString() }; - } - } - } else { - cmd = null; - operands = new String[0]; - } - - if (cmd == null) { - return Optional.empty(); - } else { - // use the origin given statement to make sure - // users can find the correct line number when parsing failed - return Optional.of(new SqlCommandCall(cmd, operands)); - } - } - - /** - * A temporary solution. We can't get the default SqlParser config through table environment now. - */ - private static SqlParser.Config createSqlParserConfig(boolean isBlinkPlanner) { - if (isBlinkPlanner) { - return SqlParser - .configBuilder() - .setParserFactory(FlinkSqlParserImpl.FACTORY) - .setConformance(FlinkSqlConformance.DEFAULT) - .setLex(Lex.JAVA) - .setIdentifierMaxLength(256) - .build(); - } else { - return SqlParser - .configBuilder() - .setParserFactory(FlinkSqlParserImpl.FACTORY) - .setConformance(FlinkSqlConformance.DEFAULT) - .setLex(Lex.JAVA) - .build(); - } - } - - // -------------------------------------------------------------------------------------------- - - private static final Function> NO_OPERANDS = - (operands) -> Optional.of(new String[0]); - - private static final int DEFAULT_PATTERN_FLAGS = Pattern.CASE_INSENSITIVE | Pattern.DOTALL; - - /** - * Supported SQL commands. - */ - public enum SqlCommand { - SELECT, - - INSERT_INTO, - - INSERT_OVERWRITE, - - CREATE_TABLE, - - ALTER_TABLE, - - DROP_TABLE, - - CREATE_VIEW, - - DROP_VIEW, - - CREATE_DATABASE, - - ALTER_DATABASE, - - DROP_DATABASE, - - USE_CATALOG, - - USE, - - SHOW_CATALOGS, - - SHOW_DATABASES, - - SHOW_TABLES, - - SHOW_FUNCTIONS, - - EXPLAIN, - - DESCRIBE_TABLE, - - RESET, - - // the following commands are not supported by SQL parser but are needed by users - - SET( - "SET", - // `SET` with operands can be parsed by SQL parser - // we keep `SET` with no operands here to print all properties - NO_OPERANDS), - - // the following commands will be supported by SQL parser in the future - // remove them once they're supported - - // FLINK-17396 - SHOW_MODULES( - "SHOW\\s+MODULES", - NO_OPERANDS), - - // FLINK-17111 - SHOW_VIEWS( - "SHOW\\s+VIEWS", - NO_OPERANDS), - - // the following commands are not supported by SQL parser but are needed by JDBC driver - // these should not be exposed to the user and should be used internally - - SHOW_CURRENT_CATALOG( - "SHOW\\s+CURRENT\\s+CATALOG", - NO_OPERANDS), - - SHOW_CURRENT_DATABASE( - "SHOW\\s+CURRENT\\s+DATABASE", - NO_OPERANDS); - - public final Pattern pattern; - public final Function> operandConverter; - - SqlCommand(String matchingRegex, Function> operandConverter) { - this.pattern = Pattern.compile(matchingRegex, DEFAULT_PATTERN_FLAGS); - this.operandConverter = operandConverter; - } - - SqlCommand() { - this.pattern = null; - this.operandConverter = null; - } - - @Override - public String toString() { - return super.toString().replace('_', ' '); - } - - boolean hasPattern() { - return pattern != null; - } - } - - /** - * Call of SQL command with operands and command type. - */ - public static class SqlCommandCall { - public final SqlCommand command; - public final String[] operands; - - public SqlCommandCall(SqlCommand command, String[] operands) { - this.command = command; - this.operands = operands; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - SqlCommandCall that = (SqlCommandCall) o; - return command == that.command && Arrays.equals(operands, that.operands); - } - - @Override - public int hashCode() { - int result = Objects.hash(command); - result = 31 * result + Arrays.hashCode(operands); - return result; - } - - @Override - public String toString() { - return command + "(" + Arrays.toString(operands) + ")"; - } - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/YarnConfLoader.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/YarnConfLoader.java deleted file mode 100644 index a8caef920e..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/client/utils/YarnConfLoader.java +++ /dev/null @@ -1,51 +0,0 @@ - -package com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils; - -import java.io.File; -import java.util.Iterator; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.conf.YarnConfiguration; - -public class YarnConfLoader { - public static YarnConfiguration getYarnConf(String yarnConfDir) { - YarnConfiguration yarnConf = new YarnConfiguration(); - try { - File dir = new File(yarnConfDir); - if (dir.exists() && dir.isDirectory()) { - File[] xmlFileList = new File(yarnConfDir).listFiles((dir1, name) -> { - if (name.endsWith(".xml")) { - return true; - } - return false; - }); - if (xmlFileList != null) { - for (File xmlFile : xmlFileList) { - yarnConf.addResource(xmlFile.toURI().toURL()); - } - } - } - } catch (Exception e) { - throw new RuntimeException(e); - } - haYarnConf(yarnConf); - return yarnConf; - } - - private static Configuration haYarnConf(Configuration yarnConf) { - Iterator> iterator = yarnConf.iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - String key = entry.getKey(); - String value = entry.getValue(); - if (key.startsWith("yarn.resourcemanager.hostname.")) { - String rm = key.substring("yarn.resourcemanager.hostname.".length()); - String addressKey = "yarn.resourcemanager.address." + rm; - if (yarnConf.get(addressKey) == null) { - yarnConf.set(addressKey, value + ":" + YarnConfiguration.DEFAULT_RM_PORT); - } - } - } - return yarnConf; - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/ExecutorInitException.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/ExecutorInitException.java deleted file mode 100644 index 1b2e71d5a9..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/ExecutorInitException.java +++ /dev/null @@ -1,47 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.exception; - - -import com.webank.wedatasphere.linkis.common.exception.ErrorException; - - -public class ExecutorInitException extends ErrorException { - - public static final int ERROR_CODE = 16021; - - private static final long serialVersionUID = 1L; - - public ExecutorInitException(int errCode, String desc) { - super(errCode, desc); - } - - public ExecutorInitException(String desc) { - super(20001, desc); - } - - public ExecutorInitException(Exception e) { - super(20001, e.getMessage()); - } - - public ExecutorInitException() { - super(20001, "argument illegal"); - } - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/FlinkInitFailedException.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/FlinkInitFailedException.java deleted file mode 100644 index c8fe2b7395..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/FlinkInitFailedException.java +++ /dev/null @@ -1,38 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.exception; - - -import com.webank.wedatasphere.linkis.common.exception.ErrorException; - -public class FlinkInitFailedException extends ErrorException { - - public static final int ERROR_CODE = 16020; - - private static final long serialVersionUID = 1L; - - public FlinkInitFailedException(String msg) { - super(ERROR_CODE, msg); - } - - public FlinkInitFailedException(String msg, Throwable cause) { - super(ERROR_CODE, msg); - initCause(cause); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/JobExecutionException.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/JobExecutionException.java deleted file mode 100644 index 832ffab5f2..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/JobExecutionException.java +++ /dev/null @@ -1,38 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.exception; - - -import com.webank.wedatasphere.linkis.common.exception.ErrorException; - -public class JobExecutionException extends ErrorException { - - private static final long serialVersionUID = 1L; - - public static final int ERROR_CODE = 16023; - - public JobExecutionException(String message) { - super(ERROR_CODE, message); - } - - public JobExecutionException(String message, Throwable e) { - super(ERROR_CODE, message); - this.initCause(e); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlExecutionException.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlExecutionException.java deleted file mode 100644 index 7c386e5100..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlExecutionException.java +++ /dev/null @@ -1,38 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.exception; - - -import com.webank.wedatasphere.linkis.common.exception.ErrorException; - -public class SqlExecutionException extends ErrorException { - - public static final int ERROR_CODE = 16022; - - private static final long serialVersionUID = 1L; - - public SqlExecutionException(String message) { - super(ERROR_CODE, message); - } - - public SqlExecutionException(String message, Throwable e) { - super(ERROR_CODE, message); - initCause(e); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlParseException.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlParseException.java deleted file mode 100644 index e0916e258c..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/exception/SqlParseException.java +++ /dev/null @@ -1,38 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.exception; - - -import com.webank.wedatasphere.linkis.common.exception.ErrorException; - -public class SqlParseException extends ErrorException { - - public static final int ERROR_CODE = 16021; - - private static final long serialVersionUID = 1L; - - public SqlParseException(String message) { - super(ERROR_CODE, message); - } - - public SqlParseException(String message, Throwable e) { - super(ERROR_CODE, message); - this.initCause(e); - } -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/RetryUtil.java b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/RetryUtil.java deleted file mode 100644 index 51c1f0eb59..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/java/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/RetryUtil.java +++ /dev/null @@ -1,54 +0,0 @@ -package com.webank.wedatasphere.linkis.engineconnplugin.flink.util; - -import com.github.rholder.retry.RetryException; -import com.github.rholder.retry.Retryer; -import com.github.rholder.retry.RetryerBuilder; -import com.github.rholder.retry.StopStrategies; -import com.github.rholder.retry.WaitStrategies; -import com.google.common.base.Predicate; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * 基于guava-retry的重试工具类 - */ -public class RetryUtil { - - private static final Logger logger = LoggerFactory.getLogger(RetryUtil.class); - - /** - * @param task 要重试执行得任务 - * @param predicate 符合预期结果需要重试 - * @param fixedWaitTime 本次重试与上次重试之间的固定间隔时长 - * @param maxEachExecuTime 一次重试的最大执行的时间 - * @param attemptNumber 重试次数 - */ - - public static T retry(Callable task, Predicate predicate, long fixedWaitTime, long maxEachExecuTime, - TimeUnit timeUnit, int attemptNumber) { - Retryer retryer = RetryerBuilder - .newBuilder() - // 抛出runtime异常、checked异常时都会重试,但是抛出error不会重试。 - .retryIfException() - // 对执行结果的预期。符合预期就重试 - .retryIfResult(predicate) - // 每次重试固定等待fixedWaitTime时间 - .withWaitStrategy(WaitStrategies.fixedWait(fixedWaitTime, timeUnit)) - // 尝试次数 - .withStopStrategy(StopStrategies.stopAfterAttempt(attemptNumber)) - .build(); - T t = null; - try { - t = retryer.call(task); - } catch (ExecutionException e) { - logger.error("", e); - } catch (RetryException e) { - logger.error("", e); - } - return t; - } - -} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/linkis-engineconn.properties b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/linkis-engineconn.properties deleted file mode 100644 index 27d74c8c34..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/linkis-engineconn.properties +++ /dev/null @@ -1,27 +0,0 @@ -# -# Copyright 2019 WeBank -# -# Licensed under the Apache License, Version 2.0 (the "License"); -# you may not use this file except in compliance with the License. -# You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - - -wds.linkis.server.version=v1 - - -wds.linkis.engineconn.debug.enable=true - -#wds.linkis.keytab.enable=true - -wds.linkis.engineconn.plugin.default.class=com.webank.wedatasphere.linkis.engineconnplugin.flink.FlinkEngineConnPlugin - -wds.linkis.engine.connector.hooks=com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.ComputationEngineConnHook,com.webank.wedatasphere.linkis.engineconn.computation.executor.hook.JarUdfEngineHook diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/log4j2-engineconn.xml b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/log4j2-engineconn.xml deleted file mode 100644 index b78b215127..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/resources/log4j2-engineconn.xml +++ /dev/null @@ -1,64 +0,0 @@ - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/FlinkEngineConnPlugin.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/FlinkEngineConnPlugin.scala deleted file mode 100644 index 63517afd56..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/FlinkEngineConnPlugin.scala +++ /dev/null @@ -1,77 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.factory.FlinkEngineConnFactory -import com.webank.wedatasphere.linkis.engineconnplugin.flink.launch.FlinkEngineConnLaunchBuilder -import com.webank.wedatasphere.linkis.engineconnplugin.flink.resource.FlinkEngineConnResourceFactory -import com.webank.wedatasphere.linkis.manager.engineplugin.common.EngineConnPlugin -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.EngineConnFactory -import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.EngineConnLaunchBuilder -import com.webank.wedatasphere.linkis.manager.engineplugin.common.resource.EngineResourceFactory -import com.webank.wedatasphere.linkis.manager.label.entity.Label - -/** - * - */ -class FlinkEngineConnPlugin extends EngineConnPlugin { - - private var engineResourceFactory: EngineResourceFactory = _ - private var engineConnLaunchBuilder: EngineConnLaunchBuilder = _ - private var engineConnFactory: EngineConnFactory = _ - - private val EP_CONTEXT_CONSTRUCTOR_LOCK = new Object() - - - override def init(params: java.util.Map[String, Any]): Unit = { - //do noting -// engineResourceFactory = new FlinkEngineConnResourceFactory -// engineConnLaunchBuilder = new FlinkEngineConnLaunchBuilder -// engineConnFactory = new FlinkEngineConnFactory - } - - override def getEngineResourceFactory: EngineResourceFactory = { - EP_CONTEXT_CONSTRUCTOR_LOCK.synchronized{ - if(null == engineResourceFactory){ - engineResourceFactory = new FlinkEngineConnResourceFactory - } - engineResourceFactory - } - } - - override def getEngineConnLaunchBuilder: EngineConnLaunchBuilder = { - EP_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - // todo check - if (null == engineConnLaunchBuilder) { - engineConnLaunchBuilder = new FlinkEngineConnLaunchBuilder() - } - engineConnLaunchBuilder - } - } - - override def getEngineConnFactory: EngineConnFactory = { - EP_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - if (null == engineConnFactory) { - engineConnFactory = new FlinkEngineConnFactory - } - engineConnFactory - } - } - - override def getDefaultLabels: java.util.List[Label[_]] = new java.util.ArrayList[Label[_]] -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkEnvConfiguration.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkEnvConfiguration.scala deleted file mode 100644 index 488cfe0d76..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkEnvConfiguration.scala +++ /dev/null @@ -1,58 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.config - -import com.webank.wedatasphere.linkis.common.conf.{CommonVars, TimeType} -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.entries.ExecutionEntry -import com.webank.wedatasphere.linkis.manager.label.entity.cluster.EnvLabel - -/** - * - */ -object FlinkEnvConfiguration { - - val FLINK_HOME_ENV = "FLINK_HOME" - val FLINK_CONF_DIR_ENV = "FLINK_CONF_DIR" - val FLINK_HOME = CommonVars("flink.home", CommonVars(FLINK_HOME_ENV, "/appcom/Install/flink").getValue) - val FLINK_CONF_DIR = CommonVars("flink.conf.dir", CommonVars(FLINK_CONF_DIR_ENV, "/appcom/config/flink-config").getValue) - val FLINK_DIST_JAR_PATH = CommonVars("flink.dist.jar.path", "/appcom/Install/flink/lib/flink-dist_2.11-1.11.1.jar") - val FLINK_LIB_REMOTE_PATH = CommonVars("flink.lib.path", "") - val FLINK_USER_LIB_REMOTE_PATH = CommonVars("flink.user.lib.path", "", "The hdfs lib path of each user in Flink EngineConn.") - val FLINK_LIB_LOCAL_PATH = CommonVars("flink.local.lib.path", "/appcom/Install/flink/lib", "The local lib path of Flink EngineConn.") - val FLINK_USER_LIB_LOCAL_PATH = CommonVars("flink.user.local.lib.path", "/appcom/Install/flink/lib", "The local lib path of each user in Flink EngineConn.") - val FLINK_SHIP_DIRECTORIES = CommonVars("flink.yarn.ship-directories", "") - - val FLINK_ENV_TYPE = CommonVars("flink.app.env.type", EnvLabel.DEV) - - val FLINK_SAVE_POINT_PATH = CommonVars("flink.app.savePointPath", "") - val FLINK_APP_ALLOW_NON_RESTORED_STATUS = CommonVars("flink.app.allowNonRestoredStatus", "false") - val FLINK_SQL_PLANNER = CommonVars("flink.sql.planner", ExecutionEntry.EXECUTION_PLANNER_VALUE_BLINK) - val FLINK_SQL_EXECUTION_TYPE = CommonVars("flink.sql.executionType", ExecutionEntry.EXECUTION_TYPE_VALUE_BATCH) - - val FLINK_SQL_DEV_SELECT_MAX_LINES = CommonVars("flink.dev.sql.select.lines.max", 500) - val FLINK_SQL_DEV_RESULT_MAX_WAIT_TIME = CommonVars("flink.dev.sql.result.wait.time.max", new TimeType("1m")) - - val FLINK_APPLICATION_ARGS = CommonVars("flink.app.args", "") - val FLINK_APPLICATION_MAIN_CLASS = CommonVars("flink.app.main.class", "") - val FLINK_APPLICATION_MAIN_CLASS_JAR = CommonVars("flink.app.main.class.jar", "") - - val FLINK_CLIENT_REQUEST_TIMEOUT = CommonVars("flink.client.request.timeout", new TimeType("30s")) - val FLINK_APPLICATION_STATUS_FETCH_INTERVAL = CommonVars("flink.app.fetch.status.interval", new TimeType("5s")) - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkResourceConfiguration.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkResourceConfiguration.scala deleted file mode 100644 index 69cf53b672..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/config/FlinkResourceConfiguration.scala +++ /dev/null @@ -1,41 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.config - -import com.webank.wedatasphere.linkis.common.conf.CommonVars - -/** - * - */ -object FlinkResourceConfiguration { - - val LINKIS_FLINK_CLIENT_MEMORY = CommonVars[Int]("flink.client.memory", 4) //单位为G - val LINKIS_FLINK_CLIENT_CORES = 1 //Fixed to 1(固定为1) CommonVars[Int]("wds.linkis.driver.cores", 1) - - - val LINKIS_FLINK_JOB_MANAGER_MEMORY = CommonVars[Int]("flink.jobmanager.memory", 2) //单位为G - val LINKIS_FLINK_TASK_MANAGER_MEMORY = CommonVars[Int]("flink.taskmanager.memory", 4) //单位为G - val LINKIS_FLINK_TASK_SLOTS = CommonVars[Int]("flink.taskmanager.numberOfTaskSlots", 2) - val LINKIS_FLINK_TASK_MANAGER_CPU_CORES = CommonVars[Int]("flink.taskmanager.cpu.cores", 2) - val LINKIS_FLINK_CONTAINERS = CommonVars[Int]("flink.container.num", 2) - val LINKIS_QUEUE_NAME = CommonVars[String]("wds.linkis.rm.yarnqueue", "default") - - - val FLINK_APP_DEFAULT_PARALLELISM = CommonVars("wds.linkis.engineconn.flink.app.parallelism", 4) - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/EnvironmentContext.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/EnvironmentContext.scala deleted file mode 100644 index fe70c30b4b..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/EnvironmentContext.scala +++ /dev/null @@ -1,106 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.context - -import java.net.URL -import java.util -import java.util.Objects - -import com.google.common.collect.Lists -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.factory.LinkisYarnClusterClientFactory -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment -import org.apache.commons.lang.StringUtils -import org.apache.flink.client.deployment.{ClusterClientServiceLoader, DefaultClusterClientServiceLoader} -import org.apache.flink.configuration.{Configuration, DeploymentOptionsInternal, GlobalConfiguration} -import org.apache.flink.yarn.configuration.{YarnConfigOptions, YarnDeploymentTarget} - -/** - * - */ -class EnvironmentContext(defaultEnv: Environment, - yarnConfDir: String, - flinkConfDir: String, - flinkHome: String, - distJarPath: String, - flinkLibRemotePath: String, - dependencies: util.List[URL]) { - - private var providedLibDirs: util.List[String] = _ - private var shipDirs: util.List[String] = _ - - private var flinkConfig: Configuration = _ - private var clusterClientServiceLoader: ClusterClientServiceLoader = _ - - private var deploymentTarget: YarnDeploymentTarget = YarnDeploymentTarget.PER_JOB - - def this(defaultEnv: Environment, systemConfiguration: Configuration, yarnConfDir: String, flinkConfDir: String, - flinkHome: String, distJarPath: String, flinkLibRemotePath: String, providedLibDirsArray: Array[String], - shipDirsArray: Array[String], dependencies: util.List[URL]) { - this(defaultEnv, yarnConfDir, flinkConfDir, flinkHome, distJarPath, flinkLibRemotePath, dependencies) - //远程资源目录 - this.providedLibDirs = Lists.newArrayList(providedLibDirsArray.filter(StringUtils.isNotBlank): _*) - //本地资源目录 - this.shipDirs = Lists.newArrayList(shipDirsArray.filter(StringUtils.isNotBlank): _*) - //加载系统级别配置 - this.flinkConfig = GlobalConfiguration.loadConfiguration(this.flinkConfDir) - if (null != systemConfiguration) this.flinkConfig.addAll(systemConfiguration) - //设置 flink conf目录 - this.flinkConfig.set(DeploymentOptionsInternal.CONF_DIR, this.flinkConfDir) - //设置 yarn conf目录 - this.flinkConfig.set(LinkisYarnClusterClientFactory.YARN_CONFIG_DIR, this.yarnConfDir) - //设置 flink dist jar - this.flinkConfig.set(YarnConfigOptions.FLINK_DIST_JAR, distJarPath) - clusterClientServiceLoader = new DefaultClusterClientServiceLoader - } - - def setDeploymentTarget(deploymentTarget: YarnDeploymentTarget): Unit = this.deploymentTarget = deploymentTarget - - def getDeploymentTarget: YarnDeploymentTarget = deploymentTarget - - def getProvidedLibDirs: util.List[String] = providedLibDirs - - def getShipDirs: util.List[String] = shipDirs - - def getYarnConfDir: String = yarnConfDir - - def getFlinkConfDir: String = flinkConfDir - - def getFlinkHome: String = flinkHome - - def getFlinkLibRemotePath: String = flinkLibRemotePath - - def getFlinkConfig: Configuration = flinkConfig - - def getDefaultEnv: Environment = defaultEnv - - def getDependencies: util.List[URL] = dependencies - - def getClusterClientServiceLoader: ClusterClientServiceLoader = clusterClientServiceLoader - - override def equals(o: Any): Boolean = o match { - case context: EnvironmentContext => - if(this eq context) return true - Objects.equals(defaultEnv, context.getDefaultEnv) && - Objects.equals(dependencies, context.getDependencies) && - Objects.equals(flinkConfig, context.flinkConfig) && - Objects.equals(clusterClientServiceLoader, context.clusterClientServiceLoader) - case _ => false - } - - override def hashCode: Int = Objects.hash(defaultEnv, dependencies, flinkConfig, clusterClientServiceLoader) -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/FlinkEngineConnContext.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/FlinkEngineConnContext.scala deleted file mode 100644 index 049c1fc14d..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/context/FlinkEngineConnContext.scala +++ /dev/null @@ -1,40 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.context - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext - -/** - * - */ -class FlinkEngineConnContext(environmentContext: EnvironmentContext) { - - private var executionContext: ExecutionContext = _ - - def getEnvironmentContext: EnvironmentContext = environmentContext - - def getExecutionContext: ExecutionContext = executionContext - - def setExecutionContext(executionContext: ExecutionContext): Unit = this.executionContext = executionContext - - def newExecutionContextBuilder(environment: Environment): ExecutionContext.Builder = - ExecutionContext.builder(environmentContext.getDefaultEnv, environment, environmentContext.getDependencies, - environmentContext.getFlinkConfig, environmentContext.getClusterClientServiceLoader) - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkApplicationJobExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkApplicationJobExecutor.scala deleted file mode 100644 index 50ea10c19a..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkApplicationJobExecutor.scala +++ /dev/null @@ -1,88 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.executor - -import java.util.concurrent.{Future, TimeUnit} - -import com.webank.wedatasphere.linkis.common.utils.Utils -import com.webank.wedatasphere.linkis.engineconn.once.executor.{ManageableOnceExecutor, OnceExecutorExecutionContext} -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.deployment.{ClusterDescriptorAdapterFactory, YarnApplicationClusterDescriptorAdapter} -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration._ -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.ExecutorInitException - -import scala.collection.convert.WrapAsScala._ -import scala.concurrent.duration.Duration - -/** - * - */ -class FlinkApplicationJobExecutor(id: Long, - override protected val flinkEngineConnContext: FlinkEngineConnContext) - extends ManageableOnceExecutor with FlinkJobExecutor { - - private var clusterDescriptor: YarnApplicationClusterDescriptorAdapter = _ - private var daemonThread: Future[_] = _ - - protected def submit(onceExecutorExecutionContext: OnceExecutorExecutionContext): Unit = { - ClusterDescriptorAdapterFactory.create(flinkEngineConnContext.getExecutionContext, null) match { - case adapter: YarnApplicationClusterDescriptorAdapter => clusterDescriptor = adapter - case _ => throw new ExecutorInitException("Not support ClusterDescriptorAdapter for flink application.") - } - val options = onceExecutorExecutionContext.getOnceExecutorContent.getJobContent.map { - case (k, v: String) => k -> v - case (k, v) if v != null => k -> v.toString - case (k, _) => k -> null - }.toMap - val programArguments = FLINK_APPLICATION_ARGS.getValue(options).split(" ") - val mainClass = FLINK_APPLICATION_MAIN_CLASS.getValue(options) - clusterDescriptor.deployCluster(programArguments, mainClass) - if (null == clusterDescriptor.getJobId || null == clusterDescriptor.getClusterID) - throw new ExecutorInitException("The app " + mainClass + " start failed, no result was returned.") - setJobID(clusterDescriptor.getJobId.toHexString) - setApplicationId(clusterDescriptor.getClusterID.toString) - setApplicationURL(clusterDescriptor.getWebInterfaceUrl) - } - - override def getId: String = "flinkApp_"+ id - - override def close(): Unit = { - if(daemonThread != null) daemonThread.cancel(true) - if(clusterDescriptor != null) { - clusterDescriptor.cancelJob() - clusterDescriptor.close() - } - flinkEngineConnContext.getExecutionContext.getClusterClientFactory.close() - super.close() - } - - override protected def waitToRunning(): Unit = { - Utils.waitUntil(() => clusterDescriptor.initJobId(), Duration.Inf) - daemonThread = Utils.defaultScheduler.scheduleAtFixedRate(new Runnable { - override def run(): Unit = { - val jobStatus = clusterDescriptor.getJobStatus - info(s"The jobStatus of $getJobID is $jobStatus.") - if(jobStatus.isGloballyTerminalState) - tryFailed() - else if(jobStatus.isTerminalState) tryShutdown() - } - }, FLINK_APPLICATION_STATUS_FETCH_INTERVAL.getValue.toLong, FLINK_APPLICATION_STATUS_FETCH_INTERVAL.getValue.toLong, TimeUnit.MILLISECONDS) - } - - override def supportCallBackLogs(): Boolean = true -} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkJobExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkJobExecutor.scala deleted file mode 100644 index cbb28fea0c..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkJobExecutor.scala +++ /dev/null @@ -1,133 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.executor - -import java.util - -import com.webank.wedatasphere.linkis.common.io.resultset.ResultSetWriter -import com.webank.wedatasphere.linkis.common.io.{MetaData, Record} -import com.webank.wedatasphere.linkis.common.utils.{OverloadUtils, Utils} -import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.EngineExecutionContext -import com.webank.wedatasphere.linkis.engineconn.executor.entity.{LabelExecutor, ResourceExecutor, YarnExecutor} -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultSet -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkResourceConfiguration -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkResourceConfiguration.LINKIS_FLINK_CLIENT_CORES -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.JobExecutionException -import com.webank.wedatasphere.linkis.engineconnplugin.flink.util.FlinkValueFormatUtil -import com.webank.wedatasphere.linkis.manager.common.entity.resource._ -import com.webank.wedatasphere.linkis.manager.common.utils.ResourceUtils -import com.webank.wedatasphere.linkis.manager.label.entity.Label -import com.webank.wedatasphere.linkis.storage.domain.{Column, DataType} -import com.webank.wedatasphere.linkis.storage.resultset.ResultSetFactory -import com.webank.wedatasphere.linkis.storage.resultset.table.{TableMetaData, TableRecord} -import org.apache.commons.io.IOUtils -import org.apache.flink.configuration.{CoreOptions, JobManagerOptions, TaskManagerOptions} -import org.apache.flink.types.Row -import org.apache.flink.yarn.configuration.YarnConfigOptions - -/** - * - */ -trait FlinkJobExecutor extends YarnExecutor with LabelExecutor with ResourceExecutor { - - private var jobID: String = _ - private var applicationId: String = _ - private var applicationURL: String = _ - private var yarnMode: String = "Client" - private var queue: String = _ - - private var executorLabels: util.List[Label[_]] = new util.ArrayList[Label[_]] - - def getJobID: String = jobID - - protected def setJobID(jobID: String): Unit = { - this.jobID = jobID - } - - override def getApplicationId: String = applicationId - def setApplicationId(applicationId: String): Unit = this.applicationId = applicationId - - override def getApplicationURL: String = applicationURL - def setApplicationURL(applicationURL: String): Unit = this.applicationURL = applicationURL - - override def getYarnMode: String = yarnMode - def setYarnMode(yarnMode: String): Unit = this.yarnMode = yarnMode - - override def getQueue: String = queue - def setQueue(queue: String): Unit = this.queue = queue - - override def getExecutorLabels(): util.List[Label[_]] = executorLabels - - override def setExecutorLabels(labels: util.List[Label[_]]): Unit = this.executorLabels = labels - - override def requestExpectedResource(expectedResource: NodeResource): NodeResource = throw new JobExecutionException("Not support method for requestExpectedResource.") - - protected val flinkEngineConnContext: FlinkEngineConnContext - - queue = flinkEngineConnContext.getEnvironmentContext.getFlinkConfig.get(YarnConfigOptions.APPLICATION_QUEUE) - - override def getCurrentNodeResource(): NodeResource = { - val flinkConfig = flinkEngineConnContext.getEnvironmentContext.getFlinkConfig - val jobManagerMemory = flinkConfig.get(JobManagerOptions.TOTAL_PROCESS_MEMORY).getBytes - val taskManagerMemory = flinkConfig.get(TaskManagerOptions.TOTAL_PROCESS_MEMORY).getBytes - val parallelism = flinkConfig.get(CoreOptions.DEFAULT_PARALLELISM) - val numOfTaskSlots =flinkConfig.get(TaskManagerOptions.NUM_TASK_SLOTS) - val containers = Math.round(parallelism * 1.0f / numOfTaskSlots) - val yarnMemory = taskManagerMemory * containers + jobManagerMemory - val yarnCores = FlinkResourceConfiguration.LINKIS_FLINK_TASK_MANAGER_CPU_CORES.getValue * containers + 1 - val resource = new DriverAndYarnResource( - new LoadInstanceResource(OverloadUtils.getProcessMaxMemory, - LINKIS_FLINK_CLIENT_CORES, - 1), - new YarnResource(yarnMemory, yarnCores, 0, queue) - ) - val engineResource = new CommonNodeResource - engineResource.setUsedResource(resource) - engineResource.setResourceType(ResourceUtils.getResourceTypeByResource(resource)) - engineResource - } - - def supportCallBackLogs(): Boolean = true -} - -object FlinkJobExecutor { - - import scala.collection.JavaConversions._ - - def writeResultSet(resultSet: ResultSet, resultSetWriter: ResultSetWriter[_ <: MetaData, _ <: Record]): Unit = { - val columns = resultSet.getColumns.map(columnInfo => Column(columnInfo.getName, DataType.toDataType(columnInfo.getType), null)).toArray - resultSetWriter.addMetaData(new TableMetaData(columns)) - resultSet.getData match { - case data: util.List[Row] => data.foreach { row => - val record = (0 until row.getArity).map(row.getField).map(FlinkValueFormatUtil.formatValue).toArray - resultSetWriter.addRecord(new TableRecord(record)) - } - case _ => - } - } - - def writeAndSendResultSet(resultSet: ResultSet, engineExecutionContext: EngineExecutionContext): Unit = { - val resultSetWriter = engineExecutionContext.createResultSetWriter(ResultSetFactory.TABLE_TYPE) - Utils.tryFinally{ - writeResultSet(resultSet, resultSetWriter) - engineExecutionContext.sendResultSet(resultSetWriter) - } (IOUtils.closeQuietly(resultSetWriter)) - } - -} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkSQLJobExecutor.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkSQLJobExecutor.scala deleted file mode 100644 index 1f87ff44a2..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/executor/FlinkSQLJobExecutor.scala +++ /dev/null @@ -1,226 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.executor - -import java.io.Closeable -import java.util -import java.util.concurrent.TimeUnit - -import com.webank.wedatasphere.linkis.common.utils.{ByteTimeUtils, Logging, Utils} -import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.{ComputationExecutor, EngineExecutionContext} -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.result.ResultKind -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.sql.operation.{AbstractJobOperation, JobOperation, OperationFactory} -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.utils.SqlCommandParser -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.SqlParseException -import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.RowsType.RowsType -import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.{FlinkStatusListener, FlinkStreamingResultSetListener} -import com.webank.wedatasphere.linkis.manager.label.entity.cluster.EnvLabel -import com.webank.wedatasphere.linkis.protocol.engine.JobProgressInfo -import com.webank.wedatasphere.linkis.scheduler.executer.{ErrorExecuteResponse, ExecuteResponse, SuccessExecuteResponse} -import com.webank.wedatasphere.linkis.storage.resultset.ResultSetFactory -import org.apache.calcite.rel.metadata.{JaninoRelMetadataProvider, RelMetadataQueryBase} -import org.apache.flink.api.common.JobStatus._ -import org.apache.flink.table.planner.plan.metadata.FlinkDefaultRelMetadataProvider - -import scala.collection.JavaConversions._ - -/** - * - */ -class FlinkSQLJobExecutor(id: Long, - override protected val flinkEngineConnContext: FlinkEngineConnContext) extends ComputationExecutor with FlinkJobExecutor { - - private var operation: JobOperation = _ - - override def executeLine(engineExecutionContext: EngineExecutionContext, code: String): ExecuteResponse = { - val callOpt = SqlCommandParser.parse(code.trim, true) - val callSQL = if (!callOpt.isPresent) throw new SqlParseException("Unknown statement: " + code) - else callOpt.get - RelMetadataQueryBase.THREAD_PROVIDERS.set(JaninoRelMetadataProvider.of(FlinkDefaultRelMetadataProvider.INSTANCE)) - val operation = OperationFactory.createOperation(callSQL, flinkEngineConnContext) - operation match { - case jobOperation: JobOperation => - this.operation = jobOperation - jobOperation.addFlinkListener(new FlinkSQLStatusListener(jobOperation, engineExecutionContext)) - if(getEnvLabel.getEnvType == EnvLabel.DEV && callSQL.command == SqlCommandParser.SqlCommand.SELECT) { - jobOperation.addFlinkListener(new FlinkSQLStreamingResultSetListener(jobOperation, engineExecutionContext)) - val properties: util.Map[String, String] = engineExecutionContext.getProperties.map { - case (k, v: String) => (k, v) - case (k, v) if v != null => (k, v.toString) - case (k, _) => (k, null) - } - jobOperation.addFlinkListener(new DevFlinkSQLStreamingListener(jobOperation, properties)) - } - case _ => - } - val resultSet = operation.execute - resultSet.getResultKind match { - case ResultKind.SUCCESS => - new SuccessExecuteResponse - case ResultKind.SUCCESS_WITH_CONTENT if !operation.isInstanceOf[JobOperation] => - FlinkJobExecutor.writeAndSendResultSet(resultSet, engineExecutionContext) - new SuccessExecuteResponse - case _ => - operation match { - case jobOperation: AbstractJobOperation => - val jobInfo = jobOperation.transformToJobInfo(resultSet) - setJobID(jobInfo.getJobId.toHexString) - setApplicationId(jobInfo.getApplicationId) - setApplicationURL(jobInfo.getWebInterfaceUrl) - setYarnMode("client") - jobOperation.getFlinkStatusListeners.get(0) match { - case listener: FlinkSQLStatusListener => listener.waitForCompleted() - case _ => - } - case jobOperation: JobOperation => - jobOperation.getFlinkListeners.find(_.isInstanceOf[FlinkSQLStatusListener]).foreach { case listener: FlinkSQLStatusListener => - listener.waitForCompleted() - } - } - } - new SuccessExecuteResponse - } - - override def executeCompletely(engineExecutorContext: EngineExecutionContext, code: String, completedLine: String): ExecuteResponse = { - val newcode = completedLine + code - info("newcode is " + newcode) - executeLine(engineExecutorContext, newcode) - } - - //TODO wait for completed. - override def progress(): Float = if(operation == null) 0 else operation.getJobStatus match { - case jobState if jobState.isGloballyTerminalState => 1 - case RUNNING => 0.5f - case _ => 0 - } - - override def getProgressInfo: Array[JobProgressInfo] = Array.empty - - private var envLabel: EnvLabel = _ - - def getEnvLabel: EnvLabel = { - if(envLabel == null) getExecutorLabels().foreach{ - case l: EnvLabel => envLabel = l - case _ => - } - envLabel - } - - override def getId: String = "flinkSQL_"+ id - - override def close(): Unit = { - if(operation != null) { - operation.cancelJob() - } - flinkEngineConnContext.getExecutionContext.createClusterDescriptor().close() - flinkEngineConnContext.getExecutionContext.getClusterClientFactory.close() - super.close() - } -} - -class FlinkSQLStatusListener(jobOperation: JobOperation, engineExecutionContext: EngineExecutionContext) extends FlinkStatusListener { - - private var resp: ExecuteResponse = _ - private val startTime = System.currentTimeMillis - - override def onSuccess(rows: Int, rowsType: RowsType): Unit = { - engineExecutionContext.appendStdout(s"Time taken: ${ByteTimeUtils.msDurationToString(System.currentTimeMillis - startTime)}, $rowsType $rows row(s).") - Utils.tryCatch{ - FlinkJobExecutor.writeAndSendResultSet(jobOperation.getJobResult.get(), engineExecutionContext) - resp = new SuccessExecuteResponse - }{ e => - resp = ErrorExecuteResponse("Fail to run statement",e) - } - synchronized(notify()) - } - - override def onFailed(message: String, t: Throwable): Unit = { - resp = ErrorExecuteResponse(message, t) - synchronized(notify()) - } - - def getResponse: ExecuteResponse = resp - - def waitForCompleted(maxWaitTime: Long): Unit = synchronized { - if(maxWaitTime < 0) wait() else wait(maxWaitTime) - } - - def waitForCompleted(): Unit = waitForCompleted(-1) -} - -class FlinkSQLStreamingResultSetListener(jobOperation: JobOperation, - engineExecutionContext: EngineExecutionContext) - extends FlinkStreamingResultSetListener with Closeable with Logging { - - private val resultSetWriter = engineExecutionContext.createResultSetWriter(ResultSetFactory.TABLE_TYPE) - - override def onResultSetPulled(rows: Int): Unit = { - info(s"$rows resultSets has pulled.") - FlinkJobExecutor.writeResultSet(jobOperation.getJobResult.get(), resultSetWriter) - } - - override def close(): Unit = engineExecutionContext.sendResultSet(resultSetWriter) -} - -class DevFlinkSQLStreamingListener(jobOperation: JobOperation, - maxWrittenLines: Int, - maxWaitForResultTime: Long) extends FlinkStreamingResultSetListener with Logging { - - def this(jobOperation: JobOperation) = - this(jobOperation, - FlinkEnvConfiguration.FLINK_SQL_DEV_SELECT_MAX_LINES.getValue, - FlinkEnvConfiguration.FLINK_SQL_DEV_RESULT_MAX_WAIT_TIME.getValue.toLong - ) - - def this(jobOperation: JobOperation, properties: util.Map[String, String]) = - this(jobOperation, - FlinkEnvConfiguration.FLINK_SQL_DEV_SELECT_MAX_LINES.getValue(properties), - FlinkEnvConfiguration.FLINK_SQL_DEV_RESULT_MAX_WAIT_TIME.getValue(properties).toLong - ) - - private var lastPulledTime = System.currentTimeMillis - private var writtenLines = 0 - - override def onResultSetPulled(rows: Int): Unit = { - lastPulledTime = System.currentTimeMillis - writtenLines += rows - if(writtenLines >= maxWrittenLines) { - warn(s"The returned resultSet reached max lines $writtenLines, now kill the job automatic. Notice: only the dev environment will touch off the automatic kill.") - stopJobOperation() - } - } - - private val future = Utils.defaultScheduler.scheduleAtFixedRate(new Runnable { - override def run(): Unit = if(System.currentTimeMillis - lastPulledTime > maxWaitForResultTime) { - warn(s"Job killed since reached the max time ${ByteTimeUtils.msDurationToString(maxWaitForResultTime)} of waiting for resultSet. Notice: only the dev environment will touch off the automatic kill.") - stopJobOperation() - } - }, maxWaitForResultTime, maxWaitForResultTime, TimeUnit.MILLISECONDS) - - def stopJobOperation(): Unit = { - Utils.tryFinally(jobOperation.cancelJob()) { - jobOperation.getFlinkListeners.foreach { - case listener: FlinkStreamingResultSetListener with Closeable => - listener.close() - } - future.cancel(false) - } - } -} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkApplicationExecutorFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkApplicationExecutorFactory.scala deleted file mode 100644 index ce8403bbd8..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkApplicationExecutorFactory.scala +++ /dev/null @@ -1,44 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.factory - -import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn -import com.webank.wedatasphere.linkis.engineconn.once.executor.OnceExecutor -import com.webank.wedatasphere.linkis.engineconn.once.executor.creation.OnceExecutorFactory -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext -import com.webank.wedatasphere.linkis.engineconnplugin.flink.executor.FlinkApplicationJobExecutor -import com.webank.wedatasphere.linkis.manager.label.entity.Label -import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType -import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType - -/** - * - */ -class FlinkApplicationExecutorFactory extends OnceExecutorFactory { - - override protected def newExecutor(id: Int, - engineCreationContext: EngineCreationContext, - engineConn: EngineConn, - labels: Array[Label[_]]): OnceExecutor = engineConn.getEngineConnSession match { - case context: FlinkEngineConnContext => - new FlinkApplicationJobExecutor(id, context) - } - - override protected def getRunType: RunType = RunType.JAR -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkEngineConnFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkEngineConnFactory.scala deleted file mode 100644 index 712ddb79ff..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkEngineConnFactory.scala +++ /dev/null @@ -1,186 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.factory - -import java.util -import java.util.Collections - -import com.google.common.collect.Lists -import com.webank.wedatasphere.linkis.common.utils.Logging -import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.Environment -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.config.entries.ExecutionEntry -import com.webank.wedatasphere.linkis.engineconnplugin.flink.client.context.ExecutionContext -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration._ -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkResourceConfiguration._ -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.{EnvironmentContext, FlinkEngineConnContext} -import com.webank.wedatasphere.linkis.engineconnplugin.flink.exception.FlinkInitFailedException -import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.EnvConfiguration -import com.webank.wedatasphere.linkis.manager.engineplugin.common.creation.{ExecutorFactory, MultiExecutorEngineConnFactory} -import com.webank.wedatasphere.linkis.manager.label.entity.Label -import com.webank.wedatasphere.linkis.manager.label.entity.engine.EngineType.EngineType -import com.webank.wedatasphere.linkis.manager.label.entity.engine._ -import org.apache.commons.lang.StringUtils -import org.apache.flink.configuration._ -import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings -import org.apache.flink.yarn.configuration.{YarnConfigOptions, YarnDeploymentTarget} - -import scala.collection.convert.decorateAsScala._ - -/** - * - */ -class FlinkEngineConnFactory extends MultiExecutorEngineConnFactory with Logging { - - override protected def createEngineConnSession(engineCreationContext: EngineCreationContext): Any = { - val options = engineCreationContext.getOptions - val environmentContext = createEnvironmentContext(engineCreationContext) - val flinkEngineConnContext = createFlinkEngineConnContext(environmentContext) - val executionContext = createExecutionContext(options, environmentContext) - flinkEngineConnContext.setExecutionContext(executionContext) - flinkEngineConnContext - } - - protected def createEnvironmentContext(engineCreationContext: EngineCreationContext): EnvironmentContext = { - val options = engineCreationContext.getOptions - val defaultEnv = Environment.parse(this.getClass.getClassLoader.getResource("flink-sql-defaults.yaml")) - val hadoopConfDir = EnvConfiguration.HADOOP_CONF_DIR.getValue(options) - val flinkHome = FLINK_HOME.getValue(options) - val flinkConfDir = FLINK_CONF_DIR.getValue(options) - val flinkLibRemotePath = FLINK_LIB_REMOTE_PATH.getValue(options) - val flinkDistJarPath = FLINK_DIST_JAR_PATH.getValue(options) - val providedLibDirsArray = FLINK_LIB_LOCAL_PATH.getValue(options).split(",") - val shipDirsArray = FLINK_SHIP_DIRECTORIES.getValue(options).split(",") - val context = new EnvironmentContext(defaultEnv, new Configuration, hadoopConfDir, flinkConfDir, flinkHome, - flinkDistJarPath, flinkLibRemotePath, providedLibDirsArray, shipDirsArray, null) - //第一步: 环境级别配置 - val jobName = options.getOrDefault("flink.app.name", "EngineConn-Flink") - val yarnQueue = LINKIS_QUEUE_NAME.getValue(options) - val parallelism = FLINK_APP_DEFAULT_PARALLELISM.getValue(options) - val jobManagerMemory = LINKIS_FLINK_JOB_MANAGER_MEMORY.getValue(options) + "G" - val taskManagerMemory = LINKIS_FLINK_TASK_MANAGER_MEMORY.getValue(options) + "G" - val numberOfTaskSlots = LINKIS_FLINK_TASK_SLOTS.getValue(options) - info(s"Use yarn queue $yarnQueue, and set parallelism = $parallelism, jobManagerMemory = $jobManagerMemory G, taskManagerMemory = $taskManagerMemory G, numberOfTaskSlots = $numberOfTaskSlots.") - //第二步: 应用级别配置 - //构建应用配置 - val flinkConfig = context.getFlinkConfig - //构建依赖jar包环境 - val flinkUserLibRemotePath = FLINK_USER_LIB_REMOTE_PATH.getValue(options).split(",") - val providedLibDirList = Lists.newArrayList(flinkUserLibRemotePath.filter(StringUtils.isNotBlank): _*) - val flinkUserRemotePathList = Lists.newArrayList(flinkLibRemotePath.split(",").filter(StringUtils.isNotBlank): _*) - if (flinkUserRemotePathList != null && flinkUserRemotePathList.size() > 0) providedLibDirList.addAll(flinkUserRemotePathList) - //if(StringUtils.isNotBlank(flinkLibRemotePath)) providedLibDirList.add(flinkLibRemotePath) - flinkConfig.set(YarnConfigOptions.PROVIDED_LIB_DIRS, providedLibDirList) - //构建依赖jar包环境 - flinkConfig.set(YarnConfigOptions.SHIP_DIRECTORIES, context.getShipDirs) - //yarn 作业名称 - flinkConfig.set(YarnConfigOptions.APPLICATION_NAME, jobName) - //yarn queue - flinkConfig.set(YarnConfigOptions.APPLICATION_QUEUE, yarnQueue) - //设置:资源/并发度 - flinkConfig.setInteger(CoreOptions.DEFAULT_PARALLELISM, parallelism) - flinkConfig.set(JobManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse(jobManagerMemory)) - flinkConfig.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse(taskManagerMemory)) - flinkConfig.setInteger(TaskManagerOptions.NUM_TASK_SLOTS, numberOfTaskSlots) - //设置 savePoint - val savePointPath = FLINK_SAVE_POINT_PATH.getValue(options) - if (StringUtils.isNotBlank(savePointPath)) { - val allowNonRestoredState = FLINK_APP_ALLOW_NON_RESTORED_STATUS.getValue(options).toBoolean - val savepointRestoreSettings = SavepointRestoreSettings.forPath(savePointPath, allowNonRestoredState) - SavepointRestoreSettings.toConfiguration(savepointRestoreSettings, flinkConfig) - } - //设置:用户入口jar:可以远程,只能设置1个jar - val flinkMainClassJar = FLINK_APPLICATION_MAIN_CLASS_JAR.getValue(options) - if(StringUtils.isNotBlank(flinkMainClassJar)) { - info(s"Ready to use $flinkMainClassJar as main class jar to submit application to Yarn.") - flinkConfig.set(PipelineOptions.JARS, Collections.singletonList(flinkMainClassJar)) - flinkConfig.set(DeploymentOptions.TARGET, YarnDeploymentTarget.APPLICATION.getName) - context.setDeploymentTarget(YarnDeploymentTarget.APPLICATION) - addApplicationLabels(engineCreationContext) - } else if(isOnceEngineConn(engineCreationContext.getLabels())) { - flinkConfig.set(DeploymentOptions.TARGET, YarnDeploymentTarget.PER_JOB.getName) - } else { - flinkConfig.set(DeploymentOptions.TARGET, YarnDeploymentTarget.SESSION.getName) - } - context - } - - protected def isOnceEngineConn(labels: util.List[Label[_]]): Boolean = { - val engineConnModeLabel = getEngineConnModeLabel(labels) - engineConnModeLabel != null && (EngineConnMode.toEngineConnMode(engineConnModeLabel.getEngineConnMode) match { - case EngineConnMode.Once | EngineConnMode.Once_With_Cluster => true - case _ => false - }) - } - - private def addApplicationLabels(engineCreationContext: EngineCreationContext): Unit = { - val labels = engineCreationContext.getLabels().asScala - if(!labels.exists(_.isInstanceOf[CodeLanguageLabel])) { - val codeLanguageLabel = new CodeLanguageLabel - codeLanguageLabel.setCodeType(RunType.JAR.toString) - engineCreationContext.getLabels().add(codeLanguageLabel) - } - if(!labels.exists(_.isInstanceOf[EngineConnModeLabel])) { - val engineConnModeLabel = new EngineConnModeLabel - engineConnModeLabel.setEngineConnMode(EngineConnMode.Once.toString) - engineCreationContext.getLabels().add(engineConnModeLabel) - } - } - - def createExecutionContext(options: util.Map[String, String], environmentContext: EnvironmentContext): ExecutionContext = { - val environment = environmentContext.getDeploymentTarget match { - case YarnDeploymentTarget.PER_JOB | YarnDeploymentTarget.SESSION => - val planner = FlinkEnvConfiguration.FLINK_SQL_PLANNER.getValue(options) - if (!ExecutionEntry.AVAILABLE_PLANNERS.contains(planner.toLowerCase)) - throw new FlinkInitFailedException("Planner must be one of these: " + String.join(", ", ExecutionEntry.AVAILABLE_PLANNERS)) - val executionType = FlinkEnvConfiguration.FLINK_SQL_EXECUTION_TYPE.getValue(options) - if (!ExecutionEntry.AVAILABLE_EXECUTION_TYPES.contains(executionType.toLowerCase)) - throw new FlinkInitFailedException("Execution type must be one of these: " + String.join(", ", ExecutionEntry.AVAILABLE_EXECUTION_TYPES)) - val properties = new util.HashMap[String, String] - properties.put(Environment.EXECUTION_ENTRY + "." + ExecutionEntry.EXECUTION_PLANNER, planner) - properties.put(Environment.EXECUTION_ENTRY + "." + ExecutionEntry.EXECUTION_TYPE, executionType) - if (executionType.equalsIgnoreCase(ExecutionEntry.EXECUTION_TYPE_VALUE_BATCH)) { - // for batch mode we ensure that results are provided in materialized form - properties.put(Environment.EXECUTION_ENTRY + "." + ExecutionEntry.EXECUTION_RESULT_MODE, ExecutionEntry.EXECUTION_RESULT_MODE_VALUE_TABLE) - } else { - // for streaming mode we ensure that results are provided in changelog form - properties.put(Environment.EXECUTION_ENTRY + "." + ExecutionEntry.EXECUTION_RESULT_MODE, ExecutionEntry.EXECUTION_RESULT_MODE_VALUE_CHANGELOG) - } - Environment.enrich(environmentContext.getDefaultEnv, properties, Collections.emptyMap()) - case YarnDeploymentTarget.APPLICATION => null - case t => - error(s"Not supported YarnDeploymentTarget ${t.getName}.") - throw new FlinkInitFailedException(s"Not supported YarnDeploymentTarget ${t.getName}.") - } - ExecutionContext.builder(environmentContext.getDefaultEnv, environment, environmentContext.getDependencies, - environmentContext.getFlinkConfig, environmentContext.getClusterClientServiceLoader).build() - } - - protected def createFlinkEngineConnContext(environmentContext: EnvironmentContext): FlinkEngineConnContext = - new FlinkEngineConnContext(environmentContext) - - override protected def getDefaultExecutorFactoryClass: Class[_ <: ExecutorFactory] = classOf[FlinkSQLExecutorFactory] - - override protected def getEngineConnType: EngineType = EngineType.FLINK - - private val executorFactoryArray = Array[ExecutorFactory](new FlinkSQLExecutorFactory, new FlinkApplicationExecutorFactory) - - - override def getExecutorFactories: Array[ExecutorFactory] = executorFactoryArray -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkSQLExecutorFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkSQLExecutorFactory.scala deleted file mode 100644 index 5f0ef8306d..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/factory/FlinkSQLExecutorFactory.scala +++ /dev/null @@ -1,63 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.factory - -import com.webank.wedatasphere.linkis.engineconn.common.creation.EngineCreationContext -import com.webank.wedatasphere.linkis.engineconn.common.engineconn.EngineConn -import com.webank.wedatasphere.linkis.engineconn.computation.executor.creation.ComputationExecutorFactory -import com.webank.wedatasphere.linkis.engineconn.computation.executor.execute.ComputationExecutor -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration -import com.webank.wedatasphere.linkis.engineconnplugin.flink.context.FlinkEngineConnContext -import com.webank.wedatasphere.linkis.engineconnplugin.flink.executor.FlinkSQLJobExecutor -import com.webank.wedatasphere.linkis.manager.label.entity.Label -import com.webank.wedatasphere.linkis.manager.label.entity.cluster.EnvLabel -import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType -import com.webank.wedatasphere.linkis.manager.label.entity.engine.RunType.RunType - -import scala.collection.JavaConversions._ - -/** - * - */ -class FlinkSQLExecutorFactory extends ComputationExecutorFactory { - - override protected def newExecutor(id: Int, - engineCreationContext: EngineCreationContext, - engineConn: EngineConn, - labels: Array[Label[_]]): ComputationExecutor = engineConn.getEngineConnSession match { - case context: FlinkEngineConnContext => - val executor = new FlinkSQLJobExecutor(id, context) - if(!labels.exists(_.isInstanceOf[EnvLabel])) { - executor.getExecutorLabels().add(getEnvLabel(engineCreationContext)) - } - if(executor.getEnvLabel.getEnvType == EnvLabel.DEV) { - context.getEnvironmentContext.getDefaultEnv - .setExecution(Map("max-table-result-rows" -> FlinkEnvConfiguration.FLINK_SQL_DEV_SELECT_MAX_LINES.getValue.asInstanceOf[Object])) - } - executor - } - - protected def getEnvLabel(engineCreationContext: EngineCreationContext): EnvLabel = { - val envType = FlinkEnvConfiguration.FLINK_ENV_TYPE.getValue(engineCreationContext.getOptions) - val envLabel = new EnvLabel - envLabel.setEnvType(envType) - envLabel - } - - override protected def getRunType: RunType = RunType.SQL -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/launch/FlinkEngineConnLaunchBuilder.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/launch/FlinkEngineConnLaunchBuilder.scala deleted file mode 100644 index d5dc3794cd..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/launch/FlinkEngineConnLaunchBuilder.scala +++ /dev/null @@ -1,42 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.launch - -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkEnvConfiguration._ -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkResourceConfiguration -import com.webank.wedatasphere.linkis.manager.engineplugin.common.conf.EnvConfiguration -import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.entity.EngineConnBuildRequest -import com.webank.wedatasphere.linkis.manager.engineplugin.common.launch.process.JavaProcessEngineConnLaunchBuilder - -/** - * - */ -class FlinkEngineConnLaunchBuilder extends JavaProcessEngineConnLaunchBuilder { - - override protected def getCommands(implicit engineConnBuildRequest: EngineConnBuildRequest): Array[String] = { - val properties = engineConnBuildRequest.engineConnCreationDesc.properties - properties.put(EnvConfiguration.ENGINE_CONN_MEMORY.key, FlinkResourceConfiguration.LINKIS_FLINK_CLIENT_MEMORY.getValue(properties) + "G") - super.getCommands - } - - override protected def getNecessaryEnvironment(implicit engineConnBuildRequest: EngineConnBuildRequest): Array[String] = - Array(FLINK_HOME_ENV, FLINK_CONF_DIR_ENV) ++: super.getNecessaryEnvironment - - override protected def ifAddHiveConfigPath: Boolean = true - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListener.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListener.scala deleted file mode 100644 index f17bf960b1..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListener.scala +++ /dev/null @@ -1,47 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.listener - -import com.webank.wedatasphere.linkis.common.listener.{Event, EventListener} -import com.webank.wedatasphere.linkis.engineconnplugin.flink.listener.RowsType.RowsType - -/** - * - */ -trait FlinkListener extends EventListener { - override def onEventError(event: Event, t: Throwable): Unit = {} -} - -trait FlinkStatusListener extends FlinkListener { - - def onSuccess(rows: Int, rowsType: RowsType): Unit - - def onFailed(message: String, t: Throwable): Unit - -} - -object RowsType extends Enumeration { - type RowsType = Value - val Fetched, Affected = Value -} - -trait FlinkStreamingResultSetListener extends FlinkListener { - - def onResultSetPulled(rows: Int): Unit - -} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListenerGroup.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListenerGroup.scala deleted file mode 100644 index e7eb3c38df..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/listener/FlinkListenerGroup.scala +++ /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 com.webank.wedatasphere.linkis.engineconnplugin.flink.listener - -import java.util - -import scala.collection.JavaConversions._ - -/** - * - */ -trait FlinkListenerGroup { - - def addFlinkListener(flinkListener: FlinkListener): Unit - - def getFlinkListeners: util.List[FlinkListener] - - def setFlinkListeners(flinkListeners: util.List[FlinkListener]): Unit - -} - -abstract class FlinkListenerGroupImpl extends FlinkListenerGroup { - - private var flinkListeners: util.List[FlinkListener] = _ - - override def addFlinkListener(flinkListener: FlinkListener): Unit = { - if (flinkListeners == null) flinkListeners = new util.ArrayList[FlinkListener] - flinkListeners.add(flinkListener) - } - - override def getFlinkListeners: util.List[FlinkListener] = flinkListeners - - override def setFlinkListeners(flinkListeners: util.List[FlinkListener]): Unit = { - this.flinkListeners = flinkListeners - } - - private def getFlinkListeners[T <: FlinkListener](clazz: Class[T]): util.List[T] = flinkListeners match { - case listeners: util.List[FlinkListener] => - listeners.filter(_.isInstanceOf[T]).map(_.asInstanceOf[T]) - case _ => new util.ArrayList[T] - } - - def getFlinkStatusListeners: util.List[FlinkStatusListener] = getFlinkListeners(classOf[FlinkStatusListener]) - - def getFlinkStreamingResultSetListeners: util.List[FlinkStreamingResultSetListener] = getFlinkListeners(classOf[FlinkStreamingResultSetListener]) - -} \ No newline at end of file diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/resource/FlinkEngineConnResourceFactory.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/resource/FlinkEngineConnResourceFactory.scala deleted file mode 100644 index d4ea352907..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/resource/FlinkEngineConnResourceFactory.scala +++ /dev/null @@ -1,47 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.resource - -import com.webank.wedatasphere.linkis.common.utils.ByteTimeUtils -import com.webank.wedatasphere.linkis.engineconnplugin.flink.config.FlinkResourceConfiguration._ -import com.webank.wedatasphere.linkis.manager.common.entity.resource.{DriverAndYarnResource, LoadInstanceResource, Resource, YarnResource} -import com.webank.wedatasphere.linkis.manager.engineplugin.common.resource.AbstractEngineResourceFactory - -/** - * - */ -class FlinkEngineConnResourceFactory extends AbstractEngineResourceFactory { - - override def getRequestResource(properties: java.util.Map[String, String]): Resource = { - val containers = if(properties.containsKey(LINKIS_FLINK_CONTAINERS)) { - val containers = LINKIS_FLINK_CONTAINERS.getValue(properties) - properties.put(FLINK_APP_DEFAULT_PARALLELISM.key, String.valueOf(containers * LINKIS_FLINK_TASK_SLOTS.getValue(properties))) - containers - } else math.round(FLINK_APP_DEFAULT_PARALLELISM.getValue(properties) * 1.0f / LINKIS_FLINK_TASK_SLOTS.getValue(properties)) - val yarnMemory = ByteTimeUtils.byteStringAsBytes(LINKIS_FLINK_TASK_MANAGER_MEMORY.getValue(properties) * containers + "G") + - ByteTimeUtils.byteStringAsBytes(LINKIS_FLINK_JOB_MANAGER_MEMORY.getValue(properties) + "G") - val yarnCores = LINKIS_FLINK_TASK_MANAGER_CPU_CORES.getValue(properties) * containers + 1 - new DriverAndYarnResource( - new LoadInstanceResource(ByteTimeUtils.byteStringAsBytes(LINKIS_FLINK_CLIENT_MEMORY.getValue(properties) + "G"), - LINKIS_FLINK_CLIENT_CORES, - 1), - new YarnResource(yarnMemory, yarnCores, 0, LINKIS_QUEUE_NAME.getValue(properties)) - ) - } - -} diff --git a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/FlinkValueFormatUtil.scala b/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/FlinkValueFormatUtil.scala deleted file mode 100644 index 91e5805c5a..0000000000 --- a/linkis-engineconn-plugins/engineconn-plugins/flink/src/main/scala/com/webank/wedatasphere/linkis/engineconnplugin/flink/util/FlinkValueFormatUtil.scala +++ /dev/null @@ -1,42 +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 com.webank.wedatasphere.linkis.engineconnplugin.flink.util - -import java.text.NumberFormat - -import com.webank.wedatasphere.linkis.common.conf.CommonVars - -/** - * - */ -object FlinkValueFormatUtil { - - val FLINK_NF_FRACTION_LENGTH = CommonVars("wds.linkis.engine.flink.fraction.length", 30) - - private val nf = NumberFormat.getInstance() - nf.setGroupingUsed(false) - nf.setMaximumFractionDigits(FLINK_NF_FRACTION_LENGTH.getValue) - - def formatValue(value: Any): Any = value match { - case value: String => value.replaceAll("\n|\t", " ") - case value: Double => nf.format(value) - case value: Any => value.toString - case _ => null - } - -} diff --git a/linkis-spring-cloud-services/linkis-service-gateway/linkis-spring-cloud-gateway/src/main/java/com/webank/wedatasphere/linkis/gateway/springcloud/LinkisGatewayApplication.java b/linkis-spring-cloud-services/linkis-service-gateway/linkis-spring-cloud-gateway/src/main/java/com/webank/wedatasphere/linkis/gateway/springcloud/LinkisGatewayApplication.java index c37ce9771f..23984cf96c 100644 --- a/linkis-spring-cloud-services/linkis-service-gateway/linkis-spring-cloud-gateway/src/main/java/com/webank/wedatasphere/linkis/gateway/springcloud/LinkisGatewayApplication.java +++ b/linkis-spring-cloud-services/linkis-service-gateway/linkis-spring-cloud-gateway/src/main/java/com/webank/wedatasphere/linkis/gateway/springcloud/LinkisGatewayApplication.java @@ -28,4 +28,4 @@ public static void main(String[] args) throws ReflectiveOperationException { logger.info("Start to running LinkisGatewayApplication"); DataWorkCloudApplication.main(args); } -} +} \ No newline at end of file From b0a0ddc34b6a20055cd5a1a3cc80139eeff5f7f1 Mon Sep 17 00:00:00 2001 From: Davidhua1996 Date: Mon, 7 Jun 2021 17:14:56 +0800 Subject: [PATCH 3/3] Remove the useless notes in code --- .../linkis/engineplugin/spark/metadata/MetaDataInfoTool.scala | 2 -- .../linkis/engineplugin/spark/metadata/SparkHiveObject.scala | 1 - 2 files changed, 3 deletions(-) diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/MetaDataInfoTool.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/MetaDataInfoTool.scala index 9488c753bb..600421c938 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/MetaDataInfoTool.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/MetaDataInfoTool.scala @@ -22,8 +22,6 @@ import org.apache.spark.sql.{DataFrame, Dataset, SQLContext, SparkLogicalPlanHel /** * * Description: - * 1.先执行sql,获取dataframe - * 2.通过dataframe的queryExecution对logicPlan进行分析 */ class MetaDataInfoTool extends Logging{ def getMetaDataInfo(sqlContext:SQLContext, sql:String, dataFrame:DataFrame):String = { diff --git a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkHiveObject.scala b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkHiveObject.scala index 82aef90940..df66839b4e 100644 --- a/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkHiveObject.scala +++ b/linkis-engineconn-plugins/engineconn-plugins/spark/src/main/scala/com/webank/wedatasphere/linkis/engineplugin/spark/metadata/SparkHiveObject.scala @@ -21,7 +21,6 @@ import com.webank.wedatasphere.linkis.cs.common.entity.metadata.CSColumn import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject /** - * created by cooperyang on 2020/8/26 * Description: */ case class SparkHiveObject(objectType: HivePrivilegeObject.HivePrivilegeObjectType, dbName: String, objectName: String,