This is an automated email from the ASF dual-hosted git repository. dkuzmenko pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push: new fa02792fe4a HIVE-28047: Iceberg: Major QB Compaction on unpartitioned tables with a single commit (Dmitriy Fingerman, reviewed by Denys Kuzmenko) fa02792fe4a is described below commit fa02792fe4a24098f9b2a874f4195d40dd85ac28 Author: Dmitriy Fingerman <dmitriy.finger...@gmail.com> AuthorDate: Tue Aug 13 05:20:50 2024 -0400 HIVE-28047: Iceberg: Major QB Compaction on unpartitioned tables with a single commit (Dmitriy Fingerman, reviewed by Denys Kuzmenko) Closes #5389 --- .../mr/hive/HiveIcebergOutputCommitter.java | 67 +++++++--------------- .../apache/iceberg/mr/hive/IcebergTableUtil.java | 37 ++++++------ .../hive/compaction/IcebergCompactionService.java | 1 - .../compaction/IcebergMajorQueryCompactor.java | 2 - .../iceberg_major_compaction_query_metadata.q | 2 + .../iceberg_major_compaction_unpartitioned.q | 2 + .../iceberg_optimize_table_unpartitioned.q | 2 + .../iceberg_major_compaction_query_metadata.q.out | 4 +- .../iceberg_major_compaction_unpartitioned.q.out | 6 +- .../iceberg_optimize_table_unpartitioned.q.out | 6 +- 10 files changed, 54 insertions(+), 75 deletions(-) diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java index 275681199e3..189661b4736 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java @@ -35,7 +35,6 @@ import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.function.Predicate; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; @@ -71,7 +70,6 @@ import org.apache.iceberg.RowDelta; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; -import org.apache.iceberg.Transaction; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -442,12 +440,14 @@ public class HiveIcebergOutputCommitter extends OutputCommitter { Table table = null; String branchName = null; + Long snapshotId = null; Expression filterExpr = Expressions.alwaysTrue(); for (JobContext jobContext : outputTable.jobContexts) { JobConf conf = jobContext.getJobConf(); table = Optional.ofNullable(table).orElse(Catalogs.loadTable(conf, catalogProperties)); branchName = conf.get(InputFormatConfig.OUTPUT_TABLE_SNAPSHOT_REF); + snapshotId = getSnapshotId(outputTable.table, branchName); Expression jobContextFilterExpr = (Expression) SessionStateUtil.getResource(conf, InputFormatConfig.QUERY_FILTERS) .orElse(Expressions.alwaysTrue()); @@ -491,7 +491,6 @@ public class HiveIcebergOutputCommitter extends OutputCommitter { table, outputTable.jobContexts.stream().map(JobContext::getJobID) .map(String::valueOf).collect(Collectors.joining(","))); } else { - Long snapshotId = getSnapshotId(outputTable.table, branchName); commitWrite(table, branchName, snapshotId, startTime, filesForCommit, operation, filterExpr); } } else { @@ -502,18 +501,12 @@ public class HiveIcebergOutputCommitter extends OutputCommitter { .orElse(RewritePolicy.DEFAULT.name())); if (rewritePolicy != RewritePolicy.DEFAULT) { - Integer partitionSpecId = outputTable.jobContexts.stream() - .findAny() - .map(x -> x.getJobConf().get(IcebergCompactionService.PARTITION_SPEC_ID)) - .map(Integer::valueOf) - .orElse(null); - String partitionPath = outputTable.jobContexts.stream() .findAny() .map(x -> x.getJobConf().get(IcebergCompactionService.PARTITION_PATH)) .orElse(null); - commitCompaction(table, startTime, filesForCommit, rewritePolicy, partitionSpecId, partitionPath); + commitCompaction(table, snapshotId, startTime, filesForCommit, partitionPath); } else { commitOverwrite(table, branchName, startTime, filesForCommit); } @@ -597,46 +590,30 @@ public class HiveIcebergOutputCommitter extends OutputCommitter { * Either full table or a selected partition contents is replaced with compacted files. * * @param table The table we are changing + * @param snapshotId The snapshot id of the table to use for validation * @param startTime The start time of the commit - used only for logging * @param results The object containing the new files - * @param rewritePolicy The rewrite policy to use for the insert overwrite commit - * @param partitionSpecId The table spec_id for partition compaction operation * @param partitionPath The path of the compacted partition */ - private void commitCompaction(Table table, long startTime, FilesForCommit results, - RewritePolicy rewritePolicy, Integer partitionSpecId, String partitionPath) { - if (rewritePolicy == RewritePolicy.FULL_TABLE) { - // Full table compaction - Transaction transaction = table.newTransaction(); - DeleteFiles delete = transaction.newDelete(); - delete.deleteFromRowFilter(Expressions.alwaysTrue()); - delete.commit(); - ReplacePartitions overwrite = transaction.newReplacePartitions(); - results.dataFiles().forEach(overwrite::addFile); - overwrite.commit(); - transaction.commitTransaction(); - LOG.debug("Compacted full table with files {}", results); - } else { - // Single partition compaction - List<DataFile> existingDataFiles = - IcebergTableUtil.getDataFiles(table, partitionSpecId, partitionPath, - partitionPath == null ? Predicate.isEqual(partitionSpecId).negate() : Predicate.isEqual(partitionSpecId)); - List<DeleteFile> existingDeleteFiles = - IcebergTableUtil.getDeleteFiles(table, partitionSpecId, partitionPath, - partitionPath == null ? Predicate.isEqual(partitionSpecId).negate() : Predicate.isEqual(partitionSpecId)); - - RewriteFiles rewriteFiles = table.newRewrite(); - rewriteFiles.validateFromSnapshot(table.currentSnapshot().snapshotId()); - - existingDataFiles.forEach(rewriteFiles::deleteFile); - existingDeleteFiles.forEach(rewriteFiles::deleteFile); - results.dataFiles().forEach(rewriteFiles::addFile); - - rewriteFiles.commit(); - LOG.debug("Compacted partition {} with files {}", partitionPath, results); + private void commitCompaction(Table table, Long snapshotId, long startTime, FilesForCommit results, + String partitionPath) { + List<DataFile> existingDataFiles = IcebergTableUtil.getDataFiles(table, partitionPath); + List<DeleteFile> existingDeleteFiles = IcebergTableUtil.getDeleteFiles(table, partitionPath); + + RewriteFiles rewriteFiles = table.newRewrite(); + rewriteFiles.validateFromSnapshot(getSnapshotId(table, null)); + if (snapshotId != null) { + rewriteFiles.validateFromSnapshot(snapshotId); } - LOG.info("Compaction commit took {} ms for table: {} with {} file(s)", System.currentTimeMillis() - startTime, - table, results.dataFiles().size()); + + existingDataFiles.forEach(rewriteFiles::deleteFile); + existingDeleteFiles.forEach(rewriteFiles::deleteFile); + results.dataFiles().forEach(rewriteFiles::addFile); + + rewriteFiles.commit(); + LOG.info("Compaction commit took {} ms for table: {} partition: {} with {} file(s)", + System.currentTimeMillis() - startTime, table, partitionPath == null ? "N/A" : partitionPath, + results.dataFiles().size()); } /** diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java index 9a661bdaa73..627e6f4b101 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java @@ -27,7 +27,6 @@ import java.util.Optional; import java.util.Properties; import java.util.function.BinaryOperator; import java.util.function.Function; -import java.util.function.Predicate; import java.util.stream.Collectors; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; @@ -395,46 +394,46 @@ public class IcebergTableUtil { } /** - * Returns list of data files filtered by specId and partitionPath as following: - * 1. If matchBySpecId is true, then filters files by specId == file's specId, else by specId != file's specId - * 2. If partitionPath is not null, then also filters files where partitionPath == file's partition path + * Returns table's list of data files as following: + * 1. If the table is unpartitioned, returns all data files. + * 2. If partitionPath is not provided, returns all data files that belong to the non-latest partition spec. + * 3. If partitionPath is provided, returns all data files that belong to the corresponding partition. * @param table the iceberg table - * @param specId partition spec id * @param partitionPath partition path - * @param matchBySpecId filter that's applied on data files' spec ids */ - public static List<DataFile> getDataFiles(Table table, int specId, String partitionPath, - Predicate<Object> matchBySpecId) { + public static List<DataFile> getDataFiles(Table table, String partitionPath) { CloseableIterable<FileScanTask> fileScanTasks = table.newScan().useSnapshot(table.currentSnapshot().snapshotId()).ignoreResiduals().planFiles(); CloseableIterable<FileScanTask> filteredFileScanTasks = CloseableIterable.filter(fileScanTasks, t -> { DataFile file = t.asFileScanTask().file(); - return matchBySpecId.test(file.specId()) && (partitionPath == null || (partitionPath != null && - table.specs().get(specId).partitionToPath(file.partition()).equals(partitionPath))); + return !table.spec().isPartitioned() || + (partitionPath == null && file.specId() != table.spec().specId()) || + (partitionPath != null && + table.specs().get(file.specId()).partitionToPath(file.partition()).equals(partitionPath)); }); return Lists.newArrayList(CloseableIterable.transform(filteredFileScanTasks, t -> t.file())); } /** - * Returns list of delete files filtered by specId and partitionPath as following: - * 1. If matchBySpecId is true, then filters files by specId == file's specId, else by specId != file's specId - * 2. If partitionPath is not null, then also filters files where partitionPath == file's partition path + * Returns table's list of delete files as following: + * 1. If the table is unpartitioned, returns all delete files. + * 2. If partitionPath is not provided, returns all delete files that belong to the non-latest partition spec. + * 3. If partitionPath is provided, returns all delete files that belong to corresponding partition. * @param table the iceberg table - * @param specId partition spec id * @param partitionPath partition path - * @param matchBySpecId filter that's applied on delete files' spec ids */ - public static List<DeleteFile> getDeleteFiles(Table table, int specId, String partitionPath, - Predicate<Object> matchBySpecId) { + public static List<DeleteFile> getDeleteFiles(Table table, String partitionPath) { Table deletesTable = MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); CloseableIterable<ScanTask> deletesScanTasks = deletesTable.newBatchScan().planFiles(); CloseableIterable<ScanTask> filteredDeletesScanTasks = CloseableIterable.filter(deletesScanTasks, t -> { DeleteFile file = ((PositionDeletesScanTask) t).file(); - return matchBySpecId.test(file.specId()) && (partitionPath == null || (partitionPath != null && - table.specs().get(specId).partitionToPath(file.partition()).equals(partitionPath))); + return !table.spec().isPartitioned() || + (partitionPath == null && file.specId() != table.spec().specId()) || + (partitionPath != null && + table.specs().get(file.specId()).partitionToPath(file.partition()).equals(partitionPath)); }); return Lists.newArrayList(CloseableIterable.transform(filteredDeletesScanTasks, t -> ((PositionDeletesScanTask) t).file())); diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java index b69714544d8..455330eb5f1 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java @@ -29,7 +29,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class IcebergCompactionService extends CompactionService { - public static final String PARTITION_SPEC_ID = "compaction_part_spec_id"; public static final String PARTITION_PATH = "compaction_partition_path"; private static final String CLASS_NAME = IcebergCompactionService.class.getName(); private static final Logger LOG = LoggerFactory.getLogger(CLASS_NAME); diff --git a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java index 411031b50c8..906854070d1 100644 --- a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java +++ b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergMajorQueryCompactor.java @@ -75,7 +75,6 @@ public class IcebergMajorQueryCompactor extends QueryCompactor { } else if (icebergTable.specs().size() > 1) { // Compacting partitions of old partition specs on a partitioned table with partition evolution HiveConf.setVar(conf, ConfVars.REWRITE_POLICY, RewritePolicy.PARTITION.name()); - conf.set(IcebergCompactionService.PARTITION_SPEC_ID, String.valueOf(icebergTable.spec().specId())); // A single filter on a virtual column causes errors during compilation, // added another filter on file_path as a workaround. compactionQuery = String.format("insert overwrite table %1$s select * from %1$s " + @@ -103,7 +102,6 @@ public class IcebergMajorQueryCompactor extends QueryCompactor { } HiveConf.setVar(conf, ConfVars.REWRITE_POLICY, RewritePolicy.PARTITION.name()); - conf.set(IcebergCompactionService.PARTITION_SPEC_ID, String.valueOf(specId.get())); conf.set(IcebergCompactionService.PARTITION_PATH, new Path(partSpec).toString()); List<FieldSchema> partitionKeys = IcebergTableUtil.getPartitionKeys(icebergTable, specId.get()); diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_query_metadata.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_query_metadata.q index f7240dd12ea..388a58d4249 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_query_metadata.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_query_metadata.q @@ -10,6 +10,8 @@ --! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ -- Mask show compactions fields that change across runs --! qt:replace:/(MAJOR\s+succeeded\s+)[a-zA-Z0-9\-\.\s+]+(\s+manual)/$1#Masked#$2/ +-- Mask removed file size +--! qt:replace:/(\S\"removed-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ set hive.llap.io.enabled=true; set hive.vectorized.execution.enabled=true; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_unpartitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_unpartitioned.q index eacb6a00d59..b9a45237991 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_unpartitioned.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_major_compaction_unpartitioned.q @@ -15,6 +15,8 @@ --! qt:replace:/(MAJOR\s+succeeded\s+)[a-zA-Z0-9\-\.\s+]+(\s+manual)/$1#Masked#$2/ -- Mask compaction id as they will be allocated in parallel threads --! qt:replace:/^[0-9]/#Masked#/ +-- Mask removed file size +--! qt:replace:/(\S\"removed-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ set hive.llap.io.enabled=true; set hive.vectorized.execution.enabled=true; diff --git a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_optimize_table_unpartitioned.q b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_optimize_table_unpartitioned.q index 5fbc108125e..1b3b1a6caab 100644 --- a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_optimize_table_unpartitioned.q +++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_optimize_table_unpartitioned.q @@ -17,6 +17,8 @@ --! qt:replace:/(MAJOR\s+succeeded\s+)[a-zA-Z0-9\-\.\s+]+(\s+manual)/$1#Masked#$2/ -- Mask compaction id as they will be allocated in parallel threads --! qt:replace:/^[0-9]/#Masked#/ +-- Mask removed file size +--! qt:replace:/(\S\"removed-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/ set hive.llap.io.enabled=true; set hive.vectorized.execution.enabled=true; diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_query_metadata.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_query_metadata.q.out index 4e369d322ef..54e6ba120fb 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_query_metadata.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_query_metadata.q.out @@ -94,7 +94,7 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"age\",\"required\":false,\"type\":\"int\"},{\"id\":3,\"name\":\"num_clicks\",\"required\":false,\"type\":\"long\"}]} current-snapshot-id #Masked# - current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"added-records\":\"7\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"7\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} + current-snapshot-summary {\"added-data-files\":\"1\",\"deleted-data-files\":\"2\",\"added-records\":\"7\",\"deleted-records\":\"7\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"7\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# external.table.purge true format-version 2 @@ -106,7 +106,7 @@ Table Parameters: #### A masked pattern was here #### rawDataSize 0 serialization.format 1 - snapshot-count 4 + snapshot-count 3 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG totalSize #Masked# diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_unpartitioned.q.out index 5ba43b21a7e..3a68668ff63 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_major_compaction_unpartitioned.q.out @@ -184,7 +184,7 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"}]} current-snapshot-id #Masked# - current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"3\",\"removed-files-size\":\"1131\",\"changed-partition-count\":\"1\",\"total-records\":\"11\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"11\",\"total-delete-files\":\"7\",\"total-position-deletes\":\"7\",\"total-equality-deletes\":\"0\"} + current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"3\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"11\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"11\",\"total-delete-files\":\"7\",\"total-position-deletes\":\"7\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# format-version 2 iceberg.orc.files.only true @@ -274,7 +274,7 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"}]} current-snapshot-id #Masked# - current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"added-records\":\"4\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"4\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} + current-snapshot-summary {\"added-data-files\":\"1\",\"deleted-data-files\":\"11\",\"removed-position-delete-files\":\"7\",\"removed-delete-files\":\"7\",\"added-records\":\"4\",\"deleted-records\":\"11\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"removed-position-deletes\":\"7\",\"changed-partition-count\":\"1\",\"total-records\":\"4\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\", [...] current-snapshot-timestamp-ms #Masked# format-version 2 iceberg.orc.files.only true @@ -285,7 +285,7 @@ Table Parameters: #### A masked pattern was here #### rawDataSize 0 serialization.format 1 - snapshot-count 17 + snapshot-count 16 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG totalSize #Masked# diff --git a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_optimize_table_unpartitioned.q.out b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_optimize_table_unpartitioned.q.out index a4ea671dd05..c5a65407570 100644 --- a/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_optimize_table_unpartitioned.q.out +++ b/iceberg/iceberg-handler/src/test/results/positive/llap/iceberg_optimize_table_unpartitioned.q.out @@ -184,7 +184,7 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"}]} current-snapshot-id #Masked# - current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"3\",\"removed-files-size\":\"1131\",\"changed-partition-count\":\"1\",\"total-records\":\"11\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"11\",\"total-delete-files\":\"7\",\"total-position-deletes\":\"7\",\"total-equality-deletes\":\"0\"} + current-snapshot-summary {\"deleted-data-files\":\"3\",\"deleted-records\":\"3\",\"removed-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"11\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"11\",\"total-delete-files\":\"7\",\"total-position-deletes\":\"7\",\"total-equality-deletes\":\"0\"} current-snapshot-timestamp-ms #Masked# format-version 2 iceberg.orc.files.only true @@ -274,7 +274,7 @@ Table Parameters: bucketing_version 2 current-schema {\"type\":\"struct\",\"schema-id\":0,\"fields\":[{\"id\":1,\"name\":\"first_name\",\"required\":false,\"type\":\"string\"},{\"id\":2,\"name\":\"last_name\",\"required\":false,\"type\":\"string\"}]} current-snapshot-id #Masked# - current-snapshot-summary {\"replace-partitions\":\"true\",\"added-data-files\":\"1\",\"added-records\":\"4\",\"added-files-size\":\"#Masked#\",\"changed-partition-count\":\"1\",\"total-records\":\"4\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\",\"total-equality-deletes\":\"0\"} + current-snapshot-summary {\"added-data-files\":\"1\",\"deleted-data-files\":\"11\",\"removed-position-delete-files\":\"7\",\"removed-delete-files\":\"7\",\"added-records\":\"4\",\"deleted-records\":\"11\",\"added-files-size\":\"#Masked#\",\"removed-files-size\":\"#Masked#\",\"removed-position-deletes\":\"7\",\"changed-partition-count\":\"1\",\"total-records\":\"4\",\"total-files-size\":\"#Masked#\",\"total-data-files\":\"1\",\"total-delete-files\":\"0\",\"total-position-deletes\":\"0\", [...] current-snapshot-timestamp-ms #Masked# format-version 2 iceberg.orc.files.only true @@ -285,7 +285,7 @@ Table Parameters: #### A masked pattern was here #### rawDataSize 0 serialization.format 1 - snapshot-count 17 + snapshot-count 16 storage_handler org.apache.iceberg.mr.hive.HiveIcebergStorageHandler table_type ICEBERG totalSize #Masked#