diff --git a/ams/ams-api/src/main/gen-java/com/netease/arctic/ams/api/OptimizeType.java b/ams/ams-api/src/main/gen-java/com/netease/arctic/ams/api/OptimizeType.java index f7ce2cfa67..54d474d56c 100644 --- a/ams/ams-api/src/main/gen-java/com/netease/arctic/ams/api/OptimizeType.java +++ b/ams/ams-api/src/main/gen-java/com/netease/arctic/ams/api/OptimizeType.java @@ -7,10 +7,11 @@ package com.netease.arctic.ams.api; -@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2022-08-22") +@javax.annotation.Generated(value = "Autogenerated by Thrift Compiler (0.13.0)", date = "2022-08-18") public enum OptimizeType implements org.apache.thrift.TEnum { Minor(0), - Major(1); + Major(1), + FullMajor(2); private final int value; @@ -36,6 +37,8 @@ public static OptimizeType findByValue(int value) { return Minor; case 1: return Major; + case 2: + return FullMajor; default: return null; } diff --git a/ams/ams-api/src/main/thrift/arctic_optimize_manager.thrift b/ams/ams-api/src/main/thrift/arctic_optimize_manager.thrift index 3cb5c63489..ae7dd34cb9 100644 --- a/ams/ams-api/src/main/thrift/arctic_optimize_manager.thrift +++ b/ams/ams-api/src/main/thrift/arctic_optimize_manager.thrift @@ -63,7 +63,8 @@ enum JobType { enum OptimizeType { Minor, - Major + Major, + FullMajor } diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/ArcticMetaStore.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/ArcticMetaStore.java index 3e3bbc00fe..54e6dad2f2 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/ArcticMetaStore.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/ArcticMetaStore.java @@ -229,6 +229,7 @@ private static void startMetaStoreThreads( startOptimizeCommit(conf.getInteger(ArcticMetaStoreConf.OPTIMIZE_COMMIT_THREAD_POOL_SIZE)); startExpiredClean(); startOrphanClean(); + startSupportHiveSync(); monitorOptimizerStatus(); tableRuntimeDataExpire(); AmsRestServer.startRestServer(httpPort); @@ -302,6 +303,14 @@ private static void startOrphanClean() { TimeUnit.MILLISECONDS); } + private static void startSupportHiveSync() { + ThreadPool.getPool(ThreadPool.Type.HIVE_SYNC).scheduleWithFixedDelay( + ServiceContainer.getSupportHiveSyncService()::checkHiveSyncTasks, + 3 * 1000L, + 60 * 1000L, + TimeUnit.MILLISECONDS); + } + private static void monitorOptimizerStatus() { OptimizeExecuteService.OptimizerMonitor monitor = new OptimizeExecuteService.OptimizerMonitor(); ThreadPool.getPool(ThreadPool.Type.OPTIMIZER_MONITOR).scheduleWithFixedDelay( diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/config/ArcticMetaStoreConf.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/config/ArcticMetaStoreConf.java index 054f90e7b1..28fd4252e3 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/config/ArcticMetaStoreConf.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/config/ArcticMetaStoreConf.java @@ -92,6 +92,12 @@ public class ArcticMetaStoreConf { .defaultValue(10) .withDescription("Number of threads in the thread pool. " + "These will be used to execute all orphan file clean processes."); + public static final ConfigOption SUPPORT_HIVE_SYNC_THREAD_POOL_SIZE = + ConfigOptions.key("arctic.ams.support.hive.sync.thread.pool-size") + .intType() + .defaultValue(10) + .withDescription("Number of threads in the thread pool. " + + "These will be used to execute all support hive sync processes."); public static final ConfigOption SYNC_FILE_INFO_CACHE_THREAD_POOL_SIZE = ConfigOptions.key("arctic.ams.file.sync.thread.pool-size") .intType() diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/mapper/OptimizeTasksMapper.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/mapper/OptimizeTasksMapper.java index 33875f81a7..1eb84cb6a8 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/mapper/OptimizeTasksMapper.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/mapper/OptimizeTasksMapper.java @@ -37,7 +37,7 @@ public interface OptimizeTasksMapper { @Select("select trace_id, optimize_type, catalog_name, db_name, table_name, `partition`," + " task_group, task_history_id, max_change_transaction_id, is_delete_pos_delete," + - " source_nodes, create_time, properties, queue_id, " + + " source_nodes, create_time, properties, queue_id," + " insert_file_size, delete_file_size, base_file_size, pos_delete_file_size," + " insert_files, delete_files, base_files, pos_delete_files" + " from " + TABLE_NAME) diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/mapper/TableOptimizeRuntimeMapper.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/mapper/TableOptimizeRuntimeMapper.java index 4d4ccebb9d..8ed627ab56 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/mapper/TableOptimizeRuntimeMapper.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/mapper/TableOptimizeRuntimeMapper.java @@ -38,7 +38,7 @@ public interface TableOptimizeRuntimeMapper { String TABLE_NAME = "optimize_table_runtime"; @Select("select catalog_name, db_name, table_name, current_snapshot_id, current_change_snapshotId," + - " latest_major_optimize_time, latest_minor_optimize_time, latest_task_history_id," + + " latest_major_optimize_time, latest_full_optimize_time, latest_minor_optimize_time, latest_task_history_id," + " optimize_status, optimize_status_start_time " + " from " + TABLE_NAME) @ConstructorArgs({ @@ -51,6 +51,8 @@ public interface TableOptimizeRuntimeMapper { @Result(property = "currentChangeSnapshotId", column = "current_change_snapshotId"), @Result(property = "latestMajorOptimizeTime", column = "latest_major_optimize_time", typeHandler = MapLong2StringConverter.class), + @Result(property = "latestFullOptimizeTime", column = "latest_full_optimize_time", + typeHandler = MapLong2StringConverter.class), @Result(property = "latestMinorOptimizeTime", column = "latest_minor_optimize_time", typeHandler = MapLong2StringConverter.class), @Result(property = "latestTaskHistoryId", column = "latest_task_history_id"), @@ -65,6 +67,8 @@ public interface TableOptimizeRuntimeMapper { "current_change_snapshotId = #{runtime.currentChangeSnapshotId}, " + "latest_major_optimize_time = #{runtime.latestMajorOptimizeTime, " + "typeHandler=com.netease.arctic.ams.server.mybatis.MapLong2StringConverter}, " + + "latest_full_optimize_time = #{runtime.latestFullOptimizeTime, " + + "typeHandler=com.netease.arctic.ams.server.mybatis.MapLong2StringConverter}, " + "latest_minor_optimize_time = #{runtime.latestMinorOptimizeTime, " + "typeHandler=com.netease.arctic.ams.server.mybatis.MapLong2StringConverter}, " + "latest_task_history_id = #{runtime.latestTaskHistoryId, jdbcType=VARCHAR}, " + @@ -84,8 +88,8 @@ public interface TableOptimizeRuntimeMapper { void deleteTableOptimizeRuntime(@Param("tableIdentifier") TableIdentifier tableIdentifier); @Insert("insert into " + TABLE_NAME + " (catalog_name, db_name, table_name, " + - "current_snapshot_id, current_change_snapshotId, latest_major_optimize_time, latest_minor_optimize_time, " + - "latest_task_history_id, optimize_status, optimize_status_start_time) values ( " + + "current_snapshot_id, current_change_snapshotId, latest_major_optimize_time, latest_full_optimize_time, " + + "latest_minor_optimize_time, latest_task_history_id, optimize_status, optimize_status_start_time) values ( " + "#{runtime.tableIdentifier.catalog}, " + "#{runtime.tableIdentifier.database}, " + "#{runtime.tableIdentifier.tableName}, " + @@ -93,6 +97,8 @@ public interface TableOptimizeRuntimeMapper { "#{runtime.currentChangeSnapshotId}, " + "#{runtime.latestMajorOptimizeTime, " + "typeHandler=com.netease.arctic.ams.server.mybatis.MapLong2StringConverter}," + + "#{runtime.latestFullOptimizeTime, " + + "typeHandler=com.netease.arctic.ams.server.mybatis.MapLong2StringConverter}," + "#{runtime.latestMinorOptimizeTime, " + "typeHandler=com.netease.arctic.ams.server.mybatis.MapLong2StringConverter}," + "#{runtime.latestTaskHistoryId, jdbcType=VARCHAR}," + diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/BaseOptimizeTask.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/BaseOptimizeTask.java index c1b81405d2..6fd74dfcc6 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/BaseOptimizeTask.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/BaseOptimizeTask.java @@ -41,6 +41,7 @@ public class BaseOptimizeTask extends OptimizeTask { protected long createTime; private long maxChangeTransactionId = INVALID_TRANSACTION_ID; + @Deprecated private int isDeletePosDelete; public BaseOptimizeTask() { @@ -70,7 +71,6 @@ public void setMaxChangeTransactionId(long maxChangeTransactionId) { this.maxChangeTransactionId = maxChangeTransactionId; } - public String getPartition() { return partition; } @@ -143,10 +143,12 @@ public void setDeleteFileCnt(int deleteFileCnt) { this.deleteFileCnt = deleteFileCnt; } + @Deprecated public int getIsDeletePosDelete() { return isDeletePosDelete; } + @Deprecated public void setIsDeletePosDelete(int isDeletePosDelete) { this.isDeletePosDelete = isDeletePosDelete; } diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/FileTree.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/FileTree.java index b43820b14d..afe364975c 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/FileTree.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/FileTree.java @@ -152,20 +152,20 @@ public void collectDeleteFiles(List collector) { } public void collectBaseFiles(List collector) { - collectBaseFiles(collector, false, 0); + collectBaseFiles(collector, false, Collections.emptyList()); } - public void collectBaseFiles(List collector, boolean isFilterSmallFile, long smallFileSize) { - if (isFilterSmallFile) { + public void collectBaseFiles(List collector, boolean isMajor, List needOptimizeFiles) { + if (isMajor) { baseFiles = baseFiles.stream() - .filter(dataFile -> dataFile.fileSizeInBytes() < smallFileSize).collect(Collectors.toList()); + .filter(needOptimizeFiles::contains).collect(Collectors.toList()); } collector.addAll(baseFiles); if (left != null) { - left.collectBaseFiles(collector, isFilterSmallFile, smallFileSize); + left.collectBaseFiles(collector, isMajor, needOptimizeFiles); } if (right != null) { - right.collectBaseFiles(collector, isFilterSmallFile, smallFileSize); + right.collectBaseFiles(collector, isMajor, needOptimizeFiles); } } diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/TableOptimizeRuntime.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/TableOptimizeRuntime.java index 83897e75c1..b25334d946 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/TableOptimizeRuntime.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/TableOptimizeRuntime.java @@ -36,7 +36,9 @@ public class TableOptimizeRuntime { private long currentChangeSnapshotId = INVALID_SNAPSHOT_ID; private TableOptimizeInfo.OptimizeStatus optimizeStatus = TableOptimizeInfo.OptimizeStatus.Idle; private long optimizeStatusStartTime = -1; + private final Map latestMajorOptimizeTime = new HashMap<>(); + private final Map latestFullOptimizeTime = new HashMap<>(); private final Map latestMinorOptimizeTime = new HashMap<>(); private String latestTaskHistoryId; private volatile boolean isRunning; @@ -77,6 +79,15 @@ public void putLatestMajorOptimizeTime(String partition, long time) { } } + public void putLatestFullOptimizeTime(String partition, long time) { + Long oldValue = latestFullOptimizeTime.putIfAbsent(partition, time); + if (oldValue != null) { + if (time > oldValue) { + latestFullOptimizeTime.put(partition, time); + } + } + } + public TableOptimizeInfo.OptimizeStatus getOptimizeStatus() { return optimizeStatus; } @@ -99,6 +110,11 @@ public long getLatestMajorOptimizeTime(String partition) { return time == null ? -1 : time; } + public long getLatestFullOptimizeTime(String partition) { + Long time = latestFullOptimizeTime.get(partition); + return time == null ? -1 : time; + } + public void putLatestMinorOptimizeTime(String partition, long time) { Long oldValue = latestMinorOptimizeTime.putIfAbsent(partition, time); if (oldValue != null) { @@ -118,6 +134,9 @@ public Set getPartitions() { if (MapUtils.isNotEmpty(latestMajorOptimizeTime)) { result.addAll(latestMajorOptimizeTime.keySet()); } + if (MapUtils.isNotEmpty(latestFullOptimizeTime)) { + result.addAll(latestFullOptimizeTime.keySet()); + } if (MapUtils.isNotEmpty(latestMinorOptimizeTime)) { result.addAll(latestMinorOptimizeTime.keySet()); } @@ -158,6 +177,7 @@ public String toString() { ", optimizeStatus=" + optimizeStatus + ", optimizeStatusStartTime=" + optimizeStatusStartTime + ", latestMajorOptimizeTime=" + latestMajorOptimizeTime + + ", latestFullOptimizeTime=" + latestFullOptimizeTime + ", latestMinorOptimizeTime=" + latestMinorOptimizeTime + ", latestTaskHistoryId='" + latestTaskHistoryId + '\'' + ", isRunning=" + isRunning + diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/TaskConfig.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/TaskConfig.java index d1873ddd93..b2fa6d19ba 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/TaskConfig.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/model/TaskConfig.java @@ -31,7 +31,8 @@ public class TaskConfig { private final String historyId; private final long createTime; - public TaskConfig(String partition, @Nullable Long maxTransactionId, String group, String historyId, + public TaskConfig(String partition, @Nullable Long maxTransactionId, + String group, String historyId, OptimizeType optimizeType, long createTime) { this.optimizeType = optimizeType; this.partition = partition; @@ -49,6 +50,7 @@ public String getPartition() { return partition; } + @org.jetbrains.annotations.Nullable public Long getMaxTransactionId() { return maxTransactionId; } diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseOptimizeCommit.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseOptimizeCommit.java index 8bf7972cdc..69846568a7 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseOptimizeCommit.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseOptimizeCommit.java @@ -35,6 +35,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; import org.apache.iceberg.OverwriteFiles; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RewriteFiles; @@ -55,10 +56,10 @@ public class BaseOptimizeCommit { private static final Logger LOG = LoggerFactory.getLogger(BaseOptimizeCommit.class); - private final ArcticTable arcticTable; - private final Map> optimizeTasksToCommit; - private final Map commitTableTaskHistory = new HashMap<>(); - private final Map partitionOptimizeType = new HashMap<>(); + protected final ArcticTable arcticTable; + protected final Map> optimizeTasksToCommit; + protected final Map commitTableTaskHistory = new HashMap<>(); + protected final Map partitionOptimizeType = new HashMap<>(); public BaseOptimizeCommit(ArcticTable arcticTable, Map> optimizeTasksToCommit) { this.arcticTable = arcticTable; @@ -114,7 +115,7 @@ public long commit(TableOptimizeRuntime tableOptimizeRuntime) throws Exception { .map(SerializationUtil::toInternalTableFile) .forEach(majorAddFiles::add); majorDeleteFiles.addAll(selectDeletedFiles(task.getOptimizeTask(), new HashSet<>())); - partitionOptimizeType.put(entry.getKey(), OptimizeType.Major); + partitionOptimizeType.put(entry.getKey(), task.getOptimizeTask().getTaskId().getType()); } String taskGroupId = task.getOptimizeTask().getTaskGroup(); @@ -153,7 +154,7 @@ public long commit(TableOptimizeRuntime tableOptimizeRuntime) throws Exception { overwriteBaseFiles.set(SnapshotSummary.SNAPSHOT_PRODUCER, CommitMetaProducer.OPTIMIZE.name()); AtomicInteger addedPosDeleteFile = new AtomicInteger(0); minorAddFiles.forEach(contentFile -> { - if (contentFile instanceof DataFile) { + if (contentFile.content() == FileContent.DATA) { overwriteBaseFiles.addFile((DataFile) contentFile); } else { overwriteBaseFiles.addFile((DeleteFile) contentFile); @@ -163,7 +164,7 @@ public long commit(TableOptimizeRuntime tableOptimizeRuntime) throws Exception { AtomicInteger deletedPosDeleteFile = new AtomicInteger(0); Set deletedPosDeleteFiles = new HashSet<>(); minorDeleteFiles.forEach(contentFile -> { - if (contentFile instanceof DataFile) { + if (contentFile.content() == FileContent.DATA) { overwriteBaseFiles.deleteFile((DataFile) contentFile); } else { deletedPosDeleteFiles.add((DeleteFile) contentFile); @@ -200,14 +201,14 @@ public long commit(TableOptimizeRuntime tableOptimizeRuntime) throws Exception { // commit major optimize content if (CollectionUtils.isNotEmpty(majorAddFiles) || CollectionUtils.isNotEmpty(majorDeleteFiles)) { Set addDataFiles = majorAddFiles.stream().map(contentFile -> { - if (contentFile instanceof DataFile) { + if (contentFile.content() == FileContent.DATA) { return (DataFile) contentFile; } return null; }).filter(Objects::nonNull).collect(Collectors.toSet()); Set addDeleteFiles = majorAddFiles.stream().map(contentFile -> { - if (contentFile instanceof DeleteFile) { + if (contentFile.content() == FileContent.POSITION_DELETES) { return (DeleteFile) contentFile; } @@ -215,14 +216,14 @@ public long commit(TableOptimizeRuntime tableOptimizeRuntime) throws Exception { }).filter(Objects::nonNull).collect(Collectors.toSet()); Set deleteDataFiles = majorDeleteFiles.stream().map(contentFile -> { - if (contentFile instanceof DataFile) { + if (contentFile.content() == FileContent.DATA) { return (DataFile) contentFile; } return null; }).filter(Objects::nonNull).collect(Collectors.toSet()); Set deleteDeleteFiles = majorDeleteFiles.stream().map(contentFile -> { - if (contentFile instanceof DeleteFile) { + if (contentFile.content() == FileContent.POSITION_DELETES) { return (DeleteFile) contentFile; } @@ -232,20 +233,27 @@ public long commit(TableOptimizeRuntime tableOptimizeRuntime) throws Exception { if (!addDeleteFiles.isEmpty()) { throw new IllegalArgumentException("for major optimize, can't add delete files " + addDeleteFiles); } - if (deleteDeleteFiles.isEmpty()) { - OverwriteFiles overwriteFiles = baseArcticTable.newOverwrite(); - overwriteFiles.set(SnapshotSummary.SNAPSHOT_PRODUCER, CommitMetaProducer.OPTIMIZE.name()); - deleteDataFiles.forEach(overwriteFiles::deleteFile); - addDataFiles.forEach(overwriteFiles::addFile); - overwriteFiles.commit(); - } else { + + // overwrite DataFiles + OverwriteFiles overwriteFiles = baseArcticTable.newOverwrite(); + overwriteFiles.set(SnapshotSummary.SNAPSHOT_PRODUCER, CommitMetaProducer.OPTIMIZE.name()); + deleteDataFiles.forEach(overwriteFiles::deleteFile); + addDataFiles.forEach(overwriteFiles::addFile); + overwriteFiles.commit(); + + // remove DeleteFiles + if (CollectionUtils.isNotEmpty(deleteDeleteFiles)) { RewriteFiles rewriteFiles = baseArcticTable.newRewrite() .validateFromSnapshot(baseArcticTable.currentSnapshot().snapshotId()); rewriteFiles.set(SnapshotSummary.SNAPSHOT_PRODUCER, CommitMetaProducer.OPTIMIZE.name()); - rewriteFiles.rewriteFiles(deleteDataFiles, deleteDeleteFiles, addDataFiles, addDeleteFiles); - rewriteFiles.commit(); + rewriteFiles.rewriteFiles(Collections.emptySet(), deleteDeleteFiles, Collections.emptySet(), addDeleteFiles); + try { + rewriteFiles.commit(); + } catch (ValidationException e) { + LOG.warn("Iceberg RewriteFiles commit failed, but ignore", e); + } } - + LOG.info("{} major optimize committed, delete {} files [{} posDelete files], " + "add {} new files [{} posDelete files]", arcticTable.id(), majorDeleteFiles.size(), deleteDeleteFiles.size(), majorAddFiles.size(), @@ -276,6 +284,7 @@ public Map getPartitionOptimizeType() { private static Set> selectDeletedFiles(BaseOptimizeTask optimizeTask, Set> addPosDeleteFiles) { switch (optimizeTask.getTaskId().getType()) { + case FullMajor: case Major: return selectMajorOptimizeDeletedFiles(optimizeTask); case Minor: @@ -288,7 +297,7 @@ private static Set> selectDeletedFiles(BaseOptimizeTask optimizeT private static Set> selectMinorOptimizeDeletedFiles(BaseOptimizeTask optimizeTask, Set> addPosDeleteFiles) { Set newFileNodes = addPosDeleteFiles.stream().map(contentFile -> { - if (contentFile instanceof DeleteFile) { + if (contentFile.content() == FileContent.POSITION_DELETES) { return DefaultKeyedFile.parseMetaFromFileName(contentFile.path().toString()).node(); } @@ -306,7 +315,7 @@ private static Set> selectMajorOptimizeDeletedFiles(BaseOptimizeT Set> result = optimizeTask.getBaseFiles().stream() .map(SerializationUtil::toInternalTableFile).collect(Collectors.toSet()); - if (optimizeTask.getIsDeletePosDelete() == 1) { + if (optimizeTask.getTaskId().getType() == OptimizeType.FullMajor) { result.addAll(optimizeTask.getPosDeleteFiles().stream() .map(SerializationUtil::toInternalTableFile).collect(Collectors.toSet())); } diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseOptimizePlan.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseOptimizePlan.java index 7e10e8a046..4aa3e8ff65 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseOptimizePlan.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseOptimizePlan.java @@ -70,6 +70,8 @@ public abstract class BaseOptimizePlan { protected final Map partitionFileTree = new LinkedHashMap<>(); // partition -> position delete file protected final Map> partitionPosDeleteFiles = new LinkedHashMap<>(); + // partition -> optimize type(FullMajor or Major or Minor) + protected final Map partitionOptimizeType = new HashMap<>(); // We store current partitions, for the next plans to decide if any partition reach the max plan interval, // if not, the new added partitions will be ignored by mistake. // After plan files, current partitions of table will be set. @@ -77,9 +79,9 @@ public abstract class BaseOptimizePlan { protected final Set allPartitions = new HashSet<>(); // for base table or unKeyed table - private long currentBaseSnapshotId = TableOptimizeRuntime.INVALID_SNAPSHOT_ID; + protected long currentBaseSnapshotId = TableOptimizeRuntime.INVALID_SNAPSHOT_ID; // for change table - private long currentChangeSnapshotId = TableOptimizeRuntime.INVALID_SNAPSHOT_ID; + protected long currentChangeSnapshotId = TableOptimizeRuntime.INVALID_SNAPSHOT_ID; // for check iceberg base table current snapshot whether cached in file cache protected Predicate snapshotIsCached; @@ -101,14 +103,16 @@ public BaseOptimizePlan(ArcticTable arcticTable, TableOptimizeRuntime tableOptim this.historyId = UUID.randomUUID().toString(); } - public abstract boolean partitionNeedPlan(String partitionToPath); + protected abstract boolean partitionNeedPlan(String partitionToPath); - public abstract void addOptimizeFilesTree(); + protected abstract void addOptimizeFilesTree(); protected abstract OptimizeType getOptimizeType(); protected abstract List collectTask(String partition); + protected abstract boolean tableChanged(); + public List plan() { long startTime = System.nanoTime(); @@ -168,7 +172,7 @@ public List collectTasks() { return results; } - public BaseOptimizeTask buildOptimizeTask(@Nullable List sourceNodes, + protected BaseOptimizeTask buildOptimizeTask(@Nullable List sourceNodes, List insertFiles, List deleteFiles, List baseFiles, @@ -246,8 +250,7 @@ public BaseOptimizeTask buildOptimizeTask(@Nullable List sourceNod if (taskConfig.getMaxTransactionId() != null) { optimizeTask.setMaxChangeTransactionId(taskConfig.getMaxTransactionId()); } - if (taskConfig.getOptimizeType() == OptimizeType.Major && - ((MajorOptimizePlan) this).isDeletePosDelete(taskConfig.getPartition())) { + if (taskConfig.getOptimizeType() == OptimizeType.FullMajor) { optimizeTask.setIsDeletePosDelete(1); } @@ -259,6 +262,10 @@ public BaseOptimizeTask buildOptimizeTask(@Nullable List sourceNod return optimizeTask; } + public Map getPartitionOptimizeType() { + return partitionOptimizeType; + } + public boolean baseTableCacheAll() { Snapshot snapshot; if (arcticTable.isKeyedTable()) { @@ -304,48 +311,6 @@ public long getCurrentChangeSnapshotId() { return currentChangeSnapshotId; } - private boolean tableChanged() { - if (this instanceof MajorOptimizePlan) { - return baseTableChanged(); - } else { - return changeTableChanged(); - } - } - - private boolean baseTableChanged() { - long lastBaseSnapshotId = tableOptimizeRuntime.getCurrentSnapshotId(); - Snapshot snapshot; - if (arcticTable.isKeyedTable()) { - snapshot = arcticTable.asKeyedTable().baseTable().currentSnapshot(); - } else { - snapshot = arcticTable.asUnkeyedTable().currentSnapshot(); - } - - if (snapshot != null) { - boolean findNewData = false; - if (snapshot.snapshotId() != lastBaseSnapshotId) { - findNewData = true; - LOG.debug("{} ==== {} find {} data in base snapshot={}", tableId(), getOptimizeType(), snapshot.operation(), - snapshot.snapshotId()); - } - - // If last snapshot not exist(may expire), then skip compaction, - // because compaction check interval is much shorter than expire time. - // Set table properties compact.major.force=true, if compaction is needed. - return findNewData; - } else { - LOG.warn("{} {} base snapshots is null, regard as table not changed", tableId(), getOptimizeType()); - return false; - } - } - - private boolean changeTableChanged() { - long lastChangeSnapshotId = tableOptimizeRuntime.getCurrentChangeSnapshotId(); - LOG.debug("{} ==== {} currentChangeSnapshotId={}, lastChangeSnapshotId={}", tableId(), getOptimizeType(), - currentChangeSnapshotId, lastChangeSnapshotId); - return currentChangeSnapshotId != lastChangeSnapshotId; - } - protected boolean anyTaskRunning(String partition) { return partitionTaskRunning.get(partition) != null && partitionTaskRunning.get(partition); } diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/MajorOptimizePlan.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/MajorOptimizePlan.java index 15d40df4f1..fa628a3ae4 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/MajorOptimizePlan.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/MajorOptimizePlan.java @@ -36,7 +36,9 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.util.BinPacking; import org.apache.iceberg.util.PropertyUtil; import org.slf4j.Logger; @@ -44,7 +46,7 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.LinkedHashMap; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -57,11 +59,10 @@ public class MajorOptimizePlan extends BaseOptimizePlan { private static final Logger LOG = LoggerFactory.getLogger(MajorOptimizePlan.class); - // partition -> small file - protected final Map> partitionSmallFiles = new LinkedHashMap<>(); - - // partition -> current optimize is merge pos-delete or only merge small files - protected final Map isMergePosDelete = new LinkedHashMap<>(); + // partition -> need major optimize files + // for keyed unSupport hive table, the files are small data files in iceberg base store + // for keyed support hive table, the files are all data files in iceberg base store not in hive store + protected final Map> partitionNeedMajorOptimizeFiles = new HashMap<>(); public MajorOptimizePlan(ArcticTable arcticTable, TableOptimizeRuntime tableOptimizeRuntime, List baseTableFileList, List posDeleteFileList, @@ -86,30 +87,28 @@ public boolean partitionNeedPlan(String partitionToPath) { long current = System.currentTimeMillis(); // check position delete file total size - long posDeleteSize = posDeleteFileList.stream().mapToLong(DataFileInfo::getSize).sum(); - if (posDeleteSize >= PropertyUtil.propertyAsLong(arcticTable.properties(), - TableProperties.MAJOR_OPTIMIZE_TRIGGER_DELETE_FILE_SIZE_BYTES, - TableProperties.MAJOR_OPTIMIZE_TRIGGER_DELETE_FILE_SIZE_BYTES_DEFAULT)) { - isMergePosDelete.put(partitionToPath, true); + if (checkPosDeleteTotalSize(partitionToPath)) { + partitionOptimizeType.put(partitionToPath, OptimizeType.FullMajor); + return true; + } + + // check full major optimize interval + if (checkFullMajorOptimizeInterval(current, partitionToPath)) { + partitionOptimizeType.put(partitionToPath, OptimizeType.FullMajor); return true; } // check small data file count - List dataFileList = partitionSmallFiles.get(partitionToPath); - if (CollectionUtils.isNotEmpty(dataFileList)) { - // file count - if (dataFileList.size() >= PropertyUtil.propertyAsInt(arcticTable.properties(), - TableProperties.MAJOR_OPTIMIZE_TRIGGER_SMALL_FILE_COUNT, - TableProperties.MAJOR_OPTIMIZE_TRIGGER_SMALL_FILE_COUNT_DEFAULT)) { - return true; - } + if (checkSmallFileCount(partitionToPath, + partitionNeedMajorOptimizeFiles.getOrDefault(partitionToPath, new ArrayList<>()))) { + partitionOptimizeType.put(partitionToPath, OptimizeType.Major); + return true; + } - // optimize interval - if (current - tableOptimizeRuntime.getLatestMajorOptimizeTime(partitionToPath) >= - PropertyUtil.propertyAsLong(arcticTable.properties(), TableProperties.MAJOR_OPTIMIZE_TRIGGER_MAX_INTERVAL, - TableProperties.MAJOR_OPTIMIZE_TRIGGER_MAX_INTERVAL_DEFAULT) && dataFileList.size() >= 2) { - return true; - } + // check major optimize interval + if (checkMajorOptimizeInterval(current, partitionToPath)) { + partitionOptimizeType.put(partitionToPath, OptimizeType.Major); + return true; } LOG.debug("{} ==== don't need {} optimize plan, skip partition {}", tableId(), getOptimizeType(), partitionToPath); @@ -119,16 +118,20 @@ public boolean partitionNeedPlan(String partitionToPath) { protected List collectTask(String partition) { List result; if (arcticTable.isUnkeyedTable()) { - List fileList = partitionSmallFiles.computeIfAbsent(partition, e -> new ArrayList<>()); + // if Major, only optimize partitionNeedMajorOptimizeFiles. + // if Full Major, optimize all files in file tree. + List fileList = partitionOptimizeType.get(partition) == OptimizeType.Major ? + partitionNeedMajorOptimizeFiles.computeIfAbsent(partition, e -> new ArrayList<>()) : + partitionFileTree.get(partition).getBaseFiles(); result = collectUnKeyedTableTasks(partition, fileList); // init files - partitionSmallFiles.put(partition, Collections.emptyList()); + partitionNeedMajorOptimizeFiles.put(partition, Collections.emptyList()); partitionFileTree.get(partition).initFiles(); } else { FileTree treeRoot = partitionFileTree.get(partition); result = collectKeyedTableTasks(partition, treeRoot); // init files - partitionSmallFiles.put(partition, Collections.emptyList()); + partitionNeedMajorOptimizeFiles.put(partition, Collections.emptyList()); partitionPosDeleteFiles.put(partition, Collections.emptyList()); partitionFileTree.get(partition).initFiles(); } @@ -136,8 +139,95 @@ protected List collectTask(String partition) { return result; } - public boolean isDeletePosDelete(String partition) { - return isMergePosDelete.containsKey(partition); + @Override + protected boolean tableChanged() { + return baseTableChanged(); + } + + protected boolean checkPosDeleteTotalSize(String partitionToPath) { + long posDeleteSize = partitionPosDeleteFiles.get(partitionToPath) == null ? + 0 : partitionPosDeleteFiles.get(partitionToPath).stream().mapToLong(DeleteFile::fileSizeInBytes).sum(); + return posDeleteSize >= PropertyUtil.propertyAsLong(arcticTable.properties(), + TableProperties.MAJOR_OPTIMIZE_TRIGGER_DELETE_FILE_SIZE_BYTES, + TableProperties.MAJOR_OPTIMIZE_TRIGGER_DELETE_FILE_SIZE_BYTES_DEFAULT); + } + + protected boolean checkFullMajorOptimizeInterval(long current, String partitionToPath) { + long fullMajorOptimizeInterval = PropertyUtil.propertyAsLong(arcticTable.properties(), + TableProperties.FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL, + TableProperties.FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL_DEFAULT); + + if (fullMajorOptimizeInterval != TableProperties.FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL_DEFAULT) { + long lastFullMajorOptimizeTime = tableOptimizeRuntime.getLatestFullOptimizeTime(partitionToPath); + return current - lastFullMajorOptimizeTime >= fullMajorOptimizeInterval; + } + + return false; + } + + protected boolean checkMajorOptimizeInterval(long current, String partitionToPath) { + if (current - tableOptimizeRuntime.getLatestMajorOptimizeTime(partitionToPath) >= + PropertyUtil.propertyAsLong(arcticTable.properties(), TableProperties.MAJOR_OPTIMIZE_TRIGGER_MAX_INTERVAL, + TableProperties.MAJOR_OPTIMIZE_TRIGGER_MAX_INTERVAL_DEFAULT)) { + long fileCount = partitionNeedMajorOptimizeFiles.get(partitionToPath) == null ? + 0 : partitionNeedMajorOptimizeFiles.get(partitionToPath).size(); + + // no need to optimize until have 2 files at least + return fileCount >= 2; + } + + return false; + } + + protected boolean checkSmallFileCount(String partition, List dataFileList) { + if (CollectionUtils.isNotEmpty(dataFileList)) { + // file count + return dataFileList.size() >= PropertyUtil.propertyAsInt(arcticTable.properties(), + TableProperties.MAJOR_OPTIMIZE_TRIGGER_SMALL_FILE_COUNT, + TableProperties.MAJOR_OPTIMIZE_TRIGGER_SMALL_FILE_COUNT_DEFAULT); + } + + return false; + } + + protected void fillPartitionNeedOptimizeFiles(String partition, ContentFile contentFile) { + // fill partition need optimize file map + // add small files in iceberg base store and not in hive store + boolean isSmallFile = contentFile.fileSizeInBytes() < PropertyUtil.propertyAsLong(arcticTable.properties(), + TableProperties.OPTIMIZE_SMALL_FILE_SIZE_BYTES_THRESHOLD, + TableProperties.OPTIMIZE_SMALL_FILE_SIZE_BYTES_THRESHOLD_DEFAULT); + if (isSmallFile) { + List files = partitionNeedMajorOptimizeFiles.computeIfAbsent(partition, e -> new ArrayList<>()); + files.add((DataFile) contentFile); + partitionNeedMajorOptimizeFiles.put(partition, files); + } + } + + private boolean baseTableChanged() { + long lastBaseSnapshotId = tableOptimizeRuntime.getCurrentSnapshotId(); + Snapshot snapshot; + if (arcticTable.isKeyedTable()) { + snapshot = arcticTable.asKeyedTable().baseTable().currentSnapshot(); + } else { + snapshot = arcticTable.asUnkeyedTable().currentSnapshot(); + } + + if (snapshot != null) { + boolean findNewData = false; + if (snapshot.snapshotId() != lastBaseSnapshotId) { + findNewData = true; + LOG.debug("{} ==== {} find {} data in base snapshot={}", tableId(), getOptimizeType(), snapshot.operation(), + snapshot.snapshotId()); + } + + // If last snapshot not exist(may expire), then skip compaction, + // because compaction check interval is much shorter than expire time. + // Set table properties compact.major.force=true, if compaction is needed. + return findNewData; + } else { + LOG.warn("{} {} base snapshots is null, regard as table not changed", tableId(), getOptimizeType()); + return false; + } } private void addBaseFileIntoFileTree() { @@ -175,20 +265,14 @@ private void addBaseFileIntoFileTree() { DataFileType.POS_DELETE_FILE : DataFileType.BASE_FILE); // fill node position delete file map - if (contentFile instanceof DeleteFile) { + if (contentFile.content() == FileContent.POSITION_DELETES) { List files = partitionPosDeleteFiles.computeIfAbsent(partition, e -> new ArrayList<>()); files.add((DeleteFile) contentFile); partitionPosDeleteFiles.put(partition, files); } - // fill partition small file map - if (contentFile instanceof DataFile && - dataFileInfo.getSize() < PropertyUtil.propertyAsLong(arcticTable.properties(), - TableProperties.OPTIMIZE_SMALL_FILE_SIZE_BYTES_THRESHOLD, - TableProperties.OPTIMIZE_SMALL_FILE_SIZE_BYTES_THRESHOLD_DEFAULT)) { - List files = partitionSmallFiles.computeIfAbsent(partition, e -> new ArrayList<>()); - files.add((DataFile) contentFile); - partitionSmallFiles.put(partition, files); + if (contentFile.content() == FileContent.DATA) { + fillPartitionNeedOptimizeFiles(partition, contentFile); } addCnt.getAndIncrement(); @@ -204,8 +288,8 @@ private List collectUnKeyedTableTasks(String partition, List collector = new ArrayList<>(); String group = UUID.randomUUID().toString(); long createTime = System.currentTimeMillis(); - TaskConfig taskPartitionConfig = new TaskConfig(partition, null, group, historyId, - OptimizeType.Major, createTime); + TaskConfig taskPartitionConfig = new TaskConfig(partition, + null, group, historyId, partitionOptimizeType.get(partition), createTime); long taskSize = PropertyUtil.propertyAsLong(arcticTable.properties(), TableProperties.MAJOR_OPTIMIZE_MAX_TASK_FILE_SIZE, @@ -227,19 +311,16 @@ private List collectKeyedTableTasks(String partition, FileTree List collector = new ArrayList<>(); String group = UUID.randomUUID().toString(); long createTime = System.currentTimeMillis(); - TaskConfig taskPartitionConfig = new TaskConfig(partition, null, group, historyId, - OptimizeType.Major, createTime); + TaskConfig taskPartitionConfig = new TaskConfig(partition, + null, group, historyId, partitionOptimizeType.get(partition), createTime); treeRoot.completeTree(false); List subTrees = new ArrayList<>(); // split tasks - boolean isFilterSmallFiles = !isMergePosDelete.containsKey(partition); - long smallFileSize = PropertyUtil.propertyAsLong(arcticTable.properties(), - TableProperties.OPTIMIZE_SMALL_FILE_SIZE_BYTES_THRESHOLD, - TableProperties.OPTIMIZE_SMALL_FILE_SIZE_BYTES_THRESHOLD_DEFAULT); - treeRoot.splitSubTree(subTrees, new CanSplitFileTree(isFilterSmallFiles, smallFileSize)); + boolean isMajor = partitionOptimizeType.get(partition) == OptimizeType.Major; + treeRoot.splitSubTree(subTrees, new CanSplitFileTree(isMajor, partitionNeedMajorOptimizeFiles.get(partition))); for (FileTree subTree : subTrees) { List baseFiles = new ArrayList<>(); - subTree.collectBaseFiles(baseFiles, isFilterSmallFiles, smallFileSize); + subTree.collectBaseFiles(baseFiles, isMajor, partitionNeedMajorOptimizeFiles.get(partition)); if (!baseFiles.isEmpty()) { List sourceNodes = Collections.singletonList(subTree.getNode()); Set baseFileNodes = baseFiles.stream() @@ -251,7 +332,7 @@ private List collectKeyedTableTasks(String partition, FileTree baseFileNodes.contains(DefaultKeyedFile.parseMetaFromFileName(deleteFile.path().toString()).node())) .collect(Collectors.toList()); // if only one base file and no position delete file, skip - if (CollectionUtils.isEmpty(posDeleteFiles) && baseFiles.size() <= 1) { + if (canSkip(posDeleteFiles, baseFiles)) { continue; } collector.add(buildOptimizeTask(sourceNodes, @@ -262,14 +343,18 @@ private List collectKeyedTableTasks(String partition, FileTree return collector; } + protected boolean canSkip(List posDeleteFiles, List baseFiles) { + return CollectionUtils.isEmpty(posDeleteFiles) && baseFiles.size() <= 1; + } + static class CanSplitFileTree implements Predicate { - private final boolean isFilterSmallFiles; - private final long smallFileSize; + private final boolean isMajor; + private final List needMajorOptimizeFiles; - public CanSplitFileTree(boolean isFilterSmallFiles, long smallFileSize) { - this.isFilterSmallFiles = isFilterSmallFiles; - this.smallFileSize = smallFileSize; + public CanSplitFileTree(boolean isMajor, List needMajorOptimizeFiles) { + this.isMajor = isMajor; + this.needMajorOptimizeFiles = needMajorOptimizeFiles; } /** @@ -286,9 +371,9 @@ public boolean test(FileTree fileTree) { return false; } List baseFiles = fileTree.getBaseFiles(); - if (isFilterSmallFiles) { + if (isMajor) { baseFiles = baseFiles.stream() - .filter(dataFile -> dataFile.fileSizeInBytes() < smallFileSize).collect(Collectors.toList()); + .filter(needMajorOptimizeFiles::contains).collect(Collectors.toList()); } return baseFiles.isEmpty(); } diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/MinorOptimizePlan.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/MinorOptimizePlan.java index f6d071696c..dda0e16406 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/MinorOptimizePlan.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/MinorOptimizePlan.java @@ -36,6 +36,7 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.util.PropertyUtil; import org.slf4j.Logger; @@ -85,6 +86,7 @@ public boolean partitionNeedPlan(String partitionToPath) { if (deleteFileList.size() >= PropertyUtil.propertyAsInt(arcticTable.properties(), TableProperties.MINOR_OPTIMIZE_TRIGGER_DELETE_FILE_COUNT, TableProperties.MINOR_OPTIMIZE_TRIGGER_DELETE_FILE_COUNT_DEFAULT)) { + partitionOptimizeType.put(partitionToPath, OptimizeType.Minor); return true; } } @@ -93,6 +95,7 @@ public boolean partitionNeedPlan(String partitionToPath) { if (current - tableOptimizeRuntime.getLatestMinorOptimizeTime(partitionToPath) >= PropertyUtil.propertyAsLong(arcticTable.properties(), TableProperties.MINOR_OPTIMIZE_TRIGGER_MAX_INTERVAL, TableProperties.MINOR_OPTIMIZE_TRIGGER_MAX_INTERVAL_DEFAULT)) { + partitionOptimizeType.put(partitionToPath, OptimizeType.Minor); return true; } LOG.debug("{} ==== don't need {} optimize plan, skip partition {}", tableId(), getOptimizeType(), partitionToPath); @@ -124,6 +127,18 @@ protected List collectTask(String partition) { return result; } + @Override + protected boolean tableChanged() { + return changeTableChanged(); + } + + private boolean changeTableChanged() { + long lastChangeSnapshotId = tableOptimizeRuntime.getCurrentChangeSnapshotId(); + LOG.debug("{} ==== {} currentChangeSnapshotId={}, lastChangeSnapshotId={}", tableId(), getOptimizeType(), + currentChangeSnapshotId, lastChangeSnapshotId); + return currentChangeSnapshotId != lastChangeSnapshotId; + } + private void addChangeFilesIntoFileTree() { LOG.debug("{} start {} plan change files", tableId(), getOptimizeType()); KeyedTable keyedArcticTable = arcticTable.asKeyedTable(); @@ -198,7 +213,7 @@ private void addBaseFileIntoFileTree() { DataFileType.POS_DELETE_FILE : DataFileType.BASE_FILE); // fill node position delete file map - if (contentFile instanceof DeleteFile) { + if (contentFile.content() == FileContent.POSITION_DELETES) { List files = partitionPosDeleteFiles.computeIfAbsent(partition, e -> new ArrayList<>()); files.add((DeleteFile) contentFile); partitionPosDeleteFiles.put(partition, files); @@ -255,7 +270,7 @@ private List collectKeyedTableTasks(String partition, FileTree private boolean isOptimized(ContentFile dataFile, String partition) { // if Pos-Delete files, ignore - if (dataFile instanceof DeleteFile) { + if (dataFile.content() == FileContent.POSITION_DELETES) { return false; } diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/OptimizeTaskItem.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/OptimizeTaskItem.java index 0b7154b073..3baa26bb8f 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/OptimizeTaskItem.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/OptimizeTaskItem.java @@ -34,7 +34,7 @@ import com.netease.arctic.utils.SerializationUtil; import org.apache.ibatis.session.SqlSession; import org.apache.iceberg.ContentFile; -import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileContent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -255,7 +255,7 @@ private void persistTaskRuntime(BaseOptimizeTaskRuntime newRuntime, boolean upda internalTableFilesMapper.deleteOptimizeTaskTargetFile(optimizeTask.getTaskId()); newRuntime.getTargetFiles().forEach(file -> { ContentFile contentFile = SerializationUtil.toInternalTableFile(file); - if (contentFile instanceof DataFile) { + if (contentFile.content() == FileContent.DATA) { internalTableFilesMapper.insertOptimizeTaskFile(optimizeTask.getTaskId(), DataFileType.BASE_FILE, 1, SerializationUtil.byteBufferToByteArray(file)); } else { @@ -273,6 +273,33 @@ private void persistTaskRuntime(BaseOptimizeTaskRuntime newRuntime, boolean upda } } + public void persistTargetFiles() { + try (SqlSession sqlSession = getSqlSession(false)) { + InternalTableFilesMapper internalTableFilesMapper = + getMapper(sqlSession, InternalTableFilesMapper.class); + + try { + internalTableFilesMapper.deleteOptimizeTaskTargetFile(optimizeTask.getTaskId()); + optimizeRuntime.getTargetFiles().forEach(file -> { + ContentFile contentFile = SerializationUtil.toInternalTableFile(file); + if (contentFile.content() == FileContent.DATA) { + internalTableFilesMapper.insertOptimizeTaskFile(optimizeTask.getTaskId(), + DataFileType.BASE_FILE, 1, SerializationUtil.byteBufferToByteArray(file)); + } else { + internalTableFilesMapper.insertOptimizeTaskFile(optimizeTask.getTaskId(), + DataFileType.POS_DELETE_FILE, 1, SerializationUtil.byteBufferToByteArray(file)); + } + }); + } catch (Exception e) { + LOG.error("Update the internal table files failed.", e); + sqlSession.rollback(true); + throw e; + } + + sqlSession.commit(true); + } + } + private TableTaskHistory constructTableTaskHistory(long currentTime) { TableTaskHistory tableTaskHistory = new TableTaskHistory(); tableTaskHistory.setTableIdentifier(new TableIdentifier(optimizeTask.getTableIdentifier())); diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/SupportHiveCommit.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/SupportHiveCommit.java new file mode 100644 index 0000000000..11e7065b5c --- /dev/null +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/SupportHiveCommit.java @@ -0,0 +1,142 @@ +package com.netease.arctic.ams.server.optimize; + +import com.google.common.base.Preconditions; +import com.netease.arctic.ams.api.OptimizeType; +import com.netease.arctic.ams.server.model.BaseOptimizeTask; +import com.netease.arctic.ams.server.model.BaseOptimizeTaskRuntime; +import com.netease.arctic.ams.server.model.TableOptimizeRuntime; +import com.netease.arctic.data.DefaultKeyedFile; +import com.netease.arctic.hive.HMSClient; +import com.netease.arctic.hive.table.SupportHive; +import com.netease.arctic.hive.utils.HivePartitionUtil; +import com.netease.arctic.hive.utils.HiveTableUtil; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.SerializationUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +public class SupportHiveCommit extends BaseOptimizeCommit { + private static final Logger LOG = LoggerFactory.getLogger(SupportHiveCommit.class); + + protected Consumer updateTargetFiles; + + public SupportHiveCommit(ArcticTable arcticTable, + Map> optimizeTasksToCommit, + Consumer updateTargetFiles) { + super(arcticTable, optimizeTasksToCommit); + Preconditions.checkArgument(HiveTableUtil.isHive(arcticTable), "The table not support hive"); + this.updateTargetFiles = updateTargetFiles; + } + + @Override + public long commit(TableOptimizeRuntime tableOptimizeRuntime) throws Exception { + HMSClient hiveClient = ((SupportHive) arcticTable).getHMSClient(); + Map partitionPathMap = new HashMap<>(); + Types.StructType partitionSchema = arcticTable.isUnkeyedTable() ? + arcticTable.asUnkeyedTable().spec().partitionType() : + arcticTable.asKeyedTable().baseTable().spec().partitionType(); + + optimizeTasksToCommit.forEach((partition, optimizeTaskItems) -> { + // if major optimize task don't contain pos-delete files in a partition, can rewrite or move data files + // to hive location + if (isPartitionMajorOptimizeSupportHive(optimizeTaskItems)) { + for (OptimizeTaskItem optimizeTaskItem : optimizeTaskItems) { + BaseOptimizeTaskRuntime optimizeRuntime = optimizeTaskItem.getOptimizeRuntime(); + List targetFiles = optimizeRuntime.getTargetFiles().stream() + .map(fileByte -> (DataFile) SerializationUtil.toInternalTableFile(fileByte)) + .collect(Collectors.toList()); + long maxTransactionId = targetFiles.stream() + .mapToLong(dataFile -> { + DefaultKeyedFile.FileMeta fileMeta = DefaultKeyedFile.parseMetaFromFileName(dataFile.path().toString()); + return fileMeta.transactionId(); + }) + .max() + .orElse(0L); + + List newTargetFiles = new ArrayList<>(targetFiles.size()); + for (DataFile targetFile : targetFiles) { + if (partitionPathMap.get(partition) == null) { + List partitionValues = + HivePartitionUtil.partitionValuesAsList(targetFile.partition(), partitionSchema); + String partitionPath; + if (arcticTable.spec().isUnpartitioned()) { + try { + Table hiveTable = ((SupportHive) arcticTable).getHMSClient().run(client -> + client.getTable(arcticTable.id().getDatabase(), arcticTable.id().getTableName())); + partitionPath = hiveTable.getSd().getLocation(); + } catch (Exception e) { + LOG.error("Get hive table failed", e); + break; + } + } else { + partitionPath = arcticTable.isKeyedTable() ? + HiveTableUtil.newKeyedHiveDataLocation( + ((SupportHive) arcticTable).hiveLocation(), arcticTable.asKeyedTable().baseTable().spec(), + targetFile.partition(), maxTransactionId) : + HiveTableUtil.newUnKeyedHiveDataLocation(((SupportHive) arcticTable).hiveLocation(), + arcticTable.asUnkeyedTable().spec(), targetFile.partition(), HiveTableUtil.getRandomSubDir()); + HivePartitionUtil + .createPartitionIfAbsent(hiveClient, arcticTable, partitionValues, partitionPath, + Collections.emptyList(), (int) (System.currentTimeMillis() / 1000)); + + partitionPath = HivePartitionUtil + .getPartition(hiveClient, arcticTable, partitionValues).getSd().getLocation(); + } + partitionPathMap.put(partition, partitionPath); + } + + DataFile finalDataFile = moveTargetFiles(targetFile, partitionPathMap.get(partition)); + newTargetFiles.add(SerializationUtil.toByteBuffer(finalDataFile)); + } + + optimizeRuntime.setTargetFiles(newTargetFiles); + updateTargetFiles.accept(optimizeTaskItem); + } + } + }); + + return super.commit(tableOptimizeRuntime); + } + + protected boolean isPartitionMajorOptimizeSupportHive(List optimizeTaskItems) { + for (OptimizeTaskItem optimizeTaskItem : optimizeTaskItems) { + BaseOptimizeTask optimizeTask = optimizeTaskItem.getOptimizeTask(); + boolean isMajorTaskSupportHive = optimizeTask.getTaskId().getType() == OptimizeType.Major && + CollectionUtils.isEmpty(optimizeTask.getPosDeleteFiles()); + if (!isMajorTaskSupportHive) { + return false; + } + } + + return true; + } + + private DataFile moveTargetFiles(DataFile targetFile, String hiveLocation) { + String oldFilePath = targetFile.path().toString(); + String newFilePath = FileUtil.getNewFilePath(hiveLocation, oldFilePath); + + if (!arcticTable.io().exists(newFilePath)) { + arcticTable.io().rename(oldFilePath, newFilePath); + LOG.debug("{} move file from {} to {}", arcticTable.id(), oldFilePath, newFilePath); + } + + // org.apache.iceberg.BaseFile.set + ((StructLike) targetFile).set(1, newFilePath); + return targetFile; + } +} diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/SupportHiveMajorOptimizePlan.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/SupportHiveMajorOptimizePlan.java new file mode 100644 index 0000000000..fcfa7a7100 --- /dev/null +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/SupportHiveMajorOptimizePlan.java @@ -0,0 +1,142 @@ +package com.netease.arctic.ams.server.optimize; + +import com.google.common.base.Preconditions; +import com.netease.arctic.ams.api.DataFileInfo; +import com.netease.arctic.ams.api.OptimizeType; +import com.netease.arctic.ams.server.model.TableOptimizeRuntime; +import com.netease.arctic.hive.table.SupportHive; +import com.netease.arctic.hive.utils.HiveTableUtil; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.TableProperties; +import org.apache.commons.collections.CollectionUtils; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +public class SupportHiveMajorOptimizePlan extends MajorOptimizePlan { + private static final Logger LOG = LoggerFactory.getLogger(SupportHiveMajorOptimizePlan.class); + + // partitions are support hive + protected final Set supportHivePartitions = new HashSet<>(); + // hive location. + protected final String hiveLocation; + // files in locations don't need to major optimize + protected final Set excludeLocations = new HashSet<>(); + + public SupportHiveMajorOptimizePlan(ArcticTable arcticTable, TableOptimizeRuntime tableOptimizeRuntime, + List baseTableFileList, List posDeleteFileList, + Map partitionTaskRunning, int queueId, long currentTime, + Predicate snapshotIsCached) { + super(arcticTable, tableOptimizeRuntime, baseTableFileList, posDeleteFileList, + partitionTaskRunning, queueId, currentTime, snapshotIsCached); + + Preconditions.checkArgument(HiveTableUtil.isHive(arcticTable), "The table not support hive"); + hiveLocation = ((SupportHive) arcticTable).hiveLocation(); + excludeLocations.add(hiveLocation); + } + + @Override + public boolean partitionNeedPlan(String partitionToPath) { + long current = System.currentTimeMillis(); + + // check position delete file total size + if (checkPosDeleteTotalSize(partitionToPath)) { + partitionOptimizeType.put(partitionToPath, OptimizeType.FullMajor); + supportHivePartitions.add(partitionToPath); + return true; + } + + // check full major optimize interval + if (checkFullMajorOptimizeInterval(current, partitionToPath)) { + partitionOptimizeType.put(partitionToPath, OptimizeType.FullMajor); + supportHivePartitions.add(partitionToPath); + return true; + } + + // check small data file count + List smallFiles = filterSmallFiles(partitionToPath, + partitionNeedMajorOptimizeFiles.getOrDefault(partitionToPath, new ArrayList<>())); + if (checkSmallFileCount(partitionToPath, smallFiles)) { + partitionOptimizeType.put(partitionToPath, OptimizeType.Major); + if (CollectionUtils.isEmpty(partitionPosDeleteFiles.get(partitionToPath))) { + supportHivePartitions.add(partitionToPath); + } + return true; + } + + // check major optimize interval + if (checkMajorOptimizeInterval(current, partitionToPath)) { + partitionOptimizeType.put(partitionToPath, OptimizeType.Major); + if (CollectionUtils.isEmpty(partitionPosDeleteFiles.get(partitionToPath))) { + supportHivePartitions.add(partitionToPath); + } + return true; + } + + LOG.debug("{} ==== don't need {} optimize plan, skip partition {}", tableId(), getOptimizeType(), partitionToPath); + return false; + } + + @Override + protected boolean checkMajorOptimizeInterval(long current, String partitionToPath) { + if (current - tableOptimizeRuntime.getLatestMajorOptimizeTime(partitionToPath) >= + PropertyUtil.propertyAsLong(arcticTable.properties(), TableProperties.MAJOR_OPTIMIZE_TRIGGER_MAX_INTERVAL, + TableProperties.MAJOR_OPTIMIZE_TRIGGER_MAX_INTERVAL_DEFAULT)) { + // need to rewrite or move all files that not in hive location to hive location. + long fileCount = partitionNeedMajorOptimizeFiles.get(partitionToPath) == null ? + 0 : partitionNeedMajorOptimizeFiles.get(partitionToPath).size(); + return fileCount >= 1; + } + + return false; + } + + @Override + protected void fillPartitionNeedOptimizeFiles(String partition, ContentFile contentFile) { + // for support hive table, add all files in iceberg base store and not in hive store + if (canInclude(contentFile.path().toString())) { + List files = partitionNeedMajorOptimizeFiles.computeIfAbsent(partition, e -> new ArrayList<>()); + files.add((DataFile) contentFile); + partitionNeedMajorOptimizeFiles.put(partition, files); + } + } + + @Override + protected boolean canSkip(List posDeleteFiles, List baseFiles) { + return CollectionUtils.isEmpty(posDeleteFiles) && baseFiles.isEmpty(); + } + + private List filterSmallFiles(String partition, List dataFileList) { + // for support hive table, filter small files + List smallFileList = dataFileList.stream().filter(file -> file.fileSizeInBytes() <= + PropertyUtil.propertyAsLong(arcticTable.properties(), TableProperties.OPTIMIZE_SMALL_FILE_SIZE_BYTES_THRESHOLD, + TableProperties.OPTIMIZE_SMALL_FILE_SIZE_BYTES_THRESHOLD_DEFAULT)).collect(Collectors.toList()); + + // if iceberg store has pos-delete, only optimize small files + if (CollectionUtils.isNotEmpty(partitionPosDeleteFiles.get(partition))) { + partitionNeedMajorOptimizeFiles.put(partition, smallFileList); + } + + return smallFileList; + } + + private boolean canInclude(String filePath) { + for (String exclude : excludeLocations) { + if (filePath.contains(exclude)) { + return false; + } + } + return true; + } +} diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/TableOptimizeItem.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/TableOptimizeItem.java index 13239d086e..143b3962d0 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/TableOptimizeItem.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/TableOptimizeItem.java @@ -50,6 +50,8 @@ import com.netease.arctic.catalog.ArcticCatalog; import com.netease.arctic.catalog.CatalogLoader; import com.netease.arctic.data.DataFileType; +import com.netease.arctic.hive.table.SupportHive; +import com.netease.arctic.hive.utils.HiveTableUtil; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableIdentifier; @@ -68,6 +70,7 @@ import java.math.RoundingMode; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -323,12 +326,14 @@ public TableOptimizeInfo buildTableOptimizeInfo() { List optimizeTasks = this.optimizeTasks.values().stream().map(OptimizeTaskItem::getOptimizeTask).collect( Collectors.toList()); - this.optimizeFileInfo = collectOptimizeFileInfo(optimizeTasks, OptimizeType.Major); + this.optimizeFileInfo = collectOptimizeFileInfo(optimizeTasks, + new HashSet<>(Arrays.asList(OptimizeType.Major, OptimizeType.FullMajor))); } else if (tableOptimizeRuntime.getOptimizeStatus() == TableOptimizeInfo.OptimizeStatus.MinorOptimizing) { List optimizeTasks = this.optimizeTasks.values().stream().map(OptimizeTaskItem::getOptimizeTask).collect( Collectors.toList()); - this.optimizeFileInfo = collectOptimizeFileInfo(optimizeTasks, OptimizeType.Minor); + this.optimizeFileInfo = collectOptimizeFileInfo(optimizeTasks, + new HashSet<>(Collections.singletonList(OptimizeType.Minor))); } if (this.optimizeFileInfo != null) { tableOptimizeInfo.setFileCount(this.optimizeFileInfo.getFileCnt()); @@ -350,10 +355,10 @@ public void updateTableOptimizeStatus() { Collectors.toList()); if (hasMajorOptimizeTask()) { tryUpdateOptimizeInfo(TableOptimizeInfo.OptimizeStatus.MajorOptimizing, optimizeTasks, - OptimizeType.Major); + new HashSet<>(Arrays.asList(OptimizeType.Major, OptimizeType.FullMajor))); } else { tryUpdateOptimizeInfo(TableOptimizeInfo.OptimizeStatus.MinorOptimizing, optimizeTasks, - OptimizeType.Minor); + new HashSet<>(Collections.singletonList(OptimizeType.Minor))); } return; } @@ -374,13 +379,15 @@ public void updateTableOptimizeStatus() { MinorOptimizePlan minorPlan = getMinorPlan(-1, System.currentTimeMillis()); List minorTasks = minorPlan.plan(); if (!CollectionUtils.isEmpty(minorTasks)) { - tryUpdateOptimizeInfo(TableOptimizeInfo.OptimizeStatus.Pending, minorTasks, OptimizeType.Minor); + tryUpdateOptimizeInfo(TableOptimizeInfo.OptimizeStatus.Pending, minorTasks, + new HashSet<>(Collections.singletonList(OptimizeType.Minor))); return; } } tryUpdateOptimizeInfo(TableOptimizeInfo.OptimizeStatus.Idle, Collections.emptyList(), null); } else { - tryUpdateOptimizeInfo(TableOptimizeInfo.OptimizeStatus.Pending, majorTasks, OptimizeType.Major); + tryUpdateOptimizeInfo(TableOptimizeInfo.OptimizeStatus.Pending, majorTasks, + new HashSet<>(Arrays.asList(OptimizeType.Major, OptimizeType.FullMajor))); } } } @@ -388,17 +395,17 @@ public void updateTableOptimizeStatus() { private boolean hasMajorOptimizeTask() { for (Map.Entry entry : optimizeTasks.entrySet()) { OptimizeTaskId key = entry.getKey(); - if (key.getType() == OptimizeType.Major) { + if (key.getType() == OptimizeType.Major || key.getType() == OptimizeType.FullMajor) { return true; } } return false; } - private FilesStatistics collectOptimizeFileInfo(Collection tasks, OptimizeType type) { + private FilesStatistics collectOptimizeFileInfo(Collection tasks, Set types) { FilesStatisticsBuilder builder = new FilesStatisticsBuilder(); for (BaseOptimizeTask task : tasks) { - if (task.getTaskId().getType() == type) { + if (types.contains(task.getTaskId().getType())) { builder.addFiles(task.getBaseFileSize(), task.getBaseFileCnt()); builder.addFiles(task.getInsertFileSize(), task.getInsertFileCnt()); builder.addFiles(task.getDeleteFileSize(), task.getDeleteFileCnt()); @@ -410,7 +417,7 @@ private FilesStatistics collectOptimizeFileInfo(Collection tas private void tryUpdateOptimizeInfo(TableOptimizeInfo.OptimizeStatus optimizeStatus, Collection optimizeTasks, - OptimizeType type) { + Set types) { if (tableOptimizeRuntime.getOptimizeStatus() != optimizeStatus) { tableOptimizeRuntime.setOptimizeStatus(optimizeStatus); tableOptimizeRuntime.setOptimizeStatusStartTime(System.currentTimeMillis()); @@ -419,7 +426,7 @@ private void tryUpdateOptimizeInfo(TableOptimizeInfo.OptimizeStatus optimizeStat } catch (Throwable t) { LOG.warn("failed to persist tableOptimizeRuntime when update OptimizeStatus, ignore", t); } - optimizeFileInfo = collectOptimizeFileInfo(optimizeTasks, type); + optimizeFileInfo = collectOptimizeFileInfo(optimizeTasks, types); } if (tableOptimizeRuntime.getOptimizeStatusStartTime() <= 0) { long createTime = PropertyUtil.propertyAsLong(getArcticTable().properties(), TableProperties.TABLE_CREATE_TIME, @@ -491,6 +498,9 @@ private void optimizeTasksCommitted( case Major: tableOptimizeRuntime.putLatestMajorOptimizeTime(partition, commitTime); break; + case FullMajor: + tableOptimizeRuntime.putLatestFullOptimizeTime(partition, commitTime); + break; } }); try { @@ -689,7 +699,7 @@ public void commitOptimizeTasks() throws Exception { // check current base table snapshot whether changed when minor optimize if (isMinorOptimizing()) { if (tableOptimizeRuntime.getCurrentSnapshotId() != - UnKeyedTableUtil.getSnapshotId(arcticTable.asKeyedTable().baseTable())) { + UnKeyedTableUtil.getSnapshotId(getArcticTable().asKeyedTable().baseTable())) { LOG.info("the latest snapshot has changed in base table {}, give up commit.", tableIdentifier); clearOptimizeTasks(); tableOptimizeRuntime.setRunning(false); @@ -701,7 +711,13 @@ public void commitOptimizeTasks() throws Exception { long taskCount = tasksToCommit.values().stream().mapToLong(Collection::size).sum(); if (MapUtils.isNotEmpty(tasksToCommit)) { LOG.info("{} get {} tasks of {} partitions to commit", tableIdentifier, taskCount, tasksToCommit.size()); - BaseOptimizeCommit optimizeCommit = new BaseOptimizeCommit(getArcticTable(true), tasksToCommit); + BaseOptimizeCommit optimizeCommit; + if (HiveTableUtil.isHive(getArcticTable())) { + optimizeCommit = new SupportHiveCommit(getArcticTable(true), + tasksToCommit, OptimizeTaskItem::persistTargetFiles); + } else { + optimizeCommit = new BaseOptimizeCommit(getArcticTable(true), tasksToCommit); + } long commitTime = optimizeCommit.commit(tableOptimizeRuntime); optimizeTasksCommitted( optimizeCommit.getCommittedTasks(), commitTime, optimizeCommit.getPartitionOptimizeType()); @@ -739,8 +755,13 @@ public MajorOptimizePlan getMajorPlan(int queueId, long currentTime) { baseFiles.addAll(filterFile(baseTableFiles, DataFileType.INSERT_FILE)); List posDeleteFiles = filterFile(baseTableFiles, DataFileType.POS_DELETE_FILE); - return new MajorOptimizePlan(getArcticTable(), tableOptimizeRuntime, - baseFiles, posDeleteFiles, generatePartitionRunning(), queueId, currentTime, snapshotIsCached); + if (getArcticTable() instanceof SupportHive) { + return new SupportHiveMajorOptimizePlan(getArcticTable(), tableOptimizeRuntime, + baseFiles, posDeleteFiles, generatePartitionRunning(), queueId, currentTime, snapshotIsCached); + } else { + return new MajorOptimizePlan(getArcticTable(), tableOptimizeRuntime, + baseFiles, posDeleteFiles, generatePartitionRunning(), queueId, currentTime, snapshotIsCached); + } } /** diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/ISupportHiveSyncService.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/ISupportHiveSyncService.java new file mode 100644 index 0000000000..37361d640c --- /dev/null +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/ISupportHiveSyncService.java @@ -0,0 +1,29 @@ +/* + * 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 com.netease.arctic.ams.server.service; + +/** + * Service for support hive sync periodically. + */ +public interface ISupportHiveSyncService { + /** + * Check support hive table tasks, sync iceberg partition location to hive table. + */ + void checkHiveSyncTasks(); +} diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/ServiceContainer.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/ServiceContainer.java index dc104f5d16..008eebd48b 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/ServiceContainer.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/ServiceContainer.java @@ -19,7 +19,6 @@ package com.netease.arctic.ams.server.service; import com.google.common.annotations.VisibleForTesting; -import com.netease.arctic.ams.api.ArcticTableMetastore; import com.netease.arctic.ams.server.handler.impl.ArcticTableMetastoreHandler; import com.netease.arctic.ams.server.handler.impl.OptimizeManagerHandler; import com.netease.arctic.ams.server.optimize.IOptimizeService; @@ -37,6 +36,7 @@ import com.netease.arctic.ams.server.service.impl.OrphanFilesCleanService; import com.netease.arctic.ams.server.service.impl.QuotaService; import com.netease.arctic.ams.server.service.impl.RuntimeDataExpireService; +import com.netease.arctic.ams.server.service.impl.SupportHiveSyncService; import com.netease.arctic.ams.server.service.impl.TableBaseInfoService; import com.netease.arctic.ams.server.service.impl.TableExpireService; import com.netease.arctic.ams.server.service.impl.TableTaskHistoryService; @@ -79,6 +79,8 @@ public class ServiceContainer { private static volatile AdaptHiveService adaptHiveService; + private static volatile ISupportHiveSyncService supportHiveSyncService; + public static IOptimizeService getOptimizeService() { if (optimizeService == null) { synchronized (ServiceContainer.class) { @@ -266,6 +268,18 @@ public static AdaptHiveService getAdaptHiveService() { return adaptHiveService; } + public static ISupportHiveSyncService getSupportHiveSyncService() { + if (supportHiveSyncService == null) { + synchronized (ServiceContainer.class) { + if (supportHiveSyncService == null) { + supportHiveSyncService = new SupportHiveSyncService(); + } + } + } + + return supportHiveSyncService; + } + public static DDLTracerService getDdlTracerService() { if (ddlTracerService == null) { synchronized (ServiceContainer.class) { diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/OptimizeQueueService.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/OptimizeQueueService.java index 7f6f1755d8..b5226d591e 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/OptimizeQueueService.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/OptimizeQueueService.java @@ -545,7 +545,6 @@ private List plan(long currentTime) { } List optimizeTasks; - OptimizeType optimizeType = OptimizeType.Major; BaseOptimizePlan optimizePlan; Map properties = tableItem.getArcticTable(false).properties(); int queueId = ServiceContainer.getOptimizeQueueService().getQueueId(properties); @@ -556,10 +555,9 @@ private List plan(long currentTime) { if (tableItem.isKeyedTable() && CollectionUtils.isEmpty(optimizeTasks)) { optimizePlan = tableItem.getMinorPlan(queueId, currentTime); optimizeTasks = optimizePlan.plan(); - optimizeType = OptimizeType.Minor; } - initTableOptimizeRuntime(tableItem, optimizePlan, optimizeTasks, optimizeType); + initTableOptimizeRuntime(tableItem, optimizePlan, optimizeTasks, optimizePlan.getPartitionOptimizeType()); LOG.debug("{} after plan get {} tasks", tableItem.getTableIdentifier(), optimizeTasks.size()); List toExecuteTasks = addTask(tableItem, optimizeTasks); @@ -640,21 +638,25 @@ private BigDecimal evalQuotaRate(TableIdentifier tableId, long currentTime) thro private void initTableOptimizeRuntime(TableOptimizeItem tableItem, BaseOptimizePlan optimizePlan, List optimizeTasks, - OptimizeType optimizeType) { + Map partitionOptimizeType) { if (CollectionUtils.isNotEmpty(optimizeTasks)) { // set latest optimize time - switch (optimizeType) { - case Minor: - for (String currentPartition : optimizePlan.getCurrentPartitions()) { - tableItem.getTableOptimizeRuntime().putLatestMinorOptimizeTime(currentPartition, -1); + for (String currentPartition : optimizePlan.getCurrentPartitions()) { + if (partitionOptimizeType.get(currentPartition) != null) { + switch (partitionOptimizeType.get(currentPartition)) { + case Minor: + tableItem.getTableOptimizeRuntime().putLatestMinorOptimizeTime(currentPartition, -1); + break; + case Major: + tableItem.getTableOptimizeRuntime().putLatestMajorOptimizeTime(currentPartition, -1); + break; + case FullMajor: + tableItem.getTableOptimizeRuntime().putLatestFullOptimizeTime(currentPartition, -1); + break; } - break; - case Major: - for (String currentPartition : optimizePlan.getCurrentPartitions()) { - tableItem.getTableOptimizeRuntime().putLatestMajorOptimizeTime(currentPartition, -1); - } - break; + } } + // set current snapshot id tableItem.getTableOptimizeRuntime().setCurrentSnapshotId(optimizePlan.getCurrentBaseSnapshotId()); if (tableItem.isKeyedTable()) { diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/OrphanFilesCleanService.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/OrphanFilesCleanService.java index 3c49ecebb5..628f024169 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/OrphanFilesCleanService.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/OrphanFilesCleanService.java @@ -23,6 +23,7 @@ import com.netease.arctic.ams.server.model.TableMetadata; import com.netease.arctic.ams.server.service.IOrphanFilesCleanService; import com.netease.arctic.ams.server.service.ServiceContainer; +import com.netease.arctic.ams.server.utils.HiveLocationUtils; import com.netease.arctic.ams.server.utils.ScheduledTasks; import com.netease.arctic.ams.server.utils.ThreadPool; import com.netease.arctic.catalog.ArcticCatalog; @@ -167,8 +168,12 @@ private static void clearInternalTableDataFiles(ArcticTable table, UnkeyedTable .getOptimizeDatafiles(table.id().buildTableIdentifier(), Constants.INNER_TABLE_BASE); exclude = dataFilesInfo.stream().map(DataFileInfo::getPath).collect(Collectors.toSet()); } + + // add hive location to exclude + exclude.addAll(HiveLocationUtils.getHiveLocation(table)); + String dataLocation = internalTable.location() + File.separator + DATA_FOLDER_NAME; - if (new File(dataLocation).exists()) { + if (table.io().exists(dataLocation)) { for (FileStatus fileStatus : table.io().list(dataLocation)) { deleteFilesCnt += deleteInvalidDataFiles(table.io(), fileStatus, diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/SupportHiveSyncService.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/SupportHiveSyncService.java new file mode 100644 index 0000000000..f2a21a1954 --- /dev/null +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/SupportHiveSyncService.java @@ -0,0 +1,288 @@ +/* + * 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 com.netease.arctic.ams.server.service.impl; + +import com.netease.arctic.ams.server.model.TableMetadata; +import com.netease.arctic.ams.server.service.ISupportHiveSyncService; +import com.netease.arctic.ams.server.service.ServiceContainer; +import com.netease.arctic.ams.server.utils.ScheduledTasks; +import com.netease.arctic.ams.server.utils.ThreadPool; +import com.netease.arctic.catalog.ArcticCatalog; +import com.netease.arctic.catalog.CatalogLoader; +import com.netease.arctic.hive.HiveTableProperties; +import com.netease.arctic.hive.table.SupportHive; +import com.netease.arctic.hive.utils.HivePartitionUtil; +import com.netease.arctic.hive.utils.HiveTableUtil; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.TableIdentifier; +import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.TablePropertyUtil; +import org.apache.commons.collections.CollectionUtils; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.util.StructLikeMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; + +public class SupportHiveSyncService implements ISupportHiveSyncService { + private static final Logger LOG = LoggerFactory.getLogger(SupportHiveSyncService.class); + private static final long SYNC_INTERVAL = 3600_000; // 1 hour + + private ScheduledTasks syncTasks; + + @Override + public void checkHiveSyncTasks() { + LOG.info("Schedule Support Hive Sync"); + if (syncTasks == null) { + syncTasks = new ScheduledTasks<>(ThreadPool.Type.HIVE_SYNC); + } + List tables = ServiceContainer.getMetaService().listTables(); + Set ids = + tables.stream().map(TableMetadata::getTableIdentifier).collect(Collectors.toSet()); + syncTasks.checkRunningTask(ids, + tableId -> SYNC_INTERVAL, + SupportHiveSyncService.SupportHiveSyncTask::new, + false); + LOG.info("Schedule Support Hive Sync finished with {} valid ids", ids.size()); + } + + public static class SupportHiveSyncTask implements ScheduledTasks.Task { + private final TableIdentifier tableIdentifier; + + SupportHiveSyncTask(TableIdentifier tableIdentifier) { + this.tableIdentifier = tableIdentifier; + } + + @Override + public void run() { + long startTime = System.currentTimeMillis(); + final String traceId = UUID.randomUUID().toString(); + try { + LOG.info("[{}] {} start hive sync", traceId, tableIdentifier); + ArcticCatalog catalog = + CatalogLoader.load(ServiceContainer.getTableMetastoreHandler(), tableIdentifier.getCatalog()); + ArcticTable arcticTable = catalog.loadTable(tableIdentifier); + if (!HiveTableUtil.isHive(arcticTable)) { + LOG.debug("[{}] {} is not a support hive table", traceId, tableIdentifier); + return; + } + + syncIcebergToHive(arcticTable, traceId); + } catch (Exception e) { + LOG.error("[{}] {} hive sync failed", traceId, tableIdentifier, e); + } finally { + LOG.info("[{}] {} hive sync finished, cost {}ms", traceId, tableIdentifier, + System.currentTimeMillis() - startTime); + } + } + + public static void syncIcebergToHive(ArcticTable arcticTable, String traceId) throws Exception { + UnkeyedTable baseTable = arcticTable.isKeyedTable() ? + arcticTable.asKeyedTable().baseTable() : arcticTable.asUnkeyedTable(); + StructLikeMap> partitionProperty = baseTable.partitionProperty(); + + if (arcticTable.spec().isUnpartitioned()) { + syncNoPartitionTable(arcticTable, partitionProperty, traceId); + } else { + syncPartitionTable(arcticTable, partitionProperty); + } + } + + /** + * once get location from iceberg property, should update hive table location, + * because only arctic update hive table location for unPartitioned table. + */ + private static void syncNoPartitionTable(ArcticTable arcticTable, + StructLikeMap> partitionProperty, + String traceId) { + Map property = partitionProperty.get(TablePropertyUtil.EMPTY_STRUCT); + if (property == null || property.get(HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION) == null) { + LOG.debug("[{}] {} has no hive location in partition property", traceId, arcticTable.id()); + return; + } + + String currentLocation = property.get(HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION); + String hiveLocation; + try { + hiveLocation = ((SupportHive) arcticTable).getHMSClient().run(client -> { + Table hiveTable = client.getTable(arcticTable.id().getDatabase(), arcticTable.id().getTableName()); + return hiveTable.getSd().getLocation(); + }); + } catch (Exception e) { + LOG.error("[{}] {} get hive location failed", traceId, arcticTable.id(), e); + return; + } + + if (!Objects.equals(currentLocation, hiveLocation)) { + try { + ((SupportHive) arcticTable).getHMSClient().run(client -> { + Table hiveTable = client.getTable(arcticTable.id().getDatabase(), arcticTable.id().getTableName()); + hiveTable.getSd().setLocation(currentLocation); + client.alter_table(arcticTable.id().getDatabase(), arcticTable.id().getTableName(), hiveTable); + return null; + }); + } catch (Exception e) { + LOG.error("[{}] {} alter hive location failed", traceId, arcticTable.id(), e); + } + } + } + + private static void syncPartitionTable(ArcticTable arcticTable, + StructLikeMap> partitionProperty) throws Exception { + Map icebergPartitionMap = new HashMap<>(); + for (StructLike structLike : partitionProperty.keySet()) { + icebergPartitionMap.put(arcticTable.spec().partitionToPath(structLike), structLike); + } + List icebergPartitions = new ArrayList<>(icebergPartitionMap.keySet()); + List hivePartitions = ((SupportHive) arcticTable).getHMSClient().run(client -> + client.listPartitions(arcticTable.id().getDatabase(), arcticTable.id().getTableName(), Short.MAX_VALUE)); + List hivePartitionNames = ((SupportHive) arcticTable).getHMSClient().run(client -> + client.listPartitionNames(arcticTable.id().getDatabase(), arcticTable.id().getTableName(), Short.MAX_VALUE)); + List partitionKeys = ((SupportHive) arcticTable).getHMSClient().run(client -> { + Table hiveTable = client.getTable(arcticTable.id().getDatabase(), arcticTable.id().getTableName()); + return hiveTable.getPartitionKeys(); + }); + Map hivePartitionMap = new HashMap<>(); + for (Partition hivePartition : hivePartitions) { + hivePartitionMap.put(Warehouse.makePartName(partitionKeys, hivePartition.getValues()), hivePartition); + } + + Set inIcebergNotInHive = icebergPartitions.stream() + .filter(partition -> !hivePartitionNames.contains(partition)) + .collect(Collectors.toSet()); + Set inHiveNotInIceberg = hivePartitionNames.stream() + .filter(partition -> !icebergPartitions.contains(partition)) + .collect(Collectors.toSet()); + Set inBoth = icebergPartitions.stream() + .filter(hivePartitionNames::contains) + .collect(Collectors.toSet()); + + if (CollectionUtils.isNotEmpty(inIcebergNotInHive)) { + handleInIcebergPartitions(arcticTable, inIcebergNotInHive, icebergPartitionMap, partitionProperty); + } + + if (CollectionUtils.isNotEmpty(inHiveNotInIceberg)) { + handleInHivePartitions(arcticTable, inHiveNotInIceberg, hivePartitionMap); + } + + if (CollectionUtils.isNotEmpty(inBoth)) { + handleInBothPartitions(arcticTable, inBoth, hivePartitionMap, icebergPartitionMap, partitionProperty); + } + } + + /** + * if iceberg partition location is existed, should update hive table location. + */ + private static void handleInIcebergPartitions(ArcticTable arcticTable, + Set inIcebergNotInHive, + Map icebergPartitionMap, + StructLikeMap> partitionProperty) { + inIcebergNotInHive.forEach(partition -> { + Map property = partitionProperty.get(icebergPartitionMap.get(partition)); + if (property == null || property.get(HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION) == null) { + return; + } + String currentLocation = property.get(HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION); + + if (arcticTable.io().exists(currentLocation)) { + int transientTime = Integer.parseInt(property + .getOrDefault(HiveTableProperties.PARTITION_PROPERTIES_KEY_TRANSIENT_TIME, "0")); + List dataFiles = getIcebergPartitionFiles(arcticTable, icebergPartitionMap.get(partition)); + HivePartitionUtil.createPartitionIfAbsent(((SupportHive) arcticTable).getHMSClient(), + arcticTable, + HivePartitionUtil.partitionValuesAsList(icebergPartitionMap.get(partition), + arcticTable.spec().partitionType()), + currentLocation, dataFiles, transientTime); + } + }); + } + + private static void handleInHivePartitions(ArcticTable arcticTable, + Set inHiveNotInIceberg, + Map hivePartitionMap) { + inHiveNotInIceberg.forEach(partition -> { + Partition hivePartition = hivePartitionMap.get(partition); + boolean isArctic = hivePartition.getParameters() + .getOrDefault(HiveTableProperties.ARCTIC_TABLE_FLAG, "false").equals("true"); + if (isArctic) { + HivePartitionUtil.dropPartition(((SupportHive) arcticTable).getHMSClient(), arcticTable, hivePartition); + } + }); + } + + private static void handleInBothPartitions(ArcticTable arcticTable, + Set inBoth, + Map hivePartitionMap, + Map icebergPartitionMap, + StructLikeMap> partitionProperty) { + Set inHiveNotInIceberg = new HashSet<>(); + inBoth.forEach(partition -> { + Map property = partitionProperty.get(icebergPartitionMap.get(partition)); + if (property == null || property.get(HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION) == null) { + inHiveNotInIceberg.add(partition); + return; + } + + String currentLocation = property.get(HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION); + Partition hivePartition = hivePartitionMap.get(partition); + + if (!Objects.equals(currentLocation, hivePartition.getSd().getLocation())) { + int transientTime = Integer.parseInt(property + .getOrDefault(HiveTableProperties.PARTITION_PROPERTIES_KEY_TRANSIENT_TIME, "0")); + List dataFiles = getIcebergPartitionFiles(arcticTable, icebergPartitionMap.get(partition)); + HivePartitionUtil.updatePartitionLocation(((SupportHive) arcticTable).getHMSClient(), + arcticTable, hivePartition, currentLocation, dataFiles, transientTime); + } + }); + + handleInHivePartitions(arcticTable, inHiveNotInIceberg, hivePartitionMap); + } + + private static List getIcebergPartitionFiles(ArcticTable arcticTable, + StructLike partition) { + UnkeyedTable baseStore; + baseStore = arcticTable.isKeyedTable() ? arcticTable.asKeyedTable().baseTable() : arcticTable.asUnkeyedTable(); + + List partitionFiles = new ArrayList<>(); + TableScan tableScan = baseStore.newScan(); + for (FileScanTask fileScanTask : tableScan.planFiles()) { + if (fileScanTask.file().partition().equals(partition)) { + partitionFiles.add(fileScanTask.file()); + } + } + return partitionFiles; + } + } +} diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/TableExpireService.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/TableExpireService.java index 62802a8349..3ebe7a4313 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/TableExpireService.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/service/impl/TableExpireService.java @@ -25,12 +25,14 @@ import com.netease.arctic.ams.server.service.ServiceContainer; import com.netease.arctic.ams.server.utils.ChangeFilesUtil; import com.netease.arctic.ams.server.utils.ContentFileUtil; +import com.netease.arctic.ams.server.utils.HiveLocationUtils; import com.netease.arctic.ams.server.utils.ScheduledTasks; import com.netease.arctic.ams.server.utils.ThreadPool; import com.netease.arctic.catalog.ArcticCatalog; import com.netease.arctic.catalog.CatalogLoader; import com.netease.arctic.data.DefaultKeyedFile; import com.netease.arctic.data.PrimaryKeyedFile; +import com.netease.arctic.hive.utils.HiveTableUtil; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableIdentifier; @@ -112,8 +114,14 @@ public void run() { .getOrDefault(TableProperties.CHANGE_SNAPSHOT_KEEP_MINUTES, TableProperties.CHANGE_SNAPSHOT_KEEP_MINUTES_DEFAULT)) * 60 * 1000; + Set hiveLocation = new HashSet<>(); + if (HiveTableUtil.isHive(arcticTable)) { + hiveLocation = HiveLocationUtils.getHiveLocation(arcticTable); + } + if (arcticTable.isKeyedTable()) { KeyedTable keyedArcticTable = arcticTable.asKeyedTable(); + Set finalHiveLocation = hiveLocation; keyedArcticTable.io().doAs(() -> { UnkeyedTable baseTable = keyedArcticTable.baseTable(); if (baseTable == null) { @@ -123,6 +131,7 @@ public void run() { List changeFilesInfo = ServiceContainer.getFileInfoCacheService() .getOptimizeDatafiles(tableIdentifier.buildTableIdentifier(), Constants.INNER_TABLE_CHANGE); Set baseExclude = changeFilesInfo.stream().map(DataFileInfo::getPath).collect(Collectors.toSet()); + baseExclude.addAll(finalHiveLocation); expireSnapshots(baseTable, startTime - baseSnapshotsKeepTime, baseExclude); long baseCleanedTime = System.currentTimeMillis(); LOG.info("[{}] {} base expire cost {} ms", traceId, arcticTable.id(), baseCleanedTime - startTime); @@ -140,6 +149,7 @@ public void run() { List baseFilesInfo = ServiceContainer.getFileInfoCacheService() .getOptimizeDatafiles(tableIdentifier.buildTableIdentifier(), Constants.INNER_TABLE_BASE); Set changeExclude = baseFilesInfo.stream().map(DataFileInfo::getPath).collect(Collectors.toSet()); + changeExclude.addAll(finalHiveLocation); expireSnapshots(changeTable, startTime - changeSnapshotsKeepTime, changeExclude); return null; }); @@ -147,7 +157,7 @@ public void run() { System.currentTimeMillis() - startTime); } else { UnkeyedTable unKeyedArcticTable = arcticTable.asUnkeyedTable(); - expireSnapshots(unKeyedArcticTable, startTime - baseSnapshotsKeepTime, new HashSet<>()); + expireSnapshots(unKeyedArcticTable, startTime - baseSnapshotsKeepTime, hiveLocation); long baseCleanedTime = System.currentTimeMillis(); LOG.info("[{}] {} unKeyedTable expire cost {} ms", traceId, arcticTable.id(), baseCleanedTime - startTime); } diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/utils/HiveLocationUtils.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/utils/HiveLocationUtils.java new file mode 100644 index 0000000000..c89fb305cd --- /dev/null +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/utils/HiveLocationUtils.java @@ -0,0 +1,64 @@ +/* + * 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 com.netease.arctic.ams.server.utils; + +import com.netease.arctic.hive.table.SupportHive; +import com.netease.arctic.hive.utils.HiveTableUtil; +import com.netease.arctic.table.ArcticTable; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class HiveLocationUtils { + private static final Logger LOG = LoggerFactory.getLogger(HiveLocationUtils.class); + + public static Set getHiveLocation(ArcticTable table) { + Set hiveLocations = new HashSet<>(); + if (HiveTableUtil.isHive(table)) { + if (table.spec().isUnpartitioned()) { + try { + Table hiveTable = ((SupportHive) table).getHMSClient().run(client -> + client.getTable(table.id().getDatabase(), table.id().getTableName())); + hiveLocations.add(hiveTable.getSd().getLocation()); + } catch (Exception e) { + LOG.error("{} get hive table error", table.id(), e); + throw new IllegalStateException("get hive table error", e); + } + } else { + try { + List partitions = ((SupportHive) table).getHMSClient().run(client -> + client.listPartitions(table.id().getDatabase(), table.id().getTableName(), Short.MAX_VALUE)); + for (Partition partition : partitions) { + hiveLocations.add(partition.getSd().getLocation()); + } + } catch (Exception e) { + LOG.error("{} get hive partitions error", table.id(), e); + throw new IllegalStateException("get hive partitions error", e); + } + } + } + + return hiveLocations; + } +} diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/utils/ThreadPool.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/utils/ThreadPool.java index e7227b1273..58b5e94897 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/utils/ThreadPool.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/utils/ThreadPool.java @@ -42,6 +42,7 @@ public class ThreadPool { private static ScheduledExecutorService commitPool; private static ScheduledExecutorService expirePool; private static ScheduledExecutorService orphanPool; + private static ScheduledExecutorService supportHiveSyncPool; private static ScheduledExecutorService optimizerMonitorPool; private static ThreadPoolExecutor syncFileInfoCachePool; private static ScheduledExecutorService tableRuntimeDataExpirePool; @@ -53,7 +54,8 @@ public enum Type { ORPHAN, SYNC_FILE_INFO_CACHE, OPTIMIZER_MONITOR, - TABLE_RUNTIME_DATA_EXPIRE + TABLE_RUNTIME_DATA_EXPIRE, + HIVE_SYNC } public static synchronized ThreadPool initialize(Configuration conf) { @@ -90,6 +92,12 @@ private ThreadPool(Configuration conf) { conf.getInteger(ArcticMetaStoreConf.ORPHAN_CLEAN_THREAD_POOL_SIZE), orphanThreadFactory); + ThreadFactory supportHiveSyncFactory = new ThreadFactoryBuilder().setDaemon(false) + .setNameFormat("Support Hive Sync Worker %d").build(); + supportHiveSyncPool = Executors.newScheduledThreadPool( + conf.getInteger(ArcticMetaStoreConf.SUPPORT_HIVE_SYNC_THREAD_POOL_SIZE), + supportHiveSyncFactory); + ThreadFactory optimizerMonitorThreadFactory = new ThreadFactoryBuilder().setDaemon(false) .setNameFormat("Metastore Scheduled Optimizer Monitor Worker %d").build(); optimizerMonitorPool = Executors.newScheduledThreadPool( @@ -131,6 +139,8 @@ public static ScheduledExecutorService getPool(Type type) { return optimizerMonitorPool; case TABLE_RUNTIME_DATA_EXPIRE: return tableRuntimeDataExpirePool; + case HIVE_SYNC: + return supportHiveSyncPool; default: throw new RuntimeException("ThreadPool not support this type: " + type); } @@ -149,6 +159,7 @@ public static synchronized void shutdown() { orphanPool.shutdown(); syncFileInfoCachePool.shutdown(); tableRuntimeDataExpirePool.shutdownNow(); + supportHiveSyncPool.shutdown(); self = null; } } diff --git a/ams/ams-server/src/main/resources/sql/derby/ams-init.sql b/ams/ams-server/src/main/resources/sql/derby/ams-init.sql index 958853aab7..1fa0de402a 100644 --- a/ams/ams-server/src/main/resources/sql/derby/ams-init.sql +++ b/ams/ams-server/src/main/resources/sql/derby/ams-init.sql @@ -103,6 +103,7 @@ CREATE TABLE optimize_table_runtime ( table_name varchar(64) NOT NULL, current_snapshot_id bigint NOT NULL DEFAULT -1, latest_major_optimize_time clob(64m), + latest_full_optimize_time clob(64m), latest_minor_optimize_time clob(64m), latest_task_history_id varchar(40) DEFAULT NULL, optimize_status varchar(20) DEFAULT 'Idle', diff --git a/ams/ams-server/src/main/resources/sql/mysql/upgrade-0.3.0-to-0.3.1.sql b/ams/ams-server/src/main/resources/sql/mysql/upgrade-0.3.0-to-0.3.1.sql index 4011a3bdbc..579c6f53cc 100644 --- a/ams/ams-server/src/main/resources/sql/mysql/upgrade-0.3.0-to-0.3.1.sql +++ b/ams/ams-server/src/main/resources/sql/mysql/upgrade-0.3.0-to-0.3.1.sql @@ -23,4 +23,6 @@ ALTER TABLE `snapshot_info_cache` modify COLUMN `table_identifier` varchar(384) ALTER TABLE `file_info_cache` ADD COLUMN `producer` varchar(64) NOT NULL DEFAULT 'INGESTION'; ALTER TABLE `file_info_cache` modify COLUMN `table_identifier` varchar(384) NOT NULL; ALTER TABLE `table_metadata` ADD COLUMN `cur_schema_id` int(11) DEFAULT NULL; -ALTER TABLE `table_transaction_meta` modify COLUMN `table_identifier` varchar(384) NOT NULL; \ No newline at end of file +ALTER TABLE `table_transaction_meta` modify COLUMN `table_identifier` varchar(384) NOT NULL; +ALTER TABLE `optimize_file` MODIFY COLUMN `optimize_type` varchar(10) NOT NULL COMMENT 'Optimize type: Major, Minor, FullMajor'; +ALTER TABLE `optimize_table_runtime` ADD COLUMN `latest_full_optimize_time` MEDIUMTEXT NULL COMMENT 'Latest Full Optimize time for all partitions'; diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/AmsTestBase.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/AmsTestBase.java index 0193e3d88a..a6262bd3e4 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/AmsTestBase.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/AmsTestBase.java @@ -29,16 +29,21 @@ import com.netease.arctic.ams.server.handler.impl.ArcticTableMetastoreHandler; import com.netease.arctic.ams.server.handler.impl.OptimizeManagerHandler; import com.netease.arctic.ams.server.optimize.TestExpiredFileClean; +import com.netease.arctic.ams.server.optimize.TestExpiredFileCleanSupportHive; import com.netease.arctic.ams.server.optimize.TestMajorOptimizeCommit; import com.netease.arctic.ams.server.optimize.TestMajorOptimizePlan; import com.netease.arctic.ams.server.optimize.TestMinorOptimizeCommit; import com.netease.arctic.ams.server.optimize.TestMinorOptimizePlan; import com.netease.arctic.ams.server.optimize.TestOrphanFileClean; +import com.netease.arctic.ams.server.optimize.TestOrphanFileCleanSupportHive; +import com.netease.arctic.ams.server.optimize.TestSupportHiveMajorOptimizeCommit; +import com.netease.arctic.ams.server.optimize.TestSupportHiveMajorOptimizePlan; import com.netease.arctic.ams.server.service.MetaService; import com.netease.arctic.ams.server.service.ServiceContainer; import com.netease.arctic.ams.server.service.TestDDLTracerService; import com.netease.arctic.ams.server.service.TestFileInfoCacheService; import com.netease.arctic.ams.server.service.impl.AdaptHiveService; +import com.netease.arctic.ams.server.service.TestSupportHiveSyncService; import com.netease.arctic.ams.server.service.impl.ArcticTransactionService; import com.netease.arctic.ams.server.service.impl.CatalogMetadataService; import com.netease.arctic.ams.server.service.impl.DDLTracerService; @@ -91,7 +96,12 @@ TestMinorOptimizePlan.class, TestOrphanFileClean.class, TestFileInfoCacheService.class, - OptimizeIntegrationTest.class}) + OptimizeIntegrationTest.class, + TestSupportHiveMajorOptimizePlan.class, + TestSupportHiveMajorOptimizeCommit.class, + TestSupportHiveSyncService.class, + TestExpiredFileCleanSupportHive.class, + TestOrphanFileCleanSupportHive.class}) @PrepareForTest({ JDBCSqlSessionFactoryProvider.class, ArcticMetaStore.class, diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/OutAmsTestBase.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/OutAmsTestBase.java index 499575771a..b9b66bfa34 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/OutAmsTestBase.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/OutAmsTestBase.java @@ -18,23 +18,9 @@ package com.netease.arctic.ams.server; -import com.netease.arctic.ams.server.controller.LoginControllerTest; -import com.netease.arctic.ams.server.controller.OptimizerControllerTest; -import com.netease.arctic.ams.server.controller.TableControllerTest; -import com.netease.arctic.ams.server.controller.TerminalControllerTest; -import com.netease.arctic.ams.server.optimize.TestBaseOptimizePlan; -import com.netease.arctic.ams.server.optimize.TestExpiredFileClean; -import com.netease.arctic.ams.server.optimize.TestMajorOptimizeCommit; -import com.netease.arctic.ams.server.optimize.TestMajorOptimizePlan; -import com.netease.arctic.ams.server.optimize.TestMinorOptimizeCommit; -import com.netease.arctic.ams.server.optimize.TestMinorOptimizePlan; -import com.netease.arctic.ams.server.optimize.TestOrphanFileClean; -import com.netease.arctic.ams.server.service.TestDDLTracerService; import org.junit.runner.RunWith; import org.junit.runners.Suite; import org.powermock.core.classloader.annotations.PowerMockIgnore; -import org.powermock.modules.junit4.PowerMockRunner; -import org.powermock.modules.junit4.PowerMockRunnerDelegate; @RunWith(Suite.class) @Suite.SuiteClasses({TestHighAvailabilityServices.class}) diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestBaseOptimizeBase.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestBaseOptimizeBase.java new file mode 100644 index 0000000000..a49800e171 --- /dev/null +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestBaseOptimizeBase.java @@ -0,0 +1,63 @@ +/* + * 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 com.netease.arctic.ams.server.optimize; + +import com.netease.arctic.TableTestBase; +import com.netease.arctic.ams.api.DataFileInfo; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.After; +import org.junit.Before; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.List; + +public class TestBaseOptimizeBase extends TableTestBase implements TestOptimizeBase { + protected List baseDataFilesInfo = new ArrayList<>(); + protected List posDeleteFilesInfo = new ArrayList<>(); + + @Before + public void initDataFileInfo() { + baseDataFilesInfo = new ArrayList<>(); + posDeleteFilesInfo = new ArrayList<>(); + } + + @After + public void clearDataFileInfo() { + baseDataFilesInfo.clear(); + posDeleteFilesInfo.clear(); + } + + @Override + public List baseRecords(int start, int length, Schema tableSchema) { + GenericRecord record = GenericRecord.create(tableSchema); + + ImmutableList.Builder builder = ImmutableList.builder(); + for (int i = start; i < start + length; i++) { + builder.add(record.copy(ImmutableMap.of("id", i, "name", "name", + "op_time", LocalDateTime.of(2022, 1, 1, 12, 0, 0)))); + } + + return builder.build(); + } +} diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestBaseOptimizePlan.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestBaseOptimizePlan.java deleted file mode 100644 index 3fe1e57403..0000000000 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestBaseOptimizePlan.java +++ /dev/null @@ -1,127 +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 com.netease.arctic.ams.server.optimize; - -import com.netease.arctic.TableTestBase; -import com.netease.arctic.ams.api.DataFileInfo; -import com.netease.arctic.ams.server.AmsTestBase; -import com.netease.arctic.ams.server.util.DataFileInfoUtils; -import com.netease.arctic.data.DataTreeNode; -import com.netease.arctic.data.DefaultKeyedFile; -import com.netease.arctic.io.writer.GenericBaseTaskWriter; -import com.netease.arctic.io.writer.GenericTaskWriters; -import com.netease.arctic.io.writer.SortedPosDeleteWriter; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.RowDelta; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; - -import java.io.IOException; -import java.time.LocalDateTime; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -public class TestBaseOptimizePlan extends TableTestBase { - protected List baseDataFilesInfo = new ArrayList<>(); - protected List posDeleteFilesInfo = new ArrayList<>(); - - protected List insertKeyedTableBaseDataFiles(long transactionId) throws IOException { - GenericBaseTaskWriter writer = GenericTaskWriters.builderFor(testKeyedTable) - .withTransactionId(transactionId).buildBaseWriter(); - - List baseDataFiles = new ArrayList<>(); - // write 1000 records to 2 partitions(2022-1-1\2022-1-2) - int length = 100; - for (int i = 1; i < length * 10; i = i + length) { - for (Record record : baseRecords(i, length)) { - writer.write(record); - } - WriteResult result = writer.complete(); - baseDataFiles.addAll(Arrays.asList(result.dataFiles())); - } - AppendFiles baseAppend = testKeyedTable.baseTable().newAppend(); - baseDataFiles.forEach(baseAppend::appendFile); - baseAppend.commit(); - long commitTime = System.currentTimeMillis(); - - baseDataFilesInfo = baseDataFiles.stream() - .map(dataFile -> DataFileInfoUtils.convertToDatafileInfo(dataFile, commitTime, testKeyedTable)) - .collect(Collectors.toList()); - return baseDataFiles; - } - - protected List insertBasePosDeleteFiles(long transactionId) throws IOException { - List dataFiles = insertKeyedTableBaseDataFiles(transactionId - 1); - Map> dataFilesPartitionMap = - new HashMap<>(dataFiles.stream().collect(Collectors.groupingBy(ContentFile::partition))); - List deleteFiles = new ArrayList<>(); - for (Map.Entry> dataFilePartitionMap : dataFilesPartitionMap.entrySet()) { - StructLike partition = dataFilePartitionMap.getKey(); - List partitionFiles = dataFilePartitionMap.getValue(); - Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() - .collect(Collectors.groupingBy(dataFile -> - DefaultKeyedFile.parseMetaFromFileName(dataFile.path().toString()).node()))); - for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { - DataTreeNode key = nodeFilePartitionMap.getKey(); - List nodeFiles = nodeFilePartitionMap.getValue(); - - // write pos delete - SortedPosDeleteWriter writer = GenericTaskWriters.builderFor(testKeyedTable) - .withTransactionId(transactionId).buildBasePosDeleteWriter(key.getMask(), key.getIndex(), partition); - for (DataFile nodeFile : nodeFiles) { - writer.delete(nodeFile.path(), 0); - } - deleteFiles.addAll(writer.complete()); - } - } - RowDelta rowDelta = testKeyedTable.baseTable().newRowDelta(); - deleteFiles.forEach(rowDelta::addDeletes); - rowDelta.commit(); - long commitTime = System.currentTimeMillis(); - - posDeleteFilesInfo.addAll(deleteFiles.stream() - .map(deleteFile -> DataFileInfoUtils.convertToDatafileInfo(deleteFile, commitTime, testKeyedTable)) - .collect(Collectors.toList())); - - return deleteFiles; - } - - protected List baseRecords(int start, int length) { - GenericRecord record = GenericRecord.create(TABLE_SCHEMA); - - ImmutableList.Builder builder = ImmutableList.builder(); - for (int i = start; i < start + length; i++) { - builder.add(record.copy(ImmutableMap.of("id", i, "name", "name" + i, "op_time", - LocalDateTime.of(2022, 1, i % 2 + 1, 12, 0, 0)))); - } - - return builder.build(); - } -} diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestExpiredFileClean.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestExpiredFileClean.java index a22be81725..0c95c54691 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestExpiredFileClean.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestExpiredFileClean.java @@ -51,7 +51,7 @@ public class TestExpiredFileClean extends TableTestBase { - private List changeTableFilesInfo = new ArrayList<>(); + private final List changeTableFilesInfo = new ArrayList<>(); @Test public void testDeleteChangeFiles() throws Exception { @@ -66,17 +66,13 @@ public void testDeleteChangeFiles() throws Exception { updateProperties.commit(); List existedDataFiles = new ArrayList<>(); testKeyedTable.changeTable().newScan().planFiles() - .forEach(fileScanTask -> { - existedDataFiles.add(fileScanTask.file()); - }); + .forEach(fileScanTask -> existedDataFiles.add(fileScanTask.file())); Assert.assertEquals(4, existedDataFiles.size()); TableExpireService.deleteChangeFile(testKeyedTable, changeTableFilesInfo); List currentDataFiles = new ArrayList<>(); testKeyedTable.changeTable().newScan().planFiles() - .forEach(fileScanTask -> { - currentDataFiles.add(fileScanTask.file()); - }); + .forEach(fileScanTask -> currentDataFiles.add(fileScanTask.file())); Assert.assertEquals(2, currentDataFiles.size()); } diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestExpiredFileCleanSupportHive.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestExpiredFileCleanSupportHive.java new file mode 100644 index 0000000000..2542e0fa87 --- /dev/null +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestExpiredFileCleanSupportHive.java @@ -0,0 +1,100 @@ +/* + * 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 com.netease.arctic.ams.server.optimize; + +import com.netease.arctic.ams.server.service.impl.TableExpireService; +import com.netease.arctic.ams.server.utils.HiveLocationUtils; +import com.netease.arctic.hive.io.writer.AdaptHiveGenericTaskWriterBuilder; +import com.netease.arctic.hive.table.HiveLocationKind; +import com.netease.arctic.hive.utils.HiveTableUtil; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.FileUtil; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFiles; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class TestExpiredFileCleanSupportHive extends TestSupportHiveBase { + @Test + public void testExpireTableFiles() throws Exception { + List hiveFiles = insertHiveDataFiles(testUnPartitionKeyedHiveTable, 1); + List s2Files = insertTableBaseDataFiles(testUnPartitionKeyedHiveTable, 2, new ArrayList<>()); + + DeleteFiles deleteHiveFiles = testUnPartitionKeyedHiveTable.baseTable().newDelete(); + for (DataFile hiveFile : hiveFiles) { + Assert.assertTrue(testUnPartitionKeyedHiveTable.io().exists(hiveFile.path().toString())); + deleteHiveFiles.deleteFile(hiveFile); + } + deleteHiveFiles.commit(); + + DeleteFiles deleteIcebergFiles = testUnPartitionKeyedHiveTable.baseTable().newDelete(); + for (DataFile s2File : s2Files) { + Assert.assertTrue(testUnPartitionKeyedHiveTable.io().exists(s2File.path().toString())); + deleteIcebergFiles.deleteFile(s2File); + } + deleteIcebergFiles.commit(); + + List s3Files = insertTableBaseDataFiles(testUnPartitionKeyedHiveTable, 3, new ArrayList<>()); + for (DataFile s3File : s3Files) { + Assert.assertTrue(testUnPartitionKeyedHiveTable.io().exists(s3File.path().toString())); + } + + Set hiveLocation = new HashSet<>(); + if (HiveTableUtil.isHive(testUnPartitionKeyedHiveTable)) { + hiveLocation.add(FileUtil.getFileDir(hiveFiles.get(0).path().toString())); + } + TableExpireService.expireSnapshots(testUnPartitionKeyedHiveTable.baseTable(), System.currentTimeMillis(), hiveLocation); + Assert.assertEquals(1, Iterables.size(testUnPartitionKeyedHiveTable.baseTable().snapshots())); + + for (DataFile hiveFile : hiveFiles) { + Assert.assertTrue(testUnPartitionKeyedHiveTable.io().exists(hiveFile.path().toString())); + } + for (DataFile s2File : s2Files) { + Assert.assertFalse(testUnPartitionKeyedHiveTable.io().exists(s2File.path().toString())); + } + for (DataFile s3File : s3Files) { + Assert.assertTrue(testUnPartitionKeyedHiveTable.io().exists(s3File.path().toString())); + } + } + + private List insertHiveDataFiles(ArcticTable arcticTable, long transactionId) throws Exception { + TaskWriter writer = AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .withTransactionId(transactionId) + .buildWriter(HiveLocationKind.INSTANT); + + List dataFiles = insertBaseDataFiles(writer, arcticTable.schema()); + UnkeyedTable baseTable = arcticTable.isKeyedTable() ? + arcticTable.asKeyedTable().baseTable() : arcticTable.asUnkeyedTable(); + AppendFiles baseAppend = baseTable.newAppend(); + dataFiles.forEach(baseAppend::appendFile); + baseAppend.commit(); + + return dataFiles; + } +} diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMajorOptimizeCommit.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMajorOptimizeCommit.java index 825492eda4..c3f32c0500 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMajorOptimizeCommit.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMajorOptimizeCommit.java @@ -19,16 +19,22 @@ package com.netease.arctic.ams.server.optimize; import com.netease.arctic.ams.api.OptimizeStatus; +import com.netease.arctic.ams.api.OptimizeType; import com.netease.arctic.ams.api.TreeNode; import com.netease.arctic.ams.server.model.BaseOptimizeTask; import com.netease.arctic.ams.server.model.BaseOptimizeTaskRuntime; import com.netease.arctic.ams.server.model.TableOptimizeRuntime; +import com.netease.arctic.ams.server.utils.JDBCSqlSessionFactoryProvider; import com.netease.arctic.data.DefaultKeyedFile; +import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.TableProperties; import com.netease.arctic.utils.SerializationUtil; import org.apache.iceberg.DataFile; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; import java.util.HashMap; import java.util.HashSet; @@ -37,11 +43,25 @@ import java.util.Set; import java.util.stream.Collectors; -public class TestMajorOptimizeCommit extends TestMajorOptimizePlan { +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.when; + +@PrepareForTest({ + JDBCSqlSessionFactoryProvider.class +}) +@PowerMockIgnore({"org.apache.logging.log4j.*", "javax.management.*", "org.apache.http.conn.ssl.*", + "com.amazonaws.http.conn.ssl.*", + "javax.net.ssl.*", "org.apache.hadoop.*", "javax.*", "com.sun.org.apache.*", "org.apache.xerces.*"}) +public class TestMajorOptimizeCommit extends TestBaseOptimizeBase { + @Before + public void mock() { + mockStatic(JDBCSqlSessionFactoryProvider.class); + when(JDBCSqlSessionFactoryProvider.get()).thenReturn(null); + } @Test public void testMajorOptimizeCommit() throws Exception { - insertBasePosDeleteFiles(2); + insertBasePosDeleteFiles(testKeyedTable, 2, baseDataFilesInfo, posDeleteFilesInfo); Set oldDataFilesPath = new HashSet<>(); Set oldDeleteFilesPath = new HashSet<>(); @@ -51,7 +71,6 @@ public void testMajorOptimizeCommit() throws Exception { fileScanTask.deletes().forEach(deleteFile -> oldDeleteFilesPath.add((String) deleteFile.path())); }); - //testKeyedTable.properties().put(TableProperties.MAJOR_OPTIMIZE_TRIGGER_DELETE_FILE_SIZE_BYTES, "0"); testKeyedTable.updateProperties(). set(TableProperties.MAJOR_OPTIMIZE_TRIGGER_DELETE_FILE_SIZE_BYTES, "0").commit(); TableOptimizeRuntime tableOptimizeRuntime = new TableOptimizeRuntime(testKeyedTable.id()); @@ -60,7 +79,7 @@ public void testMajorOptimizeCommit() throws Exception { new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); List tasks = majorOptimizePlan.plan(); - Map> resultFiles = generateTargetFiles(); + Map> resultFiles = generateTargetFiles(testKeyedTable); List taskItems = tasks.stream().map(task -> { BaseOptimizeTaskRuntime optimizeRuntime = new BaseOptimizeTaskRuntime(task.getTaskId()); List targetFiles = resultFiles.get(task.getSourceNodes().get(0)); @@ -93,8 +112,8 @@ public void testMajorOptimizeCommit() throws Exception { Assert.assertNotEquals(oldDeleteFilesPath, newDeleteFilesPath); } - private Map> generateTargetFiles() throws Exception { - List dataFiles = insertKeyedTableBaseDataFiles(3); + private Map> generateTargetFiles(ArcticTable arcticTable) throws Exception { + List dataFiles = insertOptimizeTargetDataFiles(arcticTable, OptimizeType.Major, 3); return dataFiles.stream().collect(Collectors.groupingBy(dataFile -> { DefaultKeyedFile keyedFile = new DefaultKeyedFile(dataFile); return keyedFile.node().toAmsTreeNode(); diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMajorOptimizePlan.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMajorOptimizePlan.java index e26c15df3d..0e496cd1be 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMajorOptimizePlan.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMajorOptimizePlan.java @@ -19,10 +19,10 @@ package com.netease.arctic.ams.server.optimize; +import com.netease.arctic.ams.api.OptimizeType; import com.netease.arctic.ams.server.model.BaseOptimizeTask; import com.netease.arctic.ams.server.model.TableOptimizeRuntime; import com.netease.arctic.ams.server.util.DataFileInfoUtils; -import com.netease.arctic.ams.server.utils.JDBCSqlSessionFactoryProvider; import com.netease.arctic.table.TableProperties; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; @@ -37,12 +37,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; -import org.junit.runner.RunWith; -import org.powermock.core.classloader.annotations.PowerMockIgnore; -import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; import java.io.IOException; import java.time.LocalDateTime; @@ -52,31 +47,17 @@ import java.util.Locale; import java.util.stream.Collectors; -import static org.powermock.api.mockito.PowerMockito.mockStatic; -import static org.powermock.api.mockito.PowerMockito.when; - -@PrepareForTest({ - JDBCSqlSessionFactoryProvider.class -}) -@PowerMockIgnore({"org.apache.logging.log4j.*", "javax.management.*", "org.apache.http.conn.ssl.*", - "com.amazonaws.http.conn.ssl.*", - "javax.net.ssl.*", "org.apache.hadoop.*", "javax.*", "com.sun.org.apache.*", "org.apache.xerces.*"}) -public class TestMajorOptimizePlan extends TestBaseOptimizePlan { - - @Before - public void mock() { - mockStatic(JDBCSqlSessionFactoryProvider.class); - when(JDBCSqlSessionFactoryProvider.get()).thenReturn(null); - } - +public class TestMajorOptimizePlan extends TestBaseOptimizeBase { @Test public void testKeyedTableMajorOptimize() throws IOException { - insertBasePosDeleteFiles(2); + insertBasePosDeleteFiles(testKeyedTable, 2, baseDataFilesInfo, posDeleteFilesInfo); MajorOptimizePlan majorOptimizePlan = new MajorOptimizePlan(testKeyedTable, new TableOptimizeRuntime(testKeyedTable.id()), baseDataFilesInfo, posDeleteFilesInfo, new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); List tasks = majorOptimizePlan.plan(); + + Assert.assertEquals(OptimizeType.Major, tasks.get(0).getTaskId().getType()); Assert.assertEquals(4, tasks.size()); Assert.assertEquals(10, tasks.get(0).getBaseFiles().size()); Assert.assertEquals(1, tasks.get(0).getPosDeleteFiles().size()); @@ -86,8 +67,8 @@ public void testKeyedTableMajorOptimize() throws IOException { } @Test - public void testRemovePosDeleteMajorOptimize() throws IOException { - insertBasePosDeleteFiles(2); + public void testKeyedTableFullMajorOptimize() throws IOException { + insertBasePosDeleteFiles(testKeyedTable, 2, baseDataFilesInfo, posDeleteFilesInfo); testKeyedTable.updateProperties() .set(TableProperties.MAJOR_OPTIMIZE_TRIGGER_DELETE_FILE_SIZE_BYTES, "0") @@ -97,6 +78,8 @@ public void testRemovePosDeleteMajorOptimize() throws IOException { new TableOptimizeRuntime(testKeyedTable.id()), baseDataFilesInfo, posDeleteFilesInfo, new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); List tasks = majorOptimizePlan.plan(); + + Assert.assertEquals(OptimizeType.FullMajor, tasks.get(0).getTaskId().getType()); Assert.assertEquals(4, tasks.size()); Assert.assertEquals(10, tasks.get(0).getBaseFiles().size()); Assert.assertEquals(1, tasks.get(0).getPosDeleteFiles().size()); @@ -113,6 +96,8 @@ public void testUnKeyedTableMajorOptimize() { new TableOptimizeRuntime(testTable.id()), baseDataFilesInfo, posDeleteFilesInfo, new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); List tasks = majorOptimizePlan.plan(); + + Assert.assertEquals(OptimizeType.Major, tasks.get(0).getTaskId().getType()); Assert.assertEquals(2, tasks.size()); Assert.assertEquals(5, tasks.get(0).getBaseFileCnt()); Assert.assertEquals(0, tasks.get(0).getPosDeleteFiles().size()); @@ -120,6 +105,66 @@ public void testUnKeyedTableMajorOptimize() { Assert.assertEquals(0, tasks.get(0).getDeleteFileCnt()); } + @Test + public void testUnKeyedTableFullMajorOptimize() { + testTable.updateProperties() + .set(TableProperties.FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL, "86400000") + .commit(); + insertUnKeyedTableDataFiles(); + + MajorOptimizePlan majorOptimizePlan = new MajorOptimizePlan(testTable, + new TableOptimizeRuntime(testTable.id()), baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = majorOptimizePlan.plan(); + + Assert.assertEquals(OptimizeType.FullMajor, tasks.get(0).getTaskId().getType()); + Assert.assertEquals(2, tasks.size()); + Assert.assertEquals(5, tasks.get(0).getBaseFileCnt()); + Assert.assertEquals(0, tasks.get(0).getPosDeleteFiles().size()); + Assert.assertEquals(0, tasks.get(0).getInsertFileCnt()); + Assert.assertEquals(0, tasks.get(0).getDeleteFileCnt()); + } + + @Test + public void testNoPartitionTableMajorOptimize() throws IOException { + insertBasePosDeleteFiles(testNoPartitionTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + + MajorOptimizePlan majorOptimizePlan = new MajorOptimizePlan(testNoPartitionTable, + new TableOptimizeRuntime(testNoPartitionTable.id()), baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = majorOptimizePlan.plan(); + + Assert.assertEquals(OptimizeType.Major, tasks.get(0).getTaskId().getType()); + Assert.assertEquals(4, tasks.size()); + Assert.assertEquals(10, tasks.get(0).getBaseFiles().size()); + Assert.assertEquals(1, tasks.get(0).getPosDeleteFiles().size()); + Assert.assertEquals(0, tasks.get(0).getInsertFileCnt()); + Assert.assertEquals(0, tasks.get(0).getDeleteFileCnt()); + Assert.assertEquals(0, tasks.get(0).getIsDeletePosDelete()); + } + + @Test + public void testNoPartitionTableFullMajorOptimize() throws IOException { + insertBasePosDeleteFiles(testNoPartitionTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + + testNoPartitionTable.updateProperties() + .set(TableProperties.MAJOR_OPTIMIZE_TRIGGER_DELETE_FILE_SIZE_BYTES, "0") + .commit(); + + MajorOptimizePlan majorOptimizePlan = new MajorOptimizePlan(testNoPartitionTable, + new TableOptimizeRuntime(testNoPartitionTable.id()), baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = majorOptimizePlan.plan(); + + Assert.assertEquals(OptimizeType.FullMajor, tasks.get(0).getTaskId().getType()); + Assert.assertEquals(4, tasks.size()); + Assert.assertEquals(10, tasks.get(0).getBaseFiles().size()); + Assert.assertEquals(1, tasks.get(0).getPosDeleteFiles().size()); + Assert.assertEquals(0, tasks.get(0).getInsertFileCnt()); + Assert.assertEquals(0, tasks.get(0).getDeleteFileCnt()); + Assert.assertEquals(1, tasks.get(0).getIsDeletePosDelete()); + } + private void insertUnKeyedTableDataFiles() { List dataFiles = insertUnKeyedTableDataFile(FILE_A.partition(), LocalDateTime.of(2022, 1, 1, 12, 0, 0), 5); dataFiles.addAll(insertUnKeyedTableDataFile(FILE_B.partition(), LocalDateTime.of(2022, 1, 2, 12, 0, 0), 5)); diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMinorOptimizeCommit.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMinorOptimizeCommit.java index 2026a52c11..2c59f81132 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMinorOptimizeCommit.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMinorOptimizeCommit.java @@ -28,21 +28,25 @@ import com.netease.arctic.data.DataTreeNode; import com.netease.arctic.data.DefaultKeyedFile; import com.netease.arctic.utils.SerializationUtil; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.util.StructLikeMap; import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import static org.powermock.api.mockito.PowerMockito.mockStatic; @@ -64,9 +68,9 @@ public void mock() { @Test public void testMinorOptimizeCommit() throws Exception { - insertBasePosDeleteFiles(2); - insertChangeDeleteFiles(3); - insertChangeDataFiles(4); + insertBasePosDeleteFiles(testKeyedTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + insertChangeDeleteFiles(testKeyedTable, 3); + List dataFiles = insertChangeDataFiles(testKeyedTable,4); Set oldDataFilesPath = new HashSet<>(); Set oldDeleteFilesPath = new HashSet<>(); @@ -84,10 +88,15 @@ public void testMinorOptimizeCommit() throws Exception { new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); List tasks = minorOptimizePlan.plan(); - Map> resultFiles = generateTargetFiles(); + List> resultFiles = new ArrayList<>(generateTargetFiles(dataFiles).values()); + Set partitionData = new HashSet<>(); + for (List resultFile : resultFiles) { + partitionData.addAll(resultFile.stream().map(ContentFile::partition).collect(Collectors.toList())); + } + AtomicInteger i = new AtomicInteger(); List taskItems = tasks.stream().map(task -> { BaseOptimizeTaskRuntime optimizeRuntime = new BaseOptimizeTaskRuntime(task.getTaskId()); - List targetFiles = resultFiles.get(task.getSourceNodes().get(0)); + List targetFiles = resultFiles.get(i.getAndIncrement()); optimizeRuntime.setPreparedTime(System.currentTimeMillis()); optimizeRuntime.setStatus(OptimizeStatus.Prepared); optimizeRuntime.setReportTime(System.currentTimeMillis()); @@ -96,6 +105,9 @@ public void testMinorOptimizeCommit() throws Exception { optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); } + List finalTargetFiles = optimizeRuntime.getTargetFiles(); + finalTargetFiles.addAll(task.getInsertFiles()); + optimizeRuntime.setTargetFiles(finalTargetFiles); // 1min optimizeRuntime.setCostTime(60 * 1000); return new OptimizeTaskItem(task, optimizeRuntime); @@ -113,12 +125,79 @@ public void testMinorOptimizeCommit() throws Exception { newDataFilesPath.add((String) fileScanTask.file().path()); fileScanTask.deletes().forEach(deleteFile -> newDeleteFilesPath.add((String) deleteFile.path())); }); + + StructLikeMap maxTxId = testKeyedTable.partitionMaxTransactionId(); + for (StructLike partitionDatum : partitionData) { + Assert.assertEquals(4L, (long) maxTxId.get(partitionDatum)); + } + Assert.assertNotEquals(oldDataFilesPath, newDataFilesPath); + Assert.assertNotEquals(oldDeleteFilesPath, newDeleteFilesPath); + } + + @Test + public void testNoPartitionTableMinorOptimizeCommit() throws Exception { + insertBasePosDeleteFiles(testNoPartitionTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + insertChangeDeleteFiles(testNoPartitionTable, 3); + List dataFiles = insertChangeDataFiles(testNoPartitionTable, 4); + + Set oldDataFilesPath = new HashSet<>(); + Set oldDeleteFilesPath = new HashSet<>(); + testNoPartitionTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> { + oldDataFilesPath.add((String) fileScanTask.file().path()); + fileScanTask.deletes().forEach(deleteFile -> oldDeleteFilesPath.add((String) deleteFile.path())); + }); + + List changeTableFilesInfo = new ArrayList<>(changeInsertFilesInfo); + changeTableFilesInfo.addAll(changeDeleteFilesInfo); + TableOptimizeRuntime tableOptimizeRuntime = new TableOptimizeRuntime(testNoPartitionTable.id()); + MinorOptimizePlan minorOptimizePlan = new MinorOptimizePlan(testNoPartitionTable, + tableOptimizeRuntime, baseDataFilesInfo, changeTableFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = minorOptimizePlan.plan(); + + List> resultFiles = new ArrayList<>(generateTargetFiles(dataFiles).values()); + AtomicInteger i = new AtomicInteger(); + List taskItems = tasks.stream().map(task -> { + BaseOptimizeTaskRuntime optimizeRuntime = new BaseOptimizeTaskRuntime(task.getTaskId()); + List targetFiles = resultFiles.get(i.getAndIncrement()); + optimizeRuntime.setPreparedTime(System.currentTimeMillis()); + optimizeRuntime.setStatus(OptimizeStatus.Prepared); + optimizeRuntime.setReportTime(System.currentTimeMillis()); + optimizeRuntime.setNewFileCnt(targetFiles == null ? 0 : targetFiles.size()); + if (targetFiles != null) { + optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + } + List finalTargetFiles = optimizeRuntime.getTargetFiles(); + finalTargetFiles.addAll(task.getInsertFiles()); + optimizeRuntime.setTargetFiles(finalTargetFiles); + // 1min + optimizeRuntime.setCostTime(60 * 1000); + return new OptimizeTaskItem(task, optimizeRuntime); + }).collect(Collectors.toList()); + Map> partitionTasks = taskItems.stream() + .collect(Collectors.groupingBy(taskItem -> taskItem.getOptimizeTask().getPartition())); + + BaseOptimizeCommit optimizeCommit = new BaseOptimizeCommit(testNoPartitionTable, partitionTasks); + optimizeCommit.commit(tableOptimizeRuntime); + + Set newDataFilesPath = new HashSet<>(); + Set newDeleteFilesPath = new HashSet<>(); + testNoPartitionTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> { + newDataFilesPath.add((String) fileScanTask.file().path()); + fileScanTask.deletes().forEach(deleteFile -> newDeleteFilesPath.add((String) deleteFile.path())); + }); + + StructLikeMap maxTxId = testNoPartitionTable.partitionMaxTransactionId(); + Assert.assertEquals(4L, (long) maxTxId.get(null)); Assert.assertNotEquals(oldDataFilesPath, newDataFilesPath); Assert.assertNotEquals(oldDeleteFilesPath, newDeleteFilesPath); } - private Map> generateTargetFiles() throws Exception { - List deleteFiles = insertBasePosDeleteFiles(6); + private Map> generateTargetFiles(List dataFiles) throws Exception { + List deleteFiles = insertOptimizeTargetDeleteFiles(testKeyedTable, dataFiles, 5); return deleteFiles.stream().collect(Collectors.groupingBy(deleteFile -> { DataTreeNode dataTreeNode = DefaultKeyedFile.parseMetaFromFileName(deleteFile.path().toString()).node(); return new TreeNode(dataTreeNode.mask(), dataTreeNode.index()); diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMinorOptimizePlan.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMinorOptimizePlan.java index 4f5a839680..e04f1ac461 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMinorOptimizePlan.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestMinorOptimizePlan.java @@ -23,11 +23,13 @@ import com.netease.arctic.ams.server.model.TableOptimizeRuntime; import com.netease.arctic.ams.server.util.DataFileInfoUtils; import com.netease.arctic.data.ChangeAction; -import com.netease.arctic.io.writer.GenericChangeTaskWriter; -import com.netease.arctic.io.writer.GenericTaskWriters; +import com.netease.arctic.hive.io.writer.AdaptHiveGenericTaskWriterBuilder; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.ChangeLocationKind; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriteResult; import org.junit.Assert; import org.junit.Test; @@ -39,15 +41,15 @@ import java.util.List; import java.util.stream.Collectors; -public class TestMinorOptimizePlan extends TestBaseOptimizePlan { +public class TestMinorOptimizePlan extends TestBaseOptimizeBase { protected List changeInsertFilesInfo = new ArrayList<>(); protected List changeDeleteFilesInfo = new ArrayList<>(); @Test public void testMinorOptimize() throws IOException { - insertBasePosDeleteFiles(2); - insertChangeDeleteFiles(3); - insertChangeDataFiles(4); + insertBasePosDeleteFiles(testKeyedTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + insertChangeDeleteFiles(testKeyedTable,3); + insertChangeDataFiles(testKeyedTable,4); List changeTableFilesInfo = new ArrayList<>(changeInsertFilesInfo); changeTableFilesInfo.addAll(changeDeleteFilesInfo); @@ -63,53 +65,57 @@ public void testMinorOptimize() throws IOException { Assert.assertEquals(0, tasks.get(0).getIsDeletePosDelete()); } - protected void insertChangeDeleteFiles(long transactionId) throws IOException { - GenericChangeTaskWriter writer = GenericTaskWriters.builderFor(testKeyedTable) + protected void insertChangeDeleteFiles(ArcticTable arcticTable, long transactionId) throws IOException { + TaskWriter writer = AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) .withChangeAction(ChangeAction.DELETE) - .withTransactionId(transactionId).buildChangeWriter(); + .withTransactionId(transactionId) + .buildWriter(ChangeLocationKind.INSTANT); List changeDeleteFiles = new ArrayList<>(); - // delete 1000 records in 2 partitions(2022-1-1\2022-1-2) + // delete 1000 records in 1 partitions(2022-1-1) int length = 100; for (int i = 1; i < length * 10; i = i + length) { - for (Record record : baseRecords(i, length)) { + for (Record record : baseRecords(i, length, arcticTable.schema())) { writer.write(record); } WriteResult result = writer.complete(); changeDeleteFiles.addAll(Arrays.asList(result.dataFiles())); } - AppendFiles baseAppend = testKeyedTable.changeTable().newAppend(); + AppendFiles baseAppend = arcticTable.asKeyedTable().changeTable().newAppend(); changeDeleteFiles.forEach(baseAppend::appendFile); baseAppend.commit(); long commitTime = System.currentTimeMillis(); changeDeleteFilesInfo = changeDeleteFiles.stream() - .map(deleteFile -> DataFileInfoUtils.convertToDatafileInfo(deleteFile, commitTime, testKeyedTable)) + .map(deleteFile -> DataFileInfoUtils.convertToDatafileInfo(deleteFile, commitTime, arcticTable)) .collect(Collectors.toList()); } - protected void insertChangeDataFiles(long transactionId) throws IOException { - GenericChangeTaskWriter writer = GenericTaskWriters.builderFor(testKeyedTable) + protected List insertChangeDataFiles(ArcticTable arcticTable, long transactionId) throws IOException { + TaskWriter writer = AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) .withChangeAction(ChangeAction.INSERT) - .withTransactionId(transactionId).buildChangeWriter(); + .withTransactionId(transactionId) + .buildWriter(ChangeLocationKind.INSTANT); List changeInsertFiles = new ArrayList<>(); - // write 1000 records to 2 partitions(2022-1-1\2022-1-2) + // write 1000 records to 1 partitions(2022-1-1) int length = 100; for (int i = 1; i < length * 10; i = i + length) { - for (Record record : baseRecords(i, length)) { + for (Record record : baseRecords(i, length, arcticTable.schema())) { writer.write(record); } WriteResult result = writer.complete(); changeInsertFiles.addAll(Arrays.asList(result.dataFiles())); } - AppendFiles baseAppend = testKeyedTable.changeTable().newAppend(); + AppendFiles baseAppend = arcticTable.asKeyedTable().changeTable().newAppend(); changeInsertFiles.forEach(baseAppend::appendFile); baseAppend.commit(); long commitTime = System.currentTimeMillis(); changeInsertFilesInfo = changeInsertFiles.stream() - .map(dataFile -> DataFileInfoUtils.convertToDatafileInfo(dataFile, commitTime, testKeyedTable)) + .map(dataFile -> DataFileInfoUtils.convertToDatafileInfo(dataFile, commitTime, arcticTable)) .collect(Collectors.toList()); + + return changeInsertFiles; } } diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestOptimizeBase.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestOptimizeBase.java new file mode 100644 index 0000000000..d60fed7ef9 --- /dev/null +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestOptimizeBase.java @@ -0,0 +1,188 @@ +/* + * 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 com.netease.arctic.ams.server.optimize; + +import com.netease.arctic.ams.api.DataFileInfo; +import com.netease.arctic.ams.api.OptimizeType; +import com.netease.arctic.ams.server.util.DataFileInfoUtils; +import com.netease.arctic.data.DataTreeNode; +import com.netease.arctic.data.DefaultKeyedFile; +import com.netease.arctic.hive.io.writer.AdaptHiveGenericTaskWriterBuilder; +import com.netease.arctic.io.writer.SortedPosDeleteWriter; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.BaseLocationKind; +import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.table.WriteOperationKind; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public interface TestOptimizeBase { + List baseRecords(int start, int length, Schema tableSchema); + + default List insertTableBaseDataFiles(ArcticTable arcticTable, long transactionId, List baseDataFilesInfo) throws IOException { + TaskWriter writer = arcticTable.isKeyedTable() ? + AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .withTransactionId(transactionId) + .buildWriter(BaseLocationKind.INSTANT) : + AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .buildWriter(BaseLocationKind.INSTANT) ; + + List baseDataFiles = insertBaseDataFiles(writer, arcticTable.schema()); + UnkeyedTable baseTable = arcticTable.isKeyedTable() ? + arcticTable.asKeyedTable().baseTable() : arcticTable.asUnkeyedTable(); + AppendFiles baseAppend = baseTable.newAppend(); + baseDataFiles.forEach(baseAppend::appendFile); + baseAppend.commit(); + + long commitTime = System.currentTimeMillis(); + + baseDataFilesInfo.addAll(baseDataFiles.stream() + .map(dataFile -> DataFileInfoUtils.convertToDatafileInfo(dataFile, commitTime, arcticTable)) + .collect(Collectors.toList())); + return baseDataFiles; + } + + default List insertOptimizeTargetDataFiles(ArcticTable arcticTable, + OptimizeType optimizeType, + long transactionId) throws IOException { + WriteOperationKind writeOperationKind = WriteOperationKind.MAJOR_OPTIMIZE; + switch (optimizeType) { + case FullMajor: + writeOperationKind = WriteOperationKind.FULL_OPTIMIZE; + break; + case Major: + break; + case Minor: + writeOperationKind = WriteOperationKind.MINOR_OPTIMIZE; + break; + } + TaskWriter writer = arcticTable.isKeyedTable() ? + AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .withTransactionId(transactionId) + .buildWriter(writeOperationKind) : + AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .buildWriter(writeOperationKind); + + return insertBaseDataFiles(writer, arcticTable.schema()); + } + + default List insertBasePosDeleteFiles(ArcticTable arcticTable, + long transactionId, + List baseDataFilesInfo, + List posDeleteFilesInfo) throws IOException { + List dataFiles = insertTableBaseDataFiles(arcticTable, transactionId - 1, baseDataFilesInfo); + Map> dataFilesPartitionMap = + new HashMap<>(dataFiles.stream().collect(Collectors.groupingBy(ContentFile::partition))); + List deleteFiles = new ArrayList<>(); + for (Map.Entry> dataFilePartitionMap : dataFilesPartitionMap.entrySet()) { + StructLike partition = dataFilePartitionMap.getKey(); + List partitionFiles = dataFilePartitionMap.getValue(); + Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() + .collect(Collectors.groupingBy(dataFile -> + DefaultKeyedFile.parseMetaFromFileName(dataFile.path().toString()).node()))); + for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { + DataTreeNode key = nodeFilePartitionMap.getKey(); + List nodeFiles = nodeFilePartitionMap.getValue(); + + // write pos delete + SortedPosDeleteWriter posDeleteWriter = AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .withTransactionId(transactionId) + .buildBasePosDeleteWriter(key.mask(), key.index(), partition); + for (DataFile nodeFile : nodeFiles) { + posDeleteWriter.delete(nodeFile.path(), 0); + } + deleteFiles.addAll(posDeleteWriter.complete()); + } + } + + UnkeyedTable baseTable = arcticTable.isKeyedTable() ? + arcticTable.asKeyedTable().baseTable() : arcticTable.asUnkeyedTable(); + RowDelta rowDelta = baseTable.newRowDelta(); + deleteFiles.forEach(rowDelta::addDeletes); + rowDelta.commit(); + + long commitTime = System.currentTimeMillis(); + posDeleteFilesInfo.addAll(deleteFiles.stream() + .map(deleteFile -> DataFileInfoUtils.convertToDatafileInfo(deleteFile, commitTime, arcticTable.asKeyedTable())) + .collect(Collectors.toList())); + + return deleteFiles; + } + + default List insertOptimizeTargetDeleteFiles(ArcticTable arcticTable, + List dataFiles, + long transactionId) throws IOException { + Map> dataFilesPartitionMap = + new HashMap<>(dataFiles.stream().collect(Collectors.groupingBy(ContentFile::partition))); + List deleteFiles = new ArrayList<>(); + for (Map.Entry> dataFilePartitionMap : dataFilesPartitionMap.entrySet()) { + StructLike partition = dataFilePartitionMap.getKey(); + List partitionFiles = dataFilePartitionMap.getValue(); + Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() + .collect(Collectors.groupingBy(dataFile -> + DefaultKeyedFile.parseMetaFromFileName(dataFile.path().toString()).node()))); + for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { + DataTreeNode key = nodeFilePartitionMap.getKey(); + List nodeFiles = nodeFilePartitionMap.getValue(); + + // write pos delete + SortedPosDeleteWriter posDeleteWriter = AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .withTransactionId(transactionId) + .buildBasePosDeleteWriter(key.mask(), key.index(), partition); + for (DataFile nodeFile : nodeFiles) { + posDeleteWriter.delete(nodeFile.path(), 0); + } + deleteFiles.addAll(posDeleteWriter.complete()); + } + } + + return deleteFiles; + } + + default List insertBaseDataFiles(TaskWriter writer, Schema schema) throws IOException { + List baseDataFiles = new ArrayList<>(); + // write 1000 records to 1 partitions(name="name) + int length = 100; + for (int i = 1; i < length * 10; i = i + length) { + for (Record record : baseRecords(i, length, schema)) { + writer.write(record); + } + WriteResult result = writer.complete(); + baseDataFiles.addAll(Arrays.asList(result.dataFiles())); + } + + return baseDataFiles; + } +} diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestOrphanFileClean.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestOrphanFileClean.java index be2f5405a4..c0c56b6c93 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestOrphanFileClean.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestOrphanFileClean.java @@ -18,6 +18,7 @@ package com.netease.arctic.ams.server.optimize; +import com.netease.arctic.TableTestBase; import com.netease.arctic.ams.api.DataFileInfo; import com.netease.arctic.ams.api.TableIdentifier; import com.netease.arctic.ams.server.service.ServiceContainer; @@ -28,10 +29,8 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.junit.runner.RunWith; import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; -import org.powermock.modules.junit4.PowerMockRunner; import java.io.File; import java.util.Collections; @@ -48,7 +47,7 @@ @PowerMockIgnore({"org.apache.logging.log4j.*", "javax.management.*", "org.apache.http.conn.ssl.*", "com.amazonaws.http.conn.ssl.*", "javax.net.ssl.*", "org.apache.hadoop.*", "javax.*", "com.sun.org.apache.*", "org.apache.xerces.*"}) -public class TestOrphanFileClean extends TestBaseOptimizePlan { +public class TestOrphanFileClean extends TableTestBase { @Before public void mock() { diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestOrphanFileCleanSupportHive.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestOrphanFileCleanSupportHive.java new file mode 100644 index 0000000000..fef6e389fa --- /dev/null +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestOrphanFileCleanSupportHive.java @@ -0,0 +1,87 @@ +/* + * 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 com.netease.arctic.ams.server.optimize; + +import com.netease.arctic.ams.api.DataFileInfo; +import com.netease.arctic.ams.api.TableIdentifier; +import com.netease.arctic.ams.server.service.ServiceContainer; +import com.netease.arctic.ams.server.service.impl.FileInfoCacheService; +import com.netease.arctic.ams.server.service.impl.OrphanFilesCleanService; +import com.netease.arctic.ams.server.utils.JDBCSqlSessionFactoryProvider; +import org.apache.iceberg.io.OutputFile; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; + +import java.io.File; +import java.util.Collections; +import java.util.List; + +import static com.netease.arctic.ams.server.service.impl.OrphanFilesCleanService.DATA_FOLDER_NAME; +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.when; + +@PrepareForTest({ + ServiceContainer.class, + JDBCSqlSessionFactoryProvider.class +}) +@PowerMockIgnore({"org.apache.logging.log4j.*", "javax.management.*", "org.apache.http.conn.ssl.*", + "com.amazonaws.http.conn.ssl.*", + "javax.net.ssl.*", "org.apache.hadoop.*", "javax.*", "com.sun.org.apache.*", "org.apache.xerces.*"}) +public class TestOrphanFileCleanSupportHive extends TestSupportHiveBase { + @Before + public void mock() { + mockStatic(JDBCSqlSessionFactoryProvider.class); + mockStatic(ServiceContainer.class); + when(JDBCSqlSessionFactoryProvider.get()).thenReturn(null); + TestOrphanFileCleanSupportHive.FakeFileInfoCacheService fakeFileInfoCacheService = new TestOrphanFileCleanSupportHive.FakeFileInfoCacheService(); + when(ServiceContainer.getFileInfoCacheService()).thenReturn(fakeFileInfoCacheService); + } + + @Test + public void orphanDataFileClean() { + String baseOrphanFilePath = testUnPartitionKeyedHiveTable.baseTable().location() + + File.separator + DATA_FOLDER_NAME + File.separator + "orphan.parquet"; + String hiveOrphanFilePath = testUnPartitionKeyedHiveTable.hiveLocation() + + File.separator + DATA_FOLDER_NAME + File.separator + "orphan.parquet"; + OutputFile baseOrphanDataFile = testUnPartitionKeyedHiveTable.io().newOutputFile(baseOrphanFilePath); + baseOrphanDataFile.createOrOverwrite(); + OutputFile changeOrphanDataFile = testUnPartitionKeyedHiveTable.io().newOutputFile(hiveOrphanFilePath); + changeOrphanDataFile.createOrOverwrite(); + Assert.assertTrue(testUnPartitionKeyedHiveTable.io().exists(baseOrphanFilePath)); + Assert.assertTrue(testUnPartitionKeyedHiveTable.io().exists(hiveOrphanFilePath)); + OrphanFilesCleanService.clean(testUnPartitionKeyedHiveTable, System.currentTimeMillis(), true, "all", false); + Assert.assertFalse(testUnPartitionKeyedHiveTable.io().exists(baseOrphanFilePath)); + Assert.assertTrue(testUnPartitionKeyedHiveTable.io().exists(hiveOrphanFilePath)); + } + + private static class FakeFileInfoCacheService extends FileInfoCacheService { + + public FakeFileInfoCacheService() { + super(); + } + + @Override + public List getOptimizeDatafiles(TableIdentifier tableIdentifier, String tableType) { + return Collections.emptyList(); + } + } +} diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestSupportHiveBase.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestSupportHiveBase.java new file mode 100644 index 0000000000..0ed8858303 --- /dev/null +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestSupportHiveBase.java @@ -0,0 +1,220 @@ +/* + * 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 com.netease.arctic.ams.server.optimize; + +import com.netease.arctic.CatalogMetaTestUtil; +import com.netease.arctic.ams.api.CatalogMeta; +import com.netease.arctic.ams.api.DataFileInfo; +import com.netease.arctic.ams.api.MockArcticMetastoreServer; +import com.netease.arctic.ams.api.properties.CatalogMetaProperties; +import com.netease.arctic.ams.server.AmsTestBase; +import com.netease.arctic.ams.server.service.ServiceContainer; +import com.netease.arctic.catalog.ArcticCatalog; +import com.netease.arctic.catalog.CatalogLoader; +import com.netease.arctic.hive.HMSMockServer; +import com.netease.arctic.hive.table.KeyedHiveTable; +import com.netease.arctic.hive.table.UnkeyedHiveTable; +import com.netease.arctic.table.PrimaryKeySpec; +import com.netease.arctic.table.TableIdentifier; +import org.apache.commons.io.FileUtils; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.assertj.core.util.Lists; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.math.BigDecimal; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +import static com.netease.arctic.ams.api.properties.CatalogMetaProperties.CATALOG_TYPE_HIVE; + +public class TestSupportHiveBase implements TestOptimizeBase { + public static ArcticCatalog hiveCatalog; + public static final String AMS_TEST_HIVE_CATALOG_NAME = "ams_hive_test_catalog"; + public static final String AMS_TEST_HIVE_DB_NAME = "ams_hive_test_db"; + + private static final File testHiveTableBaseDir = new File("/hive_tmp"); + private static final File testHiveBaseDir = new File("unit_test_hive_base_tmp"); + + public static final TableIdentifier HIVE_TABLE_ID = + TableIdentifier.of(AMS_TEST_HIVE_CATALOG_NAME, AMS_TEST_HIVE_DB_NAME, "test_hive_table"); + public static final TableIdentifier HIVE_PK_TABLE_ID = + TableIdentifier.of(AMS_TEST_HIVE_CATALOG_NAME, AMS_TEST_HIVE_DB_NAME, "test_pk_hive_table"); + + public static final TableIdentifier UN_PARTITION_HIVE_TABLE_ID = + TableIdentifier.of(AMS_TEST_HIVE_CATALOG_NAME, AMS_TEST_HIVE_DB_NAME, "un_partition_test_hive_table"); + public static final TableIdentifier UN_PARTITION_HIVE_PK_TABLE_ID = + TableIdentifier.of(AMS_TEST_HIVE_CATALOG_NAME, AMS_TEST_HIVE_DB_NAME, "un_partition_test_pk_hive_table"); + + public static final Schema HIVE_TABLE_SCHEMA = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "op_time", Types.TimestampType.withoutZone()), + Types.NestedField.required(3, "op_time_with_zone", Types.TimestampType.withZone()), + Types.NestedField.required(4, "d", Types.DecimalType.of(10, 0)), + Types.NestedField.required(5, "name", Types.StringType.get()) + ); + + protected static final PartitionSpec HIVE_SPEC = + PartitionSpec.builderFor(HIVE_TABLE_SCHEMA).identity("name").build(); + protected static final PrimaryKeySpec PRIMARY_KEY_SPEC = PrimaryKeySpec.builderFor(HIVE_TABLE_SCHEMA) + .addColumn("id").build(); + + protected static HMSMockServer hms; + protected static final TemporaryFolder tempFolder = new TemporaryFolder(); + protected static final AtomicInteger testCount = new AtomicInteger(0); + + protected UnkeyedHiveTable testHiveTable; + protected KeyedHiveTable testKeyedHiveTable; + + protected UnkeyedHiveTable testUnPartitionHiveTable; + protected KeyedHiveTable testUnPartitionKeyedHiveTable; + + protected List baseDataFilesInfo = new ArrayList<>(); + protected List posDeleteFilesInfo = new ArrayList<>(); + + @BeforeClass + public static void init() throws Exception { + FileUtils.deleteQuietly(testHiveBaseDir); + FileUtils.deleteQuietly(testHiveTableBaseDir); + testHiveBaseDir.mkdirs(); + setUpHMS(); + } + + @AfterClass + public static void clear() { + FileUtils.deleteQuietly(testHiveBaseDir); + FileUtils.deleteQuietly(testHiveTableBaseDir); + testHiveBaseDir.mkdirs(); + stopHMS(); + } + + @Before + public void initDataFileInfo() { + setupTables(); + baseDataFilesInfo = new ArrayList<>(); + posDeleteFilesInfo = new ArrayList<>(); + } + + @After + public void clearDataFileInfo() { + clearTable(); + baseDataFilesInfo.clear(); + posDeleteFilesInfo.clear(); + } + + public void setupTables() { + testHiveTable = (UnkeyedHiveTable) hiveCatalog + .newTableBuilder(HIVE_TABLE_ID, HIVE_TABLE_SCHEMA) + .withPartitionSpec(HIVE_SPEC) + .create().asUnkeyedTable(); + + testUnPartitionHiveTable = (UnkeyedHiveTable) hiveCatalog + .newTableBuilder(UN_PARTITION_HIVE_TABLE_ID, HIVE_TABLE_SCHEMA) + .create().asUnkeyedTable(); + + testKeyedHiveTable = (KeyedHiveTable) hiveCatalog + .newTableBuilder(HIVE_PK_TABLE_ID, HIVE_TABLE_SCHEMA) + .withPartitionSpec(HIVE_SPEC) + .withPrimaryKeySpec(PRIMARY_KEY_SPEC) + .create().asKeyedTable(); + + testUnPartitionKeyedHiveTable = (KeyedHiveTable) hiveCatalog + .newTableBuilder(UN_PARTITION_HIVE_PK_TABLE_ID, HIVE_TABLE_SCHEMA) + .withPrimaryKeySpec(PRIMARY_KEY_SPEC) + .create().asKeyedTable(); + } + + public void clearTable() { + hiveCatalog.dropTable(HIVE_TABLE_ID, true); + hiveCatalog.dropTable(UN_PARTITION_HIVE_TABLE_ID, true); + hiveCatalog.dropTable(HIVE_PK_TABLE_ID, true); + hiveCatalog.dropTable(UN_PARTITION_HIVE_PK_TABLE_ID, true); + } + + private static void setUpHMS() throws Exception { + int ref = testCount.incrementAndGet(); + if (ref == 1) { + tempFolder.create(); + hms = new HMSMockServer(tempFolder.newFolder("hive")); + hms.start(); + + Map storageConfig = new HashMap<>(); + storageConfig.put( + CatalogMetaProperties.STORAGE_CONFIGS_KEY_TYPE, + CatalogMetaProperties.STORAGE_CONFIGS_VALUE_TYPE_HDFS); + storageConfig.put(CatalogMetaProperties.STORAGE_CONFIGS_KEY_CORE_SITE, MockArcticMetastoreServer.getHadoopSite()); + storageConfig.put(CatalogMetaProperties.STORAGE_CONFIGS_KEY_HDFS_SITE, MockArcticMetastoreServer.getHadoopSite()); + storageConfig.put(CatalogMetaProperties.STORAGE_CONFIGS_KEY_HIVE_SITE, CatalogMetaTestUtil.encodingSite(hms.hiveConf())); + + + Map authConfig = new HashMap<>(); + authConfig.put(CatalogMetaProperties.AUTH_CONFIGS_KEY_TYPE, + CatalogMetaProperties.AUTH_CONFIGS_VALUE_TYPE_SIMPLE); + authConfig.put(CatalogMetaProperties.AUTH_CONFIGS_KEY_HADOOP_USERNAME, + System.getProperty("user.name")); + + Map hiveCatalogProperties = new HashMap<>(); + hiveCatalogProperties.put(CatalogMetaProperties.KEY_WAREHOUSE_DIR, "/hive_tmp"); + CatalogMeta hiveCatalogMeta = new CatalogMeta(AMS_TEST_HIVE_CATALOG_NAME, CATALOG_TYPE_HIVE, + storageConfig, authConfig, hiveCatalogProperties); + List catalogMetas = Lists.newArrayList(hiveCatalogMeta); + ServiceContainer.getCatalogMetadataService().addCatalog(catalogMetas); + hiveCatalog = CatalogLoader.load(AmsTestBase.amsHandler, AMS_TEST_HIVE_CATALOG_NAME); + hiveCatalog.createDatabase(AMS_TEST_HIVE_DB_NAME); + } + } + + private static void stopHMS() { + int ref = testCount.decrementAndGet(); + if (ref == 0){ + hms.stop(); + hms = null; + tempFolder.delete(); + } + } + + public List baseRecords(int start, int length, Schema tableSchema) { + GenericRecord record = GenericRecord.create(tableSchema); + + ImmutableList.Builder builder = ImmutableList.builder(); + for (int i = start; i < start + length; i++) { + builder.add(record.copy(ImmutableMap.of("id", i, + "op_time", LocalDateTime.of(2022, 1, 1, 12, 0, 0), + "op_time_with_zone", LocalDateTime.of(2022, 1, i % 2 + 1, 12, 0, 0).atOffset(ZoneOffset.UTC), + "d", new BigDecimal(i), "name", "name" + 1))); + } + + return builder.build(); + } +} diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestSupportHiveMajorOptimizeCommit.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestSupportHiveMajorOptimizeCommit.java new file mode 100644 index 0000000000..d3017a6419 --- /dev/null +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestSupportHiveMajorOptimizeCommit.java @@ -0,0 +1,396 @@ +/* + * 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 com.netease.arctic.ams.server.optimize; + +import com.netease.arctic.ams.api.OptimizeStatus; +import com.netease.arctic.ams.api.OptimizeType; +import com.netease.arctic.ams.api.TreeNode; +import com.netease.arctic.ams.server.model.BaseOptimizeTask; +import com.netease.arctic.ams.server.model.BaseOptimizeTaskRuntime; +import com.netease.arctic.ams.server.model.TableOptimizeRuntime; +import com.netease.arctic.data.DefaultKeyedFile; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.TableProperties; +import com.netease.arctic.utils.SerializationUtil; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class TestSupportHiveMajorOptimizeCommit extends TestSupportHiveBase { + @Test + public void testKeyedTableMajorOptimizeSupportHiveHasPosDeleteCommit() throws Exception { + insertBasePosDeleteFiles(testKeyedHiveTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + + Set oldDataFilesPath = new HashSet<>(); + Set oldDeleteFilesPath = new HashSet<>(); + testKeyedHiveTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> { + oldDataFilesPath.add((String) fileScanTask.file().path()); + fileScanTask.deletes().forEach(deleteFile -> oldDeleteFilesPath.add((String) deleteFile.path())); + }); + + TableOptimizeRuntime tableOptimizeRuntime = new TableOptimizeRuntime(testKeyedHiveTable.id()); + SupportHiveMajorOptimizePlan majorOptimizePlan = new SupportHiveMajorOptimizePlan(testKeyedHiveTable, + tableOptimizeRuntime, baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = majorOptimizePlan.plan(); + Assert.assertEquals(OptimizeType.Major, tasks.get(0).getTaskId().getType()); + + Map> resultFiles = generateTargetFiles(testKeyedHiveTable, tasks.get(0).getTaskId().getType()); + List taskItems = tasks.stream().map(task -> { + BaseOptimizeTaskRuntime optimizeRuntime = new BaseOptimizeTaskRuntime(task.getTaskId()); + List targetFiles = resultFiles.get(task.getSourceNodes().get(0)); + optimizeRuntime.setPreparedTime(System.currentTimeMillis()); + optimizeRuntime.setStatus(OptimizeStatus.Prepared); + optimizeRuntime.setReportTime(System.currentTimeMillis()); + optimizeRuntime.setNewFileCnt(targetFiles == null ? 0 : targetFiles.size()); + if (targetFiles != null) { + optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + } + // 1min + optimizeRuntime.setCostTime(60 * 1000); + return new OptimizeTaskItem(task, optimizeRuntime); + }).collect(Collectors.toList()); + Map> partitionTasks = taskItems.stream() + .collect(Collectors.groupingBy(taskItem -> taskItem.getOptimizeTask().getPartition())); + + SupportHiveCommit optimizeCommit = new SupportHiveCommit(testKeyedHiveTable, partitionTasks, taskItem -> {}); + optimizeCommit.commit(tableOptimizeRuntime); + + Set newDataFilesPath = new HashSet<>(); + Set newDeleteFilesPath = new HashSet<>(); + testKeyedHiveTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> { + newDataFilesPath.add((String) fileScanTask.file().path()); + fileScanTask.deletes().forEach(deleteFile -> newDeleteFilesPath.add((String) deleteFile.path())); + }); + for (String newFilePath : newDataFilesPath) { + Assert.assertFalse(newFilePath.contains(testKeyedHiveTable.hiveLocation())); + } + Assert.assertNotEquals(oldDataFilesPath, newDataFilesPath); + Assert.assertNotEquals(oldDeleteFilesPath, newDeleteFilesPath); + } + + @Test + public void testKeyedTableMajorOptimizeSupportHiveNoPosDeleteCommit() throws Exception { + insertTableBaseDataFiles(testKeyedHiveTable, 2, baseDataFilesInfo); + + Set oldDataFilesPath = new HashSet<>(); + testKeyedHiveTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> oldDataFilesPath.add((String) fileScanTask.file().path())); + + TableOptimizeRuntime tableOptimizeRuntime = new TableOptimizeRuntime(testKeyedHiveTable.id()); + SupportHiveMajorOptimizePlan majorOptimizePlan = new SupportHiveMajorOptimizePlan(testKeyedHiveTable, + tableOptimizeRuntime, baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = majorOptimizePlan.plan(); + Assert.assertEquals(OptimizeType.Major, tasks.get(0).getTaskId().getType()); + + Map> resultFiles = generateTargetFiles(testKeyedHiveTable, tasks.get(0).getTaskId().getType()); + List taskItems = tasks.stream().map(task -> { + BaseOptimizeTaskRuntime optimizeRuntime = new BaseOptimizeTaskRuntime(task.getTaskId()); + List targetFiles = resultFiles.get(task.getSourceNodes().get(0)); + optimizeRuntime.setPreparedTime(System.currentTimeMillis()); + optimizeRuntime.setStatus(OptimizeStatus.Prepared); + optimizeRuntime.setReportTime(System.currentTimeMillis()); + optimizeRuntime.setNewFileCnt(targetFiles == null ? 0 : targetFiles.size()); + if (targetFiles != null) { + optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + } + // 1min + optimizeRuntime.setCostTime(60 * 1000); + return new OptimizeTaskItem(task, optimizeRuntime); + }).collect(Collectors.toList()); + Map> partitionTasks = taskItems.stream() + .collect(Collectors.groupingBy(taskItem -> taskItem.getOptimizeTask().getPartition())); + + SupportHiveCommit optimizeCommit = new SupportHiveCommit(testKeyedHiveTable, partitionTasks, taskItem -> {}); + optimizeCommit.commit(tableOptimizeRuntime); + + Set newDataFilesPath = new HashSet<>(); + testKeyedHiveTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> newDataFilesPath.add((String) fileScanTask.file().path())); + for (String newFilePath : newDataFilesPath) { + Assert.assertTrue(newFilePath.contains(testKeyedHiveTable.hiveLocation())); + } + Assert.assertNotEquals(oldDataFilesPath, newDataFilesPath); + } + + @Test + public void testKeyedTableFullMajorOptimizeSupportHiveCommit() throws Exception { + testKeyedHiveTable.updateProperties() + .set(TableProperties.FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL, "86400000") + .commit(); + insertBasePosDeleteFiles(testKeyedHiveTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + + Set oldDataFilesPath = new HashSet<>(); + testKeyedHiveTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> oldDataFilesPath.add((String) fileScanTask.file().path())); + + TableOptimizeRuntime tableOptimizeRuntime = new TableOptimizeRuntime(testKeyedHiveTable.id()); + SupportHiveMajorOptimizePlan majorOptimizePlan = new SupportHiveMajorOptimizePlan(testKeyedHiveTable, + tableOptimizeRuntime, baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = majorOptimizePlan.plan(); + Assert.assertEquals(OptimizeType.FullMajor, tasks.get(0).getTaskId().getType()); + + Map> resultFiles = generateTargetFiles(testKeyedHiveTable, tasks.get(0).getTaskId().getType()); + List taskItems = tasks.stream().map(task -> { + BaseOptimizeTaskRuntime optimizeRuntime = new BaseOptimizeTaskRuntime(task.getTaskId()); + List targetFiles = resultFiles.get(task.getSourceNodes().get(0)); + optimizeRuntime.setPreparedTime(System.currentTimeMillis()); + optimizeRuntime.setStatus(OptimizeStatus.Prepared); + optimizeRuntime.setReportTime(System.currentTimeMillis()); + optimizeRuntime.setNewFileCnt(targetFiles == null ? 0 : targetFiles.size()); + if (targetFiles != null) { + optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + } + // 1min + optimizeRuntime.setCostTime(60 * 1000); + return new OptimizeTaskItem(task, optimizeRuntime); + }).collect(Collectors.toList()); + Map> partitionTasks = taskItems.stream() + .collect(Collectors.groupingBy(taskItem -> taskItem.getOptimizeTask().getPartition())); + + SupportHiveCommit optimizeCommit = new SupportHiveCommit(testKeyedHiveTable, partitionTasks, taskItem -> {}); + optimizeCommit.commit(tableOptimizeRuntime); + + Set newDataFilesPath = new HashSet<>(); + testKeyedHiveTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> newDataFilesPath.add((String) fileScanTask.file().path())); + for (String newFilePath : newDataFilesPath) { + Assert.assertTrue(newFilePath.contains(testKeyedHiveTable.hiveLocation())); + } + Assert.assertNotEquals(oldDataFilesPath, newDataFilesPath); + } + + @Test + public void testUnKeyedTableMajorOptimizeSupportHiveCommit() throws Exception { + insertTableBaseDataFiles(testHiveTable, 1, baseDataFilesInfo); + + Set oldDataFilesPath = new HashSet<>(); + testHiveTable.asUnkeyedTable().newScan().planFiles() + .forEach(fileScanTask -> oldDataFilesPath.add((String) fileScanTask.file().path())); + + TableOptimizeRuntime tableOptimizeRuntime = new TableOptimizeRuntime(testHiveTable.id()); + SupportHiveMajorOptimizePlan majorOptimizePlan = new SupportHiveMajorOptimizePlan(testHiveTable, + tableOptimizeRuntime, baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = majorOptimizePlan.plan(); + Assert.assertEquals(OptimizeType.Major, tasks.get(0).getTaskId().getType()); + + Map> resultFiles = generateTargetFiles(testHiveTable, tasks.get(0).getTaskId().getType()); + List taskItems = tasks.stream().map(task -> { + BaseOptimizeTaskRuntime optimizeRuntime = new BaseOptimizeTaskRuntime(task.getTaskId()); + ContentFile baseFile = SerializationUtil.toInternalTableFile(task.getBaseFiles().get(0)); + DefaultKeyedFile.FileMeta fileMeta = DefaultKeyedFile.parseMetaFromFileName(baseFile.path().toString()); + TreeNode treeNode = new TreeNode(fileMeta.node().getMask(), fileMeta.node().getIndex()); + List targetFiles = resultFiles.get(treeNode); + optimizeRuntime.setPreparedTime(System.currentTimeMillis()); + optimizeRuntime.setStatus(OptimizeStatus.Prepared); + optimizeRuntime.setReportTime(System.currentTimeMillis()); + optimizeRuntime.setNewFileCnt(targetFiles.size()); + optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + // 1min + optimizeRuntime.setCostTime(60 * 1000); + return new OptimizeTaskItem(task, optimizeRuntime); + }).collect(Collectors.toList()); + Map> partitionTasks = taskItems.stream() + .collect(Collectors.groupingBy(taskItem -> taskItem.getOptimizeTask().getPartition())); + + SupportHiveCommit optimizeCommit = new SupportHiveCommit(testHiveTable, partitionTasks, taskItem -> {}); + optimizeCommit.commit(tableOptimizeRuntime); + + Set newDataFilesPath = new HashSet<>(); + testHiveTable.asUnkeyedTable().newScan().planFiles() + .forEach(fileScanTask -> newDataFilesPath.add((String) fileScanTask.file().path())); + for (String newFilePath : newDataFilesPath) { + Assert.assertTrue(newFilePath.contains(testHiveTable.hiveLocation())); + } + Assert.assertNotEquals(oldDataFilesPath, newDataFilesPath); + } + + @Test + public void testUnKeyedTableFullMajorOptimizeSupportHiveCommit() throws Exception { + testHiveTable.updateProperties() + .set(TableProperties.FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL, "86400000") + .commit(); + insertTableBaseDataFiles(testHiveTable, 1, baseDataFilesInfo); + + Set oldDataFilesPath = new HashSet<>(); + testHiveTable.asUnkeyedTable().newScan().planFiles() + .forEach(fileScanTask -> oldDataFilesPath.add((String) fileScanTask.file().path())); + + TableOptimizeRuntime tableOptimizeRuntime = new TableOptimizeRuntime(testHiveTable.id()); + SupportHiveMajorOptimizePlan majorOptimizePlan = new SupportHiveMajorOptimizePlan(testHiveTable, + tableOptimizeRuntime, baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = majorOptimizePlan.plan(); + Assert.assertEquals(OptimizeType.FullMajor, tasks.get(0).getTaskId().getType()); + + Map> resultFiles = generateTargetFiles(testHiveTable, tasks.get(0).getTaskId().getType()); + List taskItems = tasks.stream().map(task -> { + BaseOptimizeTaskRuntime optimizeRuntime = new BaseOptimizeTaskRuntime(task.getTaskId()); + ContentFile baseFile = SerializationUtil.toInternalTableFile(task.getBaseFiles().get(0)); + DefaultKeyedFile.FileMeta fileMeta = DefaultKeyedFile.parseMetaFromFileName(baseFile.path().toString()); + TreeNode treeNode = new TreeNode(fileMeta.node().getMask(), fileMeta.node().getIndex()); + List targetFiles = resultFiles.get(treeNode); + optimizeRuntime.setPreparedTime(System.currentTimeMillis()); + optimizeRuntime.setStatus(OptimizeStatus.Prepared); + optimizeRuntime.setReportTime(System.currentTimeMillis()); + optimizeRuntime.setNewFileCnt(targetFiles.size()); + optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + // 1min + optimizeRuntime.setCostTime(60 * 1000); + return new OptimizeTaskItem(task, optimizeRuntime); + }).collect(Collectors.toList()); + Map> partitionTasks = taskItems.stream() + .collect(Collectors.groupingBy(taskItem -> taskItem.getOptimizeTask().getPartition())); + + SupportHiveCommit optimizeCommit = new SupportHiveCommit(testHiveTable, partitionTasks, taskItem -> {}); + optimizeCommit.commit(tableOptimizeRuntime); + + Set newDataFilesPath = new HashSet<>(); + testHiveTable.asUnkeyedTable().newScan().planFiles() + .forEach(fileScanTask -> newDataFilesPath.add((String) fileScanTask.file().path())); + for (String newFilePath : newDataFilesPath) { + Assert.assertTrue(newFilePath.contains(testHiveTable.hiveLocation())); + } + Assert.assertNotEquals(oldDataFilesPath, newDataFilesPath); + } + + + @Test + public void testUnPartitionTableMajorOptimizeSupportHiveCommit() throws Exception { + insertTableBaseDataFiles(testUnPartitionKeyedHiveTable, 2, baseDataFilesInfo); + + Set oldDataFilesPath = new HashSet<>(); + testUnPartitionKeyedHiveTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> oldDataFilesPath.add((String) fileScanTask.file().path())); + + TableOptimizeRuntime tableOptimizeRuntime = new TableOptimizeRuntime(testUnPartitionKeyedHiveTable.id()); + SupportHiveMajorOptimizePlan majorOptimizePlan = new SupportHiveMajorOptimizePlan(testUnPartitionKeyedHiveTable, + tableOptimizeRuntime, baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = majorOptimizePlan.plan(); + Assert.assertEquals(OptimizeType.Major, tasks.get(0).getTaskId().getType()); + + Map> resultFiles = generateTargetFiles(testUnPartitionKeyedHiveTable, tasks.get(0).getTaskId().getType()); + List taskItems = tasks.stream().map(task -> { + BaseOptimizeTaskRuntime optimizeRuntime = new BaseOptimizeTaskRuntime(task.getTaskId()); + List targetFiles = resultFiles.get(task.getSourceNodes().get(0)); + optimizeRuntime.setPreparedTime(System.currentTimeMillis()); + optimizeRuntime.setStatus(OptimizeStatus.Prepared); + optimizeRuntime.setReportTime(System.currentTimeMillis()); + optimizeRuntime.setNewFileCnt(targetFiles == null ? 0 : targetFiles.size()); + if (targetFiles != null) { + optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + } + // 1min + optimizeRuntime.setCostTime(60 * 1000); + return new OptimizeTaskItem(task, optimizeRuntime); + }).collect(Collectors.toList()); + Map> partitionTasks = taskItems.stream() + .collect(Collectors.groupingBy(taskItem -> taskItem.getOptimizeTask().getPartition())); + + SupportHiveCommit optimizeCommit = new SupportHiveCommit(testUnPartitionKeyedHiveTable, partitionTasks, taskItem -> {}); + optimizeCommit.commit(tableOptimizeRuntime); + + Set newDataFilesPath = new HashSet<>(); + testUnPartitionKeyedHiveTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> newDataFilesPath.add((String) fileScanTask.file().path())); + for (String newFilePath : newDataFilesPath) { + Assert.assertTrue(newFilePath.contains(testUnPartitionKeyedHiveTable.hiveLocation())); + } + Assert.assertNotEquals(oldDataFilesPath, newDataFilesPath); + } + + @Test + public void testUnPartitionTableFullMajorOptimizeSupportHiveCommit() throws Exception { + testUnPartitionKeyedHiveTable.updateProperties() + .set(TableProperties.FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL, "86400000") + .commit(); + insertBasePosDeleteFiles(testUnPartitionKeyedHiveTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + + Set oldDataFilesPath = new HashSet<>(); + testUnPartitionKeyedHiveTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> oldDataFilesPath.add((String) fileScanTask.file().path())); + + TableOptimizeRuntime tableOptimizeRuntime = new TableOptimizeRuntime(testUnPartitionKeyedHiveTable.id()); + SupportHiveMajorOptimizePlan majorOptimizePlan = new SupportHiveMajorOptimizePlan(testUnPartitionKeyedHiveTable, + tableOptimizeRuntime, baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = majorOptimizePlan.plan(); + Assert.assertEquals(OptimizeType.FullMajor, tasks.get(0).getTaskId().getType()); + + Map> resultFiles = generateTargetFiles(testUnPartitionKeyedHiveTable, tasks.get(0).getTaskId().getType()); + List taskItems = tasks.stream().map(task -> { + BaseOptimizeTaskRuntime optimizeRuntime = new BaseOptimizeTaskRuntime(task.getTaskId()); + List targetFiles = resultFiles.get(task.getSourceNodes().get(0)); + optimizeRuntime.setPreparedTime(System.currentTimeMillis()); + optimizeRuntime.setStatus(OptimizeStatus.Prepared); + optimizeRuntime.setReportTime(System.currentTimeMillis()); + optimizeRuntime.setNewFileCnt(targetFiles == null ? 0 : targetFiles.size()); + if (targetFiles != null) { + optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + } + // 1min + optimizeRuntime.setCostTime(60 * 1000); + return new OptimizeTaskItem(task, optimizeRuntime); + }).collect(Collectors.toList()); + Map> partitionTasks = taskItems.stream() + .collect(Collectors.groupingBy(taskItem -> taskItem.getOptimizeTask().getPartition())); + + SupportHiveCommit optimizeCommit = new SupportHiveCommit(testUnPartitionKeyedHiveTable, partitionTasks, taskItem -> {}); + optimizeCommit.commit(tableOptimizeRuntime); + + Set newDataFilesPath = new HashSet<>(); + testUnPartitionKeyedHiveTable.baseTable().newScan().planFiles() + .forEach(fileScanTask -> newDataFilesPath.add((String) fileScanTask.file().path())); + for (String newFilePath : newDataFilesPath) { + Assert.assertTrue(newFilePath.contains(testUnPartitionKeyedHiveTable.hiveLocation())); + } + Assert.assertNotEquals(oldDataFilesPath, newDataFilesPath); + } + + private Map> generateTargetFiles(ArcticTable arcticTable, + OptimizeType optimizeType) throws IOException { + List dataFiles = insertOptimizeTargetDataFiles(arcticTable, optimizeType,3); + return dataFiles.stream().collect(Collectors.groupingBy(dataFile -> { + DefaultKeyedFile keyedFile = new DefaultKeyedFile(dataFile); + return keyedFile.node().toAmsTreeNode(); + })); + } +} diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestSupportHiveMajorOptimizePlan.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestSupportHiveMajorOptimizePlan.java new file mode 100644 index 0000000000..4bfa52db1b --- /dev/null +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestSupportHiveMajorOptimizePlan.java @@ -0,0 +1,136 @@ +/* + * 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 com.netease.arctic.ams.server.optimize; + +import com.netease.arctic.ams.api.OptimizeType; +import com.netease.arctic.ams.server.model.BaseOptimizeTask; +import com.netease.arctic.ams.server.model.TableOptimizeRuntime; +import com.netease.arctic.table.TableProperties; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashMap; +import java.util.List; + +public class TestSupportHiveMajorOptimizePlan extends TestSupportHiveBase { + @Test + public void testKeyedTableMajorOptimizeSupportHive() throws IOException { + insertTableBaseDataFiles(testKeyedHiveTable, 1, baseDataFilesInfo); + + SupportHiveMajorOptimizePlan supportHiveMajorOptimizePlan = new SupportHiveMajorOptimizePlan(testKeyedHiveTable, + new TableOptimizeRuntime(testKeyedHiveTable.id()), baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = supportHiveMajorOptimizePlan.plan(); + Assert.assertEquals(4, tasks.size()); + Assert.assertEquals(OptimizeType.Major, tasks.get(0).getTaskId().getType()); + Assert.assertEquals(10, tasks.get(0).getBaseFiles().size()); + Assert.assertEquals(0, tasks.get(0).getPosDeleteFiles().size()); + Assert.assertEquals(0, tasks.get(0).getInsertFileCnt()); + Assert.assertEquals(0, tasks.get(0).getDeleteFileCnt()); + } + + @Test + public void testKeyedTableFullMajorOptimizeSupportHive() throws IOException { + testKeyedHiveTable.updateProperties() + .set(TableProperties.FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL, "86400000") + .commit(); + insertBasePosDeleteFiles(testKeyedHiveTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + + SupportHiveMajorOptimizePlan supportHiveMajorOptimizePlan = new SupportHiveMajorOptimizePlan(testKeyedHiveTable, + new TableOptimizeRuntime(testKeyedHiveTable.id()), baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = supportHiveMajorOptimizePlan.plan(); + Assert.assertEquals(4, tasks.size()); + Assert.assertEquals(OptimizeType.FullMajor, tasks.get(0).getTaskId().getType()); + Assert.assertEquals(10, tasks.get(0).getBaseFiles().size()); + Assert.assertEquals(1, tasks.get(0).getPosDeleteFiles().size()); + Assert.assertEquals(0, tasks.get(0).getInsertFileCnt()); + Assert.assertEquals(0, tasks.get(0).getDeleteFileCnt()); + } + + @Test + public void testUnKeyedTableMajorOptimizeSupportHive() throws IOException { + insertTableBaseDataFiles(testHiveTable, 1, baseDataFilesInfo); + + SupportHiveMajorOptimizePlan supportHiveMajorOptimizePlan = new SupportHiveMajorOptimizePlan(testHiveTable, + new TableOptimizeRuntime(testHiveTable.id()), baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = supportHiveMajorOptimizePlan.plan(); + Assert.assertEquals(1, tasks.size()); + Assert.assertEquals(OptimizeType.Major, tasks.get(0).getTaskId().getType()); + Assert.assertEquals(10, tasks.get(0).getBaseFiles().size()); + Assert.assertEquals(0, tasks.get(0).getPosDeleteFiles().size()); + Assert.assertEquals(0, tasks.get(0).getInsertFileCnt()); + Assert.assertEquals(0, tasks.get(0).getDeleteFileCnt()); + } + + @Test + public void testUnKeyedTableFullMajorOptimizeSupportHive() throws IOException { + testHiveTable.updateProperties() + .set(TableProperties.FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL, "86400000") + .commit(); + insertTableBaseDataFiles(testHiveTable, 1, baseDataFilesInfo); + + SupportHiveMajorOptimizePlan supportHiveMajorOptimizePlan = new SupportHiveMajorOptimizePlan(testHiveTable, + new TableOptimizeRuntime(testHiveTable.id()), baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = supportHiveMajorOptimizePlan.plan(); + Assert.assertEquals(1, tasks.size()); + Assert.assertEquals(OptimizeType.FullMajor, tasks.get(0).getTaskId().getType()); + Assert.assertEquals(10, tasks.get(0).getBaseFiles().size()); + Assert.assertEquals(0, tasks.get(0).getPosDeleteFiles().size()); + Assert.assertEquals(0, tasks.get(0).getInsertFileCnt()); + Assert.assertEquals(0, tasks.get(0).getDeleteFileCnt()); + } + + @Test + public void testNoPartitionTableMajorOptimizeSupportHive() throws IOException { + insertTableBaseDataFiles(testUnPartitionKeyedHiveTable, 1, baseDataFilesInfo); + SupportHiveMajorOptimizePlan supportHiveMajorOptimizePlan = new SupportHiveMajorOptimizePlan(testUnPartitionKeyedHiveTable, + new TableOptimizeRuntime(testUnPartitionKeyedHiveTable.id()), baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = supportHiveMajorOptimizePlan.plan(); + Assert.assertEquals(4, tasks.size()); + Assert.assertEquals(OptimizeType.Major, tasks.get(0).getTaskId().getType()); + Assert.assertEquals(10, tasks.get(0).getBaseFiles().size()); + Assert.assertEquals(0, tasks.get(0).getPosDeleteFiles().size()); + Assert.assertEquals(0, tasks.get(0).getInsertFileCnt()); + Assert.assertEquals(0, tasks.get(0).getDeleteFileCnt()); + } + + @Test + public void testNoPartitionTableFullMajorOptimizeSupportHive() throws IOException { + testUnPartitionKeyedHiveTable.updateProperties() + .set(TableProperties.FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL, "86400000") + .commit(); + insertTableBaseDataFiles(testUnPartitionKeyedHiveTable, 1, baseDataFilesInfo); + + SupportHiveMajorOptimizePlan supportHiveMajorOptimizePlan = new SupportHiveMajorOptimizePlan(testUnPartitionKeyedHiveTable, + new TableOptimizeRuntime(testUnPartitionKeyedHiveTable.id()), baseDataFilesInfo, posDeleteFilesInfo, + new HashMap<>(), 1, System.currentTimeMillis(), snapshotId -> true); + List tasks = supportHiveMajorOptimizePlan.plan(); + Assert.assertEquals(4, tasks.size()); + Assert.assertEquals(OptimizeType.FullMajor, tasks.get(0).getTaskId().getType()); + Assert.assertEquals(10, tasks.get(0).getBaseFiles().size()); + Assert.assertEquals(0, tasks.get(0).getPosDeleteFiles().size()); + Assert.assertEquals(0, tasks.get(0).getInsertFileCnt()); + Assert.assertEquals(0, tasks.get(0).getDeleteFileCnt()); + } +} diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/service/TestSupportHiveSyncService.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/service/TestSupportHiveSyncService.java new file mode 100644 index 0000000000..405276e615 --- /dev/null +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/service/TestSupportHiveSyncService.java @@ -0,0 +1,304 @@ +/* + * 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 com.netease.arctic.ams.server.service; + +import com.netease.arctic.ams.server.optimize.TestSupportHiveBase; +import com.netease.arctic.ams.server.service.impl.SupportHiveSyncService; +import com.netease.arctic.hive.HiveTableProperties; +import com.netease.arctic.hive.io.writer.AdaptHiveGenericTaskWriterBuilder; +import com.netease.arctic.hive.table.HiveLocationKind; +import com.netease.arctic.hive.table.SupportHive; +import com.netease.arctic.hive.utils.HivePartitionUtil; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.utils.FileUtil; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.util.StructLikeMap; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static com.netease.arctic.utils.TablePropertyUtil.EMPTY_STRUCT; + +public class TestSupportHiveSyncService extends TestSupportHiveBase { + @Test + public void testUnPartitionTableSyncInIceberg() throws Exception { + StructLikeMap> partitionProperty = testUnPartitionKeyedHiveTable.baseTable().partitionProperty(); + Assert.assertEquals(0, partitionProperty.size()); + String newLocation = createEmptyLocationForHive(testUnPartitionKeyedHiveTable); + testUnPartitionKeyedHiveTable.baseTable().updatePartitionProperties(null) + .set(EMPTY_STRUCT, HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION, newLocation).commit(); + String hiveLocation = ((SupportHive) testUnPartitionKeyedHiveTable).getHMSClient().run(client -> { + Table hiveTable = client.getTable(testUnPartitionKeyedHiveTable.id().getDatabase(), + testUnPartitionKeyedHiveTable.id().getTableName()); + return hiveTable.getSd().getLocation(); + }); + Assert.assertNotEquals(newLocation, hiveLocation); + + SupportHiveSyncService.SupportHiveSyncTask.syncIcebergToHive(testUnPartitionKeyedHiveTable, "UnitTest"); + hiveLocation = ((SupportHive) testUnPartitionKeyedHiveTable).getHMSClient().run(client -> { + Table hiveTable = client.getTable(testUnPartitionKeyedHiveTable.id().getDatabase(), + testUnPartitionKeyedHiveTable.id().getTableName()); + return hiveTable.getSd().getLocation(); + }); + Assert.assertEquals(newLocation, hiveLocation); + } + + @Test + public void testUnPartitionTableSyncNotInIceberg() throws Exception { + StructLikeMap> partitionProperty = testUnPartitionKeyedHiveTable.baseTable().partitionProperty(); + Assert.assertEquals(0, partitionProperty.size()); + + String oldHiveLocation = ((SupportHive) testUnPartitionKeyedHiveTable).getHMSClient().run(client -> { + Table hiveTable = client.getTable(testUnPartitionKeyedHiveTable.id().getDatabase(), + testUnPartitionKeyedHiveTable.id().getTableName()); + return hiveTable.getSd().getLocation(); + }); + + SupportHiveSyncService.SupportHiveSyncTask.syncIcebergToHive(testUnPartitionKeyedHiveTable, "UnitTest"); + String newHiveLocation = ((SupportHive) testUnPartitionKeyedHiveTable).getHMSClient().run(client -> { + Table hiveTable = client.getTable(testUnPartitionKeyedHiveTable.id().getDatabase(), + testUnPartitionKeyedHiveTable.id().getTableName()); + return hiveTable.getSd().getLocation(); + }); + Assert.assertEquals(oldHiveLocation, newHiveLocation); + } + + @Test + public void testSyncOnlyInIceberg() throws Exception { + StructLikeMap> partitionProperty = testKeyedHiveTable.baseTable().partitionProperty(); + Assert.assertEquals(0, partitionProperty.size()); + List dataFiles = insertTableHiveDataFiles(testKeyedHiveTable, 1); + String partitionLocation = FileUtil.getFileDir(dataFiles.get(0).path().toString()); + testKeyedHiveTable.baseTable().updatePartitionProperties(null) + .set(dataFiles.get(0).partition(), HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION, partitionLocation) + .commit(); + + List partitionValues = + HivePartitionUtil.partitionValuesAsList(dataFiles.get(0).partition(), testKeyedHiveTable.spec().partitionType()); + Assert.assertThrows(NoSuchObjectException.class, () -> ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + client.getPartition(testKeyedHiveTable.id().getDatabase(), + testKeyedHiveTable.id().getTableName(), partitionValues))); + + SupportHiveSyncService.SupportHiveSyncTask.syncIcebergToHive(testKeyedHiveTable, "UnitTest"); + Partition hivePartition = ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + client.getPartition(testKeyedHiveTable.id().getDatabase(), + testKeyedHiveTable.id().getTableName(), partitionValues)); + Assert.assertEquals(partitionLocation, hivePartition.getSd().getLocation()); + } + + @Test + public void testSyncOnlyInHiveCreateByArctic() throws Exception { + StructLikeMap> partitionProperty = testKeyedHiveTable.baseTable().partitionProperty(); + Assert.assertEquals(0, partitionProperty.size()); + + List dataFiles = insertTableHiveDataFiles(testKeyedHiveTable, 1); + String partitionLocation = FileUtil.getFileDir(dataFiles.get(0).path().toString()); + List partitionValues = + HivePartitionUtil.partitionValuesAsList(dataFiles.get(0).partition(), testKeyedHiveTable.spec().partitionType()); + ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + { + Table hiveTable = client.getTable(testKeyedHiveTable.id().getDatabase(), testKeyedHiveTable.id().getTableName()); + StorageDescriptor tableSd = hiveTable.getSd(); + PrincipalPrivilegeSet privilegeSet = hiveTable.getPrivileges(); + int lastAccessTime = (int) (System.currentTimeMillis() / 1000); + Partition p = new Partition(); + p.setValues(partitionValues); + p.setDbName(hiveTable.getDbName()); + p.setTableName(hiveTable.getTableName()); + p.setCreateTime(lastAccessTime); + p.setLastAccessTime(lastAccessTime); + StorageDescriptor sd = tableSd.deepCopy(); + sd.setLocation(partitionLocation); + p.setSd(sd); + + int files = dataFiles.size(); + long totalSize = dataFiles.stream().map(ContentFile::fileSizeInBytes).reduce(0L, Long::sum); + p.putToParameters("transient_lastDdlTime", lastAccessTime + ""); + p.putToParameters("totalSize", totalSize + ""); + p.putToParameters("numFiles", files + ""); + p.putToParameters(HiveTableProperties.ARCTIC_TABLE_FLAG, "true"); + if (privilegeSet != null) { + p.setPrivileges(privilegeSet.deepCopy()); + } + + return client.add_partition(p); + }); + + Partition hivePartition = ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + client.getPartition(testKeyedHiveTable.id().getDatabase(), + testKeyedHiveTable.id().getTableName(), partitionValues)); + Assert.assertEquals(partitionLocation, hivePartition.getSd().getLocation()); + + SupportHiveSyncService.SupportHiveSyncTask.syncIcebergToHive(testKeyedHiveTable, "UnitTest"); + + Assert.assertThrows(NoSuchObjectException.class, () -> ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + client.getPartition(testKeyedHiveTable.id().getDatabase(), + testKeyedHiveTable.id().getTableName(), partitionValues))); + } + + @Test + public void testSyncOnlyInHiveCreateNotByArctic() throws Exception { + StructLikeMap> partitionProperty = testKeyedHiveTable.baseTable().partitionProperty(); + Assert.assertEquals(0, partitionProperty.size()); + + List dataFiles = insertTableHiveDataFiles(testKeyedHiveTable, 1); + String partitionLocation = FileUtil.getFileDir(dataFiles.get(0).path().toString()); + List partitionValues = + HivePartitionUtil.partitionValuesAsList(dataFiles.get(0).partition(), testKeyedHiveTable.spec().partitionType()); + ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + { + Table hiveTable = client.getTable(testKeyedHiveTable.id().getDatabase(), testKeyedHiveTable.id().getTableName()); + StorageDescriptor tableSd = hiveTable.getSd(); + PrincipalPrivilegeSet privilegeSet = hiveTable.getPrivileges(); + int lastAccessTime = (int) (System.currentTimeMillis() / 1000); + Partition p = new Partition(); + p.setValues(partitionValues); + p.setDbName(hiveTable.getDbName()); + p.setTableName(hiveTable.getTableName()); + p.setCreateTime(lastAccessTime); + p.setLastAccessTime(lastAccessTime); + StorageDescriptor sd = tableSd.deepCopy(); + sd.setLocation(partitionLocation); + p.setSd(sd); + + int files = dataFiles.size(); + long totalSize = dataFiles.stream().map(ContentFile::fileSizeInBytes).reduce(0L, Long::sum); + p.putToParameters("transient_lastDdlTime", lastAccessTime + ""); + p.putToParameters("totalSize", totalSize + ""); + p.putToParameters("numFiles", files + ""); + if (privilegeSet != null) { + p.setPrivileges(privilegeSet.deepCopy()); + } + + return client.add_partition(p); + }); + + Partition hivePartition = ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + client.getPartition(testKeyedHiveTable.id().getDatabase(), + testKeyedHiveTable.id().getTableName(), partitionValues)); + Assert.assertEquals(partitionLocation, hivePartition.getSd().getLocation()); + + SupportHiveSyncService.SupportHiveSyncTask.syncIcebergToHive(testKeyedHiveTable, "UnitTest"); + + hivePartition = ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + client.getPartition(testKeyedHiveTable.id().getDatabase(), + testKeyedHiveTable.id().getTableName(), partitionValues)); + Assert.assertEquals(partitionLocation, hivePartition.getSd().getLocation()); + } + + @Test + public void testSyncInBoth() throws Exception { + StructLikeMap> partitionProperty = testKeyedHiveTable.baseTable().partitionProperty(); + Assert.assertEquals(0, partitionProperty.size()); + + List dataFiles = insertTableHiveDataFiles(testKeyedHiveTable, 1); + String partitionLocation = FileUtil.getFileDir(dataFiles.get(0).path().toString()); + List partitionValues = + HivePartitionUtil.partitionValuesAsList(dataFiles.get(0).partition(), testKeyedHiveTable.spec().partitionType()); + ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + { + Table hiveTable = client.getTable(testKeyedHiveTable.id().getDatabase(), testKeyedHiveTable.id().getTableName()); + StorageDescriptor tableSd = hiveTable.getSd(); + PrincipalPrivilegeSet privilegeSet = hiveTable.getPrivileges(); + int lastAccessTime = (int) (System.currentTimeMillis() / 1000); + Partition p = new Partition(); + p.setValues(partitionValues); + p.setDbName(hiveTable.getDbName()); + p.setTableName(hiveTable.getTableName()); + p.setCreateTime(lastAccessTime); + p.setLastAccessTime(lastAccessTime); + StorageDescriptor sd = tableSd.deepCopy(); + sd.setLocation(partitionLocation); + p.setSd(sd); + + int files = dataFiles.size(); + long totalSize = dataFiles.stream().map(ContentFile::fileSizeInBytes).reduce(0L, Long::sum); + p.putToParameters("transient_lastDdlTime", lastAccessTime + ""); + p.putToParameters("totalSize", totalSize + ""); + p.putToParameters("numFiles", files + ""); + if (privilegeSet != null) { + p.setPrivileges(privilegeSet.deepCopy()); + } + + return client.add_partition(p); + }); + + Partition hivePartition = ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + client.getPartition(testKeyedHiveTable.id().getDatabase(), + testKeyedHiveTable.id().getTableName(), partitionValues)); + Assert.assertEquals(partitionLocation, hivePartition.getSd().getLocation()); + + List newDataFiles = insertTableHiveDataFiles(testKeyedHiveTable, 2); + String newPartitionLocation = FileUtil.getFileDir(newDataFiles.get(0).path().toString()); + testKeyedHiveTable.baseTable().updatePartitionProperties(null) + .set(newDataFiles.get(0).partition(), HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION, newPartitionLocation) + .commit(); + Assert.assertNotEquals(newPartitionLocation, hivePartition.getSd().getLocation()); + + SupportHiveSyncService.SupportHiveSyncTask.syncIcebergToHive(testKeyedHiveTable, "UnitTest"); + + hivePartition = ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> + client.getPartition(testKeyedHiveTable.id().getDatabase(), + testKeyedHiveTable.id().getTableName(), partitionValues)); + Assert.assertEquals(newPartitionLocation, hivePartition.getSd().getLocation()); + } + + private String createEmptyLocationForHive(ArcticTable arcticTable) { + // create a new empty location for hive + String newLocation = ((SupportHive) arcticTable).hiveLocation() + "/ts_" + System.currentTimeMillis(); + OutputFile file = arcticTable.io().newOutputFile(newLocation + "/.keep"); + try { + file.createOrOverwrite().close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return newLocation; + } + + private List insertTableHiveDataFiles(ArcticTable arcticTable, long transactionId) throws IOException { + TaskWriter writer = arcticTable.isKeyedTable() ? + AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .withTransactionId(transactionId) + .buildWriter(HiveLocationKind.INSTANT) : + AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .buildWriter(HiveLocationKind.INSTANT); + + List baseDataFiles = insertBaseDataFiles(writer, arcticTable.schema()); + UnkeyedTable baseTable = arcticTable.isKeyedTable() ? + arcticTable.asKeyedTable().baseTable() : arcticTable.asUnkeyedTable(); + AppendFiles baseAppend = baseTable.newAppend(); + baseDataFiles.forEach(baseAppend::appendFile); + baseAppend.commit(); + return baseDataFiles; + } +} diff --git a/core/src/main/java/com/netease/arctic/io/ArcticHadoopFileIO.java b/core/src/main/java/com/netease/arctic/io/ArcticHadoopFileIO.java index 5532fe5b01..464a2d7b3b 100644 --- a/core/src/main/java/com/netease/arctic/io/ArcticHadoopFileIO.java +++ b/core/src/main/java/com/netease/arctic/io/ArcticHadoopFileIO.java @@ -31,8 +31,6 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.io.UncheckedIOException; @@ -160,6 +158,20 @@ public boolean isEmptyDirectory(String location) { }); } + @Override + public boolean rename(String src, String dts) { + return tableMetaStore.doAs(() -> { + Path srcPath = new Path(src); + Path dtsPath = new Path(dts); + FileSystem fs = getFs(srcPath); + try { + return fs.rename(srcPath, dtsPath); + } catch (IOException e) { + throw new UncheckedIOException("Failed to rename: from " + src + " to " + dts, e); + } + }); + } + @Override public T doAs(Callable callable) { return tableMetaStore.doAs(callable); @@ -191,20 +203,6 @@ public boolean mkdirs(String path) { }); } - @Override - public boolean rename(String oldPath, String newPath) { - return tableMetaStore.doAs(() -> { - Path srcPath = new Path(oldPath); - Path dtsPath = new Path(newPath); - FileSystem fs = getFs(srcPath); - try { - return fs.rename(srcPath, dtsPath); - } catch (IOException e) { - throw new UncheckedIOException("Failed to rename: from " + oldPath + " to " + newPath, e); - } - }); - } - public TableMetaStore getTableMetaStore() { return tableMetaStore; } diff --git a/core/src/main/java/com/netease/arctic/table/TableProperties.java b/core/src/main/java/com/netease/arctic/table/TableProperties.java index a179b465b6..979deee139 100644 --- a/core/src/main/java/com/netease/arctic/table/TableProperties.java +++ b/core/src/main/java/com/netease/arctic/table/TableProperties.java @@ -73,6 +73,9 @@ private TableProperties() { public static final String MAJOR_OPTIMIZE_TRIGGER_MAX_INTERVAL = "optimize.major.trigger.max-interval"; public static final long MAJOR_OPTIMIZE_TRIGGER_MAX_INTERVAL_DEFAULT = 86_400_000; // 1 day + public static final String FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL = "optimize.full.trigger.max-interval"; + public static final long FULL_OPTIMIZE_TRIGGER_MAX_INTERVAL_DEFAULT = -1; // default not trigger + public static final String MINOR_OPTIMIZE_TRIGGER_MAX_INTERVAL = "optimize.minor.trigger.max-interval"; public static final long MINOR_OPTIMIZE_TRIGGER_MAX_INTERVAL_DEFAULT = 3600_000; // 1h diff --git a/core/src/main/java/com/netease/arctic/utils/FileUtil.java b/core/src/main/java/com/netease/arctic/utils/FileUtil.java index 91911196ba..d4295d3525 100644 --- a/core/src/main/java/com/netease/arctic/utils/FileUtil.java +++ b/core/src/main/java/com/netease/arctic/utils/FileUtil.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.Path; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import java.io.File; import java.util.Collections; import java.util.Set; @@ -49,6 +50,12 @@ public static String getFileDir(String filePath) { return filePath.substring(0, lastSlash); } + public static String getPartitionPathFromFilePath(String fileLocation, String tableLocation, String fileName) { + int tableIndex = fileLocation.indexOf(tableLocation); + int fileIndex = fileLocation.lastIndexOf(fileName); + return fileLocation.substring(tableIndex + tableLocation.length(), fileIndex - 1); + } + public static void deleteEmptyDirectory(ArcticFileIO io, String directoryPath) { deleteEmptyDirectory(io, directoryPath, Collections.emptySet()); } @@ -73,4 +80,14 @@ public static void deleteEmptyDirectory(ArcticFileIO io, String directoryPath, S deleteEmptyDirectory(io, parent, exclude); } } + + /** + * Get the file path after move file to target directory + * @param newDirectory target directory + * @param filePath file + * @return new file path + */ + public static String getNewFilePath(String newDirectory, String filePath) { + return newDirectory + File.separator + getFileName(filePath); + } } diff --git a/core/src/test/java/com/netease/arctic/TableTestBase.java b/core/src/test/java/com/netease/arctic/TableTestBase.java index 8a2a10c098..d193645ef2 100644 --- a/core/src/test/java/com/netease/arctic/TableTestBase.java +++ b/core/src/test/java/com/netease/arctic/TableTestBase.java @@ -73,6 +73,8 @@ public class TableTestBase { TableIdentifier.of(TEST_CATALOG_NAME, TEST_DB_NAME, "test_table"); protected static final TableIdentifier PK_TABLE_ID = TableIdentifier.of(TEST_CATALOG_NAME, TEST_DB_NAME, "test_pk_table"); + protected static final TableIdentifier NO_PARTITION_TABLE_ID = + TableIdentifier.of(TEST_CATALOG_NAME, TEST_DB_NAME, "test_no_partition_table"); public static final Schema TABLE_SCHEMA = new Schema( Types.NestedField.required(1, "id", Types.IntegerType.get()), Types.NestedField.required(2, "name", Types.StringType.get()), @@ -109,6 +111,7 @@ public class TableTestBase { protected ArcticCatalog testCatalog; protected UnkeyedTable testTable; protected KeyedTable testKeyedTable; + protected KeyedTable testNoPartitionTable; @Rule public TemporaryFolder temp = new TemporaryFolder(); @@ -138,6 +141,12 @@ public void setupTables() throws Exception { .withPrimaryKeySpec(PRIMARY_KEY_SPEC) .create().asKeyedTable(); + testNoPartitionTable = testCatalog + .newTableBuilder(NO_PARTITION_TABLE_ID, TABLE_SCHEMA) + .withProperty(TableProperties.LOCATION, tableDir.getPath() + "/no_partition_table") + .withPrimaryKeySpec(PRIMARY_KEY_SPEC) + .create().asKeyedTable(); + this.before(); } @@ -152,6 +161,9 @@ public void clearTable() { testCatalog.dropTable(PK_TABLE_ID, true); AMS.handler().getTableCommitMetas().remove(PK_TABLE_ID.buildTableIdentifier()); + + testCatalog.dropTable(NO_PARTITION_TABLE_ID, true); + AMS.handler().getTableCommitMetas().remove(NO_PARTITION_TABLE_ID.buildTableIdentifier()); } public List writeBase(TableIdentifier identifier, List records) { @@ -214,7 +226,7 @@ public static List readKeyedTable(KeyedTable keyedTable) { List result = Lists.newArrayList(); try (CloseableIterable combinedScanTasks = keyedTable.newScan().planTasks()) { combinedScanTasks.forEach(combinedTask -> combinedTask.tasks().forEach(scTask -> { - try (CloseableIterator records = reader.readData(scTask);) { + try (CloseableIterator records = reader.readData(scTask)) { while (records.hasNext()) { result.add(records.next()); } diff --git a/flink/v1.12/flink/pom.xml b/flink/v1.12/flink/pom.xml index 4fc3a163c4..5b3b36c7a3 100644 --- a/flink/v1.12/flink/pom.xml +++ b/flink/v1.12/flink/pom.xml @@ -49,11 +49,6 @@ - - com.netease.arctic - arctic-hive - ${project.version} - org.apache.iceberg diff --git a/hive/src/main/java/com/netease/arctic/hive/table/KeyedHiveTable.java b/hive/src/main/java/com/netease/arctic/hive/table/KeyedHiveTable.java index 80d8bf5d7c..61ffc0c022 100644 --- a/hive/src/main/java/com/netease/arctic/hive/table/KeyedHiveTable.java +++ b/hive/src/main/java/com/netease/arctic/hive/table/KeyedHiveTable.java @@ -79,4 +79,9 @@ private void syncHiveDataToArctic() { HiveMetaSynchronizer.syncHiveDataToArctic(this, hiveClient); } } + + @Override + public HMSClient getHMSClient() { + return hiveClient; + } } diff --git a/hive/src/main/java/com/netease/arctic/hive/table/SupportHive.java b/hive/src/main/java/com/netease/arctic/hive/table/SupportHive.java index 14c8f21aa5..c204696407 100644 --- a/hive/src/main/java/com/netease/arctic/hive/table/SupportHive.java +++ b/hive/src/main/java/com/netease/arctic/hive/table/SupportHive.java @@ -18,6 +18,8 @@ package com.netease.arctic.hive.table; +import com.netease.arctic.hive.HMSClient; + /** * Mix-in interface to mark task use hive as base store */ @@ -28,4 +30,10 @@ public interface SupportHive { * @return path to store hive file */ String hiveLocation(); + + /** + * the client to operate hive table + * @return hive metastore client + */ + HMSClient getHMSClient(); } diff --git a/hive/src/main/java/com/netease/arctic/hive/table/UnkeyedHiveTable.java b/hive/src/main/java/com/netease/arctic/hive/table/UnkeyedHiveTable.java index e79c18a92d..17239f00d7 100644 --- a/hive/src/main/java/com/netease/arctic/hive/table/UnkeyedHiveTable.java +++ b/hive/src/main/java/com/netease/arctic/hive/table/UnkeyedHiveTable.java @@ -106,6 +106,11 @@ public String hiveLocation() { HiveTableUtil.hiveRootLocation(tableLocation); } + @Override + public HMSClient getHMSClient() { + return hiveClient; + } + @Override public OverwriteHiveFiles newOverwrite() { return new OverwriteHiveFiles(super.newTransaction(), false, this, hiveClient, hiveClient); diff --git a/hive/src/main/java/com/netease/arctic/hive/utils/HivePartitionUtil.java b/hive/src/main/java/com/netease/arctic/hive/utils/HivePartitionUtil.java index e1dffc87aa..951d2913a5 100644 --- a/hive/src/main/java/com/netease/arctic/hive/utils/HivePartitionUtil.java +++ b/hive/src/main/java/com/netease/arctic/hive/utils/HivePartitionUtil.java @@ -19,8 +19,10 @@ package com.netease.arctic.hive.utils; import com.netease.arctic.hive.HMSClient; +import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.TableIdentifier; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.PartitionDropOptions; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; @@ -89,7 +91,7 @@ public static Partition newPartition( p.setSd(sd); HiveTableUtil.generateTableProperties(createTimeInSeconds, dataFiles) - .forEach((key, value) -> p.putToParameters(key, value)); + .forEach(p::putToParameters); if (privilegeSet != null) { p.setPrivileges(privilegeSet.deepCopy()); @@ -97,8 +99,25 @@ public static Partition newPartition( return p; } + public static Partition getPartition(HMSClient hmsClient, + ArcticTable arcticTable, + List partitionValues) { + String db = arcticTable.id().getDatabase(); + String tableName = arcticTable.id().getTableName(); + + try { + return hmsClient.run(client -> { + Partition partition; + partition = client.getPartition(db, tableName, partitionValues); + return partition; + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + public static void rewriteHivePartitions(Partition partition, String location, List dataFiles, - int accessTimestamp) { + int accessTimestamp) { partition.getSd().setLocation(location); partition.setLastAccessTime(accessTimestamp); HiveTableUtil.generateTableProperties(accessTimestamp, dataFiles) @@ -203,4 +222,60 @@ public static void alterPartition(HMSClient hiveClient, TableIdentifier tableIde throw new IOException(e); } } + + public static void createPartitionIfAbsent(HMSClient hmsClient, + ArcticTable arcticTable, + List partitionValues, + String partitionLocation, + List dataFiles, + int accessTimestamp) { + String db = arcticTable.id().getDatabase(); + String tableName = arcticTable.id().getTableName(); + + try { + hmsClient.run(client -> { + Partition partition; + try { + partition = client.getPartition(db, tableName, partitionValues); + return partition; + } catch (NoSuchObjectException noSuchObjectException) { + Table hiveTable = client.getTable(db, tableName); + partition = newPartition(hiveTable, partitionValues, partitionLocation, + dataFiles, accessTimestamp); + client.add_partition(partition); + return partition; + } + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public static void dropPartition(HMSClient hmsClient, + ArcticTable arcticTable, + Partition hivePartition) { + try { + hmsClient.run(client -> { + PartitionDropOptions options = PartitionDropOptions.instance() + .deleteData(false) + .ifExists(true) + .purgeData(false) + .returnResults(false); + return client.dropPartition(arcticTable.id().getDatabase(), + arcticTable.id().getTableName(), hivePartition.getValues(), options); + }); + } catch (TException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + public static void updatePartitionLocation(HMSClient hmsClient, + ArcticTable arcticTable, + Partition hivePartition, + String newLocation, + List dataFiles, + int accessTimestamp) { + dropPartition(hmsClient, arcticTable, hivePartition); + createPartitionIfAbsent(hmsClient, arcticTable, hivePartition.getValues(), newLocation, dataFiles, accessTimestamp); + } } diff --git a/hive/src/test/java/com/netease/arctic/hive/HiveTableTestBase.java b/hive/src/test/java/com/netease/arctic/hive/HiveTableTestBase.java index e98ece1539..ba9730eca6 100644 --- a/hive/src/test/java/com/netease/arctic/hive/HiveTableTestBase.java +++ b/hive/src/test/java/com/netease/arctic/hive/HiveTableTestBase.java @@ -47,6 +47,9 @@ import org.junit.Before; import org.junit.BeforeClass; import org.junit.rules.TemporaryFolder; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.HashMap; import java.util.List; @@ -56,6 +59,8 @@ import static com.netease.arctic.ams.api.properties.CatalogMetaProperties.CATALOG_TYPE_HIVE; public class HiveTableTestBase extends TableTestBase { + public static final Logger LOG = LoggerFactory.getLogger(HiveTableTestBase.class); + protected static final String HIVE_DB_NAME = "hivedb"; protected static final String HIVE_CATALOG_NAME = "hive_catalog"; protected static final AtomicInteger testCount = new AtomicInteger(0); @@ -78,7 +83,7 @@ public class HiveTableTestBase extends TableTestBase { Types.NestedField.required(1, "id", Types.IntegerType.get()), Types.NestedField.required(2, "op_time", Types.TimestampType.withoutZone()), Types.NestedField.required(3, "op_time_with_zone", Types.TimestampType.withZone()), - Types.NestedField.required(4, "d", Types.DecimalType.of(3, 0)), + Types.NestedField.required(4, "d", Types.DecimalType.of(10, 0)), Types.NestedField.required(5, "name", Types.StringType.get()) ); diff --git a/optimizer/pom.xml b/optimizer/pom.xml index adf850d0df..fe8a5789fc 100644 --- a/optimizer/pom.xml +++ b/optimizer/pom.xml @@ -39,6 +39,13 @@ com.netease.arctic arctic-core + + + com.netease.arctic + arctic-hive + ${project.version} + + org.slf4j slf4j-log4j12 @@ -229,6 +236,14 @@ + + com.netease.arctic + arctic-hive + ${project.version} + tests + test + + com.netease.arctic arctic-ams-api diff --git a/optimizer/src/main/java/com/netease/arctic/optimizer/operator/BaseTaskExecutor.java b/optimizer/src/main/java/com/netease/arctic/optimizer/operator/BaseTaskExecutor.java index 8a88a6673c..a16ef969e5 100644 --- a/optimizer/src/main/java/com/netease/arctic/optimizer/operator/BaseTaskExecutor.java +++ b/optimizer/src/main/java/com/netease/arctic/optimizer/operator/BaseTaskExecutor.java @@ -112,7 +112,7 @@ public OptimizeTaskStat execute(TaskWrapper sourceTask) { try { String amsUrl = config.getAmsUrl(); table = getArcticTable(new TableIdentificationInfo(amsUrl, task.getTableIdentifier())); - setPartition(table, task); + setPartition(task); } catch (Exception e) { LOG.error("failed to set partition info {}", task.getTaskId(), e); onTaskFailed(e); @@ -181,7 +181,7 @@ private static ArcticTable buildTable(TableIdentificationInfo tableIdentifierInf return arcticCatalog.loadTable(tableIdentifierInfo.getTableIdentifier()); } - private void setPartition(ArcticTable arcticTable, NodeTask nodeTask) { + private void setPartition(NodeTask nodeTask) { // partition if (nodeTask.files().size() == 0) { LOG.warn("task: {} no files to optimize.", nodeTask.getTaskId()); diff --git a/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/ExecutorFactory.java b/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/ExecutorFactory.java index 756c6048d4..ace41cebda 100644 --- a/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/ExecutorFactory.java +++ b/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/ExecutorFactory.java @@ -32,6 +32,7 @@ public static Executor constructOptimize(NodeTask nodeTask, ArcticTable table case Minor: return new MinorExecutor(nodeTask, table, startTime, config); case Major: + case FullMajor: return new MajorExecutor(nodeTask, table, startTime, config); default: LOG.error("not support optimize type: {}", nodeTask.getOptimizeType()); diff --git a/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/MajorExecutor.java b/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/MajorExecutor.java index 31932f4371..90a11bdf87 100644 --- a/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/MajorExecutor.java +++ b/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/MajorExecutor.java @@ -22,12 +22,12 @@ import com.netease.arctic.ams.api.JobType; import com.netease.arctic.ams.api.OptimizeStatus; import com.netease.arctic.ams.api.OptimizeTaskStat; +import com.netease.arctic.ams.api.OptimizeType; import com.netease.arctic.data.DataFileType; import com.netease.arctic.data.DataTreeNode; import com.netease.arctic.data.DefaultKeyedFile; -import com.netease.arctic.io.reader.GenericArcticDataReader; -import com.netease.arctic.io.writer.GenericBaseTaskWriter; -import com.netease.arctic.io.writer.GenericTaskWriters; +import com.netease.arctic.hive.io.reader.AdaptHiveGenericArcticDataReader; +import com.netease.arctic.hive.io.writer.AdaptHiveGenericTaskWriterBuilder; import com.netease.arctic.optimizer.OptimizerConfig; import com.netease.arctic.scan.ArcticFileScanTask; import com.netease.arctic.scan.BaseArcticFileScanTask; @@ -36,30 +36,24 @@ import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.PrimaryKeySpec; -import com.netease.arctic.table.TableProperties; +import com.netease.arctic.table.WriteOperationKind; import com.netease.arctic.utils.SerializationUtil; import org.apache.commons.beanutils.BeanUtils; import org.apache.commons.collections.CollectionUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.IdentityPartitionConverters; import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.DataWriter; -import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.TaskWriter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; @@ -89,11 +83,7 @@ public OptimizeTaskResult execute() throws Exception { dataFiles.addAll(task.deleteFiles()); CloseableIterator recordIterator = openTask(dataFiles, deleteFileMap, table.schema(), task.getSourceNodes()); - if (table.isUnkeyedTable()) { - targetFiles = optimizeUnKeyedTable(recordIterator); - } else { - targetFiles = optimizeKeyedTable(recordIterator); - } + targetFiles = optimizeTable(recordIterator); long totalFileSize = 0; List baseFileBytesList = new ArrayList<>(); @@ -128,13 +118,12 @@ public OptimizeTaskResult execute() throws Exception { public void close() { } - private Iterable optimizeKeyedTable(CloseableIterator recordIterator) throws Exception { - KeyedTable keyedTable = table.asKeyedTable(); - - GenericBaseTaskWriter writer = GenericTaskWriters.builderFor(keyedTable) + private Iterable optimizeTable(CloseableIterator recordIterator) throws Exception { + TaskWriter writer = AdaptHiveGenericTaskWriterBuilder.builderFor(table) .withTransactionId(getMaxTransactionId(task.dataFiles())) .withTaskId(task.getAttemptId()) - .buildBaseWriter(); + .buildWriter(task.getOptimizeType() == OptimizeType.Major ? + WriteOperationKind.MAJOR_OPTIMIZE : WriteOperationKind.FULL_OPTIMIZE); long insertCount = 0; while (recordIterator.hasNext()) { Record baseRecord = recordIterator.next(); @@ -151,37 +140,6 @@ private Iterable optimizeKeyedTable(CloseableIterator recordIt return Arrays.asList(writer.complete().dataFiles()); } - private Iterable optimizeUnKeyedTable(CloseableIterator recordIterator) { - GenericAppenderFactory appenderFactory = new GenericAppenderFactory(table.schema(), table.spec()); - FileFormat fileFormat = FileFormat.valueOf((table.properties().getOrDefault(TableProperties.BASE_FILE_FORMAT, - TableProperties.BASE_FILE_FORMAT_DEFAULT).toUpperCase(Locale.ENGLISH))); - OutputFileFactory outputFileFactory = OutputFileFactory - .builderFor(table.asUnkeyedTable(), 0, 0).format(fileFormat).build(); - EncryptedOutputFile outputFile = outputFileFactory.newOutputFile(task.getPartition()); - DataFile targetFile = table.io().doAs(() -> { - DataWriter writer = appenderFactory - .newDataWriter(outputFile, FileFormat.PARQUET, task.getPartition()); - - long insertCount = 0; - while (recordIterator.hasNext()) { - Record baseRecord = recordIterator.next(); - writer.add(baseRecord); - insertCount++; - if (insertCount == 1 || insertCount == 100000) { - LOG.info("task {} insert records number {} and data sampling {}", - task.getTaskId(), insertCount, baseRecord); - } - } - - LOG.info("task {} insert records number {}", task.getTaskId(), insertCount); - - writer.close(); - return writer.toDataFile(); - }); - - return Collections.singletonList(targetFile); - } - private CloseableIterator openTask(List dataFiles, Map> deleteFileMap, Schema requiredSchema, Set sourceNodes) { @@ -195,8 +153,8 @@ private CloseableIterator openTask(List dataFiles, primaryKeySpec = keyedTable.primaryKeySpec(); } - GenericArcticDataReader arcticDataReader = - new GenericArcticDataReader(table.io(), table.schema(), requiredSchema, primaryKeySpec, + AdaptHiveGenericArcticDataReader arcticDataReader = + new AdaptHiveGenericArcticDataReader(table.io(), table.schema(), requiredSchema, primaryKeySpec, null, false, IdentityPartitionConverters::convertConstant, sourceNodes, false); List fileScanTasks = dataFiles.stream() diff --git a/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/MinorExecutor.java b/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/MinorExecutor.java index 4d0a978857..ebb5cacb8b 100644 --- a/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/MinorExecutor.java +++ b/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/MinorExecutor.java @@ -24,9 +24,9 @@ import com.netease.arctic.ams.api.OptimizeTaskStat; import com.netease.arctic.data.DataTreeNode; import com.netease.arctic.data.DefaultKeyedFile; +import com.netease.arctic.hive.io.reader.AdaptHiveGenericArcticDataReader; +import com.netease.arctic.hive.io.writer.AdaptHiveGenericTaskWriterBuilder; import com.netease.arctic.io.reader.BaseIcebergPosDeleteReader; -import com.netease.arctic.io.reader.GenericArcticDataReader; -import com.netease.arctic.io.writer.GenericTaskWriters; import com.netease.arctic.io.writer.SortedPosDeleteWriter; import com.netease.arctic.optimizer.OptimizerConfig; import com.netease.arctic.scan.ArcticFileScanTask; @@ -93,7 +93,7 @@ public OptimizeTaskResult execute() throws Exception { CloseableIterator iterator = openTask(dataFiles, posDeleteList, requiredSchema, task.getSourceNodes()); - SortedPosDeleteWriter posDeleteWriter = GenericTaskWriters.builderFor(keyedTable) + SortedPosDeleteWriter posDeleteWriter = AdaptHiveGenericTaskWriterBuilder.builderFor(keyedTable) .withTransactionId(getMaxTransactionId(dataFiles)) .withTaskId(task.getAttemptId()) .buildBasePosDeleteWriter(treeNode.mask(), treeNode.index(), task.getPartition()); @@ -179,8 +179,8 @@ private CloseableIterator openTask(List dataFiles, List baseDataFilesInfo = new ArrayList<>(); - protected List posDeleteFilesInfo = new ArrayList<>(); - - protected List insertKeyedTableBaseDataFiles(long transactionId) throws IOException { - GenericBaseTaskWriter writer = GenericTaskWriters.builderFor(testKeyedTable) - .withTransactionId(transactionId).buildBaseWriter(); - - List baseDataFiles = new ArrayList<>(); - // write 1000 records to 2 partitions(2022-1-1\2022-1-2) - int length = 100; - for (int i = 1; i < length * 10; i = i + length) { - for (Record record : baseRecords(i, length)) { - writer.write(record); - } - WriteResult result = writer.complete(); - baseDataFiles.addAll(Arrays.asList(result.dataFiles())); - } - AppendFiles baseAppend = testKeyedTable.baseTable().newAppend(); - baseDataFiles.forEach(baseAppend::appendFile); - baseAppend.commit(); - long commitTime = System.currentTimeMillis(); - - baseDataFilesInfo = baseDataFiles.stream() - .map(dataFile -> DataFileInfoUtils.convertToDatafileInfo(dataFile, commitTime, testKeyedTable)) - .collect(Collectors.toList()); - return baseDataFiles; - } - - protected List insertBasePosDeleteFiles(long transactionId) throws IOException { - List dataFiles = insertKeyedTableBaseDataFiles(transactionId - 1); - Map> dataFilesPartitionMap = - new HashMap<>(dataFiles.stream().collect(Collectors.groupingBy(ContentFile::partition))); - List deleteFiles = new ArrayList<>(); - for (Map.Entry> dataFilePartitionMap : dataFilesPartitionMap.entrySet()) { - StructLike partition = dataFilePartitionMap.getKey(); - List partitionFiles = dataFilePartitionMap.getValue(); - Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() - .collect(Collectors.groupingBy(dataFile -> - DefaultKeyedFile.parseMetaFromFileName(dataFile.path().toString()).node()))); - for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { - DataTreeNode key = nodeFilePartitionMap.getKey(); - List nodeFiles = nodeFilePartitionMap.getValue(); - - // write pos delete - SortedPosDeleteWriter writer = GenericTaskWriters.builderFor(testKeyedTable) - .withTransactionId(transactionId).buildBasePosDeleteWriter(key.getMask(), key.getIndex(), partition); - for (DataFile nodeFile : nodeFiles) { - writer.delete(nodeFile.path(), 0); - } - deleteFiles.addAll(writer.complete()); - } - } - RowDelta rowDelta = testKeyedTable.baseTable().newRowDelta(); - deleteFiles.forEach(rowDelta::addDeletes); - rowDelta.commit(); - long commitTime = System.currentTimeMillis(); - - posDeleteFilesInfo.addAll(deleteFiles.stream() - .map(deleteFile -> DataFileInfoUtils.convertToDatafileInfo(deleteFile, commitTime, testKeyedTable)) - .collect(Collectors.toList())); - - return deleteFiles; - } - - protected List baseRecords(int start, int length) { - GenericRecord record = GenericRecord.create(TABLE_SCHEMA); - - ImmutableList.Builder builder = ImmutableList.builder(); - for (int i = start; i < start + length; i++) { - builder.add(record.copy(ImmutableMap.of("id", i, "name", "name" + i, "op_time", - LocalDateTime.of(2022, 1, 1, 12, 0, 0)))); - } - - return builder.build(); - } -} diff --git a/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestBaseOptimizeBase.java b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestBaseOptimizeBase.java new file mode 100644 index 0000000000..2f34bdac0d --- /dev/null +++ b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestBaseOptimizeBase.java @@ -0,0 +1,63 @@ +/* + * 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 com.netease.arctic.optimizer.operator.executor; + +import com.netease.arctic.TableTestBase; +import com.netease.arctic.ams.api.DataFileInfo; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.After; +import org.junit.Before; + +import java.time.LocalDateTime; +import java.util.ArrayList; +import java.util.List; + +public class TestBaseOptimizeBase extends TableTestBase implements TestOptimizeBase { + protected List baseDataFilesInfo = new ArrayList<>(); + protected List posDeleteFilesInfo = new ArrayList<>(); + + @Before + public void initDataFileInfo() { + baseDataFilesInfo = new ArrayList<>(); + posDeleteFilesInfo = new ArrayList<>(); + } + + @After + public void clearDataFileInfo() { + baseDataFilesInfo.clear(); + posDeleteFilesInfo.clear(); + } + + @Override + public List baseRecords(int start, int length, Schema tableSchema) { + GenericRecord record = GenericRecord.create(tableSchema); + + ImmutableList.Builder builder = ImmutableList.builder(); + for (int i = start; i < start + length; i++) { + builder.add(record.copy(ImmutableMap.of("id", i, "name", "name", + "op_time", LocalDateTime.of(2022, 1, 1, 12, 0, 0)))); + } + + return builder.build(); + } +} diff --git a/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestMajorExecutor.java b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestMajorExecutor.java index d1fa760294..b192a963d5 100644 --- a/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestMajorExecutor.java +++ b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestMajorExecutor.java @@ -26,9 +26,9 @@ import com.netease.arctic.data.DataTreeNode; import com.netease.arctic.optimizer.OptimizerConfig; import com.netease.arctic.optimizer.util.ContentFileUtil; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.UnkeyedTable; import org.apache.iceberg.DataFile; -import org.apache.iceberg.PartitionKey; import org.junit.Assert; import org.junit.Test; @@ -36,38 +36,123 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; -public class TestMajorExecutor extends TestBaseExecutor { - +public class TestMajorExecutor extends TestBaseOptimizeBase { @Test public void testMajorExecutor() throws Exception { - insertBasePosDeleteFiles(2); - NodeTask nodeTask = constructNodeTask(); + insertBasePosDeleteFiles(testKeyedTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + NodeTask nodeTask = constructNodeTask(testKeyedTable, OptimizeType.Major); String[] arg = new String[0]; OptimizerConfig optimizerConfig = new OptimizerConfig(arg); optimizerConfig.setOptimizerId("UnitTest"); MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testKeyedTable, System.currentTimeMillis(), optimizerConfig); OptimizeTaskResult result = majorExecutor.execute(); Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(240, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testKeyedTable.baseLocation())); + }); + } + + @Test + public void testFullMajorExecutor() throws Exception { + insertBasePosDeleteFiles(testKeyedTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + NodeTask nodeTask = constructNodeTask(testKeyedTable, OptimizeType.FullMajor); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testKeyedTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(240, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testKeyedTable.baseLocation())); + }); + } + + @Test + public void testUnKeyedTableMajorExecutor() throws Exception { + insertTableBaseDataFiles(testTable, 1, baseDataFilesInfo); + NodeTask nodeTask = constructNodeTask(testTable, OptimizeType.Major); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 1); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(1000, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testTable.location())); + }); + } + + @Test + public void testUnKeyedTableFullMajorExecutor() throws Exception { + insertTableBaseDataFiles(testTable, 1, baseDataFilesInfo); + NodeTask nodeTask = constructNodeTask(testTable, OptimizeType.FullMajor); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 1); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(1000, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testTable.location())); + }); + } + + @Test + public void testNoPartitionTableMajorExecutor() throws Exception { + insertBasePosDeleteFiles(testNoPartitionTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + NodeTask nodeTask = constructNodeTask(testNoPartitionTable, OptimizeType.Major); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testNoPartitionTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(240, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testNoPartitionTable.baseLocation())); + }); + } + + @Test + public void testNoPartitionTableFullMajorExecutor() throws Exception { + insertBasePosDeleteFiles(testNoPartitionTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + NodeTask nodeTask = constructNodeTask(testNoPartitionTable, OptimizeType.FullMajor); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testNoPartitionTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(240, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testNoPartitionTable.baseLocation())); + }); } - private NodeTask constructNodeTask() { + private NodeTask constructNodeTask(ArcticTable arcticTable, OptimizeType optimizeType) { NodeTask nodeTask = new NodeTask(); nodeTask.setSourceNodes(baseDataFilesInfo.stream() .map(dataFileInfo -> DataTreeNode.of(dataFileInfo.getMask(), dataFileInfo.getIndex())) .collect(Collectors.toSet())); - nodeTask.setTableIdentifier(testKeyedTable.id()); - nodeTask.setTaskId(new OptimizeTaskId(OptimizeType.Major, UUID.randomUUID().toString())); + nodeTask.setTableIdentifier(arcticTable.id()); + nodeTask.setTaskId(new OptimizeTaskId(optimizeType, UUID.randomUUID().toString())); nodeTask.setAttemptId(Math.abs(ThreadLocalRandom.current().nextInt())); nodeTask.setPartition(FILE_A.partition()); + UnkeyedTable baseTable = arcticTable.isKeyedTable() ? + arcticTable.asKeyedTable().baseTable() : arcticTable.asUnkeyedTable(); for (DataFileInfo fileInfo : baseDataFilesInfo) { nodeTask.addFile( - ContentFileUtil.buildContentFile(fileInfo, testKeyedTable.baseTable().spec(), testKeyedTable.io()), + ContentFileUtil.buildContentFile(fileInfo, baseTable.spec(), arcticTable.io()), DataFileType.BASE_FILE); } for (DataFileInfo fileInfo : posDeleteFilesInfo) { nodeTask.addFile( - ContentFileUtil.buildContentFile(fileInfo, testKeyedTable.baseTable().spec(), testKeyedTable.io()), + ContentFileUtil.buildContentFile(fileInfo, baseTable.spec(), arcticTable.io()), DataFileType.POS_DELETE_FILE); } diff --git a/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestMinorExecutor.java b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestMinorExecutor.java index c995984858..f93da96fef 100644 --- a/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestMinorExecutor.java +++ b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestMinorExecutor.java @@ -46,13 +46,13 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; -public class TestMinorExecutor extends TestBaseExecutor { +public class TestMinorExecutor extends TestBaseOptimizeBase { protected List changeInsertFilesInfo = new ArrayList<>(); protected List changeDeleteFilesInfo = new ArrayList<>(); @Test public void testMinorExecutor() throws Exception { - insertBasePosDeleteFiles(2); + insertBasePosDeleteFiles(testKeyedTable, 2, baseDataFilesInfo, posDeleteFilesInfo); insertChangeDeleteFiles(3); insertChangeDataFiles(4); @@ -63,6 +63,29 @@ public void testMinorExecutor() throws Exception { MinorExecutor minorExecutor = new MinorExecutor(nodeTask, testKeyedTable, System.currentTimeMillis(), optimizerConfig); OptimizeTaskResult result = minorExecutor.execute(); Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(250, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testKeyedTable.baseLocation())); + }); + } + + @Test + public void testNoPartitionTableMinorExecutor() throws Exception { + insertBasePosDeleteFiles(testNoPartitionTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + insertChangeDeleteFiles(3); + insertChangeDataFiles(4); + + NodeTask nodeTask = constructNodeTask(); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MinorExecutor minorExecutor = new MinorExecutor(nodeTask, testNoPartitionTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = minorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(250, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testNoPartitionTable.baseLocation())); + }); } private NodeTask constructNodeTask() { @@ -71,7 +94,7 @@ private NodeTask constructNodeTask() { .map(dataFileInfo -> DataTreeNode.of(dataFileInfo.getMask(), dataFileInfo.getIndex())) .collect(Collectors.toSet())); nodeTask.setTableIdentifier(testKeyedTable.id()); - nodeTask.setTaskId(new OptimizeTaskId(OptimizeType.Major, UUID.randomUUID().toString())); + nodeTask.setTaskId(new OptimizeTaskId(OptimizeType.Minor, UUID.randomUUID().toString())); nodeTask.setAttemptId(Math.abs(ThreadLocalRandom.current().nextInt())); nodeTask.setPartition(FILE_A.partition()); @@ -108,7 +131,7 @@ protected void insertChangeDeleteFiles(long transactionId) throws IOException { // delete 1000 records in 2 partitions(2022-1-1\2022-1-2) int length = 100; for (int i = 1; i < length * 10; i = i + length) { - for (Record record : baseRecords(i, length)) { + for (Record record : baseRecords(i, length, testKeyedTable.changeTable().schema())) { writer.write(record); } WriteResult result = writer.complete(); @@ -133,7 +156,7 @@ protected void insertChangeDataFiles(long transactionId) throws IOException { // write 1000 records to 2 partitions(2022-1-1\2022-1-2) int length = 100; for (int i = 1; i < length * 10; i = i + length) { - for (Record record : baseRecords(i, length)) { + for (Record record : baseRecords(i, length, testKeyedTable.changeTable().schema())) { writer.write(record); } WriteResult result = writer.complete(); diff --git a/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestOptimizeBase.java b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestOptimizeBase.java new file mode 100644 index 0000000000..1099bcbc13 --- /dev/null +++ b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestOptimizeBase.java @@ -0,0 +1,188 @@ +/* + * 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 com.netease.arctic.optimizer.operator.executor; + +import com.netease.arctic.ams.api.DataFileInfo; +import com.netease.arctic.ams.api.OptimizeType; +import com.netease.arctic.data.DataTreeNode; +import com.netease.arctic.data.DefaultKeyedFile; +import com.netease.arctic.hive.io.writer.AdaptHiveGenericTaskWriterBuilder; +import com.netease.arctic.io.writer.SortedPosDeleteWriter; +import com.netease.arctic.optimizer.util.DataFileInfoUtils; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.BaseLocationKind; +import com.netease.arctic.table.UnkeyedTable; +import com.netease.arctic.table.WriteOperationKind; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public interface TestOptimizeBase { + List baseRecords(int start, int length, Schema tableSchema); + + default List insertTableBaseDataFiles(ArcticTable arcticTable, long transactionId, List baseDataFilesInfo) throws IOException { + TaskWriter writer = arcticTable.isKeyedTable() ? + AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .withTransactionId(transactionId) + .buildWriter(BaseLocationKind.INSTANT) : + AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .buildWriter(BaseLocationKind.INSTANT) ; + + List baseDataFiles = insertBaseDataFiles(writer, arcticTable.schema()); + UnkeyedTable baseTable = arcticTable.isKeyedTable() ? + arcticTable.asKeyedTable().baseTable() : arcticTable.asUnkeyedTable(); + AppendFiles baseAppend = baseTable.newAppend(); + baseDataFiles.forEach(baseAppend::appendFile); + baseAppend.commit(); + + long commitTime = System.currentTimeMillis(); + + baseDataFilesInfo.addAll(baseDataFiles.stream() + .map(dataFile -> DataFileInfoUtils.convertToDatafileInfo(dataFile, commitTime, arcticTable)) + .collect(Collectors.toList())); + return baseDataFiles; + } + + default List insertOptimizeTargetDataFiles(ArcticTable arcticTable, + OptimizeType optimizeType, + long transactionId) throws IOException { + WriteOperationKind writeOperationKind = WriteOperationKind.MAJOR_OPTIMIZE; + switch (optimizeType) { + case FullMajor: + writeOperationKind = WriteOperationKind.FULL_OPTIMIZE; + break; + case Major: + break; + case Minor: + writeOperationKind = WriteOperationKind.MINOR_OPTIMIZE; + break; + } + TaskWriter writer = arcticTable.isKeyedTable() ? + AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .withTransactionId(transactionId) + .buildWriter(writeOperationKind) : + AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .buildWriter(writeOperationKind); + + return insertBaseDataFiles(writer, arcticTable.schema()); + } + + default List insertBasePosDeleteFiles(ArcticTable arcticTable, + long transactionId, + List baseDataFilesInfo, + List posDeleteFilesInfo) throws IOException { + List dataFiles = insertTableBaseDataFiles(arcticTable, transactionId - 1, baseDataFilesInfo); + Map> dataFilesPartitionMap = + new HashMap<>(dataFiles.stream().collect(Collectors.groupingBy(ContentFile::partition))); + List deleteFiles = new ArrayList<>(); + for (Map.Entry> dataFilePartitionMap : dataFilesPartitionMap.entrySet()) { + StructLike partition = dataFilePartitionMap.getKey(); + List partitionFiles = dataFilePartitionMap.getValue(); + Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() + .collect(Collectors.groupingBy(dataFile -> + DefaultKeyedFile.parseMetaFromFileName(dataFile.path().toString()).node()))); + for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { + DataTreeNode key = nodeFilePartitionMap.getKey(); + List nodeFiles = nodeFilePartitionMap.getValue(); + + // write pos delete + SortedPosDeleteWriter posDeleteWriter = AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .withTransactionId(transactionId) + .buildBasePosDeleteWriter(key.mask(), key.index(), partition); + for (DataFile nodeFile : nodeFiles) { + posDeleteWriter.delete(nodeFile.path(), 0); + } + deleteFiles.addAll(posDeleteWriter.complete()); + } + } + + UnkeyedTable baseTable = arcticTable.isKeyedTable() ? + arcticTable.asKeyedTable().baseTable() : arcticTable.asUnkeyedTable(); + RowDelta rowDelta = baseTable.newRowDelta(); + deleteFiles.forEach(rowDelta::addDeletes); + rowDelta.commit(); + + long commitTime = System.currentTimeMillis(); + posDeleteFilesInfo.addAll(deleteFiles.stream() + .map(deleteFile -> DataFileInfoUtils.convertToDatafileInfo(deleteFile, commitTime, arcticTable.asKeyedTable())) + .collect(Collectors.toList())); + + return deleteFiles; + } + + default List insertOptimizeTargetDeleteFiles(ArcticTable arcticTable, + List dataFiles, + long transactionId) throws IOException { + Map> dataFilesPartitionMap = + new HashMap<>(dataFiles.stream().collect(Collectors.groupingBy(ContentFile::partition))); + List deleteFiles = new ArrayList<>(); + for (Map.Entry> dataFilePartitionMap : dataFilesPartitionMap.entrySet()) { + StructLike partition = dataFilePartitionMap.getKey(); + List partitionFiles = dataFilePartitionMap.getValue(); + Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() + .collect(Collectors.groupingBy(dataFile -> + DefaultKeyedFile.parseMetaFromFileName(dataFile.path().toString()).node()))); + for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { + DataTreeNode key = nodeFilePartitionMap.getKey(); + List nodeFiles = nodeFilePartitionMap.getValue(); + + // write pos delete + SortedPosDeleteWriter posDeleteWriter = AdaptHiveGenericTaskWriterBuilder.builderFor(arcticTable) + .withTransactionId(transactionId) + .buildBasePosDeleteWriter(key.mask(), key.index(), partition); + for (DataFile nodeFile : nodeFiles) { + posDeleteWriter.delete(nodeFile.path(), 0); + } + deleteFiles.addAll(posDeleteWriter.complete()); + } + } + + return deleteFiles; + } + + default List insertBaseDataFiles(TaskWriter writer, Schema schema) throws IOException { + List baseDataFiles = new ArrayList<>(); + // write 1000 records to 1 partitions(2022-1-1) + int length = 100; + for (int i = 1; i < length * 10; i = i + length) { + for (Record record : baseRecords(i, length, schema)) { + writer.write(record); + } + WriteResult result = writer.complete(); + baseDataFiles.addAll(Arrays.asList(result.dataFiles())); + } + + return baseDataFiles; + } +} diff --git a/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestSupportHiveMajorExecutor.java b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestSupportHiveMajorExecutor.java new file mode 100644 index 0000000000..b412ff8b26 --- /dev/null +++ b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestSupportHiveMajorExecutor.java @@ -0,0 +1,177 @@ +/* + * 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 com.netease.arctic.optimizer.operator.executor; + +import com.google.common.collect.Iterables; +import com.netease.arctic.ams.api.DataFileInfo; +import com.netease.arctic.ams.api.OptimizeTaskId; +import com.netease.arctic.ams.api.OptimizeType; +import com.netease.arctic.data.DataFileType; +import com.netease.arctic.data.DataTreeNode; +import com.netease.arctic.optimizer.OptimizerConfig; +import com.netease.arctic.optimizer.util.ContentFileUtil; +import com.netease.arctic.table.ArcticTable; +import com.netease.arctic.table.UnkeyedTable; +import org.apache.iceberg.DataFile; +import org.junit.Assert; +import org.junit.Test; + +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; + +public class TestSupportHiveMajorExecutor extends TestSupportHiveMajorOptimizeBase { + @Test + public void testMajorExecutor() throws Exception { + insertBasePosDeleteFiles(testKeyedHiveTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + NodeTask nodeTask = constructNodeTask(testKeyedHiveTable, OptimizeType.Major); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testKeyedHiveTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(240, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testKeyedHiveTable.baseLocation())); + }); + } + + @Test + public void testNoPosDeleteMajorExecutor() throws Exception { + insertTableBaseDataFiles(testKeyedHiveTable, 2, baseDataFilesInfo); + NodeTask nodeTask = constructNodeTask(testKeyedHiveTable, OptimizeType.Major); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testKeyedHiveTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(250, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testKeyedHiveTable.baseLocation())); + }); + } + + @Test + public void testFullMajorExecutor() throws Exception { + insertBasePosDeleteFiles(testKeyedHiveTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + NodeTask nodeTask = constructNodeTask(testKeyedHiveTable, OptimizeType.FullMajor); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testKeyedHiveTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(240, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testKeyedHiveTable.hiveLocation())); + }); + } + + @Test + public void testUnKeyedTableMajorExecutor() throws Exception { + insertTableBaseDataFiles(testHiveTable, 1, baseDataFilesInfo); + NodeTask nodeTask = constructNodeTask(testHiveTable, OptimizeType.Major); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testHiveTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 1); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(1000, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testHiveTable.location())); + }); + } + + @Test + public void testUnKeyedTableFullMajorExecutor() throws Exception { + insertTableBaseDataFiles(testHiveTable, 1, baseDataFilesInfo); + NodeTask nodeTask = constructNodeTask(testHiveTable, OptimizeType.FullMajor); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testHiveTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 1); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(1000, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testHiveTable.hiveLocation())); + }); + } + + @Test + public void testNoPartitionTableMajorExecutor() throws Exception { + insertBasePosDeleteFiles(testUnPartitionKeyedHiveTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + NodeTask nodeTask = constructNodeTask(testUnPartitionKeyedHiveTable, OptimizeType.Major); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testUnPartitionKeyedHiveTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(240, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testUnPartitionKeyedHiveTable.baseLocation())); + }); + } + + @Test + public void testNoPartitionTableFullMajorExecutor() throws Exception { + insertBasePosDeleteFiles(testUnPartitionKeyedHiveTable, 2, baseDataFilesInfo, posDeleteFilesInfo); + NodeTask nodeTask = constructNodeTask(testUnPartitionKeyedHiveTable, OptimizeType.FullMajor); + String[] arg = new String[0]; + OptimizerConfig optimizerConfig = new OptimizerConfig(arg); + optimizerConfig.setOptimizerId("UnitTest"); + MajorExecutor majorExecutor = new MajorExecutor(nodeTask, testUnPartitionKeyedHiveTable, System.currentTimeMillis(), optimizerConfig); + OptimizeTaskResult result = majorExecutor.execute(); + Assert.assertEquals(Iterables.size(result.getTargetFiles()), 4); + result.getTargetFiles().forEach(dataFile -> { + Assert.assertEquals(240, dataFile.recordCount()); + Assert.assertTrue(dataFile.path().toString().contains(testUnPartitionKeyedHiveTable.hiveLocation())); + }); + } + + private NodeTask constructNodeTask(ArcticTable arcticTable, OptimizeType optimizeType) { + NodeTask nodeTask = new NodeTask(); + nodeTask.setSourceNodes(baseDataFilesInfo.stream() + .map(dataFileInfo -> DataTreeNode.of(dataFileInfo.getMask(), dataFileInfo.getIndex())) + .collect(Collectors.toSet())); + nodeTask.setTableIdentifier(arcticTable.id()); + nodeTask.setTaskId(new OptimizeTaskId(optimizeType, UUID.randomUUID().toString())); + nodeTask.setAttemptId(Math.abs(ThreadLocalRandom.current().nextInt())); + nodeTask.setPartition(FILE_A.partition()); + + UnkeyedTable baseTable = arcticTable.isKeyedTable() ? + arcticTable.asKeyedTable().baseTable() : arcticTable.asUnkeyedTable(); + for (DataFileInfo fileInfo : baseDataFilesInfo) { + nodeTask.addFile( + ContentFileUtil.buildContentFile(fileInfo, baseTable.spec(), arcticTable.io()), + DataFileType.BASE_FILE); + } + for (DataFileInfo fileInfo : posDeleteFilesInfo) { + nodeTask.addFile( + ContentFileUtil.buildContentFile(fileInfo, baseTable.spec(), arcticTable.io()), + DataFileType.POS_DELETE_FILE); + } + + return nodeTask; + } +} diff --git a/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestSupportHiveMajorOptimizeBase.java b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestSupportHiveMajorOptimizeBase.java new file mode 100644 index 0000000000..be5ba1853f --- /dev/null +++ b/optimizer/src/test/java/com/netease/arctic/optimizer/operator/executor/TestSupportHiveMajorOptimizeBase.java @@ -0,0 +1,66 @@ +/* + * 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 com.netease.arctic.optimizer.operator.executor; + +import com.netease.arctic.ams.api.DataFileInfo; +import com.netease.arctic.hive.HiveTableTestBase; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.After; +import org.junit.Before; + +import java.math.BigDecimal; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.ArrayList; +import java.util.List; + +public class TestSupportHiveMajorOptimizeBase extends HiveTableTestBase implements TestOptimizeBase { + protected List baseDataFilesInfo = new ArrayList<>(); + protected List posDeleteFilesInfo = new ArrayList<>(); + + @Before + public void initDataFileInfo() { + baseDataFilesInfo = new ArrayList<>(); + posDeleteFilesInfo = new ArrayList<>(); + } + + @After + public void clearDataFileInfo() { + baseDataFilesInfo.clear(); + posDeleteFilesInfo.clear(); + } + + public List baseRecords(int start, int length, Schema tableSchema) { + GenericRecord record = GenericRecord.create(tableSchema); + + ImmutableList.Builder builder = ImmutableList.builder(); + for (int i = start; i < start + length; i++) { + builder.add(record.copy(ImmutableMap.of("id", i, + "op_time", LocalDateTime.of(2022, 1, 1, 12, 0, 0), + "op_time_with_zone", LocalDateTime.of(2022, 1, i % 2 + 1, 12, 0, 0).atOffset(ZoneOffset.UTC), + "d", new BigDecimal(i), "name", "name"))); + } + + return builder.build(); + } +}