diff --git a/flink-doris-connector/pom.xml b/flink-doris-connector/pom.xml
index 066c5e7d9..f7801ddfa 100644
--- a/flink-doris-connector/pom.xml
+++ b/flink-doris-connector/pom.xml
@@ -73,7 +73,6 @@ under the License.
3.1.1
flink-python
0.16.0
- 13.0.0
3.10.1
3.3.0
3.2.1
@@ -96,6 +95,8 @@ under the License.
8.0.26
19.3.0.0
2.17.1
+ 15.0.2
+ 0.12.0
@@ -180,13 +181,16 @@ under the License.
commons-codec
${commons-codec.version}
-
+
+ org.apache.arrow.adbc
+ adbc-driver-flight-sql
+ ${adbc.version}
+
org.apache.arrow
arrow-vector
${arrow.version}
-
org.apache.arrow
arrow-memory-netty
@@ -207,7 +211,6 @@ under the License.
-
com.fasterxml.jackson.core
@@ -429,13 +432,13 @@ under the License.
org.apache.maven.plugins
maven-shade-plugin
- 3.2.4
+ 3.4.1
-
- org.apache.arrow
- org.apache.doris.shaded.org.apache.arrow
-
+
+
+
+
io.netty
org.apache.doris.shaded.io.netty
diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSchemaFactory.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSchemaFactory.java
index 3f824a1ff..9693d433e 100644
--- a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSchemaFactory.java
+++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSchemaFactory.java
@@ -18,9 +18,12 @@
package org.apache.doris.flink.catalog.doris;
import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.StringUtils;
import org.apache.commons.collections.CollectionUtils;
import org.apache.commons.collections.MapUtils;
+import org.apache.doris.flink.exception.CreateTableException;
import org.apache.doris.flink.tools.cdc.DorisTableConfig;
import java.util.ArrayList;
@@ -30,6 +33,7 @@
import java.util.Map.Entry;
import java.util.Objects;
import java.util.regex.Pattern;
+import java.util.stream.Collectors;
/**
* Factory that creates doris schema.
@@ -103,4 +107,132 @@ public static Integer parseTableSchemaBuckets(
}
return null;
}
+
+ public static String generateCreateTableDDL(TableSchema schema) {
+ StringBuilder sb = new StringBuilder("CREATE TABLE IF NOT EXISTS ");
+ sb.append(identifier(schema.getDatabase()))
+ .append(".")
+ .append(identifier(schema.getTable()))
+ .append("(");
+
+ Map fields = schema.getFields();
+ List keys = schema.getKeys();
+ // append keys
+ for (String key : keys) {
+ if (!fields.containsKey(key)) {
+ throw new CreateTableException("key " + key + " not found in column list");
+ }
+ FieldSchema field = fields.get(key);
+ buildColumn(sb, field, true);
+ }
+
+ // append values
+ for (Map.Entry entry : fields.entrySet()) {
+ if (keys.contains(entry.getKey())) {
+ continue;
+ }
+ FieldSchema field = entry.getValue();
+ buildColumn(sb, field, false);
+ }
+ sb = sb.deleteCharAt(sb.length() - 1);
+ sb.append(" ) ");
+ // append uniq model
+ if (DataModel.UNIQUE.equals(schema.getModel())) {
+ sb.append(schema.getModel().name())
+ .append(" KEY(")
+ .append(String.join(",", identifier(schema.getKeys())))
+ .append(")");
+ }
+
+ // append table comment
+ if (!StringUtils.isNullOrWhitespaceOnly(schema.getTableComment())) {
+ sb.append(" COMMENT '").append(quoteComment(schema.getTableComment())).append("' ");
+ }
+
+ // append distribute key
+ sb.append(" DISTRIBUTED BY HASH(")
+ .append(String.join(",", identifier(schema.getDistributeKeys())))
+ .append(")");
+
+ Map properties = schema.getProperties();
+ if (schema.getTableBuckets() != null) {
+
+ int bucketsNum = schema.getTableBuckets();
+ if (bucketsNum <= 0) {
+ throw new CreateTableException("The number of buckets must be positive.");
+ }
+ sb.append(" BUCKETS ").append(bucketsNum);
+ } else {
+ sb.append(" BUCKETS AUTO ");
+ }
+ // append properties
+ int index = 0;
+ for (Map.Entry entry : properties.entrySet()) {
+ if (index == 0) {
+ sb.append(" PROPERTIES (");
+ }
+ if (index > 0) {
+ sb.append(",");
+ }
+ sb.append(quoteProperties(entry.getKey()))
+ .append("=")
+ .append(quoteProperties(entry.getValue()));
+ index++;
+
+ if (index == (schema.getProperties().size())) {
+ sb.append(")");
+ }
+ }
+ return sb.toString();
+ }
+
+ private static void buildColumn(StringBuilder sql, FieldSchema field, boolean isKey) {
+ String fieldType = field.getTypeString();
+ if (isKey && DorisType.STRING.equals(fieldType)) {
+ fieldType = String.format("%s(%s)", DorisType.VARCHAR, 65533);
+ }
+ sql.append(identifier(field.getName())).append(" ").append(fieldType);
+
+ if (field.getDefaultValue() != null) {
+ sql.append(" DEFAULT " + quoteDefaultValue(field.getDefaultValue()));
+ }
+ sql.append(" COMMENT '").append(quoteComment(field.getComment())).append("',");
+ }
+
+ private static String quoteProperties(String name) {
+ return "'" + name + "'";
+ }
+
+ private static List identifier(List names) {
+ return names.stream().map(DorisSchemaFactory::identifier).collect(Collectors.toList());
+ }
+
+ public static String identifier(String name) {
+ if (name.startsWith("`") && name.endsWith("`")) {
+ return name;
+ }
+ return "`" + name + "`";
+ }
+
+ public static String quoteDefaultValue(String defaultValue) {
+ // DEFAULT current_timestamp not need quote
+ if (defaultValue.equalsIgnoreCase("current_timestamp")) {
+ return defaultValue;
+ }
+ return "'" + defaultValue + "'";
+ }
+
+ public static String quoteComment(String comment) {
+ if (comment == null) {
+ return "";
+ } else {
+ return comment.replaceAll("'", "\\\\'");
+ }
+ }
+
+ public static String quoteTableIdentifier(String tableIdentifier) {
+ String[] dbTable = tableIdentifier.split("\\.");
+ Preconditions.checkArgument(dbTable.length == 2);
+ return identifier(dbTable[0]) + "." + identifier(dbTable[1]);
+ }
}
diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSystem.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSystem.java
index be6572d31..427eb8b34 100644
--- a/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSystem.java
+++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/catalog/doris/DorisSystem.java
@@ -18,14 +18,12 @@
package org.apache.doris.flink.catalog.doris;
import org.apache.flink.annotation.Public;
-import org.apache.flink.util.Preconditions;
import org.apache.flink.util.StringUtils;
import org.apache.commons.compress.utils.Lists;
import org.apache.doris.flink.cfg.DorisConnectionOptions;
import org.apache.doris.flink.connection.JdbcConnectionProvider;
import org.apache.doris.flink.connection.SimpleJdbcConnectionProvider;
-import org.apache.doris.flink.exception.CreateTableException;
import org.apache.doris.flink.exception.DorisRuntimeException;
import org.apache.doris.flink.exception.DorisSystemException;
import org.slf4j.Logger;
@@ -41,7 +39,6 @@
import java.util.Map;
import java.util.Objects;
import java.util.function.Predicate;
-import java.util.stream.Collectors;
import static org.apache.flink.util.Preconditions.checkArgument;
@@ -141,81 +138,7 @@ public List extractColumnValuesBySQL(
}
public static String buildCreateTableDDL(TableSchema schema) {
- StringBuilder sb = new StringBuilder("CREATE TABLE IF NOT EXISTS ");
- sb.append(identifier(schema.getDatabase()))
- .append(".")
- .append(identifier(schema.getTable()))
- .append("(");
-
- Map fields = schema.getFields();
- List keys = schema.getKeys();
- // append keys
- for (String key : keys) {
- if (!fields.containsKey(key)) {
- throw new CreateTableException("key " + key + " not found in column list");
- }
- FieldSchema field = fields.get(key);
- buildColumn(sb, field, true);
- }
-
- // append values
- for (Map.Entry entry : fields.entrySet()) {
- if (keys.contains(entry.getKey())) {
- continue;
- }
- FieldSchema field = entry.getValue();
- buildColumn(sb, field, false);
- }
- sb = sb.deleteCharAt(sb.length() - 1);
- sb.append(" ) ");
- // append uniq model
- if (DataModel.UNIQUE.equals(schema.getModel())) {
- sb.append(schema.getModel().name())
- .append(" KEY(")
- .append(String.join(",", identifier(schema.getKeys())))
- .append(")");
- }
-
- // append table comment
- if (!StringUtils.isNullOrWhitespaceOnly(schema.getTableComment())) {
- sb.append(" COMMENT '").append(quoteComment(schema.getTableComment())).append("' ");
- }
-
- // append distribute key
- sb.append(" DISTRIBUTED BY HASH(")
- .append(String.join(",", identifier(schema.getDistributeKeys())))
- .append(")");
-
- Map properties = schema.getProperties();
- if (schema.getTableBuckets() != null) {
-
- int bucketsNum = schema.getTableBuckets();
- if (bucketsNum <= 0) {
- throw new CreateTableException("The number of buckets must be positive.");
- }
- sb.append(" BUCKETS ").append(bucketsNum);
- } else {
- sb.append(" BUCKETS AUTO ");
- }
- // append properties
- int index = 0;
- for (Map.Entry entry : properties.entrySet()) {
- if (index == 0) {
- sb.append(" PROPERTIES (");
- }
- if (index > 0) {
- sb.append(",");
- }
- sb.append(quoteProperties(entry.getKey()))
- .append("=")
- .append(quoteProperties(entry.getValue()));
- index++;
-
- if (index == (schema.getProperties().size())) {
- sb.append(")");
- }
- }
- return sb.toString();
+ return DorisSchemaFactory.generateCreateTableDDL(schema);
}
public Map getTableFieldNames(String databaseName, String tableName) {
@@ -244,53 +167,23 @@ public Map getTableFieldNames(String databaseName, String tableN
}
}
- private static void buildColumn(StringBuilder sql, FieldSchema field, boolean isKey) {
- String fieldType = field.getTypeString();
- if (isKey && DorisType.STRING.equals(fieldType)) {
- fieldType = String.format("%s(%s)", DorisType.VARCHAR, 65533);
- }
- sql.append(identifier(field.getName())).append(" ").append(fieldType);
-
- if (field.getDefaultValue() != null) {
- sql.append(" DEFAULT " + quoteDefaultValue(field.getDefaultValue()));
- }
- sql.append(" COMMENT '").append(quoteComment(field.getComment())).append("',");
- }
-
+ @Deprecated
public static String quoteDefaultValue(String defaultValue) {
- // DEFAULT current_timestamp not need quote
- if (defaultValue.equalsIgnoreCase("current_timestamp")) {
- return defaultValue;
- }
- return "'" + defaultValue + "'";
+ return DorisSchemaFactory.quoteDefaultValue(defaultValue);
}
+ @Deprecated
public static String quoteComment(String comment) {
- if (comment == null) {
- return "";
- } else {
- return comment.replaceAll("'", "\\\\'");
- }
- }
-
- private static List identifier(List name) {
- return name.stream().map(DorisSystem::identifier).collect(Collectors.toList());
+ return DorisSchemaFactory.quoteComment(comment);
}
+ @Deprecated
public static String identifier(String name) {
- if (name.startsWith("`") && name.endsWith("`")) {
- return name;
- }
- return "`" + name + "`";
+ return DorisSchemaFactory.identifier(name);
}
+ @Deprecated
public static String quoteTableIdentifier(String tableIdentifier) {
- String[] dbTable = tableIdentifier.split("\\.");
- Preconditions.checkArgument(dbTable.length == 2);
- return identifier(dbTable[0]) + "." + identifier(dbTable[1]);
- }
-
- private static String quoteProperties(String name) {
- return "'" + name + "'";
+ return DorisSchemaFactory.quoteTableIdentifier(tableIdentifier);
}
}
diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/ConfigurationOptions.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/ConfigurationOptions.java
index 4a3f70b80..c249c2519 100644
--- a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/ConfigurationOptions.java
+++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/ConfigurationOptions.java
@@ -51,4 +51,10 @@ public interface ConfigurationOptions {
Boolean DORIS_DESERIALIZE_ARROW_ASYNC_DEFAULT = false;
String DORIS_DESERIALIZE_QUEUE_SIZE = "doris.deserialize.queue.size";
Integer DORIS_DESERIALIZE_QUEUE_SIZE_DEFAULT = 64;
+
+ String USE_FLIGHT_SQL = "source.use-flight-sql";
+ Boolean USE_FLIGHT_SQL_DEFAULT = false;
+
+ String FLIGHT_SQL_PORT = "source.flight-sql-port";
+ Integer FLIGHT_SQL_PORT_DEFAULT = 9040;
}
diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java
index 3669e740a..2f6cd8a86 100644
--- a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java
+++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisReadOptions.java
@@ -37,6 +37,8 @@ public class DorisReadOptions implements Serializable {
private Integer deserializeQueueSize;
private Boolean deserializeArrowAsync;
private boolean useOldApi;
+ private boolean useFlightSql;
+ private Integer flightSqlPort;
public DorisReadOptions(
String readFields,
@@ -50,7 +52,9 @@ public DorisReadOptions(
Long execMemLimit,
Integer deserializeQueueSize,
Boolean deserializeArrowAsync,
- boolean useOldApi) {
+ boolean useOldApi,
+ boolean useFlightSql,
+ Integer flightSqlPort) {
this.readFields = readFields;
this.filterQuery = filterQuery;
this.requestTabletSize = requestTabletSize;
@@ -63,6 +67,8 @@ public DorisReadOptions(
this.deserializeQueueSize = deserializeQueueSize;
this.deserializeArrowAsync = deserializeArrowAsync;
this.useOldApi = useOldApi;
+ this.useFlightSql = useFlightSql;
+ this.flightSqlPort = flightSqlPort;
}
public String getReadFields() {
@@ -121,6 +127,14 @@ public void setFilterQuery(String filterQuery) {
this.filterQuery = filterQuery;
}
+ public boolean getUseFlightSql() {
+ return useFlightSql;
+ }
+
+ public Integer getFlightSqlPort() {
+ return flightSqlPort;
+ }
+
public static Builder builder() {
return new Builder();
}
@@ -149,7 +163,9 @@ public boolean equals(Object o) {
&& Objects.equals(requestBatchSize, that.requestBatchSize)
&& Objects.equals(execMemLimit, that.execMemLimit)
&& Objects.equals(deserializeQueueSize, that.deserializeQueueSize)
- && Objects.equals(deserializeArrowAsync, that.deserializeArrowAsync);
+ && Objects.equals(deserializeArrowAsync, that.deserializeArrowAsync)
+ && Objects.equals(useFlightSql, that.useFlightSql)
+ && Objects.equals(flightSqlPort, that.flightSqlPort);
}
@Override
@@ -166,7 +182,9 @@ public int hashCode() {
execMemLimit,
deserializeQueueSize,
deserializeArrowAsync,
- useOldApi);
+ useOldApi,
+ useFlightSql,
+ flightSqlPort);
}
/** Builder of {@link DorisReadOptions}. */
@@ -184,6 +202,8 @@ public static class Builder {
private Integer deserializeQueueSize;
private Boolean deserializeArrowAsync;
private Boolean useOldApi = false;
+ private Boolean useFlightSql = false;
+ private Integer flightSqlPort;
public Builder setReadFields(String readFields) {
this.readFields = readFields;
@@ -240,11 +260,21 @@ public Builder setDeserializeArrowAsync(Boolean deserializeArrowAsync) {
return this;
}
- public Builder setUseOldApi(boolean useOldApi) {
+ public Builder setUseFlightSql(Boolean useFlightSql) {
+ this.useFlightSql = useFlightSql;
+ return this;
+ }
+
+ public Builder setUseOldApi(Boolean useOldApi) {
this.useOldApi = useOldApi;
return this;
}
+ public Builder setFlightSqlPort(Integer flightSqlPort) {
+ this.flightSqlPort = flightSqlPort;
+ return this;
+ }
+
public DorisReadOptions build() {
return new DorisReadOptions(
readFields,
@@ -258,7 +288,9 @@ public DorisReadOptions build() {
execMemLimit,
deserializeQueueSize,
deserializeArrowAsync,
- useOldApi);
+ useOldApi,
+ useFlightSql,
+ flightSqlPort);
}
}
}
diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisStreamOptions.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisStreamOptions.java
index 89e9182e8..e6d75969c 100644
--- a/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisStreamOptions.java
+++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/cfg/DorisStreamOptions.java
@@ -106,6 +106,18 @@ private void init() {
prop.getProperty(
ConfigurationOptions.DORIS_TABLET_SIZE,
ConfigurationOptions.DORIS_TABLET_SIZE_DEFAULT
+ .toString())))
+ .setUseFlightSql(
+ Boolean.valueOf(
+ prop.getProperty(
+ ConfigurationOptions.USE_FLIGHT_SQL,
+ ConfigurationOptions.USE_FLIGHT_SQL_DEFAULT
+ .toString())))
+ .setFlightSqlPort(
+ Integer.valueOf(
+ prop.getProperty(
+ ConfigurationOptions.FLIGHT_SQL_PORT,
+ ConfigurationOptions.FLIGHT_SQL_PORT_DEFAULT
.toString())));
this.options = optionsBuilder.build();
diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java
index 1dbb1fded..1663d4b39 100644
--- a/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java
+++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/RestService.java
@@ -234,6 +234,38 @@ public static String parseResponse(HttpURLConnection connection, Logger logger)
}
}
+ @VisibleForTesting
+ public static String parseFlightSql(
+ DorisReadOptions readOptions,
+ DorisOptions options,
+ PartitionDefinition partition,
+ Logger logger)
+ throws IllegalArgumentException {
+ String[] tableIdentifiers = parseIdentifier(options.getTableIdentifier(), logger);
+ String readFields =
+ StringUtils.isBlank(readOptions.getReadFields())
+ ? "*"
+ : readOptions.getReadFields();
+ String sql =
+ "select "
+ + readFields
+ + " from `"
+ + tableIdentifiers[0]
+ + "`.`"
+ + tableIdentifiers[1]
+ + "`";
+ String tablet =
+ partition.getTabletIds().stream()
+ .map(Object::toString)
+ .collect(Collectors.joining(","));
+ sql += " TABLET(" + tablet + ") ";
+ if (!StringUtils.isEmpty(readOptions.getFilterQuery())) {
+ sql += " where " + readOptions.getFilterQuery();
+ }
+ logger.info("Query SQL Sending to Doris FE is: '{}'.", sql);
+ return sql;
+ }
+
/**
* parse table identifier to array.
*
diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/SchemaUtils.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/SchemaUtils.java
index f6594b5bc..9f7d7fc64 100644
--- a/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/SchemaUtils.java
+++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/rest/SchemaUtils.java
@@ -22,6 +22,9 @@
import org.apache.doris.sdk.thrift.TScanColumnDesc;
import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
public class SchemaUtils {
@@ -46,4 +49,25 @@ public static Schema convertToSchema(List tscanColumnDescs) {
"")));
return schema;
}
+
+ public static Schema convertToSchema(
+ Schema tableSchema, org.apache.arrow.vector.types.pojo.Schema tscanColumnDescs) {
+ Schema schema = new Schema(tscanColumnDescs.getFields().size());
+ Map collect =
+ tableSchema.getProperties().stream()
+ .collect(Collectors.toMap(Field::getName, Function.identity()));
+ tscanColumnDescs
+ .getFields()
+ .forEach(
+ desc ->
+ schema.put(
+ new Field(
+ desc.getName(),
+ collect.get(desc.getName()).getType(),
+ "",
+ 0,
+ 0,
+ "")));
+ return schema;
+ }
}
diff --git a/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java b/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java
index 38c63b779..dee9c1fcb 100644
--- a/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java
+++ b/flink-doris-connector/src/main/java/org/apache/doris/flink/serialization/RowBatch.java
@@ -44,11 +44,13 @@
import org.apache.arrow.vector.complex.impl.TimeStampMicroReaderImpl;
import org.apache.arrow.vector.complex.impl.UnionMapReader;
import org.apache.arrow.vector.complex.reader.FieldReader;
+import org.apache.arrow.vector.ipc.ArrowReader;
import org.apache.arrow.vector.ipc.ArrowStreamReader;
import org.apache.arrow.vector.types.Types;
import org.apache.doris.flink.exception.DorisException;
import org.apache.doris.flink.exception.DorisRuntimeException;
import org.apache.doris.flink.rest.models.Schema;
+import org.apache.doris.flink.util.FastDateUtil;
import org.apache.doris.flink.util.IPUtils;
import org.apache.doris.sdk.thrift.TScanBatchResult;
import org.slf4j.Logger;
@@ -58,6 +60,7 @@
import java.io.IOException;
import java.math.BigDecimal;
import java.math.BigInteger;
+import java.nio.charset.StandardCharsets;
import java.time.Instant;
import java.time.LocalDate;
import java.time.LocalDateTime;
@@ -96,18 +99,19 @@ public void put(Object o) {
private int rowCountInOneBatch = 0;
private int readRowCount = 0;
private final List rowBatch = new ArrayList<>();
- private final ArrowStreamReader arrowStreamReader;
+ private final ArrowReader arrowStreamReader;
private VectorSchemaRoot root;
private List fieldVectors;
private RootAllocator rootAllocator;
private final Schema schema;
private static final String DATETIME_PATTERN = "yyyy-MM-dd HH:mm:ss";
private static final String DATETIMEV2_PATTERN = "yyyy-MM-dd HH:mm:ss.SSSSSS";
+ private static final String DATE_PATTERN = "yyyy-MM-dd";
private final DateTimeFormatter dateTimeFormatter =
DateTimeFormatter.ofPattern(DATETIME_PATTERN);
private final DateTimeFormatter dateTimeV2Formatter =
DateTimeFormatter.ofPattern(DATETIMEV2_PATTERN);
- private final DateTimeFormatter dateFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd");
+ private final DateTimeFormatter dateFormatter = DateTimeFormatter.ofPattern(DATE_PATTERN);
private static final ZoneId DEFAULT_ZONE_ID = ZoneId.systemDefault();
public List getRowBatch() {
@@ -123,6 +127,43 @@ public RowBatch(TScanBatchResult nextResult, Schema schema) {
this.offsetInRowBatch = 0;
}
+ public RowBatch(ArrowReader nextResult, Schema schema) {
+ this.schema = schema;
+ this.arrowStreamReader = nextResult;
+ this.offsetInRowBatch = 0;
+ }
+
+ public RowBatch readFlightArrow() {
+ try {
+ this.root = arrowStreamReader.getVectorSchemaRoot();
+ fieldVectors = root.getFieldVectors();
+ if (fieldVectors.size() > schema.size()) {
+ logger.error(
+ "Schema size '{}' is not equal to arrow field size '{}'.",
+ fieldVectors.size(),
+ schema.size());
+ throw new DorisException(
+ "Load Doris data failed, schema size of fetch data is wrong.");
+ }
+ if (fieldVectors.isEmpty() || root.getRowCount() == 0) {
+ logger.debug("One batch in arrow has no data.");
+ return null;
+ }
+ rowCountInOneBatch = root.getRowCount();
+ for (int i = 0; i < rowCountInOneBatch; ++i) {
+ rowBatch.add(new RowBatch.Row(fieldVectors.size()));
+ }
+ convertArrowToRowBatch();
+ readRowCount += root.getRowCount();
+ return this;
+ } catch (DorisException e) {
+ logger.error("Read Doris Data failed because: ", e);
+ throw new DorisRuntimeException(e.getMessage());
+ } catch (IOException e) {
+ return this;
+ }
+ }
+
public RowBatch readArrow() {
try {
this.root = arrowStreamReader.getVectorSchemaRoot();
@@ -297,6 +338,7 @@ public boolean doConvert(
case "DECIMAL32":
case "DECIMAL64":
case "DECIMAL128I":
+ case "DECIMAL128":
if (!minorType.equals(Types.MinorType.DECIMAL)) {
return false;
}
@@ -320,8 +362,8 @@ public boolean doConvert(
addValueToRow(rowIndex, null);
break;
}
- String stringValue = new String(date.get(rowIndex));
- LocalDate localDate = LocalDate.parse(stringValue, dateFormatter);
+ String stringValue = new String(date.get(rowIndex), StandardCharsets.UTF_8);
+ LocalDate localDate = FastDateUtil.fastParseDate(stringValue, DATE_PATTERN);
addValueToRow(rowIndex, localDate);
} else {
DateDayVector date = (DateDayVector) fieldVector;
@@ -340,8 +382,11 @@ public boolean doConvert(
addValueToRow(rowIndex, null);
break;
}
- String stringValue = new String(varCharVector.get(rowIndex));
- LocalDateTime parse = LocalDateTime.parse(stringValue, dateTimeFormatter);
+ String stringValue =
+ new String(varCharVector.get(rowIndex), StandardCharsets.UTF_8);
+ stringValue = completeMilliseconds(stringValue);
+ LocalDateTime parse =
+ FastDateUtil.fastParseDateTime(stringValue, DATETIME_PATTERN);
addValueToRow(rowIndex, parse);
} else if (fieldVector instanceof TimeStampVector) {
LocalDateTime dateTime = getDateTime(rowIndex, fieldVector);
@@ -361,9 +406,11 @@ public boolean doConvert(
addValueToRow(rowIndex, null);
break;
}
- String stringValue = new String(varCharVector.get(rowIndex));
+ String stringValue =
+ new String(varCharVector.get(rowIndex), StandardCharsets.UTF_8);
stringValue = completeMilliseconds(stringValue);
- LocalDateTime parse = LocalDateTime.parse(stringValue, dateTimeV2Formatter);
+ LocalDateTime parse =
+ FastDateUtil.fastParseDateTimeV2(stringValue, DATETIMEV2_PATTERN);
addValueToRow(rowIndex, parse);
} else if (fieldVector instanceof TimeStampVector) {
LocalDateTime dateTime = getDateTime(rowIndex, fieldVector);
@@ -405,7 +452,8 @@ public boolean doConvert(
addValueToRow(rowIndex, null);
break;
}
- String stringValue = new String(largeIntVector.get(rowIndex));
+ String stringValue =
+ new String(largeIntVector.get(rowIndex), StandardCharsets.UTF_8);
BigInteger largeInt = new BigInteger(stringValue);
addValueToRow(rowIndex, largeInt);
break;
@@ -423,7 +471,8 @@ public boolean doConvert(
addValueToRow(rowIndex, null);
break;
}
- String stringValue = new String(varCharVector.get(rowIndex));
+ String stringValue =
+ new String(varCharVector.get(rowIndex), StandardCharsets.UTF_8);
addValueToRow(rowIndex, stringValue);
break;
case "IPV6":
@@ -435,7 +484,8 @@ public boolean doConvert(
addValueToRow(rowIndex, null);
break;
}
- String ipv6Str = new String(ipv6VarcharVector.get(rowIndex));
+ String ipv6Str =
+ new String(ipv6VarcharVector.get(rowIndex), StandardCharsets.UTF_8);
String ipv6Address = IPUtils.fromBigInteger(new BigInteger(ipv6Str));
addValueToRow(rowIndex, ipv6Address);
break;
@@ -526,6 +576,14 @@ public static LocalDateTime longToLocalDateTime(long time) {
return LocalDateTime.ofInstant(instant, DEFAULT_ZONE_ID);
}
+ /**
+ * use case when to replace while "Benchmark","Mode","Threads","Samples","Score","Score Error.
+ * (99.9%)","Unit" "CaseWhenTest", "thrpt", 1, 5, 40657433.897696, 2515802.067503,"ops/s"
+ * "WhileTest", "thrpt", 1, 5, 9708130.819491, 1207453.635429,"ops/s"
+ *
+ * @param stringValue
+ * @return
+ */
@VisibleForTesting
public static String completeMilliseconds(String stringValue) {
if (stringValue.length() == DATETIMEV2_PATTERN.length()) {
@@ -536,14 +594,26 @@ public static String completeMilliseconds(String stringValue) {
return stringValue;
}
- StringBuilder sb = new StringBuilder(stringValue);
if (stringValue.length() == DATETIME_PATTERN.length()) {
- sb.append(".");
+ stringValue += ".";
}
- while (sb.toString().length() < DATETIMEV2_PATTERN.length()) {
- sb.append(0);
+ int s = DATETIMEV2_PATTERN.length() - stringValue.length();
+ switch (s) {
+ case 1:
+ return stringValue + "0";
+ case 2:
+ return stringValue + "00";
+ case 3:
+ return stringValue + "000";
+ case 4:
+ return stringValue + "0000";
+ case 5:
+ return stringValue + "00000";
+ case 6:
+ return stringValue + "000000";
+ default:
+ return stringValue;
}
- return sb.toString();
}
public List