From 260932c5f9031c70e49fa0b9abcf8016ab7097cd Mon Sep 17 00:00:00 2001 From: ZackYoung Date: Fri, 6 Sep 2024 22:47:30 +0800 Subject: [PATCH] [Feature][Flink] Added support for Flink 1.20 and updated dependencies on other Flink versions (#3786) --- .github/workflows/auto-realease-ci.yml | 24 +- .github/workflows/backend.yaml | 2 +- dinky-app/dinky-app-1.20/pom.xml | 76 + .../src/main/java/org/dinky/app/MainApp.java | 55 + dinky-app/pom.xml | 1 + .../dinky-catalog-mysql-1.20/pom.xml | 46 + .../flink/catalog/DinkyMysqlCatalog.java | 1153 +++++++++++ .../factory/DinkyMysqlCatalogFactory.java | 72 + .../DinkyMysqlCatalogFactoryOptions.java | 44 + .../org.apache.flink.table.factories.Factory | 16 + dinky-catalog/dinky-catalog-mysql/pom.xml | 1 + dinky-client/dinky-client-1.20/pom.xml | 63 + .../calcite/rel/metadata/RelColumnOrigin.java | 116 ++ .../rel/metadata/RelMdColumnOrigins.java | 534 +++++ .../org/apache/calcite/sql/SqlSelect.java | 355 ++++ .../client/program/PackagedProgramUtils.java | 262 +++ .../java/org/apache/flink/yarn/Utils.java | 773 +++++++ .../flink/yarn/YarnClusterDescriptor.java | 1801 +++++++++++++++++ .../AbstractCustomTableEnvironment.java | 123 ++ .../ClusterDescriptorAdapterImpl.java | 57 + .../executor/CustomTableEnvironmentImpl.java | 160 ++ .../dinky/executor/CustomTableResultImpl.java | 265 +++ .../DefaultStreamTableEnvironment.java | 167 ++ .../executor/DefaultTableEnvironment.java | 342 ++++ .../DefaultTableEnvironmentInternal.java | 112 + .../org/dinky/executor/ExtendedParser.java | 27 + .../org/dinky/executor/ParserWrapper.java | 70 + .../dinky/executor/StaticResultProvider.java | 119 ++ .../org/dinky/executor/TableSchemaField.java | 50 + .../org/dinky/gateway/FlinkSqlClient.java | 164 ++ .../dinky/operations/CustomNewParserImpl.java | 41 + .../operations/DinkyExecutableOperation.java | 51 + .../operations/DinkyOperationExecutor.java | 48 + .../org/dinky/operations/DinkyParser.java | 46 + .../main/java/org/dinky/utils/FlinkUtil.java | 80 + .../java/org/dinky/utils/FunctionVisitor.java | 50 + .../java/org/dinky/utils/LineageContext.java | 194 ++ .../org/dinky/utils/ObjectConvertUtil.java | 90 + dinky-client/pom.xml | 1 + dinky-flink/dinky-flink-1.14/pom.xml | 2 +- dinky-flink/dinky-flink-1.15/pom.xml | 4 +- dinky-flink/dinky-flink-1.16/pom.xml | 4 +- dinky-flink/dinky-flink-1.17/pom.xml | 17 +- dinky-flink/dinky-flink-1.18/pom.xml | 15 +- dinky-flink/dinky-flink-1.19/pom.xml | 19 +- dinky-flink/dinky-flink-1.20/pom.xml | 172 ++ dinky-flink/pom.xml | 1 + .../org/dinky/gateway/yarn/YarnGateway.java | 3 +- pom.xml | 23 +- 49 files changed, 7884 insertions(+), 27 deletions(-) create mode 100644 dinky-app/dinky-app-1.20/pom.xml create mode 100644 dinky-app/dinky-app-1.20/src/main/java/org/dinky/app/MainApp.java create mode 100644 dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/pom.xml create mode 100644 dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/DinkyMysqlCatalog.java create mode 100644 dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/factory/DinkyMysqlCatalogFactory.java create mode 100644 dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/factory/DinkyMysqlCatalogFactoryOptions.java create mode 100644 dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 dinky-client/dinky-client-1.20/pom.xml create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/sql/SqlSelect.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/yarn/Utils.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ClusterDescriptorAdapterImpl.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableResultImpl.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultStreamTableEnvironment.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultTableEnvironment.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultTableEnvironmentInternal.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ExtendedParser.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ParserWrapper.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/StaticResultProvider.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/TableSchemaField.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/gateway/FlinkSqlClient.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/CustomNewParserImpl.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyExecutableOperation.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyOperationExecutor.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyParser.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/FlinkUtil.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/FunctionVisitor.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/LineageContext.java create mode 100644 dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/ObjectConvertUtil.java create mode 100644 dinky-flink/dinky-flink-1.20/pom.xml diff --git a/.github/workflows/auto-realease-ci.yml b/.github/workflows/auto-realease-ci.yml index c1cc6ad27b..ac374e133e 100644 --- a/.github/workflows/auto-realease-ci.yml +++ b/.github/workflows/auto-realease-ci.yml @@ -108,6 +108,28 @@ jobs: -Pprod,flink-single-version,flink-1.18,maven-central,web,fast \ --no-snapshot-updates + - name: "Build Dinky ${{ inputs.version }} with Fink 1.19 Pre Release/Release Artifact" + id: build_dinky1_19 + run: | + ./mvnw -B clean package \ + -Dmaven.test.skip=false \ + -Dspotless.check.skip=true \ + -Denforcer.skip=false \ + -Dmaven.javadoc.skip=true \ + -Pprod,flink-single-version,flink-1.19,maven-central,web,fast \ + --no-snapshot-updates + + - name: "Build Dinky ${{ inputs.version }} with Fink 1.20 Pre Release/Release Artifact" + id: build_dinky1_20 + run: | + ./mvnw -B clean package \ + -Dmaven.test.skip=false \ + -Dspotless.check.skip=true \ + -Denforcer.skip=false \ + -Dmaven.javadoc.skip=true \ + -Pprod,flink-single-version,flink-1.20,maven-central,web,fast \ + --no-snapshot-updates + - name: Automatic Upload Releases to GitHub Releases Page uses: "marvinpinto/action-automatic-releases@latest" with: @@ -120,4 +142,4 @@ jobs: body: | ${{ inputs.release_notes }} files: | - ./build/dinky-release-*.tar.gz \ No newline at end of file + ./build/dinky-release-*.tar.gz diff --git a/.github/workflows/backend.yaml b/.github/workflows/backend.yaml index 95130efb42..263cef763e 100644 --- a/.github/workflows/backend.yaml +++ b/.github/workflows/backend.yaml @@ -89,7 +89,7 @@ jobs: fail-fast: true matrix: jdk: [8, 11] - flink: [1.14, 1.15, 1.16, 1.17, 1.18, 1.19] + flink: [1.14, 1.15, 1.16, 1.17, 1.18, 1.19, 1.20] timeout-minutes: 30 env: diff --git a/dinky-app/dinky-app-1.20/pom.xml b/dinky-app/dinky-app-1.20/pom.xml new file mode 100644 index 0000000000..8b69fc7600 --- /dev/null +++ b/dinky-app/dinky-app-1.20/pom.xml @@ -0,0 +1,76 @@ + + + 4.0.0 + + org.dinky + dinky-app + ${revision} + ../pom.xml + + dinky-app-1.20 + + jar + + Dinky : App 1.20 + + + org.dinky.app.MainApp + + + + + org.dinky + dinky-app-base + + + org.dinky + dinky-client-1.20 + + + org.dinky + dinky-flink-1.20 + ${scope.runtime} + + + + + + + src/main/resources + + *.properties + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + ${maven-assembly-plugin.version} + + + jar-with-dependencies + + + + + org.dinky.app.MainApp + + + ${project.parent.parent.basedir}/build/extends + + + + make-assembly + + single + + package + + + + + + diff --git a/dinky-app/dinky-app-1.20/src/main/java/org/dinky/app/MainApp.java b/dinky-app/dinky-app-1.20/src/main/java/org/dinky/app/MainApp.java new file mode 100644 index 0000000000..4b5fddac96 --- /dev/null +++ b/dinky-app/dinky-app-1.20/src/main/java/org/dinky/app/MainApp.java @@ -0,0 +1,55 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.app; + +import org.dinky.app.constant.AppParamConstant; +import org.dinky.app.db.DBUtil; +import org.dinky.app.flinksql.Submitter; +import org.dinky.data.app.AppParamConfig; +import org.dinky.utils.JsonUtils; + +import org.apache.flink.api.java.utils.ParameterTool; + +import java.util.Base64; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * MainApp + * + * @since 2022/11/05 + */ +public class MainApp { + + private static final Logger log = LoggerFactory.getLogger(Submitter.class); + + public static void main(String[] args) throws Exception { + log.info("=========================Start run dinky app job==============================="); + ParameterTool parameters = ParameterTool.fromArgs(args); + boolean isEncrypt = parameters.getBoolean(AppParamConstant.isEncrypt, true); + String config = parameters.get(AppParamConstant.config); + config = isEncrypt ? new String(Base64.getDecoder().decode(config)) : config; + AppParamConfig appConfig = JsonUtils.toJavaBean(config, AppParamConfig.class); + log.info("dinky app is Ready to run, config is {}", appConfig); + DBUtil.init(appConfig); + Submitter.submit(appConfig); + } +} diff --git a/dinky-app/pom.xml b/dinky-app/pom.xml index b3ceb07532..baef8fab4e 100644 --- a/dinky-app/pom.xml +++ b/dinky-app/pom.xml @@ -33,6 +33,7 @@ dinky-app-1.17 dinky-app-1.18 dinky-app-1.19 + dinky-app-1.20 diff --git a/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/pom.xml b/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/pom.xml new file mode 100644 index 0000000000..b87afd0495 --- /dev/null +++ b/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/pom.xml @@ -0,0 +1,46 @@ + + + 4.0.0 + + org.dinky + dinky-catalog-mysql + ${revision} + ../pom.xml + + dinky-catalog-mysql-1.20 + + jar + + Dinky : Catalog : Mysql 1.20 + + + + org.dinky + dinky-common + + + org.dinky + dinky-flink-1.20 + + + junit + junit + 4.13.2 + test + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + ${project.parent.parent.parent.basedir}/build/extends + + + + + diff --git a/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/DinkyMysqlCatalog.java b/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/DinkyMysqlCatalog.java new file mode 100644 index 0000000000..12da5b2767 --- /dev/null +++ b/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/DinkyMysqlCatalog.java @@ -0,0 +1,1153 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.flink.catalog; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; + +import org.dinky.flink.catalog.factory.DinkyMysqlCatalogFactoryOptions; + +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogFunctionImpl; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogView; +import org.apache.flink.table.catalog.FunctionLanguage; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogBaseTable; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.ResolvedCatalogView; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionAlreadyExistsException; +import org.apache.flink.table.catalog.exceptions.PartitionNotExistException; +import org.apache.flink.table.catalog.exceptions.PartitionSpecInvalidException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.exceptions.TablePartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.StringUtils; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * 自定义 catalog 检查connection done. 默认db,会被强制指定,不管输入的是什么,都会指定为 default_database + * 可以读取配置文件信息来获取数据库连接,而不是在sql语句中强制指定。 + */ +public class DinkyMysqlCatalog extends AbstractCatalog { + + private final Logger logger = LoggerFactory.getLogger(this.getClass()); + + public static final String MYSQL_DRIVER = "com.mysql.cj.jdbc.Driver"; + + public static final String DEFAULT_DATABASE = "default_database"; + + static { + try { + Class.forName(MYSQL_DRIVER); + } catch (ClassNotFoundException e) { + throw new CatalogException("未加载 mysql 驱动!", e); + } + } + + private static final String COMMENT = "comment"; + /** 判断是否发生过SQL异常,如果发生过,那么conn可能失效。要注意判断 */ + private boolean sqlExceptionHappened = false; + + /** 对象类型,例如 库、表、视图等 */ + protected static class ObjectType { + + /** 数据库 */ + public static final String DATABASE = "database"; + + /** 数据表 */ + public static final String TABLE = "TABLE"; + + /** 视图 */ + public static final String VIEW = "VIEW"; + } + + /** 对象类型,例如 库、表、视图等 */ + protected static class ColumnType { + + /** 物理字段 */ + public static final String PHYSICAL = "physical"; + + /** 计算字段 */ + public static final String COMPUTED = "computed"; + + /** 元数据字段 */ + public static final String METADATA = "metadata"; + + /** 水印 */ + public static final String WATERMARK = "watermark"; + } + + /** 数据库用户名 */ + private final String user; + /** 数据库密码 */ + private final String pwd; + /** 数据库连接 */ + private final String url; + + /** 默认database */ + private static final String defaultDatabase = "default_database"; + + /** + * 数据库用户名 + * + * @return 数据库用户名 + */ + public String getUser() { + return user; + } + + /** + * 数据库密码 + * + * @return 数据库密码 + */ + public String getPwd() { + return pwd; + } + + /** + * 数据库用户名 + * + * @return 数据库用户名 + */ + public String getUrl() { + return url; + } + + public DinkyMysqlCatalog(String name, String url, String user, String pwd) { + super(name, defaultDatabase); + this.url = url; + this.user = user; + this.pwd = pwd; + } + + public DinkyMysqlCatalog(String name) { + super(name, defaultDatabase); + this.url = DinkyMysqlCatalogFactoryOptions.URL.defaultValue(); + this.user = DinkyMysqlCatalogFactoryOptions.USERNAME.defaultValue(); + this.pwd = DinkyMysqlCatalogFactoryOptions.PASSWORD.defaultValue(); + } + + @Override + public void open() throws CatalogException { + // 验证连接是否有效 + // 获取默认db看看是否存在 + Integer defaultDbId = getDatabaseId(defaultDatabase); + if (defaultDbId == null) { + try { + createDatabase(defaultDatabase, new CatalogDatabaseImpl(new HashMap<>(), ""), true); + } catch (DatabaseAlreadyExistException a) { + logger.info("重复创建默认库"); + } + } + } + + @Override + public void close() throws CatalogException { + if (connection != null) { + try { + connection.close(); + connection = null; + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException("Fail to close connection.", e); + } + } + } + + private Connection connection; + + protected Connection getConnection() throws CatalogException { + try { + // todo: 包装一个方法用于获取连接,方便后续改造使用其他的连接生成。 + // Class.forName(MYSQL_DRIVER); + if (connection == null) { + connection = DriverManager.getConnection(url, user, pwd); + } + if (sqlExceptionHappened) { + sqlExceptionHappened = false; + if (!connection.isValid(10)) { + connection.close(); + } + if (connection.isClosed()) { + connection = null; + return getConnection(); + } + connection = null; + return getConnection(); + } + + return connection; + } catch (Exception e) { + throw new CatalogException("Fail to get connection.", e); + } + } + + @Override + public List listDatabases() throws CatalogException { + List myDatabases = new ArrayList<>(); + String querySql = "SELECT database_name FROM metadata_database"; + Connection conn = getConnection(); + try (PreparedStatement ps = conn.prepareStatement(querySql)) { + + ResultSet rs = ps.executeQuery(); + while (rs.next()) { + String dbName = rs.getString(1); + myDatabases.add(dbName); + } + + return myDatabases; + } catch (Exception e) { + throw new CatalogException(String.format("Failed listing database in catalog %s", getName()), e); + } + } + + @Override + public CatalogDatabase getDatabase(String databaseName) throws DatabaseNotExistException, CatalogException { + String querySql = "SELECT id, database_name,description " + " FROM metadata_database where database_name=?"; + Connection conn = getConnection(); + try (PreparedStatement ps = conn.prepareStatement(querySql)) { + ps.setString(1, databaseName); + ResultSet rs = ps.executeQuery(); + + if (rs.next()) { + int id = rs.getInt("id"); + String description = rs.getString("description"); + + Map map = new HashMap<>(); + + String sql = "select `key`,`value` " + "from metadata_database_property " + "where database_id=? "; + try (PreparedStatement pStat = conn.prepareStatement(sql)) { + pStat.setInt(1, id); + ResultSet prs = pStat.executeQuery(); + while (prs.next()) { + map.put(prs.getString("key"), prs.getString("value")); + } + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException( + String.format("Failed get database properties in catalog %s", getName()), e); + } + + return new CatalogDatabaseImpl(map, description); + } else { + throw new DatabaseNotExistException(getName(), databaseName); + } + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException(String.format("Failed get database in catalog %s", getName()), e); + } + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + return getDatabaseId(databaseName) != null; + } + + private Integer getDatabaseId(String databaseName) throws CatalogException { + String querySql = "select id from metadata_database where database_name=?"; + Connection conn = getConnection(); + try (PreparedStatement ps = conn.prepareStatement(querySql)) { + ps.setString(1, databaseName); + ResultSet rs = ps.executeQuery(); + boolean multiDB = false; + Integer id = null; + while (rs.next()) { + if (!multiDB) { + id = rs.getInt(1); + multiDB = true; + } else { + throw new CatalogException("存在多个同名database: " + databaseName); + } + } + return id; + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException(String.format("获取 database 信息失败:%s.%s", getName(), databaseName), e); + } + } + + @Override + public void createDatabase(String databaseName, CatalogDatabase db, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + + checkArgument(!StringUtils.isNullOrWhitespaceOnly(databaseName)); + checkNotNull(db); + if (databaseExists(databaseName)) { + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(getName(), databaseName); + } + } else { + // 在这里实现创建库的代码 + Connection conn = getConnection(); + // 启动事务 + String insertSql = "insert into metadata_database(database_name, description) values(?, ?)"; + + try (PreparedStatement stat = conn.prepareStatement(insertSql, Statement.RETURN_GENERATED_KEYS)) { + conn.setAutoCommit(false); + stat.setString(1, databaseName); + stat.setString(2, db.getComment()); + stat.executeUpdate(); + ResultSet idRs = stat.getGeneratedKeys(); + if (idRs.next() + && db.getProperties() != null + && db.getProperties().size() > 0) { + int id = idRs.getInt(1); + String propInsertSql = + "insert into metadata_database_property(database_id, " + "`key`,`value`) values (?,?,?)"; + PreparedStatement pstat = conn.prepareStatement(propInsertSql); + for (Map.Entry entry : db.getProperties().entrySet()) { + pstat.setInt(1, id); + pstat.setString(2, entry.getKey()); + pstat.setString(3, entry.getValue()); + pstat.addBatch(); + } + pstat.executeBatch(); + pstat.close(); + } + conn.commit(); + } catch (SQLException e) { + sqlExceptionHappened = true; + logger.error("创建 database 信息失败:", e); + } + } + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + if (name.equals(defaultDatabase)) { + throw new CatalogException("默认 database 不可以删除"); + } + // 1、取出db id, + Integer id = getDatabaseId(name); + if (id == null) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + return; + } + Connection conn = getConnection(); + try { + conn.setAutoCommit(false); + // 查询是否有表 + List tables = listTables(name); + if (tables.size() > 0) { + if (!cascade) { + // 有表,不做级联删除。 + throw new DatabaseNotEmptyException(getName(), name); + } + // 做级联删除 + for (String table : tables) { + try { + dropTable(new ObjectPath(name, table), true); + } catch (TableNotExistException t) { + logger.warn("表{}不存在", name + "." + table); + } + } + } + // todo: 现在是真实删除,后续设计是否做记录保留。 + String deletePropSql = "delete from metadata_database_property where database_id=?"; + PreparedStatement dStat = conn.prepareStatement(deletePropSql); + dStat.setInt(1, id); + dStat.executeUpdate(); + dStat.close(); + String deleteDbSql = "delete from metadata_database where id=?"; + dStat = conn.prepareStatement(deleteDbSql); + dStat.setInt(1, id); + dStat.executeUpdate(); + dStat.close(); + conn.commit(); + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException("删除 database 信息失败:", e); + } + } + + @Override + public void alterDatabase(String name, CatalogDatabase newDb, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + if (name.equals(defaultDatabase)) { + throw new CatalogException("默认 database 不可以修改"); + } + // 1、取出db id, + Integer id = getDatabaseId(name); + if (id == null) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + return; + } + Connection conn = getConnection(); + try { + conn.setAutoCommit(false); + // 1、名称不能改,类型不能改。只能改备注 + String updateCommentSql = "update metadata_database set description=? where id=?"; + PreparedStatement uState = conn.prepareStatement(updateCommentSql); + uState.setString(1, newDb.getComment()); + uState.setInt(2, id); + uState.executeUpdate(); + uState.close(); + if (newDb.getProperties() != null && newDb.getProperties().size() > 0) { + String upsertSql = "insert into metadata_database_property (database_id, `key`,`value`) \n" + + "values (?,?,?)\n" + + "on duplicate key update `value` =?, update_time = sysdate()\n"; + PreparedStatement pstat = conn.prepareStatement(upsertSql); + for (Map.Entry entry : newDb.getProperties().entrySet()) { + pstat.setInt(1, id); + pstat.setString(2, entry.getKey()); + pstat.setString(3, entry.getValue()); + pstat.setString(4, entry.getValue()); + pstat.addBatch(); + } + + pstat.executeBatch(); + } + conn.commit(); + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException("修改 database 信息失败:", e); + } + } + + @Override + public List listTables(String databaseName) throws DatabaseNotExistException, CatalogException { + return listTablesViews(databaseName, ObjectType.TABLE); + } + + @Override + public List listViews(String databaseName) throws DatabaseNotExistException, CatalogException { + return listTablesViews(databaseName, ObjectType.VIEW); + } + + protected List listTablesViews(String databaseName, String tableType) + throws DatabaseNotExistException, CatalogException { + Integer databaseId = getDatabaseId(databaseName); + if (null == databaseId) { + throw new DatabaseNotExistException(getName(), databaseName); + } + + // get all schemas + // 要给出table 或 view + String querySql = "SELECT table_name FROM metadata_table where table_type=? and database_id = ?"; + Connection conn = getConnection(); + try (PreparedStatement ps = conn.prepareStatement(querySql)) { + ps.setString(1, tableType); + ps.setInt(2, databaseId); + ResultSet rs = ps.executeQuery(); + + List tables = new ArrayList<>(); + + while (rs.next()) { + String table = rs.getString(1); + tables.add(table); + } + return tables; + } catch (Exception e) { + throw new CatalogException(String.format("Failed listing %s in catalog %s", tableType, getName()), e); + } + } + + @Override + public CatalogBaseTable getTable(ObjectPath tablePath) throws TableNotExistException, CatalogException { + // 还是分步骤来 + // 1、先取出表 这可能是view也可能是table + // 2、取出列 + // 3、取出属性 + Integer id = getTableId(tablePath); + + if (id == null) { + throw new TableNotExistException(getName(), tablePath); + } + + Connection conn = getConnection(); + try { + String queryTable = + "SELECT table_name " + " ,description, table_type " + " FROM metadata_table " + " where id=?"; + PreparedStatement ps = conn.prepareStatement(queryTable); + ps.setInt(1, id); + ResultSet rs = ps.executeQuery(); + String description; + String tableType; + if (rs.next()) { + description = rs.getString("description"); + tableType = rs.getString("table_type"); + ps.close(); + } else { + ps.close(); + throw new TableNotExistException(getName(), tablePath); + } + if (tableType.equals(ObjectType.TABLE)) { + // 这个是 table + String propSql = "SELECT `key`, `value` from metadata_table_property " + "WHERE table_id=?"; + PreparedStatement pState = conn.prepareStatement(propSql); + pState.setInt(1, id); + ResultSet prs = pState.executeQuery(); + Map props = new HashMap<>(); + while (prs.next()) { + String key = prs.getString("key"); + String value = prs.getString("value"); + props.put(key, value); + } + pState.close(); + props.put(COMMENT, description); + return CatalogTable.fromProperties(props); + } else if (tableType.equals(ObjectType.VIEW)) { + // 1、从库中取出table信息。(前面已做) + // 2、取出字段。 + String colSql = "SELECT column_name, column_type, data_type, description " + + " FROM metadata_column WHERE " + + " table_id=?"; + PreparedStatement cStat = conn.prepareStatement(colSql); + cStat.setInt(1, id); + ResultSet crs = cStat.executeQuery(); + + Schema.Builder builder = Schema.newBuilder(); + while (crs.next()) { + String colName = crs.getString("column_name"); + String dataType = crs.getString("data_type"); + + builder.column(colName, dataType); + String cDesc = crs.getString("description"); + if (null != cDesc && cDesc.length() > 0) { + builder.withComment(cDesc); + } + } + cStat.close(); + // 3、取出query + String qSql = "SELECT `key`, value FROM metadata_table_property" + " WHERE table_id=? "; + PreparedStatement qStat = conn.prepareStatement(qSql); + qStat.setInt(1, id); + ResultSet qrs = qStat.executeQuery(); + String originalQuery = ""; + String expandedQuery = ""; + Map options = new HashMap<>(); + while (qrs.next()) { + String key = qrs.getString("key"); + String value = qrs.getString("value"); + if ("OriginalQuery".equals(key)) { + originalQuery = value; + } else if ("ExpandedQuery".equals(key)) { + expandedQuery = value; + } else { + options.put(key, value); + } + } + // 合成view + return CatalogView.of(builder.build(), description, originalQuery, expandedQuery, options); + } else { + throw new CatalogException("不支持的数据类型。" + tableType); + } + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException("获取 表信息失败。", e); + } + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + Integer id = getTableId(tablePath); + return id != null; + } + + private Integer getTableId(ObjectPath tablePath) { + Integer dbId = getDatabaseId(tablePath.getDatabaseName()); + if (dbId == null) { + return null; + } + // 获取id + String getIdSql = "select id from metadata_table " + " where table_name=? and database_id=?"; + Connection conn = getConnection(); + try (PreparedStatement gStat = conn.prepareStatement(getIdSql)) { + gStat.setString(1, tablePath.getObjectName()); + gStat.setInt(2, dbId); + ResultSet rs = gStat.executeQuery(); + if (rs.next()) { + return rs.getInt(1); + } + } catch (SQLException e) { + sqlExceptionHappened = true; + logger.error("get table fail", e); + throw new CatalogException("get table fail.", e); + } + return null; + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + Integer id = getTableId(tablePath); + + if (id == null) { + throw new TableNotExistException(getName(), tablePath); + } + Connection conn = getConnection(); + try { + // todo: 现在是真实删除,后续设计是否做记录保留。 + conn.setAutoCommit(false); + String deletePropSql = "delete from metadata_table_property " + " where table_id=?"; + PreparedStatement dStat = conn.prepareStatement(deletePropSql); + dStat.setInt(1, id); + dStat.executeUpdate(); + dStat.close(); + String deleteColSql = "delete from metadata_column " + " where table_id=?"; + dStat = conn.prepareStatement(deleteColSql); + dStat.setInt(1, id); + dStat.executeUpdate(); + dStat.close(); + String deleteDbSql = "delete from metadata_table " + " where id=?"; + dStat = conn.prepareStatement(deleteDbSql); + dStat.setInt(1, id); + dStat.executeUpdate(); + dStat.close(); + conn.commit(); + } catch (SQLException e) { + sqlExceptionHappened = true; + logger.error("drop table fail", e); + throw new CatalogException("drop table fail.", e); + } + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException, CatalogException { + Integer id = getTableId(tablePath); + + if (id == null) { + throw new TableNotExistException(getName(), tablePath); + } + ObjectPath newPath = new ObjectPath(tablePath.getDatabaseName(), newTableName); + if (tableExists(newPath)) { + throw new TableAlreadyExistException(getName(), newPath); + } + String updateSql = "UPDATE metadata_table SET table_name=? WHERE id=?"; + Connection conn = getConnection(); + try (PreparedStatement ps = conn.prepareStatement(updateSql)) { + ps.setString(1, newTableName); + ps.setInt(2, id); + ps.executeUpdate(); + } catch (SQLException ex) { + sqlExceptionHappened = true; + throw new CatalogException("修改表名失败", ex); + } + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws TableAlreadyExistException, DatabaseNotExistException, CatalogException { + Integer dbId = getDatabaseId(tablePath.getDatabaseName()); + if (null == dbId) { + throw new DatabaseNotExistException(getName(), tablePath.getDatabaseName()); + } + if (tableExists(tablePath)) { + if (!ignoreIfExists) { + throw new TableAlreadyExistException(getName(), tablePath); + } + return; + } + // 插入表 + // 插入到table表。这里,它可能是table也可能是view + // 如果是一个table,我们认为它是一个 resolved table,就可以使用properties方式来进行序列化并保存。 + // 如果是一个view,我们认为它只能有物理字段 + if (!(table instanceof ResolvedCatalogBaseTable)) { + throw new UnsupportedOperationException("暂时不支持输入非 ResolvedCatalogBaseTable 类型的表"); + } + Connection conn = getConnection(); + try { + conn.setAutoCommit(false); + // 首先插入表信息 + CatalogBaseTable.TableKind kind = table.getTableKind(); + + String insertSql = "insert into metadata_table(\n" + + " table_name," + + " table_type," + + " database_id," + + " description)" + + " values(?,?,?,?)"; + PreparedStatement iStat = conn.prepareStatement(insertSql, Statement.RETURN_GENERATED_KEYS); + iStat.setString(1, tablePath.getObjectName()); + iStat.setString(2, kind.toString()); + iStat.setInt(3, dbId); + iStat.setString(4, table.getComment()); + iStat.executeUpdate(); + ResultSet idRs = iStat.getGeneratedKeys(); + if (!idRs.next()) { + iStat.close(); + throw new CatalogException("插入元数据表信息失败"); + } + int id = idRs.getInt(1); + iStat.close(); + // 插入属性和列 + if (table instanceof ResolvedCatalogTable) { + // table 就可以直接拿properties了。 + Map props = ((ResolvedCatalogTable) table).toProperties(); + String propInsertSql = + "insert into metadata_table_property(table_id," + "`key`,`value`) values (?,?,?)"; + PreparedStatement pStat = conn.prepareStatement(propInsertSql); + for (Map.Entry entry : props.entrySet()) { + pStat.setInt(1, id); + pStat.setString(2, entry.getKey()); + pStat.setString(3, entry.getValue()); + pStat.addBatch(); + } + pStat.executeBatch(); + pStat.close(); + } else { + // view,咱先假定它只有物理字段 + // view 还需要保存:query,expanded query + // 插入属性和列 + ResolvedCatalogView view = (ResolvedCatalogView) table; + List cols = view.getUnresolvedSchema().getColumns(); + if (cols.size() > 0) { + String colInsertSql = "insert into metadata_column(" + + " column_name, column_type, data_type" + + " , `expr`" + + " , description" + + " , table_id" + + " , `primary`) " + + " values(?,?,?,?,?,?,?)"; + PreparedStatement colIStat = conn.prepareStatement(colInsertSql); + for (Schema.UnresolvedColumn col : cols) { + if (col instanceof Schema.UnresolvedPhysicalColumn) { + Schema.UnresolvedPhysicalColumn pCol = (Schema.UnresolvedPhysicalColumn) col; + if (!(pCol.getDataType() instanceof DataType)) { + throw new UnsupportedOperationException(String.format( + "类型识别失败,该列不是有效类型:%s.%s.%s : %s", + tablePath.getDatabaseName(), + tablePath.getObjectName(), + pCol.getName(), + pCol.getDataType())); + } + DataType dataType = (DataType) pCol.getDataType(); + + colIStat.setString(1, pCol.getName()); + colIStat.setString(2, ColumnType.PHYSICAL); + colIStat.setString(3, dataType.getLogicalType().asSerializableString()); + colIStat.setObject(4, null); + colIStat.setString(5, pCol.getComment().orElse("")); + colIStat.setInt(6, id); + colIStat.setObject(7, null); // view没有主键 + colIStat.addBatch(); + } else { + throw new UnsupportedOperationException("暂时认为view 不会出现 非物理字段"); + } + } + colIStat.executeBatch(); + colIStat.close(); + + // 写 query等信息到数据库 + Map option = view.getOptions(); + if (option == null) { + option = new HashMap<>(); + } + option.put("OriginalQuery", view.getOriginalQuery()); + option.put("ExpandedQuery", view.getExpandedQuery()); + String propInsertSql = + "insert into metadata_table_property(table_id," + "`key`,`value`) values (?,?,?)"; + PreparedStatement pStat = conn.prepareStatement(propInsertSql); + for (Map.Entry entry : option.entrySet()) { + pStat.setInt(1, id); + pStat.setString(2, entry.getKey()); + pStat.setString(3, entry.getValue()); + pStat.addBatch(); + } + pStat.executeBatch(); + pStat.close(); + } + } + conn.commit(); + } catch (SQLException ex) { + sqlExceptionHappened = true; + logger.error("插入数据库失败", ex); + throw new CatalogException("插入数据库失败", ex); + } + } + + @Override + public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + Integer id = getTableId(tablePath); + + if (id == null) { + throw new TableNotExistException(getName(), tablePath); + } + + Map opts = newTable.getOptions(); + if (opts != null && opts.size() > 0) { + String updateSql = "INSERT INTO metadata_table_property(table_id," + + "`key`,`value`) values (?,?,?) " + + "on duplicate key update `value` =?, update_time = sysdate()"; + Connection conn = getConnection(); + try (PreparedStatement ps = conn.prepareStatement(updateSql)) { + for (Map.Entry entry : opts.entrySet()) { + ps.setInt(1, id); + ps.setString(2, entry.getKey()); + ps.setString(3, entry.getValue()); + ps.setString(4, entry.getValue()); + ps.addBatch(); + } + ps.executeBatch(); + } catch (SQLException ex) { + sqlExceptionHappened = true; + throw new CatalogException("修改表名失败", ex); + } + } + } + + /** ********************** partition ************************ */ + @Override + public List listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public List listPartitions(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws TableNotExistException, TableNotPartitionedException, PartitionSpecInvalidException, + CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public List listPartitionsByFilter(ObjectPath tablePath, List filters) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public void createPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition partition, + boolean ignoreIfExists) + throws TableNotExistException, TableNotPartitionedException, PartitionSpecInvalidException, + PartitionAlreadyExistsException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public void dropPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public void alterPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition newPartition, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + /** *********************Functions********************* */ + @Override + public List listFunctions(String dbName) throws DatabaseNotExistException, CatalogException { + Integer dbId = getDatabaseId(dbName); + if (null == dbId) { + throw new DatabaseNotExistException(getName(), dbName); + } + String querySql = "SELECT function_name from metadata_function " + " WHERE database_id=?"; + + Connection conn = getConnection(); + try (PreparedStatement gStat = conn.prepareStatement(querySql)) { + gStat.setInt(1, dbId); + ResultSet rs = gStat.executeQuery(); + List functions = new ArrayList<>(); + while (rs.next()) { + String n = rs.getString("function_name"); + functions.add(n); + } + return functions; + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException("获取 UDF 列表失败"); + } + } + + @Override + public CatalogFunction getFunction(ObjectPath functionPath) throws FunctionNotExistException, CatalogException { + Integer id = getFunctionId(functionPath); + if (null == id) { + throw new FunctionNotExistException(getName(), functionPath); + } + + String querySql = "SELECT class_name,function_language from metadata_function " + " WHERE id=?"; + Connection conn = getConnection(); + try (PreparedStatement gStat = conn.prepareStatement(querySql)) { + gStat.setInt(1, id); + ResultSet rs = gStat.executeQuery(); + if (rs.next()) { + String className = rs.getString("class_name"); + String language = rs.getString("function_language"); + CatalogFunctionImpl func = new CatalogFunctionImpl(className, FunctionLanguage.valueOf(language)); + return func; + } else { + throw new FunctionNotExistException(getName(), functionPath); + } + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException( + "获取 UDF 失败:" + functionPath.getDatabaseName() + "." + functionPath.getObjectName()); + } + } + + @Override + public boolean functionExists(ObjectPath functionPath) throws CatalogException { + Integer id = getFunctionId(functionPath); + return id != null; + } + + private Integer getFunctionId(ObjectPath functionPath) { + Integer dbId = getDatabaseId(functionPath.getDatabaseName()); + if (dbId == null) { + return null; + } + // 获取id + String getIdSql = "select id from metadata_function " + " where function_name=? and database_id=?"; + Connection conn = getConnection(); + try (PreparedStatement gStat = conn.prepareStatement(getIdSql)) { + gStat.setString(1, functionPath.getObjectName()); + gStat.setInt(2, dbId); + ResultSet rs = gStat.executeQuery(); + if (rs.next()) { + int id = rs.getInt(1); + return id; + } + } catch (SQLException e) { + sqlExceptionHappened = true; + logger.error("get function fail", e); + throw new CatalogException("get function fail.", e); + } + return null; + } + + @Override + public void createFunction(ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) + throws FunctionAlreadyExistException, DatabaseNotExistException, CatalogException { + Integer dbId = getDatabaseId(functionPath.getDatabaseName()); + if (null == dbId) { + throw new DatabaseNotExistException(getName(), functionPath.getDatabaseName()); + } + if (functionExists(functionPath)) { + if (!ignoreIfExists) { + throw new FunctionAlreadyExistException(getName(), functionPath); + } + } + + Connection conn = getConnection(); + String insertSql = "Insert into metadata_function " + + "(function_name,class_name,database_id,function_language) " + + " values (?,?,?,?)"; + try (PreparedStatement ps = conn.prepareStatement(insertSql)) { + ps.setString(1, functionPath.getObjectName()); + ps.setString(2, function.getClassName()); + ps.setInt(3, dbId); + ps.setString(4, function.getFunctionLanguage().toString()); + ps.executeUpdate(); + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException("创建 函数 失败", e); + } + } + + @Override + public void alterFunction(ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + Integer id = getFunctionId(functionPath); + if (null == id) { + if (!ignoreIfNotExists) { + throw new FunctionNotExistException(getName(), functionPath); + } + return; + } + + Connection conn = getConnection(); + String insertSql = "update metadata_function " + "set (class_name =?, function_language=?) " + " where id=?"; + try (PreparedStatement ps = conn.prepareStatement(insertSql)) { + ps.setString(1, newFunction.getClassName()); + ps.setString(2, newFunction.getFunctionLanguage().toString()); + ps.setInt(3, id); + ps.executeUpdate(); + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException("修改 函数 失败", e); + } + } + + @Override + public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) + throws FunctionNotExistException, CatalogException { + Integer id = getFunctionId(functionPath); + if (null == id) { + if (!ignoreIfNotExists) { + throw new FunctionNotExistException(getName(), functionPath); + } + return; + } + + Connection conn = getConnection(); + String insertSql = "delete from metadata_function " + " where id=?"; + try (PreparedStatement ps = conn.prepareStatement(insertSql)) { + ps.setInt(1, id); + ps.executeUpdate(); + } catch (SQLException e) { + sqlExceptionHappened = true; + throw new CatalogException("删除 函数 失败", e); + } + } + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + // todo: 补充完成该方法。 + checkNotNull(tablePath); + + if (!tableExists(tablePath)) { + throw new TableNotExistException(getName(), tablePath); + } + /* + * if (!isPartitionedTable(tablePath)) { CatalogTableStatistics result = tableStats.get(tablePath); return + * result != null ? result.copy() : CatalogTableStatistics.UNKNOWN; } else { return + * CatalogTableStatistics.UNKNOWN; } + */ + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + // todo: 补充完成该方法。 + checkNotNull(tablePath); + + if (!tableExists(tablePath)) { + throw new TableNotExistException(getName(), tablePath); + } + + // CatalogColumnStatistics result = tableColumnStats.get(tablePath); + // return result != null ? result.copy() : CatalogColumnStatistics.UNKNOWN; + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public void alterTableStatistics( + ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public void alterTableColumnStatistics( + ObjectPath tablePath, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException, TablePartitionedException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public void alterPartitionStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogTableStatistics partitionStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } + + @Override + public void alterPartitionColumnStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + // todo: 补充完成该方法。 + throw new UnsupportedOperationException("该方法尚未完成"); + } +} diff --git a/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/factory/DinkyMysqlCatalogFactory.java b/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/factory/DinkyMysqlCatalogFactory.java new file mode 100644 index 0000000000..afcfe2e600 --- /dev/null +++ b/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/factory/DinkyMysqlCatalogFactory.java @@ -0,0 +1,72 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.flink.catalog.factory; + +import static org.apache.flink.table.factories.FactoryUtil.PROPERTY_VERSION; +import static org.dinky.flink.catalog.factory.DinkyMysqlCatalogFactoryOptions.PASSWORD; +import static org.dinky.flink.catalog.factory.DinkyMysqlCatalogFactoryOptions.URL; +import static org.dinky.flink.catalog.factory.DinkyMysqlCatalogFactoryOptions.USERNAME; + +import org.dinky.flink.catalog.DinkyMysqlCatalog; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.flink.table.factories.FactoryUtil; + +import java.util.HashSet; +import java.util.Set; + +/** Factory for {@link DinkyMysqlCatalog}. */ +public class DinkyMysqlCatalogFactory implements CatalogFactory { + + @Override + public String factoryIdentifier() { + return DinkyMysqlCatalogFactoryOptions.IDENTIFIER; + } + + @Override + public Set> requiredOptions() { + final Set> options = new HashSet<>(); + return options; + } + + @Override + public Set> optionalOptions() { + final Set> options = new HashSet<>(); + options.add(USERNAME); + options.add(PASSWORD); + options.add(URL); + options.add(PROPERTY_VERSION); + return options; + } + + @Override + public Catalog createCatalog(Context context) { + final FactoryUtil.CatalogFactoryHelper helper = FactoryUtil.createCatalogFactoryHelper(this, context); + helper.validate(); + + return new DinkyMysqlCatalog( + context.getName(), + helper.getOptions().get(URL), + helper.getOptions().get(USERNAME), + helper.getOptions().get(PASSWORD)); + } +} diff --git a/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/factory/DinkyMysqlCatalogFactoryOptions.java b/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/factory/DinkyMysqlCatalogFactoryOptions.java new file mode 100644 index 0000000000..63bbf47369 --- /dev/null +++ b/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/java/org/dinky/flink/catalog/factory/DinkyMysqlCatalogFactoryOptions.java @@ -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 org.dinky.flink.catalog.factory; + +import org.dinky.flink.catalog.DinkyMysqlCatalog; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** {@link ConfigOption}s for {@link DinkyMysqlCatalog}. */ +@Internal +public class DinkyMysqlCatalogFactoryOptions { + + public static final String IDENTIFIER = "dinky_mysql"; + + public static final ConfigOption USERNAME = + ConfigOptions.key("username").stringType().noDefaultValue(); + + public static final ConfigOption PASSWORD = + ConfigOptions.key("password").stringType().noDefaultValue(); + + public static final ConfigOption URL = + ConfigOptions.key("url").stringType().noDefaultValue(); + + private DinkyMysqlCatalogFactoryOptions() {} +} diff --git a/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 0000000000..2ad955fc02 --- /dev/null +++ b/dinky-catalog/dinky-catalog-mysql/dinky-catalog-mysql-1.20/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.dinky.flink.catalog.factory.DinkyMysqlCatalogFactory diff --git a/dinky-catalog/dinky-catalog-mysql/pom.xml b/dinky-catalog/dinky-catalog-mysql/pom.xml index 5438919b73..7843791f24 100644 --- a/dinky-catalog/dinky-catalog-mysql/pom.xml +++ b/dinky-catalog/dinky-catalog-mysql/pom.xml @@ -46,6 +46,7 @@ dinky-catalog-mysql-1.17 dinky-catalog-mysql-1.18 dinky-catalog-mysql-1.19 + dinky-catalog-mysql-1.20 diff --git a/dinky-client/dinky-client-1.20/pom.xml b/dinky-client/dinky-client-1.20/pom.xml new file mode 100644 index 0000000000..f23cd8971b --- /dev/null +++ b/dinky-client/dinky-client-1.20/pom.xml @@ -0,0 +1,63 @@ + + + 4.0.0 + + org.dinky + dinky-client + ${revision} + ../pom.xml + + dinky-client-1.20 + + jar + + Dinky : Client 1.20 + + + + org.dinky + dinky-client-base + + + org.dinky + dinky-flink-1.20 + ${scope.runtime} + + + com.fasterxml.jackson.datatype + jackson-datatype-jsr310 + provided + + + javax.xml.bind + jaxb-api + + + com.sun.xml.bind + jaxb-impl + + + com.sun.xml.bind + jaxb-core + + + junit + junit + provided + + + + + + + org.apache.maven.plugins + maven-jar-plugin + ${maven-jar-plugin.version} + + ${project.parent.parent.basedir}/build/extends + + + + + diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java b/dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java new file mode 100644 index 0000000000..5fc8dc24cb --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/rel/metadata/RelColumnOrigin.java @@ -0,0 +1,116 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelColumnOrigin + *

+ * Modification point: + *

    + *
  1. add transform field and related code. + *
+ * + * @description: RelColumnOrigin is a data structure describing one of the origins of an + * output column produced by a relational expression. + * @author: HamaWhite + */ +public class RelColumnOrigin { + // ~ Instance fields -------------------------------------------------------- + + private final RelOptTable originTable; + + private final int iOriginColumn; + + private final boolean isDerived; + + /** + * Stores the expression for data conversion, + * which source table fields are transformed by which expression the target field + */ + private String transform; + + // ~ Constructors ----------------------------------------------------------- + + public RelColumnOrigin(RelOptTable originTable, int iOriginColumn, boolean isDerived) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + } + + public RelColumnOrigin(RelOptTable originTable, int iOriginColumn, boolean isDerived, String transform) { + this.originTable = originTable; + this.iOriginColumn = iOriginColumn; + this.isDerived = isDerived; + this.transform = transform; + } + + // ~ Methods ---------------------------------------------------------------- + + /** + * Returns table of origin. + */ + public RelOptTable getOriginTable() { + return originTable; + } + + /** + * Returns the 0-based index of column in origin table; whether this ordinal + * is flattened or unflattened depends on whether UDT flattening has already + * been performed on the relational expression which produced this + * description. + */ + public int getOriginColumnOrdinal() { + return iOriginColumn; + } + + /** + * Consider the query select a+b as c, d as e from t. The + * output column c has two origins (a and b), both of them derived. The + * output column d as one origin (c), which is not derived. + * + * @return false if value taken directly from column in origin table; true + * otherwise + */ + public boolean isDerived() { + return isDerived; + } + + public String getTransform() { + return transform; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof RelColumnOrigin)) { + return false; + } + RelColumnOrigin other = (RelColumnOrigin) obj; + return originTable.getQualifiedName().equals(other.originTable.getQualifiedName()) + && (iOriginColumn == other.iOriginColumn) + && (isDerived == other.isDerived); + } + + @Override + public int hashCode() { + return originTable.getQualifiedName().hashCode() + iOriginColumn + (isDerived ? 313 : 0); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java b/dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java new file mode 100644 index 0000000000..5c8aae002a --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/rel/metadata/RelMdColumnOrigins.java @@ -0,0 +1,534 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.rel.metadata; + +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.SingleRel; +import org.apache.calcite.rel.core.Aggregate; +import org.apache.calcite.rel.core.AggregateCall; +import org.apache.calcite.rel.core.Calc; +import org.apache.calcite.rel.core.Correlate; +import org.apache.calcite.rel.core.Exchange; +import org.apache.calcite.rel.core.Filter; +import org.apache.calcite.rel.core.Join; +import org.apache.calcite.rel.core.Match; +import org.apache.calcite.rel.core.Project; +import org.apache.calcite.rel.core.SetOp; +import org.apache.calcite.rel.core.Snapshot; +import org.apache.calcite.rel.core.Sort; +import org.apache.calcite.rel.core.TableFunctionScan; +import org.apache.calcite.rel.core.TableModify; +import org.apache.calcite.rel.core.TableScan; +import org.apache.calcite.rel.type.RelDataTypeField; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexFieldAccess; +import org.apache.calcite.rex.RexInputRef; +import org.apache.calcite.rex.RexLocalRef; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexPatternFieldRef; +import org.apache.calcite.rex.RexShuttle; +import org.apache.calcite.rex.RexVisitor; +import org.apache.calcite.rex.RexVisitorImpl; +import org.apache.calcite.util.BuiltInMethod; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Modified based on calcite's source code org.apache.calcite.rel.metadata.RelMdColumnOrigins + * + *

Modification point: + *

    + *
  1. Support lookup join, add method getColumnOrigins(Snapshot rel,RelMetadataQuery mq, int iOutputColumn) + *
  2. Support watermark, add method getColumnOrigins(SingleRel rel,RelMetadataQuery mq, int iOutputColumn) + *
  3. Support table function, add method getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) + *
  4. Support CEP, add method getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) + *
  5. Support transform, add method createDerivedColumnOrigins(Set inputSet, String transform, boolean originTransform), and related code + *
  6. Support field AS LOCALTIMESTAMP, modify method getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
  7. Support PROCTIME() is the first filed, add method computeIndexWithOffset, used by getColumnOrigins(Project rel, RelMetadataQuery mq, int iOutputColumn) + *
      + * + * @description: RelMdColumnOrigins supplies a default implementation of {@link RelMetadataQuery#getColumnOrigins} for the standard logical algebra. + * @author: HamaWhite + */ +public class RelMdColumnOrigins implements MetadataHandler { + + private static final Logger LOG = LoggerFactory.getLogger(RelMdColumnOrigins.class); + + public static final String DELIMITER = "."; + + public static final RelMetadataProvider SOURCE = ReflectiveRelMetadataProvider.reflectiveSource( + BuiltInMethod.COLUMN_ORIGIN.method, new RelMdColumnOrigins()); + + // ~ Constructors ----------------------------------------------------------- + + private RelMdColumnOrigins() {} + + // ~ Methods ---------------------------------------------------------------- + + public MetadataDef getDef() { + return BuiltInMetadata.ColumnOrigin.DEF; + } + + public Set getColumnOrigins(Aggregate rel, RelMetadataQuery mq, int iOutputColumn) { + if (iOutputColumn < rel.getGroupCount()) { + // get actual index of Group columns. + return mq.getColumnOrigins( + rel.getInput(), rel.getGroupSet().asList().get(iOutputColumn)); + } + + // Aggregate columns are derived from input columns + AggregateCall call = rel.getAggCallList().get(iOutputColumn - rel.getGroupCount()); + + final Set set = new LinkedHashSet<>(); + for (Integer iInput : call.getArgList()) { + Set inputSet = mq.getColumnOrigins(rel.getInput(), iInput); + inputSet = createDerivedColumnOrigins(inputSet, call.toString(), true); + if (inputSet != null) { + set.addAll(inputSet); + } + } + return set; + } + + public Set getColumnOrigins(Join rel, RelMetadataQuery mq, int iOutputColumn) { + int nLeftColumns = rel.getLeft().getRowType().getFieldList().size(); + Set set; + boolean derived = false; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + if (rel.getJoinType().generatesNullsOnLeft()) { + derived = true; + } + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + if (rel.getJoinType().generatesNullsOnRight()) { + derived = true; + } + } + if (derived) { + // nulls are generated due to outer join; that counts + // as derivation + set = createDerivedColumnOrigins(set); + } + return set; + } + + /** + * Support the field blood relationship of table function + */ + public Set getColumnOrigins(Correlate rel, RelMetadataQuery mq, int iOutputColumn) { + + List leftFieldList = rel.getLeft().getRowType().getFieldList(); + + int nLeftColumns = leftFieldList.size(); + Set set; + if (iOutputColumn < nLeftColumns) { + set = mq.getColumnOrigins(rel.getLeft(), iOutputColumn); + } else { + if (rel.getRight() instanceof TableFunctionScan) { + // get the field name of the left table configured in the Table Function on the right + TableFunctionScan tableFunctionScan = (TableFunctionScan) rel.getRight(); + RexCall rexCall = (RexCall) tableFunctionScan.getCall(); + // support only one field in table function + RexFieldAccess rexFieldAccess = + (RexFieldAccess) rexCall.getOperands().get(0); + String fieldName = rexFieldAccess.getField().getName(); + + int leftFieldIndex = 0; + for (int i = 0; i < nLeftColumns; i++) { + if (leftFieldList.get(i).getName().equalsIgnoreCase(fieldName)) { + leftFieldIndex = i; + break; + } + } + /** + * Get the fields from the left table, don't go to + * getColumnOrigins(TableFunctionScan rel,RelMetadataQuery mq, int iOutputColumn), + * otherwise the return is null, and the UDTF field origin cannot be parsed + */ + set = mq.getColumnOrigins(rel.getLeft(), leftFieldIndex); + + // process transform for udtf + String transform = rexCall.toString().replace(rexFieldAccess.toString(), fieldName) + + DELIMITER + + tableFunctionScan.getRowType().getFieldNames().get(iOutputColumn - nLeftColumns); + set = createDerivedColumnOrigins(set, transform, false); + } else { + set = mq.getColumnOrigins(rel.getRight(), iOutputColumn - nLeftColumns); + } + } + return set; + } + + public Set getColumnOrigins(SetOp rel, RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + for (RelNode input : rel.getInputs()) { + Set inputSet = mq.getColumnOrigins(input, iOutputColumn); + if (inputSet == null) { + return Collections.emptySet(); + } + set.addAll(inputSet); + } + return set; + } + + /** + * Support the field blood relationship of lookup join + */ + public Set getColumnOrigins(Snapshot rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support the field blood relationship of watermark + */ + public Set getColumnOrigins(SingleRel rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + /** + * Support for new fields in the source table similar to those created with the LOCALTIMESTAMP function + */ + public Set getColumnOrigins(Project rel, final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getProjects().get(iOutputColumn); + + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + int index = inputRef.getIndex(); + if (input instanceof TableScan) { + index = computeIndexWithOffset(rel.getProjects(), inputRef.getIndex(), iOutputColumn); + } + return mq.getColumnOrigins(input, index); + } else if (input instanceof TableScan + && rexNode.getClass().equals(RexCall.class) + && ((RexCall) rexNode).getOperands().isEmpty()) { + return mq.getColumnOrigins(input, iOutputColumn); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set, rexNode.toString(), true); + } + + private int computeIndexWithOffset(List projects, int baseIndex, int iOutputColumn) { + int offset = 0; + for (int index = 0; index < iOutputColumn; index++) { + RexNode rexNode = projects.get(index); + if ((rexNode.getClass().equals(RexCall.class) + && ((RexCall) rexNode).getOperands().isEmpty())) { + offset += 1; + } + } + return baseIndex + offset; + } + + /** + * Support field blood relationship of CEP. + * The first column is the field after PARTITION BY, and the other columns come from the measures in Match + */ + public Set getColumnOrigins(Match rel, RelMetadataQuery mq, int iOutputColumn) { + int orderCount = rel.getOrderKeys().getKeys().size(); + + if (iOutputColumn < orderCount) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + final RelNode input = rel.getInput(); + RexNode rexNode = rel.getMeasures().values().asList().get(iOutputColumn - orderCount); + + RexPatternFieldRef rexPatternFieldRef = searchRexPatternFieldRef(rexNode); + if (rexPatternFieldRef != null) { + final Set set = mq.getColumnOrigins(input, rexPatternFieldRef.getIndex()); + String originTransform = rexNode instanceof RexCall + ? ((RexCall) rexNode).getOperands().get(0).toString() + : null; + return createDerivedColumnOrigins(set, originTransform, true); + } + return Collections.emptySet(); + } + + private RexPatternFieldRef searchRexPatternFieldRef(RexNode rexNode) { + if (rexNode instanceof RexCall) { + RexNode operand = ((RexCall) rexNode).getOperands().get(0); + if (operand instanceof RexPatternFieldRef) { + return (RexPatternFieldRef) operand; + } else { + // recursive search + return searchRexPatternFieldRef(operand); + } + } + return null; + } + + public Set getColumnOrigins(Calc rel, final RelMetadataQuery mq, int iOutputColumn) { + final RelNode input = rel.getInput(); + final RexShuttle rexShuttle = new RexShuttle() { + + @Override + public RexNode visitLocalRef(RexLocalRef localRef) { + return rel.getProgram().expandLocalRef(localRef); + } + }; + final List projects = new ArrayList<>(); + for (RexNode rex : rexShuttle.apply(rel.getProgram().getProjectList())) { + projects.add(rex); + } + final RexNode rexNode = projects.get(iOutputColumn); + if (rexNode instanceof RexInputRef) { + // Direct reference: no derivation added. + RexInputRef inputRef = (RexInputRef) rexNode; + return mq.getColumnOrigins(input, inputRef.getIndex()); + } + // Anything else is a derivation, possibly from multiple columns. + final Set set = getMultipleColumns(rexNode, input, mq); + return createDerivedColumnOrigins(set); + } + + public Set getColumnOrigins(Filter rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Sort rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableModify rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(Exchange rel, RelMetadataQuery mq, int iOutputColumn) { + return mq.getColumnOrigins(rel.getInput(), iOutputColumn); + } + + public Set getColumnOrigins(TableFunctionScan rel, RelMetadataQuery mq, int iOutputColumn) { + final Set set = new LinkedHashSet<>(); + Set mappings = rel.getColumnMappings(); + if (mappings == null) { + if (!rel.getInputs().isEmpty()) { + // This is a non-leaf transformation: say we don't + // know about origins, because there are probably + // columns below. + return Collections.emptySet(); + } else { + // This is a leaf transformation: say there are fer sure no + // column origins. + return set; + } + } + for (RelColumnMapping mapping : mappings) { + if (mapping.iOutputColumn != iOutputColumn) { + continue; + } + final RelNode input = rel.getInputs().get(mapping.iInputRel); + final int column = mapping.iInputColumn; + Set origins = mq.getColumnOrigins(input, column); + if (origins == null) { + return Collections.emptySet(); + } + if (mapping.derived) { + origins = createDerivedColumnOrigins(origins); + } + set.addAll(origins); + } + return set; + } + + // Catch-all rule when none of the others apply. + @SuppressWarnings("squid:S1172") + public Set getColumnOrigins(RelNode rel, RelMetadataQuery mq, int iOutputColumn) { + // NOTE jvs 28-Mar-2006: We may get this wrong for a physical table + // expression which supports projections. In that case, + // it's up to the plugin writer to override with the + // correct information. + + if (!rel.getInputs().isEmpty()) { + // No generic logic available for non-leaf rels. + return Collections.emptySet(); + } + + final Set set = new LinkedHashSet<>(); + + RelOptTable table = rel.getTable(); + if (table == null) { + // Somebody is making column values up out of thin air, like a + // VALUES clause, so we return an empty set. + return set; + } + + // Detect the case where a physical table expression is performing + // projection, and say we don't know instead of making any assumptions. + // (Theoretically we could try to map the projection using column + // names.) This detection assumes the table expression doesn't handle + // rename as well. + if (table.getRowType() != rel.getRowType()) { + return Collections.emptySet(); + } + + set.add(new RelColumnOrigin(table, iOutputColumn, false)); + return set; + } + + private Set createDerivedColumnOrigins(Set inputSet) { + if (inputSet == null) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = new RelColumnOrigin(rco.getOriginTable(), rco.getOriginColumnOrdinal(), true); + set.add(derived); + } + return set; + } + + private Set createDerivedColumnOrigins( + Set inputSet, String transform, boolean originTransform) { + if (inputSet == null || inputSet.isEmpty()) { + return Collections.emptySet(); + } + final Set set = new LinkedHashSet<>(); + + String finalTransform = originTransform ? computeTransform(inputSet, transform) : transform; + for (RelColumnOrigin rco : inputSet) { + RelColumnOrigin derived = + new RelColumnOrigin(rco.getOriginTable(), rco.getOriginColumnOrdinal(), true, finalTransform); + set.add(derived); + } + return set; + } + + /** + * Replace the variable at the beginning of $ in input with the real field information + */ + private String computeTransform(Set inputSet, String transform) { + LOG.debug("origin transform: {}", transform); + Pattern pattern = Pattern.compile("\\$\\d+"); + Matcher matcher = pattern.matcher(transform); + + Set operandSet = new LinkedHashSet<>(); + while (matcher.find()) { + operandSet.add(matcher.group()); + } + + if (operandSet.isEmpty()) { + LOG.info("operandSet is empty"); + return null; + } + if (inputSet.size() != operandSet.size()) { + LOG.warn( + "The number [{}] of fields in the source tables are not equal to operands [{}]", + inputSet.size(), + operandSet.size()); + return null; + } + + Map sourceColumnMap = new HashMap<>(); + Iterator iterator = optimizeSourceColumnSet(inputSet).iterator(); + operandSet.forEach(e -> sourceColumnMap.put(e, iterator.next())); + LOG.debug("sourceColumnMap: {}", sourceColumnMap); + + matcher = pattern.matcher(transform); + String temp; + while (matcher.find()) { + temp = matcher.group(); + transform = transform.replace(temp, sourceColumnMap.get(temp)); + } + + // temporary special treatment + transform = transform.replace("_UTF-16LE", ""); + LOG.debug("transform: {}", transform); + return transform; + } + + /** + * Increase the readability of transform. + * if catalog, database and table are the same, return field. + * If the catalog and database are the same, return the table and field. + * If the catalog is the same, return the database, table, field. + * Otherwise, return all + */ + private Set optimizeSourceColumnSet(Set inputSet) { + Set catalogSet = new HashSet<>(); + Set databaseSet = new HashSet<>(); + Set tableSet = new HashSet<>(); + Set> qualifiedSet = new LinkedHashSet<>(); + for (RelColumnOrigin rco : inputSet) { + RelOptTable originTable = rco.getOriginTable(); + List qualifiedName = originTable.getQualifiedName(); + + // catalog,database,table,field + List qualifiedList = new ArrayList<>(qualifiedName); + catalogSet.add(qualifiedName.get(0)); + databaseSet.add(qualifiedName.get(1)); + tableSet.add(qualifiedName.get(2)); + + String field = rco.getTransform() != null + ? rco.getTransform() + : originTable.getRowType().getFieldNames().get(rco.getOriginColumnOrdinal()); + qualifiedList.add(field); + qualifiedSet.add(qualifiedList); + } + if (catalogSet.size() == 1 && databaseSet.size() == 1 && tableSet.size() == 1) { + return optimizeName(qualifiedSet, e -> e.get(3)); + } else if (catalogSet.size() == 1 && databaseSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(2, 4))); + } else if (catalogSet.size() == 1) { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e.subList(1, 4))); + } else { + return optimizeName(qualifiedSet, e -> String.join(DELIMITER, e)); + } + } + + private Set optimizeName(Set> qualifiedSet, Function, String> mapper) { + return qualifiedSet.stream().map(mapper).collect(Collectors.toSet()); + } + + private Set getMultipleColumns(RexNode rexNode, RelNode input, final RelMetadataQuery mq) { + final Set set = new LinkedHashSet<>(); + final RexVisitor visitor = new RexVisitorImpl(true) { + + @Override + public Void visitInputRef(RexInputRef inputRef) { + Set inputSet = mq.getColumnOrigins(input, inputRef.getIndex()); + if (inputSet != null) { + set.addAll(inputSet); + } + return null; + } + }; + rexNode.accept(visitor); + return set; + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/sql/SqlSelect.java b/dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/sql/SqlSelect.java new file mode 100644 index 0000000000..d8824d0246 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/apache/calcite/sql/SqlSelect.java @@ -0,0 +1,355 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.calcite.sql; + +import org.dinky.context.CustomTableEnvironmentContext; +import org.dinky.context.RowLevelPermissionsContext; +import org.dinky.executor.ExtendedParser; + +import org.apache.calcite.sql.parser.SqlParserPos; +import org.apache.calcite.sql.validate.SqlValidator; +import org.apache.calcite.sql.validate.SqlValidatorScope; +import org.apache.calcite.util.ImmutableNullableList; + +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; + +import javax.annotation.Nonnull; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.ImmutableList; + +/** + * A SqlSelect is a node of a parse tree which represents a select statement. It + * warrants its own node type just because we have a lot of methods to put somewhere. + * + * @description: Modify the value method of where to addCondition() to support row-level permission + * filtering + */ +public class SqlSelect extends SqlCall { + + private static final Logger LOG = LoggerFactory.getLogger(SqlSelect.class); + + public static final int FROM_OPERAND = 2; + public static final int WHERE_OPERAND = 3; + public static final int HAVING_OPERAND = 5; + + SqlNodeList keywordList; + SqlNodeList selectList; + SqlNode from; + SqlNode where; + SqlNodeList groupBy; + SqlNode having; + SqlNodeList windowDecls; + SqlNodeList orderBy; + SqlNode offset; + SqlNode fetch; + SqlNodeList hints; + + public SqlSelect( + SqlParserPos pos, + SqlNodeList keywordList, + SqlNodeList selectList, + SqlNode from, + SqlNode where, + SqlNodeList groupBy, + SqlNode having, + SqlNodeList windowDecls, + SqlNodeList orderBy, + SqlNode offset, + SqlNode fetch, + SqlNodeList hints) { + super(pos); + this.keywordList = Objects.requireNonNull(keywordList != null ? keywordList : new SqlNodeList(pos)); + this.selectList = selectList; + this.from = from; + this.groupBy = groupBy; + this.having = having; + this.windowDecls = Objects.requireNonNull(windowDecls != null ? windowDecls : new SqlNodeList(pos)); + this.orderBy = orderBy; + this.offset = offset; + this.fetch = fetch; + this.hints = hints; + + // add row level filter condition for where clause + this.where = addCondition(from, where, false); + } + + /** The main process of controlling row-level permissions */ + private SqlNode addCondition(SqlNode from, SqlNode where, boolean fromJoin) { + if (from instanceof SqlIdentifier) { + String tableName = from.toString(); + // the table name is used as an alias for join + String tableAlias = fromJoin ? tableName : null; + return addPermission(where, tableName, tableAlias); + } else if (from instanceof SqlJoin) { + SqlJoin sqlJoin = (SqlJoin) from; + // support recursive processing, such as join for three tables, process left sqlNode + where = addCondition(sqlJoin.getLeft(), where, true); + // process right sqlNode + return addCondition(sqlJoin.getRight(), where, true); + } else if (from instanceof SqlBasicCall) { + // Table has an alias or comes from a subquery + List operandList = ((SqlBasicCall) from).getOperandList(); + /** + * If there is a subquery in the Join, row-level filtering has been appended to the + * subquery. What is returned here is the SqlSelect type, just return the original where + * directly + */ + if (!(operandList.get(0) instanceof SqlIdentifier)) { + return where; + } + String tableName = operandList.get(0).toString(); + String tableAlias = operandList.get(1).toString(); + return addPermission(where, tableName, tableAlias); + } + return where; + } + + /** Add row-level filtering based on user-configured permission points */ + private SqlNode addPermission(SqlNode where, String tableName, String tableAlias) { + SqlBasicCall permissions = null; + ConcurrentHashMap permissionsMap = RowLevelPermissionsContext.get(); + if (permissionsMap != null) { + String permissionsStatement = permissionsMap.get(tableName); + if (permissionsStatement != null && !"".equals(permissionsStatement)) { + if (CustomTableEnvironmentContext.get().getParser() instanceof ExtendedParser) { + ExtendedParser extendedParser = + (ExtendedParser) CustomTableEnvironmentContext.get().getParser(); + permissions = + (SqlBasicCall) (extendedParser.getCustomParser()).parseExpression(permissionsStatement); + } else { + throw new RuntimeException("CustomParser is not set"); + } + } + } + + // add an alias + if (permissions != null && tableAlias != null) { + ImmutableList namesList = ImmutableList.of( + tableAlias, permissions.getOperandList().get(0).toString()); + permissions.getOperandList().set(0, new SqlIdentifier(namesList, null, new SqlParserPos(0, 0), null)); + } + + return buildWhereClause(where, permissions); + } + + /** Rebuild the where clause */ + private SqlNode buildWhereClause(SqlNode where, SqlBasicCall permissions) { + if (permissions != null) { + if (where == null) { + return permissions; + } + SqlBinaryOperator sqlBinaryOperator = + new SqlBinaryOperator(SqlKind.AND.name(), SqlKind.AND, 0, true, null, null, null); + SqlNode[] operands = new SqlNode[2]; + operands[0] = where; + operands[1] = permissions; + SqlParserPos sqlParserPos = new SqlParserPos(0, 0); + return new SqlBasicCall(sqlBinaryOperator, operands, sqlParserPos); + } + return where; + } + + @Override + public SqlOperator getOperator() { + return SqlSelectOperator.INSTANCE; + } + + @Override + public SqlKind getKind() { + return SqlKind.SELECT; + } + + @Override + public List getOperandList() { + return ImmutableNullableList.of( + keywordList, selectList, from, where, groupBy, having, windowDecls, orderBy, offset, fetch, hints); + } + + @Override + public void setOperand(int i, SqlNode operand) { + switch (i) { + case 0: + keywordList = Objects.requireNonNull((SqlNodeList) operand); + break; + case 1: + selectList = (SqlNodeList) operand; + break; + case 2: + from = operand; + break; + case 3: + where = operand; + break; + case 4: + groupBy = (SqlNodeList) operand; + break; + case 5: + having = operand; + break; + case 6: + windowDecls = Objects.requireNonNull((SqlNodeList) operand); + break; + case 7: + orderBy = (SqlNodeList) operand; + break; + case 8: + offset = operand; + break; + case 9: + fetch = operand; + break; + default: + throw new AssertionError(i); + } + } + + public final boolean isDistinct() { + return getModifierNode(SqlSelectKeyword.DISTINCT) != null; + } + + public final SqlNode getModifierNode(SqlSelectKeyword modifier) { + for (SqlNode keyword : keywordList) { + SqlSelectKeyword keyword2 = ((SqlLiteral) keyword).symbolValue(SqlSelectKeyword.class); + if (keyword2 == modifier) { + return keyword; + } + } + return null; + } + + public final SqlNode getFrom() { + return from; + } + + public void setFrom(SqlNode from) { + this.from = from; + } + + public final SqlNodeList getGroup() { + return groupBy; + } + + public void setGroupBy(SqlNodeList groupBy) { + this.groupBy = groupBy; + } + + public final SqlNode getHaving() { + return having; + } + + public void setHaving(SqlNode having) { + this.having = having; + } + + public final SqlNodeList getSelectList() { + return selectList; + } + + public void setSelectList(SqlNodeList selectList) { + this.selectList = selectList; + } + + public final SqlNode getWhere() { + return where; + } + + public void setWhere(SqlNode whereClause) { + this.where = whereClause; + } + + @Nonnull + public final SqlNodeList getWindowList() { + return windowDecls; + } + + public final SqlNodeList getOrderList() { + return orderBy; + } + + public void setOrderBy(SqlNodeList orderBy) { + this.orderBy = orderBy; + } + + public final SqlNode getOffset() { + return offset; + } + + public void setOffset(SqlNode offset) { + this.offset = offset; + } + + public final SqlNode getFetch() { + return fetch; + } + + public void setFetch(SqlNode fetch) { + this.fetch = fetch; + } + + public void setHints(SqlNodeList hints) { + this.hints = hints; + } + + public SqlNodeList getHints() { + return this.hints; + } + + public boolean hasHints() { + // The hints may be passed as null explicitly. + return this.hints != null && this.hints.size() > 0; + } + + @Override + public void validate(SqlValidator validator, SqlValidatorScope scope) { + validator.validateQuery(this, scope, validator.getUnknownType()); + } + + /** Override SqlCall, to introduce a sub-query frame. */ + @Override + public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { + if (!writer.inQuery()) { + // If this SELECT is the topmost item in a sub-query, introduce a new + // frame. (The topmost item in the sub-query might be a UNION or + // ORDER. In this case, we don't need a wrapper frame.) + final SqlWriter.Frame frame = writer.startList(SqlWriter.FrameTypeEnum.SUB_QUERY, "(", ")"); + writer.getDialect().unparseCall(writer, this, 0, 0); + writer.endList(frame); + } else { + writer.getDialect().unparseCall(writer, this, leftPrec, rightPrec); + } + } + + public boolean hasOrderBy() { + return orderBy != null && orderBy.size() != 0; + } + + public boolean hasWhere() { + return where != null; + } + + public boolean isKeywordPresent(SqlSelectKeyword targetKeyWord) { + return getModifierNode(targetKeyWord) != null; + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java b/dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java new file mode 100644 index 0000000000..1ca7b4a156 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java @@ -0,0 +1,262 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.flink.client.program; + +import static org.apache.flink.util.Preconditions.checkState; + +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.dag.Pipeline; +import org.apache.flink.client.FlinkPipelineTranslationUtil; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.optimizer.CompilerException; +import org.apache.flink.runtime.jobgraph.JobGraph; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.PrintStream; +import java.net.MalformedURLException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.FileSystems; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Utility class for {@link PackagedProgram} related operations. + */ +public enum PackagedProgramUtils { + ; + + private static final String PYTHON_GATEWAY_CLASS_NAME = "org.apache.flink.client.python.PythonGatewayServer"; + + private static final String PYTHON_DRIVER_CLASS_NAME = "org.apache.flink.client.python.PythonDriver"; + + /** + * Creates a {@link JobGraph} with a specified {@link JobID} from the given {@link + * PackagedProgram}. + * + * @param packagedProgram to extract the JobGraph from + * @param configuration to use for the optimizer and job graph generator + * @param defaultParallelism for the JobGraph + * @param jobID the pre-generated job id + * @return JobGraph extracted from the PackagedProgram + * @throws ProgramInvocationException if the JobGraph generation failed + */ + public static JobGraph createJobGraph( + PackagedProgram packagedProgram, + Configuration configuration, + int defaultParallelism, + @Nullable JobID jobID, + boolean suppressOutput) + throws ProgramInvocationException { + final Pipeline pipeline = + getPipelineFromProgram(packagedProgram, configuration, defaultParallelism, suppressOutput); + final JobGraph jobGraph = FlinkPipelineTranslationUtil.getJobGraphUnderUserClassLoader( + packagedProgram.getUserCodeClassLoader(), pipeline, configuration, defaultParallelism); + if (jobID != null) { + jobGraph.setJobID(jobID); + } + jobGraph.addJars(packagedProgram.getJobJarAndDependencies()); + jobGraph.setClasspaths(packagedProgram.getClasspaths()); + jobGraph.setSavepointRestoreSettings(packagedProgram.getSavepointSettings()); + + return jobGraph; + } + + /** + * Creates a {@link JobGraph} with a random {@link JobID} from the given {@link + * PackagedProgram}. + * + * @param packagedProgram to extract the JobGraph from + * @param configuration to use for the optimizer and job graph generator + * @param defaultParallelism for the JobGraph + * @param suppressOutput Whether to suppress stdout/stderr during interactive JobGraph creation. + * @return JobGraph extracted from the PackagedProgram + * @throws ProgramInvocationException if the JobGraph generation failed + */ + public static JobGraph createJobGraph( + PackagedProgram packagedProgram, + Configuration configuration, + int defaultParallelism, + boolean suppressOutput) + throws ProgramInvocationException { + return createJobGraph(packagedProgram, configuration, defaultParallelism, null, suppressOutput); + } + + public static Pipeline getPipelineFromProgram( + PackagedProgram program, Configuration configuration, int parallelism, boolean suppressOutput) + throws CompilerException, ProgramInvocationException { + final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader(); + + Thread.currentThread().setContextClassLoader(program.getUserCodeClassLoader()); + + final PrintStream originalOut = System.out; + final PrintStream originalErr = System.err; + final ByteArrayOutputStream stdOutBuffer; + final ByteArrayOutputStream stdErrBuffer; + + if (suppressOutput) { + // temporarily write STDERR and STDOUT to a byte array. + stdOutBuffer = new ByteArrayOutputStream(); + System.setOut(new PrintStream(stdOutBuffer)); + stdErrBuffer = new ByteArrayOutputStream(); + System.setErr(new PrintStream(stdErrBuffer)); + } else { + stdOutBuffer = null; + stdErrBuffer = null; + } + + // temporary hack to support the optimizer plan preview + OptimizerPlanEnvironment benv = + new OptimizerPlanEnvironment(configuration, program.getUserCodeClassLoader(), parallelism); + benv.setAsContext(); + StreamPlanEnvironment senv = + new StreamPlanEnvironment(configuration, program.getUserCodeClassLoader(), parallelism); + senv.setAsContext(); + + try { + program.invokeInteractiveModeForExecution(); + } catch (Throwable t) { + if (benv.getPipeline() != null) { + return benv.getPipeline(); + } + + if (senv.getPipeline() != null) { + return senv.getPipeline(); + } + + if (t instanceof ProgramInvocationException) { + throw t; + } + + throw generateException(program, "The program caused an error: ", t, stdOutBuffer, stdErrBuffer); + } finally { + benv.unsetAsContext(); + senv.unsetAsContext(); + if (suppressOutput) { + System.setOut(originalOut); + System.setErr(originalErr); + } + Thread.currentThread().setContextClassLoader(contextClassLoader); + } + + throw generateException( + program, + "The program plan could not be fetched - the program aborted pre-maturely. " + + "The root cause may be that the main method doesn't call `env.execute()` or `env.executeAsync()`.", + null, + stdOutBuffer, + stdErrBuffer); + } + + public static Boolean isPython(String entryPointClassName) { + return (entryPointClassName != null) + && (entryPointClassName.equals(PYTHON_DRIVER_CLASS_NAME) + || entryPointClassName.equals(PYTHON_GATEWAY_CLASS_NAME)); + } + + public static boolean isPython(String[] programArguments) { + return CollectionUtils.containsAny( + Arrays.asList(programArguments), Arrays.asList("-py", "-pym", "--python", "--pyModule")); + } + + public static URL getPythonJar() { + String flinkOptPath = System.getenv(ConfigConstants.ENV_FLINK_OPT_DIR); + final List pythonJarPath = new ArrayList<>(); + try { + if (flinkOptPath == null) { + Class pyClass = Class.forName( + PYTHON_DRIVER_CLASS_NAME, false, Thread.currentThread().getContextClassLoader()); + return pyClass.getProtectionDomain().getCodeSource().getLocation(); + } else { + Files.walkFileTree(FileSystems.getDefault().getPath(flinkOptPath), new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + FileVisitResult result = super.visitFile(file, attrs); + if (file.getFileName().toString().startsWith("flink-python")) { + pythonJarPath.add(file); + } + return result; + } + }); + } + } catch (Exception e) { + throw new RuntimeException( + "Exception encountered during finding the flink-python jar. This should not happen.", e); + } + + if (pythonJarPath.size() != 1) { + throw new RuntimeException("Found " + pythonJarPath.size() + " flink-python jar."); + } + + try { + return pythonJarPath.get(0).toUri().toURL(); + } catch (MalformedURLException e) { + throw new RuntimeException("URL is invalid. This should not happen.", e); + } + } + + public static String getPythonDriverClassName() { + return PYTHON_DRIVER_CLASS_NAME; + } + + public static URI resolveURI(String path) throws URISyntaxException { + final URI uri = new URI(path); + if (uri.getScheme() != null) { + return uri; + } + return new File(path).getAbsoluteFile().toURI(); + } + + private static ProgramInvocationException generateException( + PackagedProgram program, + String msg, + @Nullable Throwable cause, + @Nullable ByteArrayOutputStream stdoutBuffer, + @Nullable ByteArrayOutputStream stderrBuffer) { + checkState( + (stdoutBuffer != null) == (stderrBuffer != null), + "Stderr/Stdout should either both be set or both be null."); + + final String stdout = (stdoutBuffer != null) ? stdoutBuffer.toString() : ""; + final String stderr = (stderrBuffer != null) ? stderrBuffer.toString() : ""; + + return new ProgramInvocationException( + String.format( + "%s\n\nClasspath: %s\n\nSystem.out: %s\n\nSystem.err: %s", + msg, + program.getJobJarAndDependencies(), + stdout.length() == 0 ? "(none)" : stdout, + stderr.length() == 0 ? "(none)" : stderr), + cause); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/yarn/Utils.java b/dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/yarn/Utils.java new file mode 100644 index 0000000000..16a100f6fa --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/yarn/Utils.java @@ -0,0 +1,773 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.flink.yarn; + +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.yarn.YarnConfigKeys.ENV_FLINK_CLASSPATH; +import static org.apache.flink.yarn.YarnConfigKeys.LOCAL_RESOURCE_DESCRIPTOR_SEPARATOR; +import static org.apache.flink.yarn.configuration.YarnConfigOptions.YARN_CONTAINER_START_COMMAND_TEMPLATE; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigUtils; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.clusterframework.ContaineredTaskManagerParameters; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessSpec; +import org.apache.flink.runtime.clusterframework.TaskExecutorProcessUtils; +import org.apache.flink.runtime.util.HadoopUtils; +import org.apache.flink.runtime.util.config.memory.ProcessMemoryUtils; +import org.apache.flink.util.StringUtils; +import org.apache.flink.util.function.FunctionWithException; +import org.apache.flink.yarn.configuration.YarnConfigOptions; +import org.apache.flink.yarn.configuration.YarnResourceManagerDriverConfiguration; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.DataOutputBuffer; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.util.StringInterner; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; +import org.apache.hadoop.yarn.api.records.ApplicationAccessType; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.LocalResource; +import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.URL; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; +import org.apache.hadoop.yarn.util.Records; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.stream.Stream; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import cn.hutool.core.util.StrUtil; + +/** + * Utility class that provides helper methods to work with Apache Hadoop YARN. + */ +public final class Utils { + + private static final Logger LOG = LoggerFactory.getLogger(Utils.class); + + /** + * KRB5 file name populated in YARN container for secure IT run. + */ + public static final String KRB5_FILE_NAME = "krb5.conf"; + + /** + * Yarn site xml file name populated in YARN container for secure IT run. + */ + public static final String YARN_SITE_FILE_NAME = "yarn-site.xml"; + + /** + * Constant representing a wildcard access control list. + */ + private static final String WILDCARD_ACL = "*"; + + /** + * The prefixes that Flink adds to the YARN config. + */ + private static final String[] FLINK_CONFIG_PREFIXES = {"flink.yarn."}; + + @VisibleForTesting + static final String YARN_RM_FAIR_SCHEDULER_CLAZZ = + "org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.FairScheduler"; + + @VisibleForTesting + static final String YARN_RM_SLS_FAIR_SCHEDULER_CLAZZ = "org.apache.hadoop.yarn.sls.scheduler.SLSFairScheduler"; + + @VisibleForTesting + static final String YARN_RM_INCREMENT_ALLOCATION_MB_KEY = "yarn.resource-types.memory-mb.increment-allocation"; + + @VisibleForTesting + static final String YARN_RM_INCREMENT_ALLOCATION_MB_LEGACY_KEY = "yarn.scheduler.increment-allocation-mb"; + + private static final int DEFAULT_YARN_RM_INCREMENT_ALLOCATION_MB = 1024; + + @VisibleForTesting + static final String YARN_RM_INCREMENT_ALLOCATION_VCORES_KEY = "yarn.resource-types.vcores.increment-allocation"; + + @VisibleForTesting + static final String YARN_RM_INCREMENT_ALLOCATION_VCORES_LEGACY_KEY = "yarn.scheduler.increment-allocation-vcores"; + + @VisibleForTesting + static final String IGNORE_UNRECOGNIZED_VM_OPTIONS = "-XX:+IgnoreUnrecognizedVMOptions"; + + private static final int DEFAULT_YARN_RM_INCREMENT_ALLOCATION_VCORES = 1; + + public static void setupYarnClassPath(Configuration conf, Map appMasterEnv) { + addToEnvironment(appMasterEnv, Environment.CLASSPATH.name(), appMasterEnv.get(ENV_FLINK_CLASSPATH)); + String[] applicationClassPathEntries = conf.getStrings( + YarnConfiguration.YARN_APPLICATION_CLASSPATH, + Stream.of(YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH) + .map(x -> StrUtil.removeAll(x, "%")) + .map(x -> "$".equals(StrUtil.subPre(x, 1)) ? x : "$" + x) + .toArray(String[]::new)); + for (String c : applicationClassPathEntries) { + addToEnvironment(appMasterEnv, Environment.CLASSPATH.name(), c.trim()); + } + } + + /** + * Deletes the YARN application files, e.g., Flink binaries, libraries, etc., from the remote + * filesystem. + * + * @param applicationFilesDir The application files directory. + */ + public static void deleteApplicationFiles(final String applicationFilesDir) { + if (!StringUtils.isNullOrWhitespaceOnly(applicationFilesDir)) { + final org.apache.flink.core.fs.Path path = new org.apache.flink.core.fs.Path(applicationFilesDir); + try { + final org.apache.flink.core.fs.FileSystem fileSystem = path.getFileSystem(); + if (!fileSystem.delete(path, true)) { + LOG.error("Deleting yarn application files under {} was unsuccessful.", applicationFilesDir); + } + } catch (final IOException e) { + LOG.error("Could not properly delete yarn application files directory {}.", applicationFilesDir, e); + } + } else { + LOG.debug("No yarn application files directory set. Therefore, cannot clean up the data."); + } + } + + /** + * Creates a YARN resource for the remote object at the given location. + * + * @param remoteRsrcPath remote location of the resource + * @param resourceSize size of the resource + * @param resourceModificationTime last modification time of the resource + * @return YARN resource + */ + static LocalResource registerLocalResource( + Path remoteRsrcPath, + long resourceSize, + long resourceModificationTime, + LocalResourceVisibility resourceVisibility, + LocalResourceType resourceType) { + LocalResource localResource = Records.newRecord(LocalResource.class); + localResource.setResource(URL.fromURI(remoteRsrcPath.toUri())); + localResource.setSize(resourceSize); + localResource.setTimestamp(resourceModificationTime); + localResource.setType(resourceType); + localResource.setVisibility(resourceVisibility); + return localResource; + } + + /** + * Creates a YARN resource for the remote object at the given location. + * + * @param fs remote filesystem + * @param remoteRsrcPath resource path to be registered + * @return YARN resource + */ + private static LocalResource registerLocalResource( + FileSystem fs, Path remoteRsrcPath, LocalResourceType resourceType) throws IOException { + FileStatus jarStat = fs.getFileStatus(remoteRsrcPath); + return registerLocalResource( + remoteRsrcPath, + jarStat.getLen(), + jarStat.getModificationTime(), + LocalResourceVisibility.APPLICATION, + resourceType); + } + + /** + * Copied method from org.apache.hadoop.yarn.util.Apps. It was broken by YARN-1824 (2.4.0) and + * fixed for 2.4.1 by https://issues.apache.org/jira/browse/YARN-1931 + */ + public static void addToEnvironment(Map environment, String variable, String value) { + String val = environment.get(variable); + if (val == null) { + val = value; + } else { + val = val + ":" + value; + } + environment.put(StringInterner.weakIntern(variable), StringInterner.weakIntern(val)); + } + + /** + * Resolve keytab path either as absolute path or relative to working directory. + * + * @param workingDir current working directory + * @param keytabPath configured keytab path. + * @return resolved keytab path, or null if not found. + */ + public static String resolveKeytabPath(String workingDir, String keytabPath) { + String keytab = null; + if (keytabPath != null) { + File f; + f = new File(keytabPath); + if (f.exists()) { + keytab = f.getAbsolutePath(); + LOG.info("Resolved keytab path: {}", keytab); + } else { + // try using relative paths, this is the case when the keytab was shipped + // as a local resource + f = new File(workingDir, keytabPath); + if (f.exists()) { + keytab = f.getAbsolutePath(); + LOG.info("Resolved keytab path: {}", keytab); + } else { + LOG.warn("Could not resolve keytab path with: {}", keytabPath); + keytab = null; + } + } + } + return keytab; + } + + /** + * Private constructor to prevent instantiation. + */ + private Utils() { + throw new RuntimeException(); + } + + /** + * Creates the launch context, which describes how to bring up a TaskExecutor / TaskManager + * process in an allocated YARN container. + * + *

      This code is extremely YARN specific and registers all the resources that the TaskExecutor + * needs (such as JAR file, config file, ...) and all environment variables in a YARN container + * launch context. The launch context then ensures that those resources will be copied into the + * containers transient working directory. + * + * @param flinkConfig The Flink configuration object. + * @param yarnConfig The YARN configuration object. + * @param configuration The YarnResourceManagerDriver configurations. + * @param tmParams The TaskExecutor container memory parameters. + * @param taskManagerDynamicProperties The dynamic configurations to be updated for the + * TaskExecutors based on client uploaded Flink config. + * @param workingDirectory The current application master container's working directory. + * @param taskManagerMainClass The class with the main method. + * @param log The logger. + * @return The launch context for the TaskManager processes. + * @throws Exception Thrown if the launch context could not be created, for example if the + * resources could not be copied. + */ + static ContainerLaunchContext createTaskExecutorContext( + org.apache.flink.configuration.Configuration flinkConfig, + YarnConfiguration yarnConfig, + YarnResourceManagerDriverConfiguration configuration, + ContaineredTaskManagerParameters tmParams, + String taskManagerDynamicProperties, + String workingDirectory, + Class taskManagerMainClass, + Logger log) + throws Exception { + + // get and validate all relevant variables + + String remoteFlinkJarPath = checkNotNull( + configuration.getFlinkDistJar(), "Environment variable %s not set", YarnConfigKeys.FLINK_DIST_JAR); + + String shipListString = checkNotNull( + configuration.getClientShipFiles(), + "Environment variable %s not set", + YarnConfigKeys.ENV_CLIENT_SHIP_FILES); + + final String remoteKeytabPath = configuration.getRemoteKeytabPath(); + final String localKeytabPath = configuration.getLocalKeytabPath(); + final String keytabPrincipal = configuration.getKeytabPrinciple(); + final String remoteYarnConfPath = configuration.getYarnSiteXMLPath(); + final String remoteKrb5Path = configuration.getKrb5Path(); + + if (log.isDebugEnabled()) { + log.debug("TM:remote keytab path obtained {}", remoteKeytabPath); + log.debug("TM:local keytab path obtained {}", localKeytabPath); + log.debug("TM:keytab principal obtained {}", keytabPrincipal); + log.debug("TM:remote yarn conf path obtained {}", remoteYarnConfPath); + log.debug("TM:remote krb5 path obtained {}", remoteKrb5Path); + } + + String classPathString = checkNotNull( + configuration.getFlinkClasspath(), + "Environment variable %s not set", + YarnConfigKeys.ENV_FLINK_CLASSPATH); + + // register keytab + LocalResource keytabResource = null; + if (remoteKeytabPath != null) { + log.info("TM:Adding keytab {} to the container local resource bucket", remoteKeytabPath); + Path keytabPath = new Path(remoteKeytabPath); + FileSystem fs = keytabPath.getFileSystem(yarnConfig); + keytabResource = registerLocalResource(fs, keytabPath, LocalResourceType.FILE); + } + + // To support Yarn Secure Integration Test Scenario + LocalResource yarnConfResource = null; + if (remoteYarnConfPath != null) { + log.info("TM:Adding remoteYarnConfPath {} to the container local resource bucket", remoteYarnConfPath); + Path yarnConfPath = new Path(remoteYarnConfPath); + FileSystem fs = yarnConfPath.getFileSystem(yarnConfig); + yarnConfResource = registerLocalResource(fs, yarnConfPath, LocalResourceType.FILE); + } + + // register krb5.conf + LocalResource krb5ConfResource = null; + boolean hasKrb5 = false; + if (remoteKrb5Path != null) { + log.info("Adding remoteKrb5Path {} to the container local resource bucket", remoteKrb5Path); + Path krb5ConfPath = new Path(remoteKrb5Path); + FileSystem fs = krb5ConfPath.getFileSystem(yarnConfig); + krb5ConfResource = registerLocalResource(fs, krb5ConfPath, LocalResourceType.FILE); + hasKrb5 = true; + } + + Map taskManagerLocalResources = new HashMap<>(); + + // register Flink Jar with remote HDFS + final YarnLocalResourceDescriptor flinkDistLocalResourceDesc = + YarnLocalResourceDescriptor.fromString(remoteFlinkJarPath); + taskManagerLocalResources.put( + flinkDistLocalResourceDesc.getResourceKey(), flinkDistLocalResourceDesc.toLocalResource()); + + // To support Yarn Secure Integration Test Scenario + if (yarnConfResource != null) { + taskManagerLocalResources.put(YARN_SITE_FILE_NAME, yarnConfResource); + } + if (krb5ConfResource != null) { + taskManagerLocalResources.put(KRB5_FILE_NAME, krb5ConfResource); + } + if (keytabResource != null) { + taskManagerLocalResources.put(localKeytabPath, keytabResource); + } + + // prepare additional files to be shipped + decodeYarnLocalResourceDescriptorListFromString(shipListString) + .forEach(resourceDesc -> + taskManagerLocalResources.put(resourceDesc.getResourceKey(), resourceDesc.toLocalResource())); + + // now that all resources are prepared, we can create the launch context + + log.info("Creating container launch context for TaskManagers"); + + boolean hasLogback = new File(workingDirectory, "logback.xml").exists(); + boolean hasLog4j = new File(workingDirectory, "log4j.properties").exists(); + + String launchCommand = getTaskManagerShellCommand( + flinkConfig, + tmParams, + ".", + ApplicationConstants.LOG_DIR_EXPANSION_VAR, + hasLogback, + hasLog4j, + hasKrb5, + taskManagerMainClass, + taskManagerDynamicProperties); + + if (log.isDebugEnabled()) { + log.debug("Starting TaskManagers with command: " + launchCommand); + } else { + log.info("Starting TaskManagers"); + } + + ContainerLaunchContext ctx = Records.newRecord(ContainerLaunchContext.class); + ctx.setCommands(Collections.singletonList(launchCommand)); + ctx.setLocalResources(taskManagerLocalResources); + + Map containerEnv = new HashMap<>(); + containerEnv.putAll(tmParams.taskManagerEnv()); + + // add YARN classpath, etc to the container environment + containerEnv.put(ENV_FLINK_CLASSPATH, classPathString); + setupYarnClassPath(yarnConfig, containerEnv); + + containerEnv.put( + YarnConfigKeys.ENV_HADOOP_USER_NAME, + UserGroupInformation.getCurrentUser().getUserName()); + + if (remoteKeytabPath != null && localKeytabPath != null && keytabPrincipal != null) { + containerEnv.put(YarnConfigKeys.REMOTE_KEYTAB_PATH, remoteKeytabPath); + containerEnv.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, localKeytabPath); + containerEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, keytabPrincipal); + } else if (localKeytabPath != null && keytabPrincipal != null) { + containerEnv.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, localKeytabPath); + containerEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, keytabPrincipal); + } + + ctx.setEnvironment(containerEnv); + + setAclsFor(ctx, flinkConfig); + + // For TaskManager YARN container context, read the tokens from the jobmanager yarn + // container local file. + // NOTE: must read the tokens from the local file, not from the UGI context, because if UGI + // is login + // using Kerberos keytabs, there is no HDFS delegation token in the UGI context. + final String fileLocation = System.getenv(UserGroupInformation.HADOOP_TOKEN_FILE_LOCATION); + + if (fileLocation != null) { + log.debug("Adding security tokens to TaskExecutor's container launch context."); + + try (DataOutputBuffer dob = new DataOutputBuffer()) { + Credentials cred = Credentials.readTokenStorageFile( + new File(fileLocation), HadoopUtils.getHadoopConfiguration(flinkConfig)); + + // Filter out AMRMToken before setting the tokens to the TaskManager container + // context. + Credentials taskManagerCred = new Credentials(); + Collection> userTokens = cred.getAllTokens(); + for (Token token : userTokens) { + if (!token.getKind().equals(AMRMTokenIdentifier.KIND_NAME)) { + taskManagerCred.addToken(token.getService(), token); + } + } + + taskManagerCred.writeTokenStorageToStream(dob); + ByteBuffer securityTokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); + ctx.setTokens(securityTokens); + } catch (Throwable t) { + log.error("Failed to add Hadoop's security tokens.", t); + } + } else { + log.info("Could not set security tokens because Hadoop's token file location is unknown."); + } + + return ctx; + } + + /** + * Generates the shell command to start a task manager. + * + * @param flinkConfig The Flink configuration. + * @param tmParams Parameters for the task manager. + * @param configDirectory The configuration directory for the config.yaml + * @param logDirectory The log directory. + * @param hasLogback Uses logback? + * @param hasLog4j Uses log4j? + * @param mainClass The main class to start with. + * @return A String containing the task manager startup command. + */ + public static String getTaskManagerShellCommand( + org.apache.flink.configuration.Configuration flinkConfig, + ContaineredTaskManagerParameters tmParams, + String configDirectory, + String logDirectory, + boolean hasLogback, + boolean hasLog4j, + boolean hasKrb5, + Class mainClass, + String mainArgs) { + + final Map startCommandValues = new HashMap<>(); + startCommandValues.put("java", "$JAVA_HOME/bin/java"); + + final TaskExecutorProcessSpec taskExecutorProcessSpec = tmParams.getTaskExecutorProcessSpec(); + startCommandValues.put("jvmmem", ProcessMemoryUtils.generateJvmParametersStr(taskExecutorProcessSpec)); + + List> jvmOptions = Arrays.asList( + CoreOptions.FLINK_DEFAULT_JVM_OPTIONS, + CoreOptions.FLINK_JVM_OPTIONS, + CoreOptions.FLINK_DEFAULT_TM_JVM_OPTIONS, + CoreOptions.FLINK_TM_JVM_OPTIONS); + startCommandValues.put("jvmopts", generateJvmOptsString(flinkConfig, jvmOptions, hasKrb5)); + + String logging = ""; + if (hasLogback || hasLog4j) { + logging = "-Dlog.file=" + logDirectory + "/taskmanager.log"; + if (hasLogback) { + logging += " -Dlogback.configurationFile=file:" + configDirectory + "/logback.xml"; + } + if (hasLog4j) { + logging += " -Dlog4j.configuration=file:" + configDirectory + "/log4j.properties"; + logging += " -Dlog4j.configurationFile=file:" + configDirectory + "/log4j.properties"; + } + } + + startCommandValues.put("logging", logging); + startCommandValues.put("class", mainClass.getName()); + startCommandValues.put( + "redirects", "1> " + logDirectory + "/taskmanager.out " + "2> " + logDirectory + "/taskmanager.err"); + + String argsStr = TaskExecutorProcessUtils.generateDynamicConfigsStr(taskExecutorProcessSpec) + + " --configDir " + + configDirectory; + if (!mainArgs.isEmpty()) { + argsStr += " " + mainArgs; + } + startCommandValues.put("args", argsStr); + + final String commandTemplate = flinkConfig.get(YARN_CONTAINER_START_COMMAND_TEMPLATE); + String startCommand = getStartCommand(commandTemplate, startCommandValues); + LOG.debug("TaskManager start command: " + startCommand); + + return startCommand; + } + + /** + * Replaces placeholders in the template start command with values from startCommandValues. + * + *

      If the default template {@link + * ConfigConstants#DEFAULT_YARN_CONTAINER_START_COMMAND_TEMPLATE} is used, the following keys + * must be present in the map or the resulting command will still contain placeholders: + * + *

        + *
      • java = path to the Java executable + *
      • jvmmem = JVM memory limits and tweaks + *
      • jvmopts = misc options for the Java VM + *
      • logging = logging-related configuration settings + *
      • class = main class to execute + *
      • args = arguments for the main class + *
      • redirects = output redirects + *
      + * + * @param template a template start command with placeholders + * @param startCommandValues a replacement map placeholder -> value + * @return the start command with placeholders filled in + */ + public static String getStartCommand(String template, Map startCommandValues) { + for (Map.Entry variable : startCommandValues.entrySet()) { + template = template.replace("%" + variable.getKey() + "%", variable.getValue()) + .replace(" ", " ") + .trim(); + } + return template; + } + + public static String generateJvmOptsString( + org.apache.flink.configuration.Configuration conf, List> jvmOptions, boolean hasKrb5) { + StringBuilder javaOptsSb = new StringBuilder(); + for (ConfigOption option : jvmOptions) { + concatWithSpace(javaOptsSb, conf.get(option)); + } + concatWithSpace(javaOptsSb, IGNORE_UNRECOGNIZED_VM_OPTIONS); + + // krb5.conf file will be available as local resource in JM/TM container + if (hasKrb5) { + concatWithSpace(javaOptsSb, "-Djava.security.krb5.conf=krb5.conf"); + } + return javaOptsSb.toString().trim(); + } + + static boolean isRemotePath(String path) throws IOException { + org.apache.flink.core.fs.Path flinkPath = new org.apache.flink.core.fs.Path(path); + return flinkPath.getFileSystem().isDistributedFS(); + } + + private static List decodeYarnLocalResourceDescriptorListFromString(String resources) + throws Exception { + final List resourceDescriptors = new ArrayList<>(); + for (String shipResourceDescStr : resources.split(LOCAL_RESOURCE_DESCRIPTOR_SEPARATOR)) { + if (!shipResourceDescStr.isEmpty()) { + resourceDescriptors.add(YarnLocalResourceDescriptor.fromString(shipResourceDescStr)); + } + } + return resourceDescriptors; + } + + @VisibleForTesting + static Resource getUnitResource(YarnConfiguration yarnConfig) { + final int unitMemMB, unitVcore; + + final String yarnRmSchedulerClazzName = yarnConfig.get(YarnConfiguration.RM_SCHEDULER); + if (Objects.equals(yarnRmSchedulerClazzName, YARN_RM_FAIR_SCHEDULER_CLAZZ) + || Objects.equals(yarnRmSchedulerClazzName, YARN_RM_SLS_FAIR_SCHEDULER_CLAZZ)) { + String propMem = yarnConfig.get(YARN_RM_INCREMENT_ALLOCATION_MB_KEY); + String propVcore = yarnConfig.get(YARN_RM_INCREMENT_ALLOCATION_VCORES_KEY); + + unitMemMB = propMem != null + ? Integer.parseInt(propMem) + : yarnConfig.getInt( + YARN_RM_INCREMENT_ALLOCATION_MB_LEGACY_KEY, DEFAULT_YARN_RM_INCREMENT_ALLOCATION_MB); + unitVcore = propVcore != null + ? Integer.parseInt(propVcore) + : yarnConfig.getInt( + YARN_RM_INCREMENT_ALLOCATION_VCORES_LEGACY_KEY, + DEFAULT_YARN_RM_INCREMENT_ALLOCATION_VCORES); + } else { + unitMemMB = yarnConfig.getInt( + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, + YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB); + unitVcore = yarnConfig.getInt( + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, + YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES); + } + + return Resource.newInstance(unitMemMB, unitVcore); + } + + public static List getQualifiedRemoteProvidedLibDirs( + org.apache.flink.configuration.Configuration configuration, YarnConfiguration yarnConfiguration) + throws IOException { + + return getRemoteSharedLibPaths(configuration, pathStr -> { + final Path path = new Path(pathStr); + return path.getFileSystem(yarnConfiguration).makeQualified(path); + }); + } + + private static List getRemoteSharedLibPaths( + org.apache.flink.configuration.Configuration configuration, + FunctionWithException strToPathMapper) + throws IOException { + + final List providedLibDirs = + ConfigUtils.decodeListFromConfig(configuration, YarnConfigOptions.PROVIDED_LIB_DIRS, strToPathMapper); + + for (Path path : providedLibDirs) { + if (!Utils.isRemotePath(path.toString())) { + throw new IllegalArgumentException("The \"" + + YarnConfigOptions.PROVIDED_LIB_DIRS.key() + + "\" should only contain" + + " dirs accessible from all worker nodes, while the \"" + + path + + "\" is local."); + } + } + return providedLibDirs; + } + + public static boolean isUsrLibDirectory(final FileSystem fileSystem, final Path path) throws IOException { + final FileStatus fileStatus = fileSystem.getFileStatus(path); + // Use the Path obj from fileStatus to get rid of trailing slash + return fileStatus.isDirectory() + && ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR.equals( + fileStatus.getPath().getName()); + } + + public static Optional getQualifiedRemoteProvidedUsrLib( + org.apache.flink.configuration.Configuration configuration, YarnConfiguration yarnConfiguration) + throws IOException, IllegalArgumentException { + String usrlib = configuration.get(YarnConfigOptions.PROVIDED_USRLIB_DIR); + if (usrlib == null) { + return Optional.empty(); + } + final Path qualifiedUsrLibPath = FileSystem.get(yarnConfiguration).makeQualified(new Path(usrlib)); + checkArgument( + isRemotePath(qualifiedUsrLibPath.toString()), + "The \"%s\" must point to a remote dir " + "which is accessible from all worker nodes.", + YarnConfigOptions.PROVIDED_USRLIB_DIR.key()); + checkArgument( + isUsrLibDirectory(FileSystem.get(yarnConfiguration), qualifiedUsrLibPath), + "The \"%s\" should be named with \"%s\".", + YarnConfigOptions.PROVIDED_USRLIB_DIR.key(), + ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR); + return Optional.of(qualifiedUsrLibPath); + } + + public static YarnConfiguration getYarnAndHadoopConfiguration( + org.apache.flink.configuration.Configuration flinkConfig) { + final YarnConfiguration yarnConfig = getYarnConfiguration(flinkConfig); + yarnConfig.addResource(HadoopUtils.getHadoopConfiguration(flinkConfig)); + + return yarnConfig; + } + + /** + * Add additional config entries from the flink config to the yarn config. + * + * @param flinkConfig The Flink configuration object. + * @return The yarn configuration. + */ + public static YarnConfiguration getYarnConfiguration(org.apache.flink.configuration.Configuration flinkConfig) { + final YarnConfiguration yarnConfig = new YarnConfiguration(); + + for (String key : flinkConfig.keySet()) { + for (String prefix : FLINK_CONFIG_PREFIXES) { + if (key.startsWith(prefix)) { + String newKey = key.substring("flink.".length()); + String value = flinkConfig.getString(key, null); + yarnConfig.set(newKey, value); + LOG.debug("Adding Flink config entry for {} as {}={} to Yarn config", key, newKey, value); + } + } + } + + return yarnConfig; + } + + /** + * Sets the application ACLs for the given ContainerLaunchContext based on the values specified + * in the given Flink configuration. Only ApplicationAccessType.VIEW_APP and + * ApplicationAccessType.MODIFY_APP ACLs are set, and only if they are configured in the Flink + * configuration. If the viewAcls or modifyAcls string contains the WILDCARD_ACL constant, it + * will replace the entire string with the WILDCARD_ACL. The resulting map is then set as the + * application acls for the given container launch context. + * + * @param amContainer the ContainerLaunchContext to set the ACLs for. + * @param flinkConfig the Flink configuration to read the ACL values from. + */ + public static void setAclsFor( + ContainerLaunchContext amContainer, org.apache.flink.configuration.Configuration flinkConfig) { + Map acls = new HashMap<>(); + final String viewAcls = flinkConfig.get(YarnConfigOptions.APPLICATION_VIEW_ACLS); + final String modifyAcls = flinkConfig.get(YarnConfigOptions.APPLICATION_MODIFY_ACLS); + validateAclString(viewAcls); + validateAclString(modifyAcls); + + if (viewAcls != null && !viewAcls.isEmpty()) { + acls.put(ApplicationAccessType.VIEW_APP, viewAcls); + } + if (modifyAcls != null && !modifyAcls.isEmpty()) { + acls.put(ApplicationAccessType.MODIFY_APP, modifyAcls); + } + if (!acls.isEmpty()) { + amContainer.setApplicationACLs(acls); + } + } + + /* Validates the ACL string to ensure that it is either null or the wildcard ACL. */ + private static void validateAclString(String acl) { + if (acl != null && acl.contains("*") && !acl.equals("*")) { + throw new IllegalArgumentException(String.format( + "Invalid wildcard ACL %s. The ACL wildcard does not support regex. The only valid wildcard ACL is '*'.", + acl)); + } + } + + public static Path getPathFromLocalFile(File localFile) { + return new Path(localFile.toURI()); + } + + public static Path getPathFromLocalFilePathStr(String localPathStr) { + return getPathFromLocalFile(new File(localPathStr)); + } + + public static void concatWithSpace(StringBuilder sb, String value) { + if (value == null || value.isEmpty()) { + return; + } + sb.append(' '); + sb.append(value); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java b/dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java new file mode 100644 index 0000000000..f95bcd25bc --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/apache/flink/yarn/YarnClusterDescriptor.java @@ -0,0 +1,1801 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.apache.flink.yarn; + +import static org.apache.flink.client.deployment.application.ApplicationConfiguration.APPLICATION_MAIN_CLASS; +import static org.apache.flink.configuration.ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR; +import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_LIB_DIR; +import static org.apache.flink.configuration.ConfigConstants.ENV_FLINK_OPT_DIR; +import static org.apache.flink.configuration.ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX; +import static org.apache.flink.configuration.ResourceManagerOptions.CONTAINERIZED_TASK_MANAGER_ENV_PREFIX; +import static org.apache.flink.runtime.entrypoint.component.FileJobGraphRetriever.JOB_GRAPH_FILE_PATH; +import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.yarn.Utils.getPathFromLocalFile; +import static org.apache.flink.yarn.Utils.getPathFromLocalFilePathStr; +import static org.apache.flink.yarn.Utils.getStartCommand; +import static org.apache.flink.yarn.YarnConfigKeys.ENV_FLINK_CLASSPATH; +import static org.apache.flink.yarn.YarnConfigKeys.LOCAL_RESOURCE_DESCRIPTOR_SEPARATOR; +import static org.apache.flink.yarn.configuration.YarnConfigOptions.APP_MASTER_TOKEN_SERVICES; +import static org.apache.flink.yarn.configuration.YarnConfigOptions.YARN_CONTAINER_START_COMMAND_TEMPLATE; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.cache.DistributedCache; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.client.deployment.ClusterDeploymentException; +import org.apache.flink.client.deployment.ClusterDescriptor; +import org.apache.flink.client.deployment.ClusterRetrieveException; +import org.apache.flink.client.deployment.ClusterSpecification; +import org.apache.flink.client.deployment.application.ApplicationConfiguration; +import org.apache.flink.client.program.ClusterClientProvider; +import org.apache.flink.client.program.PackagedProgramUtils; +import org.apache.flink.client.program.rest.RestClusterClient; +import org.apache.flink.configuration.ConfigConstants; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.configuration.HighAvailabilityOptions; +import org.apache.flink.configuration.IllegalConfigurationException; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.configuration.ResourceManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SecurityOptions; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.core.plugin.PluginConfig; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.runtime.clusterframework.BootstrapTools; +import org.apache.flink.runtime.entrypoint.ClusterEntrypoint; +import org.apache.flink.runtime.entrypoint.ClusterEntrypointUtils; +import org.apache.flink.runtime.jobgraph.JobGraph; +import org.apache.flink.runtime.jobmanager.HighAvailabilityMode; +import org.apache.flink.runtime.jobmanager.JobManagerProcessSpec; +import org.apache.flink.runtime.jobmanager.JobManagerProcessUtils; +import org.apache.flink.runtime.security.token.DefaultDelegationTokenManager; +import org.apache.flink.runtime.security.token.DelegationTokenContainer; +import org.apache.flink.runtime.security.token.DelegationTokenManager; +import org.apache.flink.runtime.security.token.hadoop.HadoopDelegationTokenConverter; +import org.apache.flink.runtime.security.token.hadoop.KerberosLoginProvider; +import org.apache.flink.runtime.util.HadoopUtils; +import org.apache.flink.util.CollectionUtil; +import org.apache.flink.util.FlinkException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ShutdownHookUtil; +import org.apache.flink.util.StringUtils; +import org.apache.flink.util.function.FunctionUtils; +import org.apache.flink.yarn.configuration.YarnConfigOptions; +import org.apache.flink.yarn.configuration.YarnConfigOptionsInternal; +import org.apache.flink.yarn.configuration.YarnDeploymentTarget; +import org.apache.flink.yarn.configuration.YarnLogConfigUtil; +import org.apache.flink.yarn.entrypoint.YarnApplicationClusterEntryPoint; +import org.apache.flink.yarn.entrypoint.YarnJobClusterEntrypoint; +import org.apache.flink.yarn.entrypoint.YarnSessionClusterEntrypoint; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.yarn.api.ApplicationConstants; +import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; +import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; +import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; +import org.apache.hadoop.yarn.api.records.LocalResourceType; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.NodeState; +import org.apache.hadoop.yarn.api.records.Priority; +import org.apache.hadoop.yarn.api.records.QueueInfo; +import org.apache.hadoop.yarn.api.records.Resource; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; +import org.apache.hadoop.yarn.client.api.YarnClient; +import org.apache.hadoop.yarn.client.api.YarnClientApplication; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.exceptions.YarnException; +import org.apache.hadoop.yarn.util.ConverterUtils; +import org.apache.hadoop.yarn.util.Records; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.io.PrintStream; +import java.io.UnsupportedEncodingException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.net.URI; +import java.net.URLDecoder; +import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import javax.annotation.Nullable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** The descriptor with deployment information for deploying a Flink cluster on Yarn. */ +public class YarnClusterDescriptor implements ClusterDescriptor { + private static final Logger LOG = LoggerFactory.getLogger(YarnClusterDescriptor.class); + private static final String PATH_SEPARATOR = ":"; + + @VisibleForTesting + static final String IGNORE_UNRECOGNIZED_VM_OPTIONS = "-XX:+IgnoreUnrecognizedVMOptions"; + + private final YarnConfiguration yarnConfiguration; + + private final YarnClient yarnClient; + + private final YarnClusterInformationRetriever yarnClusterInformationRetriever; + + /** True if the descriptor must not shut down the YarnClient. */ + private final boolean sharedYarnClient; + + /** + * Lazily initialized list of files to ship. The path string for the files which is configured + * by {@link YarnConfigOptions#SHIP_FILES} will be converted to {@link Path} with schema and + * absolute path. + */ + private final List shipFiles = new LinkedList<>(); + + /** + * Lazily initialized list of archives to ship. The path string for the archives which is + * configured by {@link YarnConfigOptions#SHIP_ARCHIVES} will be converted to {@link Path} with + * schema and absolute path. + */ + private final List shipArchives = new LinkedList<>(); + + private final String yarnQueue; + + private Path flinkJarPath; + + private final Configuration flinkConfiguration; + + private final String customName; + + private final String nodeLabel; + + private final String applicationType; + + private YarnConfigOptions.UserJarInclusion userJarInclusion; + + public YarnClusterDescriptor( + Configuration flinkConfiguration, + YarnConfiguration yarnConfiguration, + YarnClient yarnClient, + YarnClusterInformationRetriever yarnClusterInformationRetriever, + boolean sharedYarnClient) { + + this.yarnConfiguration = Preconditions.checkNotNull(yarnConfiguration); + this.yarnClient = Preconditions.checkNotNull(yarnClient); + this.yarnClusterInformationRetriever = Preconditions.checkNotNull(yarnClusterInformationRetriever); + this.sharedYarnClient = sharedYarnClient; + + this.flinkConfiguration = Preconditions.checkNotNull(flinkConfiguration); + this.userJarInclusion = getUserJarInclusionMode(flinkConfiguration); + + adaptEnvSetting(flinkConfiguration, CoreOptions.FLINK_LOG_LEVEL, "ROOT_LOG_LEVEL"); + adaptEnvSetting(flinkConfiguration, CoreOptions.FLINK_LOG_MAX, "MAX_LOG_FILE_NUMBER"); + + getLocalFlinkDistPath(flinkConfiguration).ifPresent(this::setLocalJarPath); + decodeFilesToShipToCluster(flinkConfiguration, YarnConfigOptions.SHIP_FILES) + .ifPresent(this::addShipFiles); + decodeFilesToShipToCluster(flinkConfiguration, YarnConfigOptions.SHIP_ARCHIVES) + .ifPresent(this::addShipArchives); + + this.yarnQueue = flinkConfiguration.get(YarnConfigOptions.APPLICATION_QUEUE); + this.customName = flinkConfiguration.get(YarnConfigOptions.APPLICATION_NAME); + this.applicationType = flinkConfiguration.get(YarnConfigOptions.APPLICATION_TYPE); + this.nodeLabel = flinkConfiguration.get(YarnConfigOptions.NODE_LABEL); + } + + /** Adapt flink env setting. */ + private static void adaptEnvSetting(Configuration config, ConfigOption configOption, String envKey) { + config.getOptional(configOption).ifPresent(value -> { + config.setString(CONTAINERIZED_MASTER_ENV_PREFIX + envKey, String.valueOf(value)); + config.setString(CONTAINERIZED_TASK_MANAGER_ENV_PREFIX + envKey, String.valueOf(value)); + }); + } + + private Optional> decodeFilesToShipToCluster( + final Configuration configuration, final ConfigOption> configOption) { + checkNotNull(configuration); + checkNotNull(configOption); + + List files = ConfigUtils.decodeListFromConfig(configuration, configOption, this::createPathWithSchema); + return files.isEmpty() ? Optional.empty() : Optional.of(files); + } + + private Path createPathWithSchema(String path) { + return isWithoutSchema(new Path(path)) ? getPathFromLocalFilePathStr(path) : new Path(path); + } + + private boolean isWithoutSchema(Path path) { + return StringUtils.isNullOrWhitespaceOnly(path.toUri().getScheme()); + } + + private Optional getLocalFlinkDistPath(final Configuration configuration) { + final String localJarPath = configuration.get(YarnConfigOptions.FLINK_DIST_JAR); + if (localJarPath != null) { + return Optional.of(new Path(localJarPath)); + } + + LOG.info("No path for the flink jar passed. Using the location of " + getClass() + " to locate the jar"); + + // check whether it's actually a jar file --> when testing we execute this class without a + // flink-dist jar + final String decodedPath = getDecodedJarPath(); + return decodedPath.endsWith(".jar") ? Optional.of(getPathFromLocalFilePathStr(decodedPath)) : Optional.empty(); + } + + private String getDecodedJarPath() { + final String encodedJarPath = YarnClusterClientFactory.class + .getProtectionDomain() + .getCodeSource() + .getLocation() + .getPath(); + try { + return URLDecoder.decode(encodedJarPath, Charset.defaultCharset().name()); + } catch (UnsupportedEncodingException e) { + throw new RuntimeException("Couldn't decode the encoded Flink dist jar path: " + + encodedJarPath + + " You can supply a path manually via the command line."); + } + } + + @VisibleForTesting + List getShipFiles() { + return shipFiles; + } + + @VisibleForTesting + List getShipArchives() { + return shipArchives; + } + + public YarnClient getYarnClient() { + return yarnClient; + } + + /** + * The class to start the application master with. This class runs the main method in case of + * session cluster. + */ + protected String getYarnSessionClusterEntrypoint() { + return YarnSessionClusterEntrypoint.class.getName(); + } + + /** + * The class to start the application master with. This class runs the main method in case of + * the job cluster. + */ + protected String getYarnJobClusterEntrypoint() { + return YarnJobClusterEntrypoint.class.getName(); + } + + public Configuration getFlinkConfiguration() { + return flinkConfiguration; + } + + public void setLocalJarPath(Path localJarPath) { + if (!localJarPath.toString().endsWith("jar")) { + throw new IllegalArgumentException( + "The passed jar path ('" + localJarPath + "') does not end with the 'jar' extension"); + } + this.flinkJarPath = localJarPath; + } + + /** + * Adds the given files to the list of files to ship. + * + *

      Note that any file matching "flink-dist*.jar" will be excluded from the upload by + * {@link YarnApplicationFileUploader#registerMultipleLocalResources(Collection, String, + * LocalResourceType)} since we upload the Flink uber jar ourselves and do not need to deploy it + * multiple times. + * + * @param shipFiles files to ship + */ + public void addShipFiles(List shipFiles) { + checkArgument( + !isUsrLibDirIncludedInShipFiles(shipFiles, yarnConfiguration), + "User-shipped directories configured via : %s should not include %s.", + YarnConfigOptions.SHIP_FILES.key(), + ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR); + this.shipFiles.addAll(shipFiles); + } + + private void addShipArchives(List shipArchives) { + checkArgument( + isArchiveOnlyIncludedInShipArchiveFiles(shipArchives, yarnConfiguration), + "Directories or non-archive files are included."); + this.shipArchives.addAll(shipArchives); + } + + private static boolean isArchiveOnlyIncludedInShipArchiveFiles( + List shipFiles, YarnConfiguration yarnConfiguration) { + long archivedFileCount = shipFiles.stream() + .map(FunctionUtils.uncheckedFunction(path -> getFileStatus(path, yarnConfiguration))) + .filter(FileStatus::isFile) + .map(status -> status.getPath().getName().toLowerCase()) + .filter(name -> name.endsWith(".tar.gz") + || name.endsWith(".tar") + || name.endsWith(".tgz") + || name.endsWith(".dst") + || name.endsWith(".jar") + || name.endsWith(".zip")) + .count(); + return archivedFileCount == shipFiles.size(); + } + + private static FileStatus getFileStatus(Path path, YarnConfiguration yarnConfiguration) throws IOException { + return path.getFileSystem(yarnConfiguration).getFileStatus(path); + } + + private void isReadyForDeployment(ClusterSpecification clusterSpecification) throws Exception { + + if (this.flinkJarPath == null) { + throw new YarnDeploymentException("The Flink jar path is null"); + } + if (this.flinkConfiguration == null) { + throw new YarnDeploymentException("Flink configuration object has not been set"); + } + + // Check if we don't exceed YARN's maximum virtual cores. + final int numYarnMaxVcores = yarnClusterInformationRetriever.getMaxVcores(); + + int configuredAmVcores = flinkConfiguration.get(YarnConfigOptions.APP_MASTER_VCORES); + if (configuredAmVcores > numYarnMaxVcores) { + throw new IllegalConfigurationException(String.format( + "The number of requested virtual cores for application master %d" + + " exceeds the maximum number of virtual cores %d available in the Yarn Cluster.", + configuredAmVcores, numYarnMaxVcores)); + } + + int configuredVcores = + flinkConfiguration.get(YarnConfigOptions.VCORES, clusterSpecification.getSlotsPerTaskManager()); + // don't configure more than the maximum configured number of vcores + if (configuredVcores > numYarnMaxVcores) { + throw new IllegalConfigurationException(String.format( + "The number of requested virtual cores per node %d" + + " exceeds the maximum number of virtual cores %d available in the Yarn Cluster." + + " Please note that the number of virtual cores is set to the number of task slots by default" + + " unless configured in the Flink config with '%s.'", + configuredVcores, numYarnMaxVcores, YarnConfigOptions.VCORES.key())); + } + + // check if required Hadoop environment variables are set. If not, warn user + if (System.getenv("HADOOP_CONF_DIR") == null && System.getenv("YARN_CONF_DIR") == null) { + LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set. " + + "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + + "configuration for accessing YARN."); + } + } + + public String getNodeLabel() { + return nodeLabel; + } + + // ------------------------------------------------------------- + // Lifecycle management + // ------------------------------------------------------------- + + @Override + public void close() { + if (!sharedYarnClient) { + yarnClient.stop(); + } + } + + // ------------------------------------------------------------- + // ClusterClient overrides + // ------------------------------------------------------------- + + @Override + public ClusterClientProvider retrieve(ApplicationId applicationId) throws ClusterRetrieveException { + + try { + // check if required Hadoop environment variables are set. If not, warn user + if (System.getenv("HADOOP_CONF_DIR") == null && System.getenv("YARN_CONF_DIR") == null) { + LOG.warn("Neither the HADOOP_CONF_DIR nor the YARN_CONF_DIR environment variable is set." + + "The Flink YARN Client needs one of these to be set to properly load the Hadoop " + + "configuration for accessing YARN."); + } + + final ApplicationReport report = yarnClient.getApplicationReport(applicationId); + + if (report.getFinalApplicationStatus() != FinalApplicationStatus.UNDEFINED) { + // Flink cluster is not running anymore + LOG.error( + "The application {} doesn't run anymore. It has previously completed with final status: {}", + applicationId, + report.getFinalApplicationStatus()); + throw new RuntimeException("The Yarn application " + applicationId + " doesn't run anymore."); + } + + setClusterEntrypointInfoToConfig(report); + + return () -> { + try { + return new RestClusterClient<>(flinkConfiguration, report.getApplicationId()); + } catch (Exception e) { + throw new RuntimeException("Couldn't retrieve Yarn cluster", e); + } + }; + } catch (Exception e) { + throw new ClusterRetrieveException("Couldn't retrieve Yarn cluster", e); + } + } + + @Override + public ClusterClientProvider deploySessionCluster(ClusterSpecification clusterSpecification) + throws ClusterDeploymentException { + try { + return deployInternal( + clusterSpecification, "Flink session cluster", getYarnSessionClusterEntrypoint(), null, false); + } catch (Exception e) { + throw new ClusterDeploymentException("Couldn't deploy Yarn session cluster", e); + } + } + + @Override + public ClusterClientProvider deployApplicationCluster( + final ClusterSpecification clusterSpecification, final ApplicationConfiguration applicationConfiguration) + throws ClusterDeploymentException { + checkNotNull(clusterSpecification); + checkNotNull(applicationConfiguration); + + final YarnDeploymentTarget deploymentTarget = YarnDeploymentTarget.fromConfig(flinkConfiguration); + if (YarnDeploymentTarget.APPLICATION != deploymentTarget) { + throw new ClusterDeploymentException("Couldn't deploy Yarn Application Cluster." + + " Expected deployment.target=" + + YarnDeploymentTarget.APPLICATION.getName() + + " but actual one was \"" + + deploymentTarget.getName() + + "\""); + } + + applicationConfiguration.applyToConfiguration(flinkConfiguration); + + // No need to do pipelineJars validation if it is a PyFlink job. + if (!(PackagedProgramUtils.isPython(applicationConfiguration.getApplicationClassName()) + || PackagedProgramUtils.isPython(applicationConfiguration.getProgramArguments()))) { + final List pipelineJars = + flinkConfiguration.getOptional(PipelineOptions.JARS).orElse(Collections.emptyList()); + Preconditions.checkArgument(pipelineJars.size() == 1, "Should only have one jar"); + } + + try { + return deployInternal( + clusterSpecification, + "Flink Application Cluster", + YarnApplicationClusterEntryPoint.class.getName(), + null, + false); + } catch (Exception e) { + throw new ClusterDeploymentException("Couldn't deploy Yarn Application Cluster", e); + } + } + + @Override + public ClusterClientProvider deployJobCluster( + ClusterSpecification clusterSpecification, JobGraph jobGraph, boolean detached) + throws ClusterDeploymentException { + + LOG.warn( + "Job Clusters are deprecated since Flink 1.15. Please use an Application Cluster/Application Mode instead."); + try { + return deployInternal( + clusterSpecification, "Flink per-job cluster", getYarnJobClusterEntrypoint(), jobGraph, detached); + } catch (Exception e) { + throw new ClusterDeploymentException("Could not deploy Yarn job cluster.", e); + } + } + + @Override + public void killCluster(ApplicationId applicationId) throws FlinkException { + try { + yarnClient.killApplication(applicationId); + + try (final FileSystem fs = FileSystem.get(yarnConfiguration)) { + final Path applicationDir = + YarnApplicationFileUploader.getApplicationDirPath(getStagingDir(fs), applicationId); + + Utils.deleteApplicationFiles(applicationDir.toUri().toString()); + } + + } catch (YarnException | IOException e) { + throw new FlinkException("Could not kill the Yarn Flink cluster with id " + applicationId + '.', e); + } + } + + /** + * This method will block until the ApplicationMaster/JobManager have been deployed on YARN. + * + * @param clusterSpecification Initial cluster specification for the Flink cluster to be + * deployed + * @param applicationName name of the Yarn application to start + * @param yarnClusterEntrypoint Class name of the Yarn cluster entry point. + * @param jobGraph A job graph which is deployed with the Flink cluster, {@code null} if none + * @param detached True if the cluster should be started in detached mode + */ + private ClusterClientProvider deployInternal( + ClusterSpecification clusterSpecification, + String applicationName, + String yarnClusterEntrypoint, + @Nullable JobGraph jobGraph, + boolean detached) + throws Exception { + + final UserGroupInformation currentUser = UserGroupInformation.getCurrentUser(); + if (HadoopUtils.isKerberosSecurityEnabled(currentUser)) { + boolean useTicketCache = flinkConfiguration.get(SecurityOptions.KERBEROS_LOGIN_USETICKETCACHE); + + if (!HadoopUtils.areKerberosCredentialsValid(currentUser, useTicketCache)) { + throw new RuntimeException("Hadoop security with Kerberos is enabled but the login user " + + "does not have Kerberos credentials or delegation tokens!"); + } + + final boolean fetchToken = flinkConfiguration.get(SecurityOptions.KERBEROS_FETCH_DELEGATION_TOKEN); + final boolean yarnAccessFSEnabled = !CollectionUtil.isNullOrEmpty( + flinkConfiguration.get(SecurityOptions.KERBEROS_HADOOP_FILESYSTEMS_TO_ACCESS)); + if (!fetchToken && yarnAccessFSEnabled) { + throw new IllegalConfigurationException(String.format( + "When %s is disabled, %s must be disabled as well.", + SecurityOptions.KERBEROS_FETCH_DELEGATION_TOKEN.key(), + SecurityOptions.KERBEROS_HADOOP_FILESYSTEMS_TO_ACCESS.key())); + } + } + + isReadyForDeployment(clusterSpecification); + + // ------------------ Check if the specified queue exists -------------------- + + checkYarnQueues(yarnClient); + + // ------------------ Check if the YARN ClusterClient has the requested resources + // -------------- + + // Create application via yarnClient + final YarnClientApplication yarnApplication = yarnClient.createApplication(); + final GetNewApplicationResponse appResponse = yarnApplication.getNewApplicationResponse(); + + Resource maxRes = appResponse.getMaximumResourceCapability(); + + final ClusterResourceDescription freeClusterMem; + try { + freeClusterMem = getCurrentFreeClusterResources(yarnClient); + } catch (YarnException | IOException e) { + failSessionDuringDeployment(yarnClient, yarnApplication); + throw new YarnDeploymentException("Could not retrieve information about free cluster resources.", e); + } + + final int yarnMinAllocationMB = yarnConfiguration.getInt( + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, + YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB); + if (yarnMinAllocationMB <= 0) { + throw new YarnDeploymentException("The minimum allocation memory " + + "(" + + yarnMinAllocationMB + + " MB) configured via '" + + YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB + + "' should be greater than 0."); + } + + final ClusterSpecification validClusterSpecification; + try { + validClusterSpecification = + validateClusterResources(clusterSpecification, yarnMinAllocationMB, maxRes, freeClusterMem); + } catch (YarnDeploymentException yde) { + failSessionDuringDeployment(yarnClient, yarnApplication); + throw yde; + } + + LOG.info("Cluster specification: {}", validClusterSpecification); + + final ClusterEntrypoint.ExecutionMode executionMode = + detached ? ClusterEntrypoint.ExecutionMode.DETACHED : ClusterEntrypoint.ExecutionMode.NORMAL; + + flinkConfiguration.set(ClusterEntrypoint.INTERNAL_CLUSTER_EXECUTION_MODE, executionMode.toString()); + + ApplicationReport report = startAppMaster( + flinkConfiguration, + applicationName, + yarnClusterEntrypoint, + jobGraph, + yarnClient, + yarnApplication, + validClusterSpecification); + + // print the application id for user to cancel themselves. + if (detached) { + final ApplicationId yarnApplicationId = report.getApplicationId(); + logDetachedClusterInformation(yarnApplicationId, LOG); + } + + setClusterEntrypointInfoToConfig(report); + + return () -> { + try { + return new RestClusterClient<>(flinkConfiguration, report.getApplicationId()); + } catch (Exception e) { + throw new RuntimeException("Error while creating RestClusterClient.", e); + } + }; + } + + private ClusterSpecification validateClusterResources( + ClusterSpecification clusterSpecification, + int yarnMinAllocationMB, + Resource maximumResourceCapability, + ClusterResourceDescription freeClusterResources) + throws YarnDeploymentException { + + int jobManagerMemoryMb = clusterSpecification.getMasterMemoryMB(); + final int taskManagerMemoryMb = clusterSpecification.getTaskManagerMemoryMB(); + + logIfComponentMemNotIntegerMultipleOfYarnMinAllocation("JobManager", jobManagerMemoryMb, yarnMinAllocationMB); + logIfComponentMemNotIntegerMultipleOfYarnMinAllocation("TaskManager", taskManagerMemoryMb, yarnMinAllocationMB); + + // set the memory to minAllocationMB to do the next checks correctly + if (jobManagerMemoryMb < yarnMinAllocationMB) { + jobManagerMemoryMb = yarnMinAllocationMB; + } + + final String note = + "Please check the 'yarn.scheduler.maximum-allocation-mb' and the 'yarn.nodemanager.resource.memory-mb' configuration values\n"; + if (jobManagerMemoryMb > maximumResourceCapability.getMemorySize()) { + throw new YarnDeploymentException( + "The cluster does not have the requested resources for the JobManager available!\n" + + "Maximum Memory: " + + maximumResourceCapability.getMemorySize() + + "MB Requested: " + + jobManagerMemoryMb + + "MB. " + + note); + } + + if (taskManagerMemoryMb > maximumResourceCapability.getMemorySize()) { + throw new YarnDeploymentException( + "The cluster does not have the requested resources for the TaskManagers available!\n" + + "Maximum Memory: " + + maximumResourceCapability.getMemorySize() + + " Requested: " + + taskManagerMemoryMb + + "MB. " + + note); + } + + final String noteRsc = + "\nThe Flink YARN client will try to allocate the YARN session, but maybe not all TaskManagers are " + + "connecting from the beginning because the resources are currently not available in the cluster. " + + "The allocation might take more time than usual because the Flink YARN client needs to wait until " + + "the resources become available."; + + if (taskManagerMemoryMb > freeClusterResources.containerLimit) { + LOG.warn("The requested amount of memory for the TaskManagers (" + + taskManagerMemoryMb + + "MB) is more than " + + "the largest possible YARN container: " + + freeClusterResources.containerLimit + + noteRsc); + } + if (jobManagerMemoryMb > freeClusterResources.containerLimit) { + LOG.warn("The requested amount of memory for the JobManager (" + + jobManagerMemoryMb + + "MB) is more than " + + "the largest possible YARN container: " + + freeClusterResources.containerLimit + + noteRsc); + } + + return new ClusterSpecification.ClusterSpecificationBuilder() + .setMasterMemoryMB(jobManagerMemoryMb) + .setTaskManagerMemoryMB(taskManagerMemoryMb) + .setSlotsPerTaskManager(clusterSpecification.getSlotsPerTaskManager()) + .createClusterSpecification(); + } + + private void logIfComponentMemNotIntegerMultipleOfYarnMinAllocation( + String componentName, int componentMemoryMB, int yarnMinAllocationMB) { + int normalizedMemMB = + (componentMemoryMB + (yarnMinAllocationMB - 1)) / yarnMinAllocationMB * yarnMinAllocationMB; + if (normalizedMemMB <= 0) { + normalizedMemMB = yarnMinAllocationMB; + } + if (componentMemoryMB != normalizedMemMB) { + LOG.info( + "The configured {} memory is {} MB. YARN will allocate {} MB to make up an integer multiple of its " + + "minimum allocation memory ({} MB, configured via 'yarn.scheduler.minimum-allocation-mb'). The extra {} MB " + + "may not be used by Flink.", + componentName, + componentMemoryMB, + normalizedMemMB, + yarnMinAllocationMB, + normalizedMemMB - componentMemoryMB); + } + } + + private void checkYarnQueues(YarnClient yarnClient) { + try { + List queues = yarnClient.getAllQueues(); + if (queues.size() > 0 + && this.yarnQueue != null) { // check only if there are queues configured in yarn and for + // this session. + boolean queueFound = false; + for (QueueInfo queue : queues) { + if (queue.getQueueName().equals(this.yarnQueue) + || queue.getQueueName().equals("root." + this.yarnQueue)) { + queueFound = true; + break; + } + } + if (!queueFound) { + String queueNames = StringUtils.toQuotedListString(queues.toArray()); + LOG.warn("The specified queue '" + + this.yarnQueue + + "' does not exist. " + + "Available queues: " + + queueNames); + } + } else { + LOG.debug("The YARN cluster does not have any queues configured"); + } + } catch (Throwable e) { + LOG.warn("Error while getting queue information from YARN: " + e.getMessage()); + if (LOG.isDebugEnabled()) { + LOG.debug("Error details", e); + } + } + } + + private ApplicationReport startAppMaster( + Configuration configuration, + String applicationName, + String yarnClusterEntrypoint, + JobGraph jobGraph, + YarnClient yarnClient, + YarnClientApplication yarnApplication, + ClusterSpecification clusterSpecification) + throws Exception { + + // ------------------ Initialize the file systems ------------------------- + + org.apache.flink.core.fs.FileSystem.initialize( + configuration, PluginUtils.createPluginManagerFromRootFolder(configuration)); + + final FileSystem fs = FileSystem.get(yarnConfiguration); + + // hard coded check for the GoogleHDFS client because its not overriding the getScheme() + // method. + if (!fs.getClass().getSimpleName().equals("GoogleHadoopFileSystem") + && fs.getScheme().startsWith("file")) { + LOG.warn("The file system scheme is '" + + fs.getScheme() + + "'. This indicates that the " + + "specified Hadoop configuration path is wrong and the system is using the default Hadoop configuration values." + + "The Flink YARN client needs to store its files in a distributed file system"); + } + + ApplicationSubmissionContext appContext = yarnApplication.getApplicationSubmissionContext(); + + final List providedLibDirs = Utils.getQualifiedRemoteProvidedLibDirs(configuration, yarnConfiguration); + + final Optional providedUsrLibDir = + Utils.getQualifiedRemoteProvidedUsrLib(configuration, yarnConfiguration); + + Path stagingDirPath = getStagingDir(fs); + FileSystem stagingDirFs = stagingDirPath.getFileSystem(yarnConfiguration); + final YarnApplicationFileUploader fileUploader = YarnApplicationFileUploader.from( + stagingDirFs, stagingDirPath, providedLibDirs, appContext.getApplicationId(), getFileReplication()); + + // The files need to be shipped and added to classpath. + Set systemShipFiles = new HashSet<>(shipFiles); + + final String logConfigFilePath = configuration.get(YarnConfigOptionsInternal.APPLICATION_LOG_CONFIG_FILE); + if (logConfigFilePath != null) { + systemShipFiles.add(getPathFromLocalFilePathStr(logConfigFilePath)); + } + + // Set-up ApplicationSubmissionContext for the application + + final ApplicationId appId = appContext.getApplicationId(); + + // ------------------ Add Zookeeper namespace to local flinkConfiguration ------ + setHAClusterIdIfNotSet(configuration, appId); + + if (HighAvailabilityMode.isHighAvailabilityModeActivated(configuration)) { + // activate re-execution of failed applications + appContext.setMaxAppAttempts(configuration.getInteger( + YarnConfigOptions.APPLICATION_ATTEMPTS.key(), YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS)); + + activateHighAvailabilitySupport(appContext); + } else { + // set number of application retries to 1 in the default case + appContext.setMaxAppAttempts(configuration.getInteger(YarnConfigOptions.APPLICATION_ATTEMPTS.key(), 1)); + } + + final Set userJarFiles = new HashSet<>(); + if (jobGraph != null) { + userJarFiles.addAll(jobGraph.getUserJars().stream() + .map(f -> f.toUri()) + .map(Path::new) + .collect(Collectors.toSet())); + } + + final List jarUrls = ConfigUtils.decodeListFromConfig(configuration, PipelineOptions.JARS, URI::create); + if (jarUrls != null && YarnApplicationClusterEntryPoint.class.getName().equals(yarnClusterEntrypoint)) { + userJarFiles.addAll(jarUrls.stream().map(Path::new).collect(Collectors.toSet())); + } + + // only for per job mode + if (jobGraph != null) { + for (Map.Entry entry : + jobGraph.getUserArtifacts().entrySet()) { + // only upload local files + if (!Utils.isRemotePath(entry.getValue().filePath)) { + Path localPath = new Path(entry.getValue().filePath); + Tuple2 remoteFileInfo = fileUploader.uploadLocalFileToRemote(localPath, entry.getKey()); + jobGraph.setUserArtifactRemotePath(entry.getKey(), remoteFileInfo.f0.toString()); + } + } + + jobGraph.writeUserArtifactEntriesToConfiguration(); + } + + if (providedLibDirs == null || providedLibDirs.isEmpty()) { + addLibFoldersToShipFiles(systemShipFiles); + } + + // Register all files in provided lib dirs as local resources with public visibility + // and upload the remaining dependencies as local resources with APPLICATION visibility. + final List systemClassPaths = fileUploader.registerProvidedLocalResources(); + final List uploadedDependencies = + fileUploader.registerMultipleLocalResources(systemShipFiles, Path.CUR_DIR, LocalResourceType.FILE); + systemClassPaths.addAll(uploadedDependencies); + + // upload and register ship-only files + // Plugin files only need to be shipped and should not be added to classpath. + if (providedLibDirs == null || providedLibDirs.isEmpty()) { + Set shipOnlyFiles = new HashSet<>(); + addPluginsFoldersToShipFiles(shipOnlyFiles); + fileUploader.registerMultipleLocalResources(shipOnlyFiles, Path.CUR_DIR, LocalResourceType.FILE); + } + + if (!shipArchives.isEmpty()) { + fileUploader.registerMultipleLocalResources(shipArchives, Path.CUR_DIR, LocalResourceType.ARCHIVE); + } + + // only for application mode + // Python jar file only needs to be shipped and should not be added to classpath. + if (YarnApplicationClusterEntryPoint.class.getName().equals(yarnClusterEntrypoint) + && PackagedProgramUtils.isPython(configuration.get(APPLICATION_MAIN_CLASS))) { + fileUploader.registerMultipleLocalResources( + Collections.singletonList( + new Path(PackagedProgramUtils.getPythonJar().toURI())), + ConfigConstants.DEFAULT_FLINK_OPT_DIR, + LocalResourceType.FILE); + } + + // Upload and register user jars + final List userClassPaths = fileUploader.registerMultipleLocalResources( + userJarFiles, + userJarInclusion == YarnConfigOptions.UserJarInclusion.DISABLED + ? ConfigConstants.DEFAULT_FLINK_USR_LIB_DIR + : Path.CUR_DIR, + LocalResourceType.FILE); + + // usrlib in remote will be used first. + if (providedUsrLibDir.isPresent()) { + final List usrLibClassPaths = fileUploader.registerMultipleLocalResources( + Collections.singletonList(providedUsrLibDir.get()), Path.CUR_DIR, LocalResourceType.FILE); + userClassPaths.addAll(usrLibClassPaths); + } else if (ClusterEntrypointUtils.tryFindUserLibDirectory().isPresent()) { + // local usrlib will be automatically shipped if it exists and there is no remote + // usrlib. + final Set usrLibShipFiles = new HashSet<>(); + addUsrLibFolderToShipFiles(usrLibShipFiles); + final List usrLibClassPaths = fileUploader.registerMultipleLocalResources( + usrLibShipFiles.stream().map(e -> new Path(e.toURI())).collect(Collectors.toSet()), + Path.CUR_DIR, + LocalResourceType.FILE); + userClassPaths.addAll(usrLibClassPaths); + } + + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.ORDER) { + systemClassPaths.addAll(userClassPaths); + } + + // normalize classpath by sorting + Collections.sort(systemClassPaths); + Collections.sort(userClassPaths); + + // classpath assembler + StringBuilder classPathBuilder = new StringBuilder(); + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.FIRST) { + for (String userClassPath : userClassPaths) { + classPathBuilder.append(userClassPath).append(PATH_SEPARATOR); + } + } + for (String classPath : systemClassPaths) { + classPathBuilder.append(classPath).append(PATH_SEPARATOR); + } + + // Setup jar for ApplicationMaster + final YarnLocalResourceDescriptor localResourceDescFlinkJar = fileUploader.uploadFlinkDist(flinkJarPath); + classPathBuilder.append(localResourceDescFlinkJar.getResourceKey()).append(PATH_SEPARATOR); + + // write job graph to tmp file and add it to local resource + // TODO: server use user main method to generate job graph + if (jobGraph != null) { + File tmpJobGraphFile = null; + try { + tmpJobGraphFile = File.createTempFile(appId.toString(), null); + try (FileOutputStream output = new FileOutputStream(tmpJobGraphFile); + ObjectOutputStream obOutput = new ObjectOutputStream(output)) { + obOutput.writeObject(jobGraph); + } + + final String jobGraphFilename = "job.graph"; + configuration.set(JOB_GRAPH_FILE_PATH, jobGraphFilename); + + fileUploader.registerSingleLocalResource( + jobGraphFilename, new Path(tmpJobGraphFile.toURI()), "", LocalResourceType.FILE, true, false); + classPathBuilder.append(jobGraphFilename).append(PATH_SEPARATOR); + } catch (Exception e) { + LOG.warn("Add job graph to local resource fail."); + throw e; + } finally { + if (tmpJobGraphFile != null && !tmpJobGraphFile.delete()) { + LOG.warn("Fail to delete temporary file {}.", tmpJobGraphFile.toPath()); + } + } + } + + // Upload the flink configuration + // write out configuration file + File tmpConfigurationFile = null; + try { + String flinkConfigFileName = GlobalConfiguration.getFlinkConfFilename(); + tmpConfigurationFile = File.createTempFile(appId + "-" + flinkConfigFileName, null); + + // remove localhost bind hosts as they render production clusters unusable + removeLocalhostBindHostSetting(configuration, JobManagerOptions.BIND_HOST); + removeLocalhostBindHostSetting(configuration, TaskManagerOptions.BIND_HOST); + // this setting is unconditionally overridden anyway, so we remove it for clarity + configuration.removeConfig(TaskManagerOptions.HOST); + + BootstrapTools.writeConfiguration(configuration, tmpConfigurationFile); + + fileUploader.registerSingleLocalResource( + flinkConfigFileName, + new Path(tmpConfigurationFile.getAbsolutePath()), + "", + LocalResourceType.FILE, + true, + true); + classPathBuilder.append(flinkConfigFileName).append(PATH_SEPARATOR); + } finally { + if (tmpConfigurationFile != null && !tmpConfigurationFile.delete()) { + LOG.warn("Fail to delete temporary file {}.", tmpConfigurationFile.toPath()); + } + } + + if (userJarInclusion == YarnConfigOptions.UserJarInclusion.LAST) { + for (String userClassPath : userClassPaths) { + classPathBuilder.append(userClassPath).append(PATH_SEPARATOR); + } + } + + // To support Yarn Secure Integration Test Scenario + // In Integration test setup, the Yarn containers created by YarnMiniCluster does not have + // the Yarn site XML + // and KRB5 configuration files. We are adding these files as container local resources for + // the container + // applications (JM/TMs) to have proper secure cluster setup + Path remoteYarnSiteXmlPath = null; + if (System.getenv("IN_TESTS") != null) { + File f = new File(System.getenv("YARN_CONF_DIR"), Utils.YARN_SITE_FILE_NAME); + LOG.info("Adding Yarn configuration {} to the AM container local resource bucket", f.getAbsolutePath()); + Path yarnSitePath = new Path(f.getAbsolutePath()); + remoteYarnSiteXmlPath = fileUploader + .registerSingleLocalResource( + Utils.YARN_SITE_FILE_NAME, yarnSitePath, "", LocalResourceType.FILE, false, false) + .getPath(); + if (System.getProperty("java.security.krb5.conf") != null) { + configuration.set(SecurityOptions.KERBEROS_KRB5_PATH, System.getProperty("java.security.krb5.conf")); + } + } + + Path remoteKrb5Path = null; + boolean hasKrb5 = false; + String krb5Config = configuration.get(SecurityOptions.KERBEROS_KRB5_PATH); + if (!StringUtils.isNullOrWhitespaceOnly(krb5Config)) { + final File krb5 = new File(krb5Config); + LOG.info("Adding KRB5 configuration {} to the AM container local resource bucket", krb5.getAbsolutePath()); + final Path krb5ConfPath = new Path(krb5.getAbsolutePath()); + remoteKrb5Path = fileUploader + .registerSingleLocalResource( + Utils.KRB5_FILE_NAME, krb5ConfPath, "", LocalResourceType.FILE, false, false) + .getPath(); + hasKrb5 = true; + } + + Path remotePathKeytab = null; + String localizedKeytabPath = null; + String keytab = configuration.get(SecurityOptions.KERBEROS_LOGIN_KEYTAB); + if (keytab != null) { + boolean localizeKeytab = flinkConfiguration.get(YarnConfigOptions.SHIP_LOCAL_KEYTAB); + localizedKeytabPath = flinkConfiguration.get(YarnConfigOptions.LOCALIZED_KEYTAB_PATH); + if (localizeKeytab) { + // Localize the keytab to YARN containers via local resource. + LOG.info("Adding keytab {} to the AM container local resource bucket", keytab); + remotePathKeytab = fileUploader + .registerSingleLocalResource( + localizedKeytabPath, new Path(keytab), "", LocalResourceType.FILE, false, false) + .getPath(); + } else { + // // Assume Keytab is pre-installed in the container. + localizedKeytabPath = flinkConfiguration.get(YarnConfigOptions.LOCALIZED_KEYTAB_PATH); + } + } + + final JobManagerProcessSpec processSpec = + JobManagerProcessUtils.processSpecFromConfigWithNewOptionToInterpretLegacyHeap( + flinkConfiguration, JobManagerOptions.TOTAL_PROCESS_MEMORY); + final ContainerLaunchContext amContainer = + setupApplicationMasterContainer(yarnClusterEntrypoint, hasKrb5, processSpec); + + boolean fetchToken = configuration.get(SecurityOptions.DELEGATION_TOKENS_ENABLED); + KerberosLoginProvider kerberosLoginProvider = new KerberosLoginProvider(configuration); + if (kerberosLoginProvider.isLoginPossible(true)) { + setTokensFor(amContainer, fetchToken); + } else { + LOG.info("Cannot use kerberos delegation token manager, no valid kerberos credentials provided."); + } + + amContainer.setLocalResources(fileUploader.getRegisteredLocalResources()); + fileUploader.close(); + + Utils.setAclsFor(amContainer, flinkConfiguration); + + // Setup CLASSPATH and environment variables for ApplicationMaster + final Map appMasterEnv = generateApplicationMasterEnv( + fileUploader, classPathBuilder.toString(), localResourceDescFlinkJar.toString(), appId.toString()); + + if (localizedKeytabPath != null) { + appMasterEnv.put(YarnConfigKeys.LOCAL_KEYTAB_PATH, localizedKeytabPath); + String principal = configuration.get(SecurityOptions.KERBEROS_LOGIN_PRINCIPAL); + appMasterEnv.put(YarnConfigKeys.KEYTAB_PRINCIPAL, principal); + if (remotePathKeytab != null) { + appMasterEnv.put(YarnConfigKeys.REMOTE_KEYTAB_PATH, remotePathKeytab.toString()); + } + } + + // To support Yarn Secure Integration Test Scenario + if (remoteYarnSiteXmlPath != null) { + appMasterEnv.put(YarnConfigKeys.ENV_YARN_SITE_XML_PATH, remoteYarnSiteXmlPath.toString()); + } + if (remoteKrb5Path != null) { + appMasterEnv.put(YarnConfigKeys.ENV_KRB5_PATH, remoteKrb5Path.toString()); + } + + amContainer.setEnvironment(appMasterEnv); + + // Set up resource type requirements for ApplicationMaster + Resource capability = Records.newRecord(Resource.class); + capability.setMemorySize(clusterSpecification.getMasterMemoryMB()); + capability.setVirtualCores(flinkConfiguration.get(YarnConfigOptions.APP_MASTER_VCORES)); + + final String customApplicationName = customName != null ? customName : applicationName; + + appContext.setApplicationName(customApplicationName); + appContext.setApplicationType(applicationType != null ? applicationType : "Apache Flink"); + appContext.setAMContainerSpec(amContainer); + appContext.setResource(capability); + + // Set priority for application + int priorityNum = flinkConfiguration.get(YarnConfigOptions.APPLICATION_PRIORITY); + if (priorityNum >= 0) { + Priority priority = Priority.newInstance(priorityNum); + appContext.setPriority(priority); + } + + if (yarnQueue != null) { + appContext.setQueue(yarnQueue); + } + + setApplicationNodeLabel(appContext); + + setApplicationTags(appContext); + + // add a hook to clean up in case deployment fails + Thread deploymentFailureHook = new DeploymentFailureHook(yarnApplication, fileUploader.getApplicationDir()); + Runtime.getRuntime().addShutdownHook(deploymentFailureHook); + LOG.info("Submitting application master " + appId); + yarnClient.submitApplication(appContext); + + LOG.info("Waiting for the cluster to be allocated"); + final long startTime = System.currentTimeMillis(); + long lastLogTime = System.currentTimeMillis(); + ApplicationReport report; + YarnApplicationState lastAppState = YarnApplicationState.NEW; + loop: + while (true) { + try { + report = yarnClient.getApplicationReport(appId); + } catch (IOException e) { + throw new YarnDeploymentException("Failed to deploy the cluster.", e); + } + YarnApplicationState appState = report.getYarnApplicationState(); + LOG.debug("Application State: {}", appState); + switch (appState) { + case FAILED: + case KILLED: + throw new YarnDeploymentException("The YARN application unexpectedly switched to state " + + appState + + " during deployment. \n" + + "Diagnostics from YARN: " + + report.getDiagnostics() + + "\n" + + "If log aggregation is enabled on your cluster, use this command to further investigate the issue:\n" + + "yarn logs -applicationId " + + appId); + // break .. + case RUNNING: + LOG.info("YARN application has been deployed successfully."); + break loop; + case FINISHED: + LOG.info("YARN application has been finished successfully."); + break loop; + default: + if (appState != lastAppState) { + LOG.info("Deploying cluster, current state " + appState); + } + if (System.currentTimeMillis() - lastLogTime > 60000) { + lastLogTime = System.currentTimeMillis(); + LOG.info( + "Deployment took more than {} seconds. Please check if the requested resources are available in the YARN cluster", + (lastLogTime - startTime) / 1000); + } + } + lastAppState = appState; + Thread.sleep(250); + } + + // since deployment was successful, remove the hook + ShutdownHookUtil.removeShutdownHook(deploymentFailureHook, getClass().getSimpleName(), LOG); + return report; + } + + private void removeLocalhostBindHostSetting(Configuration configuration, ConfigOption option) { + configuration + .getOptional(option) + .filter(bindHost -> bindHost.equals("localhost")) + .ifPresent(bindHost -> { + LOG.info( + "Removing 'localhost' {} setting from effective configuration; using '0.0.0.0' instead.", + option); + configuration.removeConfig(option); + }); + } + + @VisibleForTesting + void setTokensFor(ContainerLaunchContext containerLaunchContext, boolean fetchToken) throws Exception { + Credentials credentials = new Credentials(); + + LOG.info("Loading delegation tokens available locally to add to the AM container"); + // for user + UserGroupInformation currUsr = UserGroupInformation.getCurrentUser(); + + Collection> usrTok = + currUsr.getCredentials().getAllTokens(); + for (Token token : usrTok) { + LOG.info("Adding user token " + token.getService() + " with " + token); + credentials.addToken(token.getService(), token); + } + + if (fetchToken) { + LOG.info("Fetching delegation tokens to add to the AM container."); + DelegationTokenManager delegationTokenManager = + new DefaultDelegationTokenManager(flinkConfiguration, null, null, null); + DelegationTokenContainer container = new DelegationTokenContainer(); + delegationTokenManager.obtainDelegationTokens(container); + + // This is here for backward compatibility to make log aggregation work + for (Map.Entry e : container.getTokens().entrySet()) { + if (flinkConfiguration.get(APP_MASTER_TOKEN_SERVICES).contains(e.getKey())) { + credentials.addAll(HadoopDelegationTokenConverter.deserialize(e.getValue())); + } + } + } + + ByteBuffer tokens = ByteBuffer.wrap(HadoopDelegationTokenConverter.serialize(credentials)); + containerLaunchContext.setTokens(tokens); + + LOG.info("Delegation tokens added to the AM container."); + } + + /** + * Returns the configured remote target home directory if set, otherwise returns the default + * home directory. + * + * @param defaultFileSystem default file system used + * @return the remote target home directory + */ + @VisibleForTesting + Path getStagingDir(FileSystem defaultFileSystem) throws IOException { + final String configuredStagingDir = flinkConfiguration.get(YarnConfigOptions.STAGING_DIRECTORY); + if (configuredStagingDir == null) { + return defaultFileSystem.getHomeDirectory(); + } + FileSystem stagingDirFs = new Path(configuredStagingDir).getFileSystem(defaultFileSystem.getConf()); + return stagingDirFs.makeQualified(new Path(configuredStagingDir)); + } + + private int getFileReplication() { + final int yarnFileReplication = + yarnConfiguration.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, DFSConfigKeys.DFS_REPLICATION_DEFAULT); + final int fileReplication = flinkConfiguration.get(YarnConfigOptions.FILE_REPLICATION); + return fileReplication > 0 ? fileReplication : yarnFileReplication; + } + + private static String encodeYarnLocalResourceDescriptorListToString(List resources) { + return String.join( + LOCAL_RESOURCE_DESCRIPTOR_SEPARATOR, + resources.stream().map(YarnLocalResourceDescriptor::toString).collect(Collectors.toList())); + } + + /** + * Kills YARN application and stops YARN client. + * + *

      Use this method to kill the App before it has been properly deployed + */ + private void failSessionDuringDeployment(YarnClient yarnClient, YarnClientApplication yarnApplication) { + LOG.info("Killing YARN application"); + + try { + yarnClient.killApplication( + yarnApplication.getNewApplicationResponse().getApplicationId()); + } catch (Exception e) { + // we only log a debug message here because the "killApplication" call is a best-effort + // call (we don't know if the application has been deployed when the error occurred). + LOG.debug("Error while killing YARN application", e); + } + } + + private static class ClusterResourceDescription { + public final long totalFreeMemory; + public final long containerLimit; + public final long[] nodeManagersFree; + + public ClusterResourceDescription(long totalFreeMemory, long containerLimit, long[] nodeManagersFree) { + this.totalFreeMemory = totalFreeMemory; + this.containerLimit = containerLimit; + this.nodeManagersFree = nodeManagersFree; + } + } + + private ClusterResourceDescription getCurrentFreeClusterResources(YarnClient yarnClient) + throws YarnException, IOException { + List nodes = yarnClient.getNodeReports(NodeState.RUNNING); + + int totalFreeMemory = 0; + long containerLimit = 0; + long[] nodeManagersFree = new long[nodes.size()]; + + for (int i = 0; i < nodes.size(); i++) { + NodeReport rep = nodes.get(i); + long free = rep.getCapability().getMemorySize() + - (rep.getUsed() != null ? rep.getUsed().getMemorySize() : 0); + nodeManagersFree[i] = free; + totalFreeMemory += free; + if (free > containerLimit) { + containerLimit = free; + } + } + return new ClusterResourceDescription(totalFreeMemory, containerLimit, nodeManagersFree); + } + + @Override + public String getClusterDescription() { + + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + PrintStream ps = new PrintStream(baos); + + YarnClusterMetrics metrics = yarnClient.getYarnClusterMetrics(); + + ps.append("NodeManagers in the ClusterClient " + metrics.getNumNodeManagers()); + List nodes = yarnClient.getNodeReports(NodeState.RUNNING); + final String format = "|%-16s |%-16s %n"; + ps.printf("|Property |Value %n"); + ps.println("+---------------------------------------+"); + long totalMemory = 0; + int totalCores = 0; + for (NodeReport rep : nodes) { + final Resource res = rep.getCapability(); + totalMemory += res.getMemorySize(); + totalCores += res.getVirtualCores(); + ps.format(format, "NodeID", rep.getNodeId()); + ps.format(format, "Memory", getDisplayMemory(res.getMemorySize())); + ps.format(format, "vCores", res.getVirtualCores()); + ps.format(format, "HealthReport", rep.getHealthReport()); + ps.format(format, "Containers", rep.getNumContainers()); + ps.println("+---------------------------------------+"); + } + ps.println("Summary: totalMemory " + getDisplayMemory(totalMemory) + " totalCores " + totalCores); + List qInfo = yarnClient.getAllQueues(); + for (QueueInfo q : qInfo) { + ps.println("Queue: " + + q.getQueueName() + + ", Current Capacity: " + + q.getCurrentCapacity() + + " Max Capacity: " + + q.getMaximumCapacity() + + " Applications: " + + q.getApplications().size()); + } + return baos.toString(); + } catch (Exception e) { + throw new RuntimeException("Couldn't get cluster description", e); + } + } + + private void activateHighAvailabilitySupport(ApplicationSubmissionContext appContext) + throws InvocationTargetException, IllegalAccessException { + + ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + + reflector.setKeepContainersAcrossApplicationAttempts(appContext, true); + + reflector.setAttemptFailuresValidityInterval( + appContext, flinkConfiguration.get(YarnConfigOptions.APPLICATION_ATTEMPT_FAILURE_VALIDITY_INTERVAL)); + } + + private void setApplicationTags(final ApplicationSubmissionContext appContext) + throws InvocationTargetException, IllegalAccessException { + + final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + final String tagsString = flinkConfiguration.get(YarnConfigOptions.APPLICATION_TAGS); + + final Set applicationTags = new HashSet<>(); + + // Trim whitespace and cull empty tags + for (final String tag : tagsString.split(",")) { + final String trimmedTag = tag.trim(); + if (!trimmedTag.isEmpty()) { + applicationTags.add(trimmedTag); + } + } + + reflector.setApplicationTags(appContext, applicationTags); + } + + private void setApplicationNodeLabel(final ApplicationSubmissionContext appContext) + throws InvocationTargetException, IllegalAccessException { + + if (nodeLabel != null) { + final ApplicationSubmissionContextReflector reflector = ApplicationSubmissionContextReflector.getInstance(); + reflector.setApplicationNodeLabel(appContext, nodeLabel); + } + } + + /** + * Singleton object which uses reflection to determine whether the {@link + * ApplicationSubmissionContext} supports various methods which, depending on the Hadoop + * version, may or may not be supported. + * + *

      If an unsupported method is invoked, nothing happens. + * + *

      Currently three methods are proxied: - setApplicationTags (>= 2.4.0) - + * setAttemptFailuresValidityInterval (>= 2.6.0) - setKeepContainersAcrossApplicationAttempts + * (>= 2.4.0) - setNodeLabelExpression (>= 2.6.0) + */ + private static class ApplicationSubmissionContextReflector { + private static final Logger LOG = LoggerFactory.getLogger(ApplicationSubmissionContextReflector.class); + + private static final ApplicationSubmissionContextReflector instance = + new ApplicationSubmissionContextReflector(ApplicationSubmissionContext.class); + + public static ApplicationSubmissionContextReflector getInstance() { + return instance; + } + + private static final String APPLICATION_TAGS_METHOD_NAME = "setApplicationTags"; + private static final String ATTEMPT_FAILURES_METHOD_NAME = "setAttemptFailuresValidityInterval"; + private static final String KEEP_CONTAINERS_METHOD_NAME = "setKeepContainersAcrossApplicationAttempts"; + private static final String NODE_LABEL_EXPRESSION_NAME = "setNodeLabelExpression"; + + private final Method applicationTagsMethod; + private final Method attemptFailuresValidityIntervalMethod; + private final Method keepContainersMethod; + + @Nullable + private final Method nodeLabelExpressionMethod; + + private ApplicationSubmissionContextReflector(Class clazz) { + Method applicationTagsMethod; + Method attemptFailuresValidityIntervalMethod; + Method keepContainersMethod; + Method nodeLabelExpressionMethod; + + try { + // this method is only supported by Hadoop 2.4.0 onwards + applicationTagsMethod = clazz.getMethod(APPLICATION_TAGS_METHOD_NAME, Set.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), APPLICATION_TAGS_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + applicationTagsMethod = null; + } + + this.applicationTagsMethod = applicationTagsMethod; + + try { + // this method is only supported by Hadoop 2.6.0 onwards + attemptFailuresValidityIntervalMethod = clazz.getMethod(ATTEMPT_FAILURES_METHOD_NAME, long.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), ATTEMPT_FAILURES_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + attemptFailuresValidityIntervalMethod = null; + } + + this.attemptFailuresValidityIntervalMethod = attemptFailuresValidityIntervalMethod; + + try { + // this method is only supported by Hadoop 2.4.0 onwards + keepContainersMethod = clazz.getMethod(KEEP_CONTAINERS_METHOD_NAME, boolean.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), KEEP_CONTAINERS_METHOD_NAME); + // assign null because the Hadoop version apparently does not support this call. + keepContainersMethod = null; + } + + this.keepContainersMethod = keepContainersMethod; + + try { + nodeLabelExpressionMethod = clazz.getMethod(NODE_LABEL_EXPRESSION_NAME, String.class); + LOG.debug("{} supports method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); + } catch (NoSuchMethodException e) { + LOG.debug("{} does not support method {}.", clazz.getCanonicalName(), NODE_LABEL_EXPRESSION_NAME); + nodeLabelExpressionMethod = null; + } + + this.nodeLabelExpressionMethod = nodeLabelExpressionMethod; + } + + public void setApplicationTags(ApplicationSubmissionContext appContext, Set applicationTags) + throws InvocationTargetException, IllegalAccessException { + if (applicationTagsMethod != null) { + LOG.debug( + "Calling method {} of {}.", + applicationTagsMethod.getName(), + appContext.getClass().getCanonicalName()); + applicationTagsMethod.invoke(appContext, applicationTags); + } else { + LOG.debug( + "{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + APPLICATION_TAGS_METHOD_NAME); + } + } + + public void setApplicationNodeLabel(ApplicationSubmissionContext appContext, String nodeLabel) + throws InvocationTargetException, IllegalAccessException { + if (nodeLabelExpressionMethod != null) { + LOG.debug( + "Calling method {} of {}.", + nodeLabelExpressionMethod.getName(), + appContext.getClass().getCanonicalName()); + nodeLabelExpressionMethod.invoke(appContext, nodeLabel); + } else { + LOG.debug( + "{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + NODE_LABEL_EXPRESSION_NAME); + } + } + + public void setAttemptFailuresValidityInterval(ApplicationSubmissionContext appContext, long validityInterval) + throws InvocationTargetException, IllegalAccessException { + if (attemptFailuresValidityIntervalMethod != null) { + LOG.debug( + "Calling method {} of {}.", + attemptFailuresValidityIntervalMethod.getName(), + appContext.getClass().getCanonicalName()); + attemptFailuresValidityIntervalMethod.invoke(appContext, validityInterval); + } else { + LOG.debug( + "{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + ATTEMPT_FAILURES_METHOD_NAME); + } + } + + public void setKeepContainersAcrossApplicationAttempts( + ApplicationSubmissionContext appContext, boolean keepContainers) + throws InvocationTargetException, IllegalAccessException { + + if (keepContainersMethod != null) { + LOG.debug( + "Calling method {} of {}.", + keepContainersMethod.getName(), + appContext.getClass().getCanonicalName()); + keepContainersMethod.invoke(appContext, keepContainers); + } else { + LOG.debug( + "{} does not support method {}. Doing nothing.", + appContext.getClass().getCanonicalName(), + KEEP_CONTAINERS_METHOD_NAME); + } + } + } + + private static class YarnDeploymentException extends RuntimeException { + private static final long serialVersionUID = -812040641215388943L; + + public YarnDeploymentException(String message) { + super(message); + } + + public YarnDeploymentException(String message, Throwable cause) { + super(message, cause); + } + } + + private class DeploymentFailureHook extends Thread { + + private final YarnClient yarnClient; + private final YarnClientApplication yarnApplication; + private final Path yarnFilesDir; + + DeploymentFailureHook(YarnClientApplication yarnApplication, Path yarnFilesDir) { + this.yarnApplication = Preconditions.checkNotNull(yarnApplication); + this.yarnFilesDir = Preconditions.checkNotNull(yarnFilesDir); + + // A new yarn client need to be created in shutdown hook in order to avoid + // the yarn client has been closed by YarnClusterDescriptor. + this.yarnClient = YarnClient.createYarnClient(); + this.yarnClient.init(yarnConfiguration); + } + + @Override + public void run() { + LOG.info("Cancelling deployment from Deployment Failure Hook"); + yarnClient.start(); + failSessionDuringDeployment(yarnClient, yarnApplication); + yarnClient.stop(); + LOG.info("Deleting files in {}.", yarnFilesDir); + try { + FileSystem fs = FileSystem.get(yarnConfiguration); + + if (!fs.delete(yarnFilesDir, true)) { + throw new IOException("Deleting files in " + yarnFilesDir + " was unsuccessful"); + } + + fs.close(); + } catch (IOException e) { + LOG.error("Failed to delete Flink Jar and configuration files in HDFS", e); + } + } + } + + @VisibleForTesting + void addLibFoldersToShipFiles(Collection effectiveShipFiles) { + // Add lib folder to the ship files if the environment variable is set. + // This is for convenience when running from the command-line. + // (for other files users explicitly set the ship files) + String libDir = System.getenv().get(ENV_FLINK_LIB_DIR); + if (libDir != null) { + File directoryFile = new File(libDir); + if (directoryFile.isDirectory()) { + effectiveShipFiles.add(getPathFromLocalFile(directoryFile)); + } else { + throw new YarnDeploymentException("The environment variable '" + + ENV_FLINK_LIB_DIR + + "' is set to '" + + libDir + + "' but the directory doesn't exist."); + } + } else if (shipFiles.isEmpty()) { + LOG.warn( + "Environment variable '{}' not set and ship files have not been provided manually. " + + "Not shipping any library files.", + ENV_FLINK_LIB_DIR); + } + } + + @VisibleForTesting + void addUsrLibFolderToShipFiles(Collection effectiveShipFiles) { + // Add usrlib folder to the ship files if it exists + // Classes in the folder will be loaded by UserClassLoader if CLASSPATH_INCLUDE_USER_JAR is + // DISABLED. + ClusterEntrypointUtils.tryFindUserLibDirectory().ifPresent(usrLibDirFile -> { + effectiveShipFiles.add(usrLibDirFile); + LOG.info("usrlib: {} will be shipped automatically.", usrLibDirFile.getAbsolutePath()); + }); + } + + @VisibleForTesting + void addPluginsFoldersToShipFiles(Collection effectiveShipFiles) { + final Optional pluginsDir = PluginConfig.getPluginsDir(); + pluginsDir.ifPresent(dir -> effectiveShipFiles.add(getPathFromLocalFile(dir))); + } + + ContainerLaunchContext setupApplicationMasterContainer( + String yarnClusterEntrypoint, boolean hasKrb5, JobManagerProcessSpec processSpec) { + // ------------------ Prepare Application Master Container ------------------------------ + + // respect custom JVM options in the YAML file + List> jvmOptions = Arrays.asList( + CoreOptions.FLINK_DEFAULT_JVM_OPTIONS, + CoreOptions.FLINK_JVM_OPTIONS, + CoreOptions.FLINK_DEFAULT_JM_JVM_OPTIONS, + CoreOptions.FLINK_JM_JVM_OPTIONS); + String javaOpts = Utils.generateJvmOptsString(flinkConfiguration, jvmOptions, hasKrb5); + + // Set up the container launch context for the application master + ContainerLaunchContext amContainer = Records.newRecord(ContainerLaunchContext.class); + + final Map startCommandValues = new HashMap<>(); + startCommandValues.put("java", "$JAVA_HOME/bin/java"); + + String jvmHeapMem = JobManagerProcessUtils.generateJvmParametersStr(processSpec, flinkConfiguration); + startCommandValues.put("jvmmem", jvmHeapMem); + + startCommandValues.put("jvmopts", javaOpts); + startCommandValues.put("logging", YarnLogConfigUtil.getLoggingYarnCommand(flinkConfiguration)); + + startCommandValues.put("class", yarnClusterEntrypoint); + startCommandValues.put( + "redirects", + "1> " + + ApplicationConstants.LOG_DIR_EXPANSION_VAR + + "/jobmanager.out " + + "2> " + + ApplicationConstants.LOG_DIR_EXPANSION_VAR + + "/jobmanager.err"); + String dynamicParameterListStr = JobManagerProcessUtils.generateDynamicConfigsStr(processSpec); + startCommandValues.put("args", dynamicParameterListStr); + + final String commandTemplate = flinkConfiguration.get(YARN_CONTAINER_START_COMMAND_TEMPLATE); + final String amCommand = getStartCommand(commandTemplate, startCommandValues); + + amContainer.setCommands(Collections.singletonList(amCommand)); + + LOG.debug("Application Master start command: " + amCommand); + + return amContainer; + } + + private static YarnConfigOptions.UserJarInclusion getUserJarInclusionMode( + org.apache.flink.configuration.Configuration config) { + return config.get(YarnConfigOptions.CLASSPATH_INCLUDE_USER_JAR); + } + + private static boolean isUsrLibDirIncludedInShipFiles(List shipFiles, YarnConfiguration yarnConfig) { + return shipFiles.stream() + .map(FunctionUtils.uncheckedFunction(path -> getFileStatus(path, yarnConfig))) + .filter(FileStatus::isDirectory) + .map(status -> status.getPath().getName().toLowerCase()) + .anyMatch(name -> name.equals(DEFAULT_FLINK_USR_LIB_DIR)); + } + + private void setClusterEntrypointInfoToConfig(final ApplicationReport report) { + checkNotNull(report); + + final ApplicationId appId = report.getApplicationId(); + final String host = report.getHost(); + final int port = report.getRpcPort(); + + LOG.info("Found Web Interface {}:{} of application '{}'.", host, port, appId); + + flinkConfiguration.set(JobManagerOptions.ADDRESS, host); + flinkConfiguration.set(JobManagerOptions.PORT, port); + + flinkConfiguration.set(RestOptions.ADDRESS, host); + flinkConfiguration.set(RestOptions.PORT, port); + + flinkConfiguration.set(YarnConfigOptions.APPLICATION_ID, ConverterUtils.toString(appId)); + + setHAClusterIdIfNotSet(flinkConfiguration, appId); + } + + private void setHAClusterIdIfNotSet(Configuration configuration, ApplicationId appId) { + // set cluster-id to app id if not specified + if (!configuration.contains(HighAvailabilityOptions.HA_CLUSTER_ID)) { + configuration.set(HighAvailabilityOptions.HA_CLUSTER_ID, ConverterUtils.toString(appId)); + } + } + + public static void logDetachedClusterInformation(ApplicationId yarnApplicationId, Logger logger) { + logger.info( + "The Flink YARN session cluster has been started in detached mode. In order to " + + "stop Flink gracefully, use the following command:\n" + + "$ echo \"stop\" | ./bin/yarn-session.sh -id {}\n" + + "If this should not be possible, then you can also kill Flink via YARN's web interface or via:\n" + + "$ yarn application -kill {}\n" + + "Note that killing Flink might not clean up all job artifacts and temporary files.", + yarnApplicationId, + yarnApplicationId); + } + + @VisibleForTesting + Map generateApplicationMasterEnv( + final YarnApplicationFileUploader fileUploader, + final String classPathStr, + final String localFlinkJarStr, + final String appIdStr) + throws IOException { + final Map env = new HashMap<>(); + // set user specified app master environment variables + env.putAll(ConfigurationUtils.getPrefixedKeyValuePairs( + ResourceManagerOptions.CONTAINERIZED_MASTER_ENV_PREFIX, this.flinkConfiguration)); + // set Flink app class path + env.put(ENV_FLINK_CLASSPATH, classPathStr); + // Set FLINK_LIB_DIR to `lib` folder under working dir in container + env.put(ENV_FLINK_LIB_DIR, Path.CUR_DIR + "/" + ConfigConstants.DEFAULT_FLINK_LIB_DIR); + // Set FLINK_OPT_DIR to `opt` folder under working dir in container + env.put(ENV_FLINK_OPT_DIR, Path.CUR_DIR + "/" + ConfigConstants.DEFAULT_FLINK_OPT_DIR); + // set Flink on YARN internal configuration values + env.put(YarnConfigKeys.FLINK_DIST_JAR, localFlinkJarStr); + env.put(YarnConfigKeys.ENV_APP_ID, appIdStr); + env.put(YarnConfigKeys.ENV_CLIENT_HOME_DIR, fileUploader.getHomeDir().toString()); + env.put( + YarnConfigKeys.ENV_CLIENT_SHIP_FILES, + encodeYarnLocalResourceDescriptorListToString(fileUploader.getEnvShipResourceList())); + env.put( + YarnConfigKeys.FLINK_YARN_FILES, + fileUploader.getApplicationDir().toUri().toString()); + // https://github.com/apache/hadoop/blob/trunk/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/YarnApplicationSecurity.md#identity-on-an-insecure-cluster-hadoop_user_name + env.put( + YarnConfigKeys.ENV_HADOOP_USER_NAME, + UserGroupInformation.getCurrentUser().getUserName()); + // set classpath from YARN configuration + Utils.setupYarnClassPath(this.yarnConfiguration, env); + return env; + } + + private String getDisplayMemory(long memoryMB) { + return MemorySize.ofMebiBytes(memoryMB).toHumanReadableString(); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java new file mode 100644 index 0000000000..6414271715 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/AbstractCustomTableEnvironment.java @@ -0,0 +1,123 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.dinky.data.model.LineageRel; +import org.dinky.data.result.SqlExplainResult; +import org.dinky.utils.LineageContext; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.ExplainDetail; +import org.apache.flink.table.api.ExplainFormat; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.bridge.java.internal.StreamTableEnvironmentImpl; +import org.apache.flink.table.delegation.Planner; +import org.apache.flink.table.operations.ExplainOperation; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.QueryOperation; + +import java.util.List; + +import cn.hutool.core.util.ReflectUtil; + +/** */ +public abstract class AbstractCustomTableEnvironment + implements CustomTableEnvironment, DefaultTableEnvironmentInternal, DefaultStreamTableEnvironment { + + protected StreamTableEnvironment streamTableEnvironment; + protected ClassLoader userClassLoader; + + protected AbstractCustomTableEnvironment() {} + + protected AbstractCustomTableEnvironment(StreamTableEnvironment streamTableEnvironment) { + this.streamTableEnvironment = streamTableEnvironment; + } + + @Override + public TableEnvironment getTableEnvironment() { + return streamTableEnvironment; + } + + public StreamExecutionEnvironment getStreamExecutionEnvironment() { + return ((StreamTableEnvironmentImpl) streamTableEnvironment).execEnv(); + } + + @Override + public ClassLoader getUserClassLoader() { + return userClassLoader; + } + + public Planner getPlanner() { + return ((StreamTableEnvironmentImpl) streamTableEnvironment).getPlanner(); + } + + @Override + public void injectParser(CustomParser parser) { + ReflectUtil.setFieldValue(getPlanner(), "parser", new ParserWrapper(parser)); + } + + @Override + public void injectExtendedExecutor(CustomExtendedOperationExecutor extendedExecutor) {} + + @Override + public Configuration getRootConfiguration() { + return (Configuration) this.getConfig().getRootConfiguration(); + } + + @Override + public SqlExplainResult explainSqlRecord(String statement, ExplainDetail... extraDetails) { + List operations = getParser().parse(statement); + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); + } + + Operation operation = operations.get(0); + SqlExplainResult data = new SqlExplainResult(); + data.setParseTrue(true); + data.setExplainTrue(true); + + if (operation instanceof ModifyOperation) { + data.setType("Modify DML"); + } else if (operation instanceof ExplainOperation) { + data.setType("Explain DML"); + } else if (operation instanceof QueryOperation) { + data.setType("Query DML"); + } else { + data.setExplain(operation.asSummaryString()); + data.setType("DDL"); + + // data.setExplain("DDL statement needn't comment。"); + return data; + } + + data.setExplain(getPlanner().explain(operations, ExplainFormat.TEXT, extraDetails)); + return data; + } + + @Override + public List getLineage(String statement) { + LineageContext lineageContext = new LineageContext(this); + return lineageContext.analyzeLineage(statement); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ClusterDescriptorAdapterImpl.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ClusterDescriptorAdapterImpl.java new file mode 100644 index 0000000000..68d859dc59 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ClusterDescriptorAdapterImpl.java @@ -0,0 +1,57 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.kubernetes.KubernetesClusterDescriptor; +import org.apache.flink.kubernetes.artifact.DefaultKubernetesArtifactUploader; +import org.apache.flink.kubernetes.kubeclient.FlinkKubeClient; +import org.apache.flink.kubernetes.kubeclient.FlinkKubeClientFactory; +import org.apache.flink.yarn.YarnClusterDescriptor; +import org.apache.hadoop.fs.Path; + +import java.io.File; +import java.util.List; +import java.util.stream.Collectors; + +import cn.hutool.core.util.URLUtil; + +public class ClusterDescriptorAdapterImpl extends ClusterDescriptorAdapter { + + public ClusterDescriptorAdapterImpl() {} + + public ClusterDescriptorAdapterImpl(YarnClusterDescriptor yarnClusterDescriptor) { + super(yarnClusterDescriptor); + } + + @Override + public void addShipFiles(List shipFiles) { + yarnClusterDescriptor.addShipFiles(shipFiles.stream() + .map(file -> new Path(URLUtil.getURL(file).toString())) + .collect(Collectors.toList())); + } + + @Override + public KubernetesClusterDescriptor createKubernetesClusterDescriptor( + Configuration configuration, FlinkKubeClient flinkKubeClient) { + return new KubernetesClusterDescriptor( + configuration, FlinkKubeClientFactory.getInstance(), new DefaultKubernetesArtifactUploader()); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java new file mode 100644 index 0000000000..74c58c0d64 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableEnvironmentImpl.java @@ -0,0 +1,160 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.dinky.operations.CustomNewParserImpl; + +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.runtime.jobgraph.jsonplan.JsonPlanGenerator; +import org.apache.flink.runtime.rest.messages.JobPlanInfo; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.graph.JSONGenerator; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.CachedPlan; +import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.catalog.CatalogDescriptor; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.operations.Operation; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; + +/** + * CustomTableEnvironmentImpl + * + * @since 2022/05/08 + */ +public class CustomTableEnvironmentImpl extends AbstractCustomTableEnvironment { + + private static final Logger log = LoggerFactory.getLogger(CustomTableEnvironmentImpl.class); + + private static final ObjectMapper mapper = new ObjectMapper(); + + public CustomTableEnvironmentImpl(StreamTableEnvironment streamTableEnvironment) { + super(streamTableEnvironment); + injectParser(new CustomNewParserImpl(this, getPlanner().getParser())); + } + + public static CustomTableEnvironmentImpl create( + StreamExecutionEnvironment executionEnvironment, ClassLoader classLoader) { + return create( + executionEnvironment, + EnvironmentSettings.newInstance().withClassLoader(classLoader).build()); + } + + public static CustomTableEnvironmentImpl createBatch( + StreamExecutionEnvironment executionEnvironment, ClassLoader classLoader) { + return create( + executionEnvironment, + EnvironmentSettings.newInstance() + .withClassLoader(classLoader) + .inBatchMode() + .build()); + } + + public static CustomTableEnvironmentImpl create( + StreamExecutionEnvironment executionEnvironment, EnvironmentSettings settings) { + StreamTableEnvironment streamTableEnvironment = StreamTableEnvironment.create(executionEnvironment, settings); + + return new CustomTableEnvironmentImpl(streamTableEnvironment); + } + + @Override + public ObjectNode getStreamGraph(String statement) { + List operations = super.getParser().parse(statement); + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! explainSql() only accepts a single SQL query."); + } + + List modifyOperations = operations.stream() + .filter(ModifyOperation.class::isInstance) + .map(ModifyOperation.class::cast) + .collect(Collectors.toList()); + + StreamGraph streamGraph = transOperatoinsToStreamGraph(modifyOperations); + JSONGenerator jsonGenerator = new JSONGenerator(streamGraph); + try { + return (ObjectNode) mapper.readTree(jsonGenerator.getJSON()); + } catch (JsonProcessingException e) { + log.error("read streamGraph configure error: ", e); + return mapper.createObjectNode(); + } + } + + private StreamGraph transOperatoinsToStreamGraph(List modifyOperations) { + List> trans = getPlanner().translate(modifyOperations); + final StreamExecutionEnvironment environment = getStreamExecutionEnvironment(); + trans.forEach(environment::addOperator); + + StreamGraph streamGraph = environment.getStreamGraph(); + final Configuration configuration = getConfig().getConfiguration(); + if (configuration.containsKey(PipelineOptions.NAME.key())) { + streamGraph.setJobName(configuration.getString(PipelineOptions.NAME)); + } + return streamGraph; + } + + @Override + public JobPlanInfo getJobPlanInfo(List statements) { + return new JobPlanInfo(JsonPlanGenerator.generatePlan(getJobGraphFromInserts(statements))); + } + + @Override + public StreamGraph getStreamGraphFromInserts(List statements) { + List modifyOperations = new ArrayList<>(); + statements.stream().map(statement -> getParser().parse(statement)).forEach(operations -> { + if (operations.size() != 1) { + throw new TableException("Only single statement is supported."); + } + Operation operation = operations.get(0); + if (operation instanceof ModifyOperation) { + modifyOperations.add((ModifyOperation) operation); + } else { + throw new TableException("Only insert statement is supported now."); + } + }); + + return transOperatoinsToStreamGraph(modifyOperations); + } + + @Override + public void createCatalog(String catalogName, CatalogDescriptor catalogDescriptor) { + getCatalogManager().createCatalog(catalogName, catalogDescriptor); + } + + @Override + public TableResultInternal executeCachedPlanInternal(CachedPlan cachedPlan) { + + return null; + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableResultImpl.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableResultImpl.java new file mode 100644 index 0000000000..3e1f1953d0 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/CustomTableResultImpl.java @@ -0,0 +1,265 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ResultKind; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.internal.CachedPlan; +import org.apache.flink.table.api.internal.ResultProvider; +import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.utils.print.PrintStyle; +import org.apache.flink.table.utils.print.RowDataToStringConverter; +import org.apache.flink.table.utils.print.TableauStyle; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Preconditions; + +import java.io.PrintWriter; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import javax.annotation.Nullable; + +/** Implementation for {@link TableResult}. */ +@Internal +public class CustomTableResultImpl implements TableResultInternal { + + public static final TableResult TABLE_RESULT_OK = CustomTableResultImpl.builder() + .resultKind(ResultKind.SUCCESS) + .schema(ResolvedSchema.of(Column.physical("result", DataTypes.STRING()))) + .data(Collections.singletonList(Row.of("OK"))) + .build(); + + private final JobClient jobClient; + private final ResolvedSchema resolvedSchema; + private final ResultKind resultKind; + private final ResultProvider resultProvider; + private final PrintStyle printStyle; + + private CustomTableResultImpl( + @Nullable JobClient jobClient, + ResolvedSchema resolvedSchema, + ResultKind resultKind, + ResultProvider resultProvider, + PrintStyle printStyle) { + this.jobClient = jobClient; + this.resolvedSchema = Preconditions.checkNotNull(resolvedSchema, "resolvedSchema should not be null"); + this.resultKind = Preconditions.checkNotNull(resultKind, "resultKind should not be null"); + Preconditions.checkNotNull(resultProvider, "result provider should not be null"); + this.resultProvider = resultProvider; + this.printStyle = Preconditions.checkNotNull(printStyle, "printStyle should not be null"); + } + + public static TableResult buildTableResult(List fields, List rows) { + Builder builder = builder().resultKind(ResultKind.SUCCESS); + if (fields.size() > 0) { + List columnNames = new ArrayList<>(); + List columnTypes = new ArrayList<>(); + for (int i = 0; i < fields.size(); i++) { + columnNames.add(fields.get(i).getName()); + columnTypes.add(fields.get(i).getType()); + } + builder.schema(ResolvedSchema.physical(columnNames, columnTypes)).data(rows); + } + return builder.build(); + } + + @Override + public Optional getJobClient() { + return Optional.ofNullable(jobClient); + } + + @Override + public void await() throws InterruptedException, ExecutionException { + try { + awaitInternal(-1, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + // do nothing + } + } + + @Override + public void await(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException { + awaitInternal(timeout, unit); + } + + private void awaitInternal(long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { + if (jobClient == null) { + return; + } + + ExecutorService executor = Executors.newFixedThreadPool(1, r -> new Thread(r, "TableResult-await-thread")); + try { + CompletableFuture future = CompletableFuture.runAsync( + () -> { + while (!resultProvider.isFirstRowReady()) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new TableException("Thread is interrupted"); + } + } + }, + executor); + + if (timeout >= 0) { + future.get(timeout, unit); + } else { + future.get(); + } + } finally { + executor.shutdown(); + } + } + + @Override + public ResolvedSchema getResolvedSchema() { + return resolvedSchema; + } + + @Override + public ResultKind getResultKind() { + return resultKind; + } + + @Override + public CloseableIterator collect() { + return resultProvider.toExternalIterator(); + } + + @Override + public CloseableIterator collectInternal() { + return resultProvider.toInternalIterator(); + } + + @Override + public RowDataToStringConverter getRowDataToStringConverter() { + return resultProvider.getRowDataStringConverter(); + } + + @Nullable + public CachedPlan getCachedPlan() { + return null; + } + + @Override + public void print() { + Iterator it = resultProvider.toInternalIterator(); + printStyle.print(it, new PrintWriter(System.out)); + } + + public static Builder builder() { + return new Builder(); + } + + /** Builder for creating a {@link CustomTableResultImpl}. */ + public static class Builder { + + private JobClient jobClient = null; + private ResolvedSchema resolvedSchema = null; + private ResultKind resultKind = null; + private ResultProvider resultProvider = null; + private PrintStyle printStyle = null; + + private Builder() {} + + /** + * Specifies job client which associates the submitted Flink job. + * + * @param jobClient a {@link JobClient} for the submitted Flink job. + */ + public Builder jobClient(JobClient jobClient) { + this.jobClient = jobClient; + return this; + } + + /** + * Specifies schema of the execution result. + * + * @param resolvedSchema a {@link ResolvedSchema} for the execution result. + */ + public Builder schema(ResolvedSchema resolvedSchema) { + Preconditions.checkNotNull(resolvedSchema, "resolvedSchema should not be null"); + this.resolvedSchema = resolvedSchema; + return this; + } + + /** + * Specifies result kind of the execution result. + * + * @param resultKind a {@link ResultKind} for the execution result. + */ + public Builder resultKind(ResultKind resultKind) { + Preconditions.checkNotNull(resultKind, "resultKind should not be null"); + this.resultKind = resultKind; + return this; + } + + public Builder resultProvider(ResultProvider resultProvider) { + Preconditions.checkNotNull(resultProvider, "resultProvider should not be null"); + this.resultProvider = resultProvider; + return this; + } + + /** + * Specifies an row list as the execution result. + * + * @param rowList a row list as the execution result. + */ + public Builder data(List rowList) { + Preconditions.checkNotNull(rowList, "listRows should not be null"); + this.resultProvider = new StaticResultProvider(rowList); + return this; + } + + /** Specifies print style. Default is {@link TableauStyle} with max integer column width. */ + public Builder setPrintStyle(PrintStyle printStyle) { + Preconditions.checkNotNull(printStyle, "printStyle should not be null"); + this.printStyle = printStyle; + return this; + } + + /** Returns a {@link TableResult} instance. */ + public TableResultInternal build() { + if (printStyle == null) { + printStyle = PrintStyle.rawContent(resultProvider.getRowDataStringConverter()); + } + return new CustomTableResultImpl(jobClient, resolvedSchema, resultKind, resultProvider, printStyle); + } + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultStreamTableEnvironment.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultStreamTableEnvironment.java new file mode 100644 index 0000000000..b7ef64ccba --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultStreamTableEnvironment.java @@ -0,0 +1,167 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.bridge.java.StreamStatementSet; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.functions.AggregateFunction; +import org.apache.flink.table.functions.TableAggregateFunction; +import org.apache.flink.table.functions.TableFunction; +import org.apache.flink.table.types.AbstractDataType; +import org.apache.flink.types.Row; + +/** */ +public interface DefaultStreamTableEnvironment + extends StreamTableEnvironment, DefaultTableEnvironment, TableEnvironmentInstance { + + default StreamTableEnvironment getStreamTableEnvironment() { + return (StreamTableEnvironment) getTableEnvironment(); + } + + @Override // region StreamTableEnvironment interface + default void registerFunction(String s, TableFunction tableFunction) { + getStreamTableEnvironment().registerFunction(s, tableFunction); + } + + @Override + default void registerFunction(String s, AggregateFunction aggregateFunction) { + getStreamTableEnvironment().registerFunction(s, aggregateFunction); + } + + @Override + default void registerFunction(String s, TableAggregateFunction tableAggregateFunction) { + getStreamTableEnvironment().registerFunction(s, tableAggregateFunction); + } + + @Override + default Table fromDataStream(DataStream dataStream) { + return getStreamTableEnvironment().fromDataStream(dataStream); + } + + @Override + default Table fromDataStream(DataStream dataStream, Schema schema) { + return getStreamTableEnvironment().fromDataStream(dataStream, schema); + } + + @Override + default Table fromChangelogStream(DataStream dataStream) { + return getStreamTableEnvironment().fromChangelogStream(dataStream); + } + + @Override + default Table fromChangelogStream(DataStream dataStream, Schema schema) { + return getStreamTableEnvironment().fromChangelogStream(dataStream, schema); + } + + @Override + default Table fromChangelogStream(DataStream dataStream, Schema schema, ChangelogMode changelogMode) { + return getStreamTableEnvironment().fromChangelogStream(dataStream, schema, changelogMode); + } + + @Override + default void createTemporaryView(String s, DataStream dataStream) { + getStreamTableEnvironment().createTemporaryView(s, dataStream); + } + + @Override + default void createTemporaryView(String s, DataStream dataStream, Schema schema) { + getStreamTableEnvironment().createTemporaryView(s, dataStream, schema); + } + + @Override + default DataStream toDataStream(Table table) { + return getStreamTableEnvironment().toDataStream(table); + } + + @Override + default DataStream toDataStream(Table table, Class aClass) { + return getStreamTableEnvironment().toDataStream(table, aClass); + } + + @Override + default DataStream toDataStream(Table table, AbstractDataType abstractDataType) { + return getStreamTableEnvironment().toDataStream(table, abstractDataType); + } + + @Override + default DataStream toChangelogStream(Table table) { + return getStreamTableEnvironment().toChangelogStream(table); + } + + @Override + default DataStream toChangelogStream(Table table, Schema schema) { + return getStreamTableEnvironment().toChangelogStream(table, schema); + } + + @Override + default DataStream toChangelogStream(Table table, Schema schema, ChangelogMode changelogMode) { + return getStreamTableEnvironment().toChangelogStream(table, schema, changelogMode); + } + + @Override + default StreamStatementSet createStatementSet() { + return getStreamTableEnvironment().createStatementSet(); + } + + @Override + default Table fromDataStream(DataStream dataStream, Expression... expressions) { + return getStreamTableEnvironment().fromDataStream(dataStream, expressions); + } + + @Override + default void registerDataStream(String s, DataStream dataStream) { + getStreamTableEnvironment().registerDataStream(s, dataStream); + } + + @Override + default void createTemporaryView(String s, DataStream dataStream, Expression... expressions) { + getStreamTableEnvironment().createTemporaryView(s, dataStream, expressions); + } + + @Override + default DataStream toAppendStream(Table table, Class aClass) { + return getStreamTableEnvironment().toAppendStream(table, aClass); + } + + @Override + default DataStream toAppendStream(Table table, TypeInformation typeInformation) { + return getStreamTableEnvironment().toAppendStream(table, typeInformation); + } + + @Override + default DataStream> toRetractStream(Table table, Class aClass) { + return getStreamTableEnvironment().toRetractStream(table, aClass); + } + + @Override + default DataStream> toRetractStream(Table table, TypeInformation typeInformation) { + return getStreamTableEnvironment().toRetractStream(table, typeInformation); + } + + // endregion + +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultTableEnvironment.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultTableEnvironment.java new file mode 100644 index 0000000000..dcfdc22b65 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultTableEnvironment.java @@ -0,0 +1,342 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.apache.flink.table.api.CompiledPlan; +import org.apache.flink.table.api.ExplainDetail; +import org.apache.flink.table.api.ExplainFormat; +import org.apache.flink.table.api.PlanReference; +import org.apache.flink.table.api.StatementSet; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableDescriptor; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.functions.ScalarFunction; +import org.apache.flink.table.functions.UserDefinedFunction; +import org.apache.flink.table.module.Module; +import org.apache.flink.table.module.ModuleEntry; +import org.apache.flink.table.resource.ResourceUri; +import org.apache.flink.table.types.AbstractDataType; + +import java.util.List; +import java.util.Optional; + +/** */ +public interface DefaultTableEnvironment extends TableEnvironment, TableEnvironmentInstance { + @Override + default Table fromValues(Object... values) { + return TableEnvironment.super.fromValues(values); + } + + @Override + default Table fromValues(AbstractDataType rowType, Object... values) { + return TableEnvironment.super.fromValues(rowType, values); + } + + @Override + default void createFunction(String path, String className, List resourceUris) { + getTableEnvironment().createFunction(path, className, resourceUris); + } + + @Override + default void createFunction(String path, String className, List resourceUris, boolean ignoreIfExists) { + getTableEnvironment().createFunction(path, className, resourceUris, ignoreIfExists); + } + + @Override + default void createTemporaryFunction(String path, String className, List resourceUris) { + getTableEnvironment().createTemporaryFunction(path, className, resourceUris); + } + + @Override + default void createTemporarySystemFunction(String name, String className, List resourceUris) { + getTableEnvironment().createTemporarySystemFunction(name, className, resourceUris); + } + + @Override + default String explainSql(String statement, ExplainFormat format, ExplainDetail... extraDetails) { + return getTableEnvironment().explainSql(statement, format, extraDetails); + } + + @Override + default TableResult executePlan(PlanReference planReference) throws TableException { + return TableEnvironment.super.executePlan(planReference); + } + + /// + @Override + default Table fromValues(Expression... expressions) { + return getTableEnvironment().fromValues(expressions); + } + + @Override + default Table fromValues(AbstractDataType abstractDataType, Expression... expressions) { + return getTableEnvironment().fromValues(abstractDataType, expressions); + } + + @Override + default Table fromValues(Iterable iterable) { + return getTableEnvironment().fromValues(iterable); + } + + @Override + default Table fromValues(AbstractDataType abstractDataType, Iterable iterable) { + return getTableEnvironment().fromValues(abstractDataType, iterable); + } + + @Override + default void registerCatalog(String s, Catalog catalog) { + getTableEnvironment().registerCatalog(s, catalog); + } + + @Override + default Optional getCatalog(String s) { + return getTableEnvironment().getCatalog(s); + } + + @Override + default void loadModule(String s, Module module) { + getTableEnvironment().loadModule(s, module); + } + + @Override + default void useModules(String... strings) { + getTableEnvironment().useModules(strings); + } + + @Override + default void unloadModule(String s) { + getTableEnvironment().unloadModule(s); + } + + @Override + default void registerFunction(String s, ScalarFunction scalarFunction) { + getTableEnvironment().registerFunction(s, scalarFunction); + } + + @Override + default void createTemporarySystemFunction(String s, Class aClass) { + getTableEnvironment().createTemporarySystemFunction(s, aClass); + } + + @Override + default void createTemporarySystemFunction(String s, UserDefinedFunction userDefinedFunction) { + getTableEnvironment().createTemporarySystemFunction(s, userDefinedFunction); + } + + @Override + default boolean dropTemporarySystemFunction(String s) { + return getTableEnvironment().dropTemporarySystemFunction(s); + } + + @Override + default void createFunction(String s, Class aClass) { + getTableEnvironment().createFunction(s, aClass); + } + + @Override + default void createFunction(String s, Class aClass, boolean b) { + getTableEnvironment().createFunction(s, aClass, b); + } + + @Override + default boolean dropFunction(String s) { + return getTableEnvironment().dropFunction(s); + } + + @Override + default void createTemporaryFunction(String s, Class aClass) { + getTableEnvironment().createTemporaryFunction(s, aClass); + } + + @Override + default void createTemporaryFunction(String s, UserDefinedFunction userDefinedFunction) { + getTableEnvironment().createTemporaryFunction(s, userDefinedFunction); + } + + @Override + default boolean dropTemporaryFunction(String s) { + return getTableEnvironment().dropTemporaryFunction(s); + } + + @Override + default void createTemporaryTable(String s, TableDescriptor tableDescriptor) { + getTableEnvironment().createTemporaryTable(s, tableDescriptor); + } + + @Override + default void createTable(String s, TableDescriptor tableDescriptor) { + getTableEnvironment().createTable(s, tableDescriptor); + } + + @Override + default void registerTable(String s, Table table) { + getTableEnvironment().registerTable(s, table); + } + + @Override + default void createTemporaryView(String s, Table table) { + getTableEnvironment().createTemporaryView(s, table); + } + + @Override + default Table scan(String... strings) { + return getTableEnvironment().scan(strings); + } + + @Override + default Table from(String s) { + return getTableEnvironment().from(s); + } + + @Override + default Table from(TableDescriptor tableDescriptor) { + return getTableEnvironment().from(tableDescriptor); + } + + @Override + default String[] listCatalogs() { + return getTableEnvironment().listCatalogs(); + } + + @Override + default String[] listModules() { + return getTableEnvironment().listModules(); + } + + @Override + default ModuleEntry[] listFullModules() { + return getTableEnvironment().listFullModules(); + } + + @Override + default String[] listDatabases() { + return getTableEnvironment().listDatabases(); + } + + @Override + default String[] listTables() { + return getTableEnvironment().listTables(); + } + + @Override + default String[] listTables(String s, String s1) { + return getTableEnvironment().listTables(); + } + + @Override + default String[] listViews() { + return getTableEnvironment().listViews(); + } + + @Override + default String[] listTemporaryTables() { + return getTableEnvironment().listTemporaryTables(); + } + + @Override + default String[] listTemporaryViews() { + return getTableEnvironment().listTemporaryViews(); + } + + @Override + default String[] listUserDefinedFunctions() { + return getTableEnvironment().listUserDefinedFunctions(); + } + + @Override + default String[] listFunctions() { + return getTableEnvironment().listFunctions(); + } + + @Override + default boolean dropTemporaryTable(String s) { + return getTableEnvironment().dropTemporaryTable(s); + } + + @Override + default boolean dropTemporaryView(String s) { + return getTableEnvironment().dropTemporaryView(s); + } + + @Override + default String explainSql(String s, ExplainDetail... explainDetails) { + return getTableEnvironment().explainSql(s); + } + + @Override + default String[] getCompletionHints(String s, int i) { + return getTableEnvironment().getCompletionHints(s, i); + } + + @Override + default Table sqlQuery(String s) { + return getTableEnvironment().sqlQuery(s); + } + + @Override + default TableResult executeSql(String s) { + return getTableEnvironment().executeSql(s); + } + + @Override + default String getCurrentCatalog() { + return getTableEnvironment().getCurrentCatalog(); + } + + @Override + default void useCatalog(String s) { + getTableEnvironment().useCatalog(s); + } + + @Override + default String getCurrentDatabase() { + return getTableEnvironment().getCurrentDatabase(); + } + + @Override + default void useDatabase(String s) { + getTableEnvironment().useDatabase(s); + } + + @Override + default TableConfig getConfig() { + return getTableEnvironment().getConfig(); + } + + @Override + default StatementSet createStatementSet() { + return getTableEnvironment().createStatementSet(); + } + + @Override + default CompiledPlan loadPlan(PlanReference planReference) throws TableException { + return getTableEnvironment().loadPlan(planReference); + } + + @Override + default CompiledPlan compilePlanSql(String s) throws TableException { + return getTableEnvironment().compilePlanSql(s); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultTableEnvironmentInternal.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultTableEnvironmentInternal.java new file mode 100644 index 0000000000..72f1a3b9b7 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/DefaultTableEnvironmentInternal.java @@ -0,0 +1,112 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.apache.flink.table.api.CompiledPlan; +import org.apache.flink.table.api.ExplainDetail; +import org.apache.flink.table.api.ExplainFormat; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.internal.TableEnvironmentInternal; +import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.delegation.InternalPlan; +import org.apache.flink.table.delegation.Parser; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.utils.OperationTreeBuilder; +import org.apache.flink.table.sinks.TableSink; +import org.apache.flink.table.sources.TableSource; + +import java.util.List; + +/** */ +public interface DefaultTableEnvironmentInternal extends TableEnvironmentInternal, TableEnvironmentInstance { + + default TableEnvironmentInternal getTableEnvironmentInternal() { + return (TableEnvironmentInternal) getTableEnvironment(); + } + + // region TableEnvironmentInternal interface + @Override + default Parser getParser() { + return getTableEnvironmentInternal().getParser(); + } + + @Override + default CatalogManager getCatalogManager() { + return getTableEnvironmentInternal().getCatalogManager(); + } + + @Override + default OperationTreeBuilder getOperationTreeBuilder() { + return getTableEnvironmentInternal().getOperationTreeBuilder(); + } + + @Override + default Table fromTableSource(TableSource tableSource) { + return getTableEnvironmentInternal().fromTableSource(tableSource); + } + + @Override + default TableResultInternal executeInternal(List list) { + return getTableEnvironmentInternal().executeInternal(list); + } + + @Override + default TableResultInternal executeInternal(Operation operation) { + return getTableEnvironmentInternal().executeInternal(operation); + } + + @Override + default String explainInternal(List list, ExplainDetail... explainDetails) { + return getTableEnvironmentInternal().explainInternal(list, explainDetails); + } + + @Override + default void registerTableSourceInternal(String s, TableSource tableSource) { + getTableEnvironmentInternal().registerTableSourceInternal(s, tableSource); + } + + @Override + default void registerTableSinkInternal(String s, TableSink tableSink) { + getTableEnvironmentInternal().registerTableSinkInternal(s, tableSink); + } + + @Override + default CompiledPlan compilePlan(List list) { + return getTableEnvironmentInternal().compilePlan(list); + } + + @Override + default TableResultInternal executePlan(InternalPlan internalPlan) { + return getTableEnvironmentInternal().executePlan(internalPlan); + } + + @Override + default String explainPlan(InternalPlan internalPlan, ExplainDetail... explainDetails) { + return getTableEnvironmentInternal().explainPlan(internalPlan, explainDetails); + } + + @Override + default String explainInternal(List operations, ExplainFormat format, ExplainDetail... extraDetails) { + return getTableEnvironmentInternal().explainInternal(operations, format, extraDetails); + } + // endregion +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ExtendedParser.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ExtendedParser.java new file mode 100644 index 0000000000..42ffd70783 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ExtendedParser.java @@ -0,0 +1,27 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.apache.flink.table.delegation.Parser; + +/** */ +public interface ExtendedParser extends Parser { + CustomParser getCustomParser(); +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ParserWrapper.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ParserWrapper.java new file mode 100644 index 0000000000..922ebf62ea --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/ParserWrapper.java @@ -0,0 +1,70 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; + +import java.util.List; + +import javax.annotation.Nullable; + +public class ParserWrapper implements ExtendedParser { + + private CustomParser customParser; + + public ParserWrapper(CustomParser customParser) { + this.customParser = customParser; + } + + @Override + public List parse(String statement) { + List result = customParser.parse(statement); + if (result != null) { + return result; + } + + return customParser.getParser().parse(statement); + } + + @Override + public UnresolvedIdentifier parseIdentifier(String identifier) { + return customParser.getParser().parseIdentifier(identifier); + } + + @Override + public ResolvedExpression parseSqlExpression( + String sqlExpression, RowType inputRowType, @Nullable LogicalType outputType) { + return customParser.getParser().parseSqlExpression(sqlExpression, inputRowType, outputType); + } + + @Override + public String[] getCompletionHints(String statement, int position) { + return customParser.getParser().getCompletionHints(statement, position); + } + + @Override + public CustomParser getCustomParser() { + return customParser; + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/StaticResultProvider.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/StaticResultProvider.java new file mode 100644 index 0000000000..bc88b875fa --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/StaticResultProvider.java @@ -0,0 +1,119 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.internal.ResultProvider; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.utils.print.PrintStyle; +import org.apache.flink.table.utils.print.RowDataToStringConverter; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; + +import java.util.List; +import java.util.function.Function; + +/** Create result provider from a static set of data using external types. */ +@Internal +public class StaticResultProvider implements ResultProvider { + + /** + * This converter supports only String, long, int and boolean fields. Moreover, this converter + * works only with {@link GenericRowData}. + */ + public static final RowDataToStringConverter SIMPLE_ROW_DATA_TO_STRING_CONVERTER = rowData -> { + GenericRowData genericRowData = (GenericRowData) rowData; + String[] results = new String[rowData.getArity()]; + for (int i = 0; i < results.length; i++) { + Object value = genericRowData.getField(i); + if (Boolean.TRUE.equals(value)) { + results[i] = "TRUE"; + } else if (Boolean.FALSE.equals(value)) { + results[i] = "FALSE"; + } else { + results[i] = value == null ? PrintStyle.NULL_VALUE : "" + value; + } + } + return results; + }; + + private final List rows; + private final Function externalToInternalConverter; + + public StaticResultProvider(List rows) { + this(rows, StaticResultProvider::rowToInternalRow); + } + + public StaticResultProvider(List rows, Function externalToInternalConverter) { + this.rows = rows; + this.externalToInternalConverter = externalToInternalConverter; + } + + @Override + public StaticResultProvider setJobClient(JobClient jobClient) { + return this; + } + + @Override + public CloseableIterator toInternalIterator() { + return CloseableIterator.adapterForIterator( + this.rows.stream().map(this.externalToInternalConverter).iterator()); + } + + @Override + public CloseableIterator toExternalIterator() { + return CloseableIterator.adapterForIterator(this.rows.iterator()); + } + + @Override + public RowDataToStringConverter getRowDataStringConverter() { + return SIMPLE_ROW_DATA_TO_STRING_CONVERTER; + } + + @Override + public boolean isFirstRowReady() { + return true; + } + + /** This function supports only String, long, int and boolean fields. */ + @VisibleForTesting + static RowData rowToInternalRow(Row row) { + Object[] values = new Object[row.getArity()]; + for (int i = 0; i < row.getArity(); i++) { + Object value = row.getField(i); + if (value == null) { + values[i] = null; + } else if (value instanceof String) { + values[i] = StringData.fromString((String) value); + } else if (value instanceof Boolean || value instanceof Long || value instanceof Integer) { + values[i] = value; + } else { + throw new TableException("Cannot convert row type"); + } + } + + return GenericRowData.of(values); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/TableSchemaField.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/TableSchemaField.java new file mode 100644 index 0000000000..88dc81b567 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/executor/TableSchemaField.java @@ -0,0 +1,50 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.executor; + +import org.apache.flink.table.types.DataType; + +/** @since 2022/11/04 */ +public class TableSchemaField { + + private String name; + private DataType type; + + public TableSchemaField(String name, DataType type) { + this.name = name; + this.type = type; + } + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public DataType getType() { + return type; + } + + public void setType(DataType type) { + this.type = type; + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/gateway/FlinkSqlClient.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/gateway/FlinkSqlClient.java new file mode 100644 index 0000000000..6bb269ed5f --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/gateway/FlinkSqlClient.java @@ -0,0 +1,164 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.gateway; + +import org.dinky.utils.CloseUtil; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.FileSystem; +import org.apache.flink.core.plugin.PluginUtils; +import org.apache.flink.table.client.SqlClientException; +import org.apache.flink.table.client.cli.CliClient; +import org.apache.flink.table.client.gateway.Executor; +import org.apache.flink.table.client.gateway.SingleSessionManager; +import org.apache.flink.table.gateway.SqlGateway; +import org.apache.flink.table.gateway.api.endpoint.SqlGatewayEndpointFactoryUtils; +import org.apache.flink.table.gateway.rest.SqlGatewayRestEndpointFactory; +import org.apache.flink.table.gateway.rest.util.SqlGatewayRestOptions; +import org.apache.flink.table.gateway.service.context.DefaultContext; +import org.apache.flink.util.NetUtils; + +import java.io.Closeable; +import java.net.InetSocketAddress; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.function.Supplier; + +import org.jline.terminal.Terminal; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class FlinkSqlClient implements Closeable, ISqlClient { + + public static final String CLI_NAME = "Dinky Flink SQL WEB CLI"; + + private final Supplier terminalFactory; + private final SqlClientOptions sqlClientOptions; + + private Executor executor; + private EmbeddedGateway embeddedGateway; + + public FlinkSqlClient(SqlClientOptions sqlClientOptions, Supplier terminalFactory) { + this.sqlClientOptions = sqlClientOptions; + this.terminalFactory = terminalFactory; + } + + public void startCli(Configuration configuration) { + InetSocketAddress inetSocketAddress; + DefaultContext context = createDefaultContext(configuration); + if (sqlClientOptions.getMode() == SqlCliMode.MODE_EMBEDDED) { + embeddedGateway = EmbeddedGateway.create(context); + inetSocketAddress = + InetSocketAddress.createUnresolved(embeddedGateway.getAddress(), embeddedGateway.getPort()); + } else if (sqlClientOptions.getMode() == SqlCliMode.MODE_GATEWAY) { + inetSocketAddress = sqlClientOptions + .buildConnectAddress() + .orElseThrow(() -> new SqlClientException("Gateway address is not set.")); + } else { + throw new SqlClientException("Unsupported mode: " + sqlClientOptions.getMode()); + } + + executor = Executor.create(context, inetSocketAddress, sqlClientOptions.getSessionId()); + + Path historyFilePath = Paths.get(sqlClientOptions.getHistoryFilePath()); + try (CliClient cli = new CliClient(terminalFactory, executor, historyFilePath)) { + if (sqlClientOptions.getInitSql() != null) { + boolean success = cli.executeInitialization(sqlClientOptions.getInitSql()); + } + cli.executeInInteractiveMode(); + } finally { + close(); + } + log.info("Sql Client exit : " + sqlClientOptions.getConnectAddress()); + } + + private DefaultContext createDefaultContext(Configuration configuration) { + FileSystem.initialize(configuration, PluginUtils.createPluginManagerFromRootFolder(configuration)); + return new DefaultContext(configuration, new ArrayList<>()); + } + + public void close() { + CloseUtil.closeNoErrorPrint(executor, embeddedGateway); + } + + // -------------------------------------------------------------------------------------------- + + private static class EmbeddedGateway implements Closeable { + + private static final String ADDRESS = "localhost"; + + private final NetUtils.Port port; + private final SqlGateway sqlGateway; + + public static EmbeddedGateway create(DefaultContext defaultContext) { + NetUtils.Port port = NetUtils.getAvailablePort(); + + Configuration defaultConfig = defaultContext.getFlinkConfig(); + Configuration restConfig = new Configuration(); + // always use localhost + restConfig.set(SqlGatewayRestOptions.ADDRESS, ADDRESS); + restConfig.set(SqlGatewayRestOptions.BIND_ADDRESS, ADDRESS); + restConfig.set(SqlGatewayRestOptions.PORT, port.getPort()); + restConfig.set(SqlGatewayRestOptions.BIND_PORT, port.getPort() + ""); + defaultConfig.addAll( + restConfig, + SqlGatewayEndpointFactoryUtils.getSqlGatewayOptionPrefix(SqlGatewayRestEndpointFactory.IDENTIFIER)); + SqlGateway sqlGateway = new SqlGateway(defaultConfig, new SingleSessionManager(defaultContext)); + try { + sqlGateway.start(); + log.info("Start embedded gateway on port {}", port.getPort()); + } catch (Throwable t) { + closePort(port); + throw new SqlClientException("Failed to start the embedded sql-gateway.", t); + } + + return new EmbeddedGateway(sqlGateway, port); + } + + private EmbeddedGateway(SqlGateway sqlGateway, NetUtils.Port port) { + this.sqlGateway = sqlGateway; + this.port = port; + } + + String getAddress() { + return ADDRESS; + } + + int getPort() { + return port.getPort(); + } + + @Override + public void close() { + sqlGateway.stop(); + closePort(port); + } + + private static void closePort(NetUtils.Port port) { + try { + port.close(); + } catch (Exception e) { + // ignore + } + } + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/CustomNewParserImpl.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/CustomNewParserImpl.java new file mode 100644 index 0000000000..58b8099d2e --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/CustomNewParserImpl.java @@ -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 org.dinky.operations; + +import org.dinky.parser.CustomParserImpl; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.delegation.Parser; +import org.apache.flink.table.planner.parse.ExtendedParser; + +public class CustomNewParserImpl extends CustomParserImpl { + + private final DinkyParser dinkyParser; + + public CustomNewParserImpl(TableEnvironment tableEnvironment, Parser parser) { + super(parser); + this.dinkyParser = new DinkyParser(tableEnvironment); + } + + @Override + public ExtendedParser getDinkyParser() { + return this.dinkyParser; + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyExecutableOperation.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyExecutableOperation.java new file mode 100644 index 0000000000..f30641265d --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyExecutableOperation.java @@ -0,0 +1,51 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.operations; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.operations.ExecutableOperation; +import org.apache.flink.table.operations.Operation; + +public class DinkyExecutableOperation implements ExecutableOperation { + + private final Operation innerOperation; + private final TableEnvironment tableEnvironment; + + public DinkyExecutableOperation(TableEnvironment tableEnvironment, Operation innerOperation) { + this.tableEnvironment = tableEnvironment; + this.innerOperation = innerOperation; + } + + @Override + public TableResultInternal execute(Context ctx) { + DinkyOperationExecutor operationExecutor = new DinkyOperationExecutor(tableEnvironment, ctx); + return operationExecutor.executeOperation(innerOperation).get(); + } + + public Operation getInnerOperation() { + return innerOperation; + } + + @Override + public String asSummaryString() { + return innerOperation.asSummaryString(); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyOperationExecutor.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyOperationExecutor.java new file mode 100644 index 0000000000..4892fea153 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyOperationExecutor.java @@ -0,0 +1,48 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.operations; + +import org.dinky.executor.CustomTableEnvironment; +import org.dinky.trans.ExtendOperation; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.internal.TableResultInternal; +import org.apache.flink.table.operations.ExecutableOperation; +import org.apache.flink.table.operations.Operation; + +import java.util.Optional; + +public class DinkyOperationExecutor { + private final ExecutableOperation.Context context; + + private final TableEnvironment tableEnvironment; + + public DinkyOperationExecutor(TableEnvironment tableEnvironment, ExecutableOperation.Context context) { + this.tableEnvironment = tableEnvironment; + this.context = context; + } + + public Optional executeOperation(Operation operation) { + ExtendOperation extendOperation = (ExtendOperation) operation; + return Optional.of((TableResultInternal) extendOperation + .execute((CustomTableEnvironment) tableEnvironment) + .get()); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyParser.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyParser.java new file mode 100644 index 0000000000..94ec72e4ae --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/operations/DinkyParser.java @@ -0,0 +1,46 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.operations; + +import org.dinky.parser.DinkyExtendedParser; + +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.planner.parse.ExtendedParseStrategy; + +import java.util.Optional; + +public class DinkyParser extends DinkyExtendedParser { + private final TableEnvironment tableEnvironment; + + public DinkyParser(TableEnvironment tableEnvironment) { + this.tableEnvironment = tableEnvironment; + } + + @Override + public Optional parse(String statement) { + for (ExtendedParseStrategy strategy : PARSE_STRATEGIES) { + if (strategy.match(statement)) { + return Optional.of(new DinkyExecutableOperation(this.tableEnvironment, strategy.convert(statement))); + } + } + return Optional.empty(); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/FlinkUtil.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/FlinkUtil.java new file mode 100644 index 0000000000..3ba2ba4397 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/FlinkUtil.java @@ -0,0 +1,80 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.utils; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.core.execution.SavepointFormatType; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.catalog.CatalogManager; +import org.apache.flink.table.catalog.ContextResolvedTable; +import org.apache.flink.table.catalog.ObjectIdentifier; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutionException; + +/** + * FlinkUtil + * + * @since 2022/05/08 + */ +public class FlinkUtil { + + public static List getFieldNamesFromCatalogManager( + CatalogManager catalogManager, String catalog, String database, String table) { + Optional tableOpt = + catalogManager.getTable(ObjectIdentifier.of(catalog, database, table)); + if (tableOpt.isPresent()) { + return tableOpt.get().getResolvedSchema().getColumnNames(); + } else { + return new ArrayList(); + } + } + + public static List catchColumn(TableResult tableResult) { + return tableResult.getResolvedSchema().getColumnNames(); + } + + public static String triggerSavepoint(ClusterClient clusterClient, String jobId, String savePoint) + throws ExecutionException, InterruptedException { + return clusterClient + .triggerSavepoint(JobID.fromHexString(jobId), savePoint, SavepointFormatType.DEFAULT) + .get() + .toString(); + } + + public static String stopWithSavepoint(ClusterClient clusterClient, String jobId, String savePoint) + throws ExecutionException, InterruptedException { + return clusterClient + .stopWithSavepoint(JobID.fromHexString(jobId), true, savePoint, SavepointFormatType.DEFAULT) + .get() + .toString(); + } + + public static String cancelWithSavepoint(ClusterClient clusterClient, String jobId, String savePoint) + throws ExecutionException, InterruptedException { + return clusterClient + .cancelWithSavepoint(JobID.fromHexString(jobId), savePoint, SavepointFormatType.DEFAULT) + .get() + .toString(); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/FunctionVisitor.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/FunctionVisitor.java new file mode 100644 index 0000000000..1f774297b7 --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/FunctionVisitor.java @@ -0,0 +1,50 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.utils; + +import org.apache.calcite.sql.SqlBasicCall; +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlFunction; +import org.apache.calcite.sql.SqlIdentifier; +import org.apache.calcite.sql.util.SqlBasicVisitor; +import org.apache.flink.table.catalog.UnresolvedIdentifier; + +import java.util.ArrayList; +import java.util.List; + +public class FunctionVisitor extends SqlBasicVisitor { + + private final List functionList = new ArrayList<>(); + + @Override + public Void visit(SqlCall call) { + if (call instanceof SqlBasicCall && call.getOperator() instanceof SqlFunction) { + SqlFunction function = (SqlFunction) call.getOperator(); + SqlIdentifier opName = function.getNameAsId(); + + functionList.add(UnresolvedIdentifier.of(opName.names)); + } + return super.visit(call); + } + + public List getFunctionList() { + return functionList; + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/LineageContext.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/LineageContext.java new file mode 100644 index 0000000000..c924b9c86d --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/LineageContext.java @@ -0,0 +1,194 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.utils; + +import org.dinky.data.model.FunctionResult; +import org.dinky.data.model.LineageRel; +import org.dinky.executor.CustomParser; +import org.dinky.executor.CustomTableEnvironment; + +import org.apache.calcite.plan.RelOptTable; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.metadata.RelColumnOrigin; +import org.apache.calcite.rel.metadata.RelMetadataQuery; +import org.apache.calcite.sql.SqlNode; +import org.apache.commons.collections.CollectionUtils; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.catalog.ContextResolvedFunction; +import org.apache.flink.table.catalog.FunctionCatalog; +import org.apache.flink.table.catalog.UnresolvedIdentifier; +import org.apache.flink.table.functions.FunctionIdentifier; +import org.apache.flink.table.operations.Operation; +import org.apache.flink.table.operations.SinkModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.table.planner.operations.PlannerQueryOperation; +import org.apache.flink.table.planner.plan.schema.TableSourceTable; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * LineageContext + * + * @since 2022/11/22 + */ +public class LineageContext { + + private static final Logger LOG = LoggerFactory.getLogger(LineageContext.class); + + private final CustomTableEnvironment tableEnv; + + public LineageContext(CustomTableEnvironment tableEnv) { + this.tableEnv = tableEnv; + } + + public List analyzeLineage(String statement) { + // 1. Generate original relNode tree + Tuple2 parsed = parseStatement(statement); + String sinkTable = parsed.getField(0); + RelNode oriRelNode = parsed.getField(1); + + // 2. Build lineage based from RelMetadataQuery + return buildFiledLineageResult(sinkTable, oriRelNode); + } + + private Tuple2 parseStatement(String sql) { + List operations = tableEnv.getParser().parse(sql); + + if (operations.size() != 1) { + throw new TableException("Unsupported SQL query! only accepts a single SQL statement."); + } + Operation operation = operations.get(0); + if (operation instanceof SinkModifyOperation) { + SinkModifyOperation sinkOperation = (SinkModifyOperation) operation; + + PlannerQueryOperation queryOperation = (PlannerQueryOperation) sinkOperation.getChild(); + RelNode relNode = queryOperation.getCalciteTree(); + return new Tuple2<>( + sinkOperation.getContextResolvedTable().getIdentifier().asSummaryString(), relNode); + } else { + throw new TableException("Only insert is supported now."); + } + } + + /** Check the size of query and sink fields match */ + private void validateSchema(String sinkTable, RelNode relNode, List sinkFieldList) { + List queryFieldList = relNode.getRowType().getFieldNames(); + if (queryFieldList.size() != sinkFieldList.size()) { + throw new ValidationException(String.format( + "Column types of query result and sink for %s do not match.\n" + + "Query schema: %s\n" + + "Sink schema: %s", + sinkTable, queryFieldList, sinkFieldList)); + } + } + + private List buildFiledLineageResult(String sinkTable, RelNode optRelNode) { + // target columns + List targetColumnList = + tableEnv.from(sinkTable).getResolvedSchema().getColumnNames(); + + // check the size of query and sink fields match + validateSchema(sinkTable, optRelNode, targetColumnList); + + RelMetadataQuery metadataQuery = optRelNode.getCluster().getMetadataQuery(); + List resultList = new ArrayList<>(); + + for (int index = 0; index < targetColumnList.size(); index++) { + String targetColumn = targetColumnList.get(index); + + Set relColumnOriginSet = metadataQuery.getColumnOrigins(optRelNode, index); + + if (CollectionUtils.isNotEmpty(relColumnOriginSet)) { + for (RelColumnOrigin relColumnOrigin : relColumnOriginSet) { + // table + RelOptTable table = relColumnOrigin.getOriginTable(); + String sourceTable = String.join(".", table.getQualifiedName()); + + // filed + int ordinal = relColumnOrigin.getOriginColumnOrdinal(); + List fieldNames = ((TableSourceTable) table) + .contextResolvedTable() + .getResolvedSchema() + .getColumnNames(); + String sourceColumn = fieldNames.get(ordinal); + + // add record + resultList.add(LineageRel.build( + sourceTable, sourceColumn, sinkTable, targetColumn, relColumnOrigin.getTransform())); + } + } + } + return resultList; + } + + /** + * Analyze custom functions from SQL, does not contain system functions. + * + * @param singleSql the SQL statement to analyze + * @return custom functions set + */ + public Set analyzeFunction(String singleSql) { + LOG.info("Analyze function Sql: \n {}", singleSql); + CustomParser parser = (CustomParser) tableEnv.getParser(); + + // parsing sql and return the abstract syntax tree + SqlNode sqlNode = parser.parseSql(singleSql); + + // validate the query + SqlNode validated = parser.validate(sqlNode); + + // look for all functions + FunctionVisitor visitor = new FunctionVisitor(); + validated.accept(visitor); + List fullFunctionList = visitor.getFunctionList(); + + // filter custom functions + Set resultSet = new HashSet<>(); + for (UnresolvedIdentifier unresolvedIdentifier : fullFunctionList) { + getFunctionCatalog() + .lookupFunction(unresolvedIdentifier) + .flatMap(ContextResolvedFunction::getIdentifier) + // the objectIdentifier of the built-in function is null + .flatMap(FunctionIdentifier::getIdentifier) + .ifPresent(identifier -> { + FunctionResult functionResult = new FunctionResult() + .setCatalogName(identifier.getCatalogName()) + .setDatabase(identifier.getDatabaseName()) + .setFunctionName(identifier.getObjectName()); + LOG.debug("analyzed function: {}", functionResult); + resultSet.add(functionResult); + }); + } + return resultSet; + } + + private FunctionCatalog getFunctionCatalog() { + PlannerBase planner = (PlannerBase) tableEnv.getPlanner(); + return planner.getFlinkContext().getFunctionCatalog(); + } +} diff --git a/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/ObjectConvertUtil.java b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/ObjectConvertUtil.java new file mode 100644 index 0000000000..a8000ff74d --- /dev/null +++ b/dinky-client/dinky-client-1.20/src/main/java/org/dinky/utils/ObjectConvertUtil.java @@ -0,0 +1,90 @@ +/* + * + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * + */ + +package org.dinky.utils; + +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarBinaryType; + +import java.math.BigDecimal; +import java.time.Instant; +import java.time.ZoneId; + +import javax.xml.bind.DatatypeConverter; + +/** @className: org.dinky.utils.ObjectConvertUtil @Description: */ +public class ObjectConvertUtil { + + public static Object convertValue(Object value, LogicalType logicalType) { + return ObjectConvertUtil.convertValue(value, logicalType, null); + } + + public static Object convertValue(Object value, LogicalType logicalType, ZoneId sinkTimeZone) { + if (value == null) { + return null; + } + if (sinkTimeZone == null) { + sinkTimeZone = ZoneId.of("UTC"); + } + if (logicalType instanceof DateType) { + if (value instanceof Integer) { + return Instant.ofEpochMilli(((Integer) value).longValue()) + .atZone(sinkTimeZone) + .toLocalDate(); + } else { + return Instant.ofEpochMilli((long) value) + .atZone(ZoneId.systemDefault()) + .toLocalDate(); + } + } else if (logicalType instanceof TimestampType) { + if (value instanceof Integer) { + return Instant.ofEpochMilli(((Integer) value).longValue()) + .atZone(sinkTimeZone) + .toLocalDateTime(); + } else if (value instanceof String) { + return Instant.parse((String) value) + .atZone(ZoneId.systemDefault()) + .toLocalDateTime(); + } else { + return Instant.ofEpochMilli((long) value).atZone(sinkTimeZone).toLocalDateTime(); + } + } else if (logicalType instanceof DecimalType) { + return new BigDecimal(String.valueOf(value)); + } else if (logicalType instanceof BigIntType) { + if (value instanceof Integer) { + return ((Integer) value).longValue(); + } else { + return value; + } + } else if (logicalType instanceof VarBinaryType) { + // VARBINARY AND BINARY is converted to String with encoding base64 in FlinkCDC. + if (value instanceof String) { + return DatatypeConverter.parseBase64Binary((String) value); + } else { + return value; + } + } else { + return value; + } + } +} diff --git a/dinky-client/pom.xml b/dinky-client/pom.xml index a61b34f447..8be3c04091 100644 --- a/dinky-client/pom.xml +++ b/dinky-client/pom.xml @@ -52,6 +52,7 @@ dinky-client-1.17 dinky-client-1.18 dinky-client-1.19 + dinky-client-1.20 diff --git a/dinky-flink/dinky-flink-1.14/pom.xml b/dinky-flink/dinky-flink-1.14/pom.xml index 952810b1af..03febcafa4 100644 --- a/dinky-flink/dinky-flink-1.14/pom.xml +++ b/dinky-flink/dinky-flink-1.14/pom.xml @@ -174,7 +174,7 @@ com.starrocks flink-connector-starrocks - 1.2.3_flink-1.14_2.12 + 1.2.9_flink-1.14_2.12 com.github.jsqlparser diff --git a/dinky-flink/dinky-flink-1.15/pom.xml b/dinky-flink/dinky-flink-1.15/pom.xml index 500626ced8..b649001ee2 100644 --- a/dinky-flink/dinky-flink-1.15/pom.xml +++ b/dinky-flink/dinky-flink-1.15/pom.xml @@ -133,7 +133,7 @@ com.starrocks flink-connector-starrocks - 1.2.6_flink-1.15 + 1.2.9_flink-1.15 com.github.jsqlparser @@ -184,7 +184,7 @@ org.apache.doris flink-doris-connector-1.15 - 1.2.1 + 1.6.2 org.apache.flink diff --git a/dinky-flink/dinky-flink-1.16/pom.xml b/dinky-flink/dinky-flink-1.16/pom.xml index 316bd7bf9c..c550e2ce04 100644 --- a/dinky-flink/dinky-flink-1.16/pom.xml +++ b/dinky-flink/dinky-flink-1.16/pom.xml @@ -16,7 +16,7 @@ 1.3.1 - 16.0 + 16.2 1.16.3 3.1.1 @@ -138,7 +138,7 @@ org.apache.doris flink-doris-connector-1.16 - 1.4.0 + 1.6.2 org.apache.flink diff --git a/dinky-flink/dinky-flink-1.17/pom.xml b/dinky-flink/dinky-flink-1.17/pom.xml index d190ae60cf..75baf20e2a 100644 --- a/dinky-flink/dinky-flink-1.17/pom.xml +++ b/dinky-flink/dinky-flink-1.17/pom.xml @@ -16,7 +16,7 @@ 1.3.1 - 16.1 + 17.0 1.17.2 3.1.1 @@ -44,11 +44,15 @@ flink-statebackend-rocksdb ${flink.version} - + + org.apache.flink + flink-connector-base + ${flink.version} + org.apache.flink flink-connector-jdbc - 3.1.0-1.17 + 3.1.2-1.17 org.apache.flink @@ -79,12 +83,13 @@ org.apache.flink flink-connector-kafka - ${flink.version} + 3.1.0-1.17 + org.apache.flink flink-shaded-guava - 30.1.1-jre-${flink.shaded.version} + 30.1.1-jre-16.2 org.apache.flink @@ -138,7 +143,7 @@ org.apache.doris flink-doris-connector-1.17 - 1.4.0 + 1.6.2 org.apache.flink diff --git a/dinky-flink/dinky-flink-1.18/pom.xml b/dinky-flink/dinky-flink-1.18/pom.xml index efecd8112a..1d224bd4e3 100644 --- a/dinky-flink/dinky-flink-1.18/pom.xml +++ b/dinky-flink/dinky-flink-1.18/pom.xml @@ -16,7 +16,7 @@ 1.3.1 - 17.0 + 18.0 1.18.1 3.1.1 @@ -38,10 +38,15 @@ + + org.apache.flink + flink-connector-base + ${flink.version} + org.apache.flink flink-connector-jdbc - 3.1.1-1.17 + 3.2.0-1.18 org.apache.flink @@ -77,12 +82,12 @@ org.apache.flink flink-connector-kafka - 3.0.1-1.18 + 3.2.0-1.18 org.apache.flink flink-shaded-guava - 31.1-jre-${flink.shaded.version} + 32.1.3-jre-${flink.shaded.version} org.apache.flink @@ -136,7 +141,7 @@ org.apache.doris flink-doris-connector-1.18 - 1.5.2 + 1.6.2 org.apache.flink diff --git a/dinky-flink/dinky-flink-1.19/pom.xml b/dinky-flink/dinky-flink-1.19/pom.xml index f41839fa1d..e7d460224f 100644 --- a/dinky-flink/dinky-flink-1.19/pom.xml +++ b/dinky-flink/dinky-flink-1.19/pom.xml @@ -16,8 +16,8 @@ 1.5.0 - 17.0 - 1.19.0 + 19.0 + 1.19.1 3.1.1 @@ -38,10 +38,15 @@ + + org.apache.flink + flink-connector-base + ${flink.version} + org.apache.flink flink-connector-jdbc - 3.1.1-1.17 + 3.2.0-1.19 org.apache.flink @@ -77,12 +82,12 @@ org.apache.flink flink-connector-kafka - 3.0.1-1.18 + 3.2.0-1.19 org.apache.flink flink-shaded-guava - 31.1-jre-${flink.shaded.version} + 32.1.3-jre-${flink.shaded.version} org.apache.flink @@ -135,8 +140,8 @@ org.apache.doris - flink-doris-connector-1.18 - 1.5.2 + flink-doris-connector-1.19 + 1.6.2 org.apache.flink diff --git a/dinky-flink/dinky-flink-1.20/pom.xml b/dinky-flink/dinky-flink-1.20/pom.xml new file mode 100644 index 0000000000..1db51f7273 --- /dev/null +++ b/dinky-flink/dinky-flink-1.20/pom.xml @@ -0,0 +1,172 @@ + + + 4.0.0 + + org.dinky + dinky-flink + ${revision} + ../pom.xml + + dinky-flink-1.20 + + jar + + Dinky : Flink 1.20 + + + 1.5.0 + 19.0 + 1.20.0 + 3.1.1 + + + + + org.apache.flink + flink-python + ${flink.version} + + + org.apache.flink + flink-table-planner_2.12 + ${flink.version} + + + org.slf4j + slf4j-api + + + + + org.apache.flink + flink-connector-base + ${flink.version} + + + org.apache.flink + flink-connector-jdbc + 3.2.0-1.19 + + + org.apache.flink + flink-statebackend-rocksdb + ${flink.version} + + + org.apache.flink + flink-table-api-scala-bridge_2.12 + ${flink.version} + + + org.apache.flink + flink-table-common + ${flink.version} + + + org.apache.flink + flink-yarn + ${flink.version} + + + org.slf4j + slf4j-api + + + + + org.apache.flink + flink-kubernetes + ${flink.version} + + + org.apache.flink + flink-connector-kafka + 3.2.0-1.19 + + + org.apache.flink + flink-shaded-guava + 32.1.3-jre-${flink.shaded.version} + + + org.apache.flink + flink-sql-connector-mysql-cdc + ${flinkcdc.version} + + + org.apache.flink + flink-sql-connector-oracle-cdc + ${flinkcdc.version} + + + org.apache.flink + flink-sql-connector-sqlserver-cdc + ${flinkcdc.version} + + + org.apache.flink + flink-sql-connector-postgres-cdc + ${flinkcdc.version} + + + org.apache.flink + flink-cdc-cli + ${flinkcdc.version} + + + org.apache.flink + flink-cdc-pipeline-connector-mysql + ${flinkcdc.version} + + + org.apache.flink + flink-cdc-pipeline-connector-doris + ${flinkcdc.version} + + + org.apache.flink + flink-cdc-pipeline-connector-starrocks + ${flinkcdc.version} + + + org.slf4j + slf4j-api + + + commons-cli + commons-cli + ${commons.version} + + + org.apache.doris + flink-doris-connector-1.19 + 1.6.2 + + + org.apache.flink + flink-runtime-web + ${flink.version} + + + org.apache.flink + flink-state-processor-api + ${flink.version} + + + org.apache.flink + flink-s3-fs-presto + ${flink.version} + + + org.apache.flink + flink-sql-gateway + ${flink.version} + + + org.apache.flink + flink-sql-client + ${flink.version} + + + diff --git a/dinky-flink/pom.xml b/dinky-flink/pom.xml index 0d49ae09b5..68162fca94 100644 --- a/dinky-flink/pom.xml +++ b/dinky-flink/pom.xml @@ -47,6 +47,7 @@ dinky-flink-1.17 dinky-flink-1.18 dinky-flink-1.19 + dinky-flink-1.20 diff --git a/dinky-gateway/src/main/java/org/dinky/gateway/yarn/YarnGateway.java b/dinky-gateway/src/main/java/org/dinky/gateway/yarn/YarnGateway.java index c0bc96ca9d..5277eba2d2 100644 --- a/dinky-gateway/src/main/java/org/dinky/gateway/yarn/YarnGateway.java +++ b/dinky-gateway/src/main/java/org/dinky/gateway/yarn/YarnGateway.java @@ -90,6 +90,7 @@ import cn.hutool.core.collection.CollUtil; import cn.hutool.core.collection.CollectionUtil; +import cn.hutool.core.convert.Convert; import cn.hutool.core.io.FileUtil; import cn.hutool.core.lang.Assert; import cn.hutool.core.util.ReUtil; @@ -472,7 +473,7 @@ public String getLatestJobManageHost(String appId, String oldJobManagerHost) { + HighAvailabilityOptions.HA_ZOOKEEPER_QUORUM.key() + "'."); } - int sessionTimeout = configuration.getInteger(HighAvailabilityOptions.ZOOKEEPER_SESSION_TIMEOUT); + int sessionTimeout = Convert.toInt(configuration.get(HighAvailabilityOptions.ZOOKEEPER_SESSION_TIMEOUT)); String root = configuration.getValue(HighAvailabilityOptions.HA_ZOOKEEPER_ROOT); String namespace = configuration.getValue(HighAvailabilityOptions.HA_CLUSTER_ID); diff --git a/pom.xml b/pom.xml index 97f26fb1d7..a483cde097 100644 --- a/pom.xml +++ b/pom.xml @@ -442,6 +442,11 @@ dinky-client-1.19 ${project.version} + + org.dinky + dinky-client-1.20 + ${project.version} + org.dinky dinky-catalog-mysql-1.14 @@ -472,6 +477,11 @@ dinky-catalog-mysql-1.19 ${project.version} + + org.dinky + dinky-catalog-mysql-1.20 + ${project.version} + org.dinky dinky-connector-jdbc-1.14 @@ -638,6 +648,11 @@ dinky-flink-1.19 ${project.version} + + org.dinky + dinky-flink-1.20 + ${project.version} + org.dinky dinky-cdc-core @@ -891,7 +906,7 @@ - http://www.dinky.org.cn/ + https://www.dinky.org.cn/ 2021 @@ -1090,6 +1105,12 @@ 1.19 + + flink-1.20 + + 1.20 + + flink-single-version