From bd3d77299a5ac72fc2949f5fd4b21f58361380ce Mon Sep 17 00:00:00 2001 From: lik40 Date: Thu, 27 Jun 2024 03:10:03 +0800 Subject: [PATCH 1/4] [feature](hive/iceberg) optimize hive write commiter statistics and icerberg insert overwrite --- .../doris/datasource/hive/HMSTransaction.java | 381 ++++---- .../datasource/hive/HiveCommonStatistics.java | 44 - .../datasource/hive/HiveMetadataOps.java | 19 +- .../doris/datasource/hive/HivePartition.java | 51 +- .../hive/HivePartitionStatistics.java | 69 +- .../doris/datasource/hive/HiveUtil.java | 21 +- .../iceberg/IcebergTransaction.java | 70 +- .../statistics/CommonStatistics.java | 4 + .../commands/insert/HiveInsertExecutor.java | 7 +- .../doris/datasource/TestHMSCachedClient.java | 10 +- .../doris/datasource/hive/HmsCommitTest.java | 29 +- .../test_iceberg_insert_overwrite.groovy | 856 ++++++++++++++++++ 12 files changed, 1136 insertions(+), 425 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveCommonStatistics.java create mode 100644 regression-test/suites/external_table_p0/iceberg/write/test_iceberg_insert_overwrite.groovy diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java index fe115e4521cd0e..a6bb64c48a3a95 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java @@ -24,7 +24,9 @@ import org.apache.doris.backup.Status; import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; +import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.profile.SummaryProfile; +import org.apache.doris.datasource.statistics.CommonStatistics; import org.apache.doris.fs.FileSystem; import org.apache.doris.fs.FileSystemProvider; import org.apache.doris.fs.FileSystemUtil; @@ -83,13 +85,11 @@ public class HMSTransaction implements Transaction { private static final Logger LOG = LogManager.getLogger(HMSTransaction.class); private final HiveMetadataOps hiveOps; private final FileSystem fs; - private String dbName; - private String tbName; private Optional summaryProfile = Optional.empty(); private String queryId; - private final Map> tableActions = new HashMap<>(); - private final Map, Action>> + private final Map> tableActions = new HashMap<>(); + private final Map, Action>> partitionActions = new HashMap<>(); private final Map> tableColumns = new HashMap<>(); @@ -146,14 +146,6 @@ public void commit() { doCommit(); } - public String getDbName() { - return dbName; - } - - public String getTbName() { - return tbName; - } - public List mergePartitions(List hivePUs) { Map mm = new HashMap<>(); for (THivePartitionUpdate pu : hivePUs) { @@ -190,9 +182,7 @@ public void beginInsertTable(HiveInsertCommandContext ctx) { queryId = ctx.getQueryId(); } - public void finishInsertTable(String dbName, String tbName) { - this.dbName = dbName; - this.tbName = tbName; + public void finishInsertTable(SimpleTableInfo tableInfo) { List mergedPUs = mergePartitions(hivePartitionUpdates); for (THivePartitionUpdate pu : mergedPUs) { if (pu.getS3MpuPendingUploads() != null) { @@ -202,7 +192,7 @@ public void finishInsertTable(String dbName, String tbName) { } } } - Table table = getTable(dbName, tbName); + Table table = getTable(tableInfo); List> insertExistsPartitions = new ArrayList<>(); for (THivePartitionUpdate pu : mergedPUs) { TUpdateMode updateMode = pu.getUpdateMode(); @@ -218,16 +208,15 @@ public void finishInsertTable(String dbName, String tbName) { case APPEND: finishChangingExistingTable( ActionType.INSERT_EXISTING, - dbName, - tbName, + tableInfo, writePath, pu.getFileNames(), hivePartitionStatistics, pu); break; case OVERWRITE: - dropTable(dbName, tbName); - createTable(table, writePath, pu.getFileNames(), hivePartitionStatistics, pu); + dropTable(tableInfo); + createTable(tableInfo, table, writePath, pu.getFileNames(), hivePartitionStatistics, pu); break; default: throw new RuntimeException("Not support mode:[" + updateMode + "] in unPartitioned table"); @@ -242,8 +231,7 @@ public void finishInsertTable(String dbName, String tbName) { case OVERWRITE: StorageDescriptor sd = table.getSd(); HivePartition hivePartition = new HivePartition( - dbName, - tbName, + tableInfo, false, sd.getInputFormat(), pu.getLocation().getTargetPath(), @@ -251,13 +239,13 @@ public void finishInsertTable(String dbName, String tbName) { Maps.newHashMap(), sd.getOutputFormat(), sd.getSerdeInfo().getSerializationLib(), - getTableColumns(dbName, tbName) + getTableColumns(tableInfo.getDbName(), tableInfo.getTbName()) ); if (updateMode == TUpdateMode.OVERWRITE) { - dropPartition(dbName, tbName, hivePartition.getPartitionValues(), true); + dropPartition(tableInfo, hivePartition.getPartitionValues(), true); } addPartition( - dbName, tbName, hivePartition, writePath, + tableInfo, hivePartition, writePath, pu.getName(), pu.getFileNames(), hivePartitionStatistics, pu); break; default: @@ -267,7 +255,7 @@ public void finishInsertTable(String dbName, String tbName) { } if (!insertExistsPartitions.isEmpty()) { - convertToInsertExistingPartitionAction(insertExistsPartitions); + convertToInsertExistingPartitionAction(tableInfo, insertExistsPartitions); } } @@ -275,45 +263,47 @@ public void doCommit() { hmsCommitter = new HmsCommitter(); try { - for (Map.Entry> entry : tableActions.entrySet()) { + for (Map.Entry> entry : tableActions.entrySet()) { + SimpleTableInfo tableInfo = entry.getKey(); Action action = entry.getValue(); switch (action.getType()) { case INSERT_EXISTING: - hmsCommitter.prepareInsertExistingTable(action.getData()); + hmsCommitter.prepareInsertExistingTable(tableInfo, action.getData()); break; case ALTER: - hmsCommitter.prepareAlterTable(action.getData()); + hmsCommitter.prepareAlterTable(tableInfo, action.getData()); break; default: throw new UnsupportedOperationException("Unsupported table action type: " + action.getType()); } } - for (Map.Entry, Action>> tableEntry + for (Map.Entry, Action>> tableEntry : partitionActions.entrySet()) { + SimpleTableInfo tableInfo = tableEntry.getKey(); for (Map.Entry, Action> partitionEntry : tableEntry.getValue().entrySet()) { Action action = partitionEntry.getValue(); switch (action.getType()) { case INSERT_EXISTING: - hmsCommitter.prepareInsertExistPartition(action.getData()); + hmsCommitter.prepareInsertExistPartition(tableInfo, action.getData()); break; case ADD: - hmsCommitter.prepareAddPartition(action.getData()); + hmsCommitter.prepareAddPartition(tableInfo, action.getData()); break; case ALTER: - hmsCommitter.prepareAlterPartition(action.getData()); + hmsCommitter.prepareAlterPartition(tableInfo, action.getData()); break; default: throw new UnsupportedOperationException( - "Unsupported partition action type: " + action.getType()); + "Unsupported partition action type: " + action.getType()); } } } hmsCommitter.doCommit(); } catch (Throwable t) { - LOG.warn("Failed to commit for {}.{}, abort it.", dbName, tbName); + LOG.warn("Failed to commit for {}, abort it.", queryId); try { hmsCommitter.abort(); hmsCommitter.rollback(); @@ -343,10 +333,10 @@ public long getUpdateCnt() { } private void convertToInsertExistingPartitionAction( + SimpleTableInfo tableInfo, List> partitions) { - DatabaseTableName databaseTableName = new DatabaseTableName(dbName, tbName); Map, Action> partitionActionsForTable = - partitionActions.computeIfAbsent(databaseTableName, k -> new HashMap<>()); + partitionActions.computeIfAbsent(tableInfo, k -> new HashMap<>()); for (List> partitionBatch : Iterables.partition(partitions, 100)) { @@ -364,7 +354,7 @@ private void convertToInsertExistingPartitionAction( case DROP_PRESERVE_DATA: throw new RuntimeException( "Not found partition from partition actions" - + "for " + databaseTableName + ", partitions: " + partitionNames); + + "for " + tableInfo + ", partitions: " + partitionNames); case ADD: case ALTER: case INSERT_EXISTING: @@ -379,7 +369,7 @@ private void convertToInsertExistingPartitionAction( Map partitionsByNamesMap = HiveUtil.convertToNamePartitionMap( partitionNames, - hiveOps.getClient().getPartitions(dbName, tbName, partitionNames)); + hiveOps.getClient().getPartitions(tableInfo.getDbName(), tableInfo.getTbName(), partitionNames)); for (int i = 0; i < partitionsByNamesMap.size(); i++) { String partitionName = partitionNames.get(i); @@ -388,7 +378,7 @@ private void convertToInsertExistingPartitionAction( if (partition == null) { // Prevent this partition from being deleted by other engines throw new RuntimeException( - "Not found partition from hms for " + databaseTableName + "Not found partition from hms for " + tableInfo + ", partitions: " + partitionNames); } THivePartitionUpdate pu = partitionBatch.get(i).first; @@ -398,8 +388,7 @@ private void convertToInsertExistingPartitionAction( List partitionValues = HiveUtil.toPartitionValues(pu.getName()); HivePartition hivePartition = new HivePartition( - dbName, - tbName, + tableInfo, false, sd.getInputFormat(), partition.getSd().getLocation(), @@ -407,7 +396,7 @@ private void convertToInsertExistingPartitionAction( partition.getParameters(), sd.getOutputFormat(), sd.getSerdeInfo().getSerializationLib(), - getTableColumns(dbName, tbName) + getTableColumns(tableInfo.getDbName(), tableInfo.getTbName() ); partitionActionsForTable.put( @@ -415,12 +404,12 @@ private void convertToInsertExistingPartitionAction( new Action<>( ActionType.INSERT_EXISTING, new PartitionAndMore( - hivePartition, - pu.getLocation().getWritePath(), - pu.getName(), - pu.getFileNames(), - updateStats, - pu + hivePartition, + pu.getLocation().getWritePath(), + pu.getName(), + pu.getFileNames(), + updateStats, + pu )) ); } @@ -440,18 +429,16 @@ private static void addSuppressedExceptions( } public static class UpdateStatisticsTask { - private final String dbName; - private final String tableName; + private final SimpleTableInfo tableInfo; private final Optional partitionName; private final HivePartitionStatistics updatePartitionStat; private final boolean merge; private boolean done; - public UpdateStatisticsTask(String dbName, String tableName, Optional partitionName, - HivePartitionStatistics statistics, boolean merge) { - this.dbName = Objects.requireNonNull(dbName, "dbName is null"); - this.tableName = Objects.requireNonNull(tableName, "tableName is null"); + public UpdateStatisticsTask(SimpleTableInfo tableInfo, Optional partitionName, + HivePartitionStatistics statistics, boolean merge) { + this.tableInfo = Objects.requireNonNull(tableInfo, "tableInfo is null"); this.partitionName = Objects.requireNonNull(partitionName, "partitionName is null"); this.updatePartitionStat = Objects.requireNonNull(statistics, "statistics is null"); this.merge = merge; @@ -459,9 +446,9 @@ public UpdateStatisticsTask(String dbName, String tableName, Optional pa public void run(HiveMetadataOps hiveOps) { if (partitionName.isPresent()) { - hiveOps.updatePartitionStatistics(dbName, tableName, partitionName.get(), this::updateStatistics); + hiveOps.updatePartitionStatistics(tableInfo, partitionName.get(), this::updateStatistics); } else { - hiveOps.updateTableStatistics(dbName, tableName, this::updateStatistics); + hiveOps.updateTableStatistics(tableInfo, this::updateStatistics); } done = true; } @@ -471,17 +458,17 @@ public void undo(HiveMetadataOps hmsOps) { return; } if (partitionName.isPresent()) { - hmsOps.updatePartitionStatistics(dbName, tableName, partitionName.get(), this::resetStatistics); + hmsOps.updatePartitionStatistics(tableInfo, partitionName.get(), this::resetStatistics); } else { - hmsOps.updateTableStatistics(dbName, tableName, this::resetStatistics); + hmsOps.updateTableStatistics(tableInfo, this::resetStatistics); } } public String getDescription() { if (partitionName.isPresent()) { - return "alter partition parameters " + tableName + " " + partitionName.get(); + return "alter partition parameters " + tableInfo + " " + partitionName.get(); } else { - return "alter table parameters " + tableName; + return "alter table parameters " + tableInfo; } } @@ -491,7 +478,7 @@ private HivePartitionStatistics updateStatistics(HivePartitionStatistics current private HivePartitionStatistics resetStatistics(HivePartitionStatistics currentStatistics) { return HivePartitionStatistics - .reduce(currentStatistics, updatePartitionStat, HivePartitionStatistics.ReduceOperator.SUBTRACT); + .reduce(currentStatistics, updatePartitionStat, CommonStatistics.ReduceOperator.SUBTRACT); } } @@ -513,12 +500,11 @@ public void addPartition(HivePartitionWithStatistics partition) { public void run(HiveMetadataOps hiveOps) { HivePartition firstPartition = partitions.get(0).getPartition(); - String dbName = firstPartition.getDbName(); - String tableName = firstPartition.getTblName(); + SimpleTableInfo tableInfo = firstPartition.getTableInfo(); List> batchedPartitions = Lists.partition(partitions, 20); for (List batch : batchedPartitions) { try { - hiveOps.addPartitions(dbName, tableName, batch); + hiveOps.addPartitions(tableInfo, batch); for (HivePartitionWithStatistics partition : batch) { createdPartitionValues.add(partition.getPartition().getPartitionValues()); } @@ -531,15 +517,14 @@ public void run(HiveMetadataOps hiveOps) { public List> rollback(HiveMetadataOps hiveOps) { HivePartition firstPartition = partitions.get(0).getPartition(); - String dbName = firstPartition.getDbName(); - String tableName = firstPartition.getTblName(); + SimpleTableInfo tableInfo = firstPartition.getTableInfo(); List> rollbackFailedPartitions = new ArrayList<>(); for (List createdPartitionValue : createdPartitionValues) { try { - hiveOps.dropPartition(dbName, tableName, createdPartitionValue, false); + hiveOps.dropPartition(tableInfo, createdPartitionValue, false); } catch (Throwable t) { - LOG.warn("Failed to drop partition on {}.{}.{} when rollback", - dbName, tableName, rollbackFailedPartitions); + LOG.warn("Failed to drop partition on {}.{} when rollback", + tableInfo, rollbackFailedPartitions); rollbackFailedPartitions.add(createdPartitionValue); } } @@ -567,9 +552,9 @@ public boolean isDeleteEmptyDir() { @Override public String toString() { return new StringJoiner(", ", DirectoryCleanUpTask.class.getSimpleName() + "[", "]") - .add("path=" + path) - .add("deleteEmptyDir=" + deleteEmptyDir) - .toString(); + .add("path=" + path) + .add("deleteEmptyDir=" + deleteEmptyDir) + .toString(); } } @@ -611,14 +596,13 @@ public String getRenameTo() { @Override public String toString() { return new StringJoiner(", ", RenameDirectoryTask.class.getSimpleName() + "[", "]") - .add("renameFrom:" + renameFrom) - .add("renameTo:" + renameTo) - .toString(); + .add("renameFrom:" + renameFrom) + .add("renameTo:" + renameTo) + .toString(); } } - private void recursiveDeleteItems(Path directory, boolean deleteEmptyDir, boolean reverse) { DeleteRecursivelyResult deleteResult = recursiveDeleteFiles(directory, deleteEmptyDir, reverse); @@ -706,48 +690,6 @@ public boolean deleteDirectoryIfExists(Path path) { return !fs.directoryExists(path.toString()).ok(); } - public static class DatabaseTableName { - private final String dbName; - private final String tbName; - - public DatabaseTableName(String dbName, String tbName) { - this.dbName = dbName; - this.tbName = tbName; - } - - @Override - public boolean equals(Object other) { - if (this == other) { - return true; - } - - if (other == null || getClass() != other.getClass()) { - return false; - } - - DatabaseTableName that = (DatabaseTableName) other; - return Objects.equals(dbName, that.dbName) && Objects.equals(tbName, that.tbName); - } - - @Override - public String toString() { - return dbName + "." + tbName; - } - - @Override - public int hashCode() { - return Objects.hash(dbName, tbName); - } - - public String getTbName() { - return tbName; - } - - public String getDbName() { - return dbName; - } - } - private static class TableAndMore { private final Table table; private final String currentLocation; @@ -792,9 +734,9 @@ public THivePartitionUpdate getHivePartitionUpdate() { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("table", table) - .add("statisticsUpdate", statisticsUpdate) - .toString(); + .add("table", table) + .add("statisticsUpdate", statisticsUpdate) + .toString(); } } @@ -850,10 +792,10 @@ public THivePartitionUpdate getHivePartitionUpdate() { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("partition", partition) - .add("currentLocation", currentLocation) - .add("fileNames", fileNames) - .toString(); + .add("partition", partition) + .add("currentLocation", currentLocation) + .add("fileNames", fileNames) + .toString(); } } @@ -898,16 +840,16 @@ public T getData() { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("type", type) - .add("data", data) - .toString(); + .add("type", type) + .add("data", data) + .toString(); } } - public synchronized Table getTable(String databaseName, String tableName) { - Action tableAction = tableActions.get(new DatabaseTableName(databaseName, tableName)); + public synchronized Table getTable(SimpleTableInfo tableInfo) { + Action tableAction = tableActions.get(tableInfo); if (tableAction == null) { - return hiveOps.getClient().getTable(databaseName, tableName); + return hiveOps.getClient().getTable(tableInfo.getDbName(), tableInfo.getTbName()); } switch (tableAction.getType()) { case ADD: @@ -921,7 +863,7 @@ public synchronized Table getTable(String databaseName, String tableName) { default: throw new IllegalStateException("Unknown action type: " + tableAction.getType()); } - throw new RuntimeException("Not Found table: " + databaseName + "." + tableName); + throw new RuntimeException("Not Found table: " + tableInfo); } public synchronized List getTableColumns(String databaseName, String tableName) { @@ -931,18 +873,16 @@ public synchronized List getTableColumns(String databaseName, Strin public synchronized void finishChangingExistingTable( ActionType actionType, - String databaseName, - String tableName, + SimpleTableInfo tableInfo, String location, List fileNames, HivePartitionStatistics statisticsUpdate, THivePartitionUpdate hivePartitionUpdate) { - DatabaseTableName databaseTableName = new DatabaseTableName(databaseName, tableName); - Action oldTableAction = tableActions.get(databaseTableName); + Action oldTableAction = tableActions.get(tableInfo); if (oldTableAction == null) { - Table table = hiveOps.getClient().getTable(databaseTableName.getDbName(), databaseTableName.getTbName()); + Table table = hiveOps.getClient().getTable(tableInfo.getDbName(), tableInfo.getTbName()); tableActions.put( - databaseTableName, + tableInfo, new Action<>( actionType, new TableAndMore( @@ -956,7 +896,7 @@ public synchronized void finishChangingExistingTable( switch (oldTableAction.getType()) { case DROP: - throw new RuntimeException("Not found table: " + databaseTableName); + throw new RuntimeException("Not found table: " + tableInfo); case ADD: case ALTER: case INSERT_EXISTING: @@ -972,27 +912,28 @@ public synchronized void finishChangingExistingTable( } public synchronized void createTable( - Table table, String location, List fileNames, HivePartitionStatistics statistics, + SimpleTableInfo tableInfo, + Table table, String location, List fileNames, + HivePartitionStatistics statistics, THivePartitionUpdate hivePartitionUpdate) { // When creating a table, it should never have partition actions. This is just a sanity check. - checkNoPartitionAction(dbName, tbName); - DatabaseTableName databaseTableName = new DatabaseTableName(dbName, tbName); - Action oldTableAction = tableActions.get(databaseTableName); + checkNoPartitionAction(tableInfo); + Action oldTableAction = tableActions.get(tableInfo); TableAndMore tableAndMore = new TableAndMore(table, location, fileNames, statistics, hivePartitionUpdate); if (oldTableAction == null) { - tableActions.put(databaseTableName, new Action<>(ActionType.ADD, tableAndMore)); + tableActions.put(tableInfo, new Action<>(ActionType.ADD, tableAndMore)); return; } switch (oldTableAction.getType()) { case DROP: - tableActions.put(databaseTableName, new Action<>(ActionType.ALTER, tableAndMore)); + tableActions.put(tableInfo, new Action<>(ActionType.ALTER, tableAndMore)); return; case ADD: case ALTER: case INSERT_EXISTING: case MERGE: - throw new RuntimeException("Table already exists: " + databaseTableName); + throw new RuntimeException("Table already exists: " + tableInfo); case DROP_PRESERVE_DATA: break; default: @@ -1001,18 +942,17 @@ public synchronized void createTable( } - public synchronized void dropTable(String databaseName, String tableName) { + public synchronized void dropTable(SimpleTableInfo tableInfo) { // Dropping table with partition actions requires cleaning up staging data, which is not implemented yet. - checkNoPartitionAction(databaseName, tableName); - DatabaseTableName databaseTableName = new DatabaseTableName(databaseName, tableName); - Action oldTableAction = tableActions.get(databaseTableName); + checkNoPartitionAction(tableInfo); + Action oldTableAction = tableActions.get(tableInfo); if (oldTableAction == null || oldTableAction.getType() == ActionType.ALTER) { - tableActions.put(databaseTableName, new Action<>(ActionType.DROP, null)); + tableActions.put(tableInfo, new Action<>(ActionType.DROP, null)); return; } switch (oldTableAction.getType()) { case DROP: - throw new RuntimeException("Not found table: " + databaseTableName); + throw new RuntimeException("Not found table: " + tableInfo); case ADD: case ALTER: case INSERT_EXISTING: @@ -1026,9 +966,9 @@ public synchronized void dropTable(String databaseName, String tableName) { } - private void checkNoPartitionAction(String databaseName, String tableName) { + private void checkNoPartitionAction(SimpleTableInfo tableInfo) { Map, Action> partitionActionsForTable = - partitionActions.get(new DatabaseTableName(databaseName, tableName)); + partitionActions.get(tableInfo); if (partitionActionsForTable != null && !partitionActionsForTable.isEmpty()) { throw new RuntimeException( "Cannot make schema changes to a table with modified partitions in the same transaction"); @@ -1036,8 +976,7 @@ private void checkNoPartitionAction(String databaseName, String tableName) { } public synchronized void addPartition( - String databaseName, - String tableName, + SimpleTableInfo tableInfo, HivePartition partition, String currentLocation, String partitionName, @@ -1045,7 +984,7 @@ public synchronized void addPartition( HivePartitionStatistics statistics, THivePartitionUpdate hivePartitionUpdate) { Map, Action> partitionActionsForTable = - partitionActions.computeIfAbsent(new DatabaseTableName(databaseName, tableName), k -> new HashMap<>()); + partitionActions.computeIfAbsent(tableInfo, k -> new HashMap<>()); Action oldPartitionAction = partitionActionsForTable.get(partition.getPartitionValues()); if (oldPartitionAction == null) { partitionActionsForTable.put( @@ -1073,21 +1012,20 @@ public synchronized void addPartition( case INSERT_EXISTING: case MERGE: throw new RuntimeException( - "Partition already exists for table: " - + databaseName + "." + tableName + ", partition values: " + partition.getPartitionValues()); + "Partition already exists for table: " + + tableInfo + ", partition values: " + partition + .getPartitionValues()); default: throw new IllegalStateException("Unknown action type: " + oldPartitionAction.getType()); } } public synchronized void dropPartition( - String databaseName, - String tableName, + SimpleTableInfo tableInfo, List partitionValues, boolean deleteData) { - DatabaseTableName databaseTableName = new DatabaseTableName(databaseName, tableName); Map, Action> partitionActionsForTable = - partitionActions.computeIfAbsent(databaseTableName, k -> new HashMap<>()); + partitionActions.computeIfAbsent(tableInfo, k -> new HashMap<>()); Action oldPartitionAction = partitionActionsForTable.get(partitionValues); if (oldPartitionAction == null) { if (deleteData) { @@ -1101,7 +1039,7 @@ public synchronized void dropPartition( case DROP: case DROP_PRESERVE_DATA: throw new RuntimeException( - "Not found partition from partition actions for " + databaseTableName + "Not found partition from partition actions for " + tableInfo + ", partitions: " + partitionValues); case ADD: case ALTER: @@ -1109,7 +1047,7 @@ public synchronized void dropPartition( case MERGE: throw new RuntimeException( "Dropping a partition added in the same transaction is not supported: " - + databaseTableName + ", partition values: " + partitionValues); + + tableInfo + ", partition values: " + partitionValues); default: throw new IllegalStateException("Unknown action type: " + oldPartitionAction.getType()); } @@ -1165,12 +1103,11 @@ private void undoAddPartitionsTask() { } HivePartition firstPartition = addPartitionsTask.getPartitions().get(0).getPartition(); - String dbName = firstPartition.getDbName(); - String tableName = firstPartition.getTblName(); + SimpleTableInfo tableInfo = firstPartition.getTableInfo(); List> rollbackFailedPartitions = addPartitionsTask.rollback(hiveOps); if (!rollbackFailedPartitions.isEmpty()) { - LOG.warn("Failed to rollback: add_partition for partition values {}.{}.{}", - dbName, tableName, rollbackFailedPartitions); + LOG.warn("Failed to rollback: add_partition for partition values {}.{}", + tableInfo, rollbackFailedPartitions); } } @@ -1186,7 +1123,7 @@ private void waitForAsyncFileSystemTaskSuppressThrowable() { } } - public void prepareInsertExistingTable(TableAndMore tableAndMore) { + public void prepareInsertExistingTable(SimpleTableInfo tableInfo, TableAndMore tableAndMore) { Table table = tableAndMore.getTable(); String targetPath = table.getSd().getLocation(); String writePath = tableAndMore.getCurrentLocation(); @@ -1207,15 +1144,14 @@ public void prepareInsertExistingTable(TableAndMore tableAndMore) { directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, false)); updateStatisticsTasks.add( new UpdateStatisticsTask( - dbName, - tbName, + tableInfo, Optional.empty(), tableAndMore.getStatisticsUpdate(), true )); } - public void prepareAlterTable(TableAndMore tableAndMore) { + public void prepareAlterTable(SimpleTableInfo tableInfo, TableAndMore tableAndMore) { Table table = tableAndMore.getTable(); String targetPath = table.getSd().getLocation(); String writePath = tableAndMore.getCurrentLocation(); @@ -1229,18 +1165,18 @@ public void prepareAlterTable(TableAndMore tableAndMore) { () -> renameDirectoryTasksForAbort.add(new RenameDirectoryTask(oldTablePath, targetPath))); if (!status.ok()) { throw new RuntimeException( - "Error to rename dir from " + targetPath + " to " + oldTablePath + status.getErrMsg()); + "Error to rename dir from " + targetPath + " to " + oldTablePath + status.getErrMsg()); } clearDirsForFinish.add(oldTablePath); - status = wrapperRenameDirWithProfileSummary( + status = wrapperRenameDirWithProfileSummary( writePath, targetPath, () -> directoryCleanUpTasksForAbort.add( new DirectoryCleanUpTask(targetPath, true))); if (!status.ok()) { throw new RuntimeException( - "Error to rename dir from " + writePath + " to " + targetPath + ":" + status.getErrMsg()); + "Error to rename dir from " + writePath + " to " + targetPath + ":" + status.getErrMsg()); } } else { if (!tableAndMore.hivePartitionUpdate.s3_mpu_pending_uploads.isEmpty()) { @@ -1250,16 +1186,15 @@ public void prepareAlterTable(TableAndMore tableAndMore) { } } updateStatisticsTasks.add( - new UpdateStatisticsTask( - dbName, - tbName, - Optional.empty(), - tableAndMore.getStatisticsUpdate(), - false - )); + new UpdateStatisticsTask( + tableInfo, + Optional.empty(), + tableAndMore.getStatisticsUpdate(), + false + )); } - public void prepareAddPartition(PartitionAndMore partitionAndMore) { + public void prepareAddPartition(SimpleTableInfo tableInfo, PartitionAndMore partitionAndMore) { HivePartition partition = partitionAndMore.getPartition(); String targetPath = partition.getPath(); @@ -1280,11 +1215,10 @@ public void prepareAddPartition(PartitionAndMore partitionAndMore) { } } - StorageDescriptor sd = getTable(dbName, tbName).getSd(); + StorageDescriptor sd = getTable(tableInfo).getSd(); HivePartition hivePartition = new HivePartition( - dbName, - tbName, + tableInfo, false, sd.getInputFormat(), targetPath, @@ -1292,7 +1226,7 @@ public void prepareAddPartition(PartitionAndMore partitionAndMore) { Maps.newHashMap(), sd.getOutputFormat(), sd.getSerdeInfo().getSerializationLib(), - getTableColumns(dbName, tbName) + getTableColumns(tableInfo.getDbName(), tableInfo.getTbName()) ); HivePartitionWithStatistics partitionWithStats = @@ -1303,7 +1237,7 @@ public void prepareAddPartition(PartitionAndMore partitionAndMore) { addPartitionsTask.addPartition(partitionWithStats); } - public void prepareInsertExistPartition(PartitionAndMore partitionAndMore) { + public void prepareInsertExistPartition(SimpleTableInfo tableInfo, PartitionAndMore partitionAndMore) { HivePartition partition = partitionAndMore.getPartition(); String targetPath = partition.getPath(); @@ -1326,12 +1260,11 @@ public void prepareInsertExistPartition(PartitionAndMore partitionAndMore) { } updateStatisticsTasks.add( - new UpdateStatisticsTask( - dbName, - tbName, - Optional.of(partitionAndMore.getPartitionName()), - partitionAndMore.getStatisticsUpdate(), - true)); + new UpdateStatisticsTask( + tableInfo, + Optional.of(partitionAndMore.getPartitionName()), + partitionAndMore.getStatisticsUpdate(), + true)); } private void runDirectoryClearUpTasksForAbort() { @@ -1345,7 +1278,8 @@ private void runRenameDirTasksForAbort() { for (RenameDirectoryTask task : renameDirectoryTasksForAbort) { status = fs.exists(task.getRenameFrom()); if (status.ok()) { - status = wrapperRenameDirWithProfileSummary(task.getRenameFrom(), task.getRenameTo(), () -> {}); + status = wrapperRenameDirWithProfileSummary(task.getRenameFrom(), task.getRenameTo(), () -> { + }); if (!status.ok()) { LOG.warn("Failed to abort rename dir from {} to {}:{}", task.getRenameFrom(), task.getRenameTo(), status.getErrMsg()); @@ -1370,7 +1304,7 @@ private void runS3cleanWhenSuccess() { } } - public void prepareAlterPartition(PartitionAndMore partitionAndMore) { + public void prepareAlterPartition(SimpleTableInfo tableInfo, PartitionAndMore partitionAndMore) { HivePartition partition = partitionAndMore.getPartition(); String targetPath = partition.getPath(); String writePath = partitionAndMore.getCurrentLocation(); @@ -1385,19 +1319,19 @@ public void prepareAlterPartition(PartitionAndMore partitionAndMore) { () -> renameDirectoryTasksForAbort.add(new RenameDirectoryTask(oldPartitionPath, targetPath))); if (!status.ok()) { throw new RuntimeException( - "Error to rename dir " - + "from " + targetPath - + " to " + oldPartitionPath + ":" + status.getErrMsg()); + "Error to rename dir " + + "from " + targetPath + + " to " + oldPartitionPath + ":" + status.getErrMsg()); } clearDirsForFinish.add(oldPartitionPath); status = wrapperRenameDirWithProfileSummary( - writePath, - targetPath, - () -> directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true))); + writePath, + targetPath, + () -> directoryCleanUpTasksForAbort.add(new DirectoryCleanUpTask(targetPath, true))); if (!status.ok()) { throw new RuntimeException( - "Error to rename dir from " + writePath + " to " + targetPath + ":" + status.getErrMsg()); + "Error to rename dir from " + writePath + " to " + targetPath + ":" + status.getErrMsg()); } } else { if (!partitionAndMore.hivePartitionUpdate.s3_mpu_pending_uploads.isEmpty()) { @@ -1408,13 +1342,12 @@ public void prepareAlterPartition(PartitionAndMore partitionAndMore) { } updateStatisticsTasks.add( - new UpdateStatisticsTask( - dbName, - tbName, - Optional.of(partitionAndMore.getPartitionName()), - partitionAndMore.getStatisticsUpdate(), - false - )); + new UpdateStatisticsTask( + tableInfo, + Optional.of(partitionAndMore.getPartitionName()), + partitionAndMore.getStatisticsUpdate(), + false + )); } @@ -1562,8 +1495,8 @@ public void shutdownExecutorService() { } public Status wrapperRenameDirWithProfileSummary(String origFilePath, - String destFilePath, - Runnable runWhenPathNotExist) { + String destFilePath, + Runnable runWhenPathNotExist) { summaryProfile.ifPresent(profile -> { profile.setTempStartTime(); profile.incRenameDirCnt(); @@ -1600,22 +1533,22 @@ public Status wrapperDeleteDirWithProfileSummary(String remotePath) { } public void wrapperAsyncRenameWithProfileSummary(Executor executor, - List> renameFileFutures, - AtomicBoolean cancelled, - String origFilePath, - String destFilePath, - List fileNames) { + List> renameFileFutures, + AtomicBoolean cancelled, + String origFilePath, + String destFilePath, + List fileNames) { FileSystemUtil.asyncRenameFiles( fs, executor, renameFileFutures, cancelled, origFilePath, destFilePath, fileNames); summaryProfile.ifPresent(profile -> profile.addRenameFileCnt(fileNames.size())); } public void wrapperAsyncRenameDirWithProfileSummary(Executor executor, - List> renameFileFutures, - AtomicBoolean cancelled, - String origFilePath, - String destFilePath, - Runnable runWhenPathNotExist) { + List> renameFileFutures, + AtomicBoolean cancelled, + String origFilePath, + String destFilePath, + Runnable runWhenPathNotExist) { FileSystemUtil.asyncRenameDir( fs, executor, renameFileFutures, cancelled, origFilePath, destFilePath, runWhenPathNotExist); summaryProfile.ifPresent(SummaryProfile::incRenameDirCnt); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveCommonStatistics.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveCommonStatistics.java deleted file mode 100644 index 3d8fb2512aaa80..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveCommonStatistics.java +++ /dev/null @@ -1,44 +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.datasource.hive; - -public class HiveCommonStatistics { - public static final HiveCommonStatistics EMPTY = new HiveCommonStatistics(0L, 0L, 0L); - - private final long rowCount; - private final long fileCount; - private final long totalFileBytes; - - public HiveCommonStatistics(long rowCount, long fileCount, long totalFileBytes) { - this.fileCount = fileCount; - this.rowCount = rowCount; - this.totalFileBytes = totalFileBytes; - } - - public long getRowCount() { - return rowCount; - } - - public long getFileCount() { - return fileCount; - } - - public long getTotalFileBytes() { - return totalFileBytes; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java index f438f5e1782f5a..dcfc6d1ad33f90 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveMetadataOps.java @@ -32,6 +32,7 @@ import org.apache.doris.common.ErrorCode; import org.apache.doris.common.ErrorReport; import org.apache.doris.common.UserException; +import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.security.authentication.HadoopAuthenticator; import org.apache.doris.datasource.ExternalDatabase; import org.apache.doris.datasource.jdbc.client.JdbcClient; @@ -297,25 +298,23 @@ public List listDatabaseNames() { } public void updateTableStatistics( - String dbName, - String tableName, + SimpleTableInfo tableInfo, Function update) { - client.updateTableStatistics(dbName, tableName, update); + client.updateTableStatistics(tableInfo.getDbName(), tableInfo.getTbName(), update); } void updatePartitionStatistics( - String dbName, - String tableName, + SimpleTableInfo tableInfo, String partitionName, Function update) { - client.updatePartitionStatistics(dbName, tableName, partitionName, update); + client.updatePartitionStatistics(tableInfo.getDbName(), tableInfo.getTbName(), partitionName, update); } - public void addPartitions(String dbName, String tableName, List partitions) { - client.addPartitions(dbName, tableName, partitions); + public void addPartitions(SimpleTableInfo tableInfo, List partitions) { + client.addPartitions(tableInfo.getDbName(), tableInfo.getTbName(), partitions); } - public void dropPartition(String dbName, String tableName, List partitionValues, boolean deleteData) { - client.dropPartition(dbName, tableName, partitionValues, deleteData); + public void dropPartition(SimpleTableInfo tableInfo, List partitionValues, boolean deleteData) { + client.dropPartition(tableInfo.getDbName(), tableInfo.getTbName(), partitionValues, deleteData); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartition.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartition.java index a9d97b4062891b..a114c45d4ff127 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartition.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartition.java @@ -18,6 +18,7 @@ package org.apache.doris.datasource.hive; import org.apache.doris.catalog.Column; +import org.apache.doris.common.info.SimpleTableInfo; import com.google.common.base.Preconditions; import lombok.Data; @@ -31,8 +32,7 @@ public class HivePartition { public static final String LAST_MODIFY_TIME_KEY = "transient_lastDdlTime"; public static final String FILE_NUM_KEY = "numFiles"; - private String dbName; - private String tblName; + private SimpleTableInfo tableInfo; private String inputFormat; private String path; private List partitionValues; @@ -43,10 +43,9 @@ public class HivePartition { private List columns; // If you want to read the data under a partition, you can use this constructor - public HivePartition(String dbName, String tblName, boolean isDummyPartition, + public HivePartition(SimpleTableInfo tableInfo, boolean isDummyPartition, String inputFormat, String path, List partitionValues, Map parameters) { - this.dbName = dbName; - this.tblName = tblName; + this.tableInfo = tableInfo; this.isDummyPartition = isDummyPartition; // eg: org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat this.inputFormat = inputFormat; @@ -57,17 +56,31 @@ public HivePartition(String dbName, String tblName, boolean isDummyPartition, this.parameters = parameters; } + public HivePartition(String database, String tableName, boolean isDummyPartition, + String inputFormat, String path, List partitionValues, Map parameters) { + this(new SimpleTableInfo(database, tableName), isDummyPartition, inputFormat, path, partitionValues, + parameters); + } + // If you want to update hms with partition, then you can use this constructor, // as updating hms requires some additional information, such as outputFormat and so on - public HivePartition(String dbName, String tblName, boolean isDummyPartition, - String inputFormat, String path, List partitionValues, Map parameters, - String outputFormat, String serde, List columns) { - this(dbName, tblName, isDummyPartition, inputFormat, path, partitionValues, parameters); + public HivePartition(SimpleTableInfo tableInfo, boolean isDummyPartition, + String inputFormat, String path, List partitionValues, Map parameters, + String outputFormat, String serde, List columns) { + this(tableInfo, isDummyPartition, inputFormat, path, partitionValues, parameters); this.outputFormat = outputFormat; this.serde = serde; this.columns = columns; } + public String getDbName() { + return tableInfo.getDbName(); + } + + public String getTblName() { + return tableInfo.getTbName(); + } + // return partition name like: nation=cn/city=beijing public String getPartitionName(List partColumns) { Preconditions.checkState(partColumns.size() == partitionValues.size()); @@ -94,6 +107,7 @@ public long getLastModifiedTime() { /** * If there are no files, it proves that there is no data under the partition, we return 0 + * * @return */ public long getLastModifiedTimeIgnoreInit() { @@ -112,12 +126,17 @@ public long getFileNum() { @Override public String toString() { - return "HivePartition{" - + "dbName='" + dbName + '\'' - + ", tblName='" + tblName + '\'' - + ", isDummyPartition='" + isDummyPartition + '\'' - + ", inputFormat='" + inputFormat + '\'' - + ", path='" + path + '\'' - + ", partitionValues=" + partitionValues + '}'; + final StringBuilder sb = new StringBuilder("HivePartition{"); + sb.append("tableInfo=").append(tableInfo); + sb.append(", inputFormat='").append(inputFormat).append('\''); + sb.append(", path='").append(path).append('\''); + sb.append(", partitionValues=").append(partitionValues); + sb.append(", isDummyPartition=").append(isDummyPartition); + sb.append(", parameters=").append(parameters); + sb.append(", outputFormat='").append(outputFormat).append('\''); + sb.append(", serde='").append(serde).append('\''); + sb.append(", columns=").append(columns); + sb.append('}'); + return sb.toString(); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionStatistics.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionStatistics.java index df13e6737b520a..8173f1dea3a514 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionStatistics.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HivePartitionStatistics.java @@ -20,25 +20,28 @@ package org.apache.doris.datasource.hive; +import org.apache.doris.datasource.statistics.CommonStatistics; +import org.apache.doris.datasource.statistics.CommonStatistics.ReduceOperator; + import com.google.common.collect.ImmutableMap; import java.util.Map; public class HivePartitionStatistics { public static final HivePartitionStatistics EMPTY = - new HivePartitionStatistics(HiveCommonStatistics.EMPTY, ImmutableMap.of()); + new HivePartitionStatistics(CommonStatistics.EMPTY, ImmutableMap.of()); - private final HiveCommonStatistics commonStatistics; + private final CommonStatistics commonStatistics; private final Map columnStatisticsMap; public HivePartitionStatistics( - HiveCommonStatistics commonStatistics, + CommonStatistics commonStatistics, Map columnStatisticsMap) { this.commonStatistics = commonStatistics; this.columnStatisticsMap = columnStatisticsMap; } - public HiveCommonStatistics getCommonStatistics() { + public CommonStatistics getCommonStatistics() { return commonStatistics; } @@ -48,7 +51,7 @@ public Map getColumnStatisticsMap() { public static HivePartitionStatistics fromCommonStatistics(long rowCount, long fileCount, long totalFileBytes) { return new HivePartitionStatistics( - new HiveCommonStatistics(rowCount, fileCount, totalFileBytes), + new CommonStatistics(rowCount, fileCount, totalFileBytes), ImmutableMap.of() ); } @@ -62,56 +65,32 @@ public static HivePartitionStatistics merge(HivePartitionStatistics current, Hiv } return new HivePartitionStatistics( - reduce(current.getCommonStatistics(), update.getCommonStatistics(), ReduceOperator.ADD), - // TODO merge columnStatisticsMap - current.getColumnStatisticsMap()); + CommonStatistics + .reduce(current.getCommonStatistics(), update.getCommonStatistics(), ReduceOperator.ADD), + // TODO merge columnStatisticsMap + current.getColumnStatisticsMap()); } public static HivePartitionStatistics reduce( HivePartitionStatistics first, HivePartitionStatistics second, ReduceOperator operator) { - HiveCommonStatistics left = first.getCommonStatistics(); - HiveCommonStatistics right = second.getCommonStatistics(); + CommonStatistics left = first.getCommonStatistics(); + CommonStatistics right = second.getCommonStatistics(); return HivePartitionStatistics.fromCommonStatistics( - reduce(left.getRowCount(), right.getRowCount(), operator), - reduce(left.getFileCount(), right.getFileCount(), operator), - reduce(left.getTotalFileBytes(), right.getTotalFileBytes(), operator)); + CommonStatistics.reduce(left.getRowCount(), right.getRowCount(), operator), + CommonStatistics.reduce(left.getFileCount(), right.getFileCount(), operator), + CommonStatistics.reduce(left.getTotalFileBytes(), right.getTotalFileBytes(), operator)); } - public static HiveCommonStatistics reduce( - HiveCommonStatistics current, - HiveCommonStatistics update, + public static CommonStatistics reduce( + CommonStatistics current, + CommonStatistics update, ReduceOperator operator) { - return new HiveCommonStatistics( - reduce(current.getRowCount(), update.getRowCount(), operator), - reduce(current.getFileCount(), update.getFileCount(), operator), - reduce(current.getTotalFileBytes(), update.getTotalFileBytes(), operator)); - } - - public static long reduce(long current, long update, ReduceOperator operator) { - if (current >= 0 && update >= 0) { - switch (operator) { - case ADD: - return current + update; - case SUBTRACT: - return current - update; - case MAX: - return Math.max(current, update); - case MIN: - return Math.min(current, update); - default: - throw new IllegalArgumentException("Unexpected operator: " + operator); - } - } - - return 0; + return new CommonStatistics( + CommonStatistics.reduce(current.getRowCount(), update.getRowCount(), operator), + CommonStatistics.reduce(current.getFileCount(), update.getFileCount(), operator), + CommonStatistics.reduce(current.getTotalFileBytes(), update.getTotalFileBytes(), operator)); } - public enum ReduceOperator { - ADD, - SUBTRACT, - MIN, - MAX, - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java index c12bdcfb473e88..dac5d55e5eef09 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HiveUtil.java @@ -21,6 +21,7 @@ import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.datasource.ExternalCatalog; +import org.apache.doris.datasource.statistics.CommonStatistics; import org.apache.doris.fs.remote.BrokerFileSystem; import org.apache.doris.fs.remote.RemoteFileSystem; import org.apache.doris.nereids.exceptions.AnalysisException; @@ -75,14 +76,14 @@ private HiveUtil() { /** * get input format class from inputFormatName. * - * @param jobConf jobConf used when getInputFormatClass + * @param jobConf jobConf used when getInputFormatClass * @param inputFormatName inputFormat class name - * @param symlinkTarget use target inputFormat class when inputFormat is SymlinkTextInputFormat + * @param symlinkTarget use target inputFormat class when inputFormat is SymlinkTextInputFormat * @return a class of inputFormat. * @throws UserException when class not found. */ public static InputFormat getInputFormat(JobConf jobConf, - String inputFormatName, boolean symlinkTarget) throws UserException { + String inputFormatName, boolean symlinkTarget) throws UserException { try { Class> inputFormatClass = getInputFormatClass(jobConf, inputFormatName); if (symlinkTarget && (inputFormatClass == SymlinkTextInputFormat.class)) { @@ -167,12 +168,12 @@ public static Map convertToNamePartitionMap( Map> partitionNameToPartitionValues = partitionNames - .stream() - .collect(Collectors.toMap(partitionName -> partitionName, HiveUtil::toPartitionValues)); + .stream() + .collect(Collectors.toMap(partitionName -> partitionName, HiveUtil::toPartitionValues)); Map, Partition> partitionValuesToPartition = partitions.stream() - .collect(Collectors.toMap(Partition::getValues, partition -> partition)); + .collect(Collectors.toMap(Partition::getValues, partition -> partition)); ImmutableMap.Builder resultBuilder = ImmutableMap.builder(); for (Map.Entry> entry : partitionNameToPartitionValues.entrySet()) { @@ -312,7 +313,7 @@ public static Database toHiveDatabase(HiveDatabaseMetadata hiveDb) { public static Map updateStatisticsParameters( Map parameters, - HiveCommonStatistics statistics) { + CommonStatistics statistics) { HashMap result = new HashMap<>(parameters); result.put(StatsSetupConst.NUM_FILES, String.valueOf(statistics.getFileCount())); @@ -345,8 +346,8 @@ public static Partition toMetastoreApiPartition(HivePartitionWithStatistics part public static Partition toMetastoreApiPartition(HivePartition hivePartition) { Partition result = new Partition(); - result.setDbName(hivePartition.getDbName()); - result.setTableName(hivePartition.getTblName()); + result.setDbName(hivePartition.getTableInfo().getDbName()); + result.setTableName(hivePartition.getTableInfo().getTbName()); result.setValues(hivePartition.getPartitionValues()); result.setSd(makeStorageDescriptorFromHivePartition(hivePartition)); result.setParameters(hivePartition.getParameters()); @@ -355,7 +356,7 @@ public static Partition toMetastoreApiPartition(HivePartition hivePartition) { public static StorageDescriptor makeStorageDescriptorFromHivePartition(HivePartition partition) { SerDeInfo serdeInfo = new SerDeInfo(); - serdeInfo.setName(partition.getTblName()); + serdeInfo.setName(partition.getTableInfo().getTbName()); serdeInfo.setSerializationLib(partition.getSerde()); StorageDescriptor sd = new StorageDescriptor(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java index a3a978ccd7a16e..edc613ad05b549 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/iceberg/IcebergTransaction.java @@ -96,18 +96,10 @@ private void updateManifestAfterInsert(TUpdateMode updateMode) { .convertToWriterResult(fileFormat, spec, commitDataList); List pendingResults = Lists.newArrayList(writeResult); - if (spec.isPartitioned()) { - partitionManifestUpdate(updateMode, table, pendingResults); - if (LOG.isDebugEnabled()) { - LOG.info("{} {} table partition manifest successful and writeResult : {}..", tableInfo, updateMode, - writeResult); - } + if (updateMode == TUpdateMode.APPEND) { + commitAppendTxn(table, pendingResults); } else { - tableManifestUpdate(updateMode, table, pendingResults); - if (LOG.isDebugEnabled()) { - LOG.info("{} {} table manifest successful and writeResult : {}..", tableInfo, updateMode, - writeResult); - } + commitReplaceTxn(table, pendingResults); } } @@ -133,50 +125,8 @@ private synchronized Table getNativeTable(SimpleTableInfo tableInfo) { return IcebergUtils.getRemoteTable(externalCatalog, tableInfo); } - private void partitionManifestUpdate(TUpdateMode updateMode, Table table, List pendingResults) { - if (Objects.isNull(pendingResults) || pendingResults.isEmpty()) { - LOG.warn("{} partitionManifestUp method call but pendingResults is null or empty!", table.name()); - return; - } - // Commit the appendPartitionOperator transaction. - if (updateMode == TUpdateMode.APPEND) { - commitAppendTxn(table, pendingResults); - } else { - ReplacePartitions appendPartitionOp = table.newReplacePartitions(); - for (WriteResult result : pendingResults) { - Preconditions.checkState(result.referencedDataFiles().length == 0, - "Should have no referenced data files."); - Arrays.stream(result.dataFiles()).forEach(appendPartitionOp::addFile); - } - appendPartitionOp.commit(); - } - } - - private void tableManifestUpdate(TUpdateMode updateMode, Table table, List pendingResults) { - if (Objects.isNull(pendingResults) || pendingResults.isEmpty()) { - LOG.warn("{} tableManifestUp method call but pendingResults is null or empty!", table.name()); - return; - } - // Commit the appendPartitionOperator transaction. - if (LOG.isDebugEnabled()) { - LOG.info("{} tableManifestUp method call ", table.name()); - } - if (updateMode == TUpdateMode.APPEND) { - commitAppendTxn(table, pendingResults); - } else { - ReplacePartitions appendPartitionOp = table.newReplacePartitions(); - for (WriteResult result : pendingResults) { - Preconditions.checkState(result.referencedDataFiles().length == 0, - "Should have no referenced data files."); - Arrays.stream(result.dataFiles()).forEach(appendPartitionOp::addFile); - } - appendPartitionOp.commit(); - } - } - - private void commitAppendTxn(Table table, List pendingResults) { - // To be compatible with iceberg format V1. + // commit append files. AppendFiles appendFiles = table.newAppend(); for (WriteResult result : pendingResults) { Preconditions.checkState(result.referencedDataFiles().length == 0, @@ -186,4 +136,16 @@ private void commitAppendTxn(Table table, List pendingResults) { appendFiles.commit(); } + + private void commitReplaceTxn(Table table, List pendingResults) { + // commit replace partitions + ReplacePartitions appendPartitionOp = table.newReplacePartitions(); + for (WriteResult result : pendingResults) { + Preconditions.checkState(result.referencedDataFiles().length == 0, + "Should have no referenced data files."); + Arrays.stream(result.dataFiles()).forEach(appendPartitionOp::addFile); + } + appendPartitionOp.commit(); + } + } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/statistics/CommonStatistics.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/statistics/CommonStatistics.java index 9685dfdf35aa60..0285ad09db4c46 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/statistics/CommonStatistics.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/statistics/CommonStatistics.java @@ -17,6 +17,10 @@ package org.apache.doris.datasource.statistics; +/** + * This class provides operations related to file statistics, including object and field granularity add, min, max + * and other merge operations + */ public class CommonStatistics { public static final CommonStatistics EMPTY = new CommonStatistics(0L, 0L, 0L); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java index 7095187451172a..10ff27add86708 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/HiveInsertExecutor.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.trees.plans.commands.insert; import org.apache.doris.common.UserException; +import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.datasource.hive.HMSExternalTable; import org.apache.doris.datasource.hive.HMSTransaction; @@ -42,8 +43,8 @@ public class HiveInsertExecutor extends BaseExternalTableInsertExecutor { * constructor */ public HiveInsertExecutor(ConnectContext ctx, HMSExternalTable table, - String labelName, NereidsPlanner planner, - Optional insertCtx, boolean emptyInsert) { + String labelName, NereidsPlanner planner, + Optional insertCtx, boolean emptyInsert) { super(ctx, table, labelName, planner, insertCtx, emptyInsert); } @@ -71,7 +72,7 @@ protected void doBeforeCommit() throws UserException { loadedRows = transaction.getUpdateCnt(); String dbName = ((HMSExternalTable) table).getDbName(); String tbName = table.getName(); - transaction.finishInsertTable(dbName, tbName); + transaction.finishInsertTable(new SimpleTableInfo(dbName, tbName)); } @Override diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/TestHMSCachedClient.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/TestHMSCachedClient.java index f3cb918d6f58d2..12e66398210b22 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/TestHMSCachedClient.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/TestHMSCachedClient.java @@ -18,8 +18,8 @@ package org.apache.doris.datasource; import org.apache.doris.analysis.TableName; +import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.datasource.hive.HMSCachedClient; -import org.apache.doris.datasource.hive.HMSTransaction; import org.apache.doris.datasource.hive.HiveDatabaseMetadata; import org.apache.doris.datasource.hive.HivePartitionStatistics; import org.apache.doris.datasource.hive.HivePartitionWithStatistics; @@ -48,7 +48,7 @@ public class TestHMSCachedClient implements HMSCachedClient { - public Map> partitions = new ConcurrentHashMap<>(); + public Map> partitions = new ConcurrentHashMap<>(); public Map> tables = new HashMap<>(); public List dbs = new ArrayList<>(); @@ -232,7 +232,7 @@ public void dropDatabase(String dbName) { public void dropTable(String dbName, String tableName) { Table table = getTable(dbName, tableName); this.tables.get(dbName).remove(table); - this.partitions.remove(new HMSTransaction.DatabaseTableName(dbName, tableName)); + this.partitions.remove(new SimpleTableInfo(dbName, tableName)); } @Override @@ -248,7 +248,7 @@ public void createTable(TableMetadata tbl, boolean ignoreIfExists) { List tableList = getTableList(tbl.getDbName()); tableList.add(HiveUtil.toHiveTable((HiveTableMetadata) tbl)); - HMSTransaction.DatabaseTableName key = new HMSTransaction.DatabaseTableName(dbName, tbName); + SimpleTableInfo key = new SimpleTableInfo(dbName, tbName); partitions.put(key, new ArrayList<>()); } @@ -322,7 +322,7 @@ public void dropPartition(String dbName, String tableName, List partitio } public List getPartitionList(String dbName, String tableName) { - HMSTransaction.DatabaseTableName key = new HMSTransaction.DatabaseTableName(dbName, tableName); + SimpleTableInfo key = new SimpleTableInfo(dbName, tableName); List partitionList = this.partitions.get(key); if (partitionList == null) { throw new RuntimeException("can't found table: " + key); diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java index 8644ee0ebdc8ec..7e99667b73124b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/hive/HmsCommitTest.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.Column; import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.common.info.SimpleTableInfo; import org.apache.doris.common.util.DebugUtil; import org.apache.doris.datasource.TestHMSCachedClient; import org.apache.doris.fs.FileSystem; @@ -142,9 +143,9 @@ public void before() { // create table for tbWithoutPartition HiveTableMetadata tableMetadata2 = new HiveTableMetadata( - dbName, tbWithoutPartition, Optional.of(dbLocation + tbWithPartition + UUID.randomUUID()), - columns, new ArrayList<>(), - new HashMap<>(), fileFormat, ""); + dbName, tbWithoutPartition, Optional.of(dbLocation + tbWithPartition + UUID.randomUUID()), + columns, new ArrayList<>(), + new HashMap<>(), fileFormat, ""); hmsClient.createTable(tableMetadata2, true); // context @@ -363,22 +364,22 @@ public THivePartitionUpdate genOnePartitionUpdate(String partitionValue, TUpdate public THivePartitionUpdate createRandomNew(String partition) throws IOException { return partition == null ? genOnePartitionUpdate(TUpdateMode.NEW) : - genOnePartitionUpdate("c3=" + partition, TUpdateMode.NEW); + genOnePartitionUpdate("c3=" + partition, TUpdateMode.NEW); } public THivePartitionUpdate createRandomAppend(String partition) throws IOException { return partition == null ? genOnePartitionUpdate(TUpdateMode.APPEND) : - genOnePartitionUpdate("c3=" + partition, TUpdateMode.APPEND); + genOnePartitionUpdate("c3=" + partition, TUpdateMode.APPEND); } public THivePartitionUpdate createRandomOverwrite(String partition) throws IOException { return partition == null ? genOnePartitionUpdate(TUpdateMode.OVERWRITE) : - genOnePartitionUpdate("c3=" + partition, TUpdateMode.OVERWRITE); + genOnePartitionUpdate("c3=" + partition, TUpdateMode.OVERWRITE); } public void commit(String dbName, - String tableName, - List hivePUs) { + String tableName, + List hivePUs) { HMSTransaction hmsTransaction = new HMSTransaction(hmsOps, fileSystemProvider, fileSystemExecutor); hmsTransaction.setHivePartitionUpdates(hivePUs); HiveInsertCommandContext ctx = new HiveInsertCommandContext(); @@ -386,7 +387,7 @@ public void commit(String dbName, ctx.setQueryId(queryId); ctx.setWritePath(writeLocation + queryId + "/"); hmsTransaction.beginInsertTable(ctx); - hmsTransaction.finishInsertTable(dbName, tableName); + hmsTransaction.finishInsertTable(new SimpleTableInfo(dbName, tableName)); hmsTransaction.commit(); } @@ -404,11 +405,11 @@ public void mockAsyncRenameDir(Runnable runnable) { new MockUp(HMSTransaction.class) { @Mock private void wrapperAsyncRenameDirWithProfileSummary(Executor executor, - List> renameFileFutures, - AtomicBoolean cancelled, - String origFilePath, - String destFilePath, - Runnable runWhenPathNotExist) { + List> renameFileFutures, + AtomicBoolean cancelled, + String origFilePath, + String destFilePath, + Runnable runWhenPathNotExist) { runnable.run(); throw new RuntimeException("failed to rename dir"); } diff --git a/regression-test/suites/external_table_p0/iceberg/write/test_iceberg_insert_overwrite.groovy b/regression-test/suites/external_table_p0/iceberg/write/test_iceberg_insert_overwrite.groovy new file mode 100644 index 00000000000000..db51add94076c4 --- /dev/null +++ b/regression-test/suites/external_table_p0/iceberg/write/test_iceberg_insert_overwrite.groovy @@ -0,0 +1,856 @@ +// 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. + +suite("test_iceberg_insert_overwrite", "p0,external,iceberg,external_docker,external_docker_iceberg") { + def format_compressions = ["parquet_zstd", "orc_zlib"] + + def q01 = { String format_compression, String catalog_name -> + def parts = format_compression.split("_") + def format = parts[0] + def compression = parts[1] + sql """ DROP TABLE IF EXISTS `iceberg_overwrite_all_types_${format_compression}`; """ + sql """ + CREATE TABLE `iceberg_overwrite_all_types_${format_compression}`( + `boolean_col` boolean, + `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, + `timestamp_col1` datetime, + `timestamp_col2` datetime, + `timestamp_col3` datetime, + `t_map_string` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` 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_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_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array, + `dt` int) ENGINE=iceberg + properties ( + "compression-codec" = ${compression}, + "write-format"=${format} + ) + """ + + sql """ + INSERT OVERWRITE table iceberg_overwrite_all_types_${format_compression} + VALUES ( + 1, -- boolean_col + 2147483647, -- int_col + 9223372036854775807, -- bigint_col + 123.45, -- 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 + '2024-03-20', -- date_col + '2024-03-20 12:00:00', -- timestamp_col1 + '2024-03-20 12:00:00.123456789', -- timestamp_col2 + '2024-03-20 12:00:00.123456789', -- timestamp_col3 + MAP('key1', 'value1'), -- t_map_string + 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 + MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + 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(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_field_null', null, 'struct_field_null2', null), -- t_struct_null + 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 + MAP('null_key', null), -- t_map_null_value + 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 + ARRAY(null, null, null), -- t_ARRAY_string_all_nulls + 20240320 -- dt + ); + """ + order_qt_q01 """ select * from iceberg_overwrite_all_types_${format_compression}; + """ + + sql """ + INSERT OVERWRITE table iceberg_overwrite_all_types_${format_compression} + VALUES ( + 1, -- boolean_col + 2147483647, -- int_col + 9223372036854775807, -- 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 + '2024-03-20', -- date_col + '2024-03-20 12:00:00', -- timestamp_col1 + '2024-03-20 12:00:00.123456789', -- timestamp_col2 + '2024-03-20 12:00:00.123456789', -- timestamp_col3 + MAP('key1', 'value1'), -- t_map_string + 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 + MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + 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(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(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_ARRAY_decimal_precision_18 + ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 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_field_null', null, 'struct_field_null2', null), -- t_struct_null + 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 + MAP('null_key', null), -- t_map_null_value + 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 + ARRAY(null, null, null), -- t_ARRAY_string_all_nulls + 20240320 -- dt + ), + ( + 0, -- boolean_col + -2147483648, -- int_col + -9223372036854775808, -- 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 + '2024-03-21', -- date_col + '2024-03-21 12:00:00', -- timestamp_col1 + '2024-03-21 12:00:00.123456789', -- timestamp_col2 + '2024-03-21 12:00:00.123456789', -- timestamp_col3 + MAP('key1', 'value1'), -- t_map_string + 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 + MAP(CAST(2.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(2.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(2.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + 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(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_field_null', null, 'struct_field_null2', null), -- t_struct_null + 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 + MAP('null_key', null), -- t_map_null_value + 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 + ARRAY(null, null, null), -- t_ARRAY_string_all_nulls + 20240321 -- dt + ), + ( + 0, -- boolean_col + -2147483648, -- int_col + -9223372036854775808, -- 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 + '2024-03-22', -- date_col + '2024-03-22 12:00:00', -- timestamp_col1 + '2024-03-22 12:00:00.123456789', -- timestamp_col2 + '2024-03-22 12:00:00.123456789', -- timestamp_col3 + MAP('key1', 'value1'), -- t_map_string + 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 + MAP(CAST(3.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(3.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(3.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + 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(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(8.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_field_null', null, 'struct_field_null2', null), -- t_struct_null + 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 + MAP('null_key', null), -- t_map_null_value + 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 + ARRAY(null, null, null), -- t_ARRAY_string_all_nulls + 20240322 -- dt + ); + """ + order_qt_q02 """ select * from iceberg_overwrite_all_types_${format_compression}; + """ + + sql """ + INSERT OVERWRITE table iceberg_overwrite_all_types_${format_compression}(float_col, t_map_int, t_ARRAY_decimal_precision_8, t_ARRAY_string_starting_with_nulls) + VALUES ( + CAST(123.45 AS FLOAT), -- float_col + MAP(1, 10), -- t_map_int + ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_ARRAY_decimal_precision_8 + ARRAY(null, 'value1', 'value2') -- t_ARRAY_string_starting_with_nulls + ); + """ + order_qt_q03 """ select * from iceberg_overwrite_all_types_${format_compression}; + """ + + sql """ DROP TABLE iceberg_overwrite_all_types_${format_compression}; """ + } + + + def q02 = { String format_compression, String catalog_name, String hive_catalog_name -> + def parts = format_compression.split("_") + def format = parts[0] + def compression = parts[1] + sql """ DROP TABLE IF EXISTS `iceberg_overwrite_all_types_${format_compression}`; """ + sql """ + CREATE TABLE `iceberg_overwrite_all_types_${format_compression}`( + `boolean_col` boolean, + `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, + `timestamp_col1` datetime, + `timestamp_col2` datetime, + `timestamp_col3` datetime, + `t_map_string` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` 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_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_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array, + `dt` int) ENGINE=iceberg + properties ( + "compression-codec" = ${compression}, + "write-format"=${format} + ) + """ + + sql """ + INSERT OVERWRITE TABLE iceberg_overwrite_all_types_${format_compression} + SELECT boolean_col, int_col, bigint_col, float_col, double_col, decimal_col1, decimal_col2, + decimal_col3, decimal_col4, string_col, date_col, timestamp_col1, timestamp_col2, timestamp_col3, + t_map_string, t_map_int, + t_map_bigint, t_map_float, t_map_double, t_map_boolean, t_map_decimal_precision_2, t_map_decimal_precision_4, + t_map_decimal_precision_8, t_map_decimal_precision_17, t_map_decimal_precision_18, t_map_decimal_precision_38, + t_ARRAY_string, t_ARRAY_int, t_ARRAY_bigint, t_ARRAY_float, t_ARRAY_double, t_ARRAY_boolean, + t_ARRAY_decimal_precision_2, t_ARRAY_decimal_precision_4, t_ARRAY_decimal_precision_8, + t_ARRAY_decimal_precision_17, t_ARRAY_decimal_precision_18, t_ARRAY_decimal_precision_38, t_struct_bigint, t_complex, + t_struct_nested, t_struct_null, t_struct_non_nulls_after_nulls, t_nested_struct_non_nulls_after_nulls, + t_map_null_value, t_ARRAY_string_starting_with_nulls, t_ARRAY_string_with_nulls_in_between, + t_ARRAY_string_ending_with_nulls, t_ARRAY_string_all_nulls, dt FROM ${hive_catalog_name}.write_test.all_types_parquet_snappy_src; + """ + order_qt_q01 """ select * from iceberg_overwrite_all_types_${format_compression}; + """ + + sql """ + INSERT OVERWRITE TABLE iceberg_overwrite_all_types_${format_compression}(float_col, t_map_int, t_ARRAY_decimal_precision_8, t_ARRAY_string_starting_with_nulls) + SELECT float_col, t_map_int, t_ARRAY_decimal_precision_8, t_ARRAY_string_starting_with_nulls FROM ${hive_catalog_name}.write_test.all_types_parquet_snappy_src; + """ + order_qt_q02 """ + select * from iceberg_overwrite_all_types_${format_compression}; + """ + + sql """ DROP TABLE iceberg_overwrite_all_types_${format_compression}; """ + } + + + def q03 = { String format_compression, String catalog_name -> + def parts = format_compression.split("_") + def format = parts[0] + def compression = parts[1] + sql """ DROP TABLE IF EXISTS `iceberg_overwrite_types_par_${format_compression}`; """ + sql """ + CREATE TABLE `iceberg_overwrite_types_par_${format_compression}`( + `boolean_col` boolean, + `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, + `timestamp_col1` datetime, + `timestamp_col2` datetime, + `timestamp_col3` datetime, + `t_map_string` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` 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_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_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array, + `dt` int) ENGINE=iceberg + PARTITION BY LIST (dt) () + properties ( + "compression-codec" = ${compression}, + "write-format"=${format} + ); + """ + + sql """ + INSERT OVERWRITE TABLE iceberg_overwrite_types_par_${format_compression} + VALUES ( + 1, -- boolean_col + 2147483647, -- int_col + 9223372036854775807, -- bigint_col + 123.45, -- float_col + 123456.789, -- double_col + 123456789, -- decimal_col1 + 1234.5678, -- decimal_col2 + CAST(123456.789012 AS DECIMAL(18,6)), -- decimal_col3 + CAST(123456789.012345678901 AS DECIMAL(38,12)), -- decimal_col4 + 'string_value', -- string_col + '2024-03-20', -- date_col + '2024-03-20 12:00:00', -- timestamp_col1 + '2024-03-20 12:00:00.123456789', -- timestamp_col2 + '2024-03-20 12:00:00.123456789', -- timestamp_col3 + MAP('key1', 'value1'), -- t_map_string + 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 + MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + 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(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(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_ARRAY_decimal_precision_18 + ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 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_field_null', null, 'struct_field_null2', null), -- t_struct_null + 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 + MAP('null_key', null), -- t_map_null_value + 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 + ARRAY(null, null, null), -- t_ARRAY_string_all_nulls + 20240320 -- dt + ); + """ + order_qt_q01 """ select * from iceberg_overwrite_types_par_${format_compression}; + """ + + sql """ + INSERT OVERWRITE TABLE iceberg_overwrite_types_par_${format_compression} + VALUES ( + 1, -- boolean_col + 2147483647, -- int_col + 9223372036854775807, -- 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 + '2024-03-20', -- date_col + '2024-03-20 12:00:00', -- timestamp_col1 + '2024-03-20 12:00:00.123456789', -- timestamp_col2 + '2024-03-20 12:00:00.123456789', -- timestamp_col3 + MAP('key1', 'value1'), -- t_map_string + 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 + MAP(CAST(1.1 AS DECIMAL(2,1)), CAST(1.1 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(1.23 AS DECIMAL(4,2)), CAST(1.23 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(1.2345 AS DECIMAL(8,4)), CAST(1.2345 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(1.23456789 AS DECIMAL(17,8)), CAST(1.23456789 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(1.23456789 AS DECIMAL(18,8)), CAST(1.23456789 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(1.234567890123456789 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + 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(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(CAST(1.23456789 AS DECIMAL(18,8)), CAST(2.34567891 AS DECIMAL(18,8))), -- t_ARRAY_decimal_precision_18 + ARRAY(CAST(1.234567890123456789 AS DECIMAL(38,16)), CAST(2.345678901234567890 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_field_null', null, 'struct_field_null2', null), -- t_struct_null + 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 + MAP('null_key', null), -- t_map_null_value + 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 + ARRAY(null, null, null), -- t_ARRAY_string_all_nulls + 20240320 -- dt + ), + ( + 0, -- boolean_col + -2147483648, -- int_col + -9223372036854775808, -- 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 + '2024-03-21', -- date_col + '2024-03-21 12:00:00', -- timestamp_col1 + '2024-03-21 12:00:00.123456789', -- timestamp_col2 + '2024-03-21 12:00:00.123456789', -- timestamp_col3 + MAP('key1', 'value1'), -- t_map_string + 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 + MAP(CAST(2.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(2.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(2.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + 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(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_field_null', null, 'struct_field_null2', null), -- t_struct_null + 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 + MAP('null_key', null), -- t_map_null_value + 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 + ARRAY(null, null, null), -- t_ARRAY_string_all_nulls + 20240321 -- dt + ), + ( + 0, -- boolean_col + -2147483648, -- int_col + -9223372036854775808, -- bigint_col + -123.45, -- 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 + '2024-03-22', -- date_col + '2024-03-22 12:00:00', -- timestamp_col1 + '2024-03-22 12:00:00.123456789', -- timestamp_col2 + '2024-03-22 12:00:00.123456789', -- timestamp_col3 + MAP('key1', 'value1'), -- t_map_string + 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 + MAP(CAST(3.2 AS DECIMAL(2,1)), CAST(2.2 AS DECIMAL(2,1))), -- t_map_decimal_precision_2 + MAP(CAST(3.34 AS DECIMAL(4,2)), CAST(2.34 AS DECIMAL(4,2))), -- t_map_decimal_precision_4 + MAP(CAST(2.3456 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_map_decimal_precision_8 + MAP(CAST(2.34567890 AS DECIMAL(17,8)), CAST(2.34567890 AS DECIMAL(17,8))), -- t_map_decimal_precision_17 + MAP(CAST(2.34567890 AS DECIMAL(18,8)), CAST(2.34567890 AS DECIMAL(18,8))), -- t_map_decimal_precision_18 + MAP(CAST(3.345678901234567890 AS DECIMAL(38,16)), CAST(2.345678901234567890 AS DECIMAL(38,16))), -- t_map_decimal_precision_38 + 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(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(8.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_field_null', null, 'struct_field_null2', null), -- t_struct_null + 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 + MAP('null_key', null), -- t_map_null_value + 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 + ARRAY(null, null, null), -- t_ARRAY_string_all_nulls + 20240322 -- dt + ); + """ + order_qt_q02 """ select * from iceberg_overwrite_types_par_${format_compression}; + """ + + sql """ + INSERT OVERWRITE TABLE iceberg_overwrite_types_par_${format_compression}(float_col, t_map_int, t_ARRAY_decimal_precision_8, t_ARRAY_string_starting_with_nulls, dt) + VALUES ( + 123.45, -- float_col + MAP(1, 10), -- t_map_int + ARRAY(CAST(1.2345 AS DECIMAL(8,4)), CAST(2.3456 AS DECIMAL(8,4))), -- t_ARRAY_decimal_precision_8 + ARRAY(null, 'value1', 'value2'), -- t_ARRAY_string_starting_with_nulls + 20240321 -- dt + ); + """ + order_qt_q03 """ select * from iceberg_overwrite_types_par_${format_compression}; + """ + + sql """ DROP TABLE iceberg_overwrite_types_par_${format_compression}; """ + } + + def q04 = { String format_compression, String catalog_name, String hive_catalog_name -> + def parts = format_compression.split("_") + def format = parts[0] + def compression = parts[1] + sql """ DROP TABLE IF EXISTS `iceberg_overwrite_types_par_${format_compression}`; """ + sql """ + CREATE TABLE `iceberg_overwrite_types_par_${format_compression}`( + `boolean_col` boolean, + `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, + `timestamp_col1` datetime, + `timestamp_col2` datetime, + `timestamp_col3` datetime, + `t_map_string` map, + `t_map_int` map, + `t_map_bigint` map, + `t_map_float` map, + `t_map_double` map, + `t_map_boolean` map, + `t_map_decimal_precision_2` map, + `t_map_decimal_precision_4` map, + `t_map_decimal_precision_8` map, + `t_map_decimal_precision_17` map, + `t_map_decimal_precision_18` map, + `t_map_decimal_precision_38` 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_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_null` struct, + `t_struct_non_nulls_after_nulls` struct, + `t_nested_struct_non_nulls_after_nulls` struct>, + `t_map_null_value` map, + `t_array_string_starting_with_nulls` array, + `t_array_string_with_nulls_in_between` array, + `t_array_string_ending_with_nulls` array, + `t_array_string_all_nulls` array, + `dt` int) ENGINE=iceberg + PARTITION BY LIST (dt) () + properties ( + "compression-codec" = ${compression}, + "write-format"=${format} + ); + """ + + sql """ + INSERT OVERWRITE TABLE iceberg_overwrite_types_par_${format_compression} + SELECT boolean_col, int_col, bigint_col, float_col, double_col, decimal_col1, decimal_col2, + decimal_col3, decimal_col4, string_col, date_col, timestamp_col1, timestamp_col2, timestamp_col3, + t_map_string, t_map_int, + t_map_bigint, t_map_float, t_map_double, t_map_boolean, t_map_decimal_precision_2, t_map_decimal_precision_4, + t_map_decimal_precision_8, t_map_decimal_precision_17, t_map_decimal_precision_18, t_map_decimal_precision_38, + t_ARRAY_string, t_ARRAY_int, t_ARRAY_bigint, t_ARRAY_float, t_ARRAY_double, t_ARRAY_boolean, + t_ARRAY_decimal_precision_2, t_ARRAY_decimal_precision_4, t_ARRAY_decimal_precision_8, + t_ARRAY_decimal_precision_17, t_ARRAY_decimal_precision_18, t_ARRAY_decimal_precision_38, t_struct_bigint, t_complex, + t_struct_nested, t_struct_null, t_struct_non_nulls_after_nulls, t_nested_struct_non_nulls_after_nulls, + t_map_null_value, t_ARRAY_string_starting_with_nulls, t_ARRAY_string_with_nulls_in_between, + t_ARRAY_string_ending_with_nulls, t_ARRAY_string_all_nulls, dt FROM ${hive_catalog_name}.write_test.all_types_parquet_snappy_src; + """ + order_qt_q01 """ select * from iceberg_overwrite_types_par_${format_compression}; + """ + + sql """ + INSERT OVERWRITE TABLE iceberg_overwrite_types_par_${format_compression}(float_col, t_map_int, t_ARRAY_decimal_precision_8, t_ARRAY_string_starting_with_nulls, dt) + SELECT float_col, t_map_int, t_ARRAY_decimal_precision_8, t_ARRAY_string_starting_with_nulls, dt FROM ${hive_catalog_name}.write_test.all_types_parquet_snappy_src; + """ + order_qt_q02 """ select * from iceberg_overwrite_types_par_${format_compression}; + """ + + sql """ DROP TABLE iceberg_overwrite_types_par_${format_compression}; """ + } + + String enabled = context.config.otherConfigs.get("enableHiveTest") + if (enabled == null || !enabled.equalsIgnoreCase("true")) { + logger.info("diable Hive test.") + return; + } + + for (String hivePrefix : ["hive2", "hive3"]) { + setHivePrefix(hivePrefix) + try { + String hms_port = context.config.otherConfigs.get(hivePrefix + "HmsPort") + String hdfs_port = context.config.otherConfigs.get(hivePrefix + "HdfsPort") + String externalEnvIp = context.config.otherConfigs.get("externalEnvIp") + + //create iceberg test catalog + String iceberg_catalog_name = "test_iceberg_insert_overwrite_iceberg_${hivePrefix}" + sql """drop catalog if exists ${iceberg_catalog_name}""" + sql """create catalog if not exists ${iceberg_catalog_name} properties ( + 'type'='iceberg', + 'iceberg.catalog.type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}', + 'use_meta_cache' = 'true' + );""" + + //create hive test catalog + String hive_catalog_name = "test_iceberg_insert_overwrite_hive_${hivePrefix}" + sql """drop catalog if exists ${hive_catalog_name}""" + sql """create catalog if not exists ${hive_catalog_name} properties ( + 'type'='hms', + 'hive.metastore.uris' = 'thrift://${externalEnvIp}:${hms_port}', + 'fs.defaultFS' = 'hdfs://${externalEnvIp}:${hdfs_port}', + 'use_meta_cache' = 'true' + );""" + + //using database and close planner fallback + sql """use `${iceberg_catalog_name}`.`write_test`""" + sql """set enable_fallback_to_original_planner=false;""" + + //run all external suites + for (String format_compression in format_compressions) { + logger.info("Process format_compression " + format_compression) + q01(format_compression, iceberg_catalog_name) + q02(format_compression, iceberg_catalog_name, hive_catalog_name) + q03(format_compression, iceberg_catalog_name) + q04(format_compression, iceberg_catalog_name, hive_catalog_name) + } + + //close after run external test + sql """drop catalog if exists ${iceberg_catalog_name}""" + sql """drop catalog if exists ${hive_catalog_name}""" + } finally { + } + } +} From 6b4c60a31267c67176dfe8804f2e33631f7bae55 Mon Sep 17 00:00:00 2001 From: lik40 Date: Wed, 3 Jul 2024 19:48:31 +0800 Subject: [PATCH 2/4] [merge](hive) insert overwrite command support iceberg operator --- .../insert/IcebergInsertCommandContext.java | 24 ++++++++++ .../insert/InsertOverwriteTableCommand.java | 44 ++++++++++++++++--- 2 files changed, 61 insertions(+), 7 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertCommandContext.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertCommandContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertCommandContext.java new file mode 100644 index 00000000000000..a091405b219ac3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/IcebergInsertCommandContext.java @@ -0,0 +1,24 @@ +// 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.nereids.trees.plans.commands.insert; + +/** + * For iceberg External Table + */ +public class IcebergInsertCommandContext extends BaseExternalTableInsertCommandContext { +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java index 3acd3d8123d8f0..7df3ea549ea0e6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/insert/InsertOverwriteTableCommand.java @@ -26,11 +26,13 @@ import org.apache.doris.common.UserException; import org.apache.doris.common.util.InternalDatabaseUtil; import org.apache.doris.datasource.hive.HMSExternalTable; +import org.apache.doris.datasource.iceberg.IcebergExternalTable; import org.apache.doris.insertoverwrite.InsertOverwriteUtil; import org.apache.doris.mtmv.MTMVUtil; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.analyzer.UnboundHiveTableSink; +import org.apache.doris.nereids.analyzer.UnboundIcebergTableSink; import org.apache.doris.nereids.analyzer.UnboundTableSink; import org.apache.doris.nereids.analyzer.UnboundTableSinkCreator; import org.apache.doris.nereids.exceptions.AnalysisException; @@ -111,10 +113,14 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { } TableIf targetTableIf = InsertUtils.getTargetTable(logicalQuery, ctx); - if (!(targetTableIf instanceof OlapTable || targetTableIf instanceof HMSExternalTable)) { - throw new AnalysisException("insert into overwrite only support OLAP and HMS table." - + " But current table type is " + targetTableIf.getType()); + //check allow insert overwrite + if (!allowInsertOverwrite(targetTableIf)) { + String errMsg = "insert into overwrite only support OLAP and HMS/ICEBERG table." + + " But current table type is " + targetTableIf.getType(); + LOG.error(errMsg); + throw new AnalysisException(errMsg); } + //check allow modify MTMVData if (targetTableIf instanceof MTMV && !MTMVUtil.allowModifyMTMVData(ctx)) { throw new AnalysisException("Not allowed to perform current operation on async materialized view"); } @@ -190,8 +196,16 @@ public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { } } + private boolean allowInsertOverwrite(TableIf targetTable) { + if (targetTable instanceof OlapTable) { + return true; + } else { + return targetTable instanceof HMSExternalTable || targetTable instanceof IcebergExternalTable; + } + } + private void runInsertCommand(LogicalPlan logicalQuery, InsertCommandContext insertCtx, - ConnectContext ctx, StmtExecutor executor) throws Exception { + ConnectContext ctx, StmtExecutor executor) throws Exception { InsertIntoTableCommand insertCommand = new InsertIntoTableCommand(logicalQuery, labelName, Optional.of(insertCtx), Optional.empty()); insertCommand.run(ctx, executor); @@ -205,8 +219,8 @@ private void runInsertCommand(LogicalPlan logicalQuery, InsertCommandContext ins /** * insert into select. for sepecified temp partitions * - * @param ctx ctx - * @param executor executor + * @param ctx ctx + * @param executor executor * @param tempPartitionNames tempPartitionNames */ private void insertInto(ConnectContext ctx, StmtExecutor executor, List tempPartitionNames) @@ -241,6 +255,19 @@ private void insertInto(ConnectContext ctx, StmtExecutor executor, List (LogicalPlan) (sink.child(0))); insertCtx = new HiveInsertCommandContext(); ((HiveInsertCommandContext) insertCtx).setOverwrite(true); + } else if (logicalQuery instanceof UnboundIcebergTableSink) { + UnboundIcebergTableSink sink = (UnboundIcebergTableSink) logicalQuery; + copySink = (UnboundLogicalSink) UnboundTableSinkCreator.createUnboundTableSink( + sink.getNameParts(), + sink.getColNames(), + sink.getHints(), + false, + sink.getPartitions(), + false, + sink.getDMLCommandType(), + (LogicalPlan) (sink.child(0))); + insertCtx = new IcebergInsertCommandContext(); + ((IcebergInsertCommandContext) insertCtx).setOverwrite(true); } else { throw new UserException("Current catalog does not support insert overwrite yet."); } @@ -250,7 +277,7 @@ private void insertInto(ConnectContext ctx, StmtExecutor executor, List /** * insert into auto detect partition. * - * @param ctx ctx + * @param ctx ctx * @param executor executor */ private void insertInto(ConnectContext ctx, StmtExecutor executor, long groupId) throws Exception { @@ -263,6 +290,9 @@ private void insertInto(ConnectContext ctx, StmtExecutor executor, long groupId) } else if (logicalQuery instanceof UnboundHiveTableSink) { insertCtx = new HiveInsertCommandContext(); ((HiveInsertCommandContext) insertCtx).setOverwrite(true); + } else if (logicalQuery instanceof UnboundIcebergTableSink) { + insertCtx = new IcebergInsertCommandContext(); + ((IcebergInsertCommandContext) insertCtx).setOverwrite(true); } else { throw new UserException("Current catalog does not support insert overwrite yet."); } From c5a0e40ddba882b1267b040787a030fa023ad44b Mon Sep 17 00:00:00 2001 From: morningman Date: Thu, 18 Jul 2024 16:58:13 +0800 Subject: [PATCH 3/4] rebase and add out file --- .../write/test_iceberg_insert_overwrite.out | 177 ++++++++++++++++++ 1 file changed, 177 insertions(+) create mode 100644 regression-test/data/external_table_p0/iceberg/write/test_iceberg_insert_overwrite.out diff --git a/regression-test/data/external_table_p0/iceberg/write/test_iceberg_insert_overwrite.out b/regression-test/data/external_table_p0/iceberg/write/test_iceberg_insert_overwrite.out new file mode 100644 index 00000000000000..945d3f2dbdf3c3 --- /dev/null +++ b/regression-test/data/external_table_p0/iceberg/write/test_iceberg_insert_overwrite.out @@ -0,0 +1,177 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !q01 -- +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N + +-- !q01 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N + +-- !q01 -- +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q01 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240321 +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240322 +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240320 + +-- !q01 -- +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N + +-- !q01 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N + +-- !q01 -- +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q01 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240321 +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240322 +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240320 + +-- !q01 -- +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N + +-- !q01 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N + +-- !q01 -- +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q01 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240321 +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240322 +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240320 + +-- !q01 -- +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N + +-- !q01 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N \N + +-- !q01 -- +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123457 2024-03-21T12:00:00.123457 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q03 -- +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123457 2024-03-22T12:00:00.123457 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123457 2024-03-20T12:00:00.123457 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q01 -- +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-21 2024-03-21T12:00 2024-03-21T12:00:00.123456 2024-03-21T12:00:00.123456 {"key1":"value1"} {2:20} {2:200000000000} {2.2:20.2} {2.2:20.2} {0:1} {2.2:2.2} {2.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {2.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [3.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240321 +false -2147483648 -9223372036854775808 -123.45 -123456.789 -123456789 -1234.5678 -123456.789012 -123456789.012345678901 string_value 2024-03-22 2024-03-22T12:00 2024-03-22T12:00:00.123456 2024-03-22T12:00:00.123456 {"key1":"value1"} {3:20} {3:200000000000} {3.2:20.2} {3.2:20.2} {0:1} {3.2:2.2} {3.34:2.34} {2.3456:2.3456} {2.34567890:2.34567890} {2.34567890:2.34567890} {3.3456789012345679:2.3456789012345679} ["string1", "string2"] [4, 5, 6] [300000000000, 400000000000] [3.3, 4.4] [3.123456789, 4.123456789] [0, 1] [3.3, 4.4] [3.45, 4.56] [8.4567, 4.5678] [3.45678901, 4.56789012] [3.45678901, 4.56789012] [3.4567890123456789, 4.5678901234567890] {"s_bigint":-1234567890} {"key":[{"s_int":-123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value11", "value2", null] [null, null, null] 20240322 +true 2147483647 9223372036854775807 123.45 123456.789 123456789 1234.5678 123456.789012 123456789.012345678901 string_value 2024-03-20 2024-03-20T12:00 2024-03-20T12:00:00.123456 2024-03-20T12:00:00.123456 {"key1":"value1"} {1:10} {1:100000000000} {1.1:10.1} {1.1:10.1} {1:0} {1.1:1.1} {1.23:1.23} {1.2345:1.2345} {1.23456789:1.23456789} {1.23456789:1.23456789} {1.2345678901234568:1.2345678901234568} ["string1", "string2"] [1, 2, 3] [100000000000, 200000000000] [1.1, 2.2] [1.123456789, 2.123456789] [1, 0] [1.1, 2.2] [1.23, 2.34] [1.2345, 2.3456] [1.23456789, 2.34567891] [1.23456789, 2.34567891] [1.2345678901234568, 2.3456789012345679] {"s_bigint":1234567890} {"key":[{"s_int":123}]} {"struct_field":["value1", "value2"]} {"struct_field_null":null, "struct_field_null2":null} {"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_key":null} [null, "value1", "value2"] ["value1", null, "value2"] ["value1", "value2", null] [null, null, null] 20240320 + +-- !q02 -- +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {2:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [3.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240321 +\N \N \N -123.45 \N \N \N \N \N \N \N \N \N \N \N {3:20} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [8.4567, 4.5678] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240322 +\N \N \N 123.45 \N \N \N \N \N \N \N \N \N \N \N {1:10} \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N \N [1.2345, 2.3456] \N \N \N \N \N \N \N \N \N \N [null, "value1", "value2"] \N \N \N 20240320 + From a114ba093c180b165b8b70a48e52323cd72d89c1 Mon Sep 17 00:00:00 2001 From: morningman Date: Thu, 18 Jul 2024 17:05:17 +0800 Subject: [PATCH 4/4] fix compile --- .../doris/datasource/hive/HMSTransaction.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java index a6bb64c48a3a95..82a9100be2fbfe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/HMSTransaction.java @@ -91,7 +91,7 @@ public class HMSTransaction implements Transaction { private final Map> tableActions = new HashMap<>(); private final Map, Action>> partitionActions = new HashMap<>(); - private final Map> tableColumns = new HashMap<>(); + private final Map> tableColumns = new HashMap<>(); private final Executor fileSystemExecutor; private HmsCommitter hmsCommitter; @@ -239,7 +239,7 @@ public void finishInsertTable(SimpleTableInfo tableInfo) { Maps.newHashMap(), sd.getOutputFormat(), sd.getSerdeInfo().getSerializationLib(), - getTableColumns(tableInfo.getDbName(), tableInfo.getTbName()) + getTableColumns(tableInfo) ); if (updateMode == TUpdateMode.OVERWRITE) { dropPartition(tableInfo, hivePartition.getPartitionValues(), true); @@ -396,7 +396,7 @@ private void convertToInsertExistingPartitionAction( partition.getParameters(), sd.getOutputFormat(), sd.getSerdeInfo().getSerializationLib(), - getTableColumns(tableInfo.getDbName(), tableInfo.getTbName() + getTableColumns(tableInfo) ); partitionActionsForTable.put( @@ -866,9 +866,9 @@ public synchronized Table getTable(SimpleTableInfo tableInfo) { throw new RuntimeException("Not Found table: " + tableInfo); } - public synchronized List getTableColumns(String databaseName, String tableName) { - return tableColumns.computeIfAbsent(new DatabaseTableName(databaseName, tableName), - key -> hiveOps.getClient().getSchema(dbName, tbName)); + public synchronized List getTableColumns(SimpleTableInfo tableInfo) { + return tableColumns.computeIfAbsent(tableInfo, + key -> hiveOps.getClient().getSchema(tableInfo.getDbName(), tableInfo.getTbName())); } public synchronized void finishChangingExistingTable( @@ -1226,7 +1226,7 @@ public void prepareAddPartition(SimpleTableInfo tableInfo, PartitionAndMore part Maps.newHashMap(), sd.getOutputFormat(), sd.getSerdeInfo().getSerializationLib(), - getTableColumns(tableInfo.getDbName(), tableInfo.getTbName()) + getTableColumns(tableInfo) ); HivePartitionWithStatistics partitionWithStats =