diff --git a/chunjun-connectors/chunjun-connector-starrocks/pom.xml b/chunjun-connectors/chunjun-connector-starrocks/pom.xml index ee2e5544e3..fc695fb905 100644 --- a/chunjun-connectors/chunjun-connector-starrocks/pom.xml +++ b/chunjun-connectors/chunjun-connector-starrocks/pom.xml @@ -12,56 +12,42 @@ chunjun-connector-starrocks ChunJun : Connectors : StarRocks + + + 8 + 8 + 5.0.0 + + + + mysql + mysql-connector-java + 5.1.49 + + + + com.alibaba + fastjson + 1.2.79 + + com.starrocks - flink-connector-starrocks - 1.1.13_flink-1.12 - provided - - - org.apache.flink - flink-table-common - - - org.apache.flink - flink-table-api - - - org.apache.flink - flink-table-api-java - - - flink-clients_2.11 - org.apache.flink - - - flink-table-api-java-bridge_2.11 - org.apache.flink - - - flink-table-api-scala-bridge_2.11 - org.apache.flink - - - flink-table-api-scala_2.11 - org.apache.flink - - - flink-table-planner-blink_2.11 - org.apache.flink - - - flink-table-planner_2.11 - org.apache.flink - - + starrocks-thrift-sdk + 1.0.1 + + + + org.apache.arrow + arrow-vector + ${arrow.version} + - com.dtstack.chunjun - chunjun-connector-mysql - ${project.version} - provided + org.apache.arrow + arrow-memory-netty + ${arrow.version} @@ -96,6 +82,20 @@ + + + io.netty + com.dtstack.chunjun.connector.starrocks.shaded.io.netty + + + org.apache.arrow + com.dtstack.chunjun.connector.starrocks.shaded.org.apache.arrow + + + com.google.flatbuffers + com.dtstack.chunjun.connector.starrocks.shaded.com.google.flatbuffers + + diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/conf/LoadConf.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/conf/LoadConf.java new file mode 100644 index 0000000000..f29cbdc371 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/conf/LoadConf.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.conf; + +import com.dtstack.chunjun.connector.starrocks.options.ConstantValue; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +/** @author liuliu 2022/7/12 */ +public class LoadConf implements Serializable { + + private static final Long serialVersionUID = 1L; + + /** Timeout duration of the HTTP connection when checking the connectivity with StarRocks */ + private Integer httpCheckTimeoutMs = ConstantValue.HTTP_CHECK_TIMEOUT_DEFAULT; + + private Integer queueOfferTimeoutMs = ConstantValue.QUEUE_OFFER_TIMEOUT_DEFAULT; + + private Integer queuePollTimeoutMs = ConstantValue.QUEUE_POLL_TIMEOUT_DEFAULT; + + private Long batchMaxSize = ConstantValue.SINK_BATCH_MAX_BYTES_DEFAULT; + + private Long batchMaxRows = ConstantValue.SINK_BATCH_MAX_ROWS_DEFAULT; + + private Map headProperties = new HashMap<>(); + + public Integer getHttpCheckTimeoutMs() { + return httpCheckTimeoutMs; + } + + public void setHttpCheckTimeoutMs(Integer httpCheckTimeoutMs) { + this.httpCheckTimeoutMs = httpCheckTimeoutMs; + } + + public Integer getQueueOfferTimeoutMs() { + return queueOfferTimeoutMs; + } + + public void setQueueOfferTimeoutMs(Integer queueOfferTimeoutMs) { + this.queueOfferTimeoutMs = queueOfferTimeoutMs; + } + + public Integer getQueuePollTimeoutMs() { + return queuePollTimeoutMs; + } + + public void setQueuePollTimeoutMs(Integer queuePollTimeoutMs) { + this.queuePollTimeoutMs = queuePollTimeoutMs; + } + + public Long getBatchMaxSize() { + return batchMaxSize; + } + + public void setBatchMaxSize(Long batchMaxSize) { + this.batchMaxSize = batchMaxSize; + } + + public Long getBatchMaxRows() { + return batchMaxRows; + } + + public void setBatchMaxRows(Long batchMaxRows) { + this.batchMaxRows = batchMaxRows; + } + + public Map getHeadProperties() { + return headProperties; + } + + public void setHeadProperties(Map headProperties) { + this.headProperties = headProperties; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/conf/LoadConfBuilder.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/conf/LoadConfBuilder.java new file mode 100644 index 0000000000..be3ac6d491 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/conf/LoadConfBuilder.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.conf; + +import java.util.Map; + +/** @author liuliu 2022/7/12 */ +public class LoadConfBuilder { + private final LoadConf loadConf; + + public LoadConfBuilder() { + this.loadConf = new LoadConf(); + } + + public LoadConf build() { + return this.loadConf; + } + + public LoadConfBuilder setBatchMaxSize(Long batchMaxSize) { + loadConf.setBatchMaxSize(batchMaxSize); + return this; + } + + public LoadConfBuilder setBatchMaxRows(Long batMaxRows) { + loadConf.setBatchMaxRows(batMaxRows); + return this; + } + + public LoadConfBuilder setHttpCheckTimeoutMs(int httpCheckTimeout) { + loadConf.setHttpCheckTimeoutMs(httpCheckTimeout); + return this; + } + + public LoadConfBuilder setQueueOfferTimeoutMs(int queueOfferTimeoutMs) { + loadConf.setQueueOfferTimeoutMs(queueOfferTimeoutMs); + return this; + } + + public LoadConfBuilder setQueuePollTimeoutMs(int queuePollTimeoutMs) { + loadConf.setQueuePollTimeoutMs(queuePollTimeoutMs); + return this; + } + + public LoadConfBuilder setHeadProperties(Map loadProperties) { + loadConf.setHeadProperties(loadProperties); + return this; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/conf/StarRocksConf.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/conf/StarRocksConf.java index 65da278f91..b305dacfd1 100644 --- a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/conf/StarRocksConf.java +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/conf/StarRocksConf.java @@ -17,40 +17,222 @@ */ package com.dtstack.chunjun.connector.starrocks.conf; -import com.dtstack.chunjun.connector.jdbc.conf.JdbcConf; +import com.dtstack.chunjun.conf.ChunJunCommonConf; -import java.util.ArrayList; +import org.apache.flink.table.types.DataType; + +import java.util.HashMap; import java.util.List; +import java.util.Map; /** * @author lihongwei * @date 2022/04/11 */ -public class StarRocksConf extends JdbcConf { +public class StarRocksConf extends ChunJunCommonConf { + + // common + private String url; + + private List feNodes; + + private String database; + + private String table; + + private String username; + + private String password; + + private String writeMode; + /** * default value is 3 */ + private Integer maxRetries = 3; + + // sink + /** The time to sleep when the tablet version is too large */ + private long waitRetryMills = 18000; + + /** 是否配置了NameMapping, true, RowData中将携带名称匹配后的数据库和表名, sink端配置的database和table失效* */ + private boolean nameMapped; + + private LoadConf loadConf = new LoadConf(); + + // source + private String[] fieldNames; - /** fe_ip:http_port 多个地址分号连接 */ - private String loadUrl; - /** 主键模型表需要传入主键列表 */ - private List primaryKey = new ArrayList<>(); + private DataType[] dataTypes; + + private String filterStatement; + + private int beClientKeepLiveMin = 10; + + private int beQueryTimeoutSecond = 600; + + private int beClientTimeout = 3000; + + private int beFetchRows = 1024; + + private long beFetchMaxBytes = 1024 * 1024 * 1024; + + private Map beSocketProperties = new HashMap<>(); + + public String getUrl() { + return url; + } + + public void setUrl(String url) { + this.url = url; + } + + public List getFeNodes() { + return feNodes; + } + + public void setFeNodes(List feNodes) { + this.feNodes = feNodes; + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } + + public String getUsername() { + return username; + } + + public void setUsername(String username) { + this.username = username; + } + + public String getPassword() { + return password; + } + + public void setPassword(String password) { + this.password = password; + } + + public String getWriteMode() { + return writeMode; + } + + public void setWriteMode(String writeMode) { + this.writeMode = writeMode; + } + + public Integer getMaxRetries() { + return maxRetries; + } + + public void setMaxRetries(Integer maxRetries) { + this.maxRetries = maxRetries; + } + + public long getWaitRetryMills() { + return waitRetryMills; + } + + public void setWaitRetryMills(long waitRetryMills) { + this.waitRetryMills = waitRetryMills; + } + + public boolean isNameMapped() { + return nameMapped; + } + + public void setNameMapped(boolean nameMapped) { + this.nameMapped = nameMapped; + } + + public LoadConf getLoadConf() { + return loadConf; + } + + public void setLoadConf(LoadConf loadConf) { + this.loadConf = loadConf; + } + + public String[] getFieldNames() { + return fieldNames; + } + + public void setFieldNames(String[] fieldNames) { + this.fieldNames = fieldNames; + } + + public DataType[] getDataTypes() { + return dataTypes; + } + + public void setDataTypes(DataType[] dataTypes) { + this.dataTypes = dataTypes; + } + + public String getFilterStatement() { + return filterStatement; + } + + public void setFilterStatement(String filterStatement) { + this.filterStatement = filterStatement; + } + + public int getBeClientKeepLiveMin() { + return beClientKeepLiveMin; + } + + public void setBeClientKeepLiveMin(int beClientKeepLiveMin) { + this.beClientKeepLiveMin = beClientKeepLiveMin; + } + + public int getBeQueryTimeoutSecond() { + return beQueryTimeoutSecond; + } + + public void setBeQueryTimeoutSecond(int beQueryTimeoutSecond) { + this.beQueryTimeoutSecond = beQueryTimeoutSecond; + } + + public int getBeClientTimeout() { + return beClientTimeout; + } + + public void setBeClientTimeout(int beClientTimeout) { + this.beClientTimeout = beClientTimeout; + } + + public int getBeFetchRows() { + return beFetchRows; + } - public String getLoadUrl() { - return loadUrl; + public void setBeFetchRows(int beFetchRows) { + this.beFetchRows = beFetchRows; } - public void setLoadUrl(String loadUrl) { - this.loadUrl = loadUrl; + public long getBeFetchMaxBytes() { + return beFetchMaxBytes; } - public List getPrimaryKey() { - return primaryKey; + public void setBeFetchMaxBytes(long beFetchMaxBytes) { + this.beFetchMaxBytes = beFetchMaxBytes; } - public void setPrimaryKey(List primaryKey) { - this.primaryKey = primaryKey; + public Map getBeSocketProperties() { + return beSocketProperties; } - @Override - public String toString() { - return "StarRocksConf{" + "loadUrl='" + loadUrl + '\'' + ", primaryKey=" + primaryKey + '}'; + public void setBeSocketProperties(Map beSocketProperties) { + this.beSocketProperties = beSocketProperties; } } diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/dialect/StarRocksDialect.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/connection/StarRocksJdbcConnectionIProvider.java similarity index 65% rename from chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/dialect/StarRocksDialect.java rename to chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/connection/StarRocksJdbcConnectionIProvider.java index 0a591f1a3f..70d44bc80d 100644 --- a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/dialect/StarRocksDialect.java +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/connection/StarRocksJdbcConnectionIProvider.java @@ -7,7 +7,7 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -15,18 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package com.dtstack.chunjun.connector.starrocks.dialect; -import com.dtstack.chunjun.connector.mysql.dialect.MysqlDialect; +package com.dtstack.chunjun.connector.starrocks.connection; -/** - * @author lihongwei - * @date 2022/04/11 - */ -public class StarRocksDialect extends MysqlDialect { +import org.apache.flink.annotation.Internal; + +import java.sql.Connection; + +/** connection provider. */ +@Internal +public interface StarRocksJdbcConnectionIProvider { + + Connection getConnection() throws Exception; + + Connection reestablishConnection() throws Exception; - @Override - public String dialectName() { - return "StarRocks"; - } + void close(); } diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/connection/StarRocksJdbcConnectionOptions.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/connection/StarRocksJdbcConnectionOptions.java new file mode 100644 index 0000000000..d6829bbc7c --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/connection/StarRocksJdbcConnectionOptions.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.connection; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Optional; + +import static com.dtstack.chunjun.connector.starrocks.options.ConstantValue.CJ_DRIVER_CLASS_NAME; +import static com.dtstack.chunjun.connector.starrocks.options.ConstantValue.DRIVER_CLASS_NAME; + +/** JDBC connection options. */ +@PublicEvolving +public class StarRocksJdbcConnectionOptions implements Serializable { + + private static final long serialVersionUID = 1L; + + protected final String url; + @Nullable protected final String username; + @Nullable protected final String password; + + public StarRocksJdbcConnectionOptions(String url, String username, String password) { + this.url = Preconditions.checkNotNull(url, "jdbc url is empty"); + this.username = username; + this.password = password; + } + + public String getDbURL() { + return url; + } + + public String getCjDriverName() { + return CJ_DRIVER_CLASS_NAME; + } + + public String getDriverName() { + return DRIVER_CLASS_NAME; + } + + public Optional getUsername() { + return Optional.ofNullable(username); + } + + public Optional getPassword() { + return Optional.ofNullable(password); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/connection/StarRocksJdbcConnectionProvider.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/connection/StarRocksJdbcConnectionProvider.java new file mode 100644 index 0000000000..2ee2021256 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/connection/StarRocksJdbcConnectionProvider.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 com.dtstack.chunjun.connector.starrocks.connection; + +import com.dtstack.chunjun.util.ClassUtil; +import com.dtstack.chunjun.util.RetryUtil; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.Properties; + +/** Simple JDBC connection provider. */ +public class StarRocksJdbcConnectionProvider + implements StarRocksJdbcConnectionIProvider, Serializable { + + private static final Logger LOG = + LoggerFactory.getLogger(StarRocksJdbcConnectionProvider.class); + + private static final long serialVersionUID = 1L; + + private final StarRocksJdbcConnectionOptions jdbcOptions; + + private transient volatile Connection connection; + + public StarRocksJdbcConnectionProvider(StarRocksJdbcConnectionOptions jdbcOptions) { + this.jdbcOptions = jdbcOptions; + } + + @Override + public Connection getConnection() throws ClassNotFoundException { + if (connection == null) { + synchronized (this) { + if (connection == null) { + try { + Class.forName(jdbcOptions.getCjDriverName()); + } catch (ClassNotFoundException ex) { + Class.forName(jdbcOptions.getDriverName()); + } + Properties prop = new Properties(); + if (org.apache.commons.lang3.StringUtils.isNotBlank( + jdbcOptions.getUsername().orElse(null))) { + prop.put("user", jdbcOptions.getUsername().get()); + } + if (org.apache.commons.lang3.StringUtils.isNotBlank( + jdbcOptions.getPassword().orElse(null))) { + prop.put("password", jdbcOptions.getPassword().get()); + } + synchronized (ClassUtil.LOCK_STR) { + connection = + RetryUtil.executeWithRetry( + () -> + DriverManager.getConnection( + jdbcOptions.getDbURL(), prop), + 3, + 2000, + false); + } + } + } + } + return connection; + } + + public void checkValid() throws SQLException, ClassNotFoundException { + if (connection == null || !connection.isValid(10)) { + connection = null; + getConnection(); + } + } + + @Override + public Connection reestablishConnection() throws SQLException, ClassNotFoundException { + close(); + connection = getConnection(); + return connection; + } + + @Override + public void close() { + if (connection == null) { + return; + } + try { + connection.close(); + } catch (SQLException e) { + LOG.error("JDBC connection close failed.", e); + } finally { + connection = null; + } + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/converter/StarRocksColumnConverter.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/converter/StarRocksColumnConverter.java new file mode 100644 index 0000000000..c16c2eae61 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/converter/StarRocksColumnConverter.java @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.converter; + +import com.dtstack.chunjun.conf.ChunJunCommonConf; +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.starrocks.streamload.StarRocksSinkOP; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.IDeserializationConverter; +import com.dtstack.chunjun.converter.ISerializationConverter; +import com.dtstack.chunjun.element.AbstractBaseColumn; +import com.dtstack.chunjun.element.ColumnRowData; +import com.dtstack.chunjun.element.column.BigDecimalColumn; +import com.dtstack.chunjun.element.column.BooleanColumn; +import com.dtstack.chunjun.element.column.ByteColumn; +import com.dtstack.chunjun.element.column.SqlDateColumn; +import com.dtstack.chunjun.element.column.StringColumn; +import com.dtstack.chunjun.element.column.TimestampColumn; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; + +import org.apache.commons.lang3.StringUtils; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.time.format.DateTimeFormatter; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static com.dtstack.chunjun.connector.starrocks.util.StarRocksUtil.addStrForNum; + +/** @author liuliu 2022/7/12 */ +public class StarRocksColumnConverter + extends AbstractRowConverter, LogicalType> { + + private final List columnList; + public static final String DATETIME_FORMAT_SHORT = "yyyy-MM-dd HH:mm:ss"; + + public StarRocksColumnConverter(RowType rowType, ChunJunCommonConf conf) { + super(rowType, conf); + this.columnList = + conf.getColumn().stream().map(FieldConf::getName).collect(Collectors.toList()); + for (int i = 0; i < rowType.getFieldCount(); i++) { + toInternalConverters.add( + wrapIntoNullableInternalConverter( + createInternalConverter(rowType.getTypeAt(i)))); + toExternalConverters.add( + wrapIntoNullableExternalConverter( + createExternalConverter(fieldTypes[i]), fieldTypes[i])); + } + } + + @Override + public RowData toInternal(Object[] input) throws Exception { + ColumnRowData columnRowData = new ColumnRowData(rowType.getFieldCount()); + int index = 0; + for (int pos = 0; pos < rowType.getFieldCount(); pos++) { + FieldConf fieldConf = commonConf.getColumn().get(pos); + AbstractBaseColumn val = null; + if (StringUtils.isBlank(fieldConf.getValue())) { + val = + (AbstractBaseColumn) + toInternalConverters.get(pos).deserialize(input[index++]); + } + columnRowData.addField(assembleFieldProps(fieldConf, val)); + } + return columnRowData; + } + + @Override + public RowData toInternalLookup(Object[] input) throws Exception { + return toInternal(input); + } + + @Override + protected ISerializationConverter> wrapIntoNullableExternalConverter( + ISerializationConverter> ISerializationConverter, + LogicalType type) { + return (rowData, index, output) -> { + if (rowData == null + || rowData.isNullAt(index) + || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) { + output.put(columnList.get(index), null); + } else { + ISerializationConverter.serialize(rowData, index, output); + } + }; + } + + @Override + public Map toExternal(RowData rowData, Map output) + throws Exception { + for (int index = 0; index < fieldTypes.length; index++) { + toExternalConverters.get(index).serialize(rowData, index, output); + } + output.put( + StarRocksSinkOP.COLUMN_KEY, StarRocksSinkOP.parse(rowData.getRowKind()).ordinal()); + return output; + } + + @Override + protected IDeserializationConverter createInternalConverter( + LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return val -> null; + case BOOLEAN: + return val -> new BooleanColumn((boolean) val); + case TINYINT: + return val -> new ByteColumn((byte) val); + case SMALLINT: + return val -> new BigDecimalColumn((short) val); + case INTEGER: + return val -> new BigDecimalColumn((int) val); + case BIGINT: + return val -> new BigDecimalColumn((long) val); + case FLOAT: + return val -> new BigDecimalColumn((float) val); + case DOUBLE: + return val -> new BigDecimalColumn((double) val); + case DECIMAL: + return val -> new BigDecimalColumn((BigDecimal) val); + case CHAR: + case VARCHAR: + return val -> new StringColumn((String) val); + case DATE: + return val -> new SqlDateColumn(Date.valueOf((String) val)); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + final int timestampPrecision = ((TimestampType) type).getPrecision(); + return val -> + new TimestampColumn(Timestamp.valueOf((String) val), timestampPrecision); + default: + throw new UnsupportedOperationException("Unsupported type:" + type); + } + } + + @Override + protected ISerializationConverter> createExternalConverter( + LogicalType type) { + switch (type.getTypeRoot()) { + case BOOLEAN: + return (rowData, index, map) -> + map.put( + columnList.get(index), + ((ColumnRowData) rowData).getField(index).asBoolean() ? 1 : 0); + case TINYINT: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getByte(index)); + case SMALLINT: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getShort(index)); + case INTEGER: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getInt(index)); + case BIGINT: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getLong(index)); + case FLOAT: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getFloat(index)); + case DOUBLE: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getDouble(index)); + case DECIMAL: + return (rowData, index, map) -> + map.put( + columnList.get(index), + ((ColumnRowData) rowData).getField(index).asBigDecimal()); + case CHAR: + case VARCHAR: + return (rowData, index, map) -> + map.put( + columnList.get(index), + ((ColumnRowData) rowData).getField(index).asString()); + case DATE: + return (rowData, index, map) -> + map.put( + columnList.get(index), + ((ColumnRowData) rowData).getField(index).asSqlDate().toString()); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + final int timestampPrecision = ((TimestampType) type).getPrecision(); + final String formatStr; + if (timestampPrecision > 0) { + formatStr = + addStrForNum( + DATETIME_FORMAT_SHORT + ".", + DATETIME_FORMAT_SHORT.length() + 1 + timestampPrecision, + "S"); + } else { + formatStr = DATETIME_FORMAT_SHORT; + } + return (rowData, index, map) -> { + DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(formatStr); + map.put( + columnList.get(index), + dateTimeFormatter.format( + ((ColumnRowData) rowData) + .getField(index) + .asTimestamp() + .toLocalDateTime())); + }; + default: + throw new UnsupportedOperationException("Unsupported type:" + type); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/converter/StarRocksRawTypeConverter.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/converter/StarRocksRawTypeConverter.java index bb17e5e4a5..8d759180ab 100644 --- a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/converter/StarRocksRawTypeConverter.java +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/converter/StarRocksRawTypeConverter.java @@ -17,13 +17,78 @@ */ package com.dtstack.chunjun.connector.starrocks.converter; -import com.dtstack.chunjun.connector.mysql.converter.MysqlRawTypeConverter; +import com.dtstack.chunjun.constants.ConstantValue; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.DataType; + +import org.apache.commons.lang3.StringUtils; + +import java.util.Locale; /** * @author lihongwei * @date 2022/04/11 */ -public class StarRocksRawTypeConverter extends MysqlRawTypeConverter { +public class StarRocksRawTypeConverter { - /** 目前暂时延用mysql, 可扩展StarRocks自有数据类型 */ + public static DataType apply(String type) { + type = type.toUpperCase(Locale.ENGLISH); + int left = type.indexOf(ConstantValue.LEFT_PARENTHESIS_SYMBOL); + int right = type.indexOf(ConstantValue.RIGHT_PARENTHESIS_SYMBOL); + String leftStr = type; + String rightStr = null; + if (left > 0 && right > 0) { + leftStr = type.substring(0, left); + rightStr = type.substring(left + 1, type.length() - 1); + } + switch (leftStr) { + case "BOOLEAN": + return DataTypes.BOOLEAN(); + case "TINYINT": + if (StringUtils.isNotBlank(rightStr) && rightStr.equals("1")) { + return DataTypes.BOOLEAN(); + } else { + return DataTypes.TINYINT(); + } + case "SMALLINT": + return DataTypes.SMALLINT(); + case "INT": + return DataTypes.INT(); + case "FLOAT": + return DataTypes.FLOAT(); + case "DOUBLE": + return DataTypes.DOUBLE(); + case "BIGINT": + return DataTypes.BIGINT(); + case "DECIMAL": + case "DECIMALV2": + case "DECIMAL32": + case "DECIMAL64": + case "DECIMAL128": + if (rightStr != null) { + String[] split = rightStr.split(ConstantValue.COMMA_SYMBOL); + if (split.length == 2) { + return DataTypes.DECIMAL( + Integer.parseInt(split[0].trim()), + Integer.parseInt(split[1].trim())); + } + } + return DataTypes.DECIMAL(38, 18); + case "CHAR": + case "VARCHAR": + case "STRING": + case "LARGEINT": + return DataTypes.STRING(); + case "DATE": + return DataTypes.DATE(); + case "DATETIME": + return DataTypes.TIMESTAMP(0); + case "HLL": + case "BITMAP": + case "JSON": + default: + throw new UnsupportedOperationException(type); + } + } } diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/converter/StarRocksRowConverter.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/converter/StarRocksRowConverter.java new file mode 100644 index 0000000000..2201ca4ff0 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/converter/StarRocksRowConverter.java @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.converter; + +import com.dtstack.chunjun.connector.starrocks.streamload.StarRocksSinkOP; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.IDeserializationConverter; +import com.dtstack.chunjun.converter.ISerializationConverter; + +import org.apache.flink.table.data.DecimalData; +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.data.TimestampData; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; + +import java.math.BigDecimal; +import java.sql.Date; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.util.List; +import java.util.Map; + +import static com.dtstack.chunjun.connector.starrocks.util.StarRocksUtil.addStrForNum; + +/** @author liuliu 2022/7/12 */ +public class StarRocksRowConverter + extends AbstractRowConverter, LogicalType> { + + private final List columnList; + public static final String DATETIME_FORMAT_SHORT = "yyyy-MM-dd HH:mm:ss"; + + public StarRocksRowConverter(RowType rowType, List columnList) { + super(rowType); + this.columnList = columnList; + for (int i = 0; i < rowType.getFieldCount(); i++) { + toInternalConverters.add( + wrapIntoNullableInternalConverter( + createInternalConverter(rowType.getTypeAt(i)))); + toExternalConverters.add( + wrapIntoNullableExternalConverter( + createExternalConverter(fieldTypes[i]), fieldTypes[i])); + } + } + + @Override + public RowData toInternal(Object[] input) throws Exception { + GenericRowData genericRowData = new GenericRowData(rowType.getFieldCount()); + for (int pos = 0; pos < rowType.getFieldCount(); pos++) { + Object field = input[pos]; + genericRowData.setField(pos, toInternalConverters.get(pos).deserialize(field)); + } + return genericRowData; + } + + @Override + public RowData toInternalLookup(Object[] input) throws Exception { + return toInternal(input); + } + + @Override + protected ISerializationConverter> wrapIntoNullableExternalConverter( + ISerializationConverter> ISerializationConverter, + LogicalType type) { + return (rowData, index, output) -> { + if (rowData == null + || rowData.isNullAt(index) + || LogicalTypeRoot.NULL.equals(type.getTypeRoot())) { + output.put(columnList.get(index), null); + } else { + ISerializationConverter.serialize(rowData, index, output); + } + }; + } + + @Override + public Map toExternal(RowData rowData, Map output) + throws Exception { + for (int index = 0; index < fieldTypes.length; index++) { + toExternalConverters.get(index).serialize(rowData, index, output); + } + output.put( + StarRocksSinkOP.COLUMN_KEY, StarRocksSinkOP.parse(rowData.getRowKind()).ordinal()); + return output; + } + + @Override + protected IDeserializationConverter createInternalConverter(LogicalType type) { + switch (type.getTypeRoot()) { + case NULL: + return val -> null; + case BOOLEAN: + return val -> (boolean) val; + case TINYINT: + return val -> (byte) val; + case SMALLINT: + return val -> (short) val; + case INTEGER: + return val -> (int) val; + case BIGINT: + return val -> (long) val; + case FLOAT: + return val -> (float) val; + case DOUBLE: + return val -> (double) val; + case DECIMAL: + final int precision = ((DecimalType) type).getPrecision(); + final int scale = ((DecimalType) type).getScale(); + // using decimal(20, 0) to support db type bigint unsigned, user should define + // decimal(20, 0) in SQL, + // but other precision like decimal(30, 0) can work too from lenient consideration. + return val -> DecimalData.fromBigDecimal((BigDecimal) val, precision, scale); + case CHAR: + case VARCHAR: + return val -> StringData.fromString((String) val); + case DATE: + return val -> { + String value = (String) val; + Date date = Date.valueOf(value); + return (int) date.toLocalDate().toEpochDay(); + }; + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + final int timestampPrecision = ((TimestampType) type).getPrecision(); + final int timestampLen = + timestampPrecision == 0 + ? DATETIME_FORMAT_SHORT.length() + : DATETIME_FORMAT_SHORT.length() + 1 + timestampPrecision; + String formatStr = + timestampLen == DATETIME_FORMAT_SHORT.length() + ? DATETIME_FORMAT_SHORT + : addStrForNum(DATETIME_FORMAT_SHORT + ".", timestampLen, "S"); + return val -> { + DateTimeFormatter df = DateTimeFormatter.ofPattern(formatStr); + String value = (String) val; + if (value.length() < DATETIME_FORMAT_SHORT.length()) { + throw new IllegalArgumentException( + "Date value length shorter than DATETIME_FORMAT_SHORT"); + } + if (value.length() == timestampLen) { + return TimestampData.fromLocalDateTime(LocalDateTime.parse(value, df)); + } else if (value.length() > timestampLen) { + return TimestampData.fromLocalDateTime( + LocalDateTime.parse(value.substring(0, timestampLen), df)); + } else { + if (value.length() == DATETIME_FORMAT_SHORT.length()) { + return LocalDateTime.parse( + addStrForNum(value + ".", timestampLen, "0")); + } else { + return LocalDateTime.parse(addStrForNum(value, timestampLen, "0")); + } + } + }; + default: + throw new UnsupportedOperationException("Unsupported type:" + type); + } + } + + @Override + protected ISerializationConverter> createExternalConverter( + LogicalType type) { + switch (type.getTypeRoot()) { + case BOOLEAN: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getBoolean(index) ? 1 : 0); + case TINYINT: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getByte(index)); + case SMALLINT: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getShort(index)); + case INTEGER: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getInt(index)); + case BIGINT: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getLong(index)); + case FLOAT: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getFloat(index)); + case DOUBLE: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getDouble(index)); + case DECIMAL: // for both largeint and decimal + final int decimalPrecision = ((DecimalType) type).getPrecision(); + final int decimalScale = ((DecimalType) type).getScale(); + return (rowData, index, map) -> + map.put( + columnList.get(index), + rowData.getDecimal(index, decimalPrecision, decimalScale) + .toBigDecimal()); + case CHAR: + case VARCHAR: + return (rowData, index, map) -> + map.put(columnList.get(index), rowData.getString(index).toString()); + case DATE: + return (rowData, index, map) -> + map.put( + columnList.get(index), + Date.valueOf(LocalDate.ofEpochDay(rowData.getInt(index))) + .toString()); + case TIMESTAMP_WITHOUT_TIME_ZONE: + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + final int timestampPrecision = ((TimestampType) type).getPrecision(); + return (rowData, index, map) -> + map.put( + columnList.get(index), + rowData.getTimestamp(index, timestampPrecision) + .toLocalDateTime() + .toString()); + case BINARY: + return (rowData, index, map) -> { + byte[] bts = rowData.getBinary(index); + long value = 0; + for (int i = 0; i < bts.length; i++) { + value += (bts[bts.length - i - 1] & 0xffL) << (8 * i); + } + map.put(columnList.get(index), value); + }; + default: + throw new UnsupportedOperationException("Unsupported type:" + type); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/ConstantValue.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/ConstantValue.java new file mode 100644 index 0000000000..bb3d84bdf2 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/ConstantValue.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.options; + +/** @author liuliu 2022/7/12 */ +public class ConstantValue { + + // sink + public static final String FIELD_DELIMITER = "\t"; + public static String LINE_DELIMITER = "\n"; + + // driver class name + public static final String DRIVER_CLASS_NAME = "com.mysql.jdbc.Driver"; + public static final String CJ_DRIVER_CLASS_NAME = "com.mysql.cj.jdbc.Driver"; + + public static final Integer MAX_RETRIES_DEFAULT = 3; + public static final String WRITE_MODE_DEFAULT = "APPEND"; + public static final Integer BATCH_SIZE_DEFAULT = 1024 * 10; + + // stream load + public static final Integer HTTP_CHECK_TIMEOUT_DEFAULT = 10 * 1000; + public static final Integer QUEUE_OFFER_TIMEOUT_DEFAULT = 60 * 1000; + public static final Integer QUEUE_POLL_TIMEOUT_DEFAULT = 60 * 1000; + public static final Long SINK_BATCH_MAX_BYTES_DEFAULT = 2 * 1024 * 1024 * 1024L; + public static final Long SINK_BATCH_MAX_ROWS_DEFAULT = 2048 * 100L; +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StarRocksCommonOptions.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StarRocksCommonOptions.java new file mode 100644 index 0000000000..deb19aec5d --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StarRocksCommonOptions.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 com.dtstack.chunjun.connector.starrocks.options; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import java.util.List; + +/** @author liuliu 2022/7/12 */ +public class StarRocksCommonOptions { + public static final ConfigOption> FENODES = + ConfigOptions.key("fe-nodes") + .stringType() + .asList() + .noDefaultValue() + .withDescription("StarRocks FE hostname and restful port"); + + public static final ConfigOption URL = + ConfigOptions.key("url") + .stringType() + .noDefaultValue() + .withDescription("the jdbc url of starRocks."); + + public static final ConfigOption SCHEMA_NAME = + ConfigOptions.key("schema-name") + .stringType() + .noDefaultValue() + .withDescription("the schema name of starRocks."); + + public static final ConfigOption TABLE_NAME = + ConfigOptions.key("table-name") + .stringType() + .noDefaultValue() + .withDescription("the table name of starRocks."); + + public static final ConfigOption USERNAME = + ConfigOptions.key("username") + .stringType() + .noDefaultValue() + .withDescription("StarRocks username"); + + public static final ConfigOption PASSWORD = + ConfigOptions.key("password") + .stringType() + .noDefaultValue() + .withDescription("StarRocks password"); + + public static final ConfigOption MAX_RETRIES = + ConfigOptions.key("max-retries") + .intType() + .defaultValue(ConstantValue.MAX_RETRIES_DEFAULT) + .withDescription("request retry time"); +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StarRocksSinkOptions.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StarRocksSinkOptions.java new file mode 100644 index 0000000000..e222eb41e5 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StarRocksSinkOptions.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 com.dtstack.chunjun.connector.starrocks.options; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +/** @author liuliu 2022/7/12 */ +public class StarRocksSinkOptions { + + public static final ConfigOption SINK_SEMANTIC = + ConfigOptions.key("semantic") + .stringType() + .defaultValue(ConstantValue.WRITE_MODE_DEFAULT) + .withDescription("exactly once/at least once"); + + public static final ConfigOption NAME_MAPPED = + ConfigOptions.key("name-mapped").booleanType().defaultValue(false).withDescription(""); + + public static final ConfigOption SINK_BUFFER_FLUSH_MAX_ROWS = + ConfigOptions.key("sink.buffer-flush.max-rows") + .intType() + .defaultValue(ConstantValue.BATCH_SIZE_DEFAULT) + .withDescription(""); +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StarRocksSourceOptions.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StarRocksSourceOptions.java new file mode 100644 index 0000000000..0b538aa068 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StarRocksSourceOptions.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.options; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import java.util.HashMap; +import java.util.Map; + +/** @author liuliu 2022/7/12 */ +public class StarRocksSourceOptions { + + public static final ConfigOption FILTER_STATEMENT = + ConfigOptions.key("filter-statement").stringType().defaultValue("").withDescription(""); + + public static final ConfigOption SCAN_BE_CLIENT_TIMEOUT = + ConfigOptions.key("scan.be.client.timeout") + .intType() + .defaultValue(3000) + .withDescription(""); + + public static final ConfigOption SCAN_BE_CLIENT_KEEP_LIVE_MIN = + ConfigOptions.key("scan.be.client.keep-live-min") + .intType() + .defaultValue(10) + .withDescription(""); + + public static final ConfigOption SCAN_BE_QUERY_TIMEOUT_S = + ConfigOptions.key("scan.be.query.timeout-s") + .intType() + .defaultValue(600) + .withDescription(""); + + public static final ConfigOption SCAN_BE_FETCH_ROWS = + ConfigOptions.key("scan.be.fetch-rows") + .intType() + .defaultValue(1024) + .withDescription(""); + + public static final ConfigOption SCAN_BE_FETCH_BYTES_LIMIT = + ConfigOptions.key("scan.be.fetch-bytes-limit") + .longType() + .defaultValue(1024 * 1024 * 1014L) + .withDescription(""); + + public static final ConfigOption> SCAN_BE_PARAM_PROPERTIES = + ConfigOptions.key("scan.be.param.properties") + .mapType() + .defaultValue(new HashMap<>()) + .withDescription(""); +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StreamLoadOptions.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StreamLoadOptions.java new file mode 100644 index 0000000000..3aad5e78af --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/options/StreamLoadOptions.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.options; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import java.util.HashMap; +import java.util.Map; + +/** @author liuliu 2022/7/12 */ +public class StreamLoadOptions { + + public static final ConfigOption HTTP_CHECK_TIMEOUT = + ConfigOptions.key("http.check.timeout") + .intType() + .defaultValue(ConstantValue.HTTP_CHECK_TIMEOUT_DEFAULT) + .withDescription(""); + + public static final ConfigOption QUEUE_OFFER_TIMEOUT = + ConfigOptions.key("queue.offer.timeout") + .intType() + .defaultValue(ConstantValue.QUEUE_OFFER_TIMEOUT_DEFAULT) + .withDescription(""); + + public static final ConfigOption QUEUE_POLL_TIMEOUT = + ConfigOptions.key("queue.poll.timeout") + .intType() + .defaultValue(ConstantValue.QUEUE_POLL_TIMEOUT_DEFAULT) + .withDescription(""); + + public static final ConfigOption SINK_BATCH_MAX_BYTES = + ConfigOptions.key("sink.batch.max-bytes") + .longType() + .defaultValue(ConstantValue.SINK_BATCH_MAX_BYTES_DEFAULT) + .withDescription(""); + + public static final ConfigOption SINK_BATCH_MAX_ROWS = + ConfigOptions.key("sink.batch.max-rows") + .longType() + .defaultValue(ConstantValue.SINK_BATCH_MAX_ROWS_DEFAULT) + .withDescription(""); + + public static final ConfigOption> STREAM_LOAD_HEAD_PROPERTIES = + ConfigOptions.key("stream-load.head.properties") + .mapType() + .defaultValue(new HashMap<>()) + .withDescription(""); +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/MappedWriteProcessor.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/MappedWriteProcessor.java new file mode 100644 index 0000000000..c8a4f8d187 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/MappedWriteProcessor.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.sink; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.streamload.StreamLoadManager; +import com.dtstack.chunjun.element.ColumnRowData; + +import org.apache.flink.table.data.RowData; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** @author liuliu 2022/7/28 */ +public class MappedWriteProcessor extends StarRocksWriteProcessor { + private final Set metaHeader = + Stream.of("schema", "table", "type", "opTime", "ts", "scn") + .collect(Collectors.toCollection(HashSet::new)); + + public MappedWriteProcessor(StreamLoadManager streamLoadManager, StarRocksConf starRocksConf) { + super(streamLoadManager, starRocksConf); + } + + @Override + public void write(List rowDataList) throws Exception { + Map>> identifyXValueMap = new HashMap<>(); + for (RowData data : rowDataList) { + ColumnRowData rowData = (ColumnRowData) data; + Map headerInfo = rowData.getHeaderInfo(); + String schema = starRocksConf.getDatabase(); + String table = starRocksConf.getTable(); + Map value = new HashMap<>(); + for (Map.Entry entry : headerInfo.entrySet()) { + String key = entry.getKey(); + int index = entry.getValue(); + switch (key) { + case "schema": + schema = rowData.getField(index).asString(); + case "table": + table = rowData.getField(index).asString(); + default: + if (!metaHeader.contains(key)) { + value.put(key, rowData.getField(index).asString()); + } + } + } + String identify = String.format("%s.%s", schema, table); + List> valueList = + identifyXValueMap.computeIfAbsent(identify, key -> new ArrayList<>()); + valueList.add(value); + } + for (Map.Entry>> entry : identifyXValueMap.entrySet()) { + streamLoadManager.write(entry.getKey(), null, entry.getValue()); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/NormalWriteProcessor.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/NormalWriteProcessor.java new file mode 100644 index 0000000000..112dc341b5 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/NormalWriteProcessor.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.sink; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.streamload.StreamLoadManager; +import com.dtstack.chunjun.converter.AbstractRowConverter; + +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** @author liuliu 2022/7/28 */ +public class NormalWriteProcessor extends StarRocksWriteProcessor { + + private final AbstractRowConverter, LogicalType> + rowConverter; + private final List columnNameList; + + public NormalWriteProcessor( + AbstractRowConverter, LogicalType> converter, + StreamLoadManager streamLoadManager, + StarRocksConf starRocksConf, + List columnNameList) { + super(streamLoadManager, starRocksConf); + this.rowConverter = converter; + this.columnNameList = columnNameList; + } + + @Override + public void write(List rowDataList) throws Exception { + String identify = + String.format("%s.%s", starRocksConf.getDatabase(), starRocksConf.getTable()); + List> values = new ArrayList<>(rowDataList.size()); + for (RowData rowData : rowDataList) { + values.add(rowConverter.toExternal(rowData, new HashMap<>(columnNameList.size()))); + } + streamLoadManager.write(identify, columnNameList, values); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksDynamicTableSink.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksDynamicTableSink.java new file mode 100644 index 0000000000..02e0d47018 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksDynamicTableSink.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.sink; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.converter.StarRocksRowConverter; +import com.dtstack.chunjun.sink.DtOutputFormatSinkFunction; + +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.SinkFunctionProvider; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.util.Preconditions.checkState; + +/** @author liuliu 2022/7/12 */ +public class StarRocksDynamicTableSink implements DynamicTableSink { + private final StarRocksConf sinkConf; + private final TableSchema physicalSchema; + + public StarRocksDynamicTableSink(StarRocksConf sinkConf, TableSchema physicalSchema) { + this.sinkConf = sinkConf; + this.physicalSchema = physicalSchema; + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + validatePrimaryKey(requestedMode); + return ChangelogMode.newBuilder() + .addContainedKind(RowKind.INSERT) + .addContainedKind(RowKind.DELETE) + .addContainedKind(RowKind.UPDATE_BEFORE) + .addContainedKind(RowKind.UPDATE_AFTER) + .build(); + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + final RowType rowType = (RowType) physicalSchema.toRowDataType().getLogicalType(); + StarRocksOutputFormatBuilder builder = + new StarRocksOutputFormatBuilder(new StarRocksOutputFormat()); + + sinkConf.setColumn(getFieldConfFromSchema()); + builder.setRowConverter( + new StarRocksRowConverter(rowType, Arrays.asList(physicalSchema.getFieldNames()))); + builder.setStarRocksConf(sinkConf); + + return SinkFunctionProvider.of( + new DtOutputFormatSinkFunction<>(builder.finish()), sinkConf.getParallelism()); + } + + @Override + public DynamicTableSink copy() { + return new StarRocksDynamicTableSink(sinkConf, physicalSchema); + } + + @Override + public String asSummaryString() { + return "StarRocks Sink"; + } + + private List getFieldConfFromSchema() { + + return physicalSchema.getTableColumns().stream() + .map( + tableColumn -> { + FieldConf fieldConf = new FieldConf(); + fieldConf.setName(tableColumn.getName()); + fieldConf.setType(tableColumn.getType().getConversionClass().getName()); + return fieldConf; + }) + .collect(Collectors.toList()); + } + + private void validatePrimaryKey(ChangelogMode requestedMode) { + checkState( + ChangelogMode.insertOnly().equals(requestedMode) + || physicalSchema.getPrimaryKey().isPresent(), + "please declare primary key for sink table when query contains update/delete record."); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksOutputFormat.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksOutputFormat.java new file mode 100644 index 0000000000..cbfc8ecc22 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksOutputFormat.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.sink; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.streamload.StarRocksSinkBufferEntity; +import com.dtstack.chunjun.connector.starrocks.streamload.StarRocksStreamLoadFailedException; +import com.dtstack.chunjun.connector.starrocks.streamload.StreamLoadManager; +import com.dtstack.chunjun.sink.format.BaseRichOutputFormat; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; + +import org.apache.flink.table.data.RowData; + +import com.alibaba.fastjson.JSON; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** @author liuliu 2022/7/12 */ +public class StarRocksOutputFormat extends BaseRichOutputFormat { + + StreamLoadManager streamLoadManager; + private StarRocksConf starRocksConf; + private StarRocksWriteProcessor writeProcessor; + + @Override + protected void openInternal(int taskNumber, int numTasks) throws IOException { + List columnNameList = + starRocksConf.getColumn().stream() + .map(FieldConf::getName) + .collect(Collectors.toList()); + + streamLoadManager = new StreamLoadManager(starRocksConf); + streamLoadManager.startAsyncFlushing(); + if (starRocksConf.isNameMapped()) { + writeProcessor = new MappedWriteProcessor(streamLoadManager, starRocksConf); + } else { + writeProcessor = + new NormalWriteProcessor( + rowConverter, streamLoadManager, starRocksConf, columnNameList); + } + } + + @Override + protected void writeSingleRecordInternal(RowData rowData) { + // do nothing + } + + @Override + protected void writeMultipleRecordsInternal() throws Exception { + writeProcessor.write(rows); + if (rows.size() != batchSize) { + streamLoadManager.flush(null, false); + } + } + + @Override + public synchronized void writeRecord(RowData rowData) { + checkTimerWriteException(); + int size = 0; + rows.add(rowData); + if (rows.size() >= batchSize) { + writeRecordInternal(); + size = batchSize; + } + + updateDuration(); + bytesWriteCounter.add(rowSizeCalculator.getObjectSize(rowData)); + if (checkpointEnabled) { + snapshotWriteCounter.add(size); + } + } + + @Override + protected synchronized void writeRecordInternal() { + if (flushEnable.get()) { + try { + writeMultipleRecordsInternal(); + numWriteCounter.add(rows.size()); + } catch (Exception e) { + if (e instanceof StarRocksStreamLoadFailedException) { + StarRocksStreamLoadFailedException exception = + (StarRocksStreamLoadFailedException) e; + String errMessage = handleErrMessage(exception); + StarRocksSinkBufferEntity entity = exception.getEntity(); + for (byte[] data : entity.getBuffer()) { + dirtyManager.collect(new String(data), new Throwable(errMessage), null); + } + } else { + throw new ChunJunRuntimeException("write starRocks failed.", e); + } + } finally { + // Data is either recorded dirty data or written normally + rows.clear(); + } + } + } + + public String handleErrMessage(StarRocksStreamLoadFailedException e) { + String message = e.getMessage(); + Map failedResponse = e.getFailedResponse(); + return String.format( + "write to starRocks failed.\n errMsg:%s\n failedResponse:%s", + message, JSON.toJSONString(failedResponse)); + } + + @Override + protected void closeInternal() { + if (streamLoadManager != null) { + streamLoadManager.close(); + } + } + + public void setStarRocksConf(StarRocksConf starRocksConf) { + this.starRocksConf = starRocksConf; + } + + public StarRocksConf getStarRocksConf() { + return starRocksConf; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksOutputFormatBuilder.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksOutputFormatBuilder.java new file mode 100644 index 0000000000..8585797ffe --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksOutputFormatBuilder.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.sink; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.sink.format.BaseRichOutputFormatBuilder; + +import com.google.common.base.Preconditions; + +/** @author liuliu 2022/7/12 */ +public class StarRocksOutputFormatBuilder + extends BaseRichOutputFormatBuilder { + + public StarRocksOutputFormatBuilder(StarRocksOutputFormat format) { + super(format); + } + + public void setStarRocksConf(StarRocksConf starRocksConf) { + super.setConfig(starRocksConf); + format.setStarRocksConf(starRocksConf); + } + + @Override + protected void checkFormat() { + StarRocksConf conf = format.getStarRocksConf(); + Preconditions.checkNotNull(conf.getUrl(), "starRocks url is required"); + Preconditions.checkNotNull(conf.getFeNodes(), "starRocks feNodes is required"); + if (!conf.isNameMapped()) { + Preconditions.checkNotNull( + conf.getDatabase(), + "starRocks database is required when nameMapped is not enable"); + Preconditions.checkNotNull( + conf.getTable(), "starRocks table is required when nameMapped is not enable"); + } + Preconditions.checkNotNull(conf.getUsername(), "starRocks username is required"); + Preconditions.checkNotNull(conf.getPassword(), "starRocks password is required"); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksSinkFactory.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksSinkFactory.java new file mode 100644 index 0000000000..615394d2df --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksSinkFactory.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package com.dtstack.chunjun.connector.starrocks.sink; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.conf.SyncConf; +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.converter.StarRocksColumnConverter; +import com.dtstack.chunjun.connector.starrocks.converter.StarRocksRawTypeConverter; +import com.dtstack.chunjun.connector.starrocks.converter.StarRocksRowConverter; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.RawTypeConverter; +import com.dtstack.chunjun.sink.SinkFactory; +import com.dtstack.chunjun.util.JsonUtil; +import com.dtstack.chunjun.util.TableUtil; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; + +import java.util.stream.Collectors; + +/** + * @author lihongwei + * @date 2022/04/11 + */ +public class StarRocksSinkFactory extends SinkFactory { + + private final StarRocksConf starRocksConf; + + public StarRocksSinkFactory(SyncConf syncConf) { + super(syncConf); + starRocksConf = + JsonUtil.toObject( + JsonUtil.toJson(syncConf.getWriter().getParameter()), StarRocksConf.class); + + int batchSize = syncConf.getWriter().getIntVal("batchSize", 10240); + starRocksConf.setBatchSize(batchSize); + super.initCommonConf(starRocksConf); + } + + @Override + public RawTypeConverter getRawTypeConverter() { + return StarRocksRawTypeConverter::apply; + } + + @Override + public DataStreamSink createSink(DataStream dataSet) { + StarRocksOutputFormatBuilder builder = + new StarRocksOutputFormatBuilder(new StarRocksOutputFormat()); + builder.setStarRocksConf(starRocksConf); + RowType rowType = TableUtil.createRowType(starRocksConf.getColumn(), getRawTypeConverter()); + AbstractRowConverter rowConverter; + if (useAbstractBaseColumn) { + rowConverter = new StarRocksColumnConverter(rowType, starRocksConf); + } else { + rowConverter = + new StarRocksRowConverter( + rowType, + starRocksConf.getColumn().stream() + .map(FieldConf::getName) + .collect(Collectors.toList())); + } + builder.setRowConverter(rowConverter, useAbstractBaseColumn); + return createOutput(dataSet, builder.finish()); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksWriteProcessor.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksWriteProcessor.java new file mode 100644 index 0000000000..f4686eb829 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarRocksWriteProcessor.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 com.dtstack.chunjun.connector.starrocks.sink; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.streamload.StreamLoadManager; + +import org.apache.flink.table.data.RowData; + +import java.util.List; + +/** @author liuliu 2022/7/28 */ +public abstract class StarRocksWriteProcessor { + + protected final StreamLoadManager streamLoadManager; + protected final StarRocksConf starRocksConf; + + public StarRocksWriteProcessor( + StreamLoadManager streamLoadManager, StarRocksConf starRocksConf) { + this.streamLoadManager = streamLoadManager; + this.starRocksConf = starRocksConf; + } + + public abstract void write(List rowDataList) throws Exception; +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarrocksSinkFactory.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarrocksSinkFactory.java deleted file mode 100644 index 92e9603191..0000000000 --- a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/sink/StarrocksSinkFactory.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package com.dtstack.chunjun.connector.starrocks.sink; - -import com.dtstack.chunjun.conf.FieldConf; -import com.dtstack.chunjun.conf.SyncConf; -import com.dtstack.chunjun.connector.jdbc.adapter.ConnectionAdapter; -import com.dtstack.chunjun.connector.jdbc.conf.ConnectionConf; -import com.dtstack.chunjun.connector.jdbc.exclusion.FieldNameExclusionStrategy; -import com.dtstack.chunjun.connector.jdbc.sink.JdbcOutputFormatBuilder; -import com.dtstack.chunjun.connector.jdbc.sink.JdbcSinkFactory; -import com.dtstack.chunjun.connector.jdbc.util.JdbcUtil; -import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; -import com.dtstack.chunjun.connector.starrocks.converter.StarRocksRawTypeConverter; -import com.dtstack.chunjun.connector.starrocks.dialect.StarRocksDialect; -import com.dtstack.chunjun.converter.AbstractRowConverter; -import com.dtstack.chunjun.converter.RawTypeConverter; -import com.dtstack.chunjun.util.GsonUtil; -import com.dtstack.chunjun.util.TableUtil; - -import org.apache.flink.api.common.io.OutputFormat; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSink; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.util.Preconditions; - -import com.google.gson.Gson; -import com.google.gson.GsonBuilder; -import com.starrocks.connector.flink.row.StarRocksTableRowTransformer; -import com.starrocks.connector.flink.table.StarRocksDynamicSinkFunction; -import com.starrocks.connector.flink.table.StarRocksSinkOptions; -import org.apache.commons.lang3.StringUtils; - -import java.util.Collections; -import java.util.List; - -/** - * @author lihongwei - * @date 2022/04/11 - */ -public class StarrocksSinkFactory extends JdbcSinkFactory { - - private final StarRocksConf starRocksConf; - protected TypeInformation typeInformation; - - public StarrocksSinkFactory(SyncConf syncConf) { - super(syncConf, new StarRocksDialect()); - JdbcUtil.putExtParam(jdbcConf); - Gson gson = - new GsonBuilder() - .registerTypeAdapter( - ConnectionConf.class, new ConnectionAdapter("SinkConnectionConf")) - .addDeserializationExclusionStrategy( - new FieldNameExclusionStrategy("column")) - .create(); - GsonUtil.setTypeAdapter(gson); - starRocksConf = - gson.fromJson( - gson.toJson(syncConf.getWriter().getParameter()), StarRocksConf.class); - } - - @Override - protected DataStreamSink createOutput( - DataStream dataSet, OutputFormat outputFormat) { - return createOutput(dataSet, outputFormat, this.getClass().getSimpleName().toLowerCase()); - } - - @Override - protected DataStreamSink createOutput( - DataStream dataSet, OutputFormat outputFormat, String sinkName) { - Preconditions.checkNotNull(dataSet); - Preconditions.checkNotNull(sinkName); - List fields = jdbcConf.getColumn(); - String[] fieldNames = fields.stream().map(FieldConf::getName).toArray(String[]::new); - String[] fieldTypeList = fields.stream().map(FieldConf::getType).toArray(String[]::new); - DataType[] fieldTypes = new DataType[fieldNames.length]; - List primaryKeyList = starRocksConf.getPrimaryKey(); - for (int i = 0; i < fieldNames.length; i++) { - if (primaryKeyList.contains(fieldNames[i])) { - fieldTypes[i] = StarRocksRawTypeConverter.apply(fieldTypeList[i]).notNull(); - } else { - fieldTypes[i] = StarRocksRawTypeConverter.apply(fieldTypeList[i]); - } - } - StarRocksTableRowTransformer starRocksGenericRowTransformer = - new StarRocksTableRowTransformer(typeInformation); - StarRocksDynamicSinkFunction starRocksDynamicSinkFunction = - new StarRocksDynamicSinkFunction( - getStarRocksSinkOptions(fieldNames), - primaryKeyList.size() > 0 - ? TableSchema.builder() - .primaryKey( - primaryKeyList.toArray( - new String[primaryKeyList.size()])) - .fields(fieldNames, fieldTypes) - .build() - : TableSchema.builder().fields(fieldNames, fieldTypes).build(), - starRocksGenericRowTransformer); - DataStreamSink dataStreamSink = dataSet.addSink(starRocksDynamicSinkFunction); - dataStreamSink.name(sinkName); - return dataStreamSink; - } - - public StarRocksSinkOptions getStarRocksSinkOptions(String[] fieldNames) { - return StarRocksSinkOptions.builder() - .withProperty("jdbc-url", jdbcConf.getJdbcUrl()) - .withProperty("load-url", starRocksConf.getLoadUrl()) - .withProperty("username", jdbcConf.getUsername()) - .withProperty("password", jdbcConf.getPassword()) - .withProperty("table-name", jdbcConf.getTable()) - .withProperty("database-name", jdbcConf.getSchema()) - // in case of raw data contains common delimiter like '\n' - // .withProperty("sink.properties.row_delimiter","\\x02") - // in case of raw data contains common separator like '\t' - // .withProperty("sink.properties.column_separator","\\x01") - .withProperty("sink.properties.format", "json") - .withProperty("sink.properties.strip_outer_array", "true") - .withProperty("sink.buffer-flush.interval-ms", "2000") - .withProperty("sink.properties.columns", StringUtils.join(fieldNames, ",")) - .build(); - } - - @Override - public DataStreamSink createSink(DataStream dataSet) { - - JdbcOutputFormatBuilder builder = getBuilder(); - - int connectTimeOut = jdbcConf.getConnectTimeOut(); - jdbcConf.setConnectTimeOut(connectTimeOut == 0 ? 600 : connectTimeOut); - - builder.setJdbcConf(jdbcConf); - builder.setJdbcDialect(jdbcDialect); - - AbstractRowConverter rowConverter = null; - // 同步任务使用transform - if (!useAbstractBaseColumn) { - final RowType rowType = - TableUtil.createRowType(jdbcConf.getColumn(), getRawTypeConverter()); - rowConverter = jdbcDialect.getRowConverter(rowType); - typeInformation = - TableUtil.getTypeInformation( - Collections.emptyList(), getRawTypeConverter(), false); - } else { - List fieldList = syncConf.getWriter().getFieldList(); - typeInformation = TableUtil.getTypeInformation(fieldList, getRawTypeConverter(), true); - } - builder.setRowConverter(rowConverter, useAbstractBaseColumn); - - builder.finish().initializeGlobal(1); - - return createOutput(dataSet, null); - } - - @Override - public RawTypeConverter getRawTypeConverter() { - return StarRocksRawTypeConverter::apply; - } -} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksAllTableFunction.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksAllTableFunction.java new file mode 100644 index 0000000000..519f3d2720 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksAllTableFunction.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.source; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.source.be.StarRocksQueryPlanVisitor; +import com.dtstack.chunjun.connector.starrocks.source.be.StarRocksSourceBeReader; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryBeXTablets; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryInfo; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.lookup.AbstractAllTableFunction; +import com.dtstack.chunjun.lookup.conf.LookupConf; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.functions.FunctionContext; + +import com.google.common.collect.Maps; +import org.apache.commons.collections.CollectionUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static com.dtstack.chunjun.connector.starrocks.util.StarRocksUtil.splitQueryBeXTablets; + +/** @author liuliu 2022/7/19 */ +public class StarRocksAllTableFunction extends AbstractAllTableFunction { + + private static final Logger LOG = LoggerFactory.getLogger(StarRocksAllTableFunction.class); + + private final StarRocksConf starRocksConf; + + private final int[] keyIndexes; + + private final String querySql; + + private StarRocksQueryPlanVisitor queryPlanVisitor; + + public StarRocksAllTableFunction( + StarRocksConf starRocksConf, + LookupConf lookupConf, + int[] keyIndexes, + AbstractRowConverter rowConverter) { + super(starRocksConf.getFieldNames(), null, lookupConf, rowConverter); + this.keyIndexes = keyIndexes; + this.starRocksConf = starRocksConf; + this.querySql = buildQueryStatement(); + } + + @Override + public void eval(Object... keys) { + String cacheKey = Arrays.stream(keys).map(String::valueOf).collect(Collectors.joining("_")); + List cacheList = + ((Map>) (cacheRef.get())).get(cacheKey); + if (!CollectionUtils.isEmpty(cacheList)) { + cacheList.forEach(this::collect); + } + } + + @Override + protected void initCache() { + Map> newCache = Maps.newConcurrentMap(); + cacheRef.set(newCache); + loadData(newCache); + } + + @Override + protected void reloadCache() { + // reload cacheRef and replace to old cacheRef + Map> newCache = Maps.newConcurrentMap(); + try { + loadData(newCache); + } catch (Exception e) { + throw new RuntimeException(e); + } + + cacheRef.set(newCache); + LOG.info( + "----- " + lookupConf.getTableName() + ": all cacheRef reload end:{}", + LocalDateTime.now()); + } + + @Override + protected void loadData(Object cache) { + try { + QueryInfo queryInfo = queryPlanVisitor.getQueryInfo(querySql); + List queryBeXTabletsList = splitQueryBeXTablets(1, queryInfo).get(0); + queryBeXTabletsList + .parallelStream() + .forEach( + queryBeXTablets -> { + StarRocksSourceBeReader beReader = + new StarRocksSourceBeReader( + queryBeXTablets.getBeNode(), starRocksConf); + beReader.openScanner( + queryBeXTablets.getTabletIds(), + queryInfo.getQueryPlan().getOpaqued_query_plan()); + beReader.startToRead(); + while (beReader.hasNext()) { + Object[] next = beReader.getNext(); + try { + GenericRowData rowData = + (GenericRowData) + rowConverter.toInternalLookup(next); + // add cache + ((Map>) cache) + .computeIfAbsent( + buildCacheKey(rowData), + key -> new ArrayList<>()) + .add(rowData); + } catch (Exception e) { + throw new ChunJunRuntimeException("", e); + } + } + beReader.close(); + }); + } catch (IOException e) { + throw new ChunJunRuntimeException("query from be failed", e); + } + } + + public String buildCacheKey(GenericRowData rowData) { + return Arrays.stream(keyIndexes) + .mapToObj(index -> String.valueOf(rowData.getField(index))) + .collect(Collectors.joining("_")); + } + + @Override + public void open(FunctionContext context) throws Exception { + queryPlanVisitor = new StarRocksQueryPlanVisitor(starRocksConf); + super.open(context); + } + + private String buildQueryStatement() { + String QUERY_SQL_TEMPLATE = "select %s from `%s`.`%s`"; + return String.format( + QUERY_SQL_TEMPLATE, + String.join(",", fieldsName), + starRocksConf.getDatabase(), + starRocksConf.getTable()); + } + + @Override + public void close() throws Exception { + super.close(); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksDynamicTableSource.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksDynamicTableSource.java new file mode 100644 index 0000000000..91fe7ba660 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksDynamicTableSource.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.source; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.converter.StarRocksRowConverter; +import com.dtstack.chunjun.enums.CacheType; +import com.dtstack.chunjun.lookup.conf.LookupConf; +import com.dtstack.chunjun.source.DtInputFormatSourceFunction; +import com.dtstack.chunjun.table.connector.source.ParallelAsyncTableFunctionProvider; +import com.dtstack.chunjun.table.connector.source.ParallelSourceFunctionProvider; +import com.dtstack.chunjun.table.connector.source.ParallelTableFunctionProvider; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.LookupTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.Preconditions; + +import java.util.ArrayList; +import java.util.List; + +/** @author liuliu 2022/7/19 */ +public class StarRocksDynamicTableSource + implements ScanTableSource, LookupTableSource, SupportsProjectionPushDown { + private final StarRocksConf starRocksConf; + private final LookupConf lookupConf; + private TableSchema tableSchema; + + public StarRocksDynamicTableSource( + StarRocksConf starRocksConf, LookupConf lookupConf, TableSchema tableSchema) { + this.starRocksConf = starRocksConf; + this.lookupConf = lookupConf; + this.tableSchema = tableSchema; + } + + @Override + public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { + int[] keyIndexes = new int[context.getKeys().length]; + for (int i = 0; i < keyIndexes.length; i++) { + int[] innerKeyArr = context.getKeys()[i]; + Preconditions.checkArgument( + innerKeyArr.length == 1, "StarRocks only support non-nested look up keys"); + keyIndexes[i] = innerKeyArr[0]; + } + final RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); + + if (lookupConf.getCache().equalsIgnoreCase(CacheType.ALL.toString())) { + return ParallelTableFunctionProvider.of( + new StarRocksAllTableFunction( + starRocksConf, + lookupConf, + keyIndexes, + new StarRocksRowConverter(rowType, null)), + lookupConf.getParallelism()); + } + return ParallelAsyncTableFunctionProvider.of( + new StarRocksLruTableFunction( + starRocksConf, + lookupConf, + keyIndexes, + new StarRocksRowConverter(rowType, null)), + lookupConf.getParallelism()); + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + final RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); + TypeInformation typeInformation = InternalTypeInfo.of(rowType); + StarRocksInputFormatBuilder builder = + new StarRocksInputFormatBuilder(new StarRocksInputFormat()); + List fieldConfList = new ArrayList<>(tableSchema.getFieldCount()); + String[] fieldNames = tableSchema.getFieldNames(); + for (int i = 0; i < tableSchema.getFieldCount(); i++) { + FieldConf fieldConf = new FieldConf(); + fieldConf.setName(fieldNames[i]); + fieldConf.setType(rowType.getTypeAt(i).asSummaryString()); + fieldConfList.add(fieldConf); + } + starRocksConf.setColumn(fieldConfList); + builder.setStarRocksConf(starRocksConf); + builder.setRowConverter(new StarRocksRowConverter(rowType, null)); + + return ParallelSourceFunctionProvider.of( + new DtInputFormatSourceFunction<>(builder.finish(), typeInformation), + false, + starRocksConf.getParallelism()); + } + + @Override + public DynamicTableSource copy() { + return new StarRocksDynamicTableSource(starRocksConf, lookupConf, tableSchema); + } + + @Override + public String asSummaryString() { + return "StarRocks Source"; + } + + @Override + public boolean supportsNestedProjection() { + return false; + } + + @Override + public void applyProjection(int[][] projectedFields) { + this.tableSchema = TableSchemaUtils.projectSchema(tableSchema, projectedFields); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksInputFormat.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksInputFormat.java new file mode 100644 index 0000000000..6424365228 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksInputFormat.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.source; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.source.be.StarRocksQueryPlanVisitor; +import com.dtstack.chunjun.connector.starrocks.source.be.StarRocksSourceBeReader; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryBeXTablets; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryInfo; +import com.dtstack.chunjun.constants.ConstantValue; +import com.dtstack.chunjun.source.format.BaseRichInputFormat; +import com.dtstack.chunjun.throwable.ReadRecordException; + +import org.apache.flink.core.io.InputSplit; +import org.apache.flink.table.data.RowData; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import static com.dtstack.chunjun.connector.starrocks.util.StarRocksUtil.splitQueryBeXTablets; + +/** @author liuliu 2022/7/26 */ +public class StarRocksInputFormat extends BaseRichInputFormat { + + private final Logger LOG = LoggerFactory.getLogger(StarRocksInputFormat.class); + + private StarRocksConf starRocksConf; + + private StarRocksSourceBeReader reader; + + @Override + protected InputSplit[] createInputSplitsInternal(int minNumSplits) throws Exception { + String querySql = getQueryStatement(); + LOG.info(String.format("starRocksInputFormat querySql is %s", querySql)); + StarRocksQueryPlanVisitor queryPlanVisitor = new StarRocksQueryPlanVisitor(starRocksConf); + QueryInfo queryInfo = queryPlanVisitor.getQueryInfo(querySql); + List> lists = splitQueryBeXTablets(minNumSplits, queryInfo); + List queryBeXTabletsList = new ArrayList<>(); + lists.forEach(queryBeXTabletsList::addAll); + + AtomicInteger index = new AtomicInteger(); + List res = new ArrayList<>(); + queryBeXTabletsList.forEach( + queryBeXTablets -> + res.add( + new StarRocksInputSplit( + index.getAndIncrement(), + queryBeXTabletsList.size(), + queryBeXTablets, + queryInfo.getQueryPlan().getOpaqued_query_plan()))); + return res.toArray(new StarRocksInputSplit[0]); + } + + @Override + protected void openInternal(InputSplit inputSplit) { + StarRocksInputSplit starRocksInputSplit = ((StarRocksInputSplit) inputSplit); + QueryBeXTablets queryBeXTablets = starRocksInputSplit.getQueryBeXTablets(); + reader = new StarRocksSourceBeReader(queryBeXTablets.getBeNode(), starRocksConf); + reader.openScanner( + queryBeXTablets.getTabletIds(), starRocksInputSplit.getOpaquedQueryPlan()); + reader.startToRead(); + } + + @Override + protected RowData nextRecordInternal(RowData rowData) throws ReadRecordException { + try { + return rowConverter.toInternal(reader.getNext()); + } catch (Exception e) { + throw new ReadRecordException("", e); + } + } + + @Override + protected void closeInternal() { + if (reader != null) { + reader.close(); + } + } + + @Override + public boolean reachedEnd() { + return !reader.hasNext(); + } + + public StarRocksConf getStarRocksConf() { + return starRocksConf; + } + + public void setStarRocksConf(StarRocksConf starRocksConf) { + this.starRocksConf = starRocksConf; + } + + public String getQueryStatement() { + StringBuilder builder = new StringBuilder("select "); + if (starRocksConf.getColumn().size() == 1 + && starRocksConf.getColumn().get(0).getName().equals(ConstantValue.STAR_SYMBOL)) { + builder.append(ConstantValue.STAR_SYMBOL); + } else { + builder.append(String.join(",", starRocksConf.getFieldNames())); + } + builder.append(" from ") + .append( + String.format( + "%s.%s", starRocksConf.getDatabase(), starRocksConf.getTable())); + if (StringUtils.isNotBlank(starRocksConf.getFilterStatement())) { + builder.append(" where ").append(starRocksConf.getFilterStatement()); + } + return builder.toString(); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksInputFormatBuilder.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksInputFormatBuilder.java new file mode 100644 index 0000000000..d3afcf1ee6 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksInputFormatBuilder.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.source; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.source.format.BaseRichInputFormatBuilder; + +import com.google.common.base.Preconditions; + +/** @author liuliu 2022/7/26 */ +public class StarRocksInputFormatBuilder extends BaseRichInputFormatBuilder { + public StarRocksInputFormatBuilder(StarRocksInputFormat format) { + super(format); + } + + public void setStarRocksConf(StarRocksConf starRocksConf) { + super.setConfig(starRocksConf); + format.setStarRocksConf(starRocksConf); + } + + @Override + protected void checkFormat() { + StarRocksConf conf = format.getStarRocksConf(); + Preconditions.checkNotNull(conf.getUrl(), "starRocks url is required"); + Preconditions.checkNotNull(conf.getFeNodes(), "starRocks feNodes is required"); + Preconditions.checkNotNull( + conf.getDatabase(), "starRocks database is required when nameMapped is not enable"); + Preconditions.checkNotNull( + conf.getTable(), "starRocks table is required when nameMapped is not enable"); + Preconditions.checkNotNull(conf.getUsername(), "starRocks username is required"); + Preconditions.checkNotNull(conf.getPassword(), "starRocks password is required"); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksInputSplit.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksInputSplit.java new file mode 100644 index 0000000000..f54f6476aa --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksInputSplit.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.source; + +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryBeXTablets; + +import org.apache.flink.core.io.GenericInputSplit; + +/** @author liuliu 2022/7/26 */ +public class StarRocksInputSplit extends GenericInputSplit { + + private final QueryBeXTablets queryBeXTablets; + private final String opaquedQueryPlan; + + /** + * Creates a generic input split with the given split number. + * + * @param partitionNumber The number of the split's partition. + * @param totalNumberOfPartitions The total number of the splits (partitions). + */ + public StarRocksInputSplit( + int partitionNumber, + int totalNumberOfPartitions, + QueryBeXTablets queryBeXTablets, + String opaquedQueryPlan) { + super(partitionNumber, totalNumberOfPartitions); + this.queryBeXTablets = queryBeXTablets; + this.opaquedQueryPlan = opaquedQueryPlan; + } + + public QueryBeXTablets getQueryBeXTablets() { + return queryBeXTablets; + } + + public String getOpaquedQueryPlan() { + return opaquedQueryPlan; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksLruTableFunction.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksLruTableFunction.java new file mode 100644 index 0000000000..889e641261 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksLruTableFunction.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.source; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.source.be.StarRocksQueryPlanVisitor; +import com.dtstack.chunjun.connector.starrocks.source.be.StarRocksSourceBeReader; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryBeXTablets; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryInfo; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.enums.ECacheContentType; +import com.dtstack.chunjun.lookup.AbstractLruTableFunction; +import com.dtstack.chunjun.lookup.cache.CacheMissVal; +import com.dtstack.chunjun.lookup.cache.CacheObj; +import com.dtstack.chunjun.lookup.conf.LookupConf; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; +import com.dtstack.chunjun.throwable.NoRestartException; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.functions.FunctionContext; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static com.dtstack.chunjun.connector.starrocks.util.StarRocksUtil.splitQueryBeXTablets; + +/** @author liuliu 2022/7/19 */ +public class StarRocksLruTableFunction extends AbstractLruTableFunction { + + private static final Logger LOG = LoggerFactory.getLogger(StarRocksLruTableFunction.class); + + private final StarRocksConf starRocksConf; + + private final int[] keyIndexes; + + private StarRocksQueryPlanVisitor queryPlanVisitor; + + private String queryStatement; + + public StarRocksLruTableFunction( + StarRocksConf starRocksConf, + LookupConf lookupConf, + int[] keyIndexes, + AbstractRowConverter rowConverter) { + super(lookupConf, rowConverter); + this.starRocksConf = starRocksConf; + this.keyIndexes = keyIndexes; + } + + @Override + public void open(FunctionContext context) throws Exception { + super.open(context); + this.queryPlanVisitor = new StarRocksQueryPlanVisitor(starRocksConf); + this.queryStatement = buildQueryStatement(); + } + + @Override + public void handleAsyncInvoke(CompletableFuture> future, Object... keys) { + String cacheKey = buildCacheKey(keys); + try { + QueryInfo queryInfo = queryPlanVisitor.getQueryInfo(buildFilterStatement(keys)); + List queryBeXTabletsList = splitQueryBeXTablets(1, queryInfo).get(0); + queryBeXTabletsList + .parallelStream() + .forEach( + queryBeXTablets -> { + StarRocksSourceBeReader beReader = + new StarRocksSourceBeReader( + queryBeXTablets.getBeNode(), starRocksConf); + beReader.openScanner( + queryBeXTablets.getTabletIds(), + queryInfo.getQueryPlan().getOpaqued_query_plan()); + beReader.startToRead(); + readAndDealData(beReader, cacheKey, future); + }); + } catch (IOException e) { + throw new ChunJunRuntimeException(e); + } + } + + private void readAndDealData( + StarRocksSourceBeReader beReader, + String cacheKey, + CompletableFuture> future) { + List rowDataList = new ArrayList<>(); + List cacheContent = new ArrayList<>(); + try { + while (beReader.hasNext()) { + Object[] next = beReader.getNext(); + cacheContent.add(next); + + GenericRowData rowData = (GenericRowData) rowConverter.toInternalLookup(next); + rowDataList.add(rowData); + } + } catch (Exception e) { + parseErrorRecords.inc(); + if (parseErrorRecords.getCount() > lookupConf.getErrorLimit()) { + throw new NoRestartException("lru parse error time exceeded", e); + } + } finally { + beReader.close(); + } + dealResult(cacheKey, future, rowDataList, cacheContent); + } + + private void dealResult( + String cacheKey, + CompletableFuture> future, + List rowDataList, + List cacheContent) { + if (rowDataList.size() > 0) { + dealCacheData( + cacheKey, CacheObj.buildCacheObj(ECacheContentType.MultiLine, cacheContent)); + future.complete(rowDataList); + } else { + dealMissKey(future); + dealCacheData(cacheKey, CacheMissVal.getMissKeyObj()); + } + } + + public String buildQueryStatement() { + return "select " + + String.join(",", starRocksConf.getFieldNames()) + + " from " + + starRocksConf.getDatabase() + + "." + + starRocksConf.getTable() + + " where "; + } + + public String buildFilterStatement(Object[] value) { + StringBuilder builder = new StringBuilder(queryStatement); + for (int i = 0; i < keyIndexes.length; i++) { + String fieldName = starRocksConf.getFieldNames()[keyIndexes[i]]; + LogicalTypeRoot typeRoot = + starRocksConf.getDataTypes()[keyIndexes[i]].getLogicalType().getTypeRoot(); + Object curValue = value[i]; + if (curValue == null) { + builder.append(fieldName).append("IS NULL"); + } else { + if (typeRoot.name().startsWith("TIMESTAMP")) { + DateTimeFormatter dateTimeFormatter = + DateTimeFormatter.ofPattern("yyyy-MM-dd HH:mm:ss.SSSSSS"); + curValue = + dateTimeFormatter.format(((TimestampData) curValue).toLocalDateTime()); + } else if (typeRoot == LogicalTypeRoot.DATE) { + SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd"); + Calendar c = Calendar.getInstance(); + c.setTime(new Date(0L)); + c.add(Calendar.DATE, (int) curValue); + curValue = dateFormat.format(c.getTime()); + } + builder.append(fieldName).append("=").append(String.format("'%s'", curValue)); + } + if (i + 1 != keyIndexes.length) { + builder.append(" and "); + } + } + LOG.info(String.format("startRocks lru querySql:%s", builder)); + return builder.toString(); + } + + @Override + public void close() throws Exception { + super.close(); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksSourceFactory.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksSourceFactory.java new file mode 100644 index 0000000000..c94b407393 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/StarRocksSourceFactory.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 com.dtstack.chunjun.connector.starrocks.source; + +import com.dtstack.chunjun.conf.FieldConf; +import com.dtstack.chunjun.conf.SyncConf; +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.converter.StarRocksColumnConverter; +import com.dtstack.chunjun.connector.starrocks.converter.StarRocksRawTypeConverter; +import com.dtstack.chunjun.connector.starrocks.converter.StarRocksRowConverter; +import com.dtstack.chunjun.converter.AbstractRowConverter; +import com.dtstack.chunjun.converter.RawTypeConverter; +import com.dtstack.chunjun.source.SourceFactory; +import com.dtstack.chunjun.util.JsonUtil; +import com.dtstack.chunjun.util.TableUtil; + +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; + +import org.apache.commons.lang3.StringUtils; + +import java.util.ArrayList; +import java.util.List; + +/** @author liuliu 2022/7/27 */ +public class StarRocksSourceFactory extends SourceFactory { + + private final StarRocksConf starRocksConf; + + public StarRocksSourceFactory(SyncConf syncConf, StreamExecutionEnvironment env) { + super(syncConf, env); + starRocksConf = + JsonUtil.toObject( + JsonUtil.toJson(syncConf.getReader().getParameter()), StarRocksConf.class); + List fieldList = syncConf.getReader().getFieldList(); + List fieldNameList = new ArrayList<>(); + List dataTypeList = new ArrayList<>(); + RawTypeConverter rawTypeConverter = getRawTypeConverter(); + for (FieldConf fieldConf : fieldList) { + if (StringUtils.isBlank(fieldConf.getValue())) { + fieldNameList.add(fieldConf.getName()); + dataTypeList.add(rawTypeConverter.apply(fieldConf.getType())); + } + } + + super.initCommonConf(starRocksConf); + starRocksConf.setFieldNames(fieldNameList.toArray(new String[0])); + starRocksConf.setDataTypes(dataTypeList.toArray(new DataType[0])); + } + + @Override + public RawTypeConverter getRawTypeConverter() { + return StarRocksRawTypeConverter::apply; + } + + @Override + public DataStream createSource() { + StarRocksInputFormatBuilder inputFormatBuilder = + new StarRocksInputFormatBuilder(new StarRocksInputFormat()); + inputFormatBuilder.setStarRocksConf(starRocksConf); + RowType rowType = TableUtil.createRowType(starRocksConf.getColumn(), getRawTypeConverter()); + AbstractRowConverter rowConverter; + if (useAbstractBaseColumn) { + rowConverter = new StarRocksColumnConverter(rowType, starRocksConf); + } else { + rowConverter = new StarRocksRowConverter(rowType, null); + } + inputFormatBuilder.setRowConverter(rowConverter, useAbstractBaseColumn); + return createInput(inputFormatBuilder.finish()); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksArrowReader.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksArrowReader.java new file mode 100644 index 0000000000..19041ae4ff --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksArrowReader.java @@ -0,0 +1,157 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.source.be; + +import com.dtstack.chunjun.connector.starrocks.converter.StarRocksRawTypeConverter; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.ColumnInfo; + +import com.starrocks.thrift.TScanBatchResult; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.ipc.ArrowStreamReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +public class StarRocksArrowReader { + + private static final Logger LOG = LoggerFactory.getLogger(StarRocksArrowReader.class); + private final List columnInfoList; + private int offsetOfBatchForRead; + private int rowCountOfBatch; + private int flinkRowsCount; + + private List sourceJavaRows; + private final ArrowStreamReader arrowStreamReader; + private final ConcurrentHashMap fieldVectorMap; + private final RootAllocator rootAllocator; + + public StarRocksArrowReader(TScanBatchResult nextResult, List columnInfoList) { + this.columnInfoList = columnInfoList; + this.rootAllocator = new RootAllocator(Integer.MAX_VALUE); + byte[] bytes = nextResult.getRows(); + ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(bytes); + this.arrowStreamReader = new ArrowStreamReader(byteArrayInputStream, rootAllocator); + this.offsetOfBatchForRead = 0; + this.fieldVectorMap = new ConcurrentHashMap<>(); + } + + public StarRocksArrowReader read() throws IOException { + VectorSchemaRoot root = arrowStreamReader.getVectorSchemaRoot(); + while (arrowStreamReader.loadNextBatch()) { + List fieldVectors = root.getFieldVectors(); + fieldVectors + .parallelStream() + .forEach(vector -> fieldVectorMap.put(vector.getName(), vector)); + if (fieldVectors.size() == 0 || root.getRowCount() == 0) { + continue; + } + rowCountOfBatch = root.getRowCount(); + sourceJavaRows = new ArrayList<>(rowCountOfBatch); + for (int i = 0; i < rowCountOfBatch; i++) { + sourceJavaRows.add(new Object[columnInfoList.size()]); + } + this.genData(); + flinkRowsCount += root.getRowCount(); + } + return this; + } + + public boolean hasNext() { + if (offsetOfBatchForRead < flinkRowsCount) { + return true; + } + this.close(); + return false; + } + + public Object[] next() { + if (!hasNext()) { + LOG.error("offset larger than data count"); + throw new RuntimeException("read offset larger than data count"); + } + return sourceJavaRows.get(offsetOfBatchForRead++); + } + + public int getReadRowCount() { + return flinkRowsCount; + } + + private void close() { + try { + if (arrowStreamReader != null) { + arrowStreamReader.close(); + } + if (rootAllocator != null) { + rootAllocator.close(); + } + } catch (IOException e) { + LOG.error("Failed to close StarRocksArrowReader:" + e.getMessage()); + throw new RuntimeException("Failed to close StarRocksArrowReader:" + e.getMessage()); + } + } + + private void genData() { + columnInfoList + .parallelStream() + .forEach( + columnInfo -> { + FieldVector fieldVector = getFieldVector(columnInfo.getFieldName()); + getStarRocksToJavaTrans(columnInfo) + .transToJavaData( + fieldVector, + rowCountOfBatch, + columnInfo.getIndex(), + sourceJavaRows); + }); + } + + public FieldVector getFieldVector(String fieldName) { + FieldVector fieldVector = fieldVectorMap.get(fieldName); + checkNotNull( + fieldVector, String.format("Can not find StarRocks column data[%s]", fieldName)); + return fieldVector; + } + + public StarRocksToJavaTrans getStarRocksToJavaTrans(ColumnInfo columnInfo) { + HashMap typeTransMap = + StarRocksToJavaTrans.DataTypeRelationMap.get(columnInfo.getLogicalTypeRoot()); + checkNotNull(typeTransMap, "Unsupported type,columnInfo[%s]", columnInfo); + StarRocksToJavaTrans starRocksToJavaTrans = typeTransMap.get(columnInfo.getStarRocksType()); + checkNotNull( + starRocksToJavaTrans, + "Type corresponding error,Column[%s]'s StarRocksType should be %s;LogicalTypeRoot except %s but is %s", + columnInfo.getFieldName(), + columnInfo.getStarRocksType(), + StarRocksRawTypeConverter.apply(columnInfo.getStarRocksType()) + .getLogicalType() + .getTypeRoot(), + columnInfo.getLogicalTypeRoot()); + return starRocksToJavaTrans; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksQueryPlanVisitor.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksQueryPlanVisitor.java new file mode 100644 index 0000000000..35c740cfea --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksQueryPlanVisitor.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.source.be; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryBeXTablets; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryInfo; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryPlan; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.Tablet; + +import com.alibaba.fastjson.JSONObject; +import org.apache.http.HttpEntity; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.stream.Collectors; + +import static com.dtstack.chunjun.connector.starrocks.util.StarRocksUtil.getBasicAuthHeader; + +/** @author liuliu 2022/7/20 */ +public class StarRocksQueryPlanVisitor implements Serializable { + private static final long serialVersionUID = 1L; + private static final Logger LOG = LoggerFactory.getLogger(StarRocksQueryPlanVisitor.class); + private final StarRocksConf starRocksConf; + + public StarRocksQueryPlanVisitor(StarRocksConf starRocksConf) { + this.starRocksConf = starRocksConf; + } + + public QueryInfo getQueryInfo(String querySql) throws IOException { + List httpNodeList = starRocksConf.getFeNodes(); + QueryPlan queryPlan = + getQueryPlan(querySql, httpNodeList.get(new Random().nextInt(httpNodeList.size()))); + return new QueryInfo(queryPlan, transferQueryPlanToBeXTablet(queryPlan)); + } + + private static List transferQueryPlanToBeXTablet(QueryPlan queryPlan) { + Map> beXTablets = new HashMap<>(); + queryPlan + .getPartitions() + .forEach( + (tabletId, tablet) -> allocateTabletsEqually(beXTablets, tabletId, tablet)); + + return beXTablets.entrySet().stream() + .map( + entry -> + new QueryBeXTablets( + entry.getKey(), new ArrayList<>(entry.getValue()))) + .collect(Collectors.toList()); + } + + private static void allocateTabletsEqually( + Map> beXTablets, String tabletId, Tablet tablet) { + int tabletCount = Integer.MAX_VALUE; + String currentBeNode = ""; + // Allocate tablets equally for all BeNodes + for (String beNode : tablet.getRoutings()) { + if (!beXTablets.containsKey(beNode)) { + beXTablets.put(beNode, new HashSet<>()); + currentBeNode = beNode; + break; + } + if (beXTablets.get(beNode).size() < tabletCount) { + currentBeNode = beNode; + tabletCount = beXTablets.get(beNode).size(); + } + } + beXTablets.get(currentBeNode).add(Long.valueOf(tabletId)); + } + + public QueryPlan getQueryPlan(String querySql, String httpNode) throws IOException { + String url = getRequestUrl(httpNode); + Map bodyMap = new HashMap<>(); + bodyMap.put("sql", querySql); + String body = new JSONObject(bodyMap).toString(); + + int requestCode = 0; + String respString = ""; + for (int i = 0; i < starRocksConf.getMaxRetries(); i++) { + try (CloseableHttpClient httpClient = HttpClients.createDefault()) { + HttpPost post = new HttpPost(url); + post.setHeader("Content-Type", "application/json;charset=UTF-8"); + post.setHeader( + "Authorization", + getBasicAuthHeader( + starRocksConf.getUsername(), starRocksConf.getPassword())); + post.setEntity(new ByteArrayEntity(body.getBytes())); + try (CloseableHttpResponse response = httpClient.execute(post)) { + requestCode = response.getStatusLine().getStatusCode(); + HttpEntity respEntity = response.getEntity(); + respString = EntityUtils.toString(respEntity, "UTF-8"); + } + } + if (200 == requestCode || i == starRocksConf.getMaxRetries()) { + break; + } + LOG.warn("Request of get query plan failed with code:{}", requestCode); + try { + Thread.sleep(1000L * (i + 1)); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw new IOException( + "Unable to get query plan, interrupted while doing another attempt", ex); + } + } + return dealQueryPlanResult(requestCode, respString); + } + + private QueryPlan dealQueryPlanResult(int requestCode, String respString) { + if (200 != requestCode) { + throw new RuntimeException( + "Request of get query plan failed with code " + requestCode + " " + respString); + } + if (respString.isEmpty()) { + LOG.warn("Request failed with empty response."); + throw new RuntimeException("Request failed with empty response." + requestCode); + } + JSONObject jsonObject = JSONObject.parseObject(respString); + return JSONObject.toJavaObject(jsonObject, QueryPlan.class); + } + + private String getRequestUrl(String httpNode) { + return "http://" + + httpNode + + "/api/" + + starRocksConf.getDatabase() + + "/" + + starRocksConf.getTable() + + "/_query_plan"; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksSourceBeReader.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksSourceBeReader.java new file mode 100644 index 0000000000..94af88e7c7 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksSourceBeReader.java @@ -0,0 +1,227 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.source.be; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.ColumnInfo; +import com.dtstack.chunjun.throwable.ChunJunRuntimeException; + +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import com.starrocks.shade.org.apache.thrift.TException; +import com.starrocks.shade.org.apache.thrift.protocol.TBinaryProtocol; +import com.starrocks.shade.org.apache.thrift.protocol.TProtocol; +import com.starrocks.shade.org.apache.thrift.transport.TSocket; +import com.starrocks.shade.org.apache.thrift.transport.TTransportException; +import com.starrocks.thrift.TScanBatchResult; +import com.starrocks.thrift.TScanCloseParams; +import com.starrocks.thrift.TScanColumnDesc; +import com.starrocks.thrift.TScanNextBatchParams; +import com.starrocks.thrift.TScanOpenParams; +import com.starrocks.thrift.TScanOpenResult; +import com.starrocks.thrift.TStarrocksExternalService; +import com.starrocks.thrift.TStatusCode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** @author liuliu 2022/7/20 */ +public class StarRocksSourceBeReader { + + private static final Logger LOG = LoggerFactory.getLogger(StarRocksSourceBeReader.class); + + private final StarRocksConf starRocksConf; + private final String beHost; + + private final TStarrocksExternalService.Client client; + private String contextId; + private int readerOffset = 0; + + private StarRocksArrowReader curArrowReader; + private Object[] curData; + private List columnInfoList; + + public StarRocksSourceBeReader(String beNode, StarRocksConf starRocksConf) { + this.starRocksConf = starRocksConf; + String[] beNodeInfo = beNode.split(":"); + this.beHost = beNodeInfo[0].trim(); + int bePort = Integer.parseInt(beNodeInfo[1].trim()); + TBinaryProtocol.Factory factory = new TBinaryProtocol.Factory(); + TSocket socket = + new TSocket( + beHost, + bePort, + starRocksConf.getBeClientTimeout(), + starRocksConf.getBeClientTimeout()); + try { + socket.open(); + } catch (TTransportException e) { + socket.close(); + throw new RuntimeException("Failed to create brpc source:" + e.getMessage()); + } + TProtocol protocol = factory.getProtocol(socket); + client = new TStarrocksExternalService.Client(protocol); + } + + public void openScanner(List tablets, String opaqued_query_plan) { + TScanOpenParams params = new TScanOpenParams(); + params.setTablet_ids(tablets); + params.setOpaqued_query_plan(opaqued_query_plan); + params.setCluster("default_cluster"); + params.setDatabase(starRocksConf.getDatabase()); + params.setTable(starRocksConf.getTable()); + params.setUser(starRocksConf.getUsername()); + params.setPasswd(starRocksConf.getPassword()); + params.setBatch_size(starRocksConf.getBeFetchRows()); + params.setMem_limit(starRocksConf.getBeFetchMaxBytes()); + params.setProperties(starRocksConf.getBeSocketProperties()); + params.setKeep_alive_min((short) starRocksConf.getBeClientKeepLiveMin()); + params.setQuery_timeout(starRocksConf.getBeQueryTimeoutSecond()); + LOG.info("open Scan params.mem_limit {} B", params.getMem_limit()); + LOG.info("open Scan params.keep-alive-min {} min", params.getKeep_alive_min()); + TScanOpenResult result; + int times = 0; + while (true) { + try { + result = client.open_scanner(params); + if (!TStatusCode.OK.equals(result.getStatus().getStatus_code())) { + throw new RuntimeException( + "Failed to open scanner." + + result.getStatus().getStatus_code() + + result.getStatus().getError_msgs()); + } + break; + } catch (TException e) { + if (++times <= starRocksConf.getMaxRetries()) { + LOG.info( + String.format("failed to open beScanner,current retryTimes:%s", times)); + } else { + throw new ChunJunRuntimeException( + String.format("failed to scan from be,scanParams[%s]", params), e); + } + } + } + this.contextId = result.getContext_id(); + initNameTypeMap(result.selected_columns); + } + + private void initNameTypeMap(List selectedColumnList) { + columnInfoList = new ArrayList<>(selectedColumnList.size()); + String[] fieldNames = starRocksConf.getFieldNames(); + DataType[] dataTypes = starRocksConf.getDataTypes(); + if (selectedColumnList.size() != fieldNames.length) { + throw new ChunJunRuntimeException( + "be selected column size does not match configuration column size"); + } + + for (int i = 0; i < selectedColumnList.size(); i++) { + String fieldName = fieldNames[i]; + LogicalTypeRoot logicalTypeRoot = dataTypes[i].getLogicalType().getTypeRoot(); + String starRocksType = null; + for (TScanColumnDesc columnDesc : selectedColumnList) { + if (fieldName.equalsIgnoreCase(columnDesc.getName())) { + starRocksType = columnDesc.getType().name(); + } + } + if (starRocksType == null) { + throw new ChunJunRuntimeException( + String.format( + "be selected column does not contain column[%s],please check your configuration", + fieldName)); + } + columnInfoList.add(new ColumnInfo(fieldName, logicalTypeRoot, starRocksType, i)); + } + } + + public void startToRead() { + TScanNextBatchParams params = new TScanNextBatchParams(); + params.setContext_id(this.contextId); + params.setOffset(this.readerOffset); + TScanBatchResult result; + int times = 0; + while (true) { + try { + result = client.get_next(params); + if (!TStatusCode.OK.equals(result.getStatus().getStatus_code())) { + throw new RuntimeException( + "Failed to get next from be -> ip:[" + + beHost + + "] " + + result.getStatus().getStatus_code() + + " msg:" + + result.getStatus().getError_msgs()); + } + break; + } catch (Exception e) { + if (++times <= starRocksConf.getMaxRetries()) { + LOG.info(String.format("failed to scan from be,current retryTimes:%s", times)); + } else { + throw new ChunJunRuntimeException( + String.format("failed to scan from be,scanParams[%s]", params), e); + } + } + } + if (!result.eos) { + handleResult(result); + } + } + + public boolean hasNext() { + return this.curData != null; + } + + public Object[] getNext() { + Object[] preparedData = this.curData; + this.curData = null; + if (this.curArrowReader.hasNext()) { + this.curData = curArrowReader.next(); + } + if (this.curData != null) { + return preparedData; + } + startToRead(); + return preparedData; + } + + private void handleResult(TScanBatchResult result) { + StarRocksArrowReader starRocksRowData; + try { + starRocksRowData = new StarRocksArrowReader(result, columnInfoList).read(); + } catch (IOException e) { + throw new RuntimeException(e.getMessage()); + } + this.readerOffset = starRocksRowData.getReadRowCount() + this.readerOffset; + this.curArrowReader = starRocksRowData; + this.curData = starRocksRowData.next(); + } + + public void close() { + TScanCloseParams tScanCloseParams = new TScanCloseParams(); + tScanCloseParams.setContext_id(this.contextId); + try { + this.client.close_scanner(tScanCloseParams); + } catch (TException e) { + throw new RuntimeException(e.getMessage()); + } + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksToJavaTrans.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksToJavaTrans.java new file mode 100644 index 0000000000..fe59c1177b --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/StarRocksToJavaTrans.java @@ -0,0 +1,285 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.source.be; + +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +import org.apache.arrow.vector.BigIntVector; +import org.apache.arrow.vector.BitVector; +import org.apache.arrow.vector.DecimalVector; +import org.apache.arrow.vector.FieldVector; +import org.apache.arrow.vector.Float4Vector; +import org.apache.arrow.vector.Float8Vector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.SmallIntVector; +import org.apache.arrow.vector.TinyIntVector; +import org.apache.arrow.vector.VarCharVector; + +import java.util.HashMap; +import java.util.List; + +public interface StarRocksToJavaTrans { + // StarRocks + String DATA_TYPE_STARROCKS_DATE = "DATE"; + String DATA_TYPE_STARROCKS_DATETIME = "DATETIME"; + + String DATA_TYPE_STARROCKS_CHAR = "CHAR"; + String DATA_TYPE_STARROCKS_VARCHAR = "VARCHAR"; + + String DATA_TYPE_STARROCKS_BOOLEAN = "BOOLEAN"; + + String DATA_TYPE_STARROCKS_TINYINT = "TINYINT"; + String DATA_TYPE_STARROCKS_SMALLINT = "SMALLINT"; + String DATA_TYPE_STARROCKS_INT = "INT"; + String DATA_TYPE_STARROCKS_BIGINT = "BIGINT"; + String DATA_TYPE_STARROCKS_LARGEINT = "LARGEINT"; + + String DATA_TYPE_STARROCKS_FLOAT = "FLOAT"; + String DATA_TYPE_STARROCKS_DOUBLE = "DOUBLE"; + String DATA_TYPE_STARROCKS_DECIMAL = "DECIMAL"; + String DATA_TYPE_STARROCKS_DECIMALV2 = "DECIMALV2"; + String DATA_TYPE_STARROCKS_DECIMAL32 = "DECIMAL32"; + String DATA_TYPE_STARROCKS_DECIMAL64 = "DECIMAL64"; + String DATA_TYPE_STARROCKS_DECIMAL128 = "DECIMAL128"; + + HashMap> DataTypeRelationMap = + new HashMap>() { + { + put( + LogicalTypeRoot.DATE, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_DATE, new VarCharVectorTrans()); + } + }); + put( + LogicalTypeRoot.TIMESTAMP_WITHOUT_TIME_ZONE, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_DATETIME, new VarCharVectorTrans()); + } + }); + put( + LogicalTypeRoot.TIMESTAMP_WITH_LOCAL_TIME_ZONE, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_DATETIME, new VarCharVectorTrans()); + } + }); + put( + LogicalTypeRoot.TIMESTAMP_WITH_TIME_ZONE, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_DATETIME, new VarCharVectorTrans()); + } + }); + put( + LogicalTypeRoot.VARCHAR, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_CHAR, new VarCharVectorTrans()); + put(DATA_TYPE_STARROCKS_VARCHAR, new VarCharVectorTrans()); + put(DATA_TYPE_STARROCKS_LARGEINT, new VarCharVectorTrans()); + } + }); + put( + LogicalTypeRoot.BOOLEAN, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_BOOLEAN, new BitVectorTrans()); + } + }); + put( + LogicalTypeRoot.TINYINT, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_TINYINT, new TinyIntVectorTrans()); + } + }); + put( + LogicalTypeRoot.SMALLINT, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_SMALLINT, new SmallIntVectorTrans()); + } + }); + put( + LogicalTypeRoot.INTEGER, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_INT, new IntVectorTrans()); + } + }); + put( + LogicalTypeRoot.BIGINT, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_BIGINT, new BigIntVectorTrans()); + } + }); + put( + LogicalTypeRoot.FLOAT, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_FLOAT, new Float4VectorTrans()); + } + }); + put( + LogicalTypeRoot.DOUBLE, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_DOUBLE, new Float8VectorTrans()); + } + }); + put( + LogicalTypeRoot.DECIMAL, + new HashMap() { + { + put(DATA_TYPE_STARROCKS_DECIMAL, new DecimalVectorTrans()); + put(DATA_TYPE_STARROCKS_DECIMALV2, new DecimalVectorTrans()); + put(DATA_TYPE_STARROCKS_DECIMAL32, new DecimalVectorTrans()); + put(DATA_TYPE_STARROCKS_DECIMAL64, new DecimalVectorTrans()); + put(DATA_TYPE_STARROCKS_DECIMAL128, new DecimalVectorTrans()); + } + }); + } + }; + + void transToJavaData( + FieldVector curFieldVector, int rowCount, int colIndex, List res); + + /** StarRocks Date/Timestamp/Char/Varchar/LargeInt to java String */ + class VarCharVectorTrans implements StarRocksToJavaTrans { + @Override + public void transToJavaData( + FieldVector curFieldVector, int rowCount, int colIndex, List res) { + VarCharVector varCharVector = (VarCharVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + res.get(rowIndex)[colIndex] = + varCharVector.isNull(rowIndex) + ? null + : new String(varCharVector.get(rowIndex)); + } + } + } + + /** StarRocks Boolean to java boolean */ + class BitVectorTrans implements StarRocksToJavaTrans { + @Override + public void transToJavaData( + FieldVector curFieldVector, int rowCount, int colIndex, List res) { + BitVector bitVector = (BitVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + res.get(rowIndex)[colIndex] = + bitVector.isNull(rowIndex) ? null : bitVector.get(rowIndex) != 0; + } + } + } + + /** StarRocks Tinyint to java byte */ + class TinyIntVectorTrans implements StarRocksToJavaTrans { + @Override + public void transToJavaData( + FieldVector curFieldVector, int rowCount, int colIndex, List res) { + TinyIntVector tinyIntVector = (TinyIntVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + res.get(rowIndex)[colIndex] = + tinyIntVector.isNull(rowIndex) ? null : tinyIntVector.get(rowIndex); + } + } + } + + /** StarRocks Smallint to java int */ + class SmallIntVectorTrans implements StarRocksToJavaTrans { + @Override + public void transToJavaData( + FieldVector curFieldVector, int rowCount, int colIndex, List res) { + SmallIntVector smallIntVector = (SmallIntVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + res.get(rowIndex)[colIndex] = + smallIntVector.isNull(rowIndex) ? null : smallIntVector.get(rowIndex); + } + } + } + + /** StarRocks Int to java int */ + class IntVectorTrans implements StarRocksToJavaTrans { + @Override + public void transToJavaData( + FieldVector curFieldVector, int rowCount, int colIndex, List res) { + IntVector intVector = (IntVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + res.get(rowIndex)[colIndex] = + intVector.isNull(rowIndex) ? null : intVector.get(rowIndex); + } + } + } + + /** StarRocks Bigint to java long */ + class BigIntVectorTrans implements StarRocksToJavaTrans { + @Override + public void transToJavaData( + FieldVector curFieldVector, int rowCount, int colIndex, List res) { + BigIntVector bigIntVector = (BigIntVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + res.get(rowIndex)[colIndex] = + bigIntVector.isNull(rowIndex) ? null : bigIntVector.get(rowIndex); + } + } + } + + /** StarRocks Float to java float */ + class Float4VectorTrans implements StarRocksToJavaTrans { + @Override + public void transToJavaData( + FieldVector curFieldVector, int rowCount, int colIndex, List res) { + Float4Vector float4Vector = (Float4Vector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + res.get(rowIndex)[colIndex] = + float4Vector.isNull(rowIndex) ? null : float4Vector.get(rowIndex); + } + } + } + + /** StarRocks Double to java double */ + class Float8VectorTrans implements StarRocksToJavaTrans { + @Override + public void transToJavaData( + FieldVector curFieldVector, int rowCount, int colIndex, List res) { + Float8Vector float8Vector = (Float8Vector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + res.get(rowIndex)[colIndex] = + float8Vector.isNull(rowIndex) ? null : float8Vector.get(rowIndex); + } + } + } + + /** StarRocks Decimal to java double */ + class DecimalVectorTrans implements StarRocksToJavaTrans { + @Override + public void transToJavaData( + FieldVector curFieldVector, int rowCount, int colIndex, List res) { + DecimalVector decimalVector = (DecimalVector) curFieldVector; + for (int rowIndex = 0; rowIndex < rowCount; rowIndex++) { + res.get(rowIndex)[colIndex] = + decimalVector.isNull(rowIndex) ? null : decimalVector.getObject(rowIndex); + } + } + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/ColumnInfo.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/ColumnInfo.java new file mode 100644 index 0000000000..139d34c071 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/ColumnInfo.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.source.be.entity; + +import org.apache.flink.table.types.logical.LogicalTypeRoot; + +/** @author liuliu 2022/7/22 */ +public class ColumnInfo { + private String fieldName; + private LogicalTypeRoot logicalTypeRoot; + private String starRocksType; + private int index; + + public ColumnInfo( + String fieldName, LogicalTypeRoot logicalTypeRoot, String starRocksType, int index) { + this.fieldName = fieldName; + this.logicalTypeRoot = logicalTypeRoot; + this.starRocksType = starRocksType; + this.index = index; + } + + public String getFieldName() { + return fieldName; + } + + public void setFieldName(String fieldName) { + this.fieldName = fieldName; + } + + public LogicalTypeRoot getLogicalTypeRoot() { + return logicalTypeRoot; + } + + public void setLogicalTypeRoot(LogicalTypeRoot logicalTypeRoot) { + this.logicalTypeRoot = logicalTypeRoot; + } + + public String getStarRocksType() { + return starRocksType; + } + + public void setStarRocksType(String starRocksType) { + this.starRocksType = starRocksType; + } + + public int getIndex() { + return index; + } + + public void setIndex(int index) { + this.index = index; + } + + @Override + public String toString() { + return "ColumnInfo{" + + "fieldName='" + + fieldName + + '\'' + + ", logicalTypeRoot=" + + logicalTypeRoot + + ", starRocksType='" + + starRocksType + + '\'' + + ", index=" + + index + + '}'; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/QueryBeXTablets.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/QueryBeXTablets.java new file mode 100644 index 0000000000..78637a7ee1 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/QueryBeXTablets.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.source.be.entity; + +import java.io.Serializable; +import java.util.List; + +public class QueryBeXTablets implements Serializable { + + private static final long serialVersionUID = 1L; + + private final String beNode; + private final List tabletIds; + + public QueryBeXTablets(String beNode, List tabletIds) { + this.beNode = beNode; + this.tabletIds = tabletIds; + } + + public String getBeNode() { + return beNode; + } + + public List getTabletIds() { + return tabletIds; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/QueryInfo.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/QueryInfo.java new file mode 100644 index 0000000000..b1633d58da --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/QueryInfo.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.source.be.entity; + +import java.io.Serializable; +import java.util.List; + +public class QueryInfo implements Serializable { + + private static final long serialVersionUID = 1L; + + private final QueryPlan queryPlan; + private final List beXTablets; + + public QueryInfo(QueryPlan queryPlan, List beXTablets) { + this.queryPlan = queryPlan; + this.beXTablets = beXTablets; + } + + public QueryPlan getQueryPlan() { + return queryPlan; + } + + public List getBeXTablets() { + return beXTablets; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/QueryPlan.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/QueryPlan.java new file mode 100644 index 0000000000..396a2acb24 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/QueryPlan.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.source.be.entity; + +import java.io.Serializable; +import java.util.Map; + +public class QueryPlan implements Serializable { + + private static final long serialVersionUID = 1L; + + private int status; + private String opaqued_query_plan; + private Map partitions; + + public int getStatus() { + return status; + } + + public void setStatus(int status) { + this.status = status; + } + + public String getOpaqued_query_plan() { + return opaqued_query_plan; + } + + public void setOpaqued_query_plan(String opaqued_query_plan) { + this.opaqued_query_plan = opaqued_query_plan; + } + + public Map getPartitions() { + return partitions; + } + + public void setPartitions(Map partitions) { + this.partitions = partitions; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/Tablet.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/Tablet.java new file mode 100644 index 0000000000..bf267ac278 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/source/be/entity/Tablet.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.source.be.entity; + +import java.io.Serializable; +import java.util.List; + +public class Tablet implements Serializable { + + private static final long serialVersionUID = 1L; + + private List routings; + private int version; + private long versionHash; + private long schemaHash; + + public List getRoutings() { + return routings; + } + + public void setRoutings(List routingList) { + this.routings = routingList; + } + + public int getVersion() { + return version; + } + + public void setVersion(int version) { + this.version = version; + } + + public long getVersionHash() { + return versionHash; + } + + public void setVersionHash(long versionHash) { + this.versionHash = versionHash; + } + + public long getSchemaHash() { + return schemaHash; + } + + public void setSchemaHash(long schemaHash) { + this.schemaHash = schemaHash; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksQueryVisitor.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksQueryVisitor.java new file mode 100644 index 0000000000..a813392672 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksQueryVisitor.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.streamload; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.connection.StarRocksJdbcConnectionOptions; +import com.dtstack.chunjun.connector.starrocks.connection.StarRocksJdbcConnectionProvider; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class StarRocksQueryVisitor implements Serializable { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(StarRocksQueryVisitor.class); + + private final StarRocksJdbcConnectionProvider jdbcConnProvider; + private final String database; + private final String table; + + public StarRocksQueryVisitor(StarRocksConf starRocksConf) { + StarRocksJdbcConnectionOptions jdbcOptions = + new StarRocksJdbcConnectionOptions( + starRocksConf.getUrl(), + starRocksConf.getUsername(), + starRocksConf.getPassword()); + this.jdbcConnProvider = new StarRocksJdbcConnectionProvider(jdbcOptions); + this.database = starRocksConf.getDatabase(); + this.table = starRocksConf.getTable(); + } + + public List> getTableColumnsMetaData() { + return getTableColumnsMetaData(database, table); + } + + public List> getTableColumnsMetaData(String database, String table) { + final String query = + "select `COLUMN_NAME`, `COLUMN_KEY`, `DATA_TYPE`, `COLUMN_SIZE`, `DECIMAL_DIGITS` from `information_schema`.`COLUMNS` where `TABLE_SCHEMA`=? and `TABLE_NAME`=?;"; + List> rows; + try { + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Executing query '%s'", query)); + } + rows = executeQuery(query, database, table); + } catch (ClassNotFoundException se) { + throw new IllegalArgumentException("Failed to find jdbc driver." + se.getMessage(), se); + } catch (SQLException se) { + throw new IllegalArgumentException( + "Failed to get table schema info from StarRocks. " + se.getMessage(), se); + } + return rows; + } + + public String getStarRocksVersion() { + final String query = "select current_version() as ver;"; + List> rows; + try { + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Executing query '%s'", query)); + } + rows = executeQuery(query); + if (rows.isEmpty()) { + return ""; + } + String version = rows.get(0).get("ver").toString(); + LOG.info(String.format("StarRocks version: [%s].", version)); + return version; + } catch (ClassNotFoundException se) { + throw new IllegalArgumentException("Failed to find jdbc driver." + se.getMessage(), se); + } catch (SQLException se) { + throw new IllegalArgumentException( + "Failed to get StarRocks version. " + se.getMessage(), se); + } + } + + private List> executeQuery(String query, String... args) + throws ClassNotFoundException, SQLException { + jdbcConnProvider.checkValid(); + PreparedStatement stmt = + jdbcConnProvider + .getConnection() + .prepareStatement( + query, + ResultSet.TYPE_SCROLL_INSENSITIVE, + ResultSet.CONCUR_READ_ONLY); + for (int i = 0; i < args.length; i++) { + stmt.setString(i + 1, args[i]); + } + ResultSet rs = stmt.executeQuery(); + rs.next(); + ResultSetMetaData meta = rs.getMetaData(); + int columns = meta.getColumnCount(); + List> list = new ArrayList<>(); + int currRowIndex = rs.getRow(); + rs.beforeFirst(); + while (rs.next()) { + Map row = new HashMap<>(columns); + for (int i = 1; i <= columns; ++i) { + row.put(meta.getColumnName(i), rs.getObject(i)); + } + list.add(row); + } + rs.absolute(currRowIndex); + rs.close(); + jdbcConnProvider.close(); + return list; + } + + public void close() { + jdbcConnProvider.close(); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksSinkBufferEntity.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksSinkBufferEntity.java new file mode 100644 index 0000000000..4186c493d2 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksSinkBufferEntity.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.streamload; + +import java.io.Serializable; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.util.ArrayList; +import java.util.List; +import java.util.StringJoiner; +import java.util.UUID; + +public class StarRocksSinkBufferEntity implements Serializable { + + private static final long serialVersionUID = 1L; + DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern(("yyyyMMdd_HHmmss")); + + private final ArrayList buffer = new ArrayList<>(); + private int batchCount = 0; + private long batchSize = 0; + private String label; + private String database; + private String table; + private final List columnList; + boolean supportDelete; + String httpHeadColumns; + + public StarRocksSinkBufferEntity(String database, String table, List columnList) { + this.database = database; + this.table = table; + this.columnList = columnList; + label = initBatchLabel(); + } + + public String getDatabase() { + return database; + } + + public void setDatabase(String database) { + this.database = database; + } + + public String getTable() { + return table; + } + + public void setTable(String table) { + this.table = table; + } + + public String getLabel() { + return label; + } + + public ArrayList getBuffer() { + return buffer; + } + + public void addToBuffer(byte[] bts, int count) { + incBatchCount(count); + incBatchSize(bts.length); + buffer.add(bts); + } + + public int getBatchCount() { + return batchCount; + } + + private void incBatchCount(int count) { + this.batchCount += count; + } + + public long getBatchSize() { + return batchSize; + } + + private void incBatchSize(long batchSize) { + this.batchSize += batchSize; + } + + public List getColumnList() { + return columnList; + } + + public boolean isSupportDelete() { + return supportDelete; + } + + public void setSupportDelete(boolean supportDelete, boolean __opAutoProjectionInJson) { + this.supportDelete = supportDelete; + if (columnList != null && !__opAutoProjectionInJson) { + StringJoiner joiner = new StringJoiner(","); + for (String columnName : columnList) { + String format = String.format("`%s`", columnName); + joiner.add(format); + } + String res = joiner.toString(); + if (res.length() > 0) { + res = res + String.format(",%s", "__op"); + } + httpHeadColumns = res; + } + } + + public String getHttpHeadColumns() { + return httpHeadColumns; + } + + public synchronized void clear() { + buffer.clear(); + batchCount = 0; + batchSize = 0; + label = initBatchLabel(); + } + + public void reGenerateLabel() { + label = initBatchLabel(); + } + + public String initBatchLabel() { + String formatDate = LocalDateTime.now().format(dateTimeFormatter); + return String.format( + "chunjun_connector_%s_%s", + formatDate, UUID.randomUUID().toString().replaceAll("-", "")); + } + + public String getIdentify() { + return database + "." + table; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksSinkOP.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksSinkOP.java new file mode 100644 index 0000000000..a71ae2eee5 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksSinkOP.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.streamload; + +import org.apache.flink.types.RowKind; + +/** StarRocks sink operator. */ +public enum StarRocksSinkOP { + UPSERT, + DELETE; + + public static final String COLUMN_KEY = "__op"; + + public static StarRocksSinkOP parse(RowKind kind) { + if (RowKind.INSERT.equals(kind) || RowKind.UPDATE_AFTER.equals(kind)) { + return UPSERT; + } + if (RowKind.DELETE.equals(kind) || RowKind.UPDATE_BEFORE.equals(kind)) { + return DELETE; + } + throw new RuntimeException("Unsupported row kind."); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksStreamLoadFailedException.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksStreamLoadFailedException.java new file mode 100644 index 0000000000..883e7e80d4 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksStreamLoadFailedException.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.streamload; + +import java.io.IOException; +import java.util.Map; + +public class StarRocksStreamLoadFailedException extends IOException { + + static final long serialVersionUID = 1L; + + private final Map response; + private boolean reCreateLabel; + private final StarRocksSinkBufferEntity entity; + + public StarRocksStreamLoadFailedException( + String message, Map response, StarRocksSinkBufferEntity entity) { + super(message); + this.response = response; + this.entity = entity; + } + + public StarRocksStreamLoadFailedException( + String message, + Map response, + StarRocksSinkBufferEntity entity, + boolean reCreateLabel) { + super(message); + this.response = response; + this.reCreateLabel = reCreateLabel; + this.entity = entity; + } + + public Map getFailedResponse() { + return response; + } + + public boolean needReCreateLabel() { + return reCreateLabel; + } + + public StarRocksSinkBufferEntity getEntity() { + return entity; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksStreamLoadVisitor.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksStreamLoadVisitor.java new file mode 100644 index 0000000000..cc31436497 --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StarRocksStreamLoadVisitor.java @@ -0,0 +1,330 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.streamload; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; + +import com.alibaba.fastjson.JSON; +import org.apache.commons.lang3.StringUtils; +import org.apache.http.HttpEntity; +import org.apache.http.client.config.RequestConfig; +import org.apache.http.client.methods.CloseableHttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.client.methods.HttpPut; +import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.DefaultRedirectStrategy; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.impl.client.HttpClients; +import org.apache.http.util.EntityUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.net.HttpURLConnection; +import java.net.URL; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static com.dtstack.chunjun.connector.starrocks.util.StarRocksUtil.getBasicAuthHeader; + +public class StarRocksStreamLoadVisitor implements Serializable { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(StarRocksStreamLoadVisitor.class); + + private static final int ERROR_LOG_MAX_LENGTH = 3000; + + private final StarRocksConf starRocksConf; + private long pos; + private static final String RESULT_FAILED = "Fail"; + private static final String RESULT_LABEL_EXISTED = "Label Already Exists"; + private static final String LAEBL_STATE_VISIBLE = "VISIBLE"; + private static final String LAEBL_STATE_COMMITTED = "COMMITTED"; + private static final String RESULT_LABEL_PREPARE = "PREPARE"; + private static final String RESULT_LABEL_ABORTED = "ABORTED"; + private static final String RESULT_LABEL_UNKNOWN = "UNKNOWN"; + + public StarRocksStreamLoadVisitor(StarRocksConf starRocksConf) { + this.starRocksConf = starRocksConf; + } + + public void doStreamLoad(StarRocksSinkBufferEntity bufferEntity) throws IOException { + String host = getAvailableHost(); + if (null == host) { + throw new IOException("None of the hosts in `load_url` could be connected."); + } + String loadUrl = + host + + "/api/" + + bufferEntity.getDatabase() + + "/" + + bufferEntity.getTable() + + "/_stream_load"; + LOG.info(String.format("Start to join batch data: label[%s].", bufferEntity.getLabel())); + Map loadResult = + doHttpPut( + loadUrl, + bufferEntity.getLabel(), + joinRows(bufferEntity.getBuffer(), (int) bufferEntity.getBatchSize()), + bufferEntity.getHttpHeadColumns()); + dealStreamLoadResult(host, bufferEntity, loadResult); + } + + private void dealStreamLoadResult( + String host, StarRocksSinkBufferEntity bufferEntity, Map loadResult) + throws IOException { + final String keyStatus = "Status"; + if (null == loadResult || !loadResult.containsKey(keyStatus)) { + throw new IOException( + "Unable to flush data to StarRocks: unknown result status, usually caused by: 1.authorization or permission related problems. 2.Wrong column_separator or row_delimiter. 3.Column count exceeded the limitation."); + } + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Stream Load response: \n%s\n", JSON.toJSONString(loadResult))); + } + if (RESULT_FAILED.equals(loadResult.get(keyStatus))) { + Map logMap = new HashMap<>(); + if (loadResult.containsKey("ErrorURL")) { + logMap.put("streamLoadErrorLog", getErrorLog((String) loadResult.get("ErrorURL"))); + } + throw new StarRocksStreamLoadFailedException( + String.format( + "Failed to flush data to StarRocks, Error " + "response: \n%s\n%s\n", + JSON.toJSONString(loadResult), JSON.toJSONString(logMap)), + loadResult, + bufferEntity); + } else if (RESULT_LABEL_EXISTED.equals(loadResult.get(keyStatus))) { + LOG.error(String.format("Stream Load response: \n%s\n", JSON.toJSONString(loadResult))); + // has to block-checking the state to get the final result + checkLabelState(host, bufferEntity); + } + } + + @SuppressWarnings("unchecked") + private void checkLabelState(String host, StarRocksSinkBufferEntity bufferEntity) + throws IOException { + int idx = 0; + while (true) { + try { + TimeUnit.SECONDS.sleep(Math.min(++idx, 5)); + } catch (InterruptedException ex) { + break; + } + try (CloseableHttpClient httpclient = HttpClients.createDefault()) { + HttpGet httpGet = + new HttpGet( + host + + "/api/" + + starRocksConf.getDatabase() + + "/get_load_state?label=" + + bufferEntity.getLabel()); + httpGet.setHeader( + "Authorization", + getBasicAuthHeader( + starRocksConf.getUsername(), starRocksConf.getPassword())); + httpGet.setHeader("Connection", "close"); + + try (CloseableHttpResponse resp = httpclient.execute(httpGet)) { + HttpEntity respEntity = getHttpEntity(resp); + if (respEntity == null) { + throw new StarRocksStreamLoadFailedException( + String.format( + "Failed to flush data to StarRocks, Error " + + "could not get the final state of label[%s].\n", + bufferEntity.getLabel()), + null, + bufferEntity); + } + Map result = + (Map) JSON.parse(EntityUtils.toString(respEntity)); + String labelState = (String) result.get("state"); + if (null == labelState) { + throw new StarRocksStreamLoadFailedException( + String.format( + "Failed to flush data to StarRocks, Error " + + "could not get the final state of label[%s]. response[%s]\n", + bufferEntity.getLabel(), EntityUtils.toString(respEntity)), + null, + bufferEntity); + } + LOG.info( + String.format( + "Checking label[%s] state[%s]\n", + bufferEntity.getLabel(), labelState)); + switch (labelState) { + case LAEBL_STATE_VISIBLE: + case LAEBL_STATE_COMMITTED: + return; + case RESULT_LABEL_PREPARE: + continue; + case RESULT_LABEL_ABORTED: + throw new StarRocksStreamLoadFailedException( + String.format( + "Failed to flush data to StarRocks, Error " + + "label[%s] state[%s]\n", + bufferEntity.getLabel(), labelState), + null, + bufferEntity, + true); + case RESULT_LABEL_UNKNOWN: + default: + throw new StarRocksStreamLoadFailedException( + String.format( + "Failed to flush data to StarRocks, Error " + + "label[%s] state[%s]\n", + bufferEntity.getLabel(), labelState), + null, + bufferEntity); + } + } + } + } + } + + private String getErrorLog(String errorUrl) { + if (errorUrl == null || !errorUrl.startsWith("http")) { + return null; + } + try (CloseableHttpClient httpclient = HttpClients.createDefault()) { + HttpGet httpGet = new HttpGet(errorUrl); + try (CloseableHttpResponse resp = httpclient.execute(httpGet)) { + HttpEntity respEntity = getHttpEntity(resp); + if (respEntity == null) { + return null; + } + String errorLog = EntityUtils.toString(respEntity); + if (errorLog != null && errorLog.length() > ERROR_LOG_MAX_LENGTH) { + errorLog = errorLog.substring(0, ERROR_LOG_MAX_LENGTH); + } + return errorLog; + } + } catch (Exception e) { + LOG.warn("Failed to get error log.", e); + return "Failed to get error log: " + e.getMessage(); + } + } + + private String getAvailableHost() { + List hostList = starRocksConf.getFeNodes(); + long tmp = pos + hostList.size(); + for (; pos < tmp; pos++) { + String host = "http://" + hostList.get((int) (pos % hostList.size())); + if (tryHttpConnection(host)) { + return host; + } + } + return null; + } + + private boolean tryHttpConnection(String host) { + try { + URL url = new URL(host); + HttpURLConnection co = (HttpURLConnection) url.openConnection(); + co.setConnectTimeout(starRocksConf.getLoadConf().getHttpCheckTimeoutMs()); + co.connect(); + co.disconnect(); + return true; + } catch (Exception e1) { + LOG.warn("Failed to connect to address:{}", host, e1); + return false; + } + } + + private byte[] joinRows(List rows, int totalBytes) { + ByteBuffer bos = ByteBuffer.allocate(totalBytes + (rows.isEmpty() ? 2 : 1 - rows.size())); + bos.put("[".getBytes(StandardCharsets.UTF_8)); + byte[] jsonDelimiter = ",".getBytes(StandardCharsets.UTF_8); + boolean isFirstElement = true; + for (byte[] row : rows) { + if (!isFirstElement) { + bos.put(jsonDelimiter); + } + bos.put(row, 1, row.length - 2); + isFirstElement = false; + } + bos.put("]".getBytes(StandardCharsets.UTF_8)); + return bos.array(); + } + + @SuppressWarnings("unchecked") + private Map doHttpPut( + String loadUrl, String label, byte[] data, String httpHeadColumns) throws IOException { + LOG.info( + String.format( + "Executing stream load to: '%s', size: '%s', thread: %d", + loadUrl, data.length, Thread.currentThread().getId())); + final HttpClientBuilder httpClientBuilder = + HttpClients.custom() + .setRedirectStrategy( + new DefaultRedirectStrategy() { + @Override + protected boolean isRedirectable(String method) { + return true; + } + }); + try (CloseableHttpClient httpclient = httpClientBuilder.build()) { + HttpPut httpPut = new HttpPut(loadUrl); + Map props = starRocksConf.getLoadConf().getHeadProperties(); + for (Map.Entry entry : props.entrySet()) { + httpPut.setHeader(entry.getKey(), entry.getValue()); + } + if (!props.containsKey("columns") && StringUtils.isNotBlank(httpHeadColumns)) { + httpPut.setHeader("columns", httpHeadColumns); + } + if (!httpPut.containsHeader("timeout")) { + httpPut.setHeader("timeout", "60"); + } + httpPut.setHeader("Expect", "100-continue"); + httpPut.setHeader("ignore_json_size", "true"); + httpPut.setHeader("strip_outer_array", "true"); + httpPut.setHeader("format", "json"); + httpPut.setHeader("label", label); + httpPut.setHeader( + "Authorization", + getBasicAuthHeader(starRocksConf.getUsername(), starRocksConf.getPassword())); + httpPut.setEntity(new ByteArrayEntity(data)); + httpPut.setConfig(RequestConfig.custom().setRedirectsEnabled(true).build()); + try (CloseableHttpResponse resp = httpclient.execute(httpPut)) { + HttpEntity respEntity = getHttpEntity(resp); + if (respEntity == null) return null; + return (Map) JSON.parse(EntityUtils.toString(respEntity)); + } + } + } + + private HttpEntity getHttpEntity(CloseableHttpResponse resp) { + int code = resp.getStatusLine().getStatusCode(); + if (200 != code) { + LOG.warn("Request failed with code:{}", code); + return null; + } + HttpEntity respEntity = resp.getEntity(); + if (null == respEntity) { + LOG.warn("Request failed with empty response."); + return null; + } + return respEntity; + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StreamLoadManager.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StreamLoadManager.java new file mode 100644 index 0000000000..54dd12be0b --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/streamload/StreamLoadManager.java @@ -0,0 +1,293 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES 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.dtstack.chunjun.connector.starrocks.streamload; + +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; + +import com.alibaba.fastjson.JSON; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; + +/** @author liuliu 2022/7/12 */ +public class StreamLoadManager { + final LinkedBlockingDeque flushQueue = new LinkedBlockingDeque<>(1); + private final Map bufferMap = new ConcurrentHashMap<>(); + + private final StarRocksConf starRocksConf; + private final boolean __opAutoProjectionInJson; + + private final StarRocksQueryVisitor starrocksQueryVisitor; + private final StarRocksStreamLoadVisitor starrocksStreamLoadVisitor; + + private volatile boolean closed = false; + private volatile Throwable flushException; + + private final Logger LOG = LoggerFactory.getLogger(StreamLoadManager.class); + + public StreamLoadManager(StarRocksConf starRocksConf) { + this.starRocksConf = starRocksConf; + this.starrocksQueryVisitor = new StarRocksQueryVisitor(starRocksConf); + + String version = starrocksQueryVisitor.getStarRocksVersion(); + __opAutoProjectionInJson = version.length() > 0 && !version.trim().startsWith("1."); + this.starrocksStreamLoadVisitor = new StarRocksStreamLoadVisitor(starRocksConf); + } + + public void write(String tableIdentify, List columnList, List> data) + throws Exception { + try { + checkFlushException(); + StarRocksSinkBufferEntity bufferEntity = + bufferMap.computeIfAbsent( + tableIdentify, + f -> { + String[] databaseAndTable = tableIdentify.split("\\."); + StarRocksSinkBufferEntity starRocksSinkBufferEntity = + new StarRocksSinkBufferEntity( + databaseAndTable[0], + databaseAndTable[1], + columnList); + validateTableStructure(starRocksSinkBufferEntity); + return starRocksSinkBufferEntity; + }); + bufferEntity.addToBuffer( + JSON.toJSONString(data).getBytes(StandardCharsets.UTF_8), data.size()); + if (bufferEntity.getBatchCount() >= starRocksConf.getLoadConf().getBatchMaxRows() + || bufferEntity.getBatchSize() + >= starRocksConf.getLoadConf().getBatchMaxSize()) { + LOG.info( + String.format( + "StarRocks buffer Sinking triggered: tableIdentify[%s] rows[%d] label[%s].", + tableIdentify, + bufferEntity.getBatchCount(), + bufferEntity.getLabel())); + flush(tableIdentify, false); + } + } catch (Exception e) { + throw new Exception("Writing records to StarRocks failed.", e); + } + } + + public void validateTableStructure(StarRocksSinkBufferEntity entity) { + if (starRocksConf.getLoadConf().getHeadProperties().containsKey("columns")) { + return; + } + List> rows = starrocksQueryVisitor.getTableColumnsMetaData(); + if (rows == null || rows.isEmpty()) { + throw new IllegalArgumentException("Couldn't get the sink table's column info."); + } + // validate primary keys + List primaryKeyList = new ArrayList<>(); + Set containedColumnNameSet = new HashSet<>(); + for (Map row : rows) { + String keysType = row.get("COLUMN_KEY").toString(); + String column_name = row.get("COLUMN_NAME").toString(); + if (entity.getColumnList().stream().anyMatch(cn -> cn.equalsIgnoreCase(column_name))) { + containedColumnNameSet.add(column_name); + } + if ("PRI".equals(keysType)) { + primaryKeyList.add(column_name.toLowerCase()); + } + } + if (!primaryKeyList.isEmpty()) { + entity.setSupportDelete( + new HashSet<>(entity.getColumnList()).containsAll(primaryKeyList), + __opAutoProjectionInJson); + } + if (containedColumnNameSet.size() != entity.getColumnList().size()) { + throw new IllegalArgumentException( + String.format( + "The columnList:%s contains columns that do not exist in the corresponding table[%s]", + entity.getColumnList(), + String.format("%s.%s", entity.getDatabase(), entity.getTable()))); + } + } + + public void flush(String bufferKey, boolean waitUtilDone) throws Exception { + if (bufferMap.isEmpty()) { + flushInternal(null, waitUtilDone); + return; + } + if (null == bufferKey) { + for (String key : bufferMap.keySet()) { + flushInternal(key, waitUtilDone); + } + return; + } + flushInternal(bufferKey, waitUtilDone); + } + + private synchronized void flushInternal(String bufferKey, boolean waitUtilDone) + throws Exception { + checkFlushException(); + if (null == bufferKey || bufferMap.isEmpty() || !bufferMap.containsKey(bufferKey)) { + if (waitUtilDone) { + waitAsyncFlushingDone(); + } + return; + } + offer(bufferMap.get(bufferKey)); + bufferMap.remove(bufferKey); + if (waitUtilDone) { + // wait the last flush + waitAsyncFlushingDone(); + } + } + + private void waitAsyncFlushingDone() throws InterruptedException { + // wait for previous flushings + offer(new StarRocksSinkBufferEntity(null, null, null)); + checkFlushException(); + } + + void offer(StarRocksSinkBufferEntity bufferEntity) throws InterruptedException { + if (!flushQueue.offer( + bufferEntity, + starRocksConf.getLoadConf().getQueueOfferTimeoutMs(), + TimeUnit.MILLISECONDS)) { + throw new RuntimeException("Timeout while offering data to flushQueue"); + } + } + + private void checkFlushException() { + if (flushException != null) { + StackTraceElement[] stack = Thread.currentThread().getStackTrace(); + for (StackTraceElement stackTraceElement : stack) { + LOG.info( + stackTraceElement.getClassName() + + "." + + stackTraceElement.getMethodName() + + " line:" + + stackTraceElement.getLineNumber()); + } + throw new RuntimeException("Writing records to StarRocks failed.", flushException); + } + } + + public void startAsyncFlushing() { + // start flush thread + Thread flushThread = + new Thread( + () -> { + while (true) { + try { + if (!asyncFlush()) { + LOG.info("StarRocks flush failed."); + break; + } + } catch (Exception e) { + flushException = e; + } + } + }); + + flushThread.setUncaughtExceptionHandler( + (t, e) -> { + LOG.error( + "StarRocks flush thread uncaught exception occurred: " + e.getMessage(), + e); + flushException = e; + }); + flushThread.setName("chunjun-starrocks-flush"); + flushThread.setDaemon(true); + flushThread.start(); + } + + /** @return false if met eof and flush thread will exit. */ + private boolean asyncFlush() throws Exception { + StarRocksSinkBufferEntity flushData = + flushQueue.poll( + starRocksConf.getLoadConf().getQueuePollTimeoutMs(), TimeUnit.MILLISECONDS); + if (flushData == null || 0 == flushData.getBatchCount()) { + return true; + } + LOG.info( + String.format( + "Async stream load: db[%s] table[%s] rows[%d] bytes[%d] label[%s].", + flushData.getDatabase(), + flushData.getTable(), + flushData.getBatchCount(), + flushData.getBatchSize(), + flushData.getLabel())); + for (int i = 0; i <= starRocksConf.getMaxRetries(); i++) { + try { + starrocksStreamLoadVisitor.doStreamLoad(flushData); + LOG.info( + String.format( + "Async stream load finished: label[%s].", flushData.getLabel())); + break; + } catch (Exception e) { + LOG.warn("Failed to flush batch data to StarRocks, retry times = {}", i, e); + if (i >= starRocksConf.getMaxRetries()) { + throw e; + } + if (e instanceof StarRocksStreamLoadFailedException + && ((StarRocksStreamLoadFailedException) e).needReCreateLabel()) { + String oldLabel = flushData.getLabel(); + flushData.reGenerateLabel(); + LOG.warn( + String.format( + "Batch label changed from [%s] to [%s]", + oldLabel, flushData.getLabel())); + } + try { + Thread.sleep(1000L * Math.min(i + 1, 10)); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw new IOException( + "Unable to flush, interrupted while doing another attempt", e); + } + } + } + return true; + } + + public synchronized void close() { + if (!closed) { + closed = true; + + if (starrocksQueryVisitor != null) { + starrocksQueryVisitor.close(); + } + + if (flushException != null) { + checkFlushException(); + return; + } + try { + LOG.info("StarRocks Sink is about to close."); + flush(null, true); + } catch (Exception e) { + throw new RuntimeException("Writing records to StarRocks failed.", e); + } + } + checkFlushException(); + } +} diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/table/StarRocksDynamicTableFactory.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/table/StarRocksDynamicTableFactory.java index 5797a38475..f016b79e68 100644 --- a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/table/StarRocksDynamicTableFactory.java +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/table/StarRocksDynamicTableFactory.java @@ -17,18 +17,232 @@ */ package com.dtstack.chunjun.connector.starrocks.table; -import com.starrocks.connector.flink.table.StarRocksDynamicTableSinkFactory; +import com.dtstack.chunjun.connector.starrocks.conf.LoadConf; +import com.dtstack.chunjun.connector.starrocks.conf.LoadConfBuilder; +import com.dtstack.chunjun.connector.starrocks.conf.StarRocksConf; +import com.dtstack.chunjun.connector.starrocks.sink.StarRocksDynamicTableSink; +import com.dtstack.chunjun.connector.starrocks.source.StarRocksDynamicTableSource; +import com.dtstack.chunjun.lookup.conf.LookupConf; +import com.dtstack.chunjun.lookup.conf.LookupConfFactory; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.factories.FactoryUtil; +import org.apache.flink.table.utils.TableSchemaUtils; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksCommonOptions.FENODES; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksCommonOptions.MAX_RETRIES; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksCommonOptions.PASSWORD; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksCommonOptions.SCHEMA_NAME; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksCommonOptions.TABLE_NAME; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksCommonOptions.URL; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksCommonOptions.USERNAME; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksSinkOptions.NAME_MAPPED; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksSinkOptions.SINK_BUFFER_FLUSH_MAX_ROWS; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksSinkOptions.SINK_SEMANTIC; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksSourceOptions.FILTER_STATEMENT; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksSourceOptions.SCAN_BE_CLIENT_KEEP_LIVE_MIN; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksSourceOptions.SCAN_BE_CLIENT_TIMEOUT; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksSourceOptions.SCAN_BE_FETCH_BYTES_LIMIT; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksSourceOptions.SCAN_BE_FETCH_ROWS; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksSourceOptions.SCAN_BE_PARAM_PROPERTIES; +import static com.dtstack.chunjun.connector.starrocks.options.StarRocksSourceOptions.SCAN_BE_QUERY_TIMEOUT_S; +import static com.dtstack.chunjun.connector.starrocks.options.StreamLoadOptions.HTTP_CHECK_TIMEOUT; +import static com.dtstack.chunjun.connector.starrocks.options.StreamLoadOptions.QUEUE_OFFER_TIMEOUT; +import static com.dtstack.chunjun.connector.starrocks.options.StreamLoadOptions.QUEUE_POLL_TIMEOUT; +import static com.dtstack.chunjun.connector.starrocks.options.StreamLoadOptions.SINK_BATCH_MAX_BYTES; +import static com.dtstack.chunjun.connector.starrocks.options.StreamLoadOptions.SINK_BATCH_MAX_ROWS; +import static com.dtstack.chunjun.connector.starrocks.options.StreamLoadOptions.STREAM_LOAD_HEAD_PROPERTIES; +import static com.dtstack.chunjun.lookup.options.LookupOptions.LOOKUP_ASYNC_TIMEOUT; +import static com.dtstack.chunjun.lookup.options.LookupOptions.LOOKUP_CACHE_MAX_ROWS; +import static com.dtstack.chunjun.lookup.options.LookupOptions.LOOKUP_CACHE_PERIOD; +import static com.dtstack.chunjun.lookup.options.LookupOptions.LOOKUP_CACHE_TTL; +import static com.dtstack.chunjun.lookup.options.LookupOptions.LOOKUP_CACHE_TYPE; +import static com.dtstack.chunjun.lookup.options.LookupOptions.LOOKUP_ERROR_LIMIT; +import static com.dtstack.chunjun.lookup.options.LookupOptions.LOOKUP_FETCH_SIZE; +import static com.dtstack.chunjun.lookup.options.LookupOptions.LOOKUP_MAX_RETRIES; +import static com.dtstack.chunjun.lookup.options.LookupOptions.LOOKUP_PARALLELISM; +import static com.dtstack.chunjun.table.options.SinkOptions.SINK_BUFFER_FLUSH_INTERVAL; /** * @author lihongwei * @date 2022/04/11 */ -public class StarRocksDynamicTableFactory extends StarRocksDynamicTableSinkFactory { +public class StarRocksDynamicTableFactory + implements DynamicTableSourceFactory, DynamicTableSinkFactory { private static final String IDENTIFIER = "starrocks-x"; + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + TableSchema physicalSchema = + TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); + + return new StarRocksDynamicTableSource( + createSourceConfByOptions(helper.getOptions(), physicalSchema), + createLookupConfByOptions(helper.getOptions()), + physicalSchema); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + final FactoryUtil.TableFactoryHelper helper = + FactoryUtil.createTableFactoryHelper(this, context); + ReadableConfig options = helper.getOptions(); + StarRocksConf sinkConf = createSinkConfByOptions(options); + TableSchema physicalSchema = + TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); + return new StarRocksDynamicTableSink(sinkConf, physicalSchema); + } + @Override public String factoryIdentifier() { return IDENTIFIER; } + + private StarRocksConf createSourceConfByOptions( + ReadableConfig options, TableSchema tableSchema) { + StarRocksConf sourceConf = createCommonConfByOptions(options); + // source options + String filterStatement = options.get(FILTER_STATEMENT); + Integer beClientKeepLiveMin = options.get(SCAN_BE_CLIENT_KEEP_LIVE_MIN); + Integer beQueryTimeoutSecond = options.get(SCAN_BE_QUERY_TIMEOUT_S); + Integer beClientTimeout = options.get(SCAN_BE_CLIENT_TIMEOUT); + Integer beFetchRows = options.get(SCAN_BE_FETCH_ROWS); + Long beFetchMaxBytes = options.get(SCAN_BE_FETCH_BYTES_LIMIT); + Map beSocketProperties = options.get(SCAN_BE_PARAM_PROPERTIES); + + // loading + sourceConf.setFilterStatement(filterStatement); + sourceConf.setBeClientKeepLiveMin(beClientKeepLiveMin); + sourceConf.setBeQueryTimeoutSecond(beQueryTimeoutSecond); + sourceConf.setBeClientTimeout(beClientTimeout); + sourceConf.setBeFetchRows(beFetchRows); + sourceConf.setBeFetchMaxBytes(beFetchMaxBytes); + sourceConf.setBeSocketProperties(beSocketProperties); + + sourceConf.setFieldNames(tableSchema.getFieldNames()); + sourceConf.setDataTypes(tableSchema.getFieldDataTypes()); + + return sourceConf; + } + + private LookupConf createLookupConfByOptions(ReadableConfig options) { + return LookupConfFactory.createLookupConf(options); + } + + private StarRocksConf createSinkConfByOptions(ReadableConfig options) { + StarRocksConf sinkConf = createCommonConfByOptions(options); + // sink options + boolean nameMapped = options.get(NAME_MAPPED); + Integer batchSize = options.get(SINK_BUFFER_FLUSH_MAX_ROWS); + Long sinkInternal = options.get(SINK_BUFFER_FLUSH_INTERVAL); + LoadConf loadConf = getLoadConf(options); + // loading + sinkConf.setNameMapped(nameMapped); + sinkConf.setBatchSize(batchSize); + sinkConf.setFlushIntervalMills(sinkInternal); + sinkConf.setLoadConf(loadConf); + return sinkConf; + } + + private LoadConf getLoadConf(ReadableConfig options) { + LoadConfBuilder loadConfBuilder = new LoadConfBuilder(); + return loadConfBuilder + .setBatchMaxSize(options.get(SINK_BATCH_MAX_BYTES)) + .setBatchMaxRows(options.get(SINK_BATCH_MAX_ROWS)) + .setHttpCheckTimeoutMs(options.get(HTTP_CHECK_TIMEOUT)) + .setQueueOfferTimeoutMs(options.get(QUEUE_OFFER_TIMEOUT)) + .setQueuePollTimeoutMs(options.get(QUEUE_POLL_TIMEOUT)) + .setHeadProperties(options.get(STREAM_LOAD_HEAD_PROPERTIES)) + .build(); + } + + protected StarRocksConf createCommonConfByOptions(ReadableConfig options) { + StarRocksConf conf = new StarRocksConf(); + // common options + String url = options.get(URL); + List feNodes = options.get(FENODES); + String database = options.get(SCHEMA_NAME); + String tableName = options.get(TABLE_NAME); + String username = options.get(USERNAME); + String password = options.get(PASSWORD); + Integer maxRetries = options.get(MAX_RETRIES); + // loading + conf.setUrl(url); + conf.setFeNodes(feNodes); + conf.setDatabase(database); + conf.setTable(tableName); + conf.setUsername(username); + conf.setPassword(password); + conf.setMaxRetries(maxRetries); + return conf; + } + + @Override + public Set> requiredOptions() { + Set> requiredOptions = new HashSet<>(); + requiredOptions.add(URL); + requiredOptions.add(FENODES); + requiredOptions.add(SCHEMA_NAME); + requiredOptions.add(TABLE_NAME); + requiredOptions.add(USERNAME); + requiredOptions.add(PASSWORD); + return requiredOptions; + } + + @Override + public Set> optionalOptions() { + Set> optionalOptions = new HashSet<>(); + + // common + optionalOptions.add(MAX_RETRIES); + + // source + optionalOptions.add(FILTER_STATEMENT); + optionalOptions.add(SCAN_BE_CLIENT_KEEP_LIVE_MIN); + optionalOptions.add(SCAN_BE_QUERY_TIMEOUT_S); + optionalOptions.add(SCAN_BE_CLIENT_TIMEOUT); + optionalOptions.add(SCAN_BE_FETCH_ROWS); + optionalOptions.add(SCAN_BE_FETCH_BYTES_LIMIT); + optionalOptions.add(SCAN_BE_PARAM_PROPERTIES); + + // lookup + optionalOptions.add(LOOKUP_CACHE_PERIOD); + optionalOptions.add(LOOKUP_CACHE_MAX_ROWS); + optionalOptions.add(LOOKUP_CACHE_TTL); + optionalOptions.add(LOOKUP_CACHE_TYPE); + optionalOptions.add(LOOKUP_MAX_RETRIES); + optionalOptions.add(LOOKUP_ERROR_LIMIT); + optionalOptions.add(LOOKUP_FETCH_SIZE); + optionalOptions.add(LOOKUP_ASYNC_TIMEOUT); + optionalOptions.add(LOOKUP_PARALLELISM); + + // sink + optionalOptions.add(NAME_MAPPED); + optionalOptions.add(SINK_BUFFER_FLUSH_MAX_ROWS); + optionalOptions.add(SINK_BUFFER_FLUSH_INTERVAL); + optionalOptions.add(SINK_SEMANTIC); + + // stream load + optionalOptions.add(SINK_BATCH_MAX_ROWS); + optionalOptions.add(SINK_BATCH_MAX_BYTES); + optionalOptions.add(HTTP_CHECK_TIMEOUT); + optionalOptions.add(QUEUE_OFFER_TIMEOUT); + optionalOptions.add(QUEUE_POLL_TIMEOUT); + optionalOptions.add(STREAM_LOAD_HEAD_PROPERTIES); + return optionalOptions; + } } diff --git a/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/util/StarRocksUtil.java b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/util/StarRocksUtil.java new file mode 100644 index 0000000000..cc19847a3f --- /dev/null +++ b/chunjun-connectors/chunjun-connector-starrocks/src/main/java/com/dtstack/chunjun/connector/starrocks/util/StarRocksUtil.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package com.dtstack.chunjun.connector.starrocks.util; + +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryBeXTablets; +import com.dtstack.chunjun.connector.starrocks.source.be.entity.QueryInfo; + +import org.apache.flink.api.java.tuple.Tuple2; + +import org.apache.commons.codec.binary.Base64; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** @author liuliu 2022/7/20 */ +public class StarRocksUtil { + + public static String getBasicAuthHeader(String username, String password) { + String auth = username + ":" + password; + byte[] encodedAuth = Base64.encodeBase64(auth.getBytes(StandardCharsets.UTF_8)); + return "Basic " + new String(encodedAuth); + } + + public static List> splitQueryBeXTablets( + int subTaskCount, QueryInfo queryInfo) { + List> curBeXTabletList = new ArrayList<>(); + for (int i = 0; i < subTaskCount; i++) { + curBeXTabletList.add(new ArrayList<>()); + } + int beXTabletsListCount = queryInfo.getBeXTablets().size(); + /* subTaskCount <= beXTabletsListCount */ + if (subTaskCount <= beXTabletsListCount) { + for (int i = 0; i < beXTabletsListCount; i++) { + curBeXTabletList.get(i % subTaskCount).add(queryInfo.getBeXTablets().get(i)); + } + return curBeXTabletList; + } + /* subTaskCount > beXTabletsListCount */ + // split to singleTabletList + List> beXTabletTupleList = new ArrayList<>(); + for (QueryBeXTablets queryBeXTablets : queryInfo.getBeXTablets()) { + for (Long tabletId : queryBeXTablets.getTabletIds()) { + beXTabletTupleList.add(new Tuple2<>(queryBeXTablets.getBeNode(), tabletId)); + } + } + int stepLen = beXTabletTupleList.size() / subTaskCount; + // beWithSingleTabletList.size()<=subTaskCount + if (stepLen <= 1) { + for (int i = 0; i < beXTabletTupleList.size(); i++) { + Tuple2 beXTabletTuple = beXTabletTupleList.get(i); + curBeXTabletList.set( + i, + Collections.singletonList( + new QueryBeXTablets( + beXTabletTuple.f0, + Collections.singletonList(beXTabletTuple.f1)))); + } + return curBeXTabletList; + } + // beWithSingleTabletList.size()>subTaskCount + int remain = beXTabletTupleList.size() % subTaskCount; + int end = 0; + for (int i = 0; i < subTaskCount; i++) { + int start = end; + end += stepLen; + if (i < remain) { + end += 1; + } + List> curBeTabletTupleList = + beXTabletTupleList.subList(start, end); + Map> beXTabletsMap = new HashMap<>(); + curBeTabletTupleList.forEach( + curBeTabletTuple -> { + List curTabletList = + beXTabletsMap.getOrDefault(curBeTabletTuple.f0, new ArrayList<>()); + curTabletList.add(curBeTabletTuple.f1); + beXTabletsMap.put(curBeTabletTuple.f0, curTabletList); + }); + + curBeXTabletList.set( + i, + beXTabletsMap.entrySet().stream() + .map(entry -> new QueryBeXTablets(entry.getKey(), entry.getValue())) + .collect(Collectors.toList())); + } + return curBeXTabletList; + } + + public static String addStrForNum(String str, int strLength, String appendStr) { + int strLen = str.length(); + if (strLen < strLength) { + StringBuilder strBuilder = new StringBuilder(str); + while (strLen < strLength) { + strBuilder.append(appendStr); + strLen = strBuilder.length(); + } + str = strBuilder.toString(); + } + return str; + } +} diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/sink/format/BaseRichOutputFormat.java b/chunjun-core/src/main/java/com/dtstack/chunjun/sink/format/BaseRichOutputFormat.java index db2aa489ea..1d12b55f72 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/sink/format/BaseRichOutputFormat.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/sink/format/BaseRichOutputFormat.java @@ -492,7 +492,7 @@ protected synchronized void writeRecordInternal() { } } - private void checkTimerWriteException() { + protected void checkTimerWriteException() { if (null != timerWriteException) { if (timerWriteException instanceof NoRestartException) { throw (NoRestartException) timerWriteException; diff --git a/chunjun-core/src/main/java/com/dtstack/chunjun/util/ConnectorNameConvertUtil.java b/chunjun-core/src/main/java/com/dtstack/chunjun/util/ConnectorNameConvertUtil.java index ca3677e494..432e3a2634 100644 --- a/chunjun-core/src/main/java/com/dtstack/chunjun/util/ConnectorNameConvertUtil.java +++ b/chunjun-core/src/main/java/com/dtstack/chunjun/util/ConnectorNameConvertUtil.java @@ -41,6 +41,7 @@ public class ConnectorNameConvertUtil { connectorNameMap.put("dorisbatch", new Tuple2<>("doris", "doris")); connectorNameMap.put("gbase", new Tuple2<>("gBase", "gBase")); connectorNameMap.put("protobuf", new Tuple2<>("pbformat", "pbformat")); + connectorNameMap.put("starrocks", new Tuple2<>("starrocks", "starRocks")); } public static String convertClassPrefix(String originName) { diff --git a/chunjun-examples/json/starrocks/mysql_starrocks.json b/chunjun-examples/json/starrocks/mysql_starrocks.json deleted file mode 100644 index 57672c94d4..0000000000 --- a/chunjun-examples/json/starrocks/mysql_starrocks.json +++ /dev/null @@ -1,64 +0,0 @@ -{ - "job": { - "content": [ - { - "reader": { - "name": "mysqlreader", - "parameter": { - "column": [ - {"name":"appkey", "type":"varchar"}, - {"name":"day_id", "type":"int"}, - {"name":"channel", "type":"varchar"}, - {"name":"pv", "type":"int"}, - {"name":"uv", "type":"int"}, - {"name":"retry_done", "type":"int", "value": "1"} - ], - "username": "root", - "password": "root", - "connection": [ - { - "jdbcUrl": ["jdbc:mysql://localhost:3306/test?useUnicode=true&characterEncoding=utf8&useSSL=false"], - "table": ["starrocks_test"] - } - ] - } - }, - "writer": { - "name": "starrockswriter", - "parameter": { - "username": "root", - "password": "root", - "loadUrl" : "localhost:8030", - "primaryKey": [], - "connection": [ - { - "jdbcUrl": "jdbc:mysql://localhost:9030/test?useUnicode=true&characterEncoding=utf8&useSSL=false&rewriteBatchedStatements=true&&serverTimezone=Asia/Shanghai", - "table": ["starrocks_sink_test"], - "schema":"test" - } - ], - "writeMode": "insert", - "column": [ - {"name":"name", "type":"varchar"}, - {"name":"type", "type":"int"}, - {"name":"attr", "type":"varchar"}, - {"name":"score", "type":"int"}, - {"name":"quarter", "type":"int"}, - {"name":"num", "type":"int"} - ], - "preSql":[ - "truncate table starrocks_sink_test;" - ], - "postSql":[] - } - } - } - ], - "setting": { - "speed": { - "channel": 1, - "bytes": 0 - } - } - } - } diff --git a/chunjun-examples/json/starrocks/starrocks_starrocks.json b/chunjun-examples/json/starrocks/starrocks_starrocks.json new file mode 100644 index 0000000000..c1efc8d3c7 --- /dev/null +++ b/chunjun-examples/json/starrocks/starrocks_starrocks.json @@ -0,0 +1,131 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "url": "jdbc:mysql://node01:9030", + "feNodes": [ + "node01:8030", + "node02:8030", + "node03:8030" + ], + "database": "test", + "table": "source", + "username": "root", + "password": "", + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "boolean_data", + "type": "boolean" + }, + { + "name": "tinyint_data", + "type": "tinyint" + }, + { + "name": "smallint_data", + "type": "smallint" + }, + { + "name": "integer_data", + "type": "int" + }, + { + "name": "float_data", + "type": "float" + }, + { + "name": "double_data", + "type": "double" + }, + { + "name": "decimal_data", + "type": "decimal" + }, + { + "name": "date_data", + "type": "date" + }, + { + "name": "datetime_data", + "type": "datetime" + } + ] + }, + "name": "starrocksreader" + }, + "writer": { + "parameter": { + "url": "jdbc:mysql://node01:9030", + "feNodes": [ + "node01:8030", + "node02:8030", + "node03:8030" + ], + "database": "test", + "table": "sink", + "username": "root", + "password": "", + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "boolean_data", + "type": "boolean" + }, + { + "name": "tinyint_data", + "type": "tinyint" + }, + { + "name": "smallint_data", + "type": "smallint" + }, + { + "name": "integer_data", + "type": "int" + }, + { + "name": "float_data", + "type": "float" + }, + { + "name": "double_data", + "type": "double" + }, + { + "name": "decimal_data", + "type": "decimal" + }, + { + "name": "date_data", + "type": "date" + }, + { + "name": "datetime_data", + "type": "datetime" + } + ] + }, + "name": "starrockswriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false, + "isStream": true + }, + "speed": { + "channel": 1 + } + } + } +} diff --git a/chunjun-examples/json/starrocks/starrocks_stream.json b/chunjun-examples/json/starrocks/starrocks_stream.json new file mode 100644 index 0000000000..48cbe79286 --- /dev/null +++ b/chunjun-examples/json/starrocks/starrocks_stream.json @@ -0,0 +1,80 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "url": "jdbc:mysql://node01:9030", + "feNodes": [ + "node01:8030", + "node02:8030", + "node03:8030" + ], + "database": "test", + "table": "source", + "username": "root", + "password": "", + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "boolean_data", + "type": "boolean" + }, + { + "name": "tinyint_data", + "type": "tinyint" + }, + { + "name": "smallint_data", + "type": "smallint" + }, + { + "name": "integer_data", + "type": "int" + }, + { + "name": "float_data", + "type": "float" + }, + { + "name": "double_data", + "type": "double" + }, + { + "name": "decimal_data", + "type": "decimal" + }, + { + "name": "date_data", + "type": "date" + }, + { + "name": "datetime_data", + "type": "datetime" + } + ] + }, + "name": "starrocksreader" + }, + "writer": { + "parameter": { + "print": false + }, + "name": "streamwriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false, + "isStream": true + }, + "speed": { + "channel": 1 + } + } + } +} diff --git a/chunjun-examples/json/starrocks/stream_starrocks.json b/chunjun-examples/json/starrocks/stream_starrocks.json new file mode 100644 index 0000000000..604ff17b0c --- /dev/null +++ b/chunjun-examples/json/starrocks/stream_starrocks.json @@ -0,0 +1,140 @@ +{ + "job": { + "content": [ + { + "reader": { + "parameter": { + "sliceRecordCount": [ + 10000000 + ], + "column": [ + { + "name": "id", + "type": "id" + }, + { + "name": "boolean_data", + "type": "boolean" + }, + { + "name": "tinyint_data", + "type": "tinyint" + }, + { + "name": "smallint_data", + "type": "short" + }, + { + "name": "integer_data", + "type": "int" + }, + { + "name": "bigint_data", + "type": "bigint" + }, + { + "name": "float_data", + "type": "float" + }, + { + "name": "double_data", + "type": "double" + }, + { + "name": "decimal_data", + "type": "decimal" + }, + { + "name": "string_data", + "type": "string" + }, + { + "name": "date_data", + "type": "date" + }, + { + "name": "datetime_data", + "type": "datetime" + } + ] + }, + "name": "streamreader" + }, + "writer": { + "parameter": { + "url": "jdbc:mysql://node01:9030", + "feNodes": [ + "node01:8030", + "node02:8030", + "node03:8030" + ], + "database": "test", + "table": "sink", + "username": "root", + "password": "", + "column": [ + { + "name": "id", + "type": "int" + }, + { + "name": "boolean_data", + "type": "boolean" + }, + { + "name": "tinyint_data", + "type": "tinyint" + }, + { + "name": "smallint_data", + "type": "smallint" + }, + { + "name": "integer_data", + "type": "int" + }, + { + "name": "bigint_data", + "type": "bigint" + }, + { + "name": "float_data", + "type": "float" + }, + { + "name": "double_data", + "type": "double" + }, + { + "name": "decimal_data", + "type": "decimal" + }, + { + "name": "string_data", + "type": "string" + }, + { + "name": "date_data", + "type": "date" + }, + { + "name": "datetime_data", + "type": "datetime" + } + ] + }, + "name": "starrockswriter" + } + } + ], + "setting": { + "restore": { + "isRestore": false, + "isStream": true + }, + "speed": { + "channel": 1 + } + } + } +} diff --git a/chunjun-examples/sql/starrocks/mysql_starrocks.sql b/chunjun-examples/sql/starrocks/mysql_starrocks.sql deleted file mode 100644 index f797577ea7..0000000000 --- a/chunjun-examples/sql/starrocks/mysql_starrocks.sql +++ /dev/null @@ -1,37 +0,0 @@ -CREATE TABLE source -( - appkey varchar, - day_id int, - channel varchar, - pv int, - uv int, - retry_done int -) WITH ( - 'connector' = 'mysql-x' - ,'url' = 'jdbc:mysql://localhost:3306/test?useUnicode=true&characterEncoding=utf8&useSSL=false' - ,'table-name' = 'starrocks_test' - ,'username' = 'root' - ,'password' = 'root' - ); - -CREATE TABLE sink -( - name varchar, - type int, - attr varchar, - score int, - quarter int, - num int -) WITH ( - 'connector' = 'starrocks-x' - ,'jdbc-url'='jdbc:mysql://localhost:9030/test?useUnicode=true&characterEncoding=utf8&useSSL=false&rewriteBatchedStatements=true&&serverTimezone=Asia/Shanghai' - ,'load-url'='localhost' - ,'database-name' = 'test' - ,'table-name' = 'starrocks_sink_test' - ,'username' = 'root' - ,'password' = 'root' - ); - -insert into sink -select appkey,day_id,channel,pv,uv, 1 as retry_done -from source; diff --git a/chunjun-examples/sql/starrocks/starrocks_lookup.sql b/chunjun-examples/sql/starrocks/starrocks_lookup.sql new file mode 100644 index 0000000000..c174a13600 --- /dev/null +++ b/chunjun-examples/sql/starrocks/starrocks_lookup.sql @@ -0,0 +1,49 @@ +CREATE TABLE table1 +( + id int, + name string, + date_data date, + datetime_data timestamp(0), + proctime AS PROCTIME() +) with ( + 'connector' = 'mysql-x', + 'url' = 'jdbc:mysql://localhost:3306/test?useSSL=false&allowPublicKeyRetrieval=true', + 'schema-name' = 'test', + 'table-name' = 'source', + 'username' = 'root', + 'password' = 'root' + ); + +CREATE TABLE side +( + id int, + date_data date, + datetime_data timestamp(0) +) with ( + 'connector' = 'starrocks-x', + 'url' = 'jdbc:mysql://node1:9030', + 'fe-nodes' = 'node1:8030;node2:8030;node3:8030', + 'schema-name' = 'test', + 'table-name' = 'side', + 'lookup.cache-type' = 'ALL', + 'username' = 'root', + 'password' = '' + ); + +CREATE TABLE sink +( + id int PRIMARY KEY, + date_data1 date , + date_data2 date +) WITH ( + 'connector' = 'stream-x' + ); + +insert into sink +select u.id + , u.date_data + , s.date_data +from table1 u + left join side FOR SYSTEM_TIME AS OF u.proctime AS s + on u.id = s.id and u.datetime_data=s.datetime_data and u.date_data = s.date_data; + diff --git a/chunjun-examples/sql/starrocks/starrocks_starrocks.sql b/chunjun-examples/sql/starrocks/starrocks_starrocks.sql new file mode 100644 index 0000000000..e5e4375c93 --- /dev/null +++ b/chunjun-examples/sql/starrocks/starrocks_starrocks.sql @@ -0,0 +1,52 @@ +CREATE TABLE source +( + id int, + boolean_data boolean, + tinyint_data tinyint, + smallint_data smallint, + integer_data integer, + bigint_data bigint, + float_data float, + double_data double, + decimal_data decimal, + string_data string, + date_data date, + datetime_data timestamp(0) +) with ( + 'connector' = 'starrocks-x', + 'url' = 'jdbc:mysql://node1:9030', + 'fe-nodes' = 'node1:8030;node2:8030;node3:8030', + 'schema-name' = 'test', + 'table-name' = 'source', + 'username' = 'root', + 'password' = '' + ); + + +CREATE TABLE sink +( + id int, + boolean_data boolean, + tinyint_data tinyint, + smallint_data smallint, + integer_data integer, + bigint_data bigint, + float_data float, + double_data double, + decimal_data decimal, + string_data string, + date_data date, + datetime_data timestamp(0) +) with ( + 'connector' = 'starrocks-x', + 'url' = 'jdbc:mysql://node1:9030', + 'fe-nodes' = 'node1:8030;node2:8030;node3:8030', + 'schema-name' = 'test', + 'table-name' = 'sink', + 'username' = 'root', + 'password' = '' + ); + +insert into sink +select * +from source; diff --git a/chunjun-examples/sql/starrocks/starrocks_stream.sql b/chunjun-examples/sql/starrocks/starrocks_stream.sql new file mode 100644 index 0000000000..9f37b09a10 --- /dev/null +++ b/chunjun-examples/sql/starrocks/starrocks_stream.sql @@ -0,0 +1,47 @@ +CREATE TABLE source +( + id int, + boolean_data boolean, + tinyint_data tinyint, + smallint_data smallint, + integer_data integer, + bigint_data bigint, + float_data float, + double_data double, + decimal_data decimal, + string_data string, + date_data date, + datetime_data timestamp(0) +) with ( + 'connector' = 'starrocks-x', + 'url' = 'jdbc:mysql://node01:9030', + 'fe-nodes' = 'node1:8030;node2:8030;node3:8030', + 'schema-name' = 'test', + 'table-name' = 'source', + 'username' = 'root', + 'password' = '' + ); + + +CREATE TABLE sink +( + id int, + boolean_data boolean, + tinyint_data tinyint, + smallint_data smallint, + integer_data integer, + bigint_data bigint, + float_data float, + double_data double, + decimal_data decimal, + string_data string, + date_data date, + datetime_data timestamp(0) +) with ( + 'connector' = 'stream-x', + 'print' = 'true' + ); + +insert into sink +select * +from source; diff --git a/chunjun-examples/sql/starrocks/stream_starrocks.sql b/chunjun-examples/sql/starrocks/stream_starrocks.sql new file mode 100644 index 0000000000..580e336520 --- /dev/null +++ b/chunjun-examples/sql/starrocks/stream_starrocks.sql @@ -0,0 +1,28 @@ +CREATE TABLE source +( + id int, + name string +) with ( + 'connector' = 'stream-x', + 'number-of-rows' = '10' + ); + +CREATE TABLE sink +( + id int, + name string + + +) with ( + 'connector' = 'starrocks-x', + 'url' = 'jdbc:mysql://node1:9030', + 'feNodes' = 'node1:8030;node2:8030;node3:8030', + 'schema-name' = 'test', + 'table-name' = 'sink', + 'username' = 'root', + 'password' = '' + ); + +insert into sink +select * +from source; diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/starrocks/starrocks-lookup.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/starrocks/starrocks-lookup.md" new file mode 100644 index 0000000000..431b5b503d --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/starrocks/starrocks-lookup.md" @@ -0,0 +1,197 @@ +# StarRocks Lookup + +## 一、介绍 + +StarRocks维表,支持全量和异步方式。
+全量缓存:将维表数据全部加载到内存中,建议数据量不大使用。
+异步缓存:使用异步方式查询数据,并将查询到的数据使用lru缓存到内存中,建议数据量大使用。 + +## 二、支持版本 + +StarRocks 2.x + +## 三、插件名称 + +| SQL | starrocks-x | +| --- |-------------| + +## 四、参数说明 + +- **url** + - 描述:StarRocks JdbcUrl + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **table-name** + - 描述:表名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema-name** + - 描述:schema + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **username** + - 描述:用户名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **password** + - 描述:密码 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **max-retries** + - 描述:be获取连接、读取数据的重试次数 + - 必选:否 + - 字段类型:int + - 默认值:3 + +
+ +- **scan.be.client.timeout** + - 描述:与be通信允许的超时毫秒值 + - 必选:否 + - 参数类型:int + - 默认值:3000 + +
+ +- **scan.be.client.keep-live-min** + - 描述:be连接存活时间 + - 必选:否 + - 参数类型:int + - 默 +
+ +- **scan.be.query.timeout-s** + - 描述:be查询超时时长 + - 必选:否 + - 参数类型:int + - 默认值:600 + +
+ +- **scan.be.fetch-rows** + - 描述:be批量读取的数据条数 + - 必选:否 + - 参数类型:int + - 默认值:1024 + +
+ +- **scan.be.fetch-bytes-limit** + - 描述:be批量读取的数据最大byte值 + - 必选:否 + - 参数类型:long + - 默认值:1073741824 (1G) +
+ +- **scan.be.param.properties** + - 描述:连接be的其他可配置参数 + - 必选:否 + - 参数类型:map + - 默认值:无 +
+ +- **scan.be.host-mapping** + - 描述:be节点host映射 + - 必选:否 + - 参数类型:list + - 默认值:无 +
+ +- **lookup.cache-period** + - 描述:ALL维表每隔多久加载一次数据,默认3600000毫秒(一个小时) + - 必选:否 + - 参数类型:string + - 默认值:3600000 + +
+ +- **lookup.cache.max-rows** + - 描述:lru维表缓存数据的条数,默认10000条 + - 必选:否 + - 参数类型:string + - 默认值:10000 + +
+ +- **lookup.cache.ttl** + - 描述:lru维表缓存数据的时间,默认60000毫秒(一分钟) + - 必选:否 + - 参数类型:string + - 默认值:60000 + +
+ +- **lookup.max-retries** + - 描述:LRU维表查找数据库失败时的最大重试次数 + - 必选:否 + - 参数类型:int + - 默认值:3 +
+ +- **lookup.error-limit** + - 描述:LRU维表发生超时、ALL Cache维表发送数据失败次数的容忍值 + - 必选:否 + - 参数类型:int + - 默认值:Long.MAX_VALUE +
+ +- **lookup.fetch-size** + - 描述:ALL维表每次从数据库加载的条数,默认1000条 + - 必选:否 + - 参数类型:string + - 默认值:1000 + +
+ +- **lookup.parallelism** + - 描述:维表并行度 + - 必选:否 + - 参数类型:string + - 默认值:无 + +## 五、数据类型 +| **Flink type** | **StarRocks type** | **Flinkx Column** | +| --------------------------------- | ------------------ | ----------------- | +| BOOLEAN | BOOLEAN | BooleanColumn | +| TINYINT | TINYINT | ByteColumn | +| SMALLINT | SMALLINT | BigDecimalColumn | +| INTEGER | INTEGER | BigDecimalColumn | +| BIGINT | BIGINT | BigDecimalColumn | +| FLOAT | FLOAT | BigDecimalColumn | +| DOUBLE | DOUBLE | BigDecimalColumn | +| DECIMAL | DECIMAL | BigDecimalColumn | +| BINARY | INTEGER | BigDecimalColumn | +| CHAR | STRING | StringColumn | +| VARCHAR | STRING | StringColumn | +| STRING | STRING | StringColumn | +| STRING | LARGEINT | StringColumn | +| DATE | DATE | SqlDateColumn | +| TIMESTAMP_WITHOUT_TIME_ZONE(N) | DATETIME | TimestampColumn | +| TIMESTAMP_WITH_LOCAL_TIME_ZONE(N) | DATETIME | TimestampColumn | +| ARRAY | ARRAY | 暂不支持 | +| MAP | JSON STRING | 暂不支持 | +| ROW | JSON STRING | 暂不支持 | + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/starrocks/starrocks-sink.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/starrocks/starrocks-sink.md" new file mode 100644 index 0000000000..3c14ab96f6 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/starrocks/starrocks-sink.md" @@ -0,0 +1,279 @@ +# StarRocks Sink + +## 一、介绍 + +StarRocks Sink插件使用stream-load以json格式向数据库写入数据 + +## 二、支持版本 + +StarRocks 2.x + +## 三、插件名称 + +| Sync | starrockswriter | +| ---- | --------------- | +| SQL | starrocks-x | + +## 四、插件参数 + +### 1.Sync + +- **url** + - 描述:StarRocks JdbcUrl + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称 + - 必选:是 + - 字段类型:String + - 默认值:无 +
+ +- **username** + - 描述:用户名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **password** + - 描述:密码 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **column** + + - 描述:目的表需要写入数据的字段,字段之间用英文逗号分隔。例如: "column": ["id","name","age"] + - 必选:是 + - 默认值:否 + - 字段类型:List + - 默认值:无 +
+ +- **batchSize** + + - 描述:写入内部缓存的数据批大小,不代表一次写入StarRocks的数据量 + - 必选:否 + - 字段类型:int + - 默认值:1024 +
+ +- **nameMapped** + + - 描述:配置此选项为true后,schema和table配置失效。将从上游来的数据中获取这两项值,可利用此配置实现多表写入 + - 必选:否 + - 字段类型:boolean + - 默认值:false +
+ +- **loadConf** + + - 描述:内部stream-load的配置 + + - 必选:否 + + - 字段类型:json + + - 默认值&示例 + + ``` + "loadConf":{ + "httpCheckTimeoutMs":10000,//ms + "queueOfferTimeoutMs":60000,//ms + "queuePollTimeoutMs":60000,//ms + "batchMaxSize":2147483648,//bytes + "batchMaxRows":200000, + "headProperties":{//other http head configuration + "strict_mode":true + } + } + ``` + +
+ +### 2.SQL + +- **url** + - 描述:StarRocks JdbcUrl + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **table-name** + - 描述:表名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema-name** + - 描述:schema + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **username** + - 描述:用户名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **password** + - 描述:密码 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **fe-nodes** + - 描述:StarRocks FrontendEngine地址 + - 必选:是 + - 参数类型:String + - 默认值:无 + +
+ +- **name-mapped** + - 描述:配置此选项为true后,schema-name和table-name配置失效。将从上游来的数据中获取这两项值,可利用此配置实现多表写入 + - 必选:否 + - 参数类型:boolean + - 默认值:false + +
+ +- **max-retries** + - 描述:stream-load写数据失败次数 + - 必选:否 + - 参数类型:int + - 默认值:3 + +
+ +- **sink.buffer-flush.max-rows** + - 描述:写入内部缓存的数据批大小,不代表一次写入StarRocks的数据量 + - 必选:否 + - 参数类型: + - 默认值:10240 + +
+ +- **sink.buffer-flush.max-rows** + - 描述:写入内部缓存的数据批大小,不代表一次写入StarRocks的数据量 + - 必选:否 + - 参数类型: + - 默认值:10240 + +
+ +- **sink.buffer-flush.interval** + - 描述:批量写时间间隔,单位:毫秒 + - 必选:否 + - 参数类型:String + - 默认值:10000 + +
+ +- **sink.batch.max-rows** + - 描述:以schema+table为单位的批量写入StarRocks的最大条数 + - 必选:否 + - 参数类型:long + - 默认值:204800L + +
+ +- **sink.batch.max-bytes** + - 描述:以schema+table为单位的批量写入StarRocks的最大byte + - 必选:否 + - 参数类型:long + - 默认值:2147483648L (2G) + +
+- **http.check.timeout** + - 描述:检查FE节点连通性时,允许的超时时长毫秒值 + - 必选:否 + - 参数类型:int + - 默认值:10000 + +
+ +- **queue.offer.timeout** + - 描述:数据写入内部缓冲队列允许的超时时长毫秒值 + - 必选:否 + - 参数类型:int + - 默认值:60000 + +
+ +- **queue.poll.timeout** + - 描述:从内部缓冲队列读取数据允许的超时时长毫秒值 + - 必选:否 + - 参数类型:int + - 默认值:60000 + +
+ +- **stream-load.head.properties** + - 描述:自选的stream-load的http请求头配置 + - 必选:否 + - 参数类型:map + - 默认值:无 + +
+ +- **sink.parallelism** + - 描述:写入结果的并行度 + - 必选:否 + - 参数类型:String + - 默认值:无 + +## 五、数据类型 + +| **Flink type** | **StarRocks type** | **Flinkx Column** | +| --------------------------------- | ------------------ | ----------------- | +| BOOLEAN | BOOLEAN | BooleanColumn | +| TINYINT | TINYINT | ByteColumn | +| SMALLINT | SMALLINT | BigDecimalColumn | +| INTEGER | INTEGER | BigDecimalColumn | +| BIGINT | BIGINT | BigDecimalColumn | +| FLOAT | FLOAT | BigDecimalColumn | +| DOUBLE | DOUBLE | BigDecimalColumn | +| DECIMAL | DECIMAL | BigDecimalColumn | +| BINARY | INTEGER | BigDecimalColumn | +| CHAR | STRING | StringColumn | +| VARCHAR | STRING | StringColumn | +| STRING | STRING | StringColumn | +| STRING | LARGEINT | StringColumn | +| DATE | DATE | SqlDateColumn | +| TIMESTAMP_WITHOUT_TIME_ZONE(N) | DATETIME | TimestampColumn | +| TIMESTAMP_WITH_LOCAL_TIME_ZONE(N) | DATETIME | TimestampColumn | +| ARRAY | ARRAY | 暂不支持 | +| MAP | JSON STRING | 暂不支持 | +| ROW | JSON STRING | 暂不支持 | + +## 六、脚本示例 + +见项目内`chunjun-examples`文件夹。 diff --git "a/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/starrocks/starrocks-source.md" "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/starrocks/starrocks-source.md" new file mode 100644 index 0000000000..fd95cac559 --- /dev/null +++ "b/docs_zh/ChunJun\350\277\236\346\216\245\345\231\250/starrocks/starrocks-source.md" @@ -0,0 +1,285 @@ +# StarRocks Source + +## 一、介绍 + +StarRocks 支持从StarRocks读取数据 + +## 二、支持版本 + +StarRocks2.x + +## 三、插件名称 + +| Sync | starrocksreader、starrockssource | +| ---- | -------------------------------- | +| SQL | starrocks-x | + +## 四、参数说明 + +### 1.Sync + +- **url** + - 描述:StarRocks JdbcUrl + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema** + - 描述:数据库schema名 + - 必选:否 + - 字段类型:String + - 默认值:无 + +
+ +- **table** + - 描述:目的表的表名称 + - 必选:是 + - 字段类型:String + - 默认值:无 +
+ +- **username** + - 描述:用户名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **password** + - 描述:密码 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **column** + - 描述:需要读取的字段。 + - 必选:是 + - 格式:支持3中格式
1.读取全部字段,如果字段数量很多,可以使用下面的写法: + +```bash +"column":["*"] +``` + +- 2.只指定字段名称: + +``` +"column":["id","name"] +``` + +- 3.指定具体信息: + +```json +"column": [{ + "name": "col", + "type": "datetime", + "format": "yyyy-MM-dd hh:mm:ss", + "value": "value" +}] +``` + +
+ +- **maxRetries** + - 描述:be获取连接、读取数据的重试次数 + - 必选:否 + - 字段类型:int + - 默认值:3 + +
+ +- **filterStatement** + - 描述:简单的sql过滤语句,例如id>10 + - 必选:否 + - 参数类型:string + - 默认值:无 + +
+ +- **beClientTimeout** + - 描述:与be通信允许的超时毫秒值 + - 必选:否 + - 参数类型:int + - 默认值:3000 + +
+ +- **beClientKeepLiveMin** + - 描述:be连接存活时间 + - 必选:否 + - 参数类型:int + - 默 +
+ +- **beQueryTimeoutSecond** + - 描述:be查询超时时长 + - 必选:否 + - 参数类型:int + - 默认值:600 + +
+ +- **beFetchRows** + - 描述:be批量读取的数据条数 + - 必选:否 + - 参数类型:int + - 默认值:1024 + +
+ +- **beFetchMaxBytes** + - 描述:be批量读取的数据最大byte值 + - 必选:否 + - 参数类型:long + - 默认值:1073741824 (1G) +
+- **beSocketProperties** + - 描述:连接be的其他可配置参数 + - 必选:否 + - 参数类型:map + - 默认值:无 +
+ +### 2.SQL + +- **url** + - 描述:StarRocks JdbcUrl + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **table-name** + - 描述:表名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **schema-name** + - 描述:schema + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **username** + - 描述:用户名 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **password** + - 描述:密码 + - 必选:是 + - 字段类型:String + - 默认值:无 + +
+ +- **fe-nodes** + - 描述:StarRocks FrontendEngine地址 + - 必选:是 + - 参数类型:String + - 默认值:无 + +
+ +- **max-retries** + - 描述:be获取连接、读取数据的重试次数 + - 必选:否 + - 字段类型:int + - 默认值:3 + +
+ +- **filter-statement** + - 描述:简单的sql过滤语句,例如id>10 + - 必选:否 + - 参数类型:string + - 默认值:无 + +
+ +- **scan.be.client.timeout** + - 描述:与be通信允许的超时毫秒值 + - 必选:否 + - 参数类型:int + - 默认值:3000 + +
+ +- **scan.be.client.keep-live-min** + - 描述:be连接存活时间 + - 必选:否 + - 参数类型:int + - 默 +
+ +- **scan.be.query.timeout-s** + - 描述:be查询超时时长 + - 必选:否 + - 参数类型:int + - 默认值:600 + +
+ +- **scan.be.fetch-rows** + - 描述:be批量读取的数据条数 + - 必选:否 + - 参数类型:int + - 默认值:1024 + +
+ +- **scan.be.fetch-bytes-limit** + - 描述:be批量读取的数据最大byte值 + - 必选:否 + - 参数类型:long + - 默认值:1073741824 (1G) +
+- **scan.be.param.properties** + - 描述:连接be的其他可配置参数 + - 必选:否 + - 参数类型:map + - 默认值:无 +
+ + + +## 五、数据类型 +| **Flink type** | **StarRocks type** | **Flinkx Column** | +| --------------------------------- | ------------------ | ----------------- | +| BOOLEAN | BOOLEAN | BooleanColumn | +| TINYINT | TINYINT | ByteColumn | +| SMALLINT | SMALLINT | BigDecimalColumn | +| INTEGER | INTEGER | BigDecimalColumn | +| BIGINT | BIGINT | BigDecimalColumn | +| FLOAT | FLOAT | BigDecimalColumn | +| DOUBLE | DOUBLE | BigDecimalColumn | +| DECIMAL | DECIMAL | BigDecimalColumn | +| BINARY | INTEGER | BigDecimalColumn | +| CHAR | STRING | StringColumn | +| VARCHAR | STRING | StringColumn | +| STRING | STRING | StringColumn | +| STRING | LARGEINT | StringColumn | +| DATE | DATE | SqlDateColumn | +| TIMESTAMP_WITHOUT_TIME_ZONE(N) | DATETIME | TimestampColumn | +| TIMESTAMP_WITH_LOCAL_TIME_ZONE(N) | DATETIME | TimestampColumn | +| ARRAY | ARRAY | 暂不支持 | +| MAP | JSON STRING | 暂不支持 | +| ROW | JSON STRING | 暂不支持 | + +# 六、脚本示例 + +见项目内`chunjun-examples`文件夹。