Skip to content

Commit db5c2ad

Browse files
author
Dmitriy Fingerman
committed
HIVE-28590: Iceberg: Add support for FILE_SIZE_THRESHOLD to compaction command
1 parent 7b3a4be commit db5c2ad

File tree

20 files changed

+430
-42
lines changed

20 files changed

+430
-42
lines changed

common/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -493,6 +493,7 @@ public enum ErrorMsg {
493493
ICEBERG_COMPACTION_WITH_PART_SPEC_AND_FILTER_NOT_SUPPORTED(10441, "Compaction command with both partition spec and filter is not supported on Iceberg table {0}.{1}", true),
494494
COMPACTION_THREAD_INITIALIZATION(10442, "Compaction thread failed during initialization", false),
495495
ALTER_TABLE_COMPACTION_NON_PARTITIONED_COLUMN_NOT_ALLOWED(10443, "Filter expression can contain only partition columns."),
496+
UNSUPPORTED_COMPACTION_REQUEST_WITH_FILE_SIZE_THRESHOLD(10444, "File size threshold is supported only with major and minor compaction for Iceberg tables"),
496497

497498
//========================== 20000 range starts here ========================//
498499

iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java

Lines changed: 12 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,7 @@
5050
import org.apache.hadoop.hive.ql.metadata.HiveUtils;
5151
import org.apache.hadoop.hive.ql.plan.MapWork;
5252
import org.apache.hadoop.hive.ql.session.SessionStateUtil;
53+
import org.apache.hadoop.hive.ql.txn.compactor.CompactorContext;
5354
import org.apache.hadoop.mapred.JobConf;
5455
import org.apache.hadoop.mapred.JobContext;
5556
import org.apache.hadoop.mapred.JobContextImpl;
@@ -505,7 +506,13 @@ private void commitTable(FileIO io, ExecutorService executor, OutputTable output
505506
.map(x -> x.getJobConf().get(IcebergCompactionService.PARTITION_PATH))
506507
.orElse(null);
507508

508-
commitCompaction(table, snapshotId, startTime, filesForCommit, partitionPath);
509+
Long fileSizeThreshold = jobContexts.stream()
510+
.findAny()
511+
.map(x -> x.getJobConf().get(CompactorContext.COMPACTION_FILE_SIZE_THRESHOLD))
512+
.map(Long::parseLong)
513+
.orElse(null);
514+
515+
commitCompaction(table, snapshotId, startTime, filesForCommit, partitionPath, fileSizeThreshold);
509516
} else {
510517
commitOverwrite(table, branchName, snapshotId, startTime, filesForCommit);
511518
}
@@ -597,9 +604,10 @@ private void commitWrite(Table table, String branchName, Long snapshotId, long s
597604
* @param partitionPath The path of the compacted partition
598605
*/
599606
private void commitCompaction(Table table, Long snapshotId, long startTime, FilesForCommit results,
600-
String partitionPath) {
601-
List<DataFile> existingDataFiles = IcebergTableUtil.getDataFiles(table, partitionPath);
602-
List<DeleteFile> existingDeleteFiles = IcebergTableUtil.getDeleteFiles(table, partitionPath);
607+
String partitionPath, Long fileSizeThreshold) {
608+
List<DataFile> existingDataFiles = IcebergTableUtil.getDataFiles(table, partitionPath, fileSizeThreshold);
609+
List<DeleteFile> existingDeleteFiles = fileSizeThreshold == null ?
610+
IcebergTableUtil.getDeleteFiles(table, partitionPath) : Collections.emptyList();
603611

604612
RewriteFiles rewriteFiles = table.newRewrite();
605613
existingDataFiles.forEach(rewriteFiles::deleteFile);

iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandler.java

Lines changed: 3 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -2255,24 +2255,13 @@ boolean canCompact(HiveConf hiveConf, Table icebergTable, String partitionPath,
22552255
return true;
22562256
}
22572257

2258-
int dataFilesCount = IcebergTableUtil.getDataFiles(icebergTable, partitionPath).size();
2258+
int dataFilesCount = IcebergTableUtil.getDataFiles(icebergTable, partitionPath, null).size();
22592259
if (dataFilesCount < 2) {
22602260
return false;
22612261
}
22622262

2263-
long fileSizeInBytesThreshold;
2264-
switch (compactionType) {
2265-
case MAJOR:
2266-
fileSizeInBytesThreshold = HiveConf.getSizeVar(hiveConf,
2267-
ConfVars.HIVE_ICEBERG_MAJOR_COMPACTION_FILE_SIZE_THRESHOLD);
2268-
break;
2269-
case MINOR:
2270-
fileSizeInBytesThreshold = HiveConf.getSizeVar(hiveConf,
2271-
ConfVars.HIVE_ICEBERG_MINOR_COMPACTION_FILE_SIZE_THRESHOLD);
2272-
break;
2273-
default:
2274-
throw new HiveException(String.format("Invalid compaction type: %s", compactionType.name()));
2275-
}
2263+
long fileSizeInBytesThreshold = HiveConf.toSizeBytes(IcebergTableUtil.getFileSizeTreshold(hiveConf,
2264+
compactionType));
22762265

22772266
float fileSizeRatioThreshold = HiveConf.getFloatVar(hiveConf, ConfVars.ICEBERG_COMPACTION_FILE_SIZE_RATIO);
22782267
float fileSizeRatio = IcebergTableUtil.getFileSizeRatio(icebergTable, partitionPath, fileSizeInBytesThreshold);

iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java

Lines changed: 27 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -36,11 +36,13 @@
3636
import org.apache.hadoop.hive.common.type.TimestampTZUtil;
3737
import org.apache.hadoop.hive.conf.HiveConf;
3838
import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
39+
import org.apache.hadoop.hive.metastore.api.CompactionType;
3940
import org.apache.hadoop.hive.metastore.api.FieldSchema;
4041
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
4142
import org.apache.hadoop.hive.ql.Context;
4243
import org.apache.hadoop.hive.ql.QueryState;
4344
import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
45+
import org.apache.hadoop.hive.ql.metadata.HiveException;
4446
import org.apache.hadoop.hive.ql.metadata.HiveUtils;
4547
import org.apache.hadoop.hive.ql.parse.AlterTableExecuteSpec;
4648
import org.apache.hadoop.hive.ql.parse.SemanticException;
@@ -417,19 +419,23 @@ public static PartitionData toPartitionData(StructLike sourceKey, Types.StructTy
417419
* 1. If the table is unpartitioned, returns all data files.
418420
* 2. If partitionPath is not provided, returns all data files that belong to the non-latest partition spec.
419421
* 3. If partitionPath is provided, returns all data files that belong to the corresponding partition.
422+
* 4. If fileSizeThreshold is not null, file size threshold also applied in all cases above.
420423
* @param table the iceberg table
421424
* @param partitionPath partition path
425+
* @param fileSizeThreshold file size threshold
426+
* @return list of data files
422427
*/
423-
public static List<DataFile> getDataFiles(Table table, String partitionPath) {
428+
public static List<DataFile> getDataFiles(Table table, String partitionPath, Long fileSizeThreshold) {
424429
CloseableIterable<FileScanTask> fileScanTasks =
425430
table.newScan().useSnapshot(table.currentSnapshot().snapshotId()).ignoreResiduals().planFiles();
426431
CloseableIterable<FileScanTask> filteredFileScanTasks =
427432
CloseableIterable.filter(fileScanTasks, t -> {
428433
DataFile file = t.asFileScanTask().file();
429-
return !table.spec().isPartitioned() ||
434+
return (!table.spec().isPartitioned() ||
430435
partitionPath == null && file.specId() != table.spec().specId() ||
431436
partitionPath != null &&
432-
table.specs().get(file.specId()).partitionToPath(file.partition()).equals(partitionPath);
437+
table.specs().get(file.specId()).partitionToPath(file.partition()).equals(partitionPath)) &&
438+
(fileSizeThreshold == null || file.fileSizeInBytes() < fileSizeThreshold);
433439
});
434440
return Lists.newArrayList(CloseableIterable.transform(filteredFileScanTasks, t -> t.file()));
435441
}
@@ -441,6 +447,7 @@ public static List<DataFile> getDataFiles(Table table, String partitionPath) {
441447
* 3. If partitionPath is provided, returns all delete files that belong to corresponding partition.
442448
* @param table the iceberg table
443449
* @param partitionPath partition path
450+
* @return list of delete files
444451
*/
445452
public static List<DeleteFile> getDeleteFiles(Table table, String partitionPath) {
446453
Table deletesTable =
@@ -654,4 +661,21 @@ public static Snapshot getTableSnapshot(org.apache.hadoop.hive.ql.metadata.Table
654661
}
655662
return snapshot;
656663
}
664+
665+
public static String getFileSizeTreshold(HiveConf hiveConf, CompactionType compactionType) throws HiveException {
666+
String fileSizeInBytesThreshold;
667+
switch (compactionType) {
668+
case MAJOR:
669+
fileSizeInBytesThreshold = HiveConf.getVar(hiveConf,
670+
HiveConf.ConfVars.HIVE_ICEBERG_MAJOR_COMPACTION_FILE_SIZE_THRESHOLD);
671+
break;
672+
case MINOR:
673+
fileSizeInBytesThreshold = HiveConf.getVar(hiveConf,
674+
HiveConf.ConfVars.HIVE_ICEBERG_MINOR_COMPACTION_FILE_SIZE_THRESHOLD);
675+
break;
676+
default:
677+
throw new HiveException(String.format("Invalid compaction type: %s", compactionType.name()));
678+
}
679+
return fileSizeInBytesThreshold;
680+
}
657681
}

iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/compaction/IcebergCompactionService.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -38,8 +38,8 @@ public IcebergCompactionService() {
3838

3939
public Boolean compact(Table table, CompactionInfo ci) throws Exception {
4040

41-
if (!ci.isMajorCompaction()) {
42-
ci.errorMessage = "Presently Iceberg tables support only Major compaction";
41+
if (!ci.isMajorCompaction() && !ci.isMinorCompaction()) {
42+
ci.errorMessage = String.format("Iceberg tables do not support %s compaction type", ci.type.name());
4343
LOG.error(ci.errorMessage + " Compaction info: {}", ci);
4444
try {
4545
msc.markRefused(CompactionInfo.compactionInfoToStruct(ci));
Lines changed: 23 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@
2626
import org.apache.hadoop.hive.conf.HiveConf;
2727
import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
2828
import org.apache.hadoop.hive.metastore.Warehouse;
29+
import org.apache.hadoop.hive.metastore.api.CompactionType;
2930
import org.apache.hadoop.hive.metastore.txn.entities.CompactionInfo;
3031
import org.apache.hadoop.hive.ql.Context.RewritePolicy;
3132
import org.apache.hadoop.hive.ql.DriverUtils;
@@ -43,9 +44,9 @@
4344
import org.slf4j.Logger;
4445
import org.slf4j.LoggerFactory;
4546

46-
public class IcebergMajorQueryCompactor extends QueryCompactor {
47+
public class IcebergQueryCompactor extends QueryCompactor {
4748

48-
private static final Logger LOG = LoggerFactory.getLogger(IcebergMajorQueryCompactor.class.getName());
49+
private static final Logger LOG = LoggerFactory.getLogger(IcebergQueryCompactor.class.getName());
4950

5051
@Override
5152
public boolean run(CompactorContext context) throws IOException, HiveException, InterruptedException {
@@ -62,20 +63,35 @@ public boolean run(CompactorContext context) throws IOException, HiveException,
6263
Table icebergTable = IcebergTableUtil.getTable(conf, table.getTTable());
6364
String compactionQuery;
6465
String orderBy = ci.orderByClause == null ? "" : ci.orderByClause;
66+
String fileSizeCond = null;
67+
68+
if (ci.type == CompactionType.MINOR) {
69+
String userProvidedFileSizeThreshold = context.getCompactionInfo()
70+
.getProperty(CompactorContext.COMPACTION_FILE_SIZE_THRESHOLD);
71+
Long fileSizeInBytesThreshold = HiveConf.toSizeBytes(userProvidedFileSizeThreshold == null ?
72+
IcebergTableUtil.getFileSizeTreshold(conf, ci.type) : userProvidedFileSizeThreshold);
73+
fileSizeCond = String.format("%1$s in (select file_path from %2$s.files where file_size_in_bytes < %3$d)",
74+
VirtualColumn.FILE_PATH.getName(), compactTableName, fileSizeInBytesThreshold);
75+
conf.setLong(CompactorContext.COMPACTION_FILE_SIZE_THRESHOLD, fileSizeInBytesThreshold);
76+
// IOW query containing a join with Iceberg .files metadata table fails with exception that Iceberg AVRO format
77+
// doesn't support vectorization, hence disabling it in this case.
78+
conf.setBoolVar(ConfVars.HIVE_VECTORIZATION_ENABLED, false);
79+
}
6580

6681
if (partSpec == null) {
6782
if (!icebergTable.spec().isPartitioned()) {
6883
HiveConf.setVar(conf, ConfVars.REWRITE_POLICY, RewritePolicy.FULL_TABLE.name());
69-
compactionQuery = String.format("insert overwrite table %s select * from %<s %2$s", compactTableName, orderBy);
84+
compactionQuery = String.format("insert overwrite table %s select * from %<s %2$s %3$s", compactTableName,
85+
fileSizeCond == null ? "" : "where " + fileSizeCond, orderBy);
7086
} else if (icebergTable.specs().size() > 1) {
7187
// Compacting partitions of old partition specs on a partitioned table with partition evolution
7288
HiveConf.setVar(conf, ConfVars.REWRITE_POLICY, RewritePolicy.PARTITION.name());
7389
// A single filter on a virtual column causes errors during compilation,
7490
// added another filter on file_path as a workaround.
7591
compactionQuery = String.format("insert overwrite table %1$s select * from %1$s " +
76-
"where %2$s != %3$d and %4$s is not null %5$s",
92+
"where %2$s != %3$d and %4$s is not null %5$s %6$s",
7793
compactTableName, VirtualColumn.PARTITION_SPEC_ID.getName(), icebergTable.spec().specId(),
78-
VirtualColumn.FILE_PATH.getName(), orderBy);
94+
VirtualColumn.FILE_PATH.getName(), fileSizeCond == null ? "" : "and " + fileSizeCond, orderBy);
7995
} else {
8096
// Partitioned table without partition evolution with partition spec as null in the compaction request - this
8197
// code branch is not supposed to be reachable
@@ -90,8 +106,8 @@ public boolean run(CompactorContext context) throws IOException, HiveException,
90106
Warehouse.makeSpecFromName(partSpecMap, new Path(partSpec), null);
91107

92108
compactionQuery = String.format("insert overwrite table %1$s select * from %1$s where %2$s=%3$d " +
93-
"and %4$s is not null %5$s", compactTableName, VirtualColumn.PARTITION_HASH.getName(), partitionHash,
94-
VirtualColumn.FILE_PATH.getName(), orderBy);
109+
"and %4$s is not null %5$s %6$s", compactTableName, VirtualColumn.PARTITION_HASH.getName(), partitionHash,
110+
VirtualColumn.FILE_PATH.getName(), fileSizeCond == null ? "" : "and " + fileSizeCond, orderBy);
95111
}
96112

97113
SessionState sessionState = setupQueryCompactionSession(conf, ci, tblProperties);
Lines changed: 48 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,48 @@
1+
-- SORT_QUERY_RESULTS
2+
-- Mask neededVirtualColumns due to non-strict order
3+
--! qt:replace:/(\s+neededVirtualColumns:\s)(.*)/$1#Masked#/
4+
-- Mask random uuid
5+
--! qt:replace:/(\s+uuid\s+)\S+(\s*)/$1#Masked#$2/
6+
-- Mask a random snapshot id
7+
--! qt:replace:/(\s+current-snapshot-id\s+)\S+(\s*)/$1#Masked#/
8+
-- Mask added file size
9+
--! qt:replace:/(\S\"added-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
10+
-- Mask total file size
11+
--! qt:replace:/(\S\"total-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
12+
-- Mask current-snapshot-timestamp-ms
13+
--! qt:replace:/(\s+current-snapshot-timestamp-ms\s+)\S+(\s*)/$1#Masked#$2/
14+
-- Mask the enqueue time which is based on current time
15+
--! qt:replace:/(MINOR\s+succeeded\s+)[a-zA-Z0-9\-\.\s+]+(\s+manual)/$1#Masked#$2/
16+
-- Mask compaction id as they will be allocated in parallel threads
17+
--! qt:replace:/^[0-9]/#Masked#/
18+
-- Mask removed file size
19+
--! qt:replace:/(\S\"removed-files-size\\\":\\\")(\d+)(\\\")/$1#Masked#$3/
20+
-- Mask iceberg version
21+
--! qt:replace:/(\S\"iceberg-version\\\":\\\")(\w+\s\w+\s\d+\.\d+\.\d+\s\(\w+\s\w+\))(\\\")/$1#Masked#$3/
22+
23+
set hive.llap.io.enabled=true;
24+
set hive.vectorized.execution.enabled=true;
25+
set hive.optimize.shared.work.merge.ts.schema=true;
26+
27+
create table ice_orc (
28+
first_name string,
29+
last_name string
30+
)
31+
stored by iceberg stored as orc
32+
tblproperties ('format-version'='2');
33+
34+
insert into ice_orc VALUES ('fn1','ln1');
35+
insert into ice_orc VALUES ('fn2','ln2'), ('fn3','ln3');
36+
insert into ice_orc VALUES ('fn4','ln4'), ('fn5','ln5'), ('fn6','ln6'), ('fn7','ln7');
37+
38+
select * from ice_orc;
39+
describe formatted ice_orc;
40+
41+
explain alter table ice_orc COMPACT 'minor' and wait file_size_threshold = '365bytes' pool 'iceberg';
42+
explain optimize table ice_orc rewrite data file_size_threshold = '365bytes' pool 'iceberg';
43+
44+
alter table ice_orc COMPACT 'minor' and wait file_size_threshold = '365bytes' pool 'iceberg';
45+
46+
select * from ice_orc;
47+
describe formatted ice_orc;
48+
show compactions;

0 commit comments

Comments
 (0)