diff --git a/be/src/runtime/descriptors.cpp b/be/src/runtime/descriptors.cpp index efc4396d01f9f4..bea11feb916f10 100644 --- a/be/src/runtime/descriptors.cpp +++ b/be/src/runtime/descriptors.cpp @@ -189,7 +189,21 @@ MaxComputeTableDescriptor::MaxComputeTableDescriptor(const TTableDescriptor& tde _tunnel_url(tdesc.mcTable.tunnel_url), _access_key(tdesc.mcTable.access_key), _secret_key(tdesc.mcTable.secret_key), - _public_access(tdesc.mcTable.public_access) {} + _public_access(tdesc.mcTable.public_access) { + if (tdesc.mcTable.__isset.endpoint) { + _endpoint = tdesc.mcTable.endpoint; + } else { + _init_status = Status::InvalidArgument( + "fail to init MaxComputeTableDescriptor, missing endpoint."); + } + + if (tdesc.mcTable.__isset.quota) { + _quota = tdesc.mcTable.quota; + } else { + _init_status = + Status::InvalidArgument("fail to init MaxComputeTableDescriptor, missing quota."); + } +} MaxComputeTableDescriptor::~MaxComputeTableDescriptor() = default; diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h index 3379cebefce6af..66d571d3b95541 100644 --- a/be/src/runtime/descriptors.h +++ b/be/src/runtime/descriptors.h @@ -215,16 +215,22 @@ class MaxComputeTableDescriptor : public TableDescriptor { std::string access_key() const { return _access_key; } std::string secret_key() const { return _secret_key; } std::string public_access() const { return _public_access; } + std::string endpoint() const { return _endpoint; } + std::string quota() const { return _quota; } + Status init_status() const { return _init_status; } private: - std::string _region; + std::string _region; //deprecated std::string _project; std::string _table; - std::string _odps_url; - std::string _tunnel_url; + std::string _odps_url; //deprecated + std::string _tunnel_url; //deprecated std::string _access_key; std::string _secret_key; - std::string _public_access; + std::string _public_access; //deprecated + std::string _endpoint; + std::string _quota; + Status _init_status = Status::OK(); }; class TrinoConnectorTableDescriptor : public TableDescriptor { diff --git a/be/src/vec/exec/format/table/max_compute_jni_reader.cpp b/be/src/vec/exec/format/table/max_compute_jni_reader.cpp index f3fd093ca0a391..d1a71fd1a2f9d9 100644 --- a/be/src/vec/exec/format/table/max_compute_jni_reader.cpp +++ b/be/src/vec/exec/format/table/max_compute_jni_reader.cpp @@ -63,19 +63,21 @@ MaxComputeJniReader::MaxComputeJniReader(const MaxComputeTableDescriptor* mc_des } index++; } - std::map params = {{"region", _table_desc->region()}, - {"odps_url", _table_desc->odps_url()}, - {"tunnel_url", _table_desc->tunnel_url()}, - {"access_key", _table_desc->access_key()}, - {"secret_key", _table_desc->secret_key()}, - {"project", _table_desc->project()}, - {"partition_spec", _max_compute_params.partition_spec}, - {"table", _table_desc->table()}, - {"public_access", _table_desc->public_access()}, - {"start_offset", std::to_string(_range.start_offset)}, - {"split_size", std::to_string(_range.size)}, - {"required_fields", required_fields.str()}, - {"columns_types", columns_types.str()}}; + std::map params = { + {"access_key", _table_desc->access_key()}, + {"secret_key", _table_desc->secret_key()}, + {"endpoint", _table_desc->endpoint()}, + {"quota", _table_desc->quota()}, + {"project", _table_desc->project()}, + {"table", _table_desc->table()}, + + {"session_id", _max_compute_params.session_id}, + {"scan_serializer", _max_compute_params.table_batch_read_session}, + + {"start_offset", std::to_string(_range.start_offset)}, + {"split_size", std::to_string(_range.size)}, + {"required_fields", required_fields.str()}, + {"columns_types", columns_types.str()}}; _jni_connector = std::make_unique( "org/apache/doris/maxcompute/MaxComputeJniScanner", params, column_names); } diff --git a/be/src/vec/exec/scan/vfile_scanner.cpp b/be/src/vec/exec/scan/vfile_scanner.cpp index 75ecc744ac18fb..e7dd2b1aa65061 100644 --- a/be/src/vec/exec/scan/vfile_scanner.cpp +++ b/be/src/vec/exec/scan/vfile_scanner.cpp @@ -752,6 +752,9 @@ Status VFileScanner::_get_next_reader() { range.table_format_params.table_format_type == "max_compute") { const auto* mc_desc = static_cast( _real_tuple_desc->table_desc()); + if (!mc_desc->init_status()) { + return mc_desc->init_status(); + } std::unique_ptr mc_reader = MaxComputeJniReader::create_unique( mc_desc, range.table_format_params.max_compute_params, _file_slot_descs, range, _state, _profile); diff --git a/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/vec/ColumnValue.java b/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/vec/ColumnValue.java index 0d1c522f9cbf06..640fd775c98013 100644 --- a/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/vec/ColumnValue.java +++ b/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/vec/ColumnValue.java @@ -59,6 +59,18 @@ public interface ColumnValue { LocalDate getDate(); + default String getChar() { + return getString(); + } + + default byte[] getCharAsBytes() { + return getStringAsBytes(); + } + + default boolean canGetCharAsBytes() { + return canGetStringAsBytes(); + } + LocalDateTime getDateTime(); byte[] getBytes(); diff --git a/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/vec/VectorColumn.java b/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/vec/VectorColumn.java index bf929ed9e2b77b..1542174cc1c587 100644 --- a/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/vec/VectorColumn.java +++ b/fe/be-java-extensions/java-common/src/main/java/org/apache/doris/common/jni/vec/VectorColumn.java @@ -1540,6 +1540,12 @@ public void appendValue(ColumnValue o) { appendDateTime(o.getDateTime()); break; case CHAR: + if (o.canGetCharAsBytes()) { + appendBytesAndOffset(o.getCharAsBytes()); + } else { + appendStringAndOffset(o.getChar()); + } + break; case VARCHAR: case STRING: if (o.canGetStringAsBytes()) { diff --git a/fe/be-java-extensions/max-compute-scanner/pom.xml b/fe/be-java-extensions/max-compute-scanner/pom.xml index 9f9fa50e972891..27760227b69885 100644 --- a/fe/be-java-extensions/max-compute-scanner/pom.xml +++ b/fe/be-java-extensions/max-compute-scanner/pom.xml @@ -43,6 +43,7 @@ under the License. com.aliyun.odps odps-sdk-core + ${maxcompute.version} org.codehaus.jackson @@ -54,6 +55,11 @@ under the License. + + com.aliyun.odps + odps-sdk-table-api + ${maxcompute.version} + org.apache.arrow arrow-vector diff --git a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeColumnValue.java b/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeColumnValue.java index 644caf80d97d0a..0c3a9283a8152f 100644 --- a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeColumnValue.java +++ b/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeColumnValue.java @@ -23,14 +23,17 @@ import org.apache.arrow.vector.BigIntVector; import org.apache.arrow.vector.BitVector; import org.apache.arrow.vector.DateDayVector; -import org.apache.arrow.vector.DateMilliVector; 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.TimeStampMicroTZVector; +import org.apache.arrow.vector.TimeStampMilliTZVector; +import org.apache.arrow.vector.TimeStampNanoTZVector; import org.apache.arrow.vector.TimeStampNanoVector; +import org.apache.arrow.vector.TimeStampSecTZVector; import org.apache.arrow.vector.TinyIntVector; import org.apache.arrow.vector.ValueVector; import org.apache.arrow.vector.VarBinaryVector; @@ -38,13 +41,18 @@ import org.apache.arrow.vector.complex.ListVector; import org.apache.arrow.vector.complex.MapVector; import org.apache.arrow.vector.complex.StructVector; +import org.apache.arrow.vector.holders.NullableTimeStampNanoHolder; +import org.apache.arrow.vector.types.pojo.ArrowType; import org.apache.log4j.Logger; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteOrder; +import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Arrays; import java.util.List; /** @@ -193,6 +201,33 @@ public String getString() { return v == null ? new String(new byte[0]) : v; } + + + public String getChar() { + skippedIfNull(); + VarCharVector varcharCol = (VarCharVector) column; + return varcharCol.getObject(idx++).toString().stripTrailing(); + } + + // Maybe I can use `appendBytesAndOffset(byte[] src, int offset, int length)` to reduce the creation of byte[]. + // But I haven't figured out how to write it elegantly. + public byte[] getCharAsBytes() { + skippedIfNull(); + VarCharVector varcharCol = (VarCharVector) column; + byte[] v = varcharCol.getObject(idx++).getBytes(); + + if (v == null) { + return new byte[0]; + } + + int end = v.length - 1; + while (end >= 0 && v[end] == ' ') { + end--; + } + return (end == -1) ? new byte[0] : Arrays.copyOfRange(v, 0, end + 1); + } + + @Override public byte[] getStringAsBytes() { skippedIfNull(); @@ -213,14 +248,52 @@ public LocalDate getDate() { public LocalDateTime getDateTime() { skippedIfNull(); LocalDateTime result; - if (column instanceof DateMilliVector) { - DateMilliVector datetimeCol = (DateMilliVector) column; - result = datetimeCol.getObject(idx++); + + ArrowType.Timestamp timestampType = ( ArrowType.Timestamp) column.getField().getFieldType().getType(); + if (timestampType.getUnit() == org.apache.arrow.vector.types.TimeUnit.MILLISECOND) { + result = convertToLocalDateTime((TimeStampMilliTZVector) column, idx++); } else { - TimeStampNanoVector datetimeCol = (TimeStampNanoVector) column; - result = datetimeCol.getObject(idx++); + NullableTimeStampNanoHolder valueHoder = new NullableTimeStampNanoHolder(); + ((TimeStampNanoVector) column).get(idx++, valueHoder); + long timestampNanos = valueHoder.value; + + result = LocalDateTime.ofEpochSecond(timestampNanos / 1_000_000_000, + (int) (timestampNanos % 1_000_000_000), java.time.ZoneOffset.UTC); } - return result == null ? LocalDateTime.MIN : result; + + /* + timestampType.getUnit() + result = switch (timestampType.getUnit()) { + case MICROSECOND -> convertToLocalDateTime((TimeStampMicroTZVector) column, idx++); + case SECOND -> convertToLocalDateTime((TimeStampSecTZVector) column, idx++); + case MILLISECOND -> convertToLocalDateTime((TimeStampMilliTZVector) column, idx++); + case NANOSECOND -> convertToLocalDateTime((TimeStampNanoTZVector) column, idx++); + }; + + Because : + MaxCompute type => Doris Type + DATETIME => ScalarType.createDatetimeV2Type(3) + TIMESTAMP_NTZ => ScalarType.createDatetimeV2Type(6); + + and + TableBatchReadSession + .withArrowOptions ( + ArrowOptions.newBuilder() + .withDatetimeUnit(TimestampUnit.MILLI) + .withTimestampUnit(TimestampUnit.NANO) + .build() + ) + , + TIMESTAMP_NTZ is NTZ => column is TimeStampNanoVector + + So: + case SECOND -> convertToLocalDateTime((TimeStampSecTZVector) column, idx++); + case MICROSECOND -> convertToLocalDateTime((TimeStampMicroTZVector) column, idx++); + case NANOSECOND -> convertToLocalDateTime((TimeStampNanoTZVector) column, idx++); + may never be used. + */ + + return result; } @Override @@ -248,9 +321,10 @@ public void unpackArray(List values) { public void unpackMap(List keys, List values) { skippedIfNull(); MapVector mapCol = (MapVector) column; - int elemSize = mapCol.getObject(idx).size(); - FieldVector keyList = mapCol.getDataVector().getChildrenFromFields().get(0); - FieldVector valList = mapCol.getDataVector().getChildrenFromFields().get(1); + int elemSize = mapCol.getElementEndIndex(idx) - mapCol.getElementStartIndex(idx); + List innerCols = ((StructVector) mapCol.getDataVector()).getChildrenFromFields(); + FieldVector keyList = innerCols.get(0); + FieldVector valList = innerCols.get(1); for (int i = 0; i < elemSize; i++) { MaxComputeColumnValue key = new MaxComputeColumnValue(keyList, offset); keys.add(key); @@ -265,10 +339,35 @@ public void unpackMap(List keys, List values) { public void unpackStruct(List structFieldIndex, List values) { skippedIfNull(); StructVector structCol = (StructVector) column; + List innerCols = structCol.getChildrenFromFields(); for (Integer fieldIndex : structFieldIndex) { - MaxComputeColumnValue val = new MaxComputeColumnValue(structCol.getChildByOrdinal(fieldIndex), idx); + MaxComputeColumnValue val = new MaxComputeColumnValue(innerCols.get(fieldIndex), idx); values.add(val); } idx++; } + + public static LocalDateTime convertToLocalDateTime(TimeStampMilliTZVector milliTZVector, int index) { + long timestampMillis = milliTZVector.get(index); + return LocalDateTime.ofInstant(Instant.ofEpochMilli(timestampMillis), ZoneId.systemDefault()); + } + + public static LocalDateTime convertToLocalDateTime(TimeStampNanoTZVector nanoTZVector, int index) { + long timestampNanos = nanoTZVector.get(index); + return LocalDateTime.ofInstant(Instant.ofEpochSecond(timestampNanos / 1_000_000_000, + timestampNanos % 1_000_000_000), ZoneId.systemDefault()); + } + + public static LocalDateTime convertToLocalDateTime(TimeStampSecTZVector secTZVector, int index) { + long timestampSeconds = secTZVector.get(index); + return LocalDateTime.ofInstant(Instant.ofEpochSecond(timestampSeconds), ZoneId.systemDefault()); + } + + public static LocalDateTime convertToLocalDateTime(TimeStampMicroTZVector microTZVector, int index) { + long timestampMicros = microTZVector.get(index); + long seconds = timestampMicros / 1_000_000; + long nanos = (timestampMicros % 1_000_000) * 1_000; + + return LocalDateTime.ofInstant(Instant.ofEpochSecond(seconds, nanos), ZoneId.systemDefault()); + } } diff --git a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java b/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java index 9c76a330a644e4..df8066a9fa3241 100644 --- a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java +++ b/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeJniScanner.java @@ -20,78 +20,75 @@ import org.apache.doris.common.jni.JniScanner; import org.apache.doris.common.jni.vec.ColumnType; -import com.aliyun.odps.Column; -import com.aliyun.odps.OdpsType; -import com.aliyun.odps.PartitionSpec; -import com.aliyun.odps.data.ArrowRecordReader; -import com.aliyun.odps.tunnel.TableTunnel; -import com.aliyun.odps.tunnel.TunnelException; -import com.aliyun.odps.type.TypeInfo; -import com.aliyun.odps.type.TypeInfoFactory; +import com.aliyun.odps.Odps; +import com.aliyun.odps.account.Account; +import com.aliyun.odps.account.AliyunAccount; +import com.aliyun.odps.table.configuration.CompressionCodec; +import com.aliyun.odps.table.configuration.ReaderOptions; +import com.aliyun.odps.table.enviroment.Credentials; +import com.aliyun.odps.table.enviroment.EnvironmentSettings; +import com.aliyun.odps.table.read.SplitReader; +import com.aliyun.odps.table.read.TableBatchReadSession; +import com.aliyun.odps.table.read.split.InputSplit; +import com.aliyun.odps.table.read.split.impl.IndexedInputSplit; +import com.aliyun.odps.table.read.split.impl.RowRangeInputSplit; import com.google.common.base.Strings; -import org.apache.arrow.memory.RootAllocator; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.commons.lang3.StringUtils; import org.apache.log4j.Logger; +import java.io.ByteArrayInputStream; import java.io.IOException; -import java.util.ArrayList; -import java.util.Comparator; +import java.io.ObjectInputStream; +import java.util.Base64; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; /** * MaxComputeJ JniScanner. BE will read data from the scanner object. */ public class MaxComputeJniScanner extends JniScanner { private static final Logger LOG = Logger.getLogger(MaxComputeJniScanner.class); - private static final String REGION = "region"; - private static final String PROJECT = "project"; - private static final String PARTITION_SPEC = "partition_spec"; - private static final String TABLE = "table"; + + private static final String ACCESS_KEY = "access_key"; private static final String SECRET_KEY = "secret_key"; - private static final String ODPS_URL = "odps_url"; - private static final String TUNNEL_URL = "tunnel_url"; + private static final String ENDPOINT = "endpoint"; + private static final String QUOTA = "quota"; + private static final String PROJECT = "project"; + private static final String TABLE = "table"; + private static final String START_OFFSET = "start_offset"; private static final String SPLIT_SIZE = "split_size"; - private static final String PUBLIC_ACCESS = "public_access"; - private final Map tableScans = new ConcurrentHashMap<>(); - private final String region; - private final String project; - private final String table; - private RootAllocator arrowAllocator; - private PartitionSpec partitionSpec; - private Set partitionColumns; - private MaxComputeTableScan curTableScan; + private static final String SESSION_ID = "session_id"; + private static final String SCAN_SERIALIZER = "scan_serializer"; + + + private enum SplitType { + BYTE_SIZE, + ROW_OFFSET + } + + private SplitType splitType; + private TableBatchReadSession scan; + public String sessionId; + + private String project; //final ??? + private String table; + + private SplitReader currentSplitReader; private MaxComputeColumnValue columnValue; - private long remainBatchRows = 0; - private long totalRows = 0; - private ArrowRecordReader curReader; - private List readColumns; + private Map readColumnsToId; + private long startOffset = -1L; - private int retryCount = 2; private long splitSize = -1L; - private final Map refreshParams; + public EnvironmentSettings settings; + public MaxComputeJniScanner(int batchSize, Map params) { - region = Objects.requireNonNull(params.get(REGION), "required property '" + REGION + "'."); - project = Objects.requireNonNull(params.get(PROJECT), "required property '" + PROJECT + "'."); - table = Objects.requireNonNull(params.get(TABLE), "required property '" + TABLE + "'."); - refreshParams = params; - tableScans.putIfAbsent(tableUniqKey(), newTableScan(params)); - curTableScan = tableScans.get(tableUniqKey()); - String partitionSpec = params.get(PARTITION_SPEC); - if (StringUtils.isNotEmpty(partitionSpec)) { - this.partitionSpec = new PartitionSpec(partitionSpec); - } String[] requiredFields = params.get("required_fields").split(","); String[] types = params.get("columns_types").split("#"); ColumnType[] columnTypes = new ColumnType[types.length]; @@ -99,38 +96,58 @@ public MaxComputeJniScanner(int batchSize, Map params) { columnTypes[i] = ColumnType.parseType(requiredFields[i], types[i]); } initTableInfo(columnTypes, requiredFields, batchSize); - } - - public void refreshTableScan() { - curTableScan = newTableScan(refreshParams); - tableScans.put(tableUniqKey(), curTableScan); - } - private MaxComputeTableScan newTableScan(Map params) { if (!Strings.isNullOrEmpty(params.get(START_OFFSET)) && !Strings.isNullOrEmpty(params.get(SPLIT_SIZE))) { startOffset = Long.parseLong(params.get(START_OFFSET)); splitSize = Long.parseLong(params.get(SPLIT_SIZE)); + if (splitSize == -1) { + splitType = SplitType.BYTE_SIZE; + } else { + splitType = SplitType.ROW_OFFSET; + } } + String accessKey = Objects.requireNonNull(params.get(ACCESS_KEY), "required property '" + ACCESS_KEY + "'."); String secretKey = Objects.requireNonNull(params.get(SECRET_KEY), "required property '" + SECRET_KEY + "'."); - boolean enablePublicAccess = Boolean.parseBoolean(params.getOrDefault(PUBLIC_ACCESS, "false")); - return new MaxComputeTableScan(params.get(ODPS_URL), params.get(TUNNEL_URL), region, project, table, - accessKey, secretKey, enablePublicAccess); - } + String endpoint = Objects.requireNonNull(params.get(ENDPOINT), "required property '" + ENDPOINT + "'."); + String quota = Objects.requireNonNull(params.get(QUOTA), "required property '" + QUOTA + "'."); + String scanSerializer = Objects.requireNonNull(params.get(SCAN_SERIALIZER), + "required property '" + SCAN_SERIALIZER + "'."); + project = Objects.requireNonNull(params.get(PROJECT), "required property '" + PROJECT + "'."); + table = Objects.requireNonNull(params.get(TABLE), "required property '" + TABLE + "'."); + sessionId = Objects.requireNonNull(params.get(SESSION_ID), "required property '" + SESSION_ID + "'."); + + + Account account = new AliyunAccount(accessKey, secretKey); + Odps odps = new Odps(account); + + odps.setDefaultProject(project); + odps.setEndpoint(endpoint); + + Credentials credentials = Credentials.newBuilder().withAccount(odps.getAccount()) + .withAppAccount(odps.getAppAccount()).build(); - public String tableUniqKey() { - return region + "#" + project + "." + table; + settings = EnvironmentSettings.newBuilder() + .withCredentials(credentials) + .withServiceEndpoint(odps.getEndpoint()) + .withQuotaName(quota) + .build(); + + try { + scan = (TableBatchReadSession) deserialize(scanSerializer); + } catch (Exception e) { + LOG.info("deserialize TableBatchReadSession failed.", e); + } } + @Override protected void initTableInfo(ColumnType[] requiredTypes, String[] requiredFields, int batchSize) { super.initTableInfo(requiredTypes, requiredFields, batchSize); - readColumns = new ArrayList<>(); readColumnsToId = new HashMap<>(); for (int i = 0; i < fields.length; i++) { if (!Strings.isNullOrEmpty(fields[i])) { - readColumns.add(createOdpsColumn(i, types[i])); readColumnsToId.put(fields[i], i); } } @@ -138,213 +155,92 @@ protected void initTableInfo(ColumnType[] requiredTypes, String[] requiredFields @Override public void open() throws IOException { - // reorder columns - List columnList = curTableScan.getSchema().getColumns(); - columnList.addAll(curTableScan.getSchema().getPartitionColumns()); - Map columnRank = new HashMap<>(); - for (int i = 0; i < columnList.size(); i++) { - columnRank.put(columnList.get(i).getName(), i); - } - // Downloading columns data from Max compute only supports the order of table metadata. - // We might get an error message if no sort here: Column reorder is not supported in legacy arrow mode. - readColumns.sort((Comparator.comparing(o -> columnRank.get(o.getName())))); - if (readColumns.isEmpty()) { - return; - } try { - TableTunnel.DownloadSession session; - if (partitionSpec != null) { - session = curTableScan.openDownLoadSession(partitionSpec); + InputSplit split; + if (splitType == SplitType.BYTE_SIZE) { + split = new IndexedInputSplit(sessionId, (int) startOffset); } else { - session = curTableScan.openDownLoadSession(); - } - long start = startOffset == -1L ? 0 : startOffset; - long recordCount = session.getRecordCount(); - totalRows = splitSize > 0 ? Math.min(splitSize, recordCount) : recordCount; - partitionColumns = session.getSchema().getPartitionColumns().stream() - .map(Column::getName) - .collect(Collectors.toSet()); - List pushDownColumns = new ArrayList<>(readColumns); - pushDownColumns.removeIf(e -> partitionColumns.contains(e.getName())); - if (pushDownColumns.isEmpty() && !partitionColumns.isEmpty()) { - // query columns required non-null, when query partition table - pushDownColumns.add(session.getSchema().getColumn(0)); - } - if (totalRows == 0) { - return; - } - arrowAllocator = new RootAllocator(Integer.MAX_VALUE); - curReader = session.openArrowRecordReader(start, totalRows, pushDownColumns, arrowAllocator); - remainBatchRows = totalRows; - } catch (TunnelException e) { - if (retryCount > 0 && e.getErrorMsg().contains("TableModified")) { - retryCount--; - // try to refresh table scan and re-open odps - refreshTableScan(); - open(); - } else { - retryCount = 2; - throw new IOException(e); + split = new RowRangeInputSplit(sessionId, startOffset, splitSize); } + + currentSplitReader = scan.createArrowReader(split, ReaderOptions.newBuilder().withSettings(settings) + .withCompressionCodec(CompressionCodec.ZSTD) + .withReuseBatch(true) + .build()); + + } catch (IOException e) { + LOG.info("createArrowReader failed.", e); } catch (Exception e) { close(); throw new IOException(e); } } - private Column createOdpsColumn(int colIdx, ColumnType dorisType) { - TypeInfo odpsType = getOdpsType(dorisType); - return new Column(fields[colIdx], odpsType); - } - - private static TypeInfo getOdpsType(ColumnType dorisType) { - TypeInfo odpsType; - switch (dorisType.getType()) { - case BOOLEAN: - odpsType = TypeInfoFactory.BOOLEAN; - break; - case TINYINT: - odpsType = TypeInfoFactory.TINYINT; - break; - case SMALLINT: - odpsType = TypeInfoFactory.SMALLINT; - break; - case INT: - odpsType = TypeInfoFactory.INT; - break; - case BIGINT: - odpsType = TypeInfoFactory.BIGINT; - break; - case DECIMAL32: - case DECIMAL64: - case DECIMAL128: - case DECIMALV2: - odpsType = TypeInfoFactory.getDecimalTypeInfo(dorisType.getPrecision(), dorisType.getScale()); - break; - case FLOAT: - odpsType = TypeInfoFactory.FLOAT; - break; - case DOUBLE: - odpsType = TypeInfoFactory.DOUBLE; - break; - case DATETIME: - case DATETIMEV2: - odpsType = TypeInfoFactory.DATETIME; - break; - case DATE: - case DATEV2: - odpsType = TypeInfoFactory.DATE; - break; - case CHAR: - odpsType = TypeInfoFactory.getCharTypeInfo(dorisType.getLength()); - break; - case VARCHAR: - odpsType = TypeInfoFactory.getVarcharTypeInfo(dorisType.getLength()); - break; - case STRING: - odpsType = TypeInfoFactory.getPrimitiveTypeInfo(OdpsType.STRING); - break; - case ARRAY: - TypeInfo elementType = getOdpsType(dorisType.getChildTypes().get(0)); - odpsType = TypeInfoFactory.getArrayTypeInfo(elementType); - break; - case MAP: - TypeInfo keyType = getOdpsType(dorisType.getChildTypes().get(0)); - TypeInfo valueType = getOdpsType(dorisType.getChildTypes().get(1)); - odpsType = TypeInfoFactory.getMapTypeInfo(keyType, valueType); - break; - case STRUCT: - List names = dorisType.getChildNames(); - List typeInfos = new ArrayList<>(); - for (ColumnType childType : dorisType.getChildTypes()) { - typeInfos.add(getOdpsType(childType)); - } - odpsType = TypeInfoFactory.getStructTypeInfo(names, typeInfos); - break; - default: - throw new RuntimeException("Unsupported transform for column type: " + dorisType.getType()); - } - return odpsType; - } - @Override public void close() throws IOException { - String tableName = tableUniqKey(); - MaxComputeTableScan scan = tableScans.get(tableName); - if (scan != null && scan.endOfScan()) { - tableScans.remove(tableName); - } - remainBatchRows = 0; - totalRows = 0; startOffset = -1; splitSize = -1; - if (curReader != null) { - arrowAllocator.close(); - arrowAllocator = null; - curReader.close(); - curReader = null; - } + currentSplitReader = null; + settings = null; + scan = null; + readColumnsToId.clear(); } @Override protected int getNext() throws IOException { - if (curReader == null) { + if (currentSplitReader == null) { return 0; } columnValue = new MaxComputeColumnValue(); - int expectedRows = (int) Math.min(batchSize, remainBatchRows); - int realRows = readVectors(expectedRows); - if (remainBatchRows <= 0) { - return 0; - } - remainBatchRows -= realRows; - curTableScan.increaseReadRows(realRows); - return realRows; + int expectedRows = batchSize; + return readVectors(expectedRows); } private int readVectors(int expectedRows) throws IOException { - VectorSchemaRoot batch; int curReadRows = 0; while (curReadRows < expectedRows) { - batch = curReader.read(); - if (batch == null) { + try { + if (!currentSplitReader.hasNext()) { + currentSplitReader.close(); + currentSplitReader = null; + break; + } + } catch (Exception e) { + LOG.info("currentSplitReader hasNext fail", e); break; } + try { - List fieldVectors = batch.getFieldVectors(); + VectorSchemaRoot data = currentSplitReader.get(); + if (data.getRowCount() == 0) { + break; + } + + List fieldVectors = data.getFieldVectors(); int batchRows = 0; for (FieldVector column : fieldVectors) { Integer readColumnId = readColumnsToId.get(column.getName()); + batchRows = column.getValueCount(); if (readColumnId == null) { - // use for partition if no column need to read. - batchRows = column.getValueCount(); continue; } columnValue.reset(column); - batchRows = column.getValueCount(); for (int j = 0; j < batchRows; j++) { appendData(readColumnId, columnValue); } } - if (partitionSpec != null) { - for (String partitionColumn : partitionColumns) { - String partitionValue = partitionSpec.get(partitionColumn); - Integer readColumnId = readColumnsToId.get(partitionColumn); - if (readColumnId != null && partitionValue != null) { - MaxComputePartitionValue value = new MaxComputePartitionValue(partitionValue); - for (int i = 0; i < batchRows; i++) { - appendData(readColumnId, value); - } - } - } - } curReadRows += batchRows; } catch (Exception e) { throw new RuntimeException("Fail to read arrow data, reason: " + e.getMessage(), e); - } finally { - batch.close(); } } return curReadRows; } + + private static Object deserialize(String serializedString) throws IOException, ClassNotFoundException { + byte[] serializedBytes = Base64.getDecoder().decode(serializedString); + ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(serializedBytes); + ObjectInputStream objectInputStream = new ObjectInputStream(byteArrayInputStream); + return objectInputStream.readObject(); + } } diff --git a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputePartitionValue.java b/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputePartitionValue.java deleted file mode 100644 index cb76447e589622..00000000000000 --- a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputePartitionValue.java +++ /dev/null @@ -1,137 +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 org.apache.doris.maxcompute; - -import org.apache.doris.common.jni.vec.ColumnValue; - -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.charset.StandardCharsets; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.util.List; - -/** - * MaxCompute Column value in vector column - */ -public class MaxComputePartitionValue implements ColumnValue { - private String partitionValue; - - public MaxComputePartitionValue(String partitionValue) { - reset(partitionValue); - } - - public void reset(String partitionValue) { - this.partitionValue = partitionValue; - } - - @Override - public boolean canGetStringAsBytes() { - return false; - } - - @Override - public boolean isNull() { - return false; - } - - @Override - public boolean getBoolean() { - throw new UnsupportedOperationException(); - } - - @Override - public byte getByte() { - throw new UnsupportedOperationException(); - } - - @Override - public short getShort() { - throw new UnsupportedOperationException(); - } - - @Override - public int getInt() { - return Integer.parseInt(partitionValue); - } - - @Override - public float getFloat() { - throw new UnsupportedOperationException(); - } - - @Override - public long getLong() { - return Long.parseLong(partitionValue); - } - - @Override - public double getDouble() { - throw new UnsupportedOperationException(); - } - - @Override - public BigInteger getBigInteger() { - return BigInteger.valueOf(getLong()); - } - - @Override - public BigDecimal getDecimal() { - return BigDecimal.valueOf(getDouble()); - } - - @Override - public String getString() { - return partitionValue; - } - - @Override - public byte[] getStringAsBytes() { - throw new UnsupportedOperationException(); - } - - @Override - public LocalDate getDate() { - throw new UnsupportedOperationException(); - } - - @Override - public LocalDateTime getDateTime() { - throw new UnsupportedOperationException(); - } - - @Override - public byte[] getBytes() { - return partitionValue.getBytes(StandardCharsets.UTF_8); - } - - @Override - public void unpackArray(List values) { - throw new UnsupportedOperationException(); - } - - @Override - public void unpackMap(List keys, List values) { - throw new UnsupportedOperationException(); - } - - @Override - public void unpackStruct(List structFieldIndex, List values) { - throw new UnsupportedOperationException(); - } -} diff --git a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeTableScan.java b/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeTableScan.java deleted file mode 100644 index 0de1cb17e79230..00000000000000 --- a/fe/be-java-extensions/max-compute-scanner/src/main/java/org/apache/doris/maxcompute/MaxComputeTableScan.java +++ /dev/null @@ -1,114 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.maxcompute; - -import com.aliyun.odps.Odps; -import com.aliyun.odps.PartitionSpec; -import com.aliyun.odps.TableSchema; -import com.aliyun.odps.account.AliyunAccount; -import com.aliyun.odps.tunnel.TableTunnel; -import com.aliyun.odps.tunnel.TunnelException; -import org.apache.commons.lang3.StringUtils; - -import java.io.IOException; - -/** - * MaxComputeJ JniScanner. BE will read data from the scanner object. - */ -public class MaxComputeTableScan { - private static final String odpsUrlTemplate = "http://service.{}.maxcompute.aliyun-inc.com/api"; - private static final String tunnelUrlTemplate = "http://dt.{}.maxcompute.aliyun-inc.com"; - private final Odps odps; - private final TableTunnel tunnel; - private final String project; - private final String table; - private volatile long readRows = 0; - private long totalRows = 0; - - public MaxComputeTableScan(String odpsUrl, String tunnelUrl, String region, String project, String table, - String accessKey, String secretKey, boolean enablePublicAccess) { - this.project = project; - this.table = table; - odps = new Odps(new AliyunAccount(accessKey, secretKey)); - setOdpsUrl(odpsUrl, region, enablePublicAccess); - odps.setDefaultProject(this.project); - tunnel = new TableTunnel(odps); - setTunnelUrl(tunnelUrl, region, enablePublicAccess); - } - - private void setOdpsUrl(String defaultOdpsUrl, String region, boolean enablePublicAccess) { - String odpsUrl; - if (StringUtils.isNotEmpty(defaultOdpsUrl)) { - odpsUrl = defaultOdpsUrl; - } else { - odpsUrl = odpsUrlTemplate.replace("{}", region); - if (enablePublicAccess) { - odpsUrl = odpsUrl.replace("-inc", ""); - } - } - odps.setEndpoint(odpsUrl); - } - - private void setTunnelUrl(String defaultTunnelUrl, String region, boolean enablePublicAccess) { - String tunnelUrl; - if (StringUtils.isNotEmpty(defaultTunnelUrl)) { - tunnelUrl = defaultTunnelUrl; - } else { - tunnelUrl = tunnelUrlTemplate.replace("{}", region); - if (enablePublicAccess) { - tunnelUrl = tunnelUrl.replace("-inc", ""); - } - } - tunnel.setEndpoint(tunnelUrl); - } - - public TableSchema getSchema() { - return odps.tables().get(table).getSchema(); - } - - public TableTunnel.DownloadSession openDownLoadSession() throws IOException { - TableTunnel.DownloadSession tableSession; - try { - tableSession = tunnel.getDownloadSession(project, table, null); - totalRows = tableSession.getRecordCount(); - } catch (TunnelException e) { - throw new IOException(e); - } - return tableSession; - } - - public TableTunnel.DownloadSession openDownLoadSession(PartitionSpec partitionSpec) throws IOException { - TableTunnel.DownloadSession tableSession; - try { - tableSession = tunnel.getDownloadSession(project, table, partitionSpec, null); - totalRows = tableSession.getRecordCount(); - } catch (TunnelException e) { - throw new IOException(e); - } - return tableSession; - } - - public synchronized void increaseReadRows(long rows) { - // multi-thread writing must be synchronized - readRows += rows; - } - - public boolean endOfScan() { - return readRows >= totalRows; - } -} diff --git a/fe/be-java-extensions/max-compute-scanner/src/test/java/org/apache/doris/maxcompute/MaxComputeJniScannerTest.java b/fe/be-java-extensions/max-compute-scanner/src/test/java/org/apache/doris/maxcompute/MaxComputeJniScannerTest.java deleted file mode 100644 index f14c0610d4327a..00000000000000 --- a/fe/be-java-extensions/max-compute-scanner/src/test/java/org/apache/doris/maxcompute/MaxComputeJniScannerTest.java +++ /dev/null @@ -1,158 +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 org.apache.doris.maxcompute; - -import com.aliyun.odps.Column; -import com.aliyun.odps.PartitionSpec; -import com.aliyun.odps.TableSchema; -import com.aliyun.odps.data.ArrowRecordReader; -import com.aliyun.odps.tunnel.TableTunnel; -import com.aliyun.odps.tunnel.TunnelException; -import com.aliyun.odps.type.TypeInfoFactory; -import mockit.Mock; -import mockit.MockUp; -import mockit.Mocked; -import org.apache.arrow.memory.BufferAllocator; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.io.IOException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class MaxComputeJniScannerTest { - - @Mocked - private TableTunnel.DownloadSession session; - private Map paramsMc = new HashMap() { - { - put("region", "cn-beijing"); - put("project", "test_pj"); - put("table", "test_tb"); - put("access_key", "ak"); - put("secret_key", "sk"); - put("start_offset", "0"); - put("split_size", "128"); - put("partition_spec", "p1=2022-06"); - put("required_fields", "boolean,tinyint,smallint,int,bigint,float,double," - + "date,timestamp,char,varchar,string,decimalv2,decimal64," - + "decimal18,timestamp4"); - put("columns_types", "boolean#tinyint#smallint#int#bigint#float#double#" - + "date#timestamp#char(10)#varchar(10)#string#decimalv2(12,4)#decimal64(10,3)#" - + "decimal(18,5)#timestamp(4)"); - } - }; - private MaxComputeJniScanner scanner = new MaxComputeJniScanner(32, paramsMc); - - @BeforeEach - public void init() { - new MockUp(MaxComputeJniScanner.class) { - @Mock - public TableSchema getSchema() { - return getTestSchema(); - } - }; - new MockUp(MaxComputeTableScan.class) { - @Mock - public TableSchema getSchema() { - return getTestSchema(); - } - - @Mock - public TableTunnel.DownloadSession openDownLoadSession() throws IOException { - return session; - } - - @Mock - public TableTunnel.DownloadSession openDownLoadSession(PartitionSpec partitionSpec) throws IOException { - return session; - } - }; - new MockUp(TableTunnel.DownloadSession.class) { - @Mock - public TableSchema getSchema() { - return getTestSchema(); - } - - @Mock - public long getRecordCount() { - return 10; - } - - @Mock - public ArrowRecordReader openArrowRecordReader(long start, long count, List columns, - BufferAllocator allocator) - throws TunnelException, IOException { - return null; - } - }; - } - - private TableSchema getTestSchema() { - TableSchema schema = new TableSchema(); - schema.addColumn(new Column("boolean", TypeInfoFactory.BOOLEAN)); - schema.addColumn(new Column("bigint", TypeInfoFactory.BIGINT)); - schema.addPartitionColumn(new Column("date", TypeInfoFactory.DATE)); - schema.addPartitionColumn(new Column("tinyint", TypeInfoFactory.TINYINT)); - schema.addPartitionColumn(new Column("smallint", TypeInfoFactory.SMALLINT)); - schema.addPartitionColumn(new Column("int", TypeInfoFactory.INT)); - schema.addPartitionColumn(new Column("timestamp", TypeInfoFactory.TIMESTAMP)); - schema.addPartitionColumn(new Column("char", TypeInfoFactory.getCharTypeInfo(10))); - schema.addPartitionColumn(new Column("varchar", TypeInfoFactory.getVarcharTypeInfo(10))); - schema.addPartitionColumn(new Column("string", TypeInfoFactory.STRING)); - schema.addPartitionColumn(new Column("float", TypeInfoFactory.FLOAT)); - schema.addPartitionColumn(new Column("double", TypeInfoFactory.DOUBLE)); - schema.addPartitionColumn(new Column("decimalv2", - TypeInfoFactory.getDecimalTypeInfo(12, 4))); - schema.addPartitionColumn(new Column("decimal64", - TypeInfoFactory.getDecimalTypeInfo(10, 3))); - schema.addPartitionColumn(new Column("decimal18", - TypeInfoFactory.getDecimalTypeInfo(18, 5))); - schema.addPartitionColumn(new Column("timestamp4", TypeInfoFactory.TIMESTAMP)); - return schema; - } - - @Test - public void testMaxComputeJniScanner() throws IOException { - scanner.open(); - scanner.getNext(); - scanner.close(); - } - - @Test - public void testMaxComputeJniScannerErr() { - try { - new MockUp(TableTunnel.DownloadSession.class) { - @Mock - public ArrowRecordReader openArrowRecordReader(long start, long count, List columns, - BufferAllocator allocator) - throws TunnelException, IOException { - throw new TunnelException("TableModified"); - } - }; - scanner.open(); - scanner.getNext(); - scanner.close(); - } catch (IOException e) { - Assertions.assertTrue(e.getCause() instanceof TunnelException); - Assertions.assertEquals(((TunnelException) e.getCause()).getErrorMsg(), "TableModified"); - } - } -} diff --git a/fe/fe-core/pom.xml b/fe/fe-core/pom.xml index ec5d993c6f343b..10753c14b66d30 100644 --- a/fe/fe-core/pom.xml +++ b/fe/fe-core/pom.xml @@ -442,6 +442,7 @@ under the License. com.aliyun.odps odps-sdk-core + ${maxcompute.version} antlr-runtime @@ -453,6 +454,11 @@ under the License. + + com.aliyun.odps + odps-sdk-table-api + ${maxcompute.version} + org.springframework.boot diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java index 4973a1caad9dcf..19c90f32648ac2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java @@ -730,6 +730,39 @@ public String getStringValue() { return new String(dateTimeChars, 0, 19); } + public String getStringValue(Type type) { + char[] dateTimeChars = new char[26]; // Enough to hold "YYYY-MM-DD HH:MM:SS.mmmmmm" + + // Populate the date part + fillPaddedValue(dateTimeChars, 0, year, 4); + dateTimeChars[4] = '-'; + fillPaddedValue(dateTimeChars, 5, month, 2); + dateTimeChars[7] = '-'; + fillPaddedValue(dateTimeChars, 8, day, 2); + + if (type.isDate() || type.isDateV2()) { + return new String(dateTimeChars, 0, 10); + } + + // Populate the time part + dateTimeChars[10] = ' '; + fillPaddedValue(dateTimeChars, 11, hour, 2); + dateTimeChars[13] = ':'; + fillPaddedValue(dateTimeChars, 14, minute, 2); + dateTimeChars[16] = ':'; + fillPaddedValue(dateTimeChars, 17, second, 2); + + if (type.isDatetimeV2()) { + int scale = ((ScalarType) type).getScalarScale(); + long scaledMicroseconds = (long) (microsecond / SCALE_FACTORS[scale]); + dateTimeChars[19] = '.'; + fillPaddedValue(dateTimeChars, 20, (int) scaledMicroseconds, scale); + return new String(dateTimeChars, 0, 20 + scale); + } + + return new String(dateTimeChars, 0, 19); + } + @Override public String getStringValueForArray(FormatOptions options) { return options.getNestedStringWrapper() + getStringValue() + options.getNestedStringWrapper(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java index ce200ef4fbf8b4..fcbc0a5e8fc0a3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalCatalog.java @@ -29,13 +29,17 @@ import com.aliyun.odps.Odps; import com.aliyun.odps.OdpsException; import com.aliyun.odps.Partition; +import com.aliyun.odps.Project; import com.aliyun.odps.account.Account; import com.aliyun.odps.account.AliyunAccount; -import com.aliyun.odps.tunnel.TableTunnel; -import com.google.common.base.Strings; +import com.aliyun.odps.security.SecurityManager; +import com.aliyun.odps.table.configuration.SplitOptions; +import com.aliyun.odps.table.enviroment.Credentials; +import com.aliyun.odps.table.enviroment.EnvironmentSettings; +import com.aliyun.odps.utils.StringUtils; import com.google.common.collect.ImmutableList; -import com.google.gson.annotations.SerializedName; -import org.apache.commons.lang3.StringUtils; +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; import java.util.ArrayList; import java.util.Iterator; @@ -45,87 +49,76 @@ public class MaxComputeExternalCatalog extends ExternalCatalog { private Odps odps; - private TableTunnel tunnel; - @SerializedName(value = "region") - private String region; - @SerializedName(value = "accessKey") private String accessKey; - @SerializedName(value = "secretKey") private String secretKey; - @SerializedName(value = "publicAccess") - private boolean enablePublicAccess; - private static final String odpsUrlTemplate = "http://service.{}.maxcompute.aliyun-inc.com/api"; - private static final String tunnelUrlTemplate = "http://dt.{}.maxcompute.aliyun-inc.com"; - private static String odpsUrl; - private static String tunnelUrl; + private String endpoint; + private String catalogOwner; + private String defaultProject; + private String quota; + private EnvironmentSettings settings; + + private String splitStrategy; + private SplitOptions splitOptions; + private long splitRowCount; + private long splitByteSize; + private static final List REQUIRED_PROPERTIES = ImmutableList.of( - MCProperties.REGION, - MCProperties.PROJECT + MCProperties.PROJECT, + MCProperties.ENDPOINT ); public MaxComputeExternalCatalog(long catalogId, String name, String resource, Map props, String comment) { super(catalogId, name, InitCatalogLog.Type.MAX_COMPUTE, comment); catalogProperty = new CatalogProperty(resource, props); - odpsUrl = props.getOrDefault(MCProperties.ODPS_ENDPOINT, ""); - tunnelUrl = props.getOrDefault(MCProperties.TUNNEL_SDK_ENDPOINT, ""); } @Override protected void initLocalObjectsImpl() { Map props = catalogProperty.getProperties(); - String region = props.get(MCProperties.REGION); - String defaultProject = props.get(MCProperties.PROJECT); - if (Strings.isNullOrEmpty(region)) { - throw new IllegalArgumentException("Missing required property '" + MCProperties.REGION + "'."); - } - if (Strings.isNullOrEmpty(defaultProject)) { - throw new IllegalArgumentException("Missing required property '" + MCProperties.PROJECT + "'."); - } - if (region.startsWith("oss-")) { - // may use oss-cn-beijing, ensure compatible - region = region.replace("oss-", ""); + + endpoint = props.get(MCProperties.ENDPOINT); + defaultProject = props.get(MCProperties.PROJECT); + quota = props.getOrDefault(MCProperties.QUOTA, MCProperties.DEFAULT_QUOTA); + + + splitStrategy = props.getOrDefault(MCProperties.SPLIT_STRATEGY, MCProperties.DEFAULT_SPLIT_STRATEGY); + if (splitStrategy.equals(MCProperties.SPLIT_BY_BYTE_SIZE_STRATEGY)) { + splitByteSize = Long.parseLong(props.getOrDefault(MCProperties.SPLIT_BYTE_SIZE, + MCProperties.DEFAULT_SPLIT_BYTE_SIZE)); + + splitOptions = SplitOptions.newBuilder() + .SplitByByteSize(splitByteSize) + .withCrossPartition(false) + .build(); + } else { + splitRowCount = Long.parseLong(props.getOrDefault(MCProperties.SPLIT_ROW_COUNT, + MCProperties.DEFAULT_SPLIT_ROW_COUNT)); + splitOptions = SplitOptions.newBuilder() + .SplitByRowOffset() + .withCrossPartition(false) + .build(); } - this.region = region; + + CloudCredential credential = MCProperties.getCredential(props); - if (!credential.isWhole()) { - throw new IllegalArgumentException("Max-Compute credential properties '" - + MCProperties.ACCESS_KEY + "' and '" + MCProperties.SECRET_KEY + "' are required."); - } accessKey = credential.getAccessKey(); secretKey = credential.getSecretKey(); + + + Account account = new AliyunAccount(accessKey, secretKey); this.odps = new Odps(account); - enablePublicAccess = Boolean.parseBoolean(props.getOrDefault(MCProperties.PUBLIC_ACCESS, "false")); - setOdpsUrl(region); odps.setDefaultProject(defaultProject); - tunnel = new TableTunnel(odps); - setTunnelUrl(region); - } + odps.setEndpoint(endpoint); + Credentials credentials = Credentials.newBuilder().withAccount(odps.getAccount()) + .withAppAccount(odps.getAppAccount()).build(); - private void setOdpsUrl(String region) { - if (StringUtils.isEmpty(odpsUrl)) { - odpsUrl = odpsUrlTemplate.replace("{}", region); - if (enablePublicAccess) { - odpsUrl = odpsUrl.replace("-inc", ""); - } - } - odps.setEndpoint(odpsUrl); - } - - private void setTunnelUrl(String region) { - if (StringUtils.isEmpty(tunnelUrl)) { - tunnelUrl = tunnelUrlTemplate.replace("{}", region); - if (enablePublicAccess) { - tunnelUrl = tunnelUrl.replace("-inc", ""); - } - } - tunnel.setEndpoint(tunnelUrl); - } - - public TableTunnel getTableTunnel() { - makeSureInitialized(); - return tunnel; + settings = EnvironmentSettings.newBuilder() + .withCredentials(credentials) + .withServiceEndpoint(odps.getEndpoint()) + .withQuotaName(quota) + .build(); } public Odps getClient() { @@ -136,9 +129,21 @@ public Odps getClient() { protected List listDatabaseNames() { List result = new ArrayList<>(); try { - // TODO: How to get all privileged project from max compute as databases? - // Now only have permission to show default project. - result.add(odps.projects().get(odps.getDefaultProject()).getName()); + result.add(defaultProject); + if (StringUtils.isNullOrEmpty(catalogOwner)) { + SecurityManager sm = odps.projects().get().getSecurityManager(); + String whoami = sm.runQuery("whoami", false); + + JsonObject js = JsonParser.parseString(whoami).getAsJsonObject(); + catalogOwner = js.get("DisplayName").getAsString(); + } + Iterator iterator = odps.projects().iterator(catalogOwner); + while (iterator.hasNext()) { + Project project = iterator.next(); + if (!project.getName().equals(defaultProject)) { + result.add(project.getName()); + } + } } catch (OdpsException e) { throw new RuntimeException(e); } @@ -149,7 +154,7 @@ protected List listDatabaseNames() { public boolean tableExist(SessionContext ctx, String dbName, String tblName) { makeSureInitialized(); try { - return odps.tables().exists(tblName); + return getClient().tables().exists(dbName, tblName); } catch (OdpsException e) { throw new RuntimeException(e); } @@ -195,19 +200,10 @@ public List listPartitionNames(String dbName, String tbl, long skip, lon public List listTableNames(SessionContext ctx, String dbName) { makeSureInitialized(); List result = new ArrayList<>(); - odps.tables().forEach(e -> result.add(e.getName())); + getClient().tables().forEach(e -> result.add(e.getName())); return result; } - /** - * use region to create data tunnel url - * @return region, required by jni scanner. - */ - public String getRegion() { - makeSureInitialized(); - return region; - } - public String getAccessKey() { makeSureInitialized(); return accessKey; @@ -218,26 +214,81 @@ public String getSecretKey() { return secretKey; } - public boolean enablePublicAccess() { + public String getEndpoint() { makeSureInitialized(); - return enablePublicAccess; + return endpoint; + } + + public String getDefaultProject() { + makeSureInitialized(); + return defaultProject; + } + + public String getQuota() { + return quota; + } + + public SplitOptions getSplitOption() { + return splitOptions; + } + + public EnvironmentSettings getSettings() { + return settings; + } + + public String getSplitStrategy() { + return splitStrategy; + } + + public long getSplitRowCount() { + return splitRowCount; + } + + + public long getSplitByteSize() { + return splitByteSize; } @Override public void checkProperties() throws DdlException { super.checkProperties(); + Map props = catalogProperty.getProperties(); for (String requiredProperty : REQUIRED_PROPERTIES) { - if (!catalogProperty.getProperties().containsKey(requiredProperty)) { + if (!props.containsKey(requiredProperty)) { throw new DdlException("Required property '" + requiredProperty + "' is missing"); } } - } - public String getOdpsUrl() { - return odpsUrl; - } + try { + splitStrategy = props.getOrDefault(MCProperties.SPLIT_STRATEGY, MCProperties.DEFAULT_SPLIT_STRATEGY); + if (splitStrategy.equals(MCProperties.SPLIT_BY_BYTE_SIZE_STRATEGY)) { + splitByteSize = Long.parseLong(props.getOrDefault(MCProperties.SPLIT_BYTE_SIZE, + MCProperties.DEFAULT_SPLIT_BYTE_SIZE)); + + if (splitByteSize < 10485760L) { + throw new DdlException(MCProperties.SPLIT_ROW_COUNT + " must be greater than or equal to 10485760"); + } + + } else if (splitStrategy.equals(MCProperties.SPLIT_BY_ROW_COUNT_STRATEGY)) { + splitRowCount = Long.parseLong(props.getOrDefault(MCProperties.SPLIT_ROW_COUNT, + MCProperties.DEFAULT_SPLIT_ROW_COUNT)); + if (splitRowCount <= 0) { + throw new DdlException(MCProperties.SPLIT_ROW_COUNT + " must be greater than 0"); + } - public String getTunnelUrl() { - return tunnelUrl; + } else { + throw new DdlException("property " + MCProperties.SPLIT_STRATEGY + "must is " + + MCProperties.SPLIT_BY_BYTE_SIZE_STRATEGY + " or " + MCProperties.SPLIT_BY_ROW_COUNT_STRATEGY); + } + } catch (NumberFormatException e) { + throw new DdlException("property " + MCProperties.SPLIT_BYTE_SIZE + "/" + + MCProperties.SPLIT_ROW_COUNT + "must be an integer"); + } + + CloudCredential credential = MCProperties.getCredential(props); + if (!credential.isWhole()) { + throw new DdlException("Max-Compute credential properties '" + + MCProperties.ACCESS_KEY + "' and '" + MCProperties.SECRET_KEY + "' are required."); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java index 0edae9de23bf6d..dc3232f79f5f71 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeExternalTable.java @@ -34,7 +34,6 @@ import com.aliyun.odps.OdpsType; import com.aliyun.odps.Table; -import com.aliyun.odps.tunnel.TunnelException; import com.aliyun.odps.type.ArrayTypeInfo; import com.aliyun.odps.type.CharTypeInfo; import com.aliyun.odps.type.DecimalTypeInfo; @@ -48,6 +47,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -61,6 +61,8 @@ public MaxComputeExternalTable(long id, String name, String dbName, MaxComputeEx super(id, name, catalog, dbName, TableType.MAX_COMPUTE_EXTERNAL_TABLE); } + private Map columnNameToOdpsColumn = new HashMap(); + @Override protected synchronized void makeSureInitialized() { super.makeSureInitialized(); @@ -69,27 +71,6 @@ protected synchronized void makeSureInitialized() { } } - public long getTotalRows() throws TunnelException { - // use for non-partitioned table - // partition table will read the entire partition on FE so get total rows is unnecessary. - makeSureInitialized(); - MaxComputeMetadataCache metadataCache = Env.getCurrentEnv().getExtMetaCacheMgr() - .getMaxComputeMetadataCache(catalog.getId()); - MaxComputeExternalCatalog mcCatalog = ((MaxComputeExternalCatalog) catalog); - return metadataCache.getCachedRowCount(dbName, name, null, key -> { - try { - return loadRowCount(mcCatalog, key); - } catch (TunnelException e) { - throw new RuntimeException(e); - } - }); - } - - private long loadRowCount(MaxComputeExternalCatalog catalog, MaxComputeCacheKey key) throws TunnelException { - return catalog.getTableTunnel() - .getDownloadSession(key.getDbName(), key.getTblName(), null) - .getRecordCount(); - } public List getPartitionColumns() { makeSureInitialized(); @@ -151,19 +132,34 @@ private static List parsePartitionValues(List partitionColumns, return partitionValues; } + public Map getColumnNameToOdpsColumn() { + return columnNameToOdpsColumn; + } + @Override public Optional initSchema() { // this method will be called at semantic parsing. makeSureInitialized(); - Table odpsTable = ((MaxComputeExternalCatalog) catalog).getClient().tables().get(name); + Table odpsTable = ((MaxComputeExternalCatalog) catalog).getClient().tables().get(dbName, name); List columns = odpsTable.getSchema().getColumns(); + + + for (com.aliyun.odps.Column column : columns) { + columnNameToOdpsColumn.put(column.getName(), column); + } + List schema = Lists.newArrayListWithCapacity(columns.size()); for (com.aliyun.odps.Column field : columns) { schema.add(new Column(field.getName(), mcTypeToDorisType(field.getTypeInfo()), true, null, - true, field.getComment(), true, -1)); + field.isNullable(), field.getComment(), true, -1)); } List partitionColumns = odpsTable.getSchema().getPartitionColumns(); + + for (com.aliyun.odps.Column partitionColumn : partitionColumns) { + columnNameToOdpsColumn.put(partitionColumn.getName(), partitionColumn); + } + List partitionSpecs; if (!partitionColumns.isEmpty()) { partitionSpecs = odpsTable.getPartitions().stream() @@ -239,10 +235,12 @@ private Type mcTypeToDorisType(TypeInfo typeInfo) { case DATE: { return ScalarType.createDateV2Type(); } - case DATETIME: - case TIMESTAMP: { + case DATETIME: { return ScalarType.createDatetimeV2Type(3); } + case TIMESTAMP_NTZ: { + return ScalarType.createDatetimeV2Type(6); + } case ARRAY: { ArrayTypeInfo arrayType = (ArrayTypeInfo) typeInfo; Type innerType = mcTypeToDorisType(arrayType.getElementTypeInfo()); @@ -275,17 +273,22 @@ private Type mcTypeToDorisType(TypeInfo typeInfo) { @Override public TTableDescriptor toThrift() { + // ak sk endpoint project quota List schema = getFullSchema(); TMCTable tMcTable = new TMCTable(); MaxComputeExternalCatalog mcCatalog = ((MaxComputeExternalCatalog) catalog); - tMcTable.setRegion(mcCatalog.getRegion()); + tMcTable.setAccessKey(mcCatalog.getAccessKey()); tMcTable.setSecretKey(mcCatalog.getSecretKey()); - tMcTable.setOdpsUrl(mcCatalog.getOdpsUrl()); - tMcTable.setTunnelUrl(mcCatalog.getTunnelUrl()); - tMcTable.setPublicAccess(String.valueOf(mcCatalog.enablePublicAccess())); + tMcTable.setOdpsUrl("deprecated"); + tMcTable.setRegion("deprecated"); + tMcTable.setEndpoint(mcCatalog.getEndpoint()); // use mc project as dbName tMcTable.setProject(dbName); + tMcTable.setQuota(mcCatalog.getQuota()); + + tMcTable.setTunnelUrl("deprecated"); + tMcTable.setProject("deprecated"); tMcTable.setTable(name); TTableDescriptor tTableDescriptor = new TTableDescriptor(getId(), TTableType.MAX_COMPUTE_TABLE, schema.size(), 0, getName(), dbName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCache.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCache.java index cce570a11746aa..2213ded0bcbbed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCache.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/MaxComputeMetadataCache.java @@ -30,21 +30,11 @@ public class MaxComputeMetadataCache { private final Cache partitionValuesCache; - private final Cache tableRowCountCache; public MaxComputeMetadataCache() { partitionValuesCache = Caffeine.newBuilder().maximumSize(Config.max_hive_partition_cache_num) .expireAfterAccess(Config.external_cache_expire_time_minutes_after_access, TimeUnit.MINUTES) .build(); - tableRowCountCache = Caffeine.newBuilder().maximumSize(10000) - .expireAfterAccess(Config.external_cache_expire_time_minutes_after_access, TimeUnit.MINUTES) - .build(); - } - - public Long getCachedRowCount(String dbName, String tblName, String partitionSpec, - Function loader) { - MaxComputeCacheKey tablePartitionKey = new MaxComputeCacheKey(dbName, tblName, partitionSpec); - return tableRowCountCache.get(tablePartitionKey, loader); } public TablePartitionValues getCachedPartitionValues(MaxComputeCacheKey tablePartitionKey, @@ -54,7 +44,6 @@ public TablePartitionValues getCachedPartitionValues(MaxComputeCacheKey tablePar public void cleanUp() { partitionValuesCache.invalidateAll(); - tableRowCountCache.invalidateAll(); } public void cleanDatabaseCache(String dbName) { @@ -63,17 +52,10 @@ public void cleanDatabaseCache(String dbName) { .filter(k -> k.getDbName().equalsIgnoreCase(dbName)) .collect(Collectors.toList()); partitionValuesCache.invalidateAll(removeCacheList); - - List removeCacheRowCountList = tableRowCountCache.asMap().keySet() - .stream() - .filter(k -> k.getDbName().equalsIgnoreCase(dbName)) - .collect(Collectors.toList()); - tableRowCountCache.invalidateAll(removeCacheRowCountList); } public void cleanTableCache(String dbName, String tblName) { MaxComputeCacheKey cacheKey = new MaxComputeCacheKey(dbName, tblName); partitionValuesCache.invalidate(cacheKey); - tableRowCountCache.invalidate(cacheKey); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java index 87ab365553472f..521757da20e2ea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeScanNode.java @@ -17,18 +17,29 @@ package org.apache.doris.datasource.maxcompute.source; +import org.apache.doris.analysis.BinaryPredicate; +import org.apache.doris.analysis.CastExpr; +import org.apache.doris.analysis.CompoundPredicate; +import org.apache.doris.analysis.CompoundPredicate.Operator; +import org.apache.doris.analysis.DateLiteral; +import org.apache.doris.analysis.Expr; +import org.apache.doris.analysis.InPredicate; +import org.apache.doris.analysis.IsNullPredicate; +import org.apache.doris.analysis.LiteralExpr; +import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.TupleDescriptor; -import org.apache.doris.catalog.PartitionItem; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.ScalarType; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.AnalysisException; -import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.LocationPath; import org.apache.doris.datasource.FileQueryScanNode; import org.apache.doris.datasource.TableFormatType; -import org.apache.doris.datasource.TablePartitionValues; +import org.apache.doris.datasource.maxcompute.MaxComputeExternalCatalog; import org.apache.doris.datasource.maxcompute.MaxComputeExternalTable; -import org.apache.doris.planner.ListPartitionPrunerV2; +import org.apache.doris.datasource.maxcompute.source.MaxComputeSplit.SplitType; +import org.apache.doris.datasource.property.constants.MCProperties; import org.apache.doris.planner.PlanNodeId; import org.apache.doris.spi.Split; import org.apache.doris.statistics.StatisticalType; @@ -37,22 +48,35 @@ import org.apache.doris.thrift.TMaxComputeFileDesc; import org.apache.doris.thrift.TTableFormatFileDesc; -import com.aliyun.odps.Table; -import com.aliyun.odps.tunnel.TunnelException; +import com.aliyun.odps.OdpsType; +import com.aliyun.odps.table.TableIdentifier; +import com.aliyun.odps.table.configuration.ArrowOptions; +import com.aliyun.odps.table.configuration.ArrowOptions.TimestampUnit; +import com.aliyun.odps.table.optimizer.predicate.Predicate; +import com.aliyun.odps.table.read.TableBatchReadSession; +import com.aliyun.odps.table.read.TableReadSessionBuilder; +import com.aliyun.odps.table.read.split.InputSplitAssigner; +import com.aliyun.odps.table.read.split.impl.IndexedInputSplit; import com.google.common.collect.Maps; +import jline.internal.Log; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.io.Serializable; import java.util.ArrayList; -import java.util.Collection; +import java.util.Base64; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; public class MaxComputeScanNode extends FileQueryScanNode { private final MaxComputeExternalTable table; - private static final int MIN_SPLIT_SIZE = 4096; - private static final LocationPath VIRTUAL_SLICE_PART = new LocationPath("/virtual_slice_part", Maps.newHashMap()); + TableBatchReadSession tableBatchReadSession; public MaxComputeScanNode(PlanNodeId id, TupleDescriptor desc, boolean needCheckColumnPriv) { this(id, desc, "MCScanNode", StatisticalType.MAX_COMPUTE_SCAN_NODE, needCheckColumnPriv); @@ -75,11 +99,288 @@ private void setScanParams(TFileRangeDesc rangeDesc, MaxComputeSplit maxComputeS TTableFormatFileDesc tableFormatFileDesc = new TTableFormatFileDesc(); tableFormatFileDesc.setTableFormatType(TableFormatType.MAX_COMPUTE.value()); TMaxComputeFileDesc fileDesc = new TMaxComputeFileDesc(); - if (maxComputeSplit.getPartitionSpec().isPresent()) { - fileDesc.setPartitionSpec(maxComputeSplit.getPartitionSpec().get()); - } + fileDesc.setPartitionSpec("deprecated"); + fileDesc.setTableBatchReadSession(maxComputeSplit.scanSerialize); + fileDesc.setSessionId(maxComputeSplit.getSessionId()); tableFormatFileDesc.setMaxComputeParams(fileDesc); rangeDesc.setTableFormatParams(tableFormatFileDesc); + rangeDesc.setPath("[ " + maxComputeSplit.getStart() + " , " + maxComputeSplit.getLength() + " ]"); + rangeDesc.setStartOffset(maxComputeSplit.getStart()); + rangeDesc.setSize(maxComputeSplit.getLength()); + } + + void createTableBatchReadSession() throws UserException { + Predicate filterPredicate = convertPredicate(); + + + List requiredPartitionColumns = new ArrayList<>(); + List orderedRequiredDataColumns = new ArrayList<>(); + + Set requiredSlots = + desc.getSlots().stream().map(e -> e.getColumn().getName()).collect(Collectors.toSet()); + + Set partitionColumns = + table.getPartitionColumns().stream().map(Column::getName).collect(Collectors.toSet()); + + for (Column column : table.getColumns()) { + String columnName = column.getName(); + if (!requiredSlots.contains(columnName)) { + continue; + } + if (partitionColumns.contains(columnName)) { + requiredPartitionColumns.add(columnName); + } else { + orderedRequiredDataColumns.add(columnName); + } + } + + + + MaxComputeExternalCatalog mcCatalog = (MaxComputeExternalCatalog) table.getCatalog(); + + try { + TableReadSessionBuilder scanBuilder = new TableReadSessionBuilder(); + tableBatchReadSession = + scanBuilder.identifier(TableIdentifier.of(table.getDbName(), table.getName())) + .withSettings(mcCatalog.getSettings()) + .withSplitOptions(mcCatalog.getSplitOption()) + .requiredPartitionColumns(requiredPartitionColumns) + .requiredDataColumns(orderedRequiredDataColumns) + .withArrowOptions( + ArrowOptions.newBuilder() + .withDatetimeUnit(TimestampUnit.MILLI) + .withTimestampUnit(TimestampUnit.NANO) + .build() + ) + .withFilterPredicate(filterPredicate) + .buildBatchReadSession(); + } catch (java.io.IOException e) { + throw new RuntimeException(e); + } + + } + + protected Predicate convertPredicate() { + if (conjuncts.isEmpty()) { + return Predicate.NO_PREDICATE; + } + + if (conjuncts.size() == 1) { + try { + return convertExprToOdpsPredicate(conjuncts.get(0)); + } catch (AnalysisException e) { + Log.info("Failed to convert predicate " + conjuncts.get(0) + " to odps predicate"); + Log.info("Reason: " + e.getMessage()); + return Predicate.NO_PREDICATE; + } + } + + com.aliyun.odps.table.optimizer.predicate.CompoundPredicate + filterPredicate = new com.aliyun.odps.table.optimizer.predicate.CompoundPredicate( + com.aliyun.odps.table.optimizer.predicate.CompoundPredicate.Operator.AND + ); + + for (Expr predicate : conjuncts) { + try { + filterPredicate.addPredicate(convertExprToOdpsPredicate(predicate)); + } catch (AnalysisException e) { + Log.info("Failed to convert predicate " + predicate); + Log.info("Reason: " + e.getMessage()); + return Predicate.NO_PREDICATE; + } + } + return filterPredicate; + } + + private Predicate convertExprToOdpsPredicate(Expr expr) throws AnalysisException { + Predicate odpsPredicate = null; + if (expr instanceof CompoundPredicate) { + CompoundPredicate compoundPredicate = (CompoundPredicate) expr; + + com.aliyun.odps.table.optimizer.predicate.CompoundPredicate.Operator odpsOp; + switch (compoundPredicate.getOp()) { + case AND: + odpsOp = com.aliyun.odps.table.optimizer.predicate.CompoundPredicate.Operator.AND; + break; + case OR: + odpsOp = com.aliyun.odps.table.optimizer.predicate.CompoundPredicate.Operator.OR; + break; + case NOT: + odpsOp = com.aliyun.odps.table.optimizer.predicate.CompoundPredicate.Operator.NOT; + break; + default: + throw new AnalysisException("Unknown operator: " + compoundPredicate.getOp()); + } + + List odpsPredicates = new ArrayList<>(); + + odpsPredicates.add(convertExprToOdpsPredicate(expr.getChild(0))); + + if (compoundPredicate.getOp() != Operator.NOT) { + odpsPredicates.add(convertExprToOdpsPredicate(expr.getChild(1))); + } + odpsPredicate = new com.aliyun.odps.table.optimizer.predicate.CompoundPredicate(odpsOp, odpsPredicates); + + } else if (expr instanceof InPredicate) { + + InPredicate inPredicate = (InPredicate) expr; + if (inPredicate.getChildren().size() > 2) { + return Predicate.NO_PREDICATE; + } + com.aliyun.odps.table.optimizer.predicate.InPredicate.Operator odpsOp = + inPredicate.isNotIn() + ? com.aliyun.odps.table.optimizer.predicate.InPredicate.Operator.IN + : com.aliyun.odps.table.optimizer.predicate.InPredicate.Operator.NOT_IN; + + String columnName = convertSlotRefToColumnName(expr.getChild(0)); + com.aliyun.odps.OdpsType odpsType = table.getColumnNameToOdpsColumn().get(columnName).getType(); + + StringBuilder stringBuilder = new StringBuilder(); + + + stringBuilder.append(columnName); + stringBuilder.append(" "); + stringBuilder.append(odpsOp.getDescription()); + stringBuilder.append(" ("); + + for (int i = 1; i < inPredicate.getChildren().size(); i++) { + stringBuilder.append(convertLiteralToOdpsValues(odpsType, expr.getChild(i))); + if (i < inPredicate.getChildren().size() - 1) { + stringBuilder.append(", "); + } + } + stringBuilder.append(" )"); + + odpsPredicate = new com.aliyun.odps.table.optimizer.predicate.RawPredicate(stringBuilder.toString()); + + } else if (expr instanceof BinaryPredicate) { + BinaryPredicate binaryPredicate = (BinaryPredicate) expr; + + + com.aliyun.odps.table.optimizer.predicate.BinaryPredicate.Operator odpsOp; + switch (binaryPredicate.getOp()) { + case EQ: { + odpsOp = com.aliyun.odps.table.optimizer.predicate.BinaryPredicate.Operator.EQUALS; + break; + } + case NE: { + odpsOp = com.aliyun.odps.table.optimizer.predicate.BinaryPredicate.Operator.NOT_EQUALS; + break; + } + case GE: { + odpsOp = com.aliyun.odps.table.optimizer.predicate.BinaryPredicate.Operator.GREATER_THAN_OR_EQUAL; + break; + } + case LE: { + odpsOp = com.aliyun.odps.table.optimizer.predicate.BinaryPredicate.Operator.LESS_THAN_OR_EQUAL; + break; + } + case LT: { + odpsOp = com.aliyun.odps.table.optimizer.predicate.BinaryPredicate.Operator.LESS_THAN; + break; + } + case GT: { + odpsOp = com.aliyun.odps.table.optimizer.predicate.BinaryPredicate.Operator.GREATER_THAN; + break; + } + default: { + odpsOp = null; + break; + } + } + + if (odpsOp != null) { + String columnName = convertSlotRefToColumnName(expr.getChild(0)); + com.aliyun.odps.OdpsType odpsType = table.getColumnNameToOdpsColumn().get(columnName).getType(); + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append(columnName); + stringBuilder.append(" "); + stringBuilder.append(odpsOp.getDescription()); + stringBuilder.append(" "); + stringBuilder.append(convertLiteralToOdpsValues(odpsType, expr.getChild(1))); + + odpsPredicate = new com.aliyun.odps.table.optimizer.predicate.RawPredicate(stringBuilder.toString()); + } + } else if (expr instanceof IsNullPredicate) { + IsNullPredicate isNullPredicate = (IsNullPredicate) expr; + com.aliyun.odps.table.optimizer.predicate.UnaryPredicate.Operator odpsOp = + isNullPredicate.isNotNull() + ? com.aliyun.odps.table.optimizer.predicate.UnaryPredicate.Operator.NOT_NULL + : com.aliyun.odps.table.optimizer.predicate.UnaryPredicate.Operator.IS_NULL; + + odpsPredicate = new com.aliyun.odps.table.optimizer.predicate.UnaryPredicate(odpsOp, + new com.aliyun.odps.table.optimizer.predicate.Attribute( + convertSlotRefToColumnName(expr.getChild(0)) + ) + ); + } + + + if (odpsPredicate == null) { + throw new AnalysisException("Do not support convert [" + + expr.getExprName() + "] in convertExprToOdpsPredicate."); + } + return odpsPredicate; + } + + private String convertSlotRefToColumnName(Expr expr) throws AnalysisException { + if (expr instanceof SlotRef) { + return ((SlotRef) expr).getColumnName(); + } else if (expr instanceof CastExpr) { + if (expr.getChild(0) instanceof SlotRef) { + return ((SlotRef) expr.getChild(0)).getColumnName(); + } + } + + throw new AnalysisException("Do not support convert [" + + expr.getExprName() + "] in convertSlotRefToAttribute."); + + + } + + private String convertLiteralToOdpsValues(OdpsType odpsType, Expr expr) throws AnalysisException { + if (!(expr instanceof LiteralExpr)) { + throw new AnalysisException("Do not support convert [" + + expr.getExprName() + "] in convertSlotRefToAttribute."); + } + LiteralExpr literalExpr = (LiteralExpr) expr; + + switch (odpsType) { + case BOOLEAN: + case TINYINT: + case SMALLINT: + case INT: + case BIGINT: + case DECIMAL: + case FLOAT: + case DOUBLE: { + return " " + literalExpr.toString() + " "; + } + case STRING: + case CHAR: + case VARCHAR: { + return " \"" + literalExpr.toString() + "\" "; + } + case DATE: { + DateLiteral dateLiteral = (DateLiteral) literalExpr; + ScalarType dstType = ScalarType.createDateV2Type(); + return " \"" + dateLiteral.getStringValue(dstType) + "\" "; + } + case DATETIME: { + DateLiteral dateLiteral = (DateLiteral) literalExpr; + ScalarType dstType = ScalarType.createDatetimeV2Type(3); + return " \"" + dateLiteral.getStringValue(dstType) + "\" "; + } + case TIMESTAMP_NTZ: { + DateLiteral dateLiteral = (DateLiteral) literalExpr; + ScalarType dstType = ScalarType.createDatetimeV2Type(6); + return " \"" + dateLiteral.getStringValue(dstType) + "\" "; + } + default: { + break; + } + } + throw new AnalysisException("Do not support convert odps type [" + odpsType + "] to odps values."); } @Override @@ -109,85 +410,64 @@ public List getSplits() throws UserException { if (desc.getSlots().isEmpty() || odpsTable.getFileNum() <= 0) { return result; } + createTableBatchReadSession(); + try { - if (!table.getPartitionColumns().isEmpty()) { - if (conjuncts.isEmpty()) { - throw new IllegalArgumentException("Max Compute partition table need partition predicate."); - } - List partitionSpecs = getPartitionSpecs(); - for (String partitionSpec : partitionSpecs) { - addPartitionSplits(result, odpsTable, partitionSpec); + String scanSessionSerialize = serializeSession(tableBatchReadSession); + InputSplitAssigner assigner = tableBatchReadSession.getInputSplitAssigner(); + long modificationTime = table.getOdpsTable().getLastDataModifiedTime().getTime(); + + MaxComputeExternalCatalog mcCatalog = (MaxComputeExternalCatalog) table.getCatalog(); + + if (mcCatalog.getSplitStrategy().equals(MCProperties.SPLIT_BY_BYTE_SIZE_STRATEGY)) { + + for (com.aliyun.odps.table.read.split.InputSplit split : assigner.getAllSplits()) { + MaxComputeSplit maxComputeSplit = + new MaxComputeSplit(new LocationPath("/byte_size", Maps.newHashMap()), + ((IndexedInputSplit) split).getSplitIndex(), -1, + mcCatalog.getSplitByteSize(), + modificationTime, null, + Collections.emptyList()); + + + maxComputeSplit.scanSerialize = scanSessionSerialize; + maxComputeSplit.splitType = SplitType.BYTE_SIZE; + maxComputeSplit.sessionId = split.getSessionId(); + + result.add(maxComputeSplit); } } else { - addBatchSplits(result, odpsTable, table.getTotalRows()); - } - } catch (TunnelException e) { - throw new UserException("Max Compute tunnel SDK exception: " + e.getMessage(), e); + long totalRowCount = assigner.getTotalRowCount(); - } - return result; - } + long recordsPerSplit = mcCatalog.getSplitRowCount(); + for (long offset = 0; offset < totalRowCount; offset += recordsPerSplit) { + recordsPerSplit = Math.min(recordsPerSplit, totalRowCount - offset); + com.aliyun.odps.table.read.split.InputSplit split = + assigner.getSplitByRowOffset(offset, recordsPerSplit); - private static void addPartitionSplits(List result, Table odpsTable, String partitionSpec) { - long modificationTime = odpsTable.getLastDataModifiedTime().getTime(); - // use '-1' to read whole partition, avoid expending too much time on calling table.getTotalRows() - result.add(new MaxComputeSplit(VIRTUAL_SLICE_PART, - 0, -1L, -1, modificationTime, null, Collections.emptyList(), null)); - } + MaxComputeSplit maxComputeSplit = + new MaxComputeSplit(new LocationPath("/row_offset", Maps.newHashMap()), + offset, recordsPerSplit, totalRowCount, modificationTime, null, + Collections.emptyList()); - private static void addBatchSplits(List result, Table odpsTable, long totalRows) { - List> sliceRange = new ArrayList<>(); - long fileNum = odpsTable.getFileNum(); - long start = 0; - long splitSize = (long) Math.ceil((double) totalRows / fileNum); - if (splitSize <= 0 || totalRows < MIN_SPLIT_SIZE) { - // use whole split - sliceRange.add(Pair.of(start, totalRows)); - } else { - for (int i = 0; i < fileNum; i++) { - if (start > totalRows) { - break; + maxComputeSplit.scanSerialize = scanSessionSerialize; + maxComputeSplit.splitType = SplitType.ROW_OFFSET; + maxComputeSplit.sessionId = split.getSessionId(); + + result.add(maxComputeSplit); } - sliceRange.add(Pair.of(start, splitSize)); - start += splitSize; - } - } - long modificationTime = odpsTable.getLastDataModifiedTime().getTime(); - if (!sliceRange.isEmpty()) { - for (int i = 0; i < sliceRange.size(); i++) { - Pair range = sliceRange.get(i); - result.add(new MaxComputeSplit(new LocationPath("/virtual_slice_" + i, Maps.newHashMap()), - range.first, range.second, totalRows, modificationTime, null, Collections.emptyList(), null)); } + } catch (IOException e) { + throw new RuntimeException(e); } + return result; } - private List getPartitionSpecs() throws AnalysisException { - return getPrunedPartitionSpecs(); - } - - private List getPrunedPartitionSpecs() throws AnalysisException { - List result = new ArrayList<>(); - TablePartitionValues partitionValues = table.getPartitionValues(); - // prune partitions by expr - partitionValues.readLock().lock(); - try { - Map idToPartitionItem = partitionValues.getIdToPartitionItem(); - this.totalPartitionNum = idToPartitionItem.size(); - ListPartitionPrunerV2 pruner = new ListPartitionPrunerV2(idToPartitionItem, - table.getPartitionColumns(), columnNameToRange, - partitionValues.getUidToPartitionRange(), - partitionValues.getRangeToId(), - partitionValues.getSingleColumnRangeMap(), - false); - Collection filteredPartitionIds = pruner.prune(); - this.selectedPartitionNum = filteredPartitionIds.size(); - // get partitions from cache - Map partitionIdToNameMap = partitionValues.getPartitionIdToNameMap(); - filteredPartitionIds.forEach(id -> result.add(partitionIdToNameMap.get(id))); - return result; - } finally { - partitionValues.readLock().unlock(); - } + private static String serializeSession(Serializable object) throws IOException { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + ObjectOutputStream objectOutputStream = new ObjectOutputStream(byteArrayOutputStream); + objectOutputStream.writeObject(object); + byte[] serializedBytes = byteArrayOutputStream.toByteArray(); + return Base64.getEncoder().encodeToString(serializedBytes); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeSplit.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeSplit.java index 256ee1adefb1ea..0fc9fbcbfd5f63 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeSplit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/maxcompute/source/MaxComputeSplit.java @@ -21,21 +21,27 @@ import org.apache.doris.datasource.FileSplit; import org.apache.doris.thrift.TFileType; +import lombok.Getter; + import java.util.List; -import java.util.Optional; +@Getter public class MaxComputeSplit extends FileSplit { - private final Optional partitionSpec; + public String scanSerialize; + public String sessionId; + + public enum SplitType { + ROW_OFFSET, + BYTE_SIZE + } + + public SplitType splitType; public MaxComputeSplit(LocationPath path, long start, long length, long fileLength, - long modificationTime, String[] hosts, List partitionValues, String partitionSpec) { + long modificationTime, String[] hosts, List partitionValues) { super(path, start, length, fileLength, modificationTime, hosts, partitionValues); - this.partitionSpec = Optional.ofNullable(partitionSpec); // MC always use FILE_NET type this.locationType = TFileType.FILE_NET; } - public Optional getPartitionSpec() { - return partitionSpec; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java index de1983f5753aa5..df4fa068ca706d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/property/constants/MCProperties.java @@ -27,12 +27,28 @@ public class MCProperties extends BaseProperties { public static final String REGION = "mc.region"; public static final String PROJECT = "mc.default.project"; - public static final String ACCESS_KEY = "mc.access_key"; - public static final String SECRET_KEY = "mc.secret_key"; public static final String SESSION_TOKEN = "mc.session_token"; public static final String PUBLIC_ACCESS = "mc.public_access"; - public static final String ODPS_ENDPOINT = "mc.odps_endpoint"; - public static final String TUNNEL_SDK_ENDPOINT = "mc.tunnel_endpoint"; + public static final String DEFAULT_PUBLIC_ACCESS = "false"; + + public static final String ACCESS_KEY = "mc.access_key"; + public static final String SECRET_KEY = "mc.secret_key"; + public static final String ENDPOINT = "mc.endpoint"; + + public static final String QUOTA = "mc.quota"; + public static final String DEFAULT_QUOTA = "pay-as-you-go"; + + + public static final String SPLIT_STRATEGY = "mc.split_strategy"; + public static final String SPLIT_BY_BYTE_SIZE_STRATEGY = "byte_size"; + public static final String SPLIT_BY_ROW_COUNT_STRATEGY = "row_count"; + public static final String DEFAULT_SPLIT_STRATEGY = SPLIT_BY_BYTE_SIZE_STRATEGY; + + + public static final String SPLIT_BYTE_SIZE = "mc.split_byte_size"; + public static final String DEFAULT_SPLIT_BYTE_SIZE = "268435456"; //256 * 1024L * 1024L = 256MB + public static final String SPLIT_ROW_COUNT = "mc.split_row_count"; + public static final String DEFAULT_SPLIT_ROW_COUNT = "1048576"; // 256 * 4096 public static CloudCredential getCredential(Map props) { return getCloudCredential(props, ACCESS_KEY, SECRET_KEY, SESSION_TOKEN); diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java index c9f96a79161a20..a5f2453ea363af 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/property/PropertyConverterTest.java @@ -402,10 +402,9 @@ public void testMcPropertiesConverter() throws Exception { String queryDlf1 = "create catalog hms_mc properties (\n" + " 'type'='max_compute',\n" + " 'mc.default.project' = 'project0',\n" - + " 'mc.region' = 'cn-beijing',\n" + " 'mc.access_key' = 'ak',\n" + " 'mc.secret_key' = 'sk',\n" - + " 'mc.public_access' = 'true'\n" + + " 'mc.endpoint' = 'http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api' \n" + ");"; String catalogName = "hms_mc"; CreateCatalogStmt analyzedStmt = createStmt(queryDlf1); @@ -414,10 +413,10 @@ public void testMcPropertiesConverter() throws Exception { .getCatalogMgr().getCatalog(catalogName); Map properties = catalog.getCatalogProperty().getProperties(); Assertions.assertEquals(properties.get("type"), "max_compute"); - Assertions.assertEquals(properties.get("mc.region"), "cn-beijing"); Assertions.assertEquals(properties.get("mc.access_key"), "ak"); Assertions.assertEquals(properties.get("mc.secret_key"), "sk"); - Assertions.assertEquals(properties.get("mc.public_access"), "true"); + Assertions.assertEquals(properties.get("mc.endpoint"), + "http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api"); Assertions.assertEquals(properties.get("mc.default.project"), "project0"); } diff --git a/fe/pom.xml b/fe/pom.xml index 8bba5442dc8c6d..29b2b61530a935 100644 --- a/fe/pom.xml +++ b/fe/pom.xml @@ -312,7 +312,7 @@ under the License. 1.4.3 - 0.45.2-public + 0.48.8-public 1.11.3 17.0.0 diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index 56222c2338a268..10ad6de3f6bb22 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -353,14 +353,16 @@ struct TJdbcTable { } struct TMCTable { - 1: optional string region + 1: optional string region // deprecated 2: optional string project 3: optional string table 4: optional string access_key 5: optional string secret_key - 6: optional string public_access - 7: optional string odps_url - 8: optional string tunnel_url + 6: optional string public_access // deprecated + 7: optional string odps_url // deprecated + 8: optional string tunnel_url // deprecated + 9: optional string endpoint + 10: optional string quota } struct TTrinoConnectorTable { diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift index e53289c1b865fd..c77ab48b2d404c 100644 --- a/gensrc/thrift/PlanNodes.thrift +++ b/gensrc/thrift/PlanNodes.thrift @@ -347,7 +347,10 @@ struct TTrinoConnectorFileDesc { } struct TMaxComputeFileDesc { - 1: optional string partition_spec + 1: optional string partition_spec // deprecated + 2: optional string session_id + 3: optional string table_batch_read_session + } struct THudiFileDesc { diff --git a/regression-test/data/external_table_p2/maxcompute/test_external_catalog_maxcompute.out b/regression-test/data/external_table_p2/maxcompute/test_external_catalog_maxcompute.out index e75e12c137b7a2..e1479672f232e7 100644 --- a/regression-test/data/external_table_p2/maxcompute/test_external_catalog_maxcompute.out +++ b/regression-test/data/external_table_p2/maxcompute/test_external_catalog_maxcompute.out @@ -1,69 +1,141 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !q1 -- -8639377 +5 -- !q2 -- -1 2 2000-08-15 2000-08-16 t 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22.22 23.23 +3 WS0003 2023-03-01 2023-10-31 Example Web Site 3 20230301 20231031 Healthcare Robert White 103 Health Healthcare services and products Emily Green 203 Health Corp 9101 Health Drive Floor Floor 3 Star City County TX 23456 USA -6.0 6.75 +4 WS0004 2023-04-01 2023-09-30 Example Web Site 4 20230401 20230930 Education David Black 104 EdTech Educational technology platform Fiona Grey 204 Edu Tech 1122 Education Lane Building Building 1 Smallville County FL 34567 USA -4.0 5.00 +5 WS0005 2023-05-01 2023-08-31 Example Web Site 5 20230501 20230831 Travel Sophia Blue 105 Travel Travel and booking services Daniel Red 205 Travel Inc 3344 Tourist Street Unit Unit 5 Metropolis County WA 45678 USA -7.0 8.00 -- !q3 -- -false 2 44 423432 -true 77 8920 182239402452 +false -1 -1 -1 +false 0 0 0 +false 10 1000 10000 +false 127 32767 9223372036854775807 +false 2 200 2000 +false 4 400 4000 +false 6 600 6000 +false 8 800 8000 +true -128 -32768 -9223372036854775807 +true 0 0 0 +true 1 1 1 +true 1 100 1000 +true 3 300 3000 +true 5 500 5000 +true 7 700 7000 +true 9 900 9000 + +-- !q3_1 -- +false -1 -1 -1 +false 0 0 0 +false 10 1000 10000 +false 127 32767 9223372036854775807 +false 2 200 2000 +false 4 400 4000 +false 6 600 6000 +false 8 800 8000 +true -128 -32768 -9223372036854775807 +true 0 0 0 +true 1 1 1 +true 1 100 1000 +true 3 300 3000 +true 5 500 5000 +true 7 700 7000 +true 9 900 9000 + +-- !q3_2 -- +false -1 -1 -1 +false 0 0 0 +false 10 1000 10000 +false 127 32767 9223372036854775807 +false 2 200 2000 +false 4 400 4000 +false 6 600 6000 +false 8 800 8000 +true -128 -32768 -9223372036854775807 +true 0 0 0 +true 1 1 1 +true 1 100 1000 +true 3 300 3000 +true 5 500 5000 +true 7 700 7000 +true 9 900 9000 -- !q4 -- -6223 maxam 2020-09-21 -9601 qewtoll 2020-09-21 +1003 Sample data 3 2023-08-03 -- !q5 -- -1633 siwtow 2021-08-21 +1004 Sample data 4 2023-08-04 +1005 Sample data 5 2023-08-05 -- !q6 -- -9601 qewtoll 2020-09-21 +1004 Sample data 4 2023-08-04 +1005 Sample data 5 2023-08-05 -- !q7 -- -1633 siwtow 2021-08-21 -1633 siwtow 20210821 -1633 siwtow 20210921 +1001 Sample data 1 2023-08-01 +1002 Sample data 2 2023-08-02 +1005 Sample data 5 2023-08-05 -- !replay_q6 -- -9601 qewtoll 2020-09-21 +1003 Sample data 3 2023-08-03 +1004 Sample data 4 2023-08-04 +1005 Sample data 5 2023-08-05 -- !multi_partition_q1 -- -pt=13/yy=2021/mm=12/dd=22 -pt=14/yy=2021/mm=12/dd=21 -pt=14/yy=2021/mm=12/dd=22 +yy=2023/mm=08/dd=01/pt=1 +yy=2023/mm=08/dd=02/pt=2 +yy=2023/mm=08/dd=03/pt=3 +yy=2023/mm=08/dd=04/pt=4 +yy=2023/mm=08/dd=05/pt=5 -- !multi_partition_q2 -- -17 2022-04-23T11:12:30 2021 12 22 -17 2022-04-23T11:12:30 2021 12 21 -16 2022-04-23T11:12:30 2021 12 22 +1 2023-08-01T10:30 2023 08 01 +1 2023-08-01T12:00 2023 08 01 +2 2023-08-02T08:15 2023 08 02 +2 2023-08-02T14:45 2023 08 02 +3 2023-08-03T09:00 2023 08 03 +3 2023-08-03T11:30 2023 08 03 +4 2023-08-04T07:00 2023 08 04 +4 2023-08-04T10:00 2023 08 04 +5 2023-08-05T13:00 2023 08 05 +5 2023-08-05T15:30 2023 08 05 -- !multi_partition_q3 -- -14 2022-04-23T11:12:30 2022 01 01 -14 2022-04-23T11:12:30 2022 01 02 -98 2022-04-23T11:12:30 2021 12 21 +1 2023-08-01T10:30 2023 08 01 +1 2023-08-01T12:00 2023 08 01 +2 2023-08-02T08:15 2023 08 02 +2 2023-08-02T14:45 2023 08 02 +3 2023-08-03T09:00 2023 08 03 +3 2023-08-03T11:30 2023 08 03 +4 2023-08-04T07:00 2023 08 04 +4 2023-08-04T10:00 2023 08 04 +5 2023-08-05T13:00 2023 08 05 +5 2023-08-05T15:30 2023 08 05 -- !multi_partition_q4 -- -22 +10 -- !multi_partition_q5 -- -2022-04-23T11:12:30 2021 12 21 -2022-04-23T11:12:30 2021 12 21 -2022-04-23T11:12:30 2021 12 21 +2023-08-04T07:00 2023 08 04 +2023-08-04T10:00 2023 08 04 -- !multi_partition_q6 -- -17 2021 12 +5 2023 08 -- !multi_partition_q7 -- -20 +4 -- !multi_partition_q8 -- -11 +6 -- !multi_partition_q9 -- -lweu 8920 true 2023-11-23T12:03:54.952 0.123 2022-04-23 2022-04-23T11:12:30 12 2021 12 22 -wert 8920 true 2023-11-23T12:05:01.693 0.123 2022-04-23 2022-04-23T11:12:30 12 2021 12 22 -- !multi_partition_q10 -- -12 2021 12 21 -12 2021 12 22 -12 2021 12 22 +3 2023 08 03 +3 2023 08 03 +4 2023 08 04 +4 2023 08 04 +5 2023 08 05 +5 2023 08 05 + diff --git a/regression-test/data/external_table_p2/maxcompute/test_max_compute_all_type.out b/regression-test/data/external_table_p2/maxcompute/test_max_compute_all_type.out new file mode 100644 index 00000000000000..bff928fdedb437 --- /dev/null +++ b/regression-test/data/external_table_p2/maxcompute/test_max_compute_all_type.out @@ -0,0 +1,788 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !desc -- +id int Yes true \N +boolean_col boolean Yes true \N +tinyint_col tinyint Yes true \N +smallint_col smallint Yes true \N +int_col int Yes true \N +bigint_col bigint Yes true \N +float_col float Yes true \N +double_col double Yes true \N +decimal_col1 decimal(9,0) Yes true \N +decimal_col2 decimal(8,4) Yes true \N +decimal_col3 decimal(18,6) Yes true \N +decimal_col4 decimal(38,12) Yes true \N +string_col text Yes true \N +date_col date Yes true \N +datetime_col datetime(3) Yes true \N +timestamp_ntz_col2 datetime(6) Yes true \N +timestamp_ntz_col3 datetime(6) Yes true \N +char_col1 char(50) Yes true \N +char_col2 char(100) Yes true \N +char_col3 character(255) Yes true \N +varchar_col1 varchar(50) Yes true \N +varchar_col2 varchar(100) Yes true \N +varchar_col3 varchar(255) Yes true \N +t_map_string map Yes true \N +t_map_varchar map Yes true \N +t_map_char map Yes true \N +t_map_int map Yes true \N +t_map_bigint map Yes true \N +t_map_float map Yes true \N +t_map_double map Yes true \N +t_map_boolean map Yes true \N +t_array_string array Yes true \N +t_array_int array Yes true \N +t_array_bigint array Yes true \N +t_array_float array Yes true \N +t_array_double array Yes true \N +t_array_boolean array Yes true \N +t_array_varchar array Yes true \N +t_array_char array Yes true \N +t_array_decimal_precision_2 array Yes true \N +t_array_decimal_precision_4 array Yes true \N +t_array_decimal_precision_8 array Yes true \N +t_array_decimal_precision_17 array Yes true \N +t_array_decimal_precision_18 array Yes true \N +t_array_decimal_precision_38 array Yes true \N +t_struct_bigint struct Yes true \N +t_complex map>> Yes true \N +t_struct_nested struct> Yes true \N +t_struct_non_nulls_after_nulls struct Yes true \N +t_nested_struct_non_nulls_after_nulls struct> Yes true \N +t_array_string_starting_with_nulls array Yes true \N +t_array_string_with_nulls_in_between array Yes true \N +t_array_string_ending_with_nulls array Yes true \N + +-- !test_1 -- +1 false -7 -15 16 -9223372036854775807 -123.45 -123456.789 123456789 -1234.5678 -123456.789012 -123456789.012345700000 str 2024-03-25 2024-03-25T12:00 2024-03-25T12:00:00.123456 2024-03-25T12:00:00.123456 char_value11111 char_value22222 char_value33333 varchar_value11111 varchar_value22222 varchar_value33333 {"key7":"value1"} {"key7":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [9.4567, 4.5678] [6.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456790, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":-123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] +2 true 127 32767 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345700000 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"a":"b"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] ["varchar1", "varchar2"] ["char1", "char2"] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234570, 2.3456789012345680] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_non_nulls_after_nulls1":123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":123, "nested_struct_field2":"nested_value"}} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] +3 false -7 -15 16 -9223372036854775807 -123.45 -123456.789 123456789 -1234.5678 -123456.789012 -123456789.012345700000 str 2024-03-25 2024-03-25T12:00 2024-03-25T12:00:00.123456 2024-03-25T12:00:00.123456 char_value11111 char_value22222 char_value33333 varchar_value11111 varchar_value22222 varchar_value33333 {"key7":"value1"} {"key7":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [9.4567, 4.5678] [6.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456790, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":-123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] +4 false -128 -32768 -2147483647 -9223372036854775807 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345700000 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 char_value1 char_value2 char_value3 varchar_value1 varchar_value2 varchar_value3 {"key1":"value1"} {"key1":"value1"} {"x":"y"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456790, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":-123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] + +-- !test_2 -- +1 false -7 -15 16 -9223372036854775807 -123.45 -123456.789 123456789 -1234.5678 -123456.789012 -123456789.012345700000 str 2024-03-25 2024-03-25T12:00 2024-03-25T12:00:00.123456 2024-03-25T12:00:00.123456 char_value11111 char_value22222 char_value33333 varchar_value11111 varchar_value22222 varchar_value33333 {"key7":"value1"} {"key7":"value1"} {"x":"y"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] ["varchar1", "varchar2"] ["char1", "char2"] [3.3, 4.4] [3.45, 4.56] [9.4567, 4.5678] [6.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456790, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"value"} {"struct_field1":-123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] + +-- !test_3 -- +1 false +2 true +3 false +4 false + +-- !test_4 -- +1 false +3 false +4 false + +-- !test_5 -- +1 -7 +2 127 +3 -7 +4 -128 + +-- !test_5 -- +4 -128 + +-- !test_7 -- +1 -15 +2 32767 +3 -15 +4 -32768 + +-- !test_8 -- +2 32767 + +-- !test_9 -- +1 16 +2 2147483647 +3 16 +4 -2147483647 + +-- !test_10 -- +1 16 +3 16 + +-- !test_11 -- +2 2147483647 +4 -2147483647 + +-- !test_12 -- +1 -9223372036854775807 +2 9223372036854775807 +3 -9223372036854775807 +4 -9223372036854775807 + +-- !test_13 -- +2 9223372036854775807 + +-- !test_14 -- +1 -9223372036854775807 +2 9223372036854775807 +3 -9223372036854775807 +4 -9223372036854775807 + +-- !test_15 -- +1 -123.45 +2 123.45 +3 -123.45 +4 -123.45 + +-- !test_16 -- +2 123.45 + +-- !test_17 -- + +-- !test_18 -- +1 -123.45 +3 -123.45 +4 -123.45 + +-- !test_19 -- +1 -123.45 +2 123.45 +3 -123.45 +4 -123.45 + +-- !test_20 -- +1 -123.45 +3 -123.45 +4 -123.45 + +-- !test_21 -- +1 -123.45 +3 -123.45 +4 -123.45 + +-- !test_22 -- +2 123.45 + +-- !test_23 -- +1 -123456.789 +2 123456.789 +3 -123456.789 +4 -123456.789 + +-- !test_24 -- +1 -123456.789 +3 -123456.789 +4 -123456.789 + +-- !test_25 -- +2 123456.789 + +-- !test_26 -- +1 -123456.789 +3 -123456.789 +4 -123456.789 + +-- !test_27 -- +1 -123456.789 +3 -123456.789 +4 -123456.789 + +-- !test_28 -- +1 123456789 +2 123456789 +3 123456789 +4 -123456789 + +-- !test_29 -- + +-- !test_30 -- +1 123456789 +2 123456789 +3 123456789 +4 -123456789 + +-- !test_31 -- +1 -1234.5678 +2 1234.5678 +3 -1234.5678 +4 -1234.5678 + +-- !test_32 -- +1 -1234.5678 +3 -1234.5678 +4 -1234.5678 + +-- !test_33 -- +2 1234.5678 + +-- !test_34 -- +1 -123456.789012 +2 123456.789012 +3 -123456.789012 +4 -123456.789012 + +-- !test_35 -- +1 -123456.789012 +3 -123456.789012 +4 -123456.789012 + +-- !test_36 -- +2 123456.789012 + +-- !test_37 -- +1 -123456789.012345700000 +2 123456789.012345700000 +3 -123456789.012345700000 +4 -123456789.012345700000 + +-- !test_38 -- +1 -123456789.012345700000 +3 -123456789.012345700000 +4 -123456789.012345700000 + +-- !test_39 -- +2 123456789.012345700000 + +-- !test_40 -- +1 -123456789.012345700000 +2 123456789.012345700000 +3 -123456789.012345700000 +4 -123456789.012345700000 + +-- !test_41 -- +2 123456789.012345700000 + +-- !test_42 -- +1 str +2 string_value +3 str +4 string_value + +-- !test_43 -- +1 str +3 str + +-- !test_44 -- +1 str +3 str + +-- !test_45 -- +2 string_value +4 string_value + +-- !test_46 -- +1 str +2 string_value +3 str +4 string_value + +-- !test_47 -- + +-- !test_48 -- +1 2024-03-25 +2 2024-03-20 +3 2024-03-25 +4 2024-03-21 + +-- !test_49 -- +1 2024-03-25 +3 2024-03-25 + +-- !test_50 -- +1 2024-03-25 +3 2024-03-25 + +-- !test_51 -- +1 2024-03-25 +3 2024-03-25 +4 2024-03-21 + +-- !test_52 -- +1 2024-03-25T12:00 +2 2024-03-20T12:00 +3 2024-03-25T12:00 +4 2024-03-21T12:00 + +-- !test_53 -- +2 2024-03-20T12:00 +4 2024-03-21T12:00 + +-- !test_54 -- +1 2024-03-25T12:00 +3 2024-03-25T12:00 + +-- !test_55 -- + +-- !test_56 -- +2 2024-03-20T12:00 +4 2024-03-21T12:00 + +-- !test_57 -- +1 2024-03-25T12:00 +2 2024-03-20T12:00 +3 2024-03-25T12:00 +4 2024-03-21T12:00 + +-- !test_58 -- +1 2024-03-25T12:00:00.123456 +2 2024-03-20T12:00:00.123456 +3 2024-03-25T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_59 -- +2 2024-03-20T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_60 -- +1 2024-03-25T12:00:00.123456 +3 2024-03-25T12:00:00.123456 + +-- !test_61 -- + +-- !test_62 -- +2 2024-03-20T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_63 -- +1 2024-03-25T12:00:00.123456 +2 2024-03-20T12:00:00.123456 +3 2024-03-25T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_64 -- +1 2024-03-25T12:00:00.123456 +2 2024-03-20T12:00:00.123456 +3 2024-03-25T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_65 -- +2 2024-03-20T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_66 -- +2 2024-03-20T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_67 -- +1 2024-03-25T12:00:00.123456 +3 2024-03-25T12:00:00.123456 + +-- !test_68 -- + +-- !test_69 -- +1 2024-03-25T12:00:00.123456 +2 2024-03-20T12:00:00.123456 +3 2024-03-25T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_70 -- +1 2024-03-25T12:00:00.123456 +2 2024-03-20T12:00:00.123456 +3 2024-03-25T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_71 -- +1 2024-03-25T12:00:00.123456 +2 2024-03-20T12:00:00.123456 +3 2024-03-25T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_72 -- + +-- !test_73 -- + +-- !test_74 -- +1 2024-03-25T12:00:00.123456 +2 2024-03-20T12:00:00.123456 +3 2024-03-25T12:00:00.123456 +4 2024-03-21T12:00:00.123456 + +-- !test_75 -- +1 char_value11111 +2 char_value1 +3 char_value11111 +4 char_value1 + +-- !test_76 -- +1 char_value11111 +3 char_value11111 + +-- !test_77 -- +2 char_value1 +4 char_value1 + +-- !test_78 -- +1 char_value11111 +3 char_value11111 + +-- !test_79 -- +2 char_value1 +4 char_value1 + +-- !test_80 -- +1 char_value11111 +3 char_value11111 + +-- !test_81 -- +1 char_value11111 +2 char_value1 +3 char_value11111 +4 char_value1 + +-- !test_82 -- + +-- !test_83 -- +1 char_value22222 +2 char_value2 +3 char_value22222 +4 char_value2 + +-- !test_84 -- +1 char_value22222 +3 char_value22222 + +-- !test_85 -- +2 char_value2 +4 char_value2 + +-- !test_86 -- +1 char_value22222 +3 char_value22222 + +-- !test_87 -- +2 char_value2 +4 char_value2 + +-- !test_88 -- +1 char_value22222 +3 char_value22222 + +-- !test_89 -- +1 char_value22222 +2 char_value2 +3 char_value22222 +4 char_value2 + +-- !test_90 -- + +-- !test_91 -- +1 char_value33333 +2 char_value3 +3 char_value33333 +4 char_value3 + +-- !test_92 -- +1 char_value33333 +3 char_value33333 + +-- !test_93 -- +2 char_value3 +4 char_value3 + +-- !test_94 -- +1 char_value33333 +3 char_value33333 + +-- !test_95 -- +2 char_value3 +4 char_value3 + +-- !test_96 -- +1 char_value33333 +3 char_value33333 + +-- !test_97 -- +1 char_value33333 +2 char_value3 +3 char_value33333 +4 char_value3 + +-- !test_98 -- + +-- !test_99 -- +1 varchar_value11111 +2 varchar_value1 +3 varchar_value11111 +4 varchar_value1 + +-- !test_100 -- +1 varchar_value11111 +3 varchar_value11111 + +-- !test_101 -- +2 varchar_value1 +4 varchar_value1 + +-- !test_102 -- +1 varchar_value11111 +3 varchar_value11111 + +-- !test_103 -- +2 varchar_value1 +4 varchar_value1 + +-- !test_104 -- +1 varchar_value11111 +3 varchar_value11111 + +-- !test_105 -- +1 varchar_value11111 +2 varchar_value1 +3 varchar_value11111 +4 varchar_value1 + +-- !test_106 -- + +-- !test_107 -- +1 varchar_value22222 +2 varchar_value2 +3 varchar_value22222 +4 varchar_value2 + +-- !test_108 -- +1 varchar_value22222 +3 varchar_value22222 + +-- !test_109 -- +2 varchar_value2 +4 varchar_value2 + +-- !test_110 -- +1 varchar_value22222 +3 varchar_value22222 + +-- !test_111 -- +2 varchar_value2 +4 varchar_value2 + +-- !test_112 -- +1 varchar_value22222 +3 varchar_value22222 + +-- !test_113 -- +1 varchar_value22222 +2 varchar_value2 +3 varchar_value22222 +4 varchar_value2 + +-- !test_114 -- + +-- !test_115 -- +1 varchar_value33333 +2 varchar_value3 +3 varchar_value33333 +4 varchar_value3 + +-- !test_116 -- +1 varchar_value33333 +3 varchar_value33333 + +-- !test_117 -- +2 varchar_value3 +4 varchar_value3 + +-- !test_118 -- +1 varchar_value33333 +3 varchar_value33333 + +-- !test_119 -- +2 varchar_value3 +4 varchar_value3 + +-- !test_120 -- +1 varchar_value33333 +3 varchar_value33333 + +-- !test_121 -- +1 varchar_value33333 +2 varchar_value3 +3 varchar_value33333 +4 varchar_value3 + +-- !test_122 -- + +-- !test_123 -- +1 {"key7":"value1"} +2 {"key1":"value1"} +3 {"key7":"value1"} +4 {"key1":"value1"} + +-- !test_124 -- +1 {"key7":"value1"} +2 {"key1":"value1"} +3 {"key7":"value1"} +4 {"key1":"value1"} + +-- !test_125 -- +1 {"x":"y"} +2 {"a":"b"} +3 {"x":"y"} +4 {"x":"y"} + +-- !test_126 -- +1 {3:20} +2 {1:10} +3 {3:20} +4 {2:20} + +-- !test_127 -- +1 {3:200000000000} +2 {1:100000000000} +3 {3:200000000000} +4 {2:200000000000} + +-- !test_128 -- +1 {3.2:20.2} +2 {1.1:10.1} +3 {3.2:20.2} +4 {2.2:20.2} + +-- !test_129 -- +1 {3.2:20.2} +2 {1.1:10.1} +3 {3.2:20.2} +4 {2.2:20.2} + +-- !test_130 -- +1 {0:1} +2 {1:0} +3 {0:1} +4 {0:1} + +-- !test_131 -- +1 ["string1", "string2"] +2 ["string1", "string2"] +3 ["string1", "string2"] +4 ["string1", "string2"] + +-- !test_132 -- +1 [4, 5, 6] +2 [1, 2, 3] +3 [4, 5, 6] +4 [4, 5, 6] + +-- !test_133 -- +1 [300000000000, 400000000000] +2 [100000000000, 200000000000] +3 [300000000000, 400000000000] +4 [300000000000, 400000000000] + +-- !test_134 -- +1 [3.3, 4.4] +2 [1.1, 2.2] +3 [3.3, 4.4] +4 [3.3, 4.4] + +-- !test_135 -- +1 [3.123456789, 4.123456789] +2 [1.123456789, 2.123456789] +3 [3.123456789, 4.123456789] +4 [3.123456789, 4.123456789] + +-- !test_136 -- +1 [0, 1] +2 [1, 0] +3 [0, 1] +4 [0, 1] + +-- !test_137 -- +1 ["varchar1", "varchar2"] +2 ["varchar1", "varchar2"] +3 ["varchar1", "varchar2"] +4 ["varchar1", "varchar2"] + +-- !test_138 -- +1 ["char1", "char2"] +2 ["char1", "char2"] +3 ["char1", "char2"] +4 ["char1", "char2"] + +-- !test_139 -- +1 [3.3, 4.4] +2 [1.1, 2.2] +3 [3.3, 4.4] +4 [3.3, 4.4] + +-- !test_140 -- +1 [3.45, 4.56] +2 [1.23, 2.34] +3 [3.45, 4.56] +4 [3.45, 4.56] + +-- !test_141 -- +1 [9.4567, 4.5678] +2 [1.2345, 2.3456] +3 [9.4567, 4.5678] +4 [3.4567, 4.5678] + +-- !test_142 -- +1 [6.45678901, 4.56789012] +2 [1.23456789, 2.34567891] +3 [6.45678901, 4.56789012] +4 [3.45678901, 4.56789012] + +-- !test_143 -- +1 [3.45678901, 4.56789012] +2 [1.23456789, 2.34567891] +3 [3.45678901, 4.56789012] +4 [3.45678901, 4.56789012] + +-- !test_144 -- +1 [3.45, 4.56] +2 [1.23, 2.34] +3 [3.45, 4.56] +4 [3.45, 4.56] + +-- !test_145 -- +1 [3.4567890123456790, 4.5678901234567890] +2 [1.2345678901234570, 2.3456789012345680] +3 [3.4567890123456790, 4.5678901234567890] +4 [3.4567890123456790, 4.5678901234567890] + +-- !test_146 -- +1 {"s_bigint":-1234567890} +2 {"s_bigint":1234567890} +3 {"s_bigint":-1234567890} +4 {"s_bigint":-1234567890} + +-- !test_147 -- +1 {"key":[{"s_int":-123}]} +2 {"key":[{"s_int":123}]} +3 {"key":[{"s_int":-123}]} +4 {"key":[{"s_int":-123}]} + +-- !test_148 -- +1 {"struct_field":["value1", "value2"]} +2 {"struct_field":["value1", "value2"]} +3 {"struct_field":["value1", "value2"]} +4 {"struct_field":["value1", "value2"]} + +-- !test_149 -- +1 {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"value"} +2 {"struct_non_nulls_after_nulls1":123, "struct_non_nulls_after_nulls2":"value"} +3 {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"value"} +4 {"struct_non_nulls_after_nulls1":-123, "struct_non_nulls_after_nulls2":"value"} + +-- !test_150 -- +1 {"struct_field1":-123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} +2 {"struct_field1":123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":123, "nested_struct_field2":"nested_value"}} +3 {"struct_field1":-123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} +4 {"struct_field1":-123, "struct_field2":"value", "strict_field3":{"nested_struct_field1":-123, "nested_struct_field2":"nested_value"}} + +-- !test_151 -- +1 [null, "value1", "value2"] +2 [null, "value1", "value2"] +3 [null, "value1", "value2"] +4 [null, "value1", "value2"] + +-- !test_152 -- +1 ["value1", null, "value2"] +2 ["value1", null, "value2"] +3 ["value1", null, "value2"] +4 ["value1", null, "value2"] + +-- !test_153 -- +1 ["value11", "value2", null] +2 ["value1", "value2", null] +3 ["value11", "value2", null] +4 ["value1", "value2", null] + +-- !test_154 -- +1 {"x":"y"} +2 {"a":"b"} +3 {"x":"y"} +4 {"x":"y"} + diff --git a/regression-test/data/external_table_p0/mc/test_max_compute_complex_type.out b/regression-test/data/external_table_p2/maxcompute/test_max_compute_complex_type.out similarity index 79% rename from regression-test/data/external_table_p0/mc/test_max_compute_complex_type.out rename to regression-test/data/external_table_p2/maxcompute/test_max_compute_complex_type.out index 86df55f16c56e0..226c4d8f3043f4 100644 --- a/regression-test/data/external_table_p0/mc/test_max_compute_complex_type.out +++ b/regression-test/data/external_table_p2/maxcompute/test_max_compute_complex_type.out @@ -1,8 +1,9 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !mc_q1 -- -3 [1.3] [1, 2, 3] ["2023-05-23 05:55:12.000"] ["a", "b", "c"] -2 [1.2, 1.3] [1, 2, 3] ["2023-05-23 05:55:12.000"] ["a", "b", "c"] -1 [1.2, 1.3] [1, 2, 3] ["2023-05-23 05:55:12.000"] ["a", "b", "c"] +3 [1.3] [1, 2, 3] ["2023-05-23 13:55:12.000"] ["a", "b", "c"] +2 [1.2, 1.3] [1, 2, 3] ["2023-05-23 13:55:12.000"] ["a", "b", "c"] +1 [1.2, 1.3] [1, 2, 3] ["2023-05-23 13:55:12.000"] ["a", "b", "c"] +1 [1.2, 1.3] [1, 2, 3] ["2023-05-23 13:55:12.000"] ["a", "b", "c"] -- !mc_q2 -- {1:"example1", 2:"example2"} {1:2.5, 2:3.75} @@ -14,4 +15,5 @@ -- !mc_q4 -- user1 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"time_spent":1.5, "calories":500}}}}, {"activity_date":"2024-08-02", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"time_spent":1.5, "calories":500}}}}] -user2 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"time_spent":1.5, "calories":500}}}}, {"activity_date":"2024-08-02", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"time_spent":1.5, "calories":500}}}}] +user1 [{"activity_date":"2024-08-01", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"time_spent":1.5, "calories":500}}}}, {"activity_date":"2024-08-02", "activities":{"cooking":{"details":"Made vegan meal", "metrics":{"time_spent":1.5, "calories":500}}, "movie":{"details":"Watched action movie", "metrics":{"time_spent":1.5, "calories":500}}}}] + diff --git a/regression-test/suites/external_table_p2/maxcompute/test_external_catalog_maxcompute.groovy b/regression-test/suites/external_table_p2/maxcompute/test_external_catalog_maxcompute.groovy index c016f8b91f2e97..7bcd4b5ad4198a 100644 --- a/regression-test/suites/external_table_p2/maxcompute/test_external_catalog_maxcompute.groovy +++ b/regression-test/suites/external_table_p2/maxcompute/test_external_catalog_maxcompute.groovy @@ -14,7 +14,272 @@ // KIND, either express or implied. See the License for the // specific language governing permissions and limitations // under the License. +/* + + // Test DDL and Data: + + drop table `multi_partitions`; + CREATE TABLE `multi_partitions` ( + `city` string, + `gender` boolean, + `mnt` smallint, + `order_rate` float, + `amount` decimal(24,9), + `cut_date` date, + `create_time` datetime, + `finished_time` timestamp_ntz + ) PARTITIONED BY ( + `yy` string, + `mm` string, + `dd` string, + `pt` bigint + ); + INSERT INTO `multi_partitions` PARTITION (yy='2023', mm='08', dd='01', pt=1) VALUES + ('New York', TRUE, CAST(12 AS SMALLINT), CAST(0.75 AS FLOAT), CAST(1234.567890123 AS DECIMAL(24,9)), + CAST('2023-08-01' AS DATE), CAST('2023-08-01 10:30:00' AS DATETIME), CAST('2023-08-01 11:00:00' AS timestamp_ntz)); + INSERT INTO `multi_partitions` PARTITION (yy='2023', mm='08', dd='01', pt=1) VALUES + ('Los Angeles', FALSE, CAST(10 AS SMALLINT), CAST(1.15 AS FLOAT), CAST(9876.543210987 AS DECIMAL(24,9)), + CAST('2023-08-01' AS DATE), CAST('2023-08-01 12:00:00' AS DATETIME), CAST('2023-08-01 12:30:00' AS timestamp_ntz)); + INSERT INTO `multi_partitions` PARTITION (yy='2023', mm='08', dd='02', pt=2) VALUES + ('Chicago', TRUE, CAST(8 AS SMALLINT), CAST(0.90 AS FLOAT), CAST(5555.123456789 AS DECIMAL(24,9)), + CAST('2023-08-02' AS DATE), CAST('2023-08-02 08:15:00' AS DATETIME), CAST('2023-08-02 08:45:00' AS timestamp_ntz)); + INSERT INTO `multi_partitions` PARTITION (yy='2023', mm='08', dd='02', pt=2) VALUES + ('Houston', FALSE, CAST(15 AS SMALLINT), CAST(1.25 AS FLOAT), CAST(2222.987654321 AS DECIMAL(24,9)), + CAST('2023-08-02' AS DATE), CAST('2023-08-02 14:45:00' AS DATETIME), CAST('2023-08-02 15:15:00' AS timestamp_ntz)); + INSERT INTO `multi_partitions` PARTITION (yy='2023', mm='08', dd='03', pt=3) VALUES + ('Phoenix', TRUE, CAST(5 AS SMALLINT), CAST(0.50 AS FLOAT), CAST(7777.333333333 AS DECIMAL(24,9)), + CAST('2023-08-03' AS DATE), CAST('2023-08-03 09:00:00' AS DATETIME), CAST('2023-08-03 09:30:00' AS timestamp_ntz)); + INSERT INTO `multi_partitions` PARTITION (yy='2023', mm='08', dd='03', pt=3) VALUES + ('Philadelphia', FALSE, CAST(7 AS SMALLINT), CAST(0.85 AS FLOAT), CAST(8888.222222222 AS DECIMAL(24,9)), + CAST('2023-08-03' AS DATE), CAST('2023-08-03 11:30:00' AS DATETIME), CAST('2023-08-03 12:00:00' AS timestamp_ntz)); + INSERT INTO `multi_partitions` PARTITION (yy='2023', mm='08', dd='04', pt=4) VALUES + ('San Antonio', TRUE, CAST(9 AS SMALLINT), CAST(1.05 AS FLOAT), CAST(3333.666666666 AS DECIMAL(24,9)), + CAST('2023-08-04' AS DATE), CAST('2023-08-04 07:00:00' AS DATETIME), CAST('2023-08-04 07:30:00' AS timestamp_ntz)); + INSERT INTO `multi_partitions` PARTITION (yy='2023', mm='08', dd='04', pt=4) VALUES + ('San Diego', FALSE, CAST(11 AS SMALLINT), CAST(1.10 AS FLOAT), CAST(4444.555555555 AS DECIMAL(24,9)), + CAST('2023-08-04' AS DATE), CAST('2023-08-04 10:00:00' AS DATETIME), CAST('2023-08-04 10:30:00' AS timestamp_ntz)); + INSERT INTO `multi_partitions` PARTITION (yy='2023', mm='08', dd='05', pt=5) VALUES + ('Dallas', TRUE, CAST(6 AS SMALLINT), CAST(0.65 AS FLOAT), CAST(6666.444444444 AS DECIMAL(24,9)), + CAST('2023-08-05' AS DATE), CAST('2023-08-05 13:00:00' AS DATETIME), CAST('2023-08-05 13:30:00' AS timestamp_ntz)); + INSERT INTO `multi_partitions` PARTITION (yy='2023', mm='08', dd='05', pt=5) VALUES + ('San Jose', FALSE, CAST(14 AS SMALLINT), CAST(1.20 AS FLOAT), CAST(9999.111111111 AS DECIMAL(24,9)), + CAST('2023-08-05' AS DATE), CAST('2023-08-05 15:30:00' AS DATETIME), CAST('2023-08-05 16:00:00' AS timestamp_ntz)); + drop table mc_parts; + CREATE TABLE `mc_parts` ( + `mc_bigint` bigint, + `mc_string` string + )PARTITIONED BY ( + `dt` string + ); + INSERT INTO `mc_parts` PARTITION (dt='2023-08-01') VALUES + (1001, 'Sample data 1'); + INSERT INTO `mc_parts` PARTITION (dt='2023-08-02') VALUES + (1002, 'Sample data 2'); + INSERT INTO `mc_parts` PARTITION (dt='2023-08-03') VALUES + (1003, 'Sample data 3'); + INSERT INTO `mc_parts` PARTITION (dt='2023-08-04') VALUES + (1004, 'Sample data 4'); + INSERT INTO `mc_parts` PARTITION (dt='2023-08-05') VALUES + (1005, 'Sample data 5'); + CREATE TABLE int_types ( + mc_boolean BOOLEAN, + mc_tinyint TINYINT, + mc_int SMALLINT, + mc_bigint BIGINT + ); + INSERT INTO int_types VALUES (TRUE, CAST(-128 AS TINYINT), CAST(-32768 AS SMALLINT), CAST(-9223372036854775807 AS BIGINT)); + INSERT INTO int_types VALUES (FALSE, CAST(127 AS TINYINT), CAST(32767 AS SMALLINT), CAST(9223372036854775807 AS BIGINT)); + INSERT INTO int_types VALUES (TRUE, CAST(0 AS TINYINT), CAST(0 AS SMALLINT), CAST(0 AS BIGINT)); + INSERT INTO int_types VALUES (FALSE, CAST(0 AS TINYINT), CAST(0 AS SMALLINT), CAST(0 AS BIGINT)); + INSERT INTO int_types VALUES (TRUE, CAST(1 AS TINYINT), CAST(1 AS SMALLINT), CAST(1 AS BIGINT)); + INSERT INTO int_types VALUES (FALSE, CAST(-1 AS TINYINT), CAST(-1 AS SMALLINT), CAST(-1 AS BIGINT)); + INSERT INTO int_types VALUES + (TRUE, CAST(1 AS TINYINT), CAST(100 AS SMALLINT), CAST(1000 AS BIGINT)), + (FALSE, CAST(2 AS TINYINT), CAST(200 AS SMALLINT), CAST(2000 AS BIGINT)), + (TRUE, CAST(3 AS TINYINT), CAST(300 AS SMALLINT), CAST(3000 AS BIGINT)), + (FALSE, CAST(4 AS TINYINT), CAST(400 AS SMALLINT), CAST(4000 AS BIGINT)), + (TRUE, CAST(5 AS TINYINT), CAST(500 AS SMALLINT), CAST(5000 AS BIGINT)), + (FALSE, CAST(6 AS TINYINT), CAST(600 AS SMALLINT), CAST(6000 AS BIGINT)), + (TRUE, CAST(7 AS TINYINT), CAST(700 AS SMALLINT), CAST(7000 AS BIGINT)), + (FALSE, CAST(8 AS TINYINT), CAST(800 AS SMALLINT), CAST(8000 AS BIGINT)), + (TRUE, CAST(9 AS TINYINT), CAST(900 AS SMALLINT), CAST(9000 AS BIGINT)), + (FALSE, CAST(10 AS TINYINT), CAST(1000 AS SMALLINT), CAST(10000 AS BIGINT)); + CREATE TABLE web_site ( + web_site_sk BIGINT, + web_site_id STRING, + web_rec_start_date DATE, + web_rec_end_date DATE, + web_name STRING, + web_open_date_sk BIGINT, + web_close_date_sk BIGINT, + web_class STRING, + web_manager STRING, + web_mkt_id INT, + web_mkt_class STRING, + web_mkt_desc STRING, + web_market_manager STRING, + web_company_id INT, + web_company_name STRING, + web_street_number STRING, + web_street_name STRING, + web_street_type STRING, + web_suite_number STRING, + web_city STRING, + web_county STRING, + web_state STRING, + web_zip STRING, + web_country STRING, + web_gmt_offset DOUBLE, + web_tax_percentage DECIMAL(5,2) + ); + INSERT INTO web_site VALUES + ( + CAST(1 AS BIGINT), + 'WS0001', + CAST('2023-01-01' AS DATE), + CAST('2023-12-31' AS DATE), + 'Example Web Site 1', + CAST(20230101 AS BIGINT), + CAST(20231231 AS BIGINT), + 'E-commerce', + 'John Doe', + CAST(101 AS INT), + 'Retail', + 'Online retail website', + 'Jane Smith', + CAST(201 AS INT), + 'Example Company', + '1234', + 'Main Street', + 'Apt', + 'Unit 101', + 'Metropolis', + 'County', + 'NY', + '12345', + 'USA', + CAST(-5.0 AS DOUBLE), + CAST(8.25 AS DECIMAL(5,2)) + ); + INSERT INTO web_site VALUES + ( + CAST(2 AS BIGINT), + 'WS0002', + CAST('2023-02-01' AS DATE), + CAST('2023-11-30' AS DATE), + 'Example Web Site 2', + CAST(20230201 AS BIGINT), + CAST(20231130 AS BIGINT), + 'Technology', + 'Alice Johnson', + CAST(102 AS INT), + 'Tech', + 'Tech news and reviews', + 'Bob Brown', + CAST(202 AS INT), + 'Tech Innovations', + '5678', + 'Tech Avenue', + 'Suite', + 'Suite 200', + 'Gotham', + 'County', + 'CA', + '67890', + 'USA', + CAST(-8.0 AS DOUBLE), + CAST(7.50 AS DECIMAL(5,2)) + ); + INSERT INTO web_site VALUES + ( + CAST(3 AS BIGINT), + 'WS0003', + CAST('2023-03-01' AS DATE), + CAST('2023-10-31' AS DATE), + 'Example Web Site 3', + CAST(20230301 AS BIGINT), + CAST(20231031 AS BIGINT), + 'Healthcare', + 'Robert White', + CAST(103 AS INT), + 'Health', + 'Healthcare services and products', + 'Emily Green', + CAST(203 AS INT), + 'Health Corp', + '9101', + 'Health Drive', + 'Floor', + 'Floor 3', + 'Star City', + 'County', + 'TX', + '23456', + 'USA', + CAST(-6.0 AS DOUBLE), + CAST(6.75 AS DECIMAL(5,2)) + ); + INSERT INTO web_site VALUES + ( + CAST(4 AS BIGINT), + 'WS0004', + CAST('2023-04-01' AS DATE), + CAST('2023-09-30' AS DATE), + 'Example Web Site 4', + CAST(20230401 AS BIGINT), + CAST(20230930 AS BIGINT), + 'Education', + 'David Black', + CAST(104 AS INT), + 'EdTech', + 'Educational technology platform', + 'Fiona Grey', + CAST(204 AS INT), + 'Edu Tech', + '1122', + 'Education Lane', + 'Building', + 'Building 1', + 'Smallville', + 'County', + 'FL', + '34567', + 'USA', + CAST(-4.0 AS DOUBLE), + CAST(5.00 AS DECIMAL(5,2)) + ); + INSERT INTO web_site VALUES + ( + CAST(5 AS BIGINT), + 'WS0005', + CAST('2023-05-01' AS DATE), + CAST('2023-08-31' AS DATE), + 'Example Web Site 5', + CAST(20230501 AS BIGINT), + CAST(20230831 AS BIGINT), + 'Travel', + 'Sophia Blue', + CAST(105 AS INT), + 'Travel', + 'Travel and booking services', + 'Daniel Red', + CAST(205 AS INT), + 'Travel Inc', + '3344', + 'Tourist Street', + 'Unit', + 'Unit 5', + 'Metropolis', + 'County', + 'WA', + '45678', + 'USA', + CAST(-7.0 AS DOUBLE), + CAST(8.00 AS DECIMAL(5,2)) + ); + + */ suite("test_external_catalog_maxcompute", "p2,external,maxcompute,external_remote,external_remote_maxcompute") { String enabled = context.config.otherConfigs.get("enableMaxComputeTest") if (enabled != null && enabled.equalsIgnoreCase("true")) { @@ -27,29 +292,30 @@ suite("test_external_catalog_maxcompute", "p2,external,maxcompute,external_remot sql """ create catalog if not exists ${mc_catalog_name} properties ( "type" = "max_compute", - "mc.region" = "cn-beijing", "mc.default.project" = "${mc_db}", "mc.access_key" = "${ak}", "mc.secret_key" = "${sk}", - "mc.public_access" = "true" + "mc.endpoint" = "http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api" ); """ - + // query data test def q01 = { - qt_q1 """ select count(*) from store_sales """ + order_qt_q1 """ select count(*) from web_site """ } // data type test def q02 = { - qt_q2 """ select * from web_site where web_site_id=2 order by web_site_id """ // test char,date,varchar,double,decimal - qt_q3 """ select * from int_types order by mc_boolean limit 2 """ // test bool,tinyint,int,bigint + order_qt_q2 """ select * from web_site where web_site_id>='WS0003' order by web_site_id; """ // test char,date,varchar,double,decimal + order_qt_q3 """ select * from int_types """ // test bool,tinyint,int,bigint + order_qt_q3_1 """ select * from int_types order by mc_boolean """ + order_qt_q3_2 """ select * from int_types order by mc_int""" } // test partition table filter def q03 = { - qt_q4 """ select * from mc_parts where dt = '2020-09-21' """ - qt_q5 """ select * from mc_parts where dt = '2021-08-21' """ - qt_q6 """ select * from mc_parts where dt = '2020-09-21' and mc_bigint > 6223 """ - qt_q7 """ select * from mc_parts where dt = '2020-09-21' or (mc_bigint > 0 and dt > '2020-09-20') order by mc_bigint, dt limit 3; """ + order_qt_q4 """ select * from mc_parts where dt = '2023-08-03' order by mc_bigint """ + order_qt_q5 """ select * from mc_parts where dt > '2023-08-03' order by mc_bigint """ + order_qt_q6 """ select * from mc_parts where dt > '2023-08-03' and mc_bigint > 1002 """ + order_qt_q7 """ select * from mc_parts where dt < '2023-08-03' or (mc_bigint > 1003 and dt > '2023-08-04') order by mc_bigint, dt; """ } sql """ switch `${mc_catalog_name}`; """ @@ -63,30 +329,29 @@ suite("test_external_catalog_maxcompute", "p2,external,maxcompute,external_remot sql """ create catalog if not exists ${mc_catalog_name} properties ( "type" = "max_compute", - "mc.region" = "cn-beijing", "mc.default.project" = "${mc_db}", "mc.access_key" = "${ak}", "mc.secret_key" = "${sk}", - "mc.public_access" = "true" + "mc.endpoint" = "http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api" ); """ sql """ switch `${mc_catalog_name}`; """ sql """ use `${mc_db}`; """ - qt_replay_q6 """ select * from mc_parts where dt = '2020-09-21' and mc_bigint > 6223 """ - + order_qt_replay_q6 """ select * from mc_parts where dt >= '2023-08-03' and mc_bigint > 1001 order by mc_bigint """ + // test multi partitions prune sql """ refresh catalog ${mc_catalog_name} """ sql """ switch `${mc_catalog_name}`; """ sql """ use `${mc_db}`; """ - qt_multi_partition_q1 """ show partitions from multi_partitions limit 5,3; """ - qt_multi_partition_q2 """ select pt, create_time, yy, mm, dd from multi_partitions where pt>-1 and yy > '' and mm > '' and dd >'' order by pt desc, dd desc limit 3; """ - qt_multi_partition_q3 """ select sum(pt), create_time, yy, mm, dd from multi_partitions where yy > '' and mm > '' and dd >'' group by create_time, yy, mm, dd order by dd limit 3; """ - qt_multi_partition_q4 """ select count(*) from multi_partitions where pt>-1 and yy > '' and mm > '' and dd <= '30'; """ - qt_multi_partition_q5 """ select create_time, yy, mm, dd from multi_partitions where yy = '2021' and mm='12' and dd='21' order by pt limit 3; """ - qt_multi_partition_q6 """ select max(pt), yy, mm from multi_partitions where yy = '2021' and mm='12' group by yy, mm order by yy, mm; """ - qt_multi_partition_q7 """ select count(*) from multi_partitions where yy < '2022'; """ - qt_multi_partition_q8 """ select count(*) from multi_partitions where pt>=14; """ - qt_multi_partition_q9 """ select city,mnt,gender,finished_time,order_rate,cut_date,create_time,pt, yy, mm, dd from multi_partitions where pt >= 12 and pt < 14 and finished_time is not null; """ - qt_multi_partition_q10 """ select pt, yy, mm, dd from multi_partitions where pt >= 12 and create_time > '2022-04-23 11:11:00' order by pt, yy, mm, dd limit 3; """ + order_qt_multi_partition_q1 """ show partitions from multi_partitions; """ + order_qt_multi_partition_q2 """ select pt, create_time, yy, mm, dd from multi_partitions where pt>-1 and yy > '' and mm > '' and dd >'' order by pt , dd; """ + order_qt_multi_partition_q3 """ select sum(pt), create_time, yy, mm, dd from multi_partitions where yy > '' and mm > '' and dd >'' group by create_time, yy, mm, dd order by create_time,dd ; """ + order_qt_multi_partition_q4 """ select count(*) from multi_partitions where pt>-1 and yy > '' and mm > '' and dd <= '30'; """ + order_qt_multi_partition_q5 """ select create_time, yy, mm, dd from multi_partitions where yy = '2023' and mm='08' and dd='04' order by pt ; """ + order_qt_multi_partition_q6 """ select max(pt), yy, mm from multi_partitions where yy = '2023' and mm='08' group by yy, mm order by yy, mm; """ + order_qt_multi_partition_q7 """ select count(*) from multi_partitions where yy < '2023' or dd < '03'; """ + order_qt_multi_partition_q8 """ select count(*) from multi_partitions where pt>=3; """ + order_qt_multi_partition_q9 """ select city,mnt,gender,finished_time,order_rate,cut_date,create_time,pt, yy, mm, dd from multi_partitions where pt >= 2 and pt < 4 and finished_time is not null; """ + order_qt_multi_partition_q10 """ select pt, yy, mm, dd from multi_partitions where pt >= 2 and create_time > '2023-08-03 03:11:00' order by pt, yy, mm, dd; """ } } diff --git a/regression-test/suites/external_table_p2/maxcompute/test_max_compute_all_type.groovy b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_all_type.groovy new file mode 100644 index 00000000000000..42aef6a2928a73 --- /dev/null +++ b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_all_type.groovy @@ -0,0 +1,547 @@ +// 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. + +/* + +drop table mc_all_types; + +CREATE TABLE `mc_all_types`( + `id` int, + `boolean_col` boolean, + `tinyint_col` tinyint, + `smallint_col` smallint, + `int_col` int, + `bigint_col` bigint, + `float_col` float, + `double_col` double, + `decimal_col1` decimal(9,0), + `decimal_col2` decimal(8,4), + `decimal_col3` decimal(18,6), + `decimal_col4` decimal(38,12), + `string_col` string, + `date_col` date, + `datetime_col` datetime, + `timestamp_ntz_col2` timestamp_ntz, + `timestamp_ntz_col3` timestamp_ntz, + `char_col1` char(50), + `char_col2` char(100), + `char_col3` char(255), + `varchar_col1` varchar(50), + `varchar_col2` varchar(100), + `varchar_col3` varchar(255), + `t_map_string` map, + `t_map_varchar` map, + `t_map_char` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_array_string` array, + `t_array_int` array, + `t_array_bigint` array, + `t_array_float` array, + `t_array_double` array, + `t_array_boolean` array, + `t_array_varchar` array, + `t_array_char` array, + `t_array_decimal_precision_2` array, + `t_array_decimal_precision_4` array, + `t_array_decimal_precision_8` array, + `t_array_decimal_precision_17` array, + `t_array_decimal_precision_18` array, + `t_array_decimal_precision_38` array, + `t_struct_bigint` struct, + `t_complex` map>>, + `t_struct_nested` struct>, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array +); + + + +insert into mc_all_types +VALUES ( + 1, + cast(0 as boolean), -- boolean_col + cast(-7 as tinyint), -- tinyint_col + cast(-15 as smallint), -- smallint_col + 16, -- int_col + cast(-9223372036854775807 as bigint), -- bigint_col + CAST(-123.45 AS FLOAT), -- float_col + CAST(-123456.789 AS DOUBLE), -- double_col + CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + 'str', -- string_col + cast('2024-03-25' as date), -- date_col + cast('2024-03-25 12:00:00' as datetime), + cast('2024-03-25 12:00:00.123456' as timestamp_ntz), -- timestamp_ntz_col2 + cast('2024-03-25 12:00:00.123456789' as timestamp_ntz), -- timestamp_ntz_col3 + 'char_value11111', -- char_col1 + 'char_value22222', -- char_col2 + 'char_value33333', -- char_col3 + 'varchar_value11111', -- varchar_col1 + 'varchar_value22222', -- varchar_col2 + 'varchar_value33333', -- varchar_col3 + MAP('key7', 'value1'), -- t_map_string + MAP('key7', 'value1'), -- t_map_varchar + MAP('x', 'y'), -- t_map_char + MAP(3, 20), -- t_map_int + MAP(3, 200000000000), -- t_map_bigint + MAP(CAST(3.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float + MAP(CAST(3.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double + MAP(false, true), -- t_map_boolean + ARRAY('string1', 'string2'), -- t_array_string + ARRAY(4, 5, 6), -- t_array_int + ARRAY(300000000000, 400000000000), -- t_array_bigint + ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float + ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double + ARRAY(false, true), -- t_array_boolean + ARRAY('varchar1', 'varchar2'), -- t_array_varchar + ARRAY('char1', 'char2'), -- t_array_char + ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(9.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(6.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', -1234567890), -- t_struct_bigint + MAP('key', ARRAY(NAMED_STRUCT('s_int', -123))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested + NAMED_STRUCT('struct_non_nulls_after_nulls1', -123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', -123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', -123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls + ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls + ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between + ARRAY('value11', 'value2', null) -- t_array_string_ending_with_nulls +); + + + +INSERT into mc_all_types +VALUES ( + 2, + CAST (1 as boolean), -- boolean_col + CAST (127 as tinyint), -- tinyint_col + cast(32767 as smallint), -- smallint_col + 2147483647, -- int_col + cast(9223372036854775807 as bigint), -- bigint_col + cast(123.45 as float), -- float_col + 123456.789, -- double_col + CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + 'string_value', -- string_col + cast('2024-03-20' as date), -- date_col + cast('2024-03-20 12:00:00' as datetime ), + cast('2024-03-20 12:00:00.123456' as TIMESTAMP_ntz ), -- timestamp_ntz_col2 + cast('2024-03-20 12:00:00.123456789' as TIMESTAMP_ntz ), -- timestamp_ntz_col3 + 'char_value1', -- char_col1 + 'char_value2', -- char_col2 + 'char_value3', -- char_col3 + 'varchar_value1', -- varchar_col1 + 'varchar_value2', -- varchar_col2 + 'varchar_value3', -- varchar_col3 + MAP('key1', 'value1'), -- t_map_string + MAP('key1', 'value1'), -- t_map_varchar + MAP('a', 'b'), -- t_map_char + MAP(1, 10), -- t_map_int + MAP(1, 100000000000), -- t_map_bigint + MAP(CAST(1.1 AS FLOAT), CAST(10.1 AS FLOAT)), -- t_map_float + MAP(CAST(1.1 AS DOUBLE), CAST(10.1 AS DOUBLE)), -- t_map_double + MAP(TRUE, FALSE), -- t_map_boolean + ARRAY('string1', 'string2'), -- t_array_string + ARRAY(1, 2, 3), -- t_array_int + ARRAY(100000000000, 200000000000), -- t_array_bigint + ARRAY(CAST(1.1 AS FLOAT), CAST(2.2 AS FLOAT)), -- t_array_float + ARRAY(CAST(1.123456789 AS DOUBLE), CAST(2.123456789 AS DOUBLE)), -- t_array_double + ARRAY(TRUE, FALSE), -- t_array_boolean + ARRAY('varchar1', 'varchar2'), -- t_array_varchar + ARRAY('char1', 'char2'), -- t_array_char + ARRAY(CAST(1.1 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(1.23 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(1.23456789 AS DECIMAL(17,8)), CAST(2.34567891 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(1.23456789, 2.34567891), -- t_array_decimal_precision_18 + ARRAY(1.234567890123456789, 2.345678901234567890), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', 1234567890), -- t_struct_bigint + MAP('key', ARRAY(NAMED_STRUCT('s_int', 123))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested + NAMED_STRUCT('struct_non_nulls_after_nulls1', 123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', 123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', 123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls + ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls + ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between + ARRAY('value1', 'value2', null) -- t_array_string_ending_with_nulls +); + +INSERT into mc_all_types +VALUES +( + 3, + cast(0 as boolean), -- boolean_col + cast(-7 as tinyint), -- tinyint_col + cast(-15 as smallint), -- smallint_col + 16, -- int_col + cast(-9223372036854775807 as bigint), -- bigint_col + CAST(-123.45 AS FLOAT), -- float_col + CAST(-123456.789 AS DOUBLE), -- double_col + CAST(123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + 'str', -- string_col + cast('2024-03-25' as date), -- date_col + cast('2024-03-25 12:00:00' as datetime), + cast('2024-03-25 12:00:00.123456' as timestamp_ntz), -- timestamp_ntz_col2 + cast('2024-03-25 12:00:00.123456789' as timestamp_ntz), -- timestamp_ntz_col3 + 'char_value11111', -- char_col1 + 'char_value22222', -- char_col2 + 'char_value33333', -- char_col3 + 'varchar_value11111', -- varchar_col1 + 'varchar_value22222', -- varchar_col2 + 'varchar_value33333', -- varchar_col3 + MAP('key7', 'value1'), -- t_map_string + MAP('key7', 'value1'), -- t_map_varchar + MAP('x', 'y'), -- t_map_char + MAP(3, 20), -- t_map_int + MAP(3, 200000000000), -- t_map_bigint + MAP(CAST(3.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float + MAP(CAST(3.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double + MAP(false, true), -- t_map_boolean + ARRAY('string1', 'string2'), -- t_array_string + ARRAY(4, 5, 6), -- t_array_int + ARRAY(300000000000, 400000000000), -- t_array_bigint + ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float + ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double + ARRAY(false, true), -- t_array_boolean + ARRAY('varchar1', 'varchar2'), -- t_array_varchar + ARRAY('char1', 'char2'), -- t_array_char + ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(9.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(6.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', -1234567890), -- t_struct_bigint + MAP('key', ARRAY(NAMED_STRUCT('s_int', -123))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested + NAMED_STRUCT('struct_non_nulls_after_nulls1', -123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', -123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', -123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls + ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls + ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between + ARRAY('value11', 'value2', null) -- t_array_string_ending_with_nulls +); +INSERT into mc_all_types +VALUES +( + 4, + cast(0 as boolean), -- boolean_col + cast(-128 as tinyint), -- tinyint_col + cast(-32768 as smallint), -- smallint_col + -2147483647, -- int_col + cast(-9223372036854775807 as bigint), -- bigint_col + CAST(-123.45 AS FLOAT), -- float_col + CAST(-123456.789 AS DOUBLE), -- double_col + CAST(-123456789 AS DECIMAL(9,0)), -- decimal_col1 + CAST(-1234.5678 AS DECIMAL(8,4)), -- decimal_col2 + CAST(-123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(-123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + 'string_value', -- string_col + cast('2024-03-21' as date), -- date_col + cast('2024-03-21 12:00:00' as datetime), + cast('2024-03-21 12:00:00.123456' as timestamp_ntz), -- timestamp_ntz_col2 + cast('2024-03-21 12:00:00.123456789' as timestamp_ntz), -- timestamp_ntz_col3 + 'char_value1', -- char_col1 + 'char_value2', -- char_col2 + 'char_value3', -- char_col3 + 'varchar_value1', -- varchar_col1 + 'varchar_value2', -- varchar_col2 + 'varchar_value3', -- varchar_col3 + MAP('key1', 'value1'), -- t_map_string + MAP('key1', 'value1'), -- t_map_varchar + MAP('x', 'y'), -- t_map_char + MAP(2, 20), -- t_map_int + MAP(2, 200000000000), -- t_map_bigint + MAP(CAST(2.2 AS FLOAT), CAST(20.2 AS FLOAT)), -- t_map_float + MAP(CAST(2.2 AS DOUBLE), CAST(20.2 AS DOUBLE)), -- t_map_double + MAP(false, true), -- t_map_boolean + ARRAY('string1', 'string2'), -- t_array_string + ARRAY(4, 5, 6), -- t_array_int + ARRAY(300000000000, 400000000000), -- t_array_bigint + ARRAY(CAST(3.3 AS FLOAT), CAST(4.4 AS FLOAT)), -- t_array_float + ARRAY(CAST(3.123456789 AS DOUBLE), CAST(4.123456789 AS DOUBLE)), -- t_array_double + ARRAY(false, true), -- t_array_boolean + ARRAY('varchar1', 'varchar2'), -- t_array_varchar + ARRAY('char1', 'char2'), -- t_array_char + ARRAY(CAST(3.3 AS DECIMAL(2,1)), CAST(4.4 AS DECIMAL(2,1))), -- t_array_decimal_precision_2 + ARRAY(CAST(3.45 AS DECIMAL(4,2)), CAST(4.56 AS DECIMAL(4,2))), -- t_array_decimal_precision_4 + ARRAY(CAST(3.4567 AS DECIMAL(8,4)), CAST(4.5678 AS DECIMAL(8,4))), -- t_array_decimal_precision_8 + ARRAY(CAST(3.45678901 AS DECIMAL(17,8)), CAST(4.56789012 AS DECIMAL(17,8))), -- t_array_decimal_precision_17 + ARRAY(CAST(3.45678901 AS DECIMAL(18,8)), CAST(4.56789012 AS DECIMAL(18,8))), -- t_array_decimal_precision_18 + ARRAY(CAST(3.456789012345678901 AS DECIMAL(38,16)), CAST(4.567890123456789012 AS DECIMAL(38,16))), -- t_array_decimal_precision_38 + NAMED_STRUCT('s_bigint', -1234567890), -- t_struct_bigint + MAP('key', ARRAY(NAMED_STRUCT('s_int', -123))), -- t_complex + NAMED_STRUCT('struct_field', ARRAY('value1', 'value2')), -- t_struct_nested + NAMED_STRUCT('struct_non_nulls_after_nulls1', -123, 'struct_non_nulls_after_nulls2', 'value'), -- t_struct_non_nulls_after_nulls + NAMED_STRUCT('struct_field1', -123, 'struct_field2', 'value', 'strict_field3', NAMED_STRUCT('nested_struct_field1', -123, 'nested_struct_field2', 'nested_value')), -- t_nested_struct_non_nulls_after_nulls + ARRAY(null, 'value1', 'value2'), -- t_array_string_starting_with_nulls + ARRAY('value1', null, 'value2'), -- t_array_string_with_nulls_in_between + ARRAY('value1', 'value2', null) -- t_array_string_ending_with_nulls +); + +ALTER TABLE mc_all_types MERGE SMALLFILES; + +select * from mc_all_types; + */ +suite("test_max_compute_all_type", "p2,external,maxcompute,external_remote,external_remote_maxcompute") { + String enabled = context.config.otherConfigs.get("enableMaxComputeTest") + if (enabled != null && enabled.equalsIgnoreCase("true")) { + String ak = context.config.otherConfigs.get("aliYunAk") + String sk = context.config.otherConfigs.get("aliYunSk") + String mc_catalog_name = "test_max_compute_all_type" + sql """drop catalog if exists ${mc_catalog_name} """ + + String defaultProject = "jz_datalake" + sql """ + CREATE CATALOG IF NOT EXISTS ${mc_catalog_name} PROPERTIES ( + "type" = "max_compute", + "mc.default.project" = "${defaultProject}", + "mc.access_key" = "${ak}", + "mc.secret_key" = "${sk}", + "mc.endpoint" = "http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api", + "mc.quota" = "pay-as-you-go" + ); + """ + + logger.info("catalog " + mc_catalog_name + " created") + sql """switch ${mc_catalog_name};""" + logger.info("switched to catalog " + mc_catalog_name) + sql """ show databases; """ + sql """ use ${defaultProject} """ + + String table_name = "mc_all_types" + + + qt_desc """ desc ${table_name} """ + qt_test_1 """ select * from ${table_name} order by id; """ + qt_test_2 """ select * from ${table_name} where id = 1 ; """ + + qt_test_3 """ select id,boolean_col from ${table_name} order by id """ + qt_test_4 """ select id,boolean_col from ${table_name} where boolean_col is false order by id; """ + + qt_test_5 """ select id,tinyint_col from ${table_name} order by id """ + qt_test_5 """ select id,tinyint_col from ${table_name} where tinyint_col in (-127,-128) order by id """ + + qt_test_7 """ select id,smallint_col from ${table_name} order by id """ + qt_test_8 """ select id,smallint_col from ${table_name} where smallint_col > 16 order by id """ + + qt_test_9 """ select id,int_col from ${table_name} order by id """ + qt_test_10 """ select id,int_col from ${table_name} where int_col = 16 order by id """ + qt_test_11 """ select id,int_col from ${table_name} where int_col != 16 order by id """ + + qt_test_12 """ select id,bigint_col from ${table_name} order by id """ + qt_test_13 """ select id,bigint_col from ${table_name} where bigint_col > -9223372036854775807 order by id """ + qt_test_14 """ select id,bigint_col from ${table_name} where bigint_col >= -9223372036854775807 order by id """ + + + qt_test_15 """ select id,float_col from ${table_name} order by id """ + qt_test_16 """ select id,float_col from ${table_name} where round(float_col,2) > -123.45 order by id """ + qt_test_17 """ select id,float_col from ${table_name} where round(float_col,2) < -123.45 order by id """ + qt_test_18 """ select id,float_col from ${table_name} where round(float_col,2) = -123.45 order by id """ + qt_test_19 """ select id,float_col from ${table_name} where round(float_col,2) >= -123.45 order by id """ + qt_test_20 """ select id,float_col from ${table_name} where round(float_col,2) <= -123.45 order by id """ + qt_test_21 """ select id,float_col from ${table_name} where round(float_col,2) in (-123.45,0.123) order by id """ + qt_test_22 """ select id,float_col from ${table_name} where round(float_col,2) not in (-123.45, 0.1) order by id """ + + + qt_test_23 """ select id,double_col from ${table_name} order by id """ + qt_test_24 """ select id,double_col from ${table_name} where double_col in ( -123456.789 ,0.123) order by id """ + qt_test_25 """ select id,double_col from ${table_name} where double_col != -123456.789 order by id """ + qt_test_26 """ select id,double_col from ${table_name} where double_col = -123456.789 order by id """ + qt_test_27 """ select id,double_col from ${table_name} where -123456.789 = double_col order by id """ + + qt_test_28 """ select id,decimal_col1 from ${table_name} order by id """ + qt_test_29 """ select id,decimal_col1 from ${table_name} where decimal_col1 is null order by id """ + qt_test_30 """ select id,decimal_col1 from ${table_name} where decimal_col1 is not null order by id """ + + qt_test_31 """ select id,decimal_col2 from ${table_name} order by id """ + qt_test_32 """ select id,decimal_col2 from ${table_name} where decimal_col2 = -1234.5678 order by id """ + qt_test_33 """ select id,decimal_col2 from ${table_name} where decimal_col2 != -1234.5678 order by id""" + + qt_test_34 """ select id,decimal_col3 from ${table_name} order by id """ + qt_test_35 """ select id,decimal_col3 from ${table_name} where decimal_col3 = -123456.789012 order by id """ + qt_test_36 """ select id,decimal_col3 from ${table_name} where decimal_col3 != -123456.789012 order by id """ + + qt_test_37 """ select id,decimal_col4 from ${table_name} order by id """ + qt_test_38 """ select id,decimal_col4 from ${table_name} where decimal_col4 = -123456789.0123457 order by id """ + qt_test_39 """ select id,decimal_col4 from ${table_name} where decimal_col4 != -123456789.0123457 order by id """ + qt_test_40 """ select id,decimal_col4 from ${table_name} where decimal_col4 >= -123456789.0123457 order by id """ + qt_test_41 """ select id,decimal_col4 from ${table_name} where decimal_col4 > -123456789.0123457 order by id """ + + + qt_test_42 """ select id,string_col from ${table_name} order by id """ + qt_test_43 """ select id,string_col from ${table_name} where string_col in ("str") order by id """ + qt_test_44 """ select id,string_col from ${table_name} where string_col = "str" order by id """ + qt_test_45 """ select id,string_col from ${table_name} where string_col != "str" order by id """ + qt_test_46 """ select id,string_col from ${table_name} where string_col in ("str","string_value") order by id """ + qt_test_47 """ select id,string_col from ${table_name} where string_col not in ("str","string_value") order by id """ + + qt_test_48 """ select id,date_col from ${table_name} order by id """ + qt_test_49 """ select id,date_col from ${table_name} where date_col = "2024-03-25" order by id """ + qt_test_50 """ select id,date_col from ${table_name} where date_col >= "2024-03-25" order by id """ + qt_test_51 """ select id,date_col from ${table_name} where date_col != "2024-03-20" order by id """ + + + + qt_test_52 """ select id,datetime_col from ${table_name} order by id """ + qt_test_53 """ select id,datetime_col from ${table_name} where datetime_col != "2024-03-25 12:00:00" order by id """ + qt_test_54 """ select id,datetime_col from ${table_name} where datetime_col = "2024-03-25 12:00:00" order by id """ + qt_test_55 """ select id,datetime_col from ${table_name} where datetime_col > "2024-03-25 12:00:00" order by id """ + qt_test_56 """ select id,datetime_col from ${table_name} where datetime_col < "2024-03-25 12:00:00" order by id """ + qt_test_57 """ select id,datetime_col from ${table_name} where datetime_col <= "2024-03-25 12:00:00" order by id """ + + + + qt_test_58 """ select id,timestamp_ntz_col2 from ${table_name} order by id """ + qt_test_59 """ select id,timestamp_ntz_col2 from ${table_name} where timestamp_ntz_col2 != "2024-03-25 12:00:00.123456" order by id """ + qt_test_60 """ select id,timestamp_ntz_col2 from ${table_name} where timestamp_ntz_col2 = "2024-03-25 12:00:00.123456" order by id """ + qt_test_61 """ select id,timestamp_ntz_col2 from ${table_name} where timestamp_ntz_col2 > "2024-03-25 12:00:00.123456" order by id """ + qt_test_62 """ select id,timestamp_ntz_col2 from ${table_name} where timestamp_ntz_col2 < "2024-03-25 12:00:00.123456" order by id """ + qt_test_63 """ select id,timestamp_ntz_col2 from ${table_name} where timestamp_ntz_col2 <= "2024-03-25 12:00:00.123456" order by id """ + + qt_test_64 """ select id,timestamp_ntz_col3 from ${table_name} order by id """ + qt_test_65 """ select id,timestamp_ntz_col3 from ${table_name} where timestamp_ntz_col3 != "2024-03-25 12:00:00.123456" order by id """ + qt_test_66 """ select id,timestamp_ntz_col3 from ${table_name} where timestamp_ntz_col3 < "2024-03-25 12:00:00.123456" order by id """ + qt_test_67 """ select id,timestamp_ntz_col3 from ${table_name} where timestamp_ntz_col3 = "2024-03-25 12:00:00.123456" order by id """ + qt_test_68 """ select id,timestamp_ntz_col3 from ${table_name} where timestamp_ntz_col3 > "2024-03-25 12:00:00.123456" order by id """ + qt_test_69 """ select id,timestamp_ntz_col3 from ${table_name} where timestamp_ntz_col3 <= "2024-03-25 12:00:00.123456" order by id """ + + qt_test_70 """ select id,timestamp_ntz_col3 from ${table_name} where timestamp_ntz_col3 != "2024-03-25 12:00:00.124" order by id """ + qt_test_71 """ select id,timestamp_ntz_col3 from ${table_name} where timestamp_ntz_col3 < "2024-03-25 12:00:00.124" order by id """ + qt_test_72 """ select id,timestamp_ntz_col3 from ${table_name} where timestamp_ntz_col3 = "2024-03-25 12:00:00.124" order by id """ + qt_test_73 """ select id,timestamp_ntz_col3 from ${table_name} where timestamp_ntz_col3 > "2024-03-25 12:00:00.124" order by id """ + qt_test_74 """ select id,timestamp_ntz_col3 from ${table_name} where timestamp_ntz_col3 <= "2024-03-25 12:00:00.124" order by id """ + + + + + + qt_test_75 """ select id,char_col1 from ${table_name} order by id """ + qt_test_76 """ select id,char_col1 from ${table_name} where char_col1 = "char_value11111" order by id """ + qt_test_77 """ select id,char_col1 from ${table_name} where char_col1 != "char_value11111" order by id """ + qt_test_78 """ select id,char_col1 from ${table_name} where char_col1 != "char_value1" order by id """ + qt_test_79 """ select id,char_col1 from ${table_name} where char_col1 in ("char_value1") order by id """ + qt_test_80 """ select id,char_col1 from ${table_name} where char_col1 not in ("char_value1") order by id """ + qt_test_81 """ select id,char_col1 from ${table_name} where char_col1 in ("char_value1","char_value11111") order by id """ + qt_test_82 """ select id,char_col1 from ${table_name} where char_col1 not in ("char_value1","char_value11111") order by id """ + + + qt_test_83 """ select id,char_col2 from ${table_name} order by id """ + qt_test_84 """ select id,char_col2 from ${table_name} where char_col2 = "char_value22222" order by id """ + qt_test_85 """ select id,char_col2 from ${table_name} where char_col2 != "char_value22222" order by id """ + qt_test_86 """ select id,char_col2 from ${table_name} where char_col2 != "char_value2" order by id """ + qt_test_87 """ select id,char_col2 from ${table_name} where char_col2 in ("char_value2") order by id """ + qt_test_88 """ select id,char_col2 from ${table_name} where char_col2 not in ("char_value2") order by id """ + qt_test_89 """ select id,char_col2 from ${table_name} where char_col2 in ("char_value2","char_value22222") order by id """ + qt_test_90 """ select id,char_col2 from ${table_name} where char_col2 not in ("char_value2","char_value22222") order by id """ + + qt_test_91 """ select id,char_col3 from ${table_name} order by id """ + qt_test_92 """ select id,char_col3 from ${table_name} where char_col3 = "char_value33333" order by id """ + qt_test_93 """ select id,char_col3 from ${table_name} where char_col3 != "char_value33333" order by id """ + qt_test_94 """ select id,char_col3 from ${table_name} where char_col3 != "char_value3" order by id """ + qt_test_95 """ select id,char_col3 from ${table_name} where char_col3 in ("char_value3") order by id """ + qt_test_96 """ select id,char_col3 from ${table_name} where char_col3 not in ("char_value3") order by id """ + qt_test_97 """ select id,char_col3 from ${table_name} where char_col3 in ("char_value3","char_value33333") order by id """ + qt_test_98 """ select id,char_col3 from ${table_name} where char_col3 not in ("char_value3","char_value33333") order by id """ + + + qt_test_99 """ select id,varchar_col1 from ${table_name} order by id """ + qt_test_100 """ select id,varchar_col1 from ${table_name} where varchar_col1 = "varchar_value11111" order by id """ + qt_test_101 """ select id,varchar_col1 from ${table_name} where varchar_col1 != "varchar_value11111" order by id """ + qt_test_102 """ select id,varchar_col1 from ${table_name} where varchar_col1 != "varchar_value1" order by id """ + qt_test_103 """ select id,varchar_col1 from ${table_name} where varchar_col1 in ("varchar_value1") order by id """ + qt_test_104 """ select id,varchar_col1 from ${table_name} where varchar_col1 not in ("varchar_value1") order by id """ + qt_test_105 """ select id,varchar_col1 from ${table_name} where varchar_col1 in ("varchar_value1","varchar_value11111") order by id """ + qt_test_106 """ select id,varchar_col1 from ${table_name} where varchar_col1 not in ("varchar_value1","varchar_value11111") order by id """ + + + qt_test_107 """ select id,varchar_col2 from ${table_name} order by id """ + qt_test_108 """ select id,varchar_col2 from ${table_name} where varchar_col2 = "varchar_value22222" order by id """ + qt_test_109 """ select id,varchar_col2 from ${table_name} where varchar_col2 != "varchar_value22222" order by id """ + qt_test_110 """ select id,varchar_col2 from ${table_name} where varchar_col2 != "varchar_value2" order by id """ + qt_test_111 """ select id,varchar_col2 from ${table_name} where varchar_col2 in ("varchar_value2") order by id """ + qt_test_112 """ select id,varchar_col2 from ${table_name} where varchar_col2 not in ("varchar_value2") order by id """ + qt_test_113 """ select id,varchar_col2 from ${table_name} where varchar_col2 in ("varchar_value2","varchar_value22222") order by id """ + qt_test_114 """ select id,varchar_col2 from ${table_name} where varchar_col2 not in ("varchar_value2","varchar_value22222") order by id """ + + qt_test_115 """ select id,varchar_col3 from ${table_name} order by id """ + qt_test_116 """ select id,varchar_col3 from ${table_name} where varchar_col3 = "varchar_value33333" order by id """ + qt_test_117 """ select id,varchar_col3 from ${table_name} where varchar_col3 != "varchar_value33333" order by id """ + qt_test_118 """ select id,varchar_col3 from ${table_name} where varchar_col3 != "varchar_value3" order by id """ + qt_test_119 """ select id,varchar_col3 from ${table_name} where varchar_col3 in ("varchar_value3") order by id """ + qt_test_120 """ select id,varchar_col3 from ${table_name} where varchar_col3 not in ("varchar_value3") order by id """ + qt_test_121 """ select id,varchar_col3 from ${table_name} where varchar_col3 in ("varchar_value3","varchar_value33333") order by id """ + qt_test_122 """ select id,varchar_col3 from ${table_name} where varchar_col3 not in ("varchar_value3","varchar_value33333") order by id """ + + + + + qt_test_123 """ select id,t_map_string from ${table_name} order by id """ + qt_test_124 """ select id,t_map_varchar from ${table_name} order by id """ + qt_test_125 """ select id,t_map_char from ${table_name} order by id """ + qt_test_126 """ select id,t_map_int from ${table_name} order by id """ + qt_test_127 """ select id,t_map_bigint from ${table_name} order by id """ + qt_test_128 """ select id,t_map_float from ${table_name} order by id """ + qt_test_129 """ select id,t_map_double from ${table_name} order by id """ + qt_test_130 """ select id,t_map_boolean from ${table_name} order by id """ + qt_test_131 """ select id,t_array_string from ${table_name} order by id """ + qt_test_132 """ select id,t_array_int from ${table_name} order by id """ + qt_test_133 """ select id,t_array_bigint from ${table_name} order by id """ + qt_test_134 """ select id,t_array_float from ${table_name} order by id """ + qt_test_135 """ select id,t_array_double from ${table_name} order by id """ + qt_test_136 """ select id,t_array_boolean from ${table_name} order by id """ + qt_test_137 """ select id,t_array_varchar from ${table_name} order by id """ + qt_test_138 """ select id,t_array_char from ${table_name} order by id """ + qt_test_139 """ select id,t_array_decimal_precision_2 from ${table_name} order by id """ + qt_test_140 """ select id,t_array_decimal_precision_4 from ${table_name} order by id """ + qt_test_141 """ select id,t_array_decimal_precision_8 from ${table_name} order by id """ + qt_test_142 """ select id,t_array_decimal_precision_17 from ${table_name} order by id """ + qt_test_143 """ select id,t_array_decimal_precision_18 from ${table_name} order by id """ + qt_test_144 """ select id,t_array_decimal_precision_4 from ${table_name} order by id """ + qt_test_145 """ select id,t_array_decimal_precision_38 from ${table_name} order by id """ + qt_test_146 """ select id,t_struct_bigint from ${table_name} order by id """ + qt_test_147 """ select id,t_complex from ${table_name} order by id """ + qt_test_148 """ select id,t_struct_nested from ${table_name} order by id """ + qt_test_149 """ select id,t_struct_non_nulls_after_nulls from ${table_name} order by id """ + qt_test_150 """ select id,t_nested_struct_non_nulls_after_nulls from ${table_name} order by id """ + qt_test_151 """ select id,t_array_string_starting_with_nulls from ${table_name} order by id """ + qt_test_152 """ select id,t_array_string_with_nulls_in_between from ${table_name} order by id """ + qt_test_153 """ select id,t_array_string_ending_with_nulls from ${table_name} order by id """ + qt_test_154 """ select id,t_map_char from ${table_name} order by id """ + + + + + sql """drop catalog ${mc_catalog_name};""" + } +} + diff --git a/regression-test/suites/external_table_p0/mc/test_max_compute_complex_type.groovy b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_complex_type.groovy similarity index 92% rename from regression-test/suites/external_table_p0/mc/test_max_compute_complex_type.groovy rename to regression-test/suites/external_table_p2/maxcompute/test_max_compute_complex_type.groovy index a3de3715f91fc4..4f4748099bb2a2 100644 --- a/regression-test/suites/external_table_p0/mc/test_max_compute_complex_type.groovy +++ b/regression-test/suites/external_table_p2/maxcompute/test_max_compute_complex_type.groovy @@ -26,39 +26,48 @@ arr5 ARRAY ); INSERT INTO array_table VALUES(1, array(1, 2, 3), array('a', 'b', 'c'), array(1.2, 1.3), array(date('2023-05-23')), array(datetime('2023-05-23 13:55:12'))); + INSERT INTO array_table VALUES(1, array(1, 2, 3), array('a', 'b', 'c'), array(1.2, 1.3), array(date('2023-05-23')), array(datetime('2023-05-23 13:55:12'))); INSERT INTO array_table VALUES(2, array(1, 2, 3), array('a', 'b', 'c'), array(1.2, 1.3), array(date('2023-05-23')), array(datetime('2023-05-23 13:55:12'))); INSERT INTO array_table VALUES(3, array(1, 2, 3), array('a', 'b', 'c'), array(1.3), array(date('2023-05-23')), array(datetime('2023-05-23 13:55:12'))); + drop table map_table; create table map_table ( + id int. arr1 MAP, arr2 MAP ); INSERT INTO map_table (arr1, arr2) VALUES ( + 1, MAP(1, 2.5, 2, 3.75), MAP(1, 'example1', 2, 'example2') ); INSERT INTO map_table (arr1, arr2) VALUES ( + 2, MAP(3, 2.5, 99, 3.75), MAP(349, 'asd', 324, 'uid') ); - + drop table struct_table; create table struct_table ( + id int, user_info STRUCT, contact_info STRUCT ); INSERT INTO struct_table VALUES ( + 1, named_struct('id', 'user1', 'age', 25), named_struct('phone_number', 123450, 'email', 'user1@example.com', 'addr', 'Addr1') ), ( + 2, named_struct('id', 'user2', 'age', 30), named_struct('phone_number', 2345671, 'email', 'user2@example.com', 'addr', 'Addr2') ), ( + 3, named_struct('id', 'user3', 'age', 35), named_struct('phone_number', 3456789, 'email', 'user3@example.com', 'addr', 'Addr3') ); @@ -142,7 +151,7 @@ ) ); */ -suite("test_max_compute_complex_type", "p0,external,doris,external_docker,external_docker_doris") { +suite("test_max_compute_complex_type", "p2,external,maxcompute,external_remote,external_remote_maxcompute") { String enabled = context.config.otherConfigs.get("enableMaxComputeTest") if (enabled != null && enabled.equalsIgnoreCase("true")) { String ak = context.config.otherConfigs.get("aliYunAk") @@ -152,18 +161,17 @@ suite("test_max_compute_complex_type", "p0,external,doris,external_docker,extern sql """ CREATE CATALOG IF NOT EXISTS ${mc_catalog_name} PROPERTIES ( "type" = "max_compute", - "mc.default.project" = "mc_datalake", - "mc.region" = "cn-beijing", + "mc.default.project" = "jz_datalake", "mc.access_key" = "${ak}", "mc.secret_key" = "${sk}", - "mc.public_access" = "true" + "mc.endpoint" = "http://service.cn-beijing-vpc.maxcompute.aliyun-inc.com/api" ); """ logger.info("catalog " + mc_catalog_name + " created") sql """switch ${mc_catalog_name};""" logger.info("switched to catalog " + mc_catalog_name) - sql """ use mc_datalake """ + sql """ use jz_datalake """ qt_mc_q1 """ select id,arr3,arr1,arr5,arr2 from array_table order by id desc """ qt_mc_q2 """ select arr2,arr1 from map_table order by id limit 2 """