From d64ce2baa96b8a9753754fb56c1e72e93a813f59 Mon Sep 17 00:00:00 2001 From: Cadenana <768972002@qq.com> Date: Tue, 19 Nov 2024 19:52:04 +0800 Subject: [PATCH 1/3] StarRocks StageMode use Signed-off-by: Cadenana <768972002@qq.com> --- .idea/.gitignore | 10 - ...7\347\224\250\346\226\207\346\241\2431.md" | 138 +++++++++ pom.xml | 48 ++-- .../connector/spark/catalog/SRCatalog.java | 237 +++++++++++++++ .../connector/spark/catalog/SRColumn.java | 181 ++++++++++++ .../connector/spark/catalog/SRTable.java | 202 +++++++++++++ .../spark/cfg/ConfigurationOptions.java | 2 +- .../spark/exception/CatalogException.java | 42 +++ .../connector/spark/sql/StarRocksTable.java | 2 +- .../spark/sql/conf/StarRocksConfigBase.java | 25 +- .../spark/sql/conf/WriteStarRocksConfig.java | 94 +++++- .../spark/sql/connect/StarRocksConnector.java | 132 +++++++-- .../spark/sql/schema/StarRocksSchema.java | 4 +- .../spark/sql/write/StarRocksDataWriter.java | 9 +- .../spark/sql/write/StarRocksWrite.java | 272 +++++++++++++++++- .../sql/write/StarRocksWriteBuilder.java | 20 +- .../sql/write/StarRocksWriterFactory.java | 1 - .../spark/catalog/StarRocksCatalog.scala | 1 + .../connector/spark/examples/SimpleWrite.java | 12 +- .../connector/spark/sql/ITTestBase.java | 7 +- .../connector/spark/sql/ReadWriteITTest.java | 262 ++++++++++++++--- 21 files changed, 1570 insertions(+), 131 deletions(-) delete mode 100644 .idea/.gitignore create mode 100644 "docs/StarRocks-Stage\346\250\241\345\274\217\344\275\277\347\224\250\346\226\207\346\241\2431.md" create mode 100644 src/main/java/com/starrocks/connector/spark/catalog/SRCatalog.java create mode 100644 src/main/java/com/starrocks/connector/spark/catalog/SRColumn.java create mode 100644 src/main/java/com/starrocks/connector/spark/catalog/SRTable.java create mode 100644 src/main/java/com/starrocks/connector/spark/exception/CatalogException.java diff --git a/.idea/.gitignore b/.idea/.gitignore deleted file mode 100644 index 0a8642fa..00000000 --- a/.idea/.gitignore +++ /dev/null @@ -1,10 +0,0 @@ -# Default ignored files -/shelf/ -/workspace.xml -# Editor-based HTTP Client requests -/httpRequests/ -# Datasource local storage ignored files -/dataSources/ -/dataSources.local.xml -# Zeppelin ignored files -/ZeppelinRemoteNotebooks/ diff --git "a/docs/StarRocks-Stage\346\250\241\345\274\217\344\275\277\347\224\250\346\226\207\346\241\2431.md" "b/docs/StarRocks-Stage\346\250\241\345\274\217\344\275\277\347\224\250\346\226\207\346\241\2431.md" new file mode 100644 index 00000000..61d6b359 --- /dev/null +++ "b/docs/StarRocks-Stage\346\250\241\345\274\217\344\275\277\347\224\250\346\226\207\346\241\2431.md" @@ -0,0 +1,138 @@ +## 列模式部分更新-暂存区模式 + +在列模式部分更新的场景中,一般涉及少数列大量行,暂存区模式适用于某一列或多列更新的数据行占比很大,甚至近似于全列更新的场景。 + +配置参数starrocks.write.stage.use用于控制暂存区模式的使用,支持取值为 + +always + +指定使用部分列更新时采用暂存区模式,适用于更新时行占比很大的场景 + +auto + +系统会根据更新数据涉及的列以及目标表的列数,行占比ratio决定是否使用暂存区模式 + +更新数据的行占比超过60,且更新列数少于4个 + +更新列数占所有列数的百分比小于25% + +反之,系统不会使用暂存区模式 + +starrocks.write.stage.columns.update.ratio:被更新数据列的行占比,可指定占比,默认20 + +never(默认值) + +指定不使用暂存区模式,涉及行占比不大时采用 + + + +**sql配置** + +暂存区模式采用在StarRocks中创建临时表实现,使用暂存区模式更新导入数据时需要执行update语句,可以使用StarRocks系统变量配置sql执行的参数,对应参数为starrocks.write.stage.session.* + +参数映射:exec_mem_limit->exec.mem.limit //todo + +比如:.option ("starrocks.write.stage.session.exec.mem.limit","8589934592") ,可以实现update执行时的StarRocks内存限制。 + +**暂存表清理** + +正常流程中,在执行完update后会将暂存表清楚,完成整个部分列更新流程。一些特殊情况下暂存表未被正确删除,会占用数据库空间。可以定期执行SparkDrop作业,删除存在时长超过一天的暂存表实现对冗余暂存表的清理。 + +参数: + +1. FE MySQL Server 端口(默认127.0.0.1:9030) + +2.用户名(username) + +3.密码(password) + +4.Spark作业执行环境(默认local[*]) 用--master指定也可(优先级更高) + +用例 + +``` +spark-submit \ + --class com.TableDrop.SparkDrop \ + --master local[*] \ +/opt/SparkTask/original-SparkDrop-1.0-SNAPSHOT.jar 192.168.181.1:9030 root "password" local[*] +``` + +**使用示例** + +建表语句: + +``` +CREATE TABLE `test`.`stage_test` +( + `id` BIGINT NOT NULL COMMENT "主键", + `name` VARCHAR(65533) NULL , + `score` INT NOT NULL , + `is_active` BOOLEAN NULL , + `decimal_score` DECIMAL(10, 2) NULL , + `double_value` DOUBLE NULL , + `float_value` FLOAT NULL , + `largeint_value` LARGEINT NULL , + `smallint_value` SMALLINT NULL , + `tinyint_value` TINYINT NULL , + `char_value` CHAR(10) NULL , + `string_value` STRING NULL , + `create_date` DATE NULL +) +ENGINE=OLAP +PRIMARY KEY(`id`) +COMMENT "OLAP" +DISTRIBUTED BY HASH(`id`) +BUCKETS 4; + +``` + +插入数据 + +``` +INSERT INTO `test`.`stage_test` (`id`, `name`, `score`, `is_active`, `decimal_score`, `double_value`, `float_value`, `largeint_value`, `smallint_value`, `tinyint_value`, `char_value`, `string_value`, `create_date`) +VALUES + (1, 'Alice', 95, true, 95.50, 95.5, 95.5, 10000000000, 100, 10, 'A', 'Alice String', '2024-01-01'), + (2, 'Bob', 88, true, 88.00, 88.0, 88.0, 20000000000, 90, 9, 'B', 'Bob String', '2024-01-02'), + (3, 'Charlie', 76, false, 76.75, 76.75, 76.75, 30000000000, 80, 8, 'C', 'Charlie String', '2024-01-03'), + (4, 'David', 82, true, 82.25, 82.25, 82.25, 40000000000, 70, 7, 'D', 'David String', '2024-01-04'), + (5, 'Eva', 90, true, 90.00, 90.0, 90.0, 50000000000, 60, 6, 'E', 'Eva String', '2024-01-05'), + (6, 'Frank', 65, false, 65.50, 65.5, 65.5, 60000000000, 50, 5, 'F', 'Frank String', '2024-01-06'), + (7, 'Grace', 70, true, 70.00, 70.0, 70.0, 70000000000, 40, 4, 'G', 'Grace String', '2024-01-07'), + (8, 'Heidi', 80, true, 80.00, 80.0, 80.0, 80000000000, 30, 3, 'H', 'Heidi String', '2024-01-08'), + (9, 'Ivan', 92, true, 92.00, 92.0, 92.0, 90000000000, 20, 2, 'I', 'Ivan String', '2024-01-09'), + (10, 'Judy', 85, false, 85.00, 85.0, 85.0, 10000000000, 10, 1, 'J', 'Judy String', '2024-01-10'); + +``` + +以Scala语言为例,在Spark-Shell中运行 + +``` +import org.apache.spark.sql.SparkSession + +val data = Seq((1, "starrocks", 124578), (2, "spark", 235689)) +val df = data.toDF("id", "name", "score") + +df.write.format("starrocks") + .option("starrocks.fe.http.url", "http://127.0.0.1:8030") + .option("starrocks.fe.jdbc.url", "jdbc:mysql://127.0.0.1:9030") + .option("starrocks.table.identifier", "test.stage_test") + .option("starrocks.user", "root") + .option("starrocks.password", "") + .option("starrocks.write.stage.use","always") + .option("starrocks.write.properties.partial_update_mode","column") + .option("starrocks.write.properties.partial_update","true") + .option("starrocks.columns", "id,name,score") + .option("starrocks.write.stage.session.query.timeout","309") + .option("starrocks.write.stage.session.query.mem.limit","100000000") + .option("starrocks.write.stage.session.exec.mem.limit","8589934592") + .option("starrocks.write.stage.columns.update.ratio","80") + .mode("append") + .save() +``` + +运行后会在temp_db下创建具有"id", "name", "score"字段的临时表,并更新test.stage_test,更新完成后删除临时表。 + + + + + diff --git a/pom.xml b/pom.xml index aaa28ecc..d5d51d7c 100644 --- a/pom.xml +++ b/pom.xml @@ -497,9 +497,9 @@ src/test/resources/sql/** **/target/** **/*.md - - - + + + .github/** @@ -641,25 +641,25 @@ - - ossrh - https://s01.oss.sonatype.org/content/repositories/snapshots - - - ossrh - https://s01.oss.sonatype.org/service/local/staging/deploy/maven2/ - - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - 3.1.0 - - checkstyle.xml - - - - + + ossrh + https://s01.oss.sonatype.org/content/repositories/snapshots + + + ossrh + https://s01.oss.sonatype.org/service/local/staging/deploy/maven2/ + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + 3.1.0 + + checkstyle.xml + + + + diff --git a/src/main/java/com/starrocks/connector/spark/catalog/SRCatalog.java b/src/main/java/com/starrocks/connector/spark/catalog/SRCatalog.java new file mode 100644 index 00000000..c997ee2b --- /dev/null +++ b/src/main/java/com/starrocks/connector/spark/catalog/SRCatalog.java @@ -0,0 +1,237 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +package com.starrocks.connector.spark.catalog; + +import com.google.common.base.Preconditions; +import com.starrocks.connector.spark.exception.CatalogException; +import com.starrocks.connector.spark.sql.connect.StarRocksConnector; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +public class SRCatalog implements Serializable { +//supply someBase sql Method for BatchWrite StageMode + +private static final long serialVersionUID = 1L; + +private static final Logger LOG = LoggerFactory.getLogger(StarRocksCatalog.class); + + private final String jdbcUrl; + private final String username; + private final String password; + public SRCatalog(String jdbcUrl, String username, String password) { + this.jdbcUrl = jdbcUrl; + this.username = username; + this.password = password; + } + public Connection getJdbcConnection() throws Exception { + Connection jdbcConnection = StarRocksConnector.createJdbcConnection(jdbcUrl, username, password); + return jdbcConnection; + } + +public void executeDDLBySql(String ddlSql) throws Exception { + try (Connection connection = getJdbcConnection()) { + Statement statement = connection.createStatement(); + statement.execute(ddlSql); + } + catch (Exception e) { + throw new IllegalStateException("execute ddl error, " + e.getMessage(), e); + } +} + + //create db if not exists + public void createDatabase (String dbNames,boolean ignoreIfExists) { + if (StringUtils.isEmpty(dbNames)) { + throw new CatalogException("create db error"); + } + try { + String buildSql = buildCreateDatabaseSql(dbNames, ignoreIfExists); + executeDDLBySql(buildSql); + } + catch (Exception e) { + + throw new CatalogException("create db error", e); + } + LOG.info("successfully created database {}", dbNames); + } + + public void createTable(SRTable table, boolean ignoreIfExists) + { + try { + String sql = buildCreateTableSql(table, ignoreIfExists); + executeDDLBySql(sql); + } + catch (Exception e) { + throw new CatalogException("create table error", e); + } + LOG.info("successfully created {}", table); + } + + public void dropTable(String dbName,String tableName) { + String sql = String.format( + "SELECT TABLE_NAME FROM information_schema.tables WHERE TABLE_SCHEMA = '%s' AND TABLE_NAME = '%s'", + dbName, tableName + ); + try (Connection connection = getJdbcConnection(); Statement stmt = connection.createStatement()) { + ResultSet resultSet = stmt.executeQuery(sql); + if (resultSet.next()) { + stmt.executeUpdate("DROP TABLE `" + dbName + "`.`" + tableName + "`"); + } + } +catch (Exception e) { + throw new CatalogException("drop table error", e); +} + LOG.info("successfully dropped {}", tableName); + } + + private String buildCreateDatabaseSql(String databaseName, boolean ignoreIfExists) { + StringBuilder sql = new StringBuilder("CREATE DATABASE "); + if (ignoreIfExists) { + sql.append("IF NOT EXISTS "); + } + sql.append(databaseName); + return sql.toString(); + } + + private String buildCreateTableSql(SRTable table, boolean ignoreIfExists) { + StringBuilder builder = new StringBuilder(); + builder.append( + String.format( + "CREATE TABLE %s`%s`.`%s`", + ignoreIfExists ? "IF NOT EXISTS " : "", + table.getDatabaseName(), + table.getTableName())); + builder.append(" (\n"); + + List tableKeys = table.getTableKeys().orElse(Collections.emptyList()); + + String columnsStmt = + table.getColumns().stream() + .map(column -> buildColumnStmt(column, tableKeys)) + .collect(Collectors.joining(",\n")); + + builder.append(columnsStmt); + builder.append("\n) "); + + String keyModel; + switch (table.getTableType()) { + case DUPLICATE_KEYS: + keyModel = "DUPLICATE KEY"; + break; + case PRIMARY_KEYS: + keyModel = "PRIMARY KEY"; + break; + case UNIQUE_KEYS: + keyModel = "UNIQUE KEY"; + break; + default: + throw new UnsupportedOperationException( + "Not support to build create table sql for table type " + table.getTableType()); + } + + builder.append(String.format("%s (%s)\n", keyModel, String.join(", ", tableKeys))); + + if (!table.getDistributionKeys().isPresent()) { + Preconditions.checkArgument( + table.getTableType() == SRTable.TableType.DUPLICATE_KEYS, + "Can't build create table sql because there is no distribution keys"); + } else { + String distributionKeys = + table.getDistributionKeys().get().stream() + .map(key -> "`" + key + "`") + .collect(Collectors.joining(", ")); + builder.append(String.format("DISTRIBUTED BY HASH (%s)", distributionKeys)); + } + + if (table.getNumBuckets().isPresent()) { + builder.append(" BUCKETS "); + builder.append(table.getNumBuckets().get()); + } + + if (!table.getProperties().isEmpty()) { + builder.append("\nPROPERTIES (\n"); + String properties = + table.getProperties().entrySet().stream() + .map(entry -> String.format("\"%s\" = \"%s\"", entry.getKey(), entry.getValue())) + .collect(Collectors.joining(",\n")); + builder.append(properties); + builder.append("\n)"); + } + + builder.append(";"); + return builder.toString(); + } + + private String buildColumnStmt(SRColumn column, List tableKeys) { + StringBuilder builder = new StringBuilder(); + builder.append("`"); + builder.append(column.getColumnName()); + builder.append("` "); + + builder.append( + getFullColumnType( + column.getDataType(), column.getColumnSize(), column.getDecimalDigits())); + builder.append(" "); + + if (tableKeys.contains(column.getColumnName())) { + builder.append("NOT NULL"); + } else { + builder.append(column.isNullable() ? "NULL" : "NOT NULL"); + } + + if (column.getDefaultValue().isPresent()) { + builder.append(String.format(" DEFAULT \"%s\"", column.getDefaultValue().get())); + } + + if (column.getColumnComment().isPresent()) { + builder.append(String.format(" COMMENT \"%s\"", column.getColumnComment().get())); + } + + return builder.toString(); + } + + private String getFullColumnType( + String type, Optional columnSize, Optional decimalDigits) { + String dataType = type.toUpperCase(); + switch (dataType) { + case "DECIMAL": + Preconditions.checkArgument( + columnSize.isPresent(), "DECIMAL type must have column size"); + Preconditions.checkArgument( + decimalDigits.isPresent(), "DECIMAL type must have decimal digits"); + return String.format("DECIMAL(%d, %s)", columnSize.get(), decimalDigits.get()); + case "CHAR": + case "VARCHAR": + Preconditions.checkArgument( + columnSize.isPresent(), type + " type must have column size"); + return String.format("%s(%d)", dataType, columnSize.get()); + default: + return dataType; + } + } +} diff --git a/src/main/java/com/starrocks/connector/spark/catalog/SRColumn.java b/src/main/java/com/starrocks/connector/spark/catalog/SRColumn.java new file mode 100644 index 00000000..95f03c6d --- /dev/null +++ b/src/main/java/com/starrocks/connector/spark/catalog/SRColumn.java @@ -0,0 +1,181 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +package com.starrocks.connector.spark.catalog; + +import com.google.common.base.Preconditions; +import javax.annotation.Nullable; +import java.io.Serializable; +import java.util.Objects; +import java.util.Optional; +//to describe a column of SR, info from information_schema.COLUMNS +public class SRColumn implements Serializable { + private static final long serialVersionUID = 1L; + + private final String columnName; + //column's position + private final int ordinalPosition; + //column type + private final String dataType; + //can be null or not + private final boolean isNullable; + // column's defaultValue + @Nullable + private final String defaultValue; + //column' maxSize + @Nullable + private final Integer columnSize; + //decimal 0 0.1 null if not a number + @Nullable + private final Integer decimalDigits; + //comment about a column + @Nullable + private final String columnComment; + private SRColumn( + String columnName, + int ordinalPosition, + String dataType, + boolean isNullable, + @Nullable String defaultValue, + @Nullable Integer columnSize, + @Nullable Integer decimalDigits, + @Nullable String columnComment) { + this.columnName = Preconditions.checkNotNull(columnName); + this.ordinalPosition = ordinalPosition; + this.dataType = Preconditions.checkNotNull(dataType); + this.isNullable = isNullable; + this.defaultValue = defaultValue; + this.columnSize = columnSize; + this.decimalDigits = decimalDigits; + this.columnComment = columnComment; + } + public String getColumnName() { + return columnName; + } + public int getOrdinalPosition() { + return ordinalPosition; + } + public String getDataType() { + return dataType; + } + public boolean isNullable() { + return isNullable; + } + public Optional getDefaultValue() { + return Optional.ofNullable(defaultValue); + } + public Optional getColumnSize() { + return Optional.ofNullable(columnSize); + } + public Optional getDecimalDigits() { + return Optional.ofNullable(decimalDigits); + } + public Optional getColumnComment() { + return Optional.ofNullable(columnComment); + } + @Override + public String toString() { + return "StarRocksColumn{" + + "columnName='" + columnName + '\'' + + ", ordinalPosition=" + ordinalPosition + + ", dataType='" + dataType + '\'' + + ", isNullable=" + isNullable + + ", defaultValue='" + defaultValue + '\'' + + ", columnSize=" + columnSize + + ", decimalDigits=" + decimalDigits + + ", columnComment='" + columnComment + '\'' + + '}'; + } + public static class Builder { + private String columnName; + private int ordinalPosition; + private String dataType; + private boolean isNullable = true; + private String defaultValue; + private Integer columnSize; + private Integer decimalDigits; + private String columnComment; + public SRColumn.Builder setColumnName(String columnName) { + this.columnName = columnName; + return this; + } + public SRColumn.Builder setOrdinalPosition(int ordinalPosition) { + this.ordinalPosition = ordinalPosition; + return this; + } + public SRColumn.Builder setDataType(String dataType,Integer size) { + if ("tinyint".equalsIgnoreCase(dataType) && size == null) { + this.dataType = "boolean"; + } + else { + this.dataType = dataType; + } + + return this; + } + public SRColumn.Builder setDataType(String dataType) { + this.dataType = dataType; + return this; + } + public SRColumn.Builder setNullable(String isNULL) { + + if ("YES".equalsIgnoreCase(isNULL)) { + this.isNullable = true; + } + else + { + this.isNullable = false; + } + return this; + } + public SRColumn.Builder setDefaultValue(String defaultValue) { + this.defaultValue = defaultValue; + return this; + } + public SRColumn.Builder setColumnSize(Integer columnSize) { + this.columnSize = columnSize; + return this; + } + public SRColumn.Builder setDecimalDigits(String decimalDigits) { + if (Objects.isNull(decimalDigits)) { + this.decimalDigits = null; + } + else + { + this.decimalDigits = Integer.parseInt(decimalDigits); + } + return this; + } + + public SRColumn.Builder setColumnComment(String columnComment) { + this.columnComment = columnComment; + return this; + } + public SRColumn build() { + return new SRColumn( + columnName, + ordinalPosition, + dataType, + isNullable, + defaultValue, + columnSize, + decimalDigits, + columnComment); + } + } +} diff --git a/src/main/java/com/starrocks/connector/spark/catalog/SRTable.java b/src/main/java/com/starrocks/connector/spark/catalog/SRTable.java new file mode 100644 index 00000000..34a257b5 --- /dev/null +++ b/src/main/java/com/starrocks/connector/spark/catalog/SRTable.java @@ -0,0 +1,202 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +package com.starrocks.connector.spark.catalog; +import com.google.common.base.Preconditions; + + +import javax.annotation.Nullable; +import java.util.*; + +//to describe a column of SR,info from information_schema.tables + +public class SRTable { + + + public enum TableType { + UNKNOWN, + DUPLICATE_KEYS, + AGGREGATES, + UNIQUE_KEYS, + PRIMARY_KEYS + } + //database's name + private final String databaseName; + //table's name + private final String tableName; + //SR table's type only primary can use stageMode + private final TableType tableType; + //columns describe + private final List columns; + //table's primaryKey for pks table, can more than one + @Nullable + private final List tableKeys; + //table properties + private final Map properties; + @Nullable + private final List distributionKeys; + @Nullable + private final Integer numBuckets; + //comment about table + @Nullable private final String comment; + private SRTable( + String databaseName, + String tableName, + TableType tableType, + List columns, + @Nullable List tableKeys, + @Nullable List distributionKeys, + @Nullable Integer numBuckets, + @Nullable String comment, + Map properties) { + Preconditions.checkNotNull(databaseName); + Preconditions.checkNotNull(tableName); + Preconditions.checkNotNull(tableType); + Preconditions.checkArgument(columns != null && !columns.isEmpty()); + this.databaseName = databaseName; + this.tableName = tableName; + this.tableType = tableType; + this.columns = columns; + this.tableKeys = tableKeys; + this.distributionKeys = distributionKeys; + this.numBuckets = numBuckets; + this.comment = comment; + this.properties = Preconditions.checkNotNull(properties); + } + public String getDatabaseName() { + return databaseName; + } + public String getTableName() { + return tableName; + } + public TableType getTableType() { + return tableType; + } + public List getColumns() { + return columns; + } + + public Optional> getTableKeys() { + return Optional.ofNullable(tableKeys); + } + public Optional> getDistributionKeys() { + return Optional.ofNullable(distributionKeys); + } + public Optional getNumBuckets() { + return Optional.ofNullable(numBuckets); + } + public Optional getComment() { + return Optional.ofNullable(comment); + } + public Map getProperties() { + return properties; + } + + @Override + public String toString() { + return "StarRocksTable{" + + "databaseName='" + databaseName + '\'' + + ", tableName='" + tableName + '\'' + + ", tableType=" + tableType + + ", columns=" + columns + + ", tableKeys=" + tableKeys + + ", comment='" + comment + '\'' + + ", properties=" + properties + + '}'; + } + public static class Builder { + private String databaseName; + private String tableName; + private TableType tableType; + private List columns = new ArrayList<>(); + private List tableKeys; + private List distributionKeys; + private Integer numBuckets; + private String comment; + private Map properties = new HashMap<>(); + public Builder setDatabaseName(String databaseName) { + this.databaseName = databaseName; + return this; + } + public Builder setTableName(String tableName) { + this.tableName = tableName; + return this; + } + public Builder setTableType(String tableType) { + switch (tableType) { + case "DUPLICATE_KEYS" : + this.tableType = TableType.DUPLICATE_KEYS; + break; + case "AGGREGATES": + this.tableType = TableType.AGGREGATES; + break; + case "UNIQUE_KEYS": + this.tableType = TableType.UNIQUE_KEYS; + break; + case "PRIMARY_KEYS": + this.tableType = TableType.PRIMARY_KEYS; + break; + default: + this.tableType = TableType.UNKNOWN; + } + return this; + } + public Builder setTableType(TableType tableType) + { + this.tableType = tableType; + return this; + } + public Builder setColumns(List columns) { + this.columns = columns; + return this; + } + public Builder setTableKeys(List tableKeys) { + this.tableKeys = tableKeys; + return this; + } + public Builder setDistributionKeys(List distributionKeys) { + this.distributionKeys = distributionKeys; + return this; + } + public Builder setNumBuckets(Integer numBuckets) { + this.numBuckets = numBuckets; + return this; + } + public Builder setComment(String comment) { + this.comment = comment; + return this; + } + public Builder setTableProperties(Map properties) { + this.properties = properties; + return this; + } + public SRTable build() { + return new SRTable( + databaseName, + tableName, + tableType, + columns, + tableKeys, + distributionKeys, + numBuckets, + comment, + properties); + } + } + +} diff --git a/src/main/java/com/starrocks/connector/spark/cfg/ConfigurationOptions.java b/src/main/java/com/starrocks/connector/spark/cfg/ConfigurationOptions.java index 40c0d7c3..242b3cf4 100644 --- a/src/main/java/com/starrocks/connector/spark/cfg/ConfigurationOptions.java +++ b/src/main/java/com/starrocks/connector/spark/cfg/ConfigurationOptions.java @@ -79,7 +79,7 @@ public interface ConfigurationOptions { String STARROCKS_DESERIALIZE_QUEUE_SIZE = "starrocks.deserialize.queue.size"; int STARROCKS_DESERIALIZE_QUEUE_SIZE_DEFAULT = 64; - + String STARROCKS_TEMP_DBNAME="temp_db"; static Map makeWriteCompatibleWithRead(Map options) { // user and password compatible Map configMap = new HashMap(options); diff --git a/src/main/java/com/starrocks/connector/spark/exception/CatalogException.java b/src/main/java/com/starrocks/connector/spark/exception/CatalogException.java new file mode 100644 index 00000000..1ad45fe2 --- /dev/null +++ b/src/main/java/com/starrocks/connector/spark/exception/CatalogException.java @@ -0,0 +1,42 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. +package com.starrocks.connector.spark.exception; + +public class CatalogException extends RuntimeException { + + public CatalogException(String message) { + super(message); + } + + /** + * @param cause the cause. + */ + public CatalogException(Throwable cause) { + super(cause); + } + + /** + * @param message the detail message. + * @param cause the cause. + */ + public CatalogException(String message, Throwable cause) { + super(message, cause); + } + +} diff --git a/src/main/java/com/starrocks/connector/spark/sql/StarRocksTable.java b/src/main/java/com/starrocks/connector/spark/sql/StarRocksTable.java index d1cc815c..31a550f3 100644 --- a/src/main/java/com/starrocks/connector/spark/sql/StarRocksTable.java +++ b/src/main/java/com/starrocks/connector/spark/sql/StarRocksTable.java @@ -75,7 +75,7 @@ public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { WriteStarRocksConfig writeConfig = new WriteStarRocksConfig(properties.getPropertyMap(), schema, starRocksSchema); checkWriteParameter(writeConfig); - return new StarRocksWriteBuilder(info, writeConfig); + return new StarRocksWriteBuilder(info, writeConfig, starRocksSchema); } @Override diff --git a/src/main/java/com/starrocks/connector/spark/sql/conf/StarRocksConfigBase.java b/src/main/java/com/starrocks/connector/spark/sql/conf/StarRocksConfigBase.java index db74c565..7e03eafa 100644 --- a/src/main/java/com/starrocks/connector/spark/sql/conf/StarRocksConfigBase.java +++ b/src/main/java/com/starrocks/connector/spark/sql/conf/StarRocksConfigBase.java @@ -61,14 +61,14 @@ public abstract class StarRocksConfigBase implements StarRocksConfig { // data type mapping instead of all columns public static final String KEY_COLUMN_TYPES = PREFIX + "column.types"; - protected final Map originOptions; + protected Map originOptions; private String[] feHttpUrls; private String feJdbcUrl; private String username; private String password; - private String database; - private String table; + protected String database; + protected String table; @Nullable private String[] columns; @Nullable @@ -77,7 +77,24 @@ public abstract class StarRocksConfigBase implements StarRocksConfig { private int httpRequestConnectTimeoutMs; private int httpRequestSocketTimeoutMs; private ZoneId timeZone; - + // Just for copy() + protected StarRocksConfigBase() { + } + protected void copy(StarRocksConfigBase other) { + other.originOptions = new HashMap<>(originOptions); + other.feHttpUrls = feHttpUrls; + other.feJdbcUrl = feJdbcUrl; + other.username = username; + other.password = password; + other.database = database; + other.table = table; + other.columns = columns; + other.columnTypes = columnTypes; + other.httpRequestRetries = httpRequestRetries; + other.httpRequestConnectTimeoutMs = httpRequestConnectTimeoutMs; + other.httpRequestSocketTimeoutMs = httpRequestSocketTimeoutMs; + other.timeZone = timeZone; + } public StarRocksConfigBase(Map options) { this.originOptions = new HashMap<>(options); load(); diff --git a/src/main/java/com/starrocks/connector/spark/sql/conf/WriteStarRocksConfig.java b/src/main/java/com/starrocks/connector/spark/sql/conf/WriteStarRocksConfig.java index 0f30f0f9..6b5007e3 100644 --- a/src/main/java/com/starrocks/connector/spark/sql/conf/WriteStarRocksConfig.java +++ b/src/main/java/com/starrocks/connector/spark/sql/conf/WriteStarRocksConfig.java @@ -36,12 +36,7 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.util.Utils; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.stream.Collectors; public class WriteStarRocksConfig extends StarRocksConfigBase { @@ -65,6 +60,7 @@ public class WriteStarRocksConfig extends StarRocksConfigBase { private static final String KEY_BUFFER_SIZE = WRITE_PREFIX + "buffer.size"; // The number of rows buffered before sending to StarRocks. private static final String KEY_BUFFER_ROWS = WRITE_PREFIX + "buffer.rows"; + // Flush interval of the row batch in millisecond private static final String KEY_FLUSH_INTERVAL = WRITE_PREFIX + "flush.interval.ms"; private static final String KEY_MAX_RETIES = WRITE_PREFIX + "max.retries"; @@ -76,6 +72,17 @@ public class WriteStarRocksConfig extends StarRocksConfigBase { private static final String KEY_NUM_PARTITIONS = WRITE_PREFIX + "num.partitions"; private static final String KEY_PARTITION_COLUMNS = WRITE_PREFIX + "partition.columns"; + // start use stage mode or not, if not stage mode will never use + //decide stageMode when to use always,auto,never + private static final String STAGE_MODE_USE = WRITE_PREFIX + "stage.use"; + // stage config header + private static final String STAGE_CONFIG = WRITE_PREFIX + "stage."; + + public enum StageUse { + ALWAYS, + AUTO, + NEVER + } private String labelPrefix = "spark"; private int socketTimeoutMs = -1; @@ -95,7 +102,13 @@ public class WriteStarRocksConfig extends StarRocksConfigBase { private String rowDelimiter = "\n"; private String columnSeparator = "\t"; private boolean supportTransactionStreamLoad = true; + private StageUse stageUse = StageUse.NEVER; + private Map stageConfig; + + //starrocks.write.stage.columns.update.ratio + // a rough number describe the ratio of row ro be updated 1-100 + //if any column in stageUpdateColumn to be updated use stage mode default "" // According to Spark RequiresDistributionAndOrdering#requiredNumPartitions(), // any value less than 1 mean no requirement private int numPartitions = 0; @@ -104,11 +117,17 @@ public class WriteStarRocksConfig extends StarRocksConfigBase { private String streamLoadColumnProperty; private String[] streamLoadColumnNames; - private final Set starRocksJsonColumnNames; + private Set starRocksJsonColumnNames; + private StructType sparkSchema; + + public StructType getSparkSchema() { + return sparkSchema; + } public WriteStarRocksConfig(Map originOptions, StructType sparkSchema, StarRocksSchema starRocksSchema) { super(originOptions); load(sparkSchema); + this.sparkSchema = sparkSchema; genStreamLoadColumns(sparkSchema, starRocksSchema); this.starRocksJsonColumnNames = new HashSet<>(); for (StarRocksField column : starRocksSchema.getColumns()) { @@ -130,6 +149,16 @@ private void load(StructType sparkSchema) { flushInterval = getInt(KEY_FLUSH_INTERVAL, 300000); maxRetries = getInt(KEY_MAX_RETIES, 3); retryIntervalInMs = getInt(KEY_RETRY_INTERVAL_MS, 10000); + stageConfig= originOptions.entrySet().stream() + .filter(entry -> entry.getKey().startsWith(STAGE_CONFIG)) + .collect( + Collectors.toMap( + entry -> entry.getKey().replaceFirst(STAGE_CONFIG, ""), + Map.Entry::getValue + ) + ); + + stageUse =StageUse.valueOf(get(STAGE_MODE_USE, "never").toUpperCase()); properties = originOptions.entrySet().stream() .filter(entry -> entry.getKey().startsWith(PROPS_PREFIX)) @@ -168,6 +197,39 @@ private void load(StructType sparkSchema) { supportTransactionStreamLoad = StreamLoadUtils.isStarRocksSupportTransactionLoad( Arrays.asList(getFeHttpUrls()), getHttpRequestConnectTimeoutMs(), getUsername(), getPassword()); } + private WriteStarRocksConfig() { + super(); + } + public WriteStarRocksConfig copy(String dataBase, String table, List excludeStreamLoadProperties) { + WriteStarRocksConfig copyConfig = new WriteStarRocksConfig(); + + super.copy(copyConfig); + copyConfig.database = dataBase; + copyConfig.table = table; + copyConfig.labelPrefix = labelPrefix; + copyConfig.waitForContinueTimeoutMs = waitForContinueTimeoutMs; + copyConfig.ioThreadCount = ioThreadCount; + copyConfig.chunkLimit = chunkLimit; + copyConfig.scanFrequencyInMs = scanFrequencyInMs; + copyConfig.enableTransactionStreamLoad = enableTransactionStreamLoad; + copyConfig.bufferSize = bufferSize; + copyConfig.bufferRows = bufferRows; + copyConfig.flushInterval = flushInterval; + copyConfig.maxRetries = maxRetries; + copyConfig.retryIntervalInMs = retryIntervalInMs; + copyConfig.properties = new HashMap<>(properties); + excludeStreamLoadProperties.forEach(copyConfig.properties::remove); + copyConfig.format = format; + copyConfig.rowDelimiter = rowDelimiter; + copyConfig.columnSeparator = columnSeparator; + copyConfig.supportTransactionStreamLoad = supportTransactionStreamLoad; + copyConfig.numPartitions = numPartitions; + copyConfig.partitionColumns = partitionColumns; + copyConfig.streamLoadColumnProperty = streamLoadColumnProperty; + copyConfig.streamLoadColumnNames = streamLoadColumnNames; + + return copyConfig; + } private void genStreamLoadColumns(StructType sparkSchema, StarRocksSchema starRocksSchema) { streamLoadColumnNames = new String[sparkSchema.length()]; @@ -223,6 +285,15 @@ private String getBitmapFunction(StructField field) { } } + + public StageUse getStageUse() { + return stageUse; + } + + + public Map getStageConfig() { + return stageConfig; + } public String getFormat() { return format; } @@ -247,11 +318,18 @@ public Set getStarRocksJsonColumnNames() { return starRocksJsonColumnNames; } + + + + public boolean isPartialUpdate() { String val = properties.get("partial_update"); return val != null && val.equalsIgnoreCase("true"); } - + public boolean isPartialUpdateColumnMode() { + String val = properties.get("partial_update_mode"); + return val != null && val.equalsIgnoreCase("column"); + } public StreamLoadProperties toStreamLoadProperties() { StreamLoadDataFormat dataFormat = "json".equalsIgnoreCase(format) ? StreamLoadDataFormat.JSON : new StreamLoadDataFormat.CSVFormat(rowDelimiter); diff --git a/src/main/java/com/starrocks/connector/spark/sql/connect/StarRocksConnector.java b/src/main/java/com/starrocks/connector/spark/sql/connect/StarRocksConnector.java index 67e76ec6..aec5d610 100644 --- a/src/main/java/com/starrocks/connector/spark/sql/connect/StarRocksConnector.java +++ b/src/main/java/com/starrocks/connector/spark/sql/connect/StarRocksConnector.java @@ -19,6 +19,9 @@ package com.starrocks.connector.spark.sql.connect; +import com.starrocks.connector.spark.catalog.SRColumn; +import com.starrocks.connector.spark.catalog.SRTable; +import com.starrocks.connector.spark.exception.CatalogException; import com.starrocks.connector.spark.exception.StarRocksException; import com.starrocks.connector.spark.sql.conf.StarRocksConfig; import com.starrocks.connector.spark.sql.schema.StarRocksField; @@ -26,32 +29,33 @@ import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.sql.Connection; import java.sql.DriverManager; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.ResultSetMetaData; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; +import java.util.*; public class StarRocksConnector { - private static Logger logger = LoggerFactory.getLogger(StarRocksConnector.class); + private static final Logger logger = LoggerFactory.getLogger(StarRocksConnector.class); private static final String TABLE_SCHEMA_QUERY = "SELECT `COLUMN_NAME`, `ORDINAL_POSITION`, `COLUMN_KEY`, `DATA_TYPE`, `COLUMN_SIZE`, `DECIMAL_DIGITS` " + "FROM `information_schema`.`COLUMNS` WHERE `TABLE_SCHEMA`=? AND `TABLE_NAME`=?;"; private static final String ALL_DBS_QUERY = "show databases;"; + private static final String LOAD_DB_QUERY = "select SCHEMA_NAME from information_schema.schemata where SCHEMA_NAME in (?) AND CATALOG_NAME = 'def';"; + private static final String ALL_TABLES_QUERY = "select TABLE_SCHEMA, TABLE_NAME from information_schema.tables " + "where TABLE_TYPE = 'BASE TABLE' AND TABLE_SCHEMA in (?) ;"; - + private static final String DB_EXISTS = + "select SCHEMA_NAME from information_schema.schemata where SCHEMA_NAME =? AND CATALOG_NAME = 'def';"; + private static final String Table_COLUMNS_QUERY= + "select COLUMN_NAME,ORDINAL_POSITION,DATA_TYPE,COLUMN_SIZE,DECIMAL_DIGITS,COLUMN_DEFAULT,IS_NULLABLE,COLUMN_COMMENT from INFORMATION_SCHEMA.columns where TABLE_SCHEMA=? and TABLE_NAME=?"; + private static final String TABLES_QUERY = + "select TABLE_SCHEMA,TABLE_NAME,TABLE_COMMENT from INFORMATION_SCHEMA.tables where TABLE_SCHEMA=? AND TABLE_NAME = ?"; + private static final String TABLE_CONFIG_QUERY="select TABLE_MODEL,PRIMARY_KEY,DISTRIBUTE_KEY,DISTRIBUTE_BUCKET,PROPERTIES from INFORMATION_SCHEMA.tables_config where TABLE_SCHEMA=? AND TABLE_NAME = ?"; // Driver name for mysql connector 5.1 which is deprecated in 8.0 private static final String MYSQL_51_DRIVER_NAME = "com.mysql.jdbc.Driver"; // Driver name for mysql connector 8.0 @@ -110,12 +114,11 @@ public static Map loadDatabase(StarRocksConfig config, List getTables(StarRocksConfig config, List dbNames) { - List parameters = Arrays.asList(String.join(",", dbNames)); + List parameters = Arrays.asList(java.lang.String.join(",", dbNames)); List> tables = extractColumnValuesBySql(config, ALL_TABLES_QUERY, parameters); Map table2Db = new HashMap<>(); - for (Map db : tables) { String dbName = Optional.ofNullable(db.get("TABLE_SCHEMA")) .orElseThrow(() -> new StarRocksException("get table header error")); @@ -128,14 +131,111 @@ public static Map getTables(StarRocksConfig config, List return table2Db; } - private static Connection createJdbcConnection(String jdbcUrl, String username, String password) throws Exception { +public static boolean dbExists(StarRocksConfig config, String dbName) { + + List> res = extractColumnValuesBySql(config, DB_EXISTS, Arrays.asList(dbName)); + return !res.isEmpty(); +} + + + public static SRTable getSRTable(StarRocksConfig config, String dbName, String tableName) { + + Map tableConfig = getTableConfig(config, dbName, tableName); + List srColumns = getSRColumns(config, dbName, tableName); + List> maps = extractColumnValuesBySql(config, TABLES_QUERY, Arrays.asList(dbName, tableName)); + if (maps.isEmpty()) { + throw new StarRocksException("table does not exist: " + dbName + "." + tableName); + } + + Map tables = maps.get(0); + String db_name = tables.get("TABLE_SCHEMA"); + String table_name = tables.get("TABLE_NAME"); + String table_comment = tables.get("TABLE_COMMENT"); + String table_type = tableConfig.get("TABLE_MODEL"); + String[] arr = tableConfig.get("PRIMARY_KEY").replace("`","").split(","); + String[] trimmedArr = Arrays.stream(arr) + .map(String::trim) + .toArray(String[]::new); + List primary_key = Arrays.asList(trimmedArr); + int num_buckets = Integer.parseInt(tableConfig.get("DISTRIBUTE_BUCKET")); + String[] distributeKeys = tableConfig.get("DISTRIBUTE_KEY").replace("`","").split(","); + String[] trimmedDistributeKeys = Arrays.stream(distributeKeys) + .map(String::trim) + .toArray(String[]::new); + List distribution_keys = Arrays.asList(trimmedDistributeKeys); + + String proper = tableConfig.get("PROPERTIES"); + proper=proper.trim().replaceAll("[{}]",""); + String[] entries = proper.split(","); + Map table_properties = new HashMap<>(); + for (String entry : entries) { + String[] keyValue = entry.split(":"); + table_properties.put(keyValue[0].replace("\"", "").trim(), keyValue[1].replace("\"", "").trim()); + } + SRTable res=new SRTable.Builder() + .setTableName(table_name) + .setComment(table_comment) + .setColumns(srColumns) + .setTableKeys(primary_key) + .setDatabaseName(db_name) + .setNumBuckets(num_buckets) + .setTableType(table_type) + .setTableProperties(table_properties) + .setDistributionKeys(distribution_keys) + .build(); + return res; + } + + public static Map getTableConfig(StarRocksConfig config, String dbName, String tableName) { + List> res = extractColumnValuesBySql(config, TABLE_CONFIG_QUERY, Arrays.asList(dbName, tableName)); + if (!res.isEmpty()) { + return res.get(0); + } + else + { + throw new StarRocksException("table does not exist: " + dbName + "." + tableName); + } + } + + public static List getSRColumns(StarRocksConfig config, String dbName, String tableName) { + ArrayList srColumns = new ArrayList<>(); + List> res = extractColumnValuesBySql(config, Table_COLUMNS_QUERY, Arrays.asList(dbName, tableName)); + if (!res.isEmpty()) { + + for (Map columnMap : res) { + String columnSize = columnMap.get("COLUMN_SIZE"); + Integer size=null; + if (columnSize != null) { + size = Integer.parseInt(columnSize); + } + SRColumn column = new SRColumn.Builder() + .setColumnName(columnMap.get("COLUMN_NAME")) + .setOrdinalPosition(Integer.parseInt(columnMap.get("ORDINAL_POSITION"))) + .setDataType(columnMap.get("DATA_TYPE"),size) + .setNullable(columnMap.get("IS_NULLABLE")) + .setDefaultValue(columnMap.get("COLUMN_DEFAULT")) + .setColumnSize(size) + .setDecimalDigits((columnMap.get("DECIMAL_DIGITS"))) + .setColumnComment(columnMap.get("COLUMN_COMMENT")) + .build(); + srColumns.add(column); + } + } + else + { + throw new CatalogException("table does not exist: " + dbName + "." + tableName); + } + return srColumns; + } + + public static Connection createJdbcConnection(String jdbcUrl, String username, String password) throws Exception { try { Class.forName(MYSQL_80_DRIVER_NAME); } catch (ClassNotFoundException e) { try { Class.forName(MYSQL_51_DRIVER_NAME); } catch (ClassNotFoundException ie) { - String msg = String.format("Can't find mysql jdbc driver, please download it and " + String msg = java.lang.String.format("Can't find mysql jdbc driver, please download it and " + "put it in your classpath manually. Note that the connector does not include " + "the mysql driver since version 1.1.1 because of the limitation of GPL license " + "used by the driver. You can download it from MySQL site %s, or Maven Central %s", MYSQL_SITE_URL, @@ -172,7 +272,7 @@ private static List> extractColumnValuesBySql(StarRocksConfi } if (columnValues.isEmpty()) { - String errMsg = String.format("Can't get schema of table [%s.%s] from StarRocks. The possible reasons: " + String errMsg = java.lang.String.format("Can't get schema of table [%s.%s] from StarRocks. The possible reasons: " + "1) The table does not exist. 2) The user does not have [SELECT] privilege on the " + "table, and can't read the schema. Please make sure that the table exists in StarRocks, " + "and grant [SELECT] privilege to the user. If you are loading data to the table, also need " diff --git a/src/main/java/com/starrocks/connector/spark/sql/schema/StarRocksSchema.java b/src/main/java/com/starrocks/connector/spark/sql/schema/StarRocksSchema.java index 6909971c..69609cf1 100644 --- a/src/main/java/com/starrocks/connector/spark/sql/schema/StarRocksSchema.java +++ b/src/main/java/com/starrocks/connector/spark/sql/schema/StarRocksSchema.java @@ -43,7 +43,9 @@ public StarRocksSchema(List columns) { public StarRocksSchema(List columns, List keyColumns) { this(columns, keyColumns, null); } - + public List getPrimaryKeys() { + return keyColumns; + } public StarRocksSchema(List columns, List keyColumns, Long tableId) { diff --git a/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksDataWriter.java b/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksDataWriter.java index 950a84e9..82bc91fb 100644 --- a/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksDataWriter.java +++ b/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksDataWriter.java @@ -33,7 +33,6 @@ import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.Serializable; @@ -46,7 +45,6 @@ public class StarRocksDataWriter implements DataWriter, Serializabl private final long epochId; private final RowStringConverter converter; private final StreamLoadManager manager; - public StarRocksDataWriter(WriteStarRocksConfig config, StructType schema, int partitionId, @@ -75,6 +73,7 @@ public void open() { @Override public void write(InternalRow internalRow) throws IOException { + String data = converter.fromRow(internalRow); manager.write(null, config.getDatabase(), config.getTable(), data); @@ -82,11 +81,13 @@ public void write(InternalRow internalRow) throws IOException { partitionId, taskId, epochId, internalRow); log.debug("partitionId: {}, taskId: {}, epochId: {}, receive converted row: {}", partitionId, taskId, epochId, data); + } @Override public WriterCommitMessage commit() throws IOException { log.info("partitionId: {}, taskId: {}, epochId: {} commit", partitionId, taskId, epochId); + try { manager.flush(); return new StarRocksWriterCommitMessage(partitionId, taskId, epochId, null); @@ -121,4 +122,8 @@ public void close() throws IOException { log.info("partitionId: {}, taskId: {}, epochId: {} close", partitionId, taskId, epochId); manager.close(); } + + + + } diff --git a/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWrite.java b/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWrite.java index c03204f8..f08736c0 100644 --- a/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWrite.java +++ b/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWrite.java @@ -19,7 +19,14 @@ package com.starrocks.connector.spark.sql.write; +import com.starrocks.connector.spark.catalog.SRCatalog; +import com.starrocks.connector.spark.catalog.SRColumn; +import com.starrocks.connector.spark.catalog.SRTable; +import com.starrocks.connector.spark.exception.CatalogException; import com.starrocks.connector.spark.sql.conf.WriteStarRocksConfig; +import com.starrocks.connector.spark.sql.connect.StarRocksConnector; +import com.starrocks.connector.spark.sql.schema.StarRocksField; +import com.starrocks.connector.spark.sql.schema.StarRocksSchema; import org.apache.spark.sql.connector.write.BatchWrite; import org.apache.spark.sql.connector.write.DataWriterFactory; import org.apache.spark.sql.connector.write.LogicalWriteInfo; @@ -27,26 +34,54 @@ import org.apache.spark.sql.connector.write.WriterCommitMessage; import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory; import org.apache.spark.sql.connector.write.streaming.StreamingWrite; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import java.util.*; +import java.util.stream.Collectors; + +import static com.starrocks.connector.spark.cfg.ConfigurationOptions.STARROCKS_TEMP_DBNAME; + public class StarRocksWrite implements BatchWrite, StreamingWrite { private static final Logger log = LoggerFactory.getLogger(StarRocksWrite.class); private final LogicalWriteInfo logicalInfo; private final WriteStarRocksConfig config; + private final StarRocksSchema starRocksSchema; + private final boolean isBatch; + private boolean useStage; + @Nullable + private transient String tempTableName; + @Nullable + private transient SRTable srTable; + @Nullable + private transient Map TableConfig; + //todo - public StarRocksWrite(LogicalWriteInfo logicalInfo, WriteStarRocksConfig config) { + public StarRocksWrite(LogicalWriteInfo logicalInfo, WriteStarRocksConfig config, StarRocksSchema starRocksSchema,boolean isBatch) { this.logicalInfo = logicalInfo; this.config = config; + this.isBatch = isBatch; + this.starRocksSchema = starRocksSchema; + } @Override public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { - return new StarRocksWriterFactory(logicalInfo.schema(), config); + + WriteStarRocksConfig writeConfig = doPrepare(); + + return new StarRocksWriterFactory(logicalInfo.schema(), writeConfig); + } + + @Override public boolean useCommitCoordinator() { return true; @@ -55,11 +90,13 @@ public boolean useCommitCoordinator() { @Override public void commit(WriterCommitMessage[] messages) { log.info("batch query `{}` commit", logicalInfo.queryId()); + doCommit(messages); } @Override public void abort(WriterCommitMessage[] messages) { log.info("batch query `{}` abort", logicalInfo.queryId()); + doAbort(messages); } @Override @@ -72,8 +109,239 @@ public void commit(long epochId, WriterCommitMessage[] messages) { log.info("streaming query `{}` commit", logicalInfo.queryId()); } + @Override public void abort(long epochId, WriterCommitMessage[] messages) { log.info("streaming query `{}` abort", logicalInfo.queryId()); } + + private void doAbort(WriterCommitMessage[] messages) { +if(!useStage) +{ + return; +} +dropTempTable(); + +} + public boolean useStageMode() { + String database = config.getDatabase(); + String table = config.getTable(); + if (this.TableConfig==null) + { + this.TableConfig = StarRocksConnector.getTableConfig(config, database, table); + } + + String tableModel = TableConfig.get("TABLE_MODEL"); + + + if(isBatch&&config.isPartialUpdate()&&config.isPartialUpdateColumnMode()&&tableModel.equals("PRIMARY_KEYS")) + { + + this.srTable = StarRocksConnector.getSRTable(config, config.getDatabase(), config.getTable()); + //always + if (config.getStageUse()== WriteStarRocksConfig.StageUse.ALWAYS) + { + return true; + } + //todo auto when to use stageMode need a better strategy + //auto + else if (config.getStageUse()== WriteStarRocksConfig.StageUse.AUTO) + { + Map stageConfig = config.getStageConfig(); + int ratio = Integer.parseInt(stageConfig.getOrDefault("columns.update.ratio","60")); + + StructType schema = config.getSparkSchema(); + Set fieldNames = new HashSet<>(); + for (StructField field : schema.fields()) { + fieldNames.add(field.name()); + } + + int srColumnSize = srTable.getColumns().size(); + + return ratio >= 60 && fieldNames.size() * 4 >= srColumnSize; + + } + //never + else + { + return false; + } + } + + else + { + return false; + } + } + + private WriteStarRocksConfig doPrepare() { + this.useStage=useStageMode(); + if (!useStage) + { + return config; + } + SRTable tempTable = toSRTable(starRocksSchema); + SRCatalog catalog = getSRCatalog(); + if (!StarRocksConnector.dbExists(config, STARROCKS_TEMP_DBNAME)) + { + catalog.createDatabase(STARROCKS_TEMP_DBNAME,true); + } + try { + catalog.createTable(tempTable, false); + } catch (Exception e) { + log.error("Failed to create temp table {}.{},already exists", STARROCKS_TEMP_DBNAME, tempTable.getTableName(), e); + throw e; + } + this.tempTableName = tempTable.getTableName(); + return config.copy(tempTable.getDatabaseName(),tempTable.getTableName(), Arrays.asList("partial_update", "partial_update_mode")); + } + + private void doCommit(WriterCommitMessage[] messages) { + if (!useStage) { + return; + } + + String srcTableId = String.format("`%s`.`%s`", STARROCKS_TEMP_DBNAME, tempTableName); + String targetTableId = String.format("`%s`.`%s`", config.getDatabase(), config.getTable()); + List primaryKeys = starRocksSchema.getPrimaryKeys().stream() + .map(StarRocksField::getName) + .collect(Collectors.toList()); + String joinedKeys = primaryKeys.stream() + .map(key -> String.format("%s.`%s` = t2.`%s`", targetTableId, key, key)) + .collect(Collectors.joining(" AND ")); + + List columns = srTable.getColumns(); + + SRTable srTable1 = StarRocksConnector.getSRTable(config, STARROCKS_TEMP_DBNAME, tempTableName); + List tempColumns = srTable1.getColumns().stream() + .map(SRColumn::getColumnName) + .collect(Collectors.toList()); + + String joinedColumns = columns.stream() + .map(SRColumn::getColumnName) + .filter(tempColumns::contains) + .filter(col -> !primaryKeys.contains(col)) + .map(col -> String.format("`%s` = `t2`.`%s`", col, col)) + .collect(Collectors.joining(", ")); + + Map stageConfig = config.getStageConfig(); + + Map sessionMap = stageConfig.entrySet().stream().filter(entry -> entry.getKey().startsWith("session.")).collect( + Collectors.toMap( + entry -> entry.getKey().substring(8).replace('.', '_'), + Map.Entry::getValue + ) + ); + StringBuilder updateHeader = new StringBuilder("UPDATE /* + SET_VAR ("); + for (Map.Entry sessionSet : sessionMap.entrySet()) { + if(!updateHeader.toString().endsWith("(")) + { + updateHeader.append(","); + } + updateHeader.append(sessionSet.getKey()).append("=").append(sessionSet.getValue()); + } + updateHeader.append(") */"); + + String updateSql = String.format(updateHeader + " %s SET %s FROM %s AS `t2` WHERE %s;", + targetTableId, joinedColumns, srcTableId, joinedKeys); + + String setVar = "SET partial_update_mode = 'column';"; + + log.info("Update sql: {}", updateSql); + + try { + getSRCatalog().executeDDLBySql(setVar+updateSql); + + } + catch (Exception e) { + log.error("Failed to execute update, temp table: {}, target table: {}", srcTableId, targetTableId, e); + throw new CatalogException("Failed to execute update, temp table: " + srcTableId, e); + } + dropTempTable(); + log.info("Success to execute update, temp table: {}, target table: {}", srcTableId, targetTableId); + } + + +private void dropTempTable() { + if (tempTableName!=null) + { + try { + getSRCatalog().dropTable(STARROCKS_TEMP_DBNAME, tempTableName); + } + catch (Exception e) { + log.error("Failed to drop temp table {}.{}", STARROCKS_TEMP_DBNAME, tempTableName, e); + } + } +} + +private SRTable toSRTable(StarRocksSchema starRocksSchema) +{ + String uuid = UUID.randomUUID().toString().replace("-", "_"); + return new SRTable.Builder() + .setDatabaseName(STARROCKS_TEMP_DBNAME) + .setTableName(srTable.getDatabaseName() + "_" + srTable.getTableName() + "_temp_"+uuid) + .setTableType(SRTable.TableType.DUPLICATE_KEYS) + .setTableKeys( + starRocksSchema.getPrimaryKeys().stream() + .map(StarRocksField::getName) + .collect(Collectors.toList()) + ) + .setColumns( + Arrays.stream(config.getSparkSchema().fields()) + .map(f -> starRocksSchema.getField(f.name())) + .map(this::toStarRocksColumn) + .collect(Collectors.toList()) + ) + .setComment( + String.format("Spark partial update with column mode, table: %s.%s, query: %s", + config.getDatabase(), config.getTable(), logicalInfo.queryId()) + ) + .setTableProperties(srTable.getProperties()) + .setDistributionKeys(srTable.getDistributionKeys().get()) + .setNumBuckets(srTable.getNumBuckets().get()) + .build(); +} + private SRColumn toStarRocksColumn(StarRocksField field) { + return new SRColumn.Builder() + .setColumnName(field.getName()) + .setDataType(toStarRocksType(field)) + .setColumnSize(field.getSize() == null ? null : field.getSize()) + .setDecimalDigits(field.getScale() == null ? null : String.valueOf(field.getScale())) + .setNullable("YES") + .build(); + } + private String toStarRocksType(StarRocksField field) { + String type = field.getType().toLowerCase(Locale.ROOT); + switch (type) { + case "tinyint": + // mysql does not have boolean type, and starrocks `information_schema`.`COLUMNS` will return + // a "tinyint" data type for both StarRocks BOOLEAN and TINYINT type, We distinguish them by + // column size, and the size of BOOLEAN is null + return field.getSize() == null ? "BOOLEAN" : "TINYINT"; + case "bigint unsigned": + return "LARGEINT"; + case "smallint": + case "int": + case "bigint": + case "float": + case "double": + case "decimal": + case "char": + case "varchar": + case "json": + case "date": + case "datetime": + return type.toUpperCase(); + default: + throw new UnsupportedOperationException(String.format( + "Unsupported starrocks type, column name: %s, data type: %s", field.getName(), field.getType())); + } + } + +private SRCatalog getSRCatalog() +{ + return new SRCatalog(config.getFeJdbcUrl(), config.getUsername(), config.getPassword()); +} + + } diff --git a/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWriteBuilder.java b/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWriteBuilder.java index 4ae4418e..79361115 100644 --- a/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWriteBuilder.java +++ b/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWriteBuilder.java @@ -20,6 +20,7 @@ package com.starrocks.connector.spark.sql.write; import com.starrocks.connector.spark.sql.conf.WriteStarRocksConfig; +import com.starrocks.connector.spark.sql.schema.StarRocksSchema; import org.apache.spark.sql.connector.distributions.Distribution; import org.apache.spark.sql.connector.distributions.Distributions; import org.apache.spark.sql.connector.expressions.Expression; @@ -35,25 +36,27 @@ public class StarRocksWriteBuilder implements WriteBuilder { private final LogicalWriteInfo info; private final WriteStarRocksConfig config; - - public StarRocksWriteBuilder(LogicalWriteInfo info, WriteStarRocksConfig config) { + private final StarRocksSchema starRocksSchema; + public StarRocksWriteBuilder(LogicalWriteInfo info, WriteStarRocksConfig config, StarRocksSchema starRocksSchema) { this.info = info; this.config = config; + this.starRocksSchema = starRocksSchema; } @Override public Write build() { - return new StarRocksWriteImpl(info, config); + return new StarRocksWriteImpl(info, config, starRocksSchema); } private static class StarRocksWriteImpl implements Write, RequiresDistributionAndOrdering { private final LogicalWriteInfo info; private final WriteStarRocksConfig config; - - public StarRocksWriteImpl(LogicalWriteInfo info, WriteStarRocksConfig config) { + private final StarRocksSchema starRocksSchema; + public StarRocksWriteImpl(LogicalWriteInfo info, WriteStarRocksConfig config, StarRocksSchema starRocksSchema) { this.info = info; this.config = config; + this.starRocksSchema = starRocksSchema; } @Override @@ -63,12 +66,12 @@ public String description() { @Override public BatchWrite toBatch() { - return new StarRocksWrite(info, config); + return new StarRocksWrite(info, config, starRocksSchema, true); } @Override public StreamingWrite toStreaming() { - return new StarRocksWrite(info, config); + return new StarRocksWrite(info, config, starRocksSchema, false); } @Override @@ -100,5 +103,8 @@ public Distribution requiredDistribution() { public SortOrder[] requiredOrdering() { return new SortOrder[0]; } + + + } } diff --git a/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWriterFactory.java b/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWriterFactory.java index c3aba06a..144d83f7 100644 --- a/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWriterFactory.java +++ b/src/main/java/com/starrocks/connector/spark/sql/write/StarRocksWriterFactory.java @@ -39,7 +39,6 @@ public StarRocksWriterFactory(StructType schema, WriteStarRocksConfig config) { this.schema = schema; this.config = config; } - @Override public DataWriter createWriter(int partitionId, long taskId) { return createAndOpenWriter(partitionId, taskId, -1); diff --git a/src/main/scala/com/starrocks/connector/spark/catalog/StarRocksCatalog.scala b/src/main/scala/com/starrocks/connector/spark/catalog/StarRocksCatalog.scala index 05159d20..cc9b9cc8 100644 --- a/src/main/scala/com/starrocks/connector/spark/catalog/StarRocksCatalog.scala +++ b/src/main/scala/com/starrocks/connector/spark/catalog/StarRocksCatalog.scala @@ -93,4 +93,5 @@ class StarRocksCatalog extends TableCatalog override def dropNamespace(namespace: Array[String], cascade: Boolean): Boolean = throw new UnsupportedOperationException + } diff --git a/src/test/java/com/starrocks/connector/spark/examples/SimpleWrite.java b/src/test/java/com/starrocks/connector/spark/examples/SimpleWrite.java index 7756772f..39944882 100644 --- a/src/test/java/com/starrocks/connector/spark/examples/SimpleWrite.java +++ b/src/test/java/com/starrocks/connector/spark/examples/SimpleWrite.java @@ -56,8 +56,8 @@ public class SimpleWrite { // "replication_num" = "1" // ); - private static final String FE_HTTP = "127.0.0.1:11901"; - private static final String FE_JDBC = "jdbc:mysql://127.0.0.1:11903"; + private static final String FE_HTTP = "127.0.0.1:8030"; + private static final String FE_JDBC = "jdbc:mysql://127.0.0.1:9030"; private static final String DB = "test"; private static final String TABLE = "score_board"; private static final String TABLE_ID = DB + "." + TABLE; @@ -66,8 +66,8 @@ public class SimpleWrite { public static void main(String[] args) throws Exception { dataFrameBatchWrite(); - dataFrameSteamingWrite(); - sqlWrite(); +// dataFrameSteamingWrite(); +// sqlWrite(); } // write using DataFrame in batch mode @@ -82,8 +82,8 @@ private static void dataFrameBatchWrite() { // 2. create a source DataFrame from a list of data, and define // the schema which is mapped to the StarRocks table List data = Arrays.asList( - RowFactory.create(1, "row1", 1), - RowFactory.create(2, "row2", 2) + RowFactory.create(1, "row1666", 1), + RowFactory.create(2, "row27777", 2) ); StructType schema = new StructType(new StructField[] { new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), diff --git a/src/test/java/com/starrocks/connector/spark/sql/ITTestBase.java b/src/test/java/com/starrocks/connector/spark/sql/ITTestBase.java index 5166c5ef..77c01651 100644 --- a/src/test/java/com/starrocks/connector/spark/sql/ITTestBase.java +++ b/src/test/java/com/starrocks/connector/spark/sql/ITTestBase.java @@ -55,6 +55,7 @@ public abstract class ITTestBase { protected static String FE_HTTP = "10.37.42.50:8031"; protected static String FE_JDBC = "jdbc:mysql://10.37.42.50:9031"; + protected static String USER = "root"; protected static String PASSWORD = ""; private static final boolean DEBUG_MODE = false; @@ -65,10 +66,10 @@ public abstract class ITTestBase { @BeforeEach public void beforeClass() throws Exception { Properties props = loadConnProps(); - FE_HTTP = props.getProperty("starrocks.fe.http.url", FE_HTTP); - FE_JDBC = props.getProperty("starrocks.fe.jdbc.url", FE_JDBC); + FE_HTTP = "127.0.0.1:8030"; + FE_JDBC = "jdbc:mysql://127.0.0.1:9030"; USER = props.getProperty("starrocks.user", USER); - PASSWORD = props.getProperty("starrocks.password", PASSWORD); + PASSWORD =""; try { DB_CONNECTION = DriverManager.getConnection(FE_JDBC, USER, PASSWORD); diff --git a/src/test/java/com/starrocks/connector/spark/sql/ReadWriteITTest.java b/src/test/java/com/starrocks/connector/spark/sql/ReadWriteITTest.java index d4a07603..49e599e0 100644 --- a/src/test/java/com/starrocks/connector/spark/sql/ReadWriteITTest.java +++ b/src/test/java/com/starrocks/connector/spark/sql/ReadWriteITTest.java @@ -19,22 +19,6 @@ package com.starrocks.connector.spark.sql; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.RowFactory; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; -import org.apache.spark.sql.types.ArrayType; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.DecimalType; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.types.StructType; -import org.junit.Ignore; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; - import java.math.BigDecimal; import java.sql.Date; import java.sql.Statement; @@ -51,8 +35,20 @@ import java.util.Map; import java.util.stream.Collectors; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; public class ReadWriteITTest extends ITTestBase { @Test @@ -1279,40 +1275,216 @@ private void prepareTimestampTypeWithMilliseconds(String tableName) throws Excep DB_NAME, tableName); executeSrSQL(createStarRocksTable); } + @Test + public void TestStageWrite() throws Exception { + String tableName = "testStageWrite" + genRandomUuid(); + prepareStageTest(tableName); + SparkSession spark = SparkSession + .builder() + .master("local[1]") + .appName("TestStageWrite") + .getOrCreate(); + //test always mode + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("c3_double", DataTypes.DoubleType, true, Metadata.empty()), + new StructField("c4_float", DataTypes.FloatType, true, Metadata.empty()), + new StructField("c5_int", DataTypes.IntegerType, true, Metadata.empty()), + new StructField("c6_largeint", DataTypes.LongType, true, Metadata.empty()), // 对应 LARGEINT,使用 LongType + new StructField("c7_smallint", DataTypes.ShortType, true, Metadata.empty()), + new StructField("c8_tinyint", DataTypes.ByteType, true, Metadata.empty()), + new StructField("c9_date", DataTypes.DateType, true, Metadata.empty()), + new StructField("c10_datetime", DataTypes.TimestampType, true, Metadata.empty()), + new StructField("c11_char", DataTypes.StringType, true, Metadata.empty()), + }); + List data = new ArrayList<>(); + data.add(RowFactory.create(1, 12345.12, 12345.12f, 123456, 987654321012345671L, (short) 1234, (byte) 123, Date.valueOf("2024-01-01"), Timestamp.valueOf("2024-01-01 12:34:56"), "char_value_11")); + data.add(RowFactory.create(2, 23456.23, 23456.23f, 234567, 887654321012345672L, (short) 2345, (byte) 234, Date.valueOf("2024-02-02"), Timestamp.valueOf("2024-02-02 12:34:56"), "char_value_22")); + data.add(RowFactory.create(3, 34567.34, 34567.34f, 345678, 787654321012345673L, (short) 3456, (byte) 345, Date.valueOf("2024-03-03"), Timestamp.valueOf("2024-03-03 12:34:56"), "char_value_33")); + Dataset df = spark.createDataFrame(data, schema); + Map options = new HashMap<>(); + options.put("starrocks.fe.http.url", FE_HTTP); + options.put("starrocks.fe.jdbc.url", FE_JDBC); + options.put("starrocks.table.identifier", String.join(".", DB_NAME, tableName)); + options.put("starrocks.user", USER); + options.put("starrocks.password", PASSWORD); + options.put("starrocks.write.stage.mode", "true"); + options.put("starrocks.write.properties.partial_update_mode", "column"); + options.put("starrocks.write.properties.partial_update", "true"); + options.put("starrocks.write.stage.session.query.timeout", "309"); + options.put("starrocks.write.stage.session.query.mem.limit", "100000000"); + options.put("starrocks.write.stage.session.exec.mem.limit", "8589934592"); + options.put("starrocks.write.stage.columns.update.ratio", "20"); + + options.put("starrocks.write.stage.use", "always"); + options.put("starrocks.columns", "id,c3_double,c4_float,c5_int,c6_largeint,c7_smallint,c8_tinyint,c9_date,c10_datetime,c11_char"); + df.write().format("starrocks") + .mode(SaveMode.Append) + .options(options) + .save(); + List> expectedData = Arrays.asList( + Arrays.asList(1, 12345.12, 12345.12f, 123456, 987654321012345671L, (short) 1234, (byte) 123, Date.valueOf("2024-01-01"), Timestamp.valueOf("2024-01-01 12:34:56"), "char_value_11"), + Arrays.asList(2, 23456.23, 23456.23f, 234567, 887654321012345672L, (short) 2345, (byte) 234, Date.valueOf("2024-02-02"), Timestamp.valueOf("2024-02-02 12:34:56"), "char_value_22"), + Arrays.asList(3, 34567.34, 34567.34f, 345678, 787654321012345673L, (short) 3456, (byte) 345, Date.valueOf("2024-03-03"), Timestamp.valueOf("2024-03-03 12:34:56"), "char_value_33") + ); + String selectSql = String.format("SELECT id,c3_double,c4_float,c5_int,c6_largeint,c7_smallint,c8_tinyint,c9_date,c10_datetime,c11_char FROM %s.%s where id between 1 and 3 ", DB_NAME, tableName); + List> actual = queryTable(DB_CONNECTION, selectSql); + verifyResult(expectedData, actual); + //test auto mode + List data1 = Arrays.asList( + RowFactory.create(1, 101L, false, new BigDecimal("111.1")), + RowFactory.create(2, 102L, true, new BigDecimal("111.2")), + RowFactory.create(3, 103L, false, new BigDecimal("111.3")), + RowFactory.create(4, 104L, true, new BigDecimal("111.4")) + ); + StructType schema1 = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("c0_bigint", DataTypes.LongType, true, Metadata.empty()), + new StructField("c1_boolean", DataTypes.BooleanType, true, Metadata.empty()), + new StructField("c2_decimal", DataTypes.createDecimalType(10, 1), true, Metadata.empty()), - // To enable this test, need to inject response delay (>10s) on BE. - // Have verified this test manually - @Ignore - @org.junit.Test - public void testSocketTimeout() throws Exception { - String tableName = "testSocketTimeout" + genRandomUuid(); - prepareScoreBoardTable(tableName); + }); + Dataset df1 = spark.createDataFrame(data1, schema1); + options.put("starrocks.write.stage.use", "auto"); + options.put("starrocks.columns", "id,c0_bigint,c1_boolean,c2_decimal"); + options.put("starrocks.write.stage.columns.update.ratio", "60"); + df1.write().format("starrocks") + .mode(SaveMode.Append) + .options(options) + .save(); + String selectSql1 = String.format("SELECT id,c0_bigint,c1_boolean,c2_decimal FROM %s.%s where id between 1 and 4 ", DB_NAME, tableName); + List> actual1 = queryTable(DB_CONNECTION, selectSql1); + List> expectedData1 = Arrays.asList( + Arrays.asList(1, 101L, false, new BigDecimal("111.1")), + Arrays.asList(2, 102L, true, new BigDecimal("111.2")), + Arrays.asList(3, 103L, false, new BigDecimal("111.3")), + Arrays.asList(4, 104L, true, new BigDecimal("111.4")) + ); + verifyResult(expectedData1, actual1); + +//test never mode + List data2 = Arrays.asList( + RowFactory.create(1, 102L, false, new BigDecimal("111.1")), + RowFactory.create(2, 103L, true, new BigDecimal("111.2")), + RowFactory.create(3, 104L, false, new BigDecimal("111.3")), + RowFactory.create(4, 105L, true, new BigDecimal("111.4")) + ); + List> expectedData2 = Arrays.asList( + Arrays.asList(1, 102L, false, new BigDecimal("111.1")), + Arrays.asList(2, 103L, true, new BigDecimal("111.2")), + Arrays.asList(3, 104L, false, new BigDecimal("111.3")), + Arrays.asList(4, 105L, true, new BigDecimal("111.4")) + ); + options.put("starrocks.write.stage.use", "never"); + Dataset df2 = spark.createDataFrame(data2, schema1); + df2.write().format("starrocks") + .mode(SaveMode.Append) + .options(options) + .save(); + List> actual2 = queryTable(DB_CONNECTION, selectSql1); + verifyResult(expectedData2,actual2); + spark.stop(); + + } + + + + + + private void prepareStageTest(String tableName) throws Exception { + String createStarRocksTable = + String.format("CREATE TABLE %s.%s (" + + "id INT," + // 主键 + "c0_bigint BIGINT," + + "c1_boolean BOOLEAN," + + "c2_decimal DECIMAL(10, 1)," + + "c3_double DOUBLE," + + "c4_float FLOAT," + + "c5_int INT," + + "c6_largeint LARGEINT," + + "c7_smallint SMALLINT," + + "c8_tinyint TINYINT," + + "c9_date DATE," + + "c10_datetime DATETIME," + + "c11_char CHAR(50)," + + "c12_string STRING," + + "c13_varchar VARCHAR(255)" + + ") ENGINE=OLAP " + + "PRIMARY KEY(id) " + + "DISTRIBUTED BY HASH(id) BUCKETS 2 " + + "PROPERTIES (" + + "\"replication_num\" = \"1\"" + + ")", + DB_NAME, tableName); + + executeSrSQL(createStarRocksTable); SparkSession spark = SparkSession .builder() .master("local[1]") - .appName("testSocketTimeout") + .appName("prepareTestStage") .getOrCreate(); - String ddl = String.format("CREATE TABLE sr_table \n" + - " USING starrocks\n" + - "OPTIONS(\n" + - " \"starrocks.table.identifier\"=\"%s\",\n" + - " \"starrocks.fe.http.url\"=\"%s\",\n" + - " \"starrocks.fe.jdbc.url\"=\"%s\",\n" + - " \"starrocks.write.max.retries\"=\"0\",\n" + - " \"starrocks.write.socket.timeout.ms\"=\"10000\",\n" + - " \"starrocks.user\"=\"%s\",\n" + - " \"starrocks.password\"=\"%s\"\n" + - ")", String.join(".", DB_NAME, tableName), FE_HTTP, FE_JDBC, USER, PASSWORD); - spark.sql(ddl); - try { - spark.sql("INSERT INTO sr_table VALUES (1, \"2\", 3), (2, \"3\", 4)"); - fail("Should throw exception"); - } catch (Exception e) { - assertTrue(e.getMessage().contains("Read timed out")); - } + StructType schema = new StructType(new StructField[]{ + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("c0_bigint", DataTypes.LongType, true, Metadata.empty()), + new StructField("c1_boolean", DataTypes.BooleanType, true, Metadata.empty()), + new StructField("c2_decimal", DataTypes.createDecimalType(10, 1), true, Metadata.empty()), + new StructField("c3_double", DataTypes.DoubleType, true, Metadata.empty()), + new StructField("c4_float", DataTypes.FloatType, true, Metadata.empty()), + new StructField("c5_int", DataTypes.IntegerType, true, Metadata.empty()), + new StructField("c6_largeint", DataTypes.LongType, true, Metadata.empty()), // 对应 LARGEINT,使用 LongType + new StructField("c7_smallint", DataTypes.ShortType, true, Metadata.empty()), + new StructField("c8_tinyint", DataTypes.ByteType, true, Metadata.empty()), + new StructField("c9_date", DataTypes.DateType, true, Metadata.empty()), + new StructField("c10_datetime", DataTypes.TimestampType, true, Metadata.empty()), + new StructField("c11_char", DataTypes.StringType, true, Metadata.empty()), + new StructField("c12_string", DataTypes.StringType, true, Metadata.empty()), + new StructField("c13_varchar", DataTypes.StringType, true, Metadata.empty()) + }); + + + List data = new ArrayList<>(); + data.add(RowFactory.create(1, 1234567890123L, true, new BigDecimal("12345.6"), 12345.67, 12345.67f, 12345, 987654321012345678L, (short) 123, (byte) 12, Date.valueOf("2023-01-01"), Timestamp.valueOf("2023-01-01 12:34:56"), "char_value_1", "string_value_1", "varchar_value_1")); + data.add(RowFactory.create(2, 2234567890123L, false, new BigDecimal("23456.7"), 23456.78, 23456.78f, 23456, 887654321012345678L, (short) 234, (byte) 23, Date.valueOf("2023-02-02"), Timestamp.valueOf("2023-02-02 12:34:56"), "char_value_2", "string_value_2", "varchar_value_2")); + data.add(RowFactory.create(3, 3234567890123L, true, new BigDecimal("34567.8"), 34567.89, 34567.89f, 34567, 787654321012345678L, (short) 345, (byte) 34, Date.valueOf("2023-03-03"), Timestamp.valueOf("2023-03-03 12:34:56"), "char_value_3", "string_value_3", "varchar_value_3")); + data.add(RowFactory.create(4, 4234567890123L, false, new BigDecimal("45678.9"), 45678.90, 45678.90f, 45678, 687654321012345678L, (short) 456, (byte) 45, Date.valueOf("2023-04-04"), Timestamp.valueOf("2023-04-04 12:34:56"), "char_value_4", "string_value_4", "varchar_value_4")); + data.add(RowFactory.create(5, 5234567890123L, true, new BigDecimal("56789.0"), 56789.01, 56789.01f, 56789, 587654321012345678L, (short) 567, (byte) 56, Date.valueOf("2023-05-05"), Timestamp.valueOf("2023-05-05 12:34:56"), "char_value_5", "string_value_5", "varchar_value_5")); + data.add(RowFactory.create(6, 6234567890123L, false, new BigDecimal("67890.1"), 67890.12, 67890.12f, 67890, 487654321012345678L, (short) 678, (byte) 67, Date.valueOf("2023-06-06"), Timestamp.valueOf("2023-06-06 12:34:56"), "char_value_6", "string_value_6", "varchar_value_6")); + data.add(RowFactory.create(7, 7234567890123L, true, new BigDecimal("78901.2"), 78901.23, 78901.23f, 78901, 387654321012345678L, (short) 789, (byte) 78, Date.valueOf("2023-07-07"), Timestamp.valueOf("2023-07-07 12:34:56"), "char_value_7", "string_value_7", "varchar_value_7")); + data.add(RowFactory.create(8, 8234567890123L, false, new BigDecimal("89012.3"), 89012.34, 89012.34f, 89012, 287654321012345678L, (short) 890, (byte) 89, Date.valueOf("2023-08-08"), Timestamp.valueOf("2023-08-08 12:34:56"), "char_value_8", "string_value_8", "varchar_value_8")); + data.add(RowFactory.create(9, 9234567890123L, true, new BigDecimal("90123.4"), 90123.45, 90123.45f, 90123, 187654321012345678L, (short) 901, (byte) 90, Date.valueOf("2023-09-09"), Timestamp.valueOf("2023-09-09 12:34:56"), "char_value_9", "string_value_9", "varchar_value_9")); + data.add(RowFactory.create(10, 10234567890123L, false, new BigDecimal("01234.5"), 01234.56, 01234.56f, 1234, 98765432101234567L, (short) 12, (byte) 1, Date.valueOf("2023-10-10"), Timestamp.valueOf("2023-10-10 12:34:56"), "char_value_10", "string_value_10", "varchar_value_10")); + + Dataset df = spark.createDataFrame(data, schema); + + df.write() + .format("starrocks") + .option("starrocks.fe.http.url", FE_HTTP) + .option("starrocks.fe.jdbc.url", FE_JDBC) + .option("starrocks.user", USER) + .option("starrocks.password", PASSWORD) + .option("starrocks.table.identifier", DB_NAME + "." + tableName) + .mode(SaveMode.Append) + .save(); + + String selectSql = String.format("SELECT * FROM %s.%s", DB_NAME, tableName); + + List> actualRows = queryTable(DB_CONNECTION, selectSql); + + List> expectedData = Arrays.asList( + Arrays.asList(1, 1234567890123L, true, new BigDecimal("12345.6"), 12345.67, 12345.67f, 12345, 987654321012345678L, (short) 123, (byte) 12, Date.valueOf("2023-01-01"), Timestamp.valueOf("2023-01-01 12:34:56"), "char_value_1", "string_value_1", "varchar_value_1"), + Arrays.asList(2, 2234567890123L, false, new BigDecimal("23456.7"), 23456.78, 23456.78f, 23456, 887654321012345678L, (short) 234, (byte) 23, Date.valueOf("2023-02-02"), Timestamp.valueOf("2023-02-02 12:34:56"), "char_value_2", "string_value_2", "varchar_value_2"), + Arrays.asList(3, 3234567890123L, true, new BigDecimal("34567.8"), 34567.89, 34567.89f, 34567, 787654321012345678L, (short) 345, (byte) 34, Date.valueOf("2023-03-03"), Timestamp.valueOf("2023-03-03 12:34:56"), "char_value_3", "string_value_3", "varchar_value_3"), + Arrays.asList(4, 4234567890123L, false, new BigDecimal("45678.9"), 45678.90, 45678.90f, 45678, 687654321012345678L, (short) 456, (byte) 45, Date.valueOf("2023-04-04"), Timestamp.valueOf("2023-04-04 12:34:56"), "char_value_4", "string_value_4", "varchar_value_4"), + Arrays.asList(5, 5234567890123L, true, new BigDecimal("56789.0"), 56789.01, 56789.01f, 56789, 587654321012345678L, (short) 567, (byte) 56, Date.valueOf("2023-05-05"), Timestamp.valueOf("2023-05-05 12:34:56"), "char_value_5", "string_value_5", "varchar_value_5"), + Arrays.asList(6, 6234567890123L, false, new BigDecimal("67890.1"), 67890.12, 67890.12f, 67890, 487654321012345678L, (short) 678, (byte) 67, Date.valueOf("2023-06-06"), Timestamp.valueOf("2023-06-06 12:34:56"), "char_value_6", "string_value_6", "varchar_value_6"), + Arrays.asList(7, 7234567890123L, true, new BigDecimal("78901.2"), 78901.23, 78901.23f, 78901, 387654321012345678L, (short) 789, (byte) 78, Date.valueOf("2023-07-07"), Timestamp.valueOf("2023-07-07 12:34:56"), "char_value_7", "string_value_7", "varchar_value_7"), + Arrays.asList(8, 8234567890123L, false, new BigDecimal("89012.3"), 89012.34, 89012.34f, 89012, 287654321012345678L, (short) 890, (byte) 89, Date.valueOf("2023-08-08"), Timestamp.valueOf("2023-08-08 12:34:56"), "char_value_8", "string_value_8", "varchar_value_8"), + Arrays.asList(9, 9234567890123L, true, new BigDecimal("90123.4"), 90123.45, 90123.45f, 90123, 187654321012345678L, (short) 901, (byte) 90, Date.valueOf("2023-09-09"), Timestamp.valueOf("2023-09-09 12:34:56"), "char_value_9", "string_value_9", "varchar_value_9"), + Arrays.asList(10, 10234567890123L, false, new BigDecimal("01234.5"), 01234.56, 01234.56f, 1234, 98765432101234567L, (short) 12, (byte) 1, Date.valueOf("2023-10-10"), Timestamp.valueOf("2023-10-10 12:34:56"), "char_value_10", "string_value_10", "varchar_value_10") + ); + verifyResult(expectedData, actualRows); - spark.stop(); } } From 3e28077c730e156858972ac40216531ebcd37ffd Mon Sep 17 00:00:00 2001 From: Cadenana <768972002@qq.com> Date: Tue, 19 Nov 2024 20:07:39 +0800 Subject: [PATCH 2/3] StarRocks StageMode use Signed-off-by: Cadenana <768972002@qq.com> --- ...7\347\224\250\346\226\207\346\241\2431.md" | 138 ------------------ 1 file changed, 138 deletions(-) delete mode 100644 "docs/StarRocks-Stage\346\250\241\345\274\217\344\275\277\347\224\250\346\226\207\346\241\2431.md" diff --git "a/docs/StarRocks-Stage\346\250\241\345\274\217\344\275\277\347\224\250\346\226\207\346\241\2431.md" "b/docs/StarRocks-Stage\346\250\241\345\274\217\344\275\277\347\224\250\346\226\207\346\241\2431.md" deleted file mode 100644 index 61d6b359..00000000 --- "a/docs/StarRocks-Stage\346\250\241\345\274\217\344\275\277\347\224\250\346\226\207\346\241\2431.md" +++ /dev/null @@ -1,138 +0,0 @@ -## 列模式部分更新-暂存区模式 - -在列模式部分更新的场景中,一般涉及少数列大量行,暂存区模式适用于某一列或多列更新的数据行占比很大,甚至近似于全列更新的场景。 - -配置参数starrocks.write.stage.use用于控制暂存区模式的使用,支持取值为 - -always - -指定使用部分列更新时采用暂存区模式,适用于更新时行占比很大的场景 - -auto - -系统会根据更新数据涉及的列以及目标表的列数,行占比ratio决定是否使用暂存区模式 - -更新数据的行占比超过60,且更新列数少于4个 - -更新列数占所有列数的百分比小于25% - -反之,系统不会使用暂存区模式 - -starrocks.write.stage.columns.update.ratio:被更新数据列的行占比,可指定占比,默认20 - -never(默认值) - -指定不使用暂存区模式,涉及行占比不大时采用 - - - -**sql配置** - -暂存区模式采用在StarRocks中创建临时表实现,使用暂存区模式更新导入数据时需要执行update语句,可以使用StarRocks系统变量配置sql执行的参数,对应参数为starrocks.write.stage.session.* - -参数映射:exec_mem_limit->exec.mem.limit //todo - -比如:.option ("starrocks.write.stage.session.exec.mem.limit","8589934592") ,可以实现update执行时的StarRocks内存限制。 - -**暂存表清理** - -正常流程中,在执行完update后会将暂存表清楚,完成整个部分列更新流程。一些特殊情况下暂存表未被正确删除,会占用数据库空间。可以定期执行SparkDrop作业,删除存在时长超过一天的暂存表实现对冗余暂存表的清理。 - -参数: - -1. FE MySQL Server 端口(默认127.0.0.1:9030) - -2.用户名(username) - -3.密码(password) - -4.Spark作业执行环境(默认local[*]) 用--master指定也可(优先级更高) - -用例 - -``` -spark-submit \ - --class com.TableDrop.SparkDrop \ - --master local[*] \ -/opt/SparkTask/original-SparkDrop-1.0-SNAPSHOT.jar 192.168.181.1:9030 root "password" local[*] -``` - -**使用示例** - -建表语句: - -``` -CREATE TABLE `test`.`stage_test` -( - `id` BIGINT NOT NULL COMMENT "主键", - `name` VARCHAR(65533) NULL , - `score` INT NOT NULL , - `is_active` BOOLEAN NULL , - `decimal_score` DECIMAL(10, 2) NULL , - `double_value` DOUBLE NULL , - `float_value` FLOAT NULL , - `largeint_value` LARGEINT NULL , - `smallint_value` SMALLINT NULL , - `tinyint_value` TINYINT NULL , - `char_value` CHAR(10) NULL , - `string_value` STRING NULL , - `create_date` DATE NULL -) -ENGINE=OLAP -PRIMARY KEY(`id`) -COMMENT "OLAP" -DISTRIBUTED BY HASH(`id`) -BUCKETS 4; - -``` - -插入数据 - -``` -INSERT INTO `test`.`stage_test` (`id`, `name`, `score`, `is_active`, `decimal_score`, `double_value`, `float_value`, `largeint_value`, `smallint_value`, `tinyint_value`, `char_value`, `string_value`, `create_date`) -VALUES - (1, 'Alice', 95, true, 95.50, 95.5, 95.5, 10000000000, 100, 10, 'A', 'Alice String', '2024-01-01'), - (2, 'Bob', 88, true, 88.00, 88.0, 88.0, 20000000000, 90, 9, 'B', 'Bob String', '2024-01-02'), - (3, 'Charlie', 76, false, 76.75, 76.75, 76.75, 30000000000, 80, 8, 'C', 'Charlie String', '2024-01-03'), - (4, 'David', 82, true, 82.25, 82.25, 82.25, 40000000000, 70, 7, 'D', 'David String', '2024-01-04'), - (5, 'Eva', 90, true, 90.00, 90.0, 90.0, 50000000000, 60, 6, 'E', 'Eva String', '2024-01-05'), - (6, 'Frank', 65, false, 65.50, 65.5, 65.5, 60000000000, 50, 5, 'F', 'Frank String', '2024-01-06'), - (7, 'Grace', 70, true, 70.00, 70.0, 70.0, 70000000000, 40, 4, 'G', 'Grace String', '2024-01-07'), - (8, 'Heidi', 80, true, 80.00, 80.0, 80.0, 80000000000, 30, 3, 'H', 'Heidi String', '2024-01-08'), - (9, 'Ivan', 92, true, 92.00, 92.0, 92.0, 90000000000, 20, 2, 'I', 'Ivan String', '2024-01-09'), - (10, 'Judy', 85, false, 85.00, 85.0, 85.0, 10000000000, 10, 1, 'J', 'Judy String', '2024-01-10'); - -``` - -以Scala语言为例,在Spark-Shell中运行 - -``` -import org.apache.spark.sql.SparkSession - -val data = Seq((1, "starrocks", 124578), (2, "spark", 235689)) -val df = data.toDF("id", "name", "score") - -df.write.format("starrocks") - .option("starrocks.fe.http.url", "http://127.0.0.1:8030") - .option("starrocks.fe.jdbc.url", "jdbc:mysql://127.0.0.1:9030") - .option("starrocks.table.identifier", "test.stage_test") - .option("starrocks.user", "root") - .option("starrocks.password", "") - .option("starrocks.write.stage.use","always") - .option("starrocks.write.properties.partial_update_mode","column") - .option("starrocks.write.properties.partial_update","true") - .option("starrocks.columns", "id,name,score") - .option("starrocks.write.stage.session.query.timeout","309") - .option("starrocks.write.stage.session.query.mem.limit","100000000") - .option("starrocks.write.stage.session.exec.mem.limit","8589934592") - .option("starrocks.write.stage.columns.update.ratio","80") - .mode("append") - .save() -``` - -运行后会在temp_db下创建具有"id", "name", "score"字段的临时表,并更新test.stage_test,更新完成后删除临时表。 - - - - - From 5f927b1f093d2442b50465102ac015a2c4d684a9 Mon Sep 17 00:00:00 2001 From: Cadenana <768972002@qq.com> Date: Tue, 19 Nov 2024 20:19:42 +0800 Subject: [PATCH 3/3] StarRocks StageMode use Signed-off-by: Cadenana <768972002@qq.com> --- .../java/com/starrocks/connector/spark/sql/ITTestBase.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/test/java/com/starrocks/connector/spark/sql/ITTestBase.java b/src/test/java/com/starrocks/connector/spark/sql/ITTestBase.java index 77c01651..5166c5ef 100644 --- a/src/test/java/com/starrocks/connector/spark/sql/ITTestBase.java +++ b/src/test/java/com/starrocks/connector/spark/sql/ITTestBase.java @@ -55,7 +55,6 @@ public abstract class ITTestBase { protected static String FE_HTTP = "10.37.42.50:8031"; protected static String FE_JDBC = "jdbc:mysql://10.37.42.50:9031"; - protected static String USER = "root"; protected static String PASSWORD = ""; private static final boolean DEBUG_MODE = false; @@ -66,10 +65,10 @@ public abstract class ITTestBase { @BeforeEach public void beforeClass() throws Exception { Properties props = loadConnProps(); - FE_HTTP = "127.0.0.1:8030"; - FE_JDBC = "jdbc:mysql://127.0.0.1:9030"; + FE_HTTP = props.getProperty("starrocks.fe.http.url", FE_HTTP); + FE_JDBC = props.getProperty("starrocks.fe.jdbc.url", FE_JDBC); USER = props.getProperty("starrocks.user", USER); - PASSWORD =""; + PASSWORD = props.getProperty("starrocks.password", PASSWORD); try { DB_CONNECTION = DriverManager.getConnection(FE_JDBC, USER, PASSWORD);