diff --git a/LICENSE b/LICENSE index 5d38daae5f..5f6951a343 100644 --- a/LICENSE +++ b/LICENSE @@ -208,8 +208,8 @@ This product includes code from Apache Iceberg. * org.apache.iceberg.data.DeleteFilter copied and modified to com.netease.arctic.iceberg.optimize.DeleteFilter * org.apache.iceberg.deletes.Deletes copied and modified to com.netease.arctic.iceberg.optimize.Deletes * org.apache.iceberg.data.InternalRecordWrapper copied and modified to com.netease.arctic.iceberg.optimize.InternalRecordWrapper -* org.apache.iceberg.util.StructLikeMap copied and modified to com.netease.arctic.iceberg.optimize.StructLikeMap -* org.apache.iceberg.util.StructLikeSet copied and modified to com.netease.arctic.iceberg.optimize.StructLikeSet +* org.apache.iceberg.util.StructLikeMap copied and modified to com.netease.arctic.utils.map.StructLikeMemoryMap +* org.apache.iceberg.util.StructLikeSet copied and modified to com.netease.arctic.utils.StructLikeSet * org.apache.iceberg.util.StructLikeWrapper copied and modified to com.netease.arctic.iceberg.optimize.StructLikeWrapper * org.apache.iceberg.util.StructProjection copied and modified to com.netease.arctic.iceberg.optimize.StructProjection * org.apache.iceberg.data.parquet.BaseParquetReaders copied and modified to org.apache.iceberg.data.parquet.AdaptHiveBaseParquetReaders diff --git a/ams/ams-api/src/main/java/com/netease/arctic/ams/api/client/ArcticThriftUrl.java b/ams/ams-api/src/main/java/com/netease/arctic/ams/api/client/ArcticThriftUrl.java index 400ea756a6..a219a6feb2 100644 --- a/ams/ams-api/src/main/java/com/netease/arctic/ams/api/client/ArcticThriftUrl.java +++ b/ams/ams-api/src/main/java/com/netease/arctic/ams/api/client/ArcticThriftUrl.java @@ -20,6 +20,7 @@ import com.alibaba.fastjson.JSONObject; import com.netease.arctic.ams.api.properties.AmsHAProperties; + import java.net.URI; import java.net.URISyntaxException; import java.util.Locale; diff --git a/ams/ams-api/src/test/java/com/netease/arctic/ams/api/MockZookeeperServer.java b/ams/ams-api/src/test/java/com/netease/arctic/ams/api/MockZookeeperServer.java index 6d0e74f5f6..b12a558307 100644 --- a/ams/ams-api/src/test/java/com/netease/arctic/ams/api/MockZookeeperServer.java +++ b/ams/ams-api/src/test/java/com/netease/arctic/ams/api/MockZookeeperServer.java @@ -18,16 +18,14 @@ package com.netease.arctic.ams.api; -import java.util.Random; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.retry.ExponentialBackoffRetry; import org.apache.curator.test.TestingServer; -import org.junit.AfterClass; -import org.junit.BeforeClass; import org.junit.Test; import java.io.IOException; +import java.util.Random; /** * Provides mock zookeeper server. diff --git a/ams/ams-dashboard/src/assets/icons/svg/slide.svg b/ams/ams-dashboard/src/assets/icons/svg/slide.svg index c5b97d9391..3ff294e92c 100644 --- a/ams/ams-dashboard/src/assets/icons/svg/slide.svg +++ b/ams/ams-dashboard/src/assets/icons/svg/slide.svg @@ -1,5 +1,5 @@ - + slide Created with Sketch. diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseArcticOptimizePlan.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseArcticOptimizePlan.java index f7bc83b7dc..b7a9753fb1 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseArcticOptimizePlan.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseArcticOptimizePlan.java @@ -33,10 +33,9 @@ import com.netease.arctic.hive.utils.HiveTableUtil; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.Snapshot; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -103,19 +102,19 @@ protected BaseOptimizeTask buildOptimizeTask(@Nullable List source List baseFileBytesList = baseFiles.stream() - .map(SerializationUtil::toByteBuffer) + .map(SerializationUtils::toByteBuffer) .collect(Collectors.toList()); List insertFileBytesList = insertFiles.stream() - .map(SerializationUtil::toByteBuffer) + .map(SerializationUtils::toByteBuffer) .collect(Collectors.toList()); List deleteFileBytesList = deleteFiles.stream() - .map(SerializationUtil::toByteBuffer) + .map(SerializationUtils::toByteBuffer) .collect(Collectors.toList()); List posDeleteFileBytesList = posDeleteFiles.stream() - .map(SerializationUtil::toByteBuffer) + .map(SerializationUtils::toByteBuffer) .collect(Collectors.toList()); optimizeTask.setBaseFiles(baseFileBytesList); optimizeTask.setInsertFiles(insertFileBytesList); diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseIcebergOptimizePlan.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseIcebergOptimizePlan.java index 52cc6072aa..334f97ad33 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseIcebergOptimizePlan.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/BaseIcebergOptimizePlan.java @@ -30,7 +30,7 @@ import com.netease.arctic.data.IcebergContentFile; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileContent; @@ -118,25 +118,25 @@ protected BaseOptimizeTask buildOptimizeTask(List insertFiles, baseFiles.stream().map(dataFile -> { IcebergContentFile icebergContentFile = new IcebergContentFile(dataFile, sequenceNumberFetcher.sequenceNumberOf(dataFile.path().toString())); - return SerializationUtil.toByteBuffer(icebergContentFile); + return SerializationUtils.toByteBuffer(icebergContentFile); }).collect(Collectors.toList()); List insertFileBytesList = insertFiles.stream().map(dataFile -> { IcebergContentFile icebergContentFile = new IcebergContentFile(dataFile, sequenceNumberFetcher.sequenceNumberOf(dataFile.path().toString())); - return SerializationUtil.toByteBuffer(icebergContentFile); + return SerializationUtils.toByteBuffer(icebergContentFile); }).collect(Collectors.toList()); List eqDeleteFileBytesList = eqDeleteFiles.stream().map(deleteFile -> { IcebergContentFile icebergContentFile = new IcebergContentFile(deleteFile, sequenceNumberFetcher.sequenceNumberOf(deleteFile.path().toString())); - return SerializationUtil.toByteBuffer(icebergContentFile); + return SerializationUtils.toByteBuffer(icebergContentFile); }).collect(Collectors.toList()); List posDeleteFileBytesList = posDeleteFiles.stream().map(deleteFile -> { IcebergContentFile icebergContentFile = new IcebergContentFile(deleteFile, sequenceNumberFetcher.sequenceNumberOf(deleteFile.path().toString())); - return SerializationUtil.toByteBuffer(icebergContentFile); + return SerializationUtils.toByteBuffer(icebergContentFile); }).collect(Collectors.toList()); optimizeTask.setBaseFiles(baseFileBytesList); optimizeTask.setInsertFiles(insertFileBytesList); 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 fc10cce842..5b3d02dce4 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 @@ -31,8 +31,8 @@ import com.netease.arctic.table.UnkeyedTable; import com.netease.arctic.trace.SnapshotSummary; import com.netease.arctic.utils.ArcticDataFiles; -import com.netease.arctic.utils.FileUtil; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; +import com.netease.arctic.utils.TableFileUtils; import com.netease.arctic.utils.TablePropertyUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.collections.MapUtils; @@ -100,7 +100,7 @@ public boolean commit(long baseSnapshotId) throws Exception { // tasks in partition if (task.getOptimizeTask().getTaskId().getType() == OptimizeType.Minor) { task.getOptimizeRuntime().getTargetFiles().stream() - .map(SerializationUtil::toInternalTableFile) + .map(SerializationUtils::toInternalTableFile) .forEach(minorAddFiles::add); minorDeleteFiles.addAll(selectDeletedFiles(task, minorAddFiles)); @@ -128,7 +128,7 @@ public boolean commit(long baseSnapshotId) throws Exception { partitionOptimizeType.put(entry.getKey(), OptimizeType.Minor); } else { task.getOptimizeRuntime().getTargetFiles().stream() - .map(SerializationUtil::toInternalTableFile) + .map(SerializationUtils::toInternalTableFile) .forEach(majorAddFiles::add); majorDeleteFiles.addAll(selectDeletedFiles(task, new HashSet<>())); partitionOptimizeType.put(entry.getKey(), task.getOptimizeTask().getTaskId().getType()); @@ -395,15 +395,15 @@ private static Set> selectMinorOptimizeDeletedFiles(BaseOptimizeT Set> addPosDeleteFiles) { Set newFileNodes = addPosDeleteFiles.stream().map(contentFile -> { if (contentFile.content() == FileContent.POSITION_DELETES) { - return FileUtil.parseFileNodeFromFileName(contentFile.path().toString()); + return TableFileUtils.parseFileNodeFromFileName(contentFile.path().toString()); } return null; }).filter(Objects::nonNull).collect(Collectors.toSet()); - return optimizeTask.getPosDeleteFiles().stream().map(SerializationUtil::toInternalTableFile) + return optimizeTask.getPosDeleteFiles().stream().map(SerializationUtils::toInternalTableFile) .filter(posDeleteFile -> - newFileNodes.contains(FileUtil.parseFileNodeFromFileName(posDeleteFile.path().toString()))) + newFileNodes.contains(TableFileUtils.parseFileNodeFromFileName(posDeleteFile.path().toString()))) .collect(Collectors.toSet()); } @@ -411,13 +411,13 @@ private static Set> selectMajorOptimizeDeletedFiles(BaseOptimizeT BaseOptimizeTaskRuntime optimizeTaskRuntime) { // add base deleted files Set> result = optimizeTask.getBaseFiles().stream() - .map(SerializationUtil::toInternalTableFile).collect(Collectors.toSet()); + .map(SerializationUtils::toInternalTableFile).collect(Collectors.toSet()); // if full optimize or new DataFiles is empty, can delete DeleteFiles if (optimizeTask.getTaskId().getType() == OptimizeType.FullMajor || CollectionUtils.isEmpty(optimizeTaskRuntime.getTargetFiles())) { result.addAll(optimizeTask.getPosDeleteFiles().stream() - .map(SerializationUtil::toInternalTableFile).collect(Collectors.toSet())); + .map(SerializationUtils::toInternalTableFile).collect(Collectors.toSet())); } return result; diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/FullOptimizePlan.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/FullOptimizePlan.java index ec2e58716f..be49a09009 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/FullOptimizePlan.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/FullOptimizePlan.java @@ -32,8 +32,8 @@ import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.UnkeyedTable; import com.netease.arctic.utils.CompatiblePropertyUtil; -import com.netease.arctic.utils.FileUtil; import com.netease.arctic.utils.IdGenerator; +import com.netease.arctic.utils.TableFileUtils; import org.apache.commons.collections.CollectionUtils; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -219,12 +219,12 @@ private List collectKeyedTableTasks(String partition, FileTree if (!baseFiles.isEmpty()) { List sourceNodes = Collections.singletonList(subTree.getNode()); Set baseFileNodes = baseFiles.stream() - .map(dataFile -> FileUtil.parseFileNodeFromFileName(dataFile.path().toString())) + .map(dataFile -> TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())) .collect(Collectors.toSet()); List posDeleteFiles = partitionPosDeleteFiles .computeIfAbsent(partition, e -> Collections.emptyList()).stream() .filter(deleteFile -> - baseFileNodes.contains(FileUtil.parseFileNodeFromFileName(deleteFile.path().toString()))) + baseFileNodes.contains(TableFileUtils.parseFileNodeFromFileName(deleteFile.path().toString()))) .collect(Collectors.toList()); if (nodeTaskNeedBuild(posDeleteFiles, baseFiles)) { @@ -288,7 +288,7 @@ private void addBaseFileIntoFileTree() { private long getMaxTransactionId(List dataFiles) { OptionalLong maxTransactionId = dataFiles.stream() - .mapToLong(file -> FileUtil.parseFileTidFromFileName(file.path().toString())).max(); + .mapToLong(file -> TableFileUtils.parseFileTidFromFileName(file.path().toString())).max(); if (maxTransactionId.isPresent()) { return maxTransactionId.getAsLong(); } diff --git a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/IcebergOptimizeCommit.java b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/IcebergOptimizeCommit.java index 28836e0e7f..dcaed9a2c4 100644 --- a/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/IcebergOptimizeCommit.java +++ b/ams/ams-server/src/main/java/com/netease/arctic/ams/server/optimize/IcebergOptimizeCommit.java @@ -25,7 +25,7 @@ import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.UnkeyedTable; import com.netease.arctic.trace.SnapshotSummary; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; import org.apache.commons.collections.CollectionUtils; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -77,14 +77,14 @@ public boolean commit(long baseSnapshotId) throws Exception { // tasks in partition if (task.getOptimizeTask().getTaskId().getType() == OptimizeType.Minor) { task.getOptimizeRuntime().getTargetFiles().stream() - .map(SerializationUtil::toInternalTableFile) + .map(SerializationUtils::toInternalTableFile) .forEach(minorAddFiles::add); minorDeleteFiles.addAll(selectDeletedFiles(task)); partitionOptimizeType.put(entry.getKey(), OptimizeType.Minor); } else { task.getOptimizeRuntime().getTargetFiles().stream() - .map(SerializationUtil::toInternalTableFile) + .map(SerializationUtils::toInternalTableFile) .forEach(majorAddFiles::add); majorDeleteFiles.addAll(selectDeletedFiles(task)); partitionOptimizeType.put(entry.getKey(), task.getOptimizeTask().getTaskId().getType()); @@ -248,15 +248,15 @@ private static Set> selectMinorOptimizeDeletedFiles(BaseOptimizeT if (CollectionUtils.isNotEmpty(optimizeTask.getInsertFiles())) { // small data files for (ByteBuffer insertFile : optimizeTask.getInsertFiles()) { - deletedFiles.add(SerializationUtil.toIcebergContentFile(insertFile).asDataFile()); + deletedFiles.add(SerializationUtils.toIcebergContentFile(insertFile).asDataFile()); } } else { // delete files for (ByteBuffer eqDeleteFile : optimizeTask.getDeleteFiles()) { - deletedFiles.add(SerializationUtil.toIcebergContentFile(eqDeleteFile).asDeleteFile()); + deletedFiles.add(SerializationUtils.toIcebergContentFile(eqDeleteFile).asDeleteFile()); } for (ByteBuffer posDeleteFile : optimizeTask.getPosDeleteFiles()) { - deletedFiles.add(SerializationUtil.toIcebergContentFile(posDeleteFile).asDeleteFile()); + deletedFiles.add(SerializationUtils.toIcebergContentFile(posDeleteFile).asDeleteFile()); } } @@ -267,18 +267,18 @@ private static Set> selectMajorOptimizeDeletedFiles(BaseOptimizeT Set> deletedFiles = new HashSet<>(); // data files for (ByteBuffer insertFile : optimizeTask.getInsertFiles()) { - deletedFiles.add(SerializationUtil.toIcebergContentFile(insertFile).asDataFile()); + deletedFiles.add(SerializationUtils.toIcebergContentFile(insertFile).asDataFile()); } for (ByteBuffer baseFile : optimizeTask.getBaseFiles()) { - deletedFiles.add(SerializationUtil.toIcebergContentFile(baseFile).asDataFile()); + deletedFiles.add(SerializationUtils.toIcebergContentFile(baseFile).asDataFile()); } // delete files for (ByteBuffer eqDeleteFile : optimizeTask.getDeleteFiles()) { - deletedFiles.add(SerializationUtil.toIcebergContentFile(eqDeleteFile).asDeleteFile()); + deletedFiles.add(SerializationUtils.toIcebergContentFile(eqDeleteFile).asDeleteFile()); } for (ByteBuffer posDeleteFile : optimizeTask.getPosDeleteFiles()) { - deletedFiles.add(SerializationUtil.toIcebergContentFile(posDeleteFile).asDeleteFile()); + deletedFiles.add(SerializationUtils.toIcebergContentFile(posDeleteFile).asDeleteFile()); } return deletedFiles; 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 8af3b391c4..f9f4183c91 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 @@ -32,7 +32,7 @@ import com.netease.arctic.table.TableProperties; import com.netease.arctic.table.UnkeyedTable; import com.netease.arctic.utils.CompatiblePropertyUtil; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.commons.collections.CollectionUtils; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -258,12 +258,12 @@ private List collectKeyedTableTasks(String partition, FileTree if (!baseFiles.isEmpty()) { List sourceNodes = Collections.singletonList(subTree.getNode()); Set baseFileNodes = baseFiles.stream() - .map(dataFile -> FileUtil.parseFileNodeFromFileName(dataFile.path().toString())) + .map(dataFile -> TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())) .collect(Collectors.toSet()); List posDeleteFiles = partitionPosDeleteFiles .computeIfAbsent(partition, e -> Collections.emptyList()).stream() .filter(deleteFile -> - baseFileNodes.contains(FileUtil.parseFileNodeFromFileName(deleteFile.path().toString()))) + baseFileNodes.contains(TableFileUtils.parseFileNodeFromFileName(deleteFile.path().toString()))) .collect(Collectors.toList()); if (nodeTaskNeedBuild(posDeleteFiles, baseFiles)) { 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 5d140e51a8..dadc24f8c3 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 @@ -32,7 +32,7 @@ import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableProperties; import com.netease.arctic.utils.CompatiblePropertyUtil; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import com.netease.arctic.utils.TablePropertyUtil; import org.apache.commons.collections.CollectionUtils; import org.apache.iceberg.ContentFile; @@ -247,7 +247,7 @@ public long getTransactionId() { public long getLegacyTransactionId() { if (legacyTransactionId == -1) { - legacyTransactionId = FileUtil.parseFileTidFromFileName(dataFileInfo.getPath()); + legacyTransactionId = TableFileUtils.parseFileTidFromFileName(dataFileInfo.getPath()); } return legacyTransactionId; } @@ -338,12 +338,12 @@ private List collectKeyedTableTasks(String partition, FileTree sourceNodes = Collections.singletonList(subTree.getNode()); } Set baseFileNodes = baseFiles.stream() - .map(dataFile -> FileUtil.parseFileNodeFromFileName(dataFile.path().toString())) + .map(dataFile -> TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())) .collect(Collectors.toSet()); List posDeleteFiles = partitionPosDeleteFiles .computeIfAbsent(partition, e -> Collections.emptyList()).stream() .filter(deleteFile -> - baseFileNodes.contains(FileUtil.parseFileNodeFromFileName(deleteFile.path().toString()))) + baseFileNodes.contains(TableFileUtils.parseFileNodeFromFileName(deleteFile.path().toString()))) .collect(Collectors.toList()); // if no insert files and no eq-delete file, skip if (CollectionUtils.isEmpty(insertFiles) && CollectionUtils.isEmpty(deleteFiles)) { 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 d5005b60c0..735717177f 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 @@ -38,7 +38,7 @@ import com.netease.arctic.table.TableIdentifier; import com.netease.arctic.table.TableProperties; import com.netease.arctic.utils.CompatiblePropertyUtil; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; import org.apache.ibatis.session.SqlSession; import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; @@ -238,13 +238,13 @@ public void clearFiles() { public void setFiles() { List insertFiles = selectOptimizeTaskFiles(DataFileType.INSERT_FILE.name(), 0) - .stream().map(SerializationUtil::byteArrayToByteBuffer).collect(Collectors.toList()); + .stream().map(SerializationUtils::byteArrayToByteBuffer).collect(Collectors.toList()); List deleteFiles = selectOptimizeTaskFiles(DataFileType.EQ_DELETE_FILE.name(), 0) - .stream().map(SerializationUtil::byteArrayToByteBuffer).collect(Collectors.toList()); + .stream().map(SerializationUtils::byteArrayToByteBuffer).collect(Collectors.toList()); List baseFiles = selectOptimizeTaskFiles(DataFileType.BASE_FILE.name(), 0) - .stream().map(SerializationUtil::byteArrayToByteBuffer).collect(Collectors.toList()); + .stream().map(SerializationUtils::byteArrayToByteBuffer).collect(Collectors.toList()); List posDeleteFiles = selectOptimizeTaskFiles(DataFileType.POS_DELETE_FILE.name(), 0) - .stream().map(SerializationUtil::byteArrayToByteBuffer).collect(Collectors.toList()); + .stream().map(SerializationUtils::byteArrayToByteBuffer).collect(Collectors.toList()); optimizeTask.setInsertFiles(insertFiles); optimizeTask.setDeleteFiles(deleteFiles); optimizeTask.setBaseFiles(baseFiles); @@ -255,7 +255,7 @@ public void setFiles() { targetFiles.addAll( selectOptimizeTaskFiles(DataFileType.POS_DELETE_FILE.name(), 1)); optimizeRuntime.setTargetFiles(targetFiles.stream() - .map(SerializationUtil::byteArrayToByteBuffer).collect(Collectors.toList())); + .map(SerializationUtils::byteArrayToByteBuffer).collect(Collectors.toList())); } public void setMaxExecuteTime() { @@ -292,13 +292,13 @@ private void persistTaskRuntime(BaseOptimizeTaskRuntime newRuntime, boolean upda try { internalTableFilesMapper.deleteOptimizeTaskTargetFile(optimizeTask.getTaskId()); newRuntime.getTargetFiles().forEach(file -> { - ContentFile contentFile = SerializationUtil.toInternalTableFile(file); + ContentFile contentFile = SerializationUtils.toInternalTableFile(file); if (contentFile.content() == FileContent.DATA) { internalTableFilesMapper.insertOptimizeTaskFile(optimizeTask.getTaskId(), - DataFileType.BASE_FILE.name(), 1, SerializationUtil.byteBufferToByteArray(file)); + DataFileType.BASE_FILE.name(), 1, SerializationUtils.byteBufferToByteArray(file)); } else { internalTableFilesMapper.insertOptimizeTaskFile(optimizeTask.getTaskId(), - DataFileType.POS_DELETE_FILE.name(), 1, SerializationUtil.byteBufferToByteArray(file)); + DataFileType.POS_DELETE_FILE.name(), 1, SerializationUtils.byteBufferToByteArray(file)); } }); } catch (Exception e) { @@ -319,13 +319,13 @@ public void persistTargetFiles() { try { internalTableFilesMapper.deleteOptimizeTaskTargetFile(optimizeTask.getTaskId()); optimizeRuntime.getTargetFiles().forEach(file -> { - ContentFile contentFile = SerializationUtil.toInternalTableFile(file); + ContentFile contentFile = SerializationUtils.toInternalTableFile(file); if (contentFile.content() == FileContent.DATA) { internalTableFilesMapper.insertOptimizeTaskFile(optimizeTask.getTaskId(), - DataFileType.BASE_FILE.name(), 1, SerializationUtil.byteBufferToByteArray(file)); + DataFileType.BASE_FILE.name(), 1, SerializationUtils.byteBufferToByteArray(file)); } else { internalTableFilesMapper.insertOptimizeTaskFile(optimizeTask.getTaskId(), - DataFileType.POS_DELETE_FILE.name(), 1, SerializationUtil.byteBufferToByteArray(file)); + DataFileType.POS_DELETE_FILE.name(), 1, SerializationUtils.byteBufferToByteArray(file)); } }); } catch (Exception e) { @@ -392,25 +392,25 @@ public void persistOptimizeTask() { .insertOptimizeTaskFile(optimizeTaskId, DataFileType.INSERT_FILE.name(), 0, - SerializationUtil.byteBufferToByteArray(f))); + SerializationUtils.byteBufferToByteArray(f))); optimizeTask.getDeleteFiles() .forEach(f -> internalTableFilesMapper .insertOptimizeTaskFile(optimizeTaskId, DataFileType.EQ_DELETE_FILE.name(), 0, - SerializationUtil.byteBufferToByteArray(f))); + SerializationUtils.byteBufferToByteArray(f))); optimizeTask.getBaseFiles() .forEach(f -> internalTableFilesMapper .insertOptimizeTaskFile(optimizeTaskId, DataFileType.BASE_FILE.name(), 0, - SerializationUtil.byteBufferToByteArray(f))); + SerializationUtils.byteBufferToByteArray(f))); optimizeTask.getPosDeleteFiles() .forEach(f -> internalTableFilesMapper .insertOptimizeTaskFile(optimizeTaskId, DataFileType.POS_DELETE_FILE.name(), 0, - SerializationUtil.byteBufferToByteArray(f))); + SerializationUtils.byteBufferToByteArray(f))); sqlSession.commit(true); } catch (Exception e) { 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 index 5c9e5ffcf6..fd307a4241 100644 --- 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 @@ -28,9 +28,9 @@ import com.netease.arctic.hive.utils.HiveTableUtil; import com.netease.arctic.hive.utils.TableTypeUtil; import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.utils.FileUtil; import com.netease.arctic.utils.IdGenerator; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; +import com.netease.arctic.utils.TableFileUtils; import org.apache.commons.collections.CollectionUtils; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; @@ -79,10 +79,10 @@ public boolean commit(long baseSnapshotId) throws Exception { for (OptimizeTaskItem optimizeTaskItem : optimizeTaskItems) { BaseOptimizeTaskRuntime optimizeRuntime = optimizeTaskItem.getOptimizeRuntime(); List targetFiles = optimizeRuntime.getTargetFiles().stream() - .map(fileByte -> (DataFile) SerializationUtil.toInternalTableFile(fileByte)) + .map(fileByte -> (DataFile) SerializationUtils.toInternalTableFile(fileByte)) .collect(Collectors.toList()); long maxTransactionId = targetFiles.stream() - .mapToLong(dataFile -> FileUtil.parseFileTidFromFileName(dataFile.path().toString())) + .mapToLong(dataFile -> TableFileUtils.parseFileTidFromFileName(dataFile.path().toString())) .max() .orElse(0L); @@ -117,7 +117,7 @@ public boolean commit(long baseSnapshotId) throws Exception { } DataFile finalDataFile = moveTargetFiles(targetFile, partitionPathMap.get(partition)); - newTargetFiles.add(SerializationUtil.toByteBuffer(finalDataFile)); + newTargetFiles.add(SerializationUtils.toByteBuffer(finalDataFile)); } optimizeRuntime.setTargetFiles(newTargetFiles); @@ -146,7 +146,7 @@ protected boolean isPartitionMajorOptimizeSupportHive(String partition, List FileUtil.deleteEmptyDirectory(arcticInternalTable.io(), parent, exclude)); + parentDirectory.forEach(parent -> TableFileUtils.deleteEmptyDirectory(arcticInternalTable.io(), parent, exclude)); LOG.info("to delete {} files, success delete {} files", toDeleteFiles.get(), deleteFiles.get()); } } diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/controller/CatalogControllerTest.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/controller/CatalogControllerTest.java index c6a2526bee..32082d5b51 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/controller/CatalogControllerTest.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/controller/CatalogControllerTest.java @@ -25,7 +25,6 @@ import com.netease.arctic.ams.server.model.CatalogSettingInfo; import io.javalin.testtools.HttpClient; import io.javalin.testtools.JavalinTest; -import org.junit.Assert; import org.junit.Test; import java.util.List; 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 index 961e9efe86..89d81cc8ed 100644 --- 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 @@ -24,7 +24,7 @@ import com.netease.arctic.hive.utils.TableTypeUtil; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.UnkeyedTable; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFiles; @@ -65,7 +65,7 @@ public void testExpireTableFiles() throws Exception { Set hiveLocation = new HashSet<>(); if (TableTypeUtil.isHive(testUnPartitionKeyedHiveTable)) { - hiveLocation.add(FileUtil.getFileDir(hiveFiles.get(0).path().toString())); + hiveLocation.add(TableFileUtils.getFileDir(hiveFiles.get(0).path().toString())); } TableExpireService.expireSnapshots(testUnPartitionKeyedHiveTable.baseTable(), System.currentTimeMillis(), hiveLocation); Assert.assertEquals(1, Iterables.size(testUnPartitionKeyedHiveTable.baseTable().snapshots())); diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestIcebergFullOptimizeCommit.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestIcebergFullOptimizeCommit.java index e5341c9f66..f79b96d75f 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestIcebergFullOptimizeCommit.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestIcebergFullOptimizeCommit.java @@ -6,7 +6,7 @@ import com.netease.arctic.ams.server.model.TableOptimizeRuntime; import com.netease.arctic.ams.server.utils.JDBCSqlSessionFactoryProvider; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.io.CloseableIterable; @@ -73,7 +73,7 @@ public void testNoPartitionTableMajorOptimizeCommit() throws Exception { optimizeRuntime.setReportTime(System.currentTimeMillis()); if (resultFiles != null) { optimizeRuntime.setNewFileSize(resultFiles.get(0).fileSizeInBytes()); - optimizeRuntime.setTargetFiles(resultFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + optimizeRuntime.setTargetFiles(resultFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } List finalTargetFiles = optimizeRuntime.getTargetFiles(); optimizeRuntime.setTargetFiles(finalTargetFiles); @@ -142,7 +142,7 @@ public void testPartitionTableMajorOptimizeCommit() throws Exception { optimizeRuntime.setReportTime(System.currentTimeMillis()); if (resultFiles != null) { optimizeRuntime.setNewFileSize(resultFiles.get(0).fileSizeInBytes()); - optimizeRuntime.setTargetFiles(resultFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + optimizeRuntime.setTargetFiles(resultFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } List finalTargetFiles = optimizeRuntime.getTargetFiles(); optimizeRuntime.setTargetFiles(finalTargetFiles); diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestIcebergMinorOptimizeCommit.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestIcebergMinorOptimizeCommit.java index 1f3ca64de7..640b837230 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestIcebergMinorOptimizeCommit.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/optimize/TestIcebergMinorOptimizeCommit.java @@ -6,7 +6,7 @@ import com.netease.arctic.ams.server.model.TableOptimizeRuntime; import com.netease.arctic.ams.server.utils.JDBCSqlSessionFactoryProvider; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -18,7 +18,6 @@ import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; -import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; @@ -81,7 +80,7 @@ public void testNoPartitionTableMinorOptimizeCommit() throws Exception { optimizeRuntime.setReportTime(System.currentTimeMillis()); if (resultFiles != null) { optimizeRuntime.setNewFileSize(resultFiles.get(0).fileSizeInBytes()); - optimizeRuntime.setTargetFiles(resultFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + optimizeRuntime.setTargetFiles(resultFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } List finalTargetFiles = optimizeRuntime.getTargetFiles(); optimizeRuntime.setTargetFiles(finalTargetFiles); @@ -145,7 +144,7 @@ public void testPartitionTableMinorOptimizeCommit() throws Exception { optimizeRuntime.setReportTime(System.currentTimeMillis()); if (resultFiles != null) { optimizeRuntime.setNewFileSize(resultFiles.get(0).fileSizeInBytes()); - optimizeRuntime.setTargetFiles(resultFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + optimizeRuntime.setTargetFiles(resultFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } List finalTargetFiles = optimizeRuntime.getTargetFiles(); optimizeRuntime.setTargetFiles(finalTargetFiles); 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 14970de0fe..91d34484b3 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 @@ -30,7 +30,7 @@ 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 com.netease.arctic.utils.SerializationUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Snapshot; @@ -110,7 +110,7 @@ public void testMajorOptimizeCommit() throws Exception { 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())); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } // 1min optimizeRuntime.setCostTime(60 * 1000); 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 26d7c5d463..7ddcde7280 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 @@ -27,8 +27,8 @@ import com.netease.arctic.ams.server.util.DataFileInfoUtils; import com.netease.arctic.ams.server.utils.JDBCSqlSessionFactoryProvider; import com.netease.arctic.data.DataTreeNode; -import com.netease.arctic.utils.FileUtil; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; +import com.netease.arctic.utils.TableFileUtils; import com.netease.arctic.utils.TablePropertyUtil; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -121,7 +121,7 @@ public void testMinorOptimizeCommit() throws Exception { optimizeRuntime.setReportTime(System.currentTimeMillis()); if (targetFiles != null) { optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); - optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } List finalTargetFiles = optimizeRuntime.getTargetFiles(); finalTargetFiles.addAll(task.getInsertFiles()); @@ -201,7 +201,7 @@ public void testNoPartitionTableMinorOptimizeCommit() throws Exception { optimizeRuntime.setReportTime(System.currentTimeMillis()); if (targetFiles != null) { optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); - optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } List finalTargetFiles = optimizeRuntime.getTargetFiles(); finalTargetFiles.addAll(task.getInsertFiles()); @@ -323,7 +323,7 @@ public void testNoPartitionOnlyEqDeleteInChangeCommit() throws Exception { private Map> generateTargetFiles(List dataFiles) throws Exception { List deleteFiles = insertOptimizeTargetDeleteFiles(testKeyedTable, dataFiles, 5); return deleteFiles.stream().collect(Collectors.groupingBy(deleteFile -> { - DataTreeNode dataTreeNode = FileUtil.parseFileNodeFromFileName(deleteFile.path().toString()); + DataTreeNode dataTreeNode = TableFileUtils.parseFileNodeFromFileName(deleteFile.path().toString()); return new TreeNode(dataTreeNode.mask(), dataTreeNode.index()); })); } 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 index b056f75585..c110e3b9ca 100644 --- 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 @@ -26,7 +26,7 @@ import com.netease.arctic.table.BaseLocationKind; import com.netease.arctic.table.UnkeyedTable; import com.netease.arctic.table.WriteOperationKind; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -115,7 +115,7 @@ default Pair> insertBasePosDeleteFiles(ArcticTable ar List partitionFiles = dataFilePartitionMap.getValue(); Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() .collect(Collectors.groupingBy(dataFile -> - FileUtil.parseFileNodeFromFileName(dataFile.path().toString())))); + TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())))); for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { DataTreeNode key = nodeFilePartitionMap.getKey(); List nodeFiles = nodeFilePartitionMap.getValue(); @@ -157,7 +157,7 @@ default List insertOptimizeTargetDeleteFiles(ArcticTable arcticTable List partitionFiles = dataFilePartitionMap.getValue(); Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() .collect(Collectors.groupingBy(dataFile -> - FileUtil.parseFileNodeFromFileName(dataFile.path().toString())))); + TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())))); for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { DataTreeNode key = nodeFilePartitionMap.getKey(); List nodeFiles = nodeFilePartitionMap.getValue(); 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 index 281f6738b6..78e5e6f4ae 100644 --- 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 @@ -30,8 +30,8 @@ import com.netease.arctic.hive.table.SupportHive; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.utils.FileUtil; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; +import com.netease.arctic.utils.TableFileUtils; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.iceberg.ContentFile; @@ -95,7 +95,7 @@ public void testKeyedTableMajorOptimizeSupportHiveHasPosDeleteCommit() throws Ex 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())); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } // 1min optimizeRuntime.setCostTime(60 * 1000); @@ -165,7 +165,7 @@ public void testKeyedTableMajorOptimizeSupportHiveNoPosDeleteCommit() throws Exc 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())); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } // 1min optimizeRuntime.setCostTime(60 * 1000); @@ -247,7 +247,7 @@ public void testKeyedTableFullMajorOptimizeSupportHiveCommit() throws Exception 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())); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } // 1min optimizeRuntime.setCostTime(60 * 1000); @@ -308,8 +308,8 @@ public void testUnKeyedTableMajorOptimizeSupportHiveCommit() throws Exception { 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)); - DataTreeNode dataTreeNode = FileUtil.parseFileNodeFromFileName(baseFile.path().toString()); + ContentFile baseFile = SerializationUtils.toInternalTableFile(task.getBaseFiles().get(0)); + DataTreeNode dataTreeNode = TableFileUtils.parseFileNodeFromFileName(baseFile.path().toString()); TreeNode treeNode = new TreeNode(dataTreeNode.getMask(), dataTreeNode.getIndex()); List targetFiles = resultFiles.get(treeNode); optimizeRuntime.setPreparedTime(System.currentTimeMillis()); @@ -317,7 +317,7 @@ public void testUnKeyedTableMajorOptimizeSupportHiveCommit() throws Exception { optimizeRuntime.setReportTime(System.currentTimeMillis()); optimizeRuntime.setNewFileCnt(targetFiles.size()); optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); - optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); // 1min optimizeRuntime.setCostTime(60 * 1000); return new OptimizeTaskItem(task, optimizeRuntime); @@ -380,8 +380,8 @@ public void testUnKeyedTableFullMajorOptimizeSupportHiveCommit() throws Exceptio 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)); - DataTreeNode dataTreeNode = FileUtil.parseFileNodeFromFileName(baseFile.path().toString()); + ContentFile baseFile = SerializationUtils.toInternalTableFile(task.getBaseFiles().get(0)); + DataTreeNode dataTreeNode = TableFileUtils.parseFileNodeFromFileName(baseFile.path().toString()); TreeNode treeNode = new TreeNode(dataTreeNode.getMask(), dataTreeNode.getIndex()); List targetFiles = resultFiles.get(treeNode); optimizeRuntime.setPreparedTime(System.currentTimeMillis()); @@ -389,7 +389,7 @@ public void testUnKeyedTableFullMajorOptimizeSupportHiveCommit() throws Exceptio optimizeRuntime.setReportTime(System.currentTimeMillis()); optimizeRuntime.setNewFileCnt(targetFiles.size()); optimizeRuntime.setNewFileSize(targetFiles.get(0).fileSizeInBytes()); - optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtil::toByteBuffer).collect(Collectors.toList())); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); // 1min optimizeRuntime.setCostTime(60 * 1000); return new OptimizeTaskItem(task, optimizeRuntime); @@ -457,7 +457,7 @@ public void testUnPartitionTableMajorOptimizeSupportHiveCommit() throws Exceptio 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())); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } // 1min optimizeRuntime.setCostTime(60 * 1000); @@ -539,7 +539,7 @@ public void testUnPartitionTableFullMajorOptimizeSupportHiveCommit() throws Exce 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())); + optimizeRuntime.setTargetFiles(targetFiles.stream().map(SerializationUtils::toByteBuffer).collect(Collectors.toList())); } // 1min optimizeRuntime.setCostTime(60 * 1000); 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 index 4f0bd397f1..3d5fe25469 100644 --- 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 @@ -27,7 +27,7 @@ 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 com.netease.arctic.utils.TableFileUtils; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet; @@ -98,7 +98,7 @@ 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()); + String partitionLocation = TableFileUtils.getFileDir(dataFiles.get(0).path().toString()); testKeyedHiveTable.baseTable().updatePartitionProperties(null) .set(dataFiles.get(0).partition(), HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION, partitionLocation) .commit(); @@ -122,7 +122,7 @@ public void testSyncOnlyInHiveCreateByArctic() throws Exception { Assert.assertEquals(0, partitionProperty.size()); List dataFiles = insertTableHiveDataFiles(testKeyedHiveTable, 1); - String partitionLocation = FileUtil.getFileDir(dataFiles.get(0).path().toString()); + String partitionLocation = TableFileUtils.getFileDir(dataFiles.get(0).path().toString()); List partitionValues = HivePartitionUtil.partitionValuesAsList(dataFiles.get(0).partition(), testKeyedHiveTable.spec().partitionType()); ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> @@ -172,7 +172,7 @@ public void testSyncOnlyInHiveCreateNotByArctic() throws Exception { Assert.assertEquals(0, partitionProperty.size()); List dataFiles = insertTableHiveDataFiles(testKeyedHiveTable, 1); - String partitionLocation = FileUtil.getFileDir(dataFiles.get(0).path().toString()); + String partitionLocation = TableFileUtils.getFileDir(dataFiles.get(0).path().toString()); List partitionValues = HivePartitionUtil.partitionValuesAsList(dataFiles.get(0).partition(), testKeyedHiveTable.spec().partitionType()); ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> @@ -222,7 +222,7 @@ public void testSyncInBoth() throws Exception { Assert.assertEquals(0, partitionProperty.size()); List dataFiles = insertTableHiveDataFiles(testKeyedHiveTable, 1); - String partitionLocation = FileUtil.getFileDir(dataFiles.get(0).path().toString()); + String partitionLocation = TableFileUtils.getFileDir(dataFiles.get(0).path().toString()); List partitionValues = HivePartitionUtil.partitionValuesAsList(dataFiles.get(0).partition(), testKeyedHiveTable.spec().partitionType()); ((SupportHive) testKeyedHiveTable).getHMSClient().run(client -> @@ -259,7 +259,7 @@ public void testSyncInBoth() throws Exception { Assert.assertEquals(partitionLocation, hivePartition.getSd().getLocation()); List newDataFiles = insertTableHiveDataFiles(testKeyedHiveTable, 2); - String newPartitionLocation = FileUtil.getFileDir(newDataFiles.get(0).path().toString()); + String newPartitionLocation = TableFileUtils.getFileDir(newDataFiles.get(0).path().toString()); testKeyedHiveTable.baseTable().updatePartitionProperties(null) .set(newDataFiles.get(0).partition(), HiveTableProperties.PARTITION_PROPERTIES_KEY_HIVE_LOCATION, newPartitionLocation) .commit(); diff --git a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/util/DataFileInfoUtils.java b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/util/DataFileInfoUtils.java index 886f9078d1..3ac97070cc 100644 --- a/ams/ams-server/src/test/java/com/netease/arctic/ams/server/util/DataFileInfoUtils.java +++ b/ams/ams-server/src/test/java/com/netease/arctic/ams/server/util/DataFileInfoUtils.java @@ -25,7 +25,7 @@ import com.netease.arctic.data.DefaultKeyedFile; import com.netease.arctic.data.PrimaryKeyedFile; import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.PartitionField; @@ -67,7 +67,7 @@ public static DataFileInfo convertToDatafileInfo(DeleteFile deleteFile, Snapshot dataFileInfo.setSpecId(arcticTable.spec().specId()); dataFileInfo.setRecordCount(deleteFile.recordCount()); dataFileInfo.setType(DataFileType.POS_DELETE_FILE.name()); - DataTreeNode node = FileUtil.parseFileNodeFromFileName(deleteFile.path().toString()); + DataTreeNode node = TableFileUtils.parseFileNodeFromFileName(deleteFile.path().toString()); dataFileInfo.setIndex(node.getIndex()); dataFileInfo.setMask(node.getMask()); dataFileInfo.setCommitTime(snapshot.timestampMillis()); diff --git a/core/pom.xml b/core/pom.xml index 04d0b0dcce..a3c27cb588 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -41,6 +41,24 @@ ${project.version} + + org.rocksdb + rocksdbjni + 5.17.2 + + + + com.esotericsoftware.kryo + kryo + 2.24.0 + + + + org.openjdk.jol + jol-core + 0.16 + + org.apache.iceberg iceberg-core diff --git a/core/src/main/java/com/netease/arctic/ArcticIOException.java b/core/src/main/java/com/netease/arctic/ArcticIOException.java new file mode 100644 index 0000000000..492a5e55ff --- /dev/null +++ b/core/src/main/java/com/netease/arctic/ArcticIOException.java @@ -0,0 +1,29 @@ +package com.netease.arctic; + +/** + *

+ * Exception thrown for table IO-related failures. + *

+ */ +public class ArcticIOException extends RuntimeException { + + private Throwable ioException; + + public ArcticIOException(String msg, Throwable t) { + super(msg, t); + this.ioException = t; + } + + public ArcticIOException(Throwable t) { + super(t.getMessage(), t); + this.ioException = t; + } + + public ArcticIOException(String message) { + super(message); + } + + public Throwable getInternalError() { + return ioException; + } +} diff --git a/core/src/main/java/com/netease/arctic/catalog/BaseArcticCatalog.java b/core/src/main/java/com/netease/arctic/catalog/BaseArcticCatalog.java index a5be59bdf8..a0a30681c0 100644 --- a/core/src/main/java/com/netease/arctic/catalog/BaseArcticCatalog.java +++ b/core/src/main/java/com/netease/arctic/catalog/BaseArcticCatalog.java @@ -45,7 +45,6 @@ import com.netease.arctic.utils.CatalogUtil; import com.netease.arctic.utils.CompatiblePropertyUtil; import com.netease.arctic.utils.ConvertStructUtil; -import com.netease.arctic.utils.TablePropertyUtil; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.Path; import org.apache.iceberg.PartitionSpec; diff --git a/core/src/main/java/com/netease/arctic/data/ChangedLsn.java b/core/src/main/java/com/netease/arctic/data/ChangedLsn.java index e586166477..11e3c94aa3 100644 --- a/core/src/main/java/com/netease/arctic/data/ChangedLsn.java +++ b/core/src/main/java/com/netease/arctic/data/ChangedLsn.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.Objects; + /** * Global row sequence number. *

@@ -40,6 +41,25 @@ public static ChangedLsn of(long transactionId, long fileOffset) { return new ChangedLsn(transactionId, fileOffset); } + public static ChangedLsn of(byte[] bytes) { + return of(((long) bytes[15] << 56) | + ((long) bytes[14] & 0xff) << 48 | + ((long) bytes[13] & 0xff) << 40 | + ((long) bytes[12] & 0xff) << 32 | + ((long) bytes[11] & 0xff) << 24 | + ((long) bytes[10] & 0xff) << 16 | + ((long) bytes[9] & 0xff) << 8 | + ((long) bytes[8] & 0xff), + ((long) bytes[7] << 56) | + ((long) bytes[6] & 0xff) << 48 | + ((long) bytes[5] & 0xff) << 40 | + ((long) bytes[4] & 0xff) << 32 | + ((long) bytes[3] & 0xff) << 24 | + ((long) bytes[2] & 0xff) << 16 | + ((long) bytes[1] & 0xff) << 8 | + ((long) bytes[0] & 0xff)); + } + private ChangedLsn(long transactionId, long fileOffset) { this.transactionId = transactionId; this.fileOffset = fileOffset; @@ -77,7 +97,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) return false; ChangedLsn recordLsn = (ChangedLsn) o; return transactionId == recordLsn.transactionId && - fileOffset == recordLsn.fileOffset; + fileOffset == recordLsn.fileOffset; } @Override @@ -88,6 +108,27 @@ public int hashCode() { @Override public String toString() { return new StringBuilder("RecordLsn(").append(transactionId) - .append(", ").append(fileOffset).append(")").toString(); + .append(", ").append(fileOffset).append(")").toString(); + } + + public byte[] toBytes() { + return new byte[] { + (byte) transactionId, + (byte) (transactionId >> 8), + (byte) (transactionId >> 16), + (byte) (transactionId >> 24), + (byte) (transactionId >> 32), + (byte) (transactionId >> 40), + (byte) (transactionId >> 48), + (byte) (transactionId >> 56), + (byte) fileOffset, + (byte) (fileOffset >> 8), + (byte) (fileOffset >> 16), + (byte) (fileOffset >> 24), + (byte) (fileOffset >> 32), + (byte) (fileOffset >> 40), + (byte) (fileOffset >> 48), + (byte) (fileOffset >> 56) + }; } } diff --git a/core/src/main/java/com/netease/arctic/data/DefaultKeyedFile.java b/core/src/main/java/com/netease/arctic/data/DefaultKeyedFile.java index c71f111e37..6b52aa733a 100644 --- a/core/src/main/java/com/netease/arctic/data/DefaultKeyedFile.java +++ b/core/src/main/java/com/netease/arctic/data/DefaultKeyedFile.java @@ -18,13 +18,10 @@ package com.netease.arctic.data; -import com.netease.arctic.table.MetadataColumns; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.StructLike; -import org.apache.iceberg.types.Conversions; -import org.apache.iceberg.types.Types; import java.io.Serializable; import java.nio.ByteBuffer; @@ -47,7 +44,8 @@ public DefaultKeyedFile(DataFile internalFile) { } private void parse() { - this.meta = FileUtil.parseFileMetaFromFileName(FileUtil.getFileName(internalFile.path().toString())); + this.meta = TableFileUtils.parseFileMetaFromFileName(TableFileUtils + .getFileName(internalFile.path().toString())); } @Override diff --git a/core/src/main/java/com/netease/arctic/iceberg/CombinedDeleteFilter.java b/core/src/main/java/com/netease/arctic/iceberg/CombinedDeleteFilter.java index 0c4d0ef29f..d6a38d4ac2 100644 --- a/core/src/main/java/com/netease/arctic/iceberg/CombinedDeleteFilter.java +++ b/core/src/main/java/com/netease/arctic/iceberg/CombinedDeleteFilter.java @@ -20,10 +20,10 @@ import com.netease.arctic.data.IcebergContentFile; import com.netease.arctic.iceberg.optimize.InternalRecordWrapper; -import com.netease.arctic.iceberg.optimize.StructLikeMap; import com.netease.arctic.iceberg.optimize.StructProjection; import com.netease.arctic.io.ArcticFileIO; import com.netease.arctic.scan.CombinedIcebergScanTask; +import com.netease.arctic.utils.map.StructLikeMemoryMap; import org.apache.iceberg.Accessor; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.MetadataColumns; @@ -188,7 +188,7 @@ private Predicate applyEqDeletes() { InternalRecordWrapper internalRecordWrapper = new InternalRecordWrapper(deleteSchema.asStruct()); - StructLikeMap structLikeMap = StructLikeMap.create(pkSchema.asStruct()); + StructLikeMemoryMap structLikeMap = StructLikeMemoryMap.create(pkSchema.asStruct()); //init map try (CloseableIterable deletes = deleteRecords) { Iterator it = getArcticFileIo() == null ? deletes.iterator() diff --git a/core/src/main/java/com/netease/arctic/iceberg/optimize/DeleteFilter.java b/core/src/main/java/com/netease/arctic/iceberg/optimize/DeleteFilter.java index 66832d5a97..3c5b5b4562 100644 --- a/core/src/main/java/com/netease/arctic/iceberg/optimize/DeleteFilter.java +++ b/core/src/main/java/com/netease/arctic/iceberg/optimize/DeleteFilter.java @@ -18,6 +18,7 @@ package com.netease.arctic.iceberg.optimize; +import com.netease.arctic.utils.StructLikeSet; import org.apache.iceberg.Accessor; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; diff --git a/core/src/main/java/com/netease/arctic/iceberg/optimize/Deletes.java b/core/src/main/java/com/netease/arctic/iceberg/optimize/Deletes.java index e69f3ab082..5360b218ac 100644 --- a/core/src/main/java/com/netease/arctic/iceberg/optimize/Deletes.java +++ b/core/src/main/java/com/netease/arctic/iceberg/optimize/Deletes.java @@ -18,13 +18,13 @@ package com.netease.arctic.iceberg.optimize; +import com.netease.arctic.utils.StructLikeSet; import org.apache.iceberg.Accessor; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Comparators; @@ -66,8 +66,10 @@ public static CloseableIterable filter(CloseableIterable rows, Functio public static StructLikeSet toEqualitySet(CloseableIterable eqDeletes, Types.StructType eqType) { try (CloseableIterable deletes = eqDeletes) { - StructLikeSet deleteSet = StructLikeSet.create(eqType); - Iterables.addAll(deleteSet, deletes); + StructLikeSet deleteSet = StructLikeSet.createMemorySet(eqType); + for (StructLike delete : deletes) { + deleteSet.add(delete); + } return deleteSet; } catch (IOException e) { throw new UncheckedIOException("Failed to close equality delete source", e); diff --git a/core/src/main/java/com/netease/arctic/iceberg/optimize/StructLikeMap.java b/core/src/main/java/com/netease/arctic/iceberg/optimize/StructLikeMap.java deleted file mode 100644 index 95ec52b99c..0000000000 --- a/core/src/main/java/com/netease/arctic/iceberg/optimize/StructLikeMap.java +++ /dev/null @@ -1,183 +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.iceberg.optimize; - -import org.apache.iceberg.StructLike; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.StructLikeSet; - -import java.util.AbstractMap; -import java.util.Collection; -import java.util.Map; -import java.util.Objects; -import java.util.Set; - -/** - * Copy form iceberg {@link org.apache.iceberg.util.StructLikeMap}. Make using StructLikeWrapper more cheap - */ -public class StructLikeMap extends AbstractMap implements Map { - - public static StructLikeMap create(Types.StructType type) { - return new StructLikeMap<>(type); - } - - private final Types.StructType type; - private final Map wrapperMap; - private final ThreadLocal wrappers; - private final StructLikeWrapperFactory structLikeWrapperFactory; - - private StructLikeMap(Types.StructType type) { - this.type = type; - this.wrapperMap = Maps.newHashMap(); - this.structLikeWrapperFactory = new StructLikeWrapperFactory(type); - this.wrappers = ThreadLocal.withInitial(() -> structLikeWrapperFactory.create()); - } - - @Override - public int size() { - return wrapperMap.size(); - } - - @Override - public boolean isEmpty() { - return wrapperMap.isEmpty(); - } - - @Override - public boolean containsKey(Object key) { - if (key instanceof StructLike || key == null) { - StructLikeWrapper wrapper = wrappers.get(); - boolean result = wrapperMap.containsKey(wrapper.set((StructLike) key)); - wrapper.set(null); // don't hold a reference to the key. - return result; - } - return false; - } - - @Override - public boolean containsValue(Object value) { - return wrapperMap.containsValue(value); - } - - @Override - public T get(Object key) { - if (key instanceof StructLike || key == null) { - StructLikeWrapper wrapper = wrappers.get(); - T value = wrapperMap.get(wrapper.set((StructLike) key)); - wrapper.set(null); // don't hold a reference to the key. - return value; - } - return null; - } - - @Override - public T put(StructLike key, T value) { - return wrapperMap.put(structLikeWrapperFactory.create().set(key), value); - } - - @Override - public T remove(Object key) { - if (key instanceof StructLike || key == null) { - StructLikeWrapper wrapper = wrappers.get(); - T value = wrapperMap.remove(wrapper.set((StructLike) key)); - wrapper.set(null); // don't hold a reference to the key. - return value; - } - return null; - } - - @Override - public void clear() { - wrapperMap.clear(); - } - - @Override - public Set keySet() { - org.apache.iceberg.util.StructLikeSet keySet = StructLikeSet.create(type); - for (StructLikeWrapper wrapper : wrapperMap.keySet()) { - keySet.add(wrapper.get()); - } - return keySet; - } - - @Override - public Collection values() { - return wrapperMap.values(); - } - - @Override - public Set> entrySet() { - Set> entrySet = Sets.newHashSet(); - for (Entry entry : wrapperMap.entrySet()) { - entrySet.add(new StructLikeEntry<>(entry)); - } - return entrySet; - } - - private static class StructLikeEntry implements Entry { - - private Map.Entry inner; - - private StructLikeEntry(Map.Entry inner) { - this.inner = inner; - } - - @Override - public StructLike getKey() { - return inner.getKey().get(); - } - - @Override - public R getValue() { - return inner.getValue(); - } - - @Override - public int hashCode() { - int hashCode = getKey().hashCode(); - if (getValue() != null) { - hashCode ^= getValue().hashCode(); - } - return hashCode; - } - - @Override - @SuppressWarnings("unchecked") - public boolean equals(Object o) { - if (this == o) { - return true; - } else if (!(o instanceof StructLikeEntry)) { - return false; - } else { - StructLikeEntry - that = (StructLikeEntry) o; - return Objects.equals(getKey(), that.getKey()) && - Objects.equals(getValue(), that.getValue()); - } - } - - @Override - public R setValue(R value) { - throw new UnsupportedOperationException("Does not support setValue."); - } - } -} - diff --git a/core/src/main/java/com/netease/arctic/iceberg/optimize/StructLikeSet.java b/core/src/main/java/com/netease/arctic/iceberg/optimize/StructLikeSet.java deleted file mode 100644 index c2679da0bf..0000000000 --- a/core/src/main/java/com/netease/arctic/iceberg/optimize/StructLikeSet.java +++ /dev/null @@ -1,187 +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.iceberg.optimize; - -import org.apache.iceberg.StructLike; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Iterators; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Types; - -import java.util.AbstractSet; -import java.util.Collection; -import java.util.Iterator; -import java.util.Objects; -import java.util.Set; - -/** - * Copy form iceberg {@link org.apache.iceberg.util.StructLikeSet}. Make using StructLikeWrapper more cheap - */ -public class StructLikeSet extends AbstractSet implements Set { - public static StructLikeSet create(Types.StructType type) { - return new StructLikeSet(type); - } - - private final Types.StructType type; - private final Set wrapperSet; - private final ThreadLocal wrappers; - - private StructLikeWrapperFactory structLikeWrapperFactory; - - private StructLikeSet(Types.StructType type) { - this.type = type; - this.wrapperSet = Sets.newHashSet(); - this.structLikeWrapperFactory = new StructLikeWrapperFactory(type); - this.wrappers = ThreadLocal.withInitial(() -> structLikeWrapperFactory.create()); - } - - @Override - public int size() { - return wrapperSet.size(); - } - - @Override - public boolean isEmpty() { - return wrapperSet.isEmpty(); - } - - @Override - public boolean contains(Object obj) { - if (obj instanceof StructLike || obj == null) { - StructLikeWrapper wrapper = wrappers.get(); - boolean result = wrapperSet.contains(wrapper.set((StructLike) obj)); - wrapper.set(null); // don't hold a reference to the value - return result; - } - return false; - } - - @Override - public Iterator iterator() { - return Iterators.transform(wrapperSet.iterator(), StructLikeWrapper::get); - } - - @Override - public Object[] toArray() { - return Iterators.toArray(iterator(), StructLike.class); - } - - @Override - @SuppressWarnings("unchecked") - public T[] toArray(T[] destArray) { - int size = wrapperSet.size(); - if (destArray.length < size) { - return (T[]) toArray(); - } - - Iterator iter = iterator(); - int ind = 0; - while (iter.hasNext()) { - destArray[ind] = (T) iter.next(); - ind += 1; - } - - if (destArray.length > size) { - destArray[size] = null; - } - - return destArray; - } - - @Override - public boolean add(StructLike struct) { - return wrapperSet.add(structLikeWrapperFactory.create().set(struct)); - } - - @Override - public boolean remove(Object obj) { - if (obj instanceof StructLike || obj == null) { - StructLikeWrapper wrapper = wrappers.get(); - boolean result = wrapperSet.remove(wrapper.set((StructLike) obj)); - wrapper.set(null); // don't hold a reference to the value - return result; - } - return false; - } - - @Override - public boolean containsAll(Collection objects) { - if (objects != null) { - return Iterables.all(objects, this::contains); - } - return false; - } - - @Override - public boolean addAll(Collection structs) { - if (structs != null) { - return Iterables.addAll(wrapperSet, - Iterables.transform(structs, struct -> structLikeWrapperFactory.create().set(struct))); - } - return false; - } - - @Override - public boolean retainAll(Collection objects) { - throw new UnsupportedOperationException("retailAll is not supported"); - } - - @Override - public boolean removeAll(Collection objects) { - boolean changed = false; - if (objects != null) { - for (Object object : objects) { - changed |= remove(object); - } - } - return changed; - } - - @Override - public void clear() { - wrapperSet.clear(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - StructLikeSet that = (StructLikeSet) o; - if (!type.equals(that.type)) { - return false; - } - - if (wrapperSet.size() != that.wrapperSet.size()) { - return false; - } - - return containsAll(that); - } - - @Override - public int hashCode() { - return Objects.hash(type) + wrapperSet.stream().mapToInt(StructLikeWrapper::hashCode).sum(); - } -} diff --git a/core/src/main/java/com/netease/arctic/iceberg/optimize/StructLikeWrapper.java b/core/src/main/java/com/netease/arctic/iceberg/optimize/StructLikeWrapper.java index 52ba70305b..8b4719bb67 100644 --- a/core/src/main/java/com/netease/arctic/iceberg/optimize/StructLikeWrapper.java +++ b/core/src/main/java/com/netease/arctic/iceberg/optimize/StructLikeWrapper.java @@ -23,21 +23,22 @@ import org.apache.iceberg.types.JavaHash; import org.apache.iceberg.types.Types; +import java.io.Serializable; import java.util.Comparator; /** * Copy from iceberg {@link org.apache.iceberg.util.StructLikeWrapper}, * Add new constructor to create StructLikeWrapper more cheap */ -public class StructLikeWrapper { +public class StructLikeWrapper implements Serializable { public static StructLikeWrapper forType(Types.StructType struct) { return new StructLikeWrapper(struct); } - private final Comparator comparator; - private final JavaHash structHash; - private Integer hashCode; + private final transient Comparator comparator; + private final transient JavaHash structHash; + private transient Integer hashCode; private StructLike struct; private StructLikeWrapper(Types.StructType type) { diff --git a/core/src/main/java/com/netease/arctic/io/reader/ArcticDeleteFilter.java b/core/src/main/java/com/netease/arctic/io/reader/ArcticDeleteFilter.java index 5108e95201..0d124c2e24 100644 --- a/core/src/main/java/com/netease/arctic/io/reader/ArcticDeleteFilter.java +++ b/core/src/main/java/com/netease/arctic/io/reader/ArcticDeleteFilter.java @@ -22,7 +22,6 @@ import com.netease.arctic.data.DataTreeNode; import com.netease.arctic.data.PrimaryKeyedFile; import com.netease.arctic.iceberg.optimize.InternalRecordWrapper; -import com.netease.arctic.iceberg.optimize.StructLikeMap; import com.netease.arctic.iceberg.optimize.StructProjection; import com.netease.arctic.io.ArcticFileIO; import com.netease.arctic.scan.ArcticFileScanTask; @@ -30,6 +29,7 @@ import com.netease.arctic.table.MetadataColumns; import com.netease.arctic.table.PrimaryKeySpec; import com.netease.arctic.utils.NodeFilter; +import com.netease.arctic.utils.map.StructLikeMemoryMap; import org.apache.iceberg.Accessor; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Schema; @@ -70,13 +70,13 @@ public abstract class ArcticDeleteFilter { private static final Schema POS_DELETE_SCHEMA = new Schema( - org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH, - org.apache.iceberg.MetadataColumns.DELETE_FILE_POS); + org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH, + org.apache.iceberg.MetadataColumns.DELETE_FILE_POS); private static final Accessor FILENAME_ACCESSOR = POS_DELETE_SCHEMA - .accessorForField(org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH.fieldId()); + .accessorForField(org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH.fieldId()); private static final Accessor POSITION_ACCESSOR = POS_DELETE_SCHEMA - .accessorForField(org.apache.iceberg.MetadataColumns.DELETE_FILE_POS.fieldId()); + .accessorForField(org.apache.iceberg.MetadataColumns.DELETE_FILE_POS.fieldId()); private final Set eqDeletes; private final List posDeletes; @@ -98,19 +98,19 @@ public abstract class ArcticDeleteFilter { private Set currentPosSet; protected ArcticDeleteFilter( - KeyedTableScanTask keyedTableScanTask, Schema tableSchema, - Schema requestedSchema, PrimaryKeySpec primaryKeySpec) { + KeyedTableScanTask keyedTableScanTask, Schema tableSchema, + Schema requestedSchema, PrimaryKeySpec primaryKeySpec) { this(keyedTableScanTask, tableSchema, requestedSchema, primaryKeySpec, null); } protected ArcticDeleteFilter( - KeyedTableScanTask keyedTableScanTask, Schema tableSchema, - Schema requestedSchema, PrimaryKeySpec primaryKeySpec, - Set sourceNodes) { + KeyedTableScanTask keyedTableScanTask, Schema tableSchema, + Schema requestedSchema, PrimaryKeySpec primaryKeySpec, + Set sourceNodes) { this.eqDeletes = keyedTableScanTask.arcticEquityDeletes().stream() - .map(ArcticFileScanTask::file) - .sorted(Comparator.comparingLong(PrimaryKeyedFile::transactionId)) - .collect(Collectors.toSet()); + .map(ArcticFileScanTask::file) + .sorted(Comparator.comparingLong(PrimaryKeyedFile::transactionId)) + .collect(Collectors.toSet()); Map map = new HashMap<>(); for (ArcticFileScanTask arcticFileScanTask : keyedTableScanTask.dataTasks()) { @@ -121,10 +121,10 @@ protected ArcticDeleteFilter( this.posDeletes = map.values().stream().collect(Collectors.toList()); this.pathSets = - keyedTableScanTask.dataTasks().stream().map(s -> s.file().path().toString()).collect(Collectors.toSet()); + keyedTableScanTask.dataTasks().stream().map(s -> s.file().path().toString()).collect(Collectors.toSet()); this.primaryKeyId = primaryKeySpec.primaryKeyStruct().fields().stream() - .map(Types.NestedField::fieldId).collect(Collectors.toSet()); + .map(Types.NestedField::fieldId).collect(Collectors.toSet()); this.requiredSchema = fileProjection(tableSchema, requestedSchema, eqDeletes, posDeletes); Set deleteIds = Sets.newHashSet(primaryKeyId); deleteIds.add(MetadataColumns.TRANSACTION_ID_FILED.fieldId()); @@ -213,24 +213,24 @@ private Predicate applyEqDeletes() { StructProjection dataPKProjectRow = StructProjection.create(requiredSchema, pkSchema); Iterable> deleteRecords = Iterables.transform( - eqDeletes, - this::openDeletes); + eqDeletes, + this::openDeletes); // copy the delete records because they will be held in a map CloseableIterable records = CloseableIterable.transform( - CloseableIterable.concat(deleteRecords), Record::copy); + CloseableIterable.concat(deleteRecords), Record::copy); if (deleteNodeFilter != null) { records = deleteNodeFilter.filter(records); } CloseableIterable structLikeIterable = CloseableIterable.transform( - records, record -> new InternalRecordWrapper(deleteSchema.asStruct()).wrap(record)); + records, record -> new InternalRecordWrapper(deleteSchema.asStruct()).wrap(record)); - StructLikeMap structLikeMap = StructLikeMap.create(pkSchema.asStruct()); + StructLikeMemoryMap structLikeMap = StructLikeMemoryMap.create(pkSchema.asStruct()); //init map try (CloseableIterable deletes = structLikeIterable) { Iterator it = getArcticFileIo() == null ? deletes.iterator() - : getArcticFileIo().doAs(deletes::iterator); + : getArcticFileIo().doAs(deletes::iterator); while (it.hasNext()) { StructLike structLike = it.next(); StructLike deletePK = deletePKProjectRow.copyWrap(structLike); @@ -283,24 +283,25 @@ private CloseableIterable openDeletes(PrimaryKeyedFile deleteFile) { switch (deleteFile.format()) { case AVRO: return Avro.read(input) - .project(deleteSchema) - .reuseContainers() - .createReaderFunc(fileSchema -> DataReader.create(deleteSchema, fileSchema, idToConstant)) - .build(); + .project(deleteSchema) + .reuseContainers() + .createReaderFunc(fileSchema -> DataReader.create(deleteSchema, fileSchema, idToConstant)) + .build(); case PARQUET: Parquet.ReadBuilder builder = Parquet.read(input) - .project(deleteSchema) - .reuseContainers() - .createReaderFunc(fileSchema -> - GenericParquetReaders.buildReader(deleteSchema, fileSchema, idToConstant)); + .project(deleteSchema) + .reuseContainers() + .createReaderFunc(fileSchema -> + GenericParquetReaders.buildReader(deleteSchema, fileSchema, idToConstant)); return builder.build(); case ORC: default: throw new UnsupportedOperationException(String.format( - "Cannot read deletes, %s is not a supported format: %s", deleteFile.format().name(), deleteFile.path())); + "Cannot read deletes, %s is not a supported format: %s", + deleteFile.format().name(), deleteFile.path())); } } @@ -365,29 +366,30 @@ private CloseableIterable openPositionDeletes(DeleteFile deleteFile, Sch switch (deleteFile.format()) { case AVRO: return Avro.read(input) - .project(deleteSchema) - .reuseContainers() - .createReaderFunc(DataReader::create) - .build(); + .project(deleteSchema) + .reuseContainers() + .createReaderFunc(DataReader::create) + .build(); case PARQUET: Parquet.ReadBuilder builder = Parquet.read(input) - .project(deleteSchema) - .reuseContainers() - .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(deleteSchema, fileSchema)); + .project(deleteSchema) + .reuseContainers() + .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(deleteSchema, fileSchema)); return builder.build(); case ORC: default: throw new UnsupportedOperationException(String.format( - "Cannot read deletes, %s is not a supported format: %s", deleteFile.format().name(), deleteFile.path())); + "Cannot read deletes, %s is not a supported format: %s", + deleteFile.format().name(), deleteFile.path())); } } private Schema fileProjection( - Schema tableSchema, Schema requestedSchema, Collection eqDeletes, - Collection posDeletes) { + Schema tableSchema, Schema requestedSchema, Collection eqDeletes, + Collection posDeletes) { if (eqDeletes.isEmpty() && posDeletes.isEmpty()) { return requestedSchema; } @@ -405,15 +407,15 @@ private Schema fileProjection( } Set missingIds = Sets.newLinkedHashSet( - Sets.difference(requiredIds, TypeUtil.getProjectedIds(requestedSchema))); + Sets.difference(requiredIds, TypeUtil.getProjectedIds(requestedSchema))); // TODO: support adding nested columns. this will currently fail when finding nested columns to add List columns = Lists.newArrayList(requestedSchema.columns()); for (int fieldId : missingIds) { if (fieldId == org.apache.iceberg.MetadataColumns.ROW_POSITION.fieldId() || - fieldId == org.apache.iceberg.MetadataColumns.FILE_PATH.fieldId() || - fieldId == MetadataColumns.TRANSACTION_ID_FILED.fieldId() || - fieldId == MetadataColumns.FILE_OFFSET_FILED.fieldId() + fieldId == org.apache.iceberg.MetadataColumns.FILE_PATH.fieldId() || + fieldId == MetadataColumns.TRANSACTION_ID_FILED.fieldId() || + fieldId == MetadataColumns.FILE_OFFSET_FILED.fieldId() ) { continue; } diff --git a/core/src/main/java/com/netease/arctic/io/reader/GenericCombinedIcebergDataReader.java b/core/src/main/java/com/netease/arctic/io/reader/GenericCombinedIcebergDataReader.java index fff57ed2f0..c62777f6cc 100644 --- a/core/src/main/java/com/netease/arctic/io/reader/GenericCombinedIcebergDataReader.java +++ b/core/src/main/java/com/netease/arctic/io/reader/GenericCombinedIcebergDataReader.java @@ -18,13 +18,11 @@ package com.netease.arctic.io.reader; -import com.netease.arctic.data.DataTreeNode; import com.netease.arctic.data.IcebergContentFile; import com.netease.arctic.iceberg.CombinedDeleteFilter; import com.netease.arctic.iceberg.optimize.InternalRecordWrapper; import com.netease.arctic.io.ArcticFileIO; import com.netease.arctic.scan.CombinedIcebergScanTask; -import com.netease.arctic.table.PrimaryKeySpec; import org.apache.iceberg.DataFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; diff --git a/core/src/main/java/com/netease/arctic/io/writer/IcebergFanoutPosDeleteWriter.java b/core/src/main/java/com/netease/arctic/io/writer/IcebergFanoutPosDeleteWriter.java index 56f2e1ae0c..e7ce3fea64 100644 --- a/core/src/main/java/com/netease/arctic/io/writer/IcebergFanoutPosDeleteWriter.java +++ b/core/src/main/java/com/netease/arctic/io/writer/IcebergFanoutPosDeleteWriter.java @@ -18,7 +18,7 @@ package com.netease.arctic.io.writer; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.StructLike; @@ -146,12 +146,12 @@ private void flushDeletes() { return; } posDeletes.sort(Comparator.comparingLong(PosRow::pos)); - String fileName = FileUtil.getFileName(filePath.get().toString()); + String fileName = TableFileUtils.getFileName(filePath.get().toString()); FileFormat fileFormat = FileFormat.fromFileName(fileName); if (fileFormat != null) { fileName = fileName.substring(0, fileName.length() - fileFormat.name().length() - 1); } - String fileDir = FileUtil.getFileDir(filePath.get().toString()); + String fileDir = TableFileUtils.getFileDir(filePath.get().toString()); String deleteFilePath = format.addExtension(String.format("%s/%s-delete-%s", fileDir, fileName, fileNameSuffix)); EncryptedOutputFile outputFile = encryptionManager.encrypt(fileIO.newOutputFile(deleteFilePath)); diff --git a/core/src/main/java/com/netease/arctic/scan/BaseArcticFileScanTask.java b/core/src/main/java/com/netease/arctic/scan/BaseArcticFileScanTask.java index 6383c00d9e..c1b93b0915 100644 --- a/core/src/main/java/com/netease/arctic/scan/BaseArcticFileScanTask.java +++ b/core/src/main/java/com/netease/arctic/scan/BaseArcticFileScanTask.java @@ -20,7 +20,7 @@ import com.netease.arctic.data.DefaultKeyedFile; import com.netease.arctic.data.PrimaryKeyedFile; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; @@ -56,7 +56,7 @@ public BaseArcticFileScanTask( this.baseFile = baseFile; this.posDeleteFiles = posDeleteFiles == null ? Collections.emptyList() : posDeleteFiles.stream().filter(s -> { - DefaultKeyedFile.FileMeta fileMeta = FileUtil.parseFileMetaFromFileName(s.path().toString()); + DefaultKeyedFile.FileMeta fileMeta = TableFileUtils.parseFileMetaFromFileName(s.path().toString()); return fileMeta.node().index() == baseFile.node().index() && fileMeta.node().mask() == baseFile.node().mask(); }).collect(Collectors.toList()); diff --git a/core/src/main/java/com/netease/arctic/scan/BaseChangeTableIncrementalScan.java b/core/src/main/java/com/netease/arctic/scan/BaseChangeTableIncrementalScan.java index e8f55c9540..e48e2307a7 100644 --- a/core/src/main/java/com/netease/arctic/scan/BaseChangeTableIncrementalScan.java +++ b/core/src/main/java/com/netease/arctic/scan/BaseChangeTableIncrementalScan.java @@ -22,7 +22,7 @@ import com.netease.arctic.data.DefaultKeyedFile; import com.netease.arctic.table.ChangeTable; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.Snapshot; @@ -88,7 +88,7 @@ public CloseableIterable planTasks(PartitionDataFilter shoul Boolean shouldKeep = shouldKeepFile.shouldKeep(partition, sequenceNumber); if (shouldKeep == null) { String filePath = entry.getFile().path().toString(); - return shouldKeepFileWithLegacyTxId.shouldKeep(partition, FileUtil.parseFileTidFromFileName(filePath)); + return shouldKeepFileWithLegacyTxId.shouldKeep(partition, TableFileUtils.parseFileTidFromFileName(filePath)); } else { return shouldKeep; } diff --git a/core/src/main/java/com/netease/arctic/utils/CatalogUtil.java b/core/src/main/java/com/netease/arctic/utils/CatalogUtil.java index 6906d8f8f4..180dd118d3 100644 --- a/core/src/main/java/com/netease/arctic/utils/CatalogUtil.java +++ b/core/src/main/java/com/netease/arctic/utils/CatalogUtil.java @@ -41,7 +41,6 @@ import org.slf4j.LoggerFactory; import java.util.Arrays; -import java.util.HashMap; import java.util.Locale; import java.util.Map; import java.util.Set; diff --git a/core/src/main/java/com/netease/arctic/utils/ConvertStructUtil.java b/core/src/main/java/com/netease/arctic/utils/ConvertStructUtil.java index aed644e7ba..b635e784e5 100644 --- a/core/src/main/java/com/netease/arctic/utils/ConvertStructUtil.java +++ b/core/src/main/java/com/netease/arctic/utils/ConvertStructUtil.java @@ -72,13 +72,13 @@ public static com.netease.arctic.ams.api.DataFile convertToAmsDatafile( */ FileContent content = dataFile.content(); if (content == FileContent.DATA) { - DefaultKeyedFile.FileMeta fileMeta = FileUtil.parseFileMetaFromFileName(dataFile.path().toString()); + DefaultKeyedFile.FileMeta fileMeta = TableFileUtils.parseFileMetaFromFileName(dataFile.path().toString()); validateArcticFileType(content, dataFile.path().toString(), fileMeta.type()); amsDataFile.setFileType(fileMeta.type().name()); amsDataFile.setIndex(fileMeta.node().index()); amsDataFile.setMask(fileMeta.node().mask()); } else if (content == FileContent.POSITION_DELETES) { - DefaultKeyedFile.FileMeta fileMeta = FileUtil.parseFileMetaFromFileName(dataFile.path().toString()); + DefaultKeyedFile.FileMeta fileMeta = TableFileUtils.parseFileMetaFromFileName(dataFile.path().toString()); amsDataFile.setFileType(DataFileType.POS_DELETE_FILE.name()); if (fileMeta.type() == DataFileType.POS_DELETE_FILE || fileMeta.type() == DataFileType.BASE_FILE) { amsDataFile.setIndex(fileMeta.node().index()); diff --git a/core/src/main/java/com/netease/arctic/utils/LocalFileUtils.java b/core/src/main/java/com/netease/arctic/utils/LocalFileUtils.java new file mode 100644 index 0000000000..f0cf0ae848 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/utils/LocalFileUtils.java @@ -0,0 +1,30 @@ +package com.netease.arctic.utils; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Comparator; + +public class LocalFileUtils { + + public static void deleteDirectory(File directory) throws IOException { + if (directory.exists()) { + Files.walk(directory.toPath()).sorted(Comparator.reverseOrder()) + .map(java.nio.file.Path::toFile).forEach(File::delete); + directory.delete(); + if (directory.exists()) { + throw new IOException("Unable to delete directory " + directory); + } + } + } + + public static void mkdir(File directory) throws IOException { + if (!directory.exists()) { + directory.mkdirs(); + } + + if (!directory.isDirectory()) { + throw new IOException("Unable to create :" + directory); + } + } +} diff --git a/core/src/main/java/com/netease/arctic/utils/SerializationUtil.java b/core/src/main/java/com/netease/arctic/utils/SerializationUtil.java deleted file mode 100644 index 50eb59de93..0000000000 --- a/core/src/main/java/com/netease/arctic/utils/SerializationUtil.java +++ /dev/null @@ -1,86 +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.utils; - -import com.netease.arctic.data.IcebergContentFile; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.util.ByteBuffers; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.nio.ByteBuffer; - -public class SerializationUtil { - - public static ByteBuffer toByteBuffer(Object obj) { - try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) { - try (ObjectOutputStream oos = new ObjectOutputStream(bos)) { - oos.writeObject(obj); - oos.flush(); - return ByteBuffer.wrap(bos.toByteArray()); - } - } catch (IOException e) { - throw new IllegalArgumentException("serialization error of " + obj, e); - } - } - - public static ByteBuffer byteArrayToByteBuffer(byte[] bytes) { - return ByteBuffer.wrap(bytes); - } - - public static byte[] byteBufferToByteArray(ByteBuffer buffer) { - return ByteBuffers.toByteArray(buffer); - } - - public static Object toObject(ByteBuffer buffer) { - byte[] bytes = ByteBuffers.toByteArray(buffer); - try (ByteArrayInputStream bis = new ByteArrayInputStream(bytes)) { - try (ObjectInputStream ois = new ObjectInputStream(bis)) { - return ois.readObject(); - } - } catch (IOException | ClassNotFoundException e) { - throw new IllegalArgumentException("deserialization error ", e); - } - } - - public static Object toObject(byte[] bytes) { - try (ByteArrayInputStream bis = new ByteArrayInputStream(bytes)) { - try (ObjectInputStream ois = new ObjectInputStream(bis)) { - return ois.readObject(); - } - } catch (IOException | ClassNotFoundException e) { - throw new IllegalArgumentException("deserialization error ", e); - } - } - - public static ContentFile toInternalTableFile(ByteBuffer buffer) { - return (ContentFile) toObject(buffer); - } - - public static ContentFile toInternalTableFile(byte[] bytes) { - return (ContentFile) toObject(bytes); - } - - public static IcebergContentFile toIcebergContentFile(ByteBuffer buffer) { - return (IcebergContentFile) toObject(buffer); - } -} diff --git a/core/src/main/java/com/netease/arctic/utils/SerializationUtils.java b/core/src/main/java/com/netease/arctic/utils/SerializationUtils.java new file mode 100644 index 0000000000..91aa4a50b3 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/utils/SerializationUtils.java @@ -0,0 +1,173 @@ +/* + * 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.utils; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.netease.arctic.data.IcebergContentFile; +import org.apache.avro.util.Utf8; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.util.ByteBuffers; +import org.objenesis.strategy.StdInstantiatorStrategy; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.nio.ByteBuffer; + +public class SerializationUtils { + + private static final ThreadLocal SERIALIZER = + ThreadLocal.withInitial(KryoSerializerInstance::new); + + public static ByteBuffer toByteBuffer(Object obj) { + try (ByteArrayOutputStream bos = new ByteArrayOutputStream()) { + try (ObjectOutputStream oos = new ObjectOutputStream(bos)) { + oos.writeObject(obj); + oos.flush(); + return ByteBuffer.wrap(bos.toByteArray()); + } + } catch (IOException e) { + throw new IllegalArgumentException("serialization error of " + obj, e); + } + } + + public static ByteBuffer byteArrayToByteBuffer(byte[] bytes) { + return ByteBuffer.wrap(bytes); + } + + public static byte[] byteBufferToByteArray(ByteBuffer buffer) { + return ByteBuffers.toByteArray(buffer); + } + + public static Object toObject(ByteBuffer buffer) { + byte[] bytes = ByteBuffers.toByteArray(buffer); + try (ByteArrayInputStream bis = new ByteArrayInputStream(bytes)) { + try (ObjectInputStream ois = new ObjectInputStream(bis)) { + return ois.readObject(); + } + } catch (IOException | ClassNotFoundException e) { + throw new IllegalArgumentException("deserialization error ", e); + } + } + + public static Object toObject(byte[] bytes) { + try (ByteArrayInputStream bis = new ByteArrayInputStream(bytes)) { + try (ObjectInputStream ois = new ObjectInputStream(bis)) { + return ois.readObject(); + } + } catch (IOException | ClassNotFoundException e) { + throw new IllegalArgumentException("deserialization error ", e); + } + } + + public static ContentFile toInternalTableFile(ByteBuffer buffer) { + return (ContentFile) toObject(buffer); + } + + public static ContentFile toInternalTableFile(byte[] bytes) { + return (ContentFile) toObject(bytes); + } + + public static IcebergContentFile toIcebergContentFile(ByteBuffer buffer) { + return (IcebergContentFile) toObject(buffer); + } + + public static byte[] serialize(final Object obj) throws IOException { + return SERIALIZER.get().serialize(obj); + } + + public static T deserialize(final byte[] objectData) { + if (objectData == null) { + throw new IllegalArgumentException("The byte[] must not be null"); + } + return (T) SERIALIZER.get().deserialize(objectData); + } + + private static class KryoSerializerInstance implements Serializable { + public static final int KRYO_SERIALIZER_INITIAL_BUFFER_SIZE = 1048576; + private final Kryo kryo; + private final ByteArrayOutputStream baos; + + KryoSerializerInstance() { + KryoInstantiator kryoInstantiator = new KryoInstantiator(); + kryo = kryoInstantiator.newKryo(); + baos = new ByteArrayOutputStream(KRYO_SERIALIZER_INITIAL_BUFFER_SIZE); + kryo.setRegistrationRequired(false); + } + + byte[] serialize(Object obj) { + kryo.reset(); + baos.reset(); + Output output = new Output(baos); + this.kryo.writeClassAndObject(output, obj); + output.close(); + return baos.toByteArray(); + } + + Object deserialize(byte[] objectData) { + return this.kryo.readClassAndObject(new Input(objectData)); + } + } + + private static class KryoInstantiator implements Serializable { + + public Kryo newKryo() { + Kryo kryo = new Kryo(); + + // This instance of Kryo should not require prior registration of classes + kryo.setRegistrationRequired(false); + Kryo.DefaultInstantiatorStrategy instantiatorStrategy = new Kryo.DefaultInstantiatorStrategy(); + instantiatorStrategy.setFallbackInstantiatorStrategy(new StdInstantiatorStrategy()); + kryo.setInstantiatorStrategy(instantiatorStrategy); + // Handle cases where we may have an odd classloader setup like with libjars + // for hadoop + kryo.setClassLoader(Thread.currentThread().getContextClassLoader()); + + // Register serializers + kryo.register(Utf8.class, new AvroUtf8Serializer()); + + return kryo; + } + + } + + private static class AvroUtf8Serializer extends Serializer { + + @SuppressWarnings("unchecked") + @Override + public void write(Kryo kryo, Output output, Utf8 utf8String) { + Serializer bytesSerializer = kryo.getDefaultSerializer(byte[].class); + bytesSerializer.write(kryo, output, utf8String.getBytes()); + } + + @SuppressWarnings("unchecked") + @Override + public Utf8 read(Kryo kryo, Input input, Class type) { + Serializer bytesSerializer = kryo.getDefaultSerializer(byte[].class); + byte[] bytes = bytesSerializer.read(kryo, input, byte[].class); + return new Utf8(bytes); + } + } +} diff --git a/core/src/main/java/com/netease/arctic/utils/StructLikeSet.java b/core/src/main/java/com/netease/arctic/utils/StructLikeSet.java new file mode 100644 index 0000000000..e16b7e4677 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/utils/StructLikeSet.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.utils; + +import com.netease.arctic.utils.map.StructLikeBaseMap; +import com.netease.arctic.utils.map.StructLikeMemoryMap; +import com.netease.arctic.utils.map.StructLikeSpillableMap; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types; + +public class StructLikeSet { + + public static StructLikeSet createMemorySet(Types.StructType type) { + return new StructLikeSet(type); + } + + public static StructLikeSet createSpillableSet(Types.StructType type, + Long maxInMemorySizeInBytes, String mapIdentifier) { + return new StructLikeSet(type, maxInMemorySizeInBytes, mapIdentifier); + } + + private static final Integer _V = 0; + private StructLikeBaseMap structLikeMap; + + private StructLikeSet(Types.StructType type) { + this.structLikeMap = StructLikeMemoryMap.create(type); + } + + private StructLikeSet(Types.StructType type, Long maxInMemorySizeInBytes, String mapIdentifier) { + this.structLikeMap = StructLikeSpillableMap.create(type, maxInMemorySizeInBytes, mapIdentifier); + } + + public boolean contains(StructLike key) { + return structLikeMap.get(key) != null; + } + + public void add(StructLike struct) { + structLikeMap.put(struct, _V); + } + + public void remove(StructLike struct) { + structLikeMap.delete(struct); + } + + public void close() { + structLikeMap.close(); + } +} \ No newline at end of file diff --git a/core/src/main/java/com/netease/arctic/utils/FileUtil.java b/core/src/main/java/com/netease/arctic/utils/TableFileUtils.java similarity index 94% rename from core/src/main/java/com/netease/arctic/utils/FileUtil.java rename to core/src/main/java/com/netease/arctic/utils/TableFileUtils.java index de826d411e..562d346490 100644 --- a/core/src/main/java/com/netease/arctic/utils/FileUtil.java +++ b/core/src/main/java/com/netease/arctic/utils/TableFileUtils.java @@ -33,8 +33,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -public class FileUtil { - private static final Logger LOG = LoggerFactory.getLogger(FileUtil.class); +public class TableFileUtils { + private static final Logger LOG = LoggerFactory.getLogger(TableFileUtils.class); private static final String KEYED_FILE_NAME_PATTERN_STRING = "(\\d+)-(\\w+)-(\\d+)-(\\d+)-(\\d+)-(\\d+)\\.\\w+"; private static final Pattern KEYED_FILE_NAME_PATTERN = Pattern.compile(KEYED_FILE_NAME_PATTERN_STRING); @@ -110,7 +110,7 @@ public static String getNewFilePath(String newDirectory, String filePath) { * @return fileMeta */ public static DefaultKeyedFile.FileMeta parseFileMetaFromFileName(String fileName) { - fileName = FileUtil.getFileName(fileName); + fileName = TableFileUtils.getFileName(fileName); Matcher matcher = KEYED_FILE_NAME_PATTERN.matcher(fileName); long nodeId = 1; DataFileType type = DataFileType.BASE_FILE; @@ -130,7 +130,7 @@ public static DefaultKeyedFile.FileMeta parseFileMetaFromFileName(String fileNam * @return DataFileType */ public static DataFileType parseFileTypeFromFileName(String fileName) { - fileName = FileUtil.getFileName(fileName); + fileName = TableFileUtils.getFileName(fileName); Matcher matcher = KEYED_FILE_NAME_PATTERN.matcher(fileName); DataFileType type = DataFileType.BASE_FILE; if (matcher.matches()) { @@ -145,7 +145,7 @@ public static DataFileType parseFileTypeFromFileName(String fileName) { * @return transaction id */ public static long parseFileTidFromFileName(String fileName) { - fileName = FileUtil.getFileName(fileName); + fileName = TableFileUtils.getFileName(fileName); Matcher matcher = KEYED_FILE_NAME_PATTERN.matcher(fileName); long transactionId = 0L; if (matcher.matches()) { @@ -160,7 +160,7 @@ public static long parseFileTidFromFileName(String fileName) { * @return node id */ public static DataTreeNode parseFileNodeFromFileName(String fileName) { - fileName = FileUtil.getFileName(fileName); + fileName = TableFileUtils.getFileName(fileName); Matcher matcher = KEYED_FILE_NAME_PATTERN.matcher(fileName); long nodeId = 1; if (matcher.matches()) { diff --git a/core/src/main/java/com/netease/arctic/utils/map/RocksDBBackend.java b/core/src/main/java/com/netease/arctic/utils/map/RocksDBBackend.java new file mode 100644 index 0000000000..9cec42d02f --- /dev/null +++ b/core/src/main/java/com/netease/arctic/utils/map/RocksDBBackend.java @@ -0,0 +1,377 @@ +/* + * 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.utils.map; + +import com.netease.arctic.ArcticIOException; +import com.netease.arctic.utils.LocalFileUtils; +import com.netease.arctic.utils.SerializationUtils; +import org.apache.commons.lang.Validate; +import org.rocksdb.AbstractImmutableNativeReference; +import org.rocksdb.ColumnFamilyDescriptor; +import org.rocksdb.ColumnFamilyHandle; +import org.rocksdb.ColumnFamilyOptions; +import org.rocksdb.DBOptions; +import org.rocksdb.InfoLogLevel; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.rocksdb.RocksIterator; +import org.rocksdb.Statistics; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; + +public class RocksDBBackend { + private static final Logger LOG = LoggerFactory.getLogger(RocksDBBackend.class); + private static final String BACKEND_BASE_DIR = System.getProperty("rocksdb.dir"); + private static final ThreadLocal instance = + new ThreadLocal<>().withInitial(() -> create()); + + public static RocksDBBackend getOrCreateInstance() { + RocksDBBackend backend = instance.get(); + if (backend.closed) { + backend = create(); + instance.set(backend); + } + return backend; + } + + private Map handlesMap = new HashMap<>(); + private Map descriptorMap = new HashMap<>(); + private RocksDB rocksDB; + private boolean closed = false; + private final String rocksDBBasePath; + private long totalBytesWritten; + + private static RocksDBBackend create() { + return new RocksDBBackend(); + } + + private RocksDBBackend() { + this.rocksDBBasePath = BACKEND_BASE_DIR == null ? UUID.randomUUID().toString() : + String.format("%s/%s", BACKEND_BASE_DIR, UUID.randomUUID().toString()); + totalBytesWritten = 0L; + setup(); + } + + /** + * Initialized Rocks DB instance. + */ + private void setup() { + try { + LOG.info("DELETING RocksDB instance persisted at " + rocksDBBasePath); + LocalFileUtils.deleteDirectory(new File(rocksDBBasePath)); + + final DBOptions dbOptions = new DBOptions().setCreateIfMissing(true).setCreateMissingColumnFamilies(true) + .setWalDir(rocksDBBasePath).setStatsDumpPeriodSec(300).setStatistics(new Statistics()); + dbOptions.setLogger(new org.rocksdb.Logger(dbOptions) { + @Override + protected void log(InfoLogLevel infoLogLevel, String logMsg) { + LOG.info("From Rocks DB : " + logMsg); + } + }); + final List managedColumnFamilies = loadManagedColumnFamilies(dbOptions); + final List managedHandles = new ArrayList<>(); + LocalFileUtils.mkdir(new File(rocksDBBasePath)); + rocksDB = RocksDB.open(dbOptions, rocksDBBasePath, managedColumnFamilies, managedHandles); + + Validate.isTrue(managedHandles.size() == managedColumnFamilies.size(), + "Unexpected number of handles are returned"); + for (int index = 0; index < managedHandles.size(); index++) { + ColumnFamilyHandle handle = managedHandles.get(index); + ColumnFamilyDescriptor descriptor = managedColumnFamilies.get(index); + String familyNameFromHandle = new String(handle.getName()); + String familyNameFromDescriptor = new String(descriptor.getName()); + + Validate.isTrue(familyNameFromDescriptor.equals(familyNameFromHandle), + "Family Handles not in order with descriptors"); + handlesMap.put(familyNameFromHandle, handle); + descriptorMap.put(familyNameFromDescriptor, descriptor); + } + addShutDownHook(); + } catch (RocksDBException | IOException re) { + LOG.error("Got exception opening Rocks DB instance ", re); + if (rocksDB != null) { + close(); + } + throw new ArcticIOException(re); + } + } + + private void addShutDownHook() { + Runtime.getRuntime().addShutdownHook(new Thread(this::close)); + } + + /** + * Helper to load managed column family descriptors. + */ + private List loadManagedColumnFamilies(DBOptions dbOptions) throws RocksDBException { + final List managedColumnFamilies = new ArrayList<>(); + final Options options = new Options(dbOptions, new ColumnFamilyOptions()); + List existing = RocksDB.listColumnFamilies(options, rocksDBBasePath); + + if (existing.isEmpty()) { + LOG.info("No column family found. Loading default"); + managedColumnFamilies.add(getColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY)); + } else { + LOG.info("Loading column families :" + existing.stream().map(String::new).collect(Collectors.toList())); + managedColumnFamilies + .addAll(existing.stream().map(this::getColumnFamilyDescriptor).collect(Collectors.toList())); + } + return managedColumnFamilies; + } + + private ColumnFamilyDescriptor getColumnFamilyDescriptor(byte[] columnFamilyName) { + return new ColumnFamilyDescriptor(columnFamilyName, new ColumnFamilyOptions()); + } + + /** + * Perform single PUT on a column-family. + * + * @param columnFamilyName Column family name + * @param key Key + * @param value Payload + * @param Type of Payload + */ + public void put(String columnFamilyName, String key, T value) { + try { + Validate.isTrue(key != null && value != null, + "values or keys in rocksdb can not be null!"); + ColumnFamilyHandle cfHandler = handlesMap.get(columnFamilyName); + Validate.isTrue(cfHandler != null, "column family " + + columnFamilyName + " does not exists in rocksdb"); + rocksDB.put(cfHandler, key.getBytes(), serializePayload(value)); + } catch (Exception e) { + throw new ArcticIOException(e); + } + } + + /** + * Perform single PUT on a column-family. + * + * @param columnFamilyName Column family name + * @param key Key + * @param value Payload + * @param Type of Payload + */ + public void put(String columnFamilyName, K key, T value) { + try { + Validate.isTrue(key != null && value != null, + "values or keys in rocksdb can not be null!"); + byte[] payload = serializePayload(value); + rocksDB.put(handlesMap.get(columnFamilyName), SerializationUtils.serialize(key), payload); + } catch (Exception e) { + throw new ArcticIOException(e); + } + } + + /** + * Perform a single Delete operation. + * + * @param columnFamilyName Column Family name + * @param key Key to be deleted + */ + public void delete(String columnFamilyName, String key) { + try { + Validate.isTrue(key != null, "keys in rocksdb can not be null!"); + rocksDB.delete(handlesMap.get(columnFamilyName), key.getBytes()); + } catch (RocksDBException e) { + throw new ArcticIOException(e); + } + } + + /** + * Perform a single Delete operation. + * + * @param columnFamilyName Column Family name + * @param key Key to be deleted + */ + public void delete(String columnFamilyName, K key) { + try { + Validate.isTrue(key != null, "keys in rocksdb can not be null!"); + rocksDB.delete(handlesMap.get(columnFamilyName), SerializationUtils.serialize(key)); + } catch (Exception e) { + throw new ArcticIOException(e); + } + } + + /** + * Retrieve a value for a given key in a column family. + * + * @param columnFamilyName Column Family Name + * @param key Key to be retrieved + * @param Type of object stored. + */ + public T get(String columnFamilyName, String key) { + Validate.isTrue(!closed); + try { + Validate.isTrue(key != null, "keys in rocksdb can not be null!"); + ColumnFamilyHandle cfHandler = handlesMap.get(columnFamilyName); + Validate.isTrue(cfHandler != null, "column family " + + columnFamilyName + " does not exists in rocksdb"); + byte[] val = rocksDB.get(handlesMap.get(columnFamilyName), key.getBytes()); + return val == null ? null : SerializationUtils.deserialize(val); + } catch (RocksDBException e) { + throw new ArcticIOException(e); + } + } + + /** + * Retrieve a value for a given key in a column family. + * + * @param columnFamilyName Column Family Name + * @param key Key to be retrieved + * @param Type of object stored. + */ + public T get(String columnFamilyName, K key) { + Validate.isTrue(!closed); + try { + Validate.isTrue(key != null, "keys in rocksdb can not be null!"); + byte[] val = rocksDB.get(handlesMap.get(columnFamilyName), SerializationUtils.serialize(key)); + return val == null ? null : SerializationUtils.deserialize(val); + } catch (Exception e) { + throw new ArcticIOException(e); + } + } + + /** + * Return Iterator of key-value pairs from RocksIterator. + * + * @param columnFamilyName Column Family Name + * @param Type of value stored + */ + public Iterator iterator(String columnFamilyName) { + return new IteratorWrapper<>(rocksDB.newIterator(handlesMap.get(columnFamilyName))); + } + + /** + * Add a new column family to store. + * + * @param columnFamilyName Column family name + */ + public void addColumnFamily(String columnFamilyName) { + Validate.isTrue(!closed); + + descriptorMap.computeIfAbsent(columnFamilyName, colFamilyName -> { + try { + ColumnFamilyDescriptor descriptor = getColumnFamilyDescriptor(colFamilyName.getBytes()); + ColumnFamilyHandle handle = rocksDB.createColumnFamily(descriptor); + handlesMap.put(colFamilyName, handle); + return descriptor; + } catch (RocksDBException e) { + throw new ArcticIOException(e); + } + }); + } + + /** + * Note : Does not delete from underlying DB. Just closes the handle. + * + * @param columnFamilyName Column Family Name + */ + public void dropColumnFamily(String columnFamilyName) { + Validate.isTrue(!closed); + + descriptorMap.computeIfPresent(columnFamilyName, (colFamilyName, descriptor) -> { + ColumnFamilyHandle handle = handlesMap.get(colFamilyName); + try { + rocksDB.dropColumnFamily(handle); + handle.close(); + } catch (RocksDBException e) { + throw new ArcticIOException(e); + } + handlesMap.remove(columnFamilyName); + return null; + }); + } + + public List listColumnFamilies() { + return new ArrayList<>(descriptorMap.values()); + } + + /** + * Close the DAO object. + */ + public void close() { + if (!closed) { + closed = true; + handlesMap.values().forEach(AbstractImmutableNativeReference::close); + handlesMap.clear(); + descriptorMap.clear(); + rocksDB.close(); + try { + LocalFileUtils.deleteDirectory(new File(rocksDBBasePath)); + } catch (IOException e) { + throw new ArcticIOException(e.getMessage(), e); + } + } + } + + public String getRocksDBBasePath() { + return rocksDBBasePath; + } + + public long getTotalBytesWritten() { + return totalBytesWritten; + } + + private byte[] serializePayload(T value) throws IOException { + byte[] payload = SerializationUtils.serialize(value); + totalBytesWritten += payload.length; + return payload; + } + + /** + * {@link Iterator} wrapper for RocksDb Iterator {@link RocksIterator}. + */ + private static class IteratorWrapper implements Iterator { + + private final RocksIterator iterator; + + public IteratorWrapper(final RocksIterator iterator) { + this.iterator = iterator; + iterator.seekToFirst(); + } + + @Override + public boolean hasNext() { + return iterator.isValid(); + } + + @Override + public R next() { + if (!hasNext()) { + throw new IllegalStateException("next() called on rocksDB with no more valid entries"); + } + R val = SerializationUtils.deserialize(iterator.value()); + iterator.next(); + return val; + } + } +} diff --git a/core/src/main/java/com/netease/arctic/utils/map/SimpleMap.java b/core/src/main/java/com/netease/arctic/utils/map/SimpleMap.java new file mode 100644 index 0000000000..d1c06e9a3d --- /dev/null +++ b/core/src/main/java/com/netease/arctic/utils/map/SimpleMap.java @@ -0,0 +1,30 @@ +/* + * 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.utils.map; + +public interface SimpleMap { + + void put(T key, K value); + + void delete(T key); + + K get(T key); + + void close(); +} diff --git a/core/src/main/java/com/netease/arctic/utils/map/SimpleSpillableMap.java b/core/src/main/java/com/netease/arctic/utils/map/SimpleSpillableMap.java new file mode 100644 index 0000000000..e8a7a96377 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/utils/map/SimpleSpillableMap.java @@ -0,0 +1,152 @@ +/* + * 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.utils.map; + +import org.apache.commons.lang.Validate; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.openjdk.jol.info.GraphLayout; + +import java.io.Serializable; +import java.util.Map; +import java.util.Optional; + +public class SimpleSpillableMap implements SimpleMap { + + private static final int RECORDS_TO_SKIP_FOR_ESTIMATING = 200; + private final long maxInMemorySizeInBytes; + private final String mapIdentifier; + private Map memoryMap; + private Optional> diskBasedMap = Optional.empty(); + private Long currentInMemoryMapSize; + private long estimatedPayloadSize = 0; + private int putCount = 0; + + protected SimpleSpillableMap(Long maxInMemorySizeInBytes, String mapIdentifier) { + Validate.isTrue(mapIdentifier != null, "Map identifier can not be null"); + this.memoryMap = Maps.newHashMap(); + this.maxInMemorySizeInBytes = maxInMemorySizeInBytes; + this.currentInMemoryMapSize = 0L; + this.mapIdentifier = mapIdentifier; + } + + /** + * Number of bytes spilled to disk. + */ + public long getSizeOfFileOnDiskInBytes() { + return diskBasedMap.map(SimpleSpilledMap::sizeOfFileOnDiskInBytes).orElse(0L); + } + + /** + * Number of entries in InMemoryMap. + */ + public int getMemoryMapSize() { + return memoryMap.size(); + } + + /** + * Approximate memory footprint of the in-memory map. + */ + public long getMemoryMapSpaceSize() { + return currentInMemoryMapSize; + } + + public boolean containsKey(T key) { + return memoryMap.containsKey(key) || + diskBasedMap.map(diskMap -> diskMap.containsKey(key)).orElse(false); + } + + public K get(T key) { + return Optional.ofNullable(memoryMap.get(key)) + .orElse(diskBasedMap.map(diskMap -> diskMap.get(key)).orElse(null)); + } + + public void put(T key, K value) { + if (estimatedPayloadSize == 0) { + this.estimatedPayloadSize = estimateSize(key) + estimateSize(value); + } else if (++putCount % RECORDS_TO_SKIP_FOR_ESTIMATING == 0) { + this.estimatedPayloadSize = (long) (this.estimatedPayloadSize * 0.9 + + (estimateSize(key) + estimateSize(value)) * 0.1); + this.currentInMemoryMapSize = this.memoryMap.size() * this.estimatedPayloadSize; + } + + if (this.currentInMemoryMapSize < maxInMemorySizeInBytes) { + if (memoryMap.put(key, value) == null) { + currentInMemoryMapSize += this.estimatedPayloadSize; + } + } else { + if (!diskBasedMap.isPresent()) { + diskBasedMap = Optional.of(new SimpleSpilledMap<>()); + } + diskBasedMap.get().put(key, value); + } + } + + public void delete(T key) { + if (memoryMap.containsKey(key)) { + currentInMemoryMapSize -= estimatedPayloadSize; + memoryMap.remove(key); + } else { + diskBasedMap.ifPresent(map -> map.delete(key)); + } + } + + public void close() { + memoryMap = null; + diskBasedMap.ifPresent(SimpleSpilledMap::close); + currentInMemoryMapSize = 0L; + } + + private long estimateSize(Object obj) { + return obj == null ? 0 : GraphLayout.parseInstance(obj).totalSize(); + } + + protected class SimpleSpilledMap + implements SimpleMap { + + private final RocksDBBackend rocksDB = RocksDBBackend.getOrCreateInstance(); + + public SimpleSpilledMap() { + rocksDB.addColumnFamily(mapIdentifier); + } + + public boolean containsKey(T key) { + return rocksDB.get(mapIdentifier, key) != null; + } + + public K get(T key) { + return rocksDB.get(mapIdentifier, key); + } + + public void put(T key, K value) { + rocksDB.put(mapIdentifier, key, value); + } + + public void delete(T key) { + rocksDB.delete(mapIdentifier, key); + } + + public void close() { + rocksDB.dropColumnFamily(mapIdentifier); + } + + public long sizeOfFileOnDiskInBytes() { + return rocksDB.getTotalBytesWritten(); + } + } +} diff --git a/core/src/main/java/com/netease/arctic/utils/map/StructLikeBaseMap.java b/core/src/main/java/com/netease/arctic/utils/map/StructLikeBaseMap.java new file mode 100644 index 0000000000..f12bf5a7a1 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/utils/map/StructLikeBaseMap.java @@ -0,0 +1,62 @@ +/* + * 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.utils.map; + +import com.netease.arctic.iceberg.optimize.StructLikeWrapper; +import com.netease.arctic.iceberg.optimize.StructLikeWrapperFactory; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types; + +public abstract class StructLikeBaseMap implements SimpleMap { + + protected final ThreadLocal wrappers; + protected final StructLikeWrapperFactory structLikeWrapperFactory; + + protected StructLikeBaseMap(Types.StructType type) { + this.structLikeWrapperFactory = new StructLikeWrapperFactory(type); + this.wrappers = ThreadLocal.withInitial(() -> structLikeWrapperFactory.create()); + } + + @Override + public T get(StructLike key) { + StructLikeWrapper wrapper = wrappers.get(); + T value = getInternalMap().get(wrapper.set((key))); + wrapper.set(null); // don't hold a reference to the key. + return value; + } + + @Override + public void put(StructLike key, T value) { + getInternalMap().put(structLikeWrapperFactory.create().set(key), value); + } + + @Override + public void delete(StructLike key) { + StructLikeWrapper wrapper = wrappers.get(); + getInternalMap().delete(wrapper.set(key)); + wrapper.set(null); // don't hold a reference to the key. + } + + @Override + public void close() { + getInternalMap().close(); + } + + protected abstract SimpleMap getInternalMap(); +} diff --git a/core/src/main/java/com/netease/arctic/utils/map/StructLikeMemoryMap.java b/core/src/main/java/com/netease/arctic/utils/map/StructLikeMemoryMap.java new file mode 100644 index 0000000000..cebfc55e85 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/utils/map/StructLikeMemoryMap.java @@ -0,0 +1,75 @@ +/* + * 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.utils.map; + +import com.netease.arctic.iceberg.optimize.StructLikeWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +import java.util.HashMap; + +/** + * Copy form iceberg {@link org.apache.iceberg.util.StructLikeMap}. Make using StructLikeWrapper more cheap + */ +public class StructLikeMemoryMap extends StructLikeBaseMap { + + public static StructLikeMemoryMap create(Types.StructType type) { + return new StructLikeMemoryMap<>(type); + } + + private final SimpleMap wrapperMap; + + private StructLikeMemoryMap(Types.StructType type) { + super(type); + this.wrapperMap = new SimpleMemoryMap(); + } + + @Override + protected SimpleMap getInternalMap() { + return wrapperMap; + } + + + private class SimpleMemoryMap implements SimpleMap { + + HashMap map = Maps.newHashMap(); + + @Override + public void put(StructLikeWrapper key, T value) { + map.put(key, value); + } + + @Override + public void delete(StructLikeWrapper key) { + map.remove(key); + } + + @Override + public T get(StructLikeWrapper key) { + return map.get(key); + } + + + @Override + public void close() { + //do nothing and gc will digard it + } + } +} + diff --git a/core/src/main/java/com/netease/arctic/utils/map/StructLikeSpillableMap.java b/core/src/main/java/com/netease/arctic/utils/map/StructLikeSpillableMap.java new file mode 100644 index 0000000000..58307a3ff4 --- /dev/null +++ b/core/src/main/java/com/netease/arctic/utils/map/StructLikeSpillableMap.java @@ -0,0 +1,75 @@ +/* + * 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.utils.map; + +import com.netease.arctic.iceberg.optimize.StructLikeWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +import java.util.HashMap; + +/** + * Copy form iceberg {@link org.apache.iceberg.util.StructLikeMap}. Make using StructLikeWrapper more cheap + */ +public class StructLikeSpillableMap extends StructLikeBaseMap { + + public static StructLikeSpillableMap create(Types.StructType type, + Long maxInMemorySizeInBytes, String mapIdentifier) { + return new StructLikeSpillableMap<>(type, maxInMemorySizeInBytes, mapIdentifier); + } + + private final SimpleMap wrapperMap; + + private StructLikeSpillableMap(Types.StructType type, Long maxInMemorySizeInBytes, + String mapIdentifier) { + super(type); + this.wrapperMap = new SimpleSpillableMap(maxInMemorySizeInBytes, mapIdentifier); + } + + @Override + protected SimpleMap getInternalMap() { + return wrapperMap; + } + + private class SimpleMemoryMap implements SimpleMap { + + HashMap map = Maps.newHashMap(); + + @Override + public void put(StructLikeWrapper key, T value) { + map.put(key, value); + } + + @Override + public void delete(StructLikeWrapper key) { + map.remove(key); + } + + @Override + public T get(StructLikeWrapper key) { + return map.get(key); + } + + @Override + public void close() { + //do nothing and gc will digard it + } + } +} + diff --git a/core/src/test/java/com/netease/arctic/io/ArcticFileIoDummy.java b/core/src/test/java/com/netease/arctic/io/ArcticFileIoDummy.java index fee1653cdf..000fb56ced 100644 --- a/core/src/test/java/com/netease/arctic/io/ArcticFileIoDummy.java +++ b/core/src/test/java/com/netease/arctic/io/ArcticFileIoDummy.java @@ -18,13 +18,14 @@ package com.netease.arctic.io; -import java.util.List; -import java.util.concurrent.Callable; import org.apache.hadoop.fs.FileStatus; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import java.util.List; +import java.util.concurrent.Callable; + public class ArcticFileIoDummy implements ArcticFileIO{ private FileIO fileIO; diff --git a/core/src/test/java/com/netease/arctic/io/IcebergFanoutPosDeleteWriterTest.java b/core/src/test/java/com/netease/arctic/io/IcebergFanoutPosDeleteWriterTest.java index b8d9e0342b..6b8ebfd4fe 100644 --- a/core/src/test/java/com/netease/arctic/io/IcebergFanoutPosDeleteWriterTest.java +++ b/core/src/test/java/com/netease/arctic/io/IcebergFanoutPosDeleteWriterTest.java @@ -20,7 +20,7 @@ import com.netease.arctic.IcebergTableBase; import com.netease.arctic.io.writer.IcebergFanoutPosDeleteWriter; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.hadoop.fs.Path; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -75,12 +75,12 @@ public void testWritePosDelete() throws IOException { String dataDir = temp.newFolder("data").getPath(); - String dataFile1Path = new Path(FileUtil.getNewFilePath(dataDir, "data-1.parquet")).toString(); + String dataFile1Path = new Path(TableFileUtils.getNewFilePath(dataDir, "data-1.parquet")).toString(); icebergPosDeleteWriter.delete(dataFile1Path, 0); icebergPosDeleteWriter.delete(dataFile1Path, 1); icebergPosDeleteWriter.delete(dataFile1Path, 3); - String dataFile2Path = new Path(FileUtil.getNewFilePath(dataDir, "data-2.parquet")).toString(); + String dataFile2Path = new Path(TableFileUtils.getNewFilePath(dataDir, "data-2.parquet")).toString(); icebergPosDeleteWriter.delete(dataFile2Path, 10); icebergPosDeleteWriter.delete(dataFile2Path, 9); icebergPosDeleteWriter.delete(dataFile2Path, 8); @@ -90,7 +90,7 @@ public void testWritePosDelete() throws IOException { Map deleteFileMap = deleteFiles.stream().collect(Collectors.toMap(f -> f.path().toString(), f -> f)); DeleteFile deleteFile1 = deleteFileMap.get( - new Path(FileUtil.getNewFilePath(dataDir, "data-1-delete-suffix.parquet")).toString()); + new Path(TableFileUtils.getNewFilePath(dataDir, "data-1-delete-suffix.parquet")).toString()); Assert.assertNotNull(deleteFile1); Assert.assertEquals(3, deleteFile1.recordCount()); // Check whether the path-pos pairs are sorted as expected. @@ -104,10 +104,10 @@ public void testWritePosDelete() throws IOException { Assert.assertEquals(expectedDeletes, readRecordsAsList(FileFormat.PARQUET, pathPosSchema, deleteFile1.path())); DeleteFile deleteFile2 = deleteFileMap.get( - new Path(FileUtil.getNewFilePath(dataDir, "data-2-delete-suffix.parquet")).toString()); + new Path(TableFileUtils.getNewFilePath(dataDir, "data-2-delete-suffix.parquet")).toString()); Assert.assertNotNull(deleteFile2); Assert.assertEquals( - new Path(FileUtil.getNewFilePath(dataDir, "data-2-delete-suffix.parquet")).toString(), + new Path(TableFileUtils.getNewFilePath(dataDir, "data-2-delete-suffix.parquet")).toString(), deleteFile2.path().toString()); Assert.assertEquals(3, deleteFile2.recordCount()); // Check whether the path-pos pairs are sorted as expected. diff --git a/core/src/test/java/com/netease/arctic/scan/TableEntriesScanTest.java b/core/src/test/java/com/netease/arctic/scan/TableEntriesScanTest.java index 44990fc11a..eef2db3e91 100644 --- a/core/src/test/java/com/netease/arctic/scan/TableEntriesScanTest.java +++ b/core/src/test/java/com/netease/arctic/scan/TableEntriesScanTest.java @@ -21,7 +21,7 @@ import com.netease.arctic.IcebergFileEntry; import com.netease.arctic.data.DataFileType; import com.netease.arctic.io.TableTestBaseWithInitData; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -45,7 +45,7 @@ public void testScanDataEntries() { for (IcebergFileEntry entry : dataFileScan.entries()) { cnt++; DataFile file = (DataFile) entry.getFile(); - DataFileType dataFileType = FileUtil.parseFileTypeFromFileName(file.path().toString()); + DataFileType dataFileType = TableFileUtils.parseFileTypeFromFileName(file.path().toString()); if (dataFileType == DataFileType.INSERT_FILE) { Assert.assertEquals(1, entry.getSequenceNumber()); } else if (dataFileType == DataFileType.EQ_DELETE_FILE) { @@ -112,7 +112,7 @@ public void testScanEntriesWithFilter() { for (IcebergFileEntry entry : dataFileScan.entries()) { cnt++; DataFile file = (DataFile) entry.getFile(); - DataFileType dataFileType = FileUtil.parseFileTypeFromFileName(file.path().toString()); + DataFileType dataFileType = TableFileUtils.parseFileTypeFromFileName(file.path().toString()); if (dataFileType == DataFileType.INSERT_FILE) { Assert.assertEquals(1, entry.getSequenceNumber()); } else if (dataFileType == DataFileType.EQ_DELETE_FILE) { diff --git a/core/src/test/java/com/netease/arctic/utils/FileUtilTest.java b/core/src/test/java/com/netease/arctic/utils/FileUtilTest.java index e3ddd6f3d6..51dea04e77 100644 --- a/core/src/test/java/com/netease/arctic/utils/FileUtilTest.java +++ b/core/src/test/java/com/netease/arctic/utils/FileUtilTest.java @@ -28,14 +28,14 @@ public class FileUtilTest { @Test public void getFileName() { - String fileName = FileUtil.getFileName("hdfs://easyops-sloth/user/warehouse/animal_partition_two/base/" + + String fileName = TableFileUtils.getFileName("hdfs://easyops-sloth/user/warehouse/animal_partition_two/base/" + "opt_mon=202109/opt_day=26/00000-0-3-1-37128f07-0845-43d8-905b-bd69b4ca351c-0000000001.parquet"); Assert.assertEquals("00000-0-3-1-37128f07-0845-43d8-905b-bd69b4ca351c-0000000001.parquet", fileName); } @Test public void getFileDir() { - String fileDir = FileUtil.getFileDir("hdfs://easyops-sloth/user/warehouse/animal_partition_two/base/" + + String fileDir = TableFileUtils.getFileDir("hdfs://easyops-sloth/user/warehouse/animal_partition_two/base/" + "opt_mon=202109/opt_day=26/00000-0-3-1-37128f07-0845-43d8-905b-bd69b4ca351c-0000000001.parquet"); Assert.assertEquals("hdfs://easyops-sloth/user/warehouse/animal_partition_two/base/opt_mon=202109/opt_day=26", fileDir); } @@ -44,14 +44,14 @@ public void getFileDir() { public void testParseFileName() { String fileName = "hdfs://easyops-sloth/user/warehouse/animal_partition_two/base/5-I-2-00000-941953957-0000000001.parquet"; - DefaultKeyedFile.FileMeta fileMeta = FileUtil.parseFileMetaFromFileName(fileName); + DefaultKeyedFile.FileMeta fileMeta = TableFileUtils.parseFileMetaFromFileName(fileName); Assert.assertEquals(DataFileType.INSERT_FILE, fileMeta.type()); Assert.assertEquals(DataTreeNode.of(3,1), fileMeta.node()); Assert.assertEquals(2, fileMeta.transactionId()); - Assert.assertEquals(DataFileType.INSERT_FILE, FileUtil.parseFileTypeFromFileName(fileName)); - Assert.assertEquals(DataTreeNode.of(3,1), FileUtil.parseFileNodeFromFileName(fileName)); - Assert.assertEquals(2, FileUtil.parseFileTidFromFileName(fileName)); + Assert.assertEquals(DataFileType.INSERT_FILE, TableFileUtils.parseFileTypeFromFileName(fileName)); + Assert.assertEquals(DataTreeNode.of(3,1), TableFileUtils.parseFileNodeFromFileName(fileName)); + Assert.assertEquals(2, TableFileUtils.parseFileTidFromFileName(fileName)); } } \ No newline at end of file diff --git a/core/src/test/java/com/netease/arctic/utils/map/TestRocksDBBackend.java b/core/src/test/java/com/netease/arctic/utils/map/TestRocksDBBackend.java new file mode 100644 index 0000000000..5a915424e3 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/utils/map/TestRocksDBBackend.java @@ -0,0 +1,102 @@ +package com.netease.arctic.utils.map; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +public class TestRocksDBBackend { + + public static final String CF_NAME = "TEST"; + + @Before + public void setup() { + } + + @Test + public void testAddAndDropColumnFamily() throws Exception { + RocksDBBackend rocksDBBackend = RocksDBBackend.getOrCreateInstance(); + int originalCfCount = rocksDBBackend.listColumnFamilies().size(); + rocksDBBackend.addColumnFamily(CF_NAME); + Assert.assertEquals(originalCfCount + 1, rocksDBBackend.listColumnFamilies().size()); + Assert.assertEquals(CF_NAME, new String(rocksDBBackend.listColumnFamilies().get(originalCfCount).getName(), "utf8")); + rocksDBBackend.dropColumnFamily(CF_NAME); + Assert.assertEquals(originalCfCount, rocksDBBackend.listColumnFamilies().size()); + } + + @Test + public void testPutGetDelete() { + RocksDBBackend rocksDBBackend = RocksDBBackend.getOrCreateInstance(); + rocksDBBackend.addColumnFamily(CF_NAME); + rocksDBBackend.put(CF_NAME, "name", "mj"); + rocksDBBackend.put(CF_NAME, 2, "zjs"); + rocksDBBackend.put(CF_NAME, 4556, "zyx"); + Assert.assertEquals("zyx", rocksDBBackend.get(CF_NAME, 4556)); + Assert.assertEquals("zjs", rocksDBBackend.get(CF_NAME, 2)); + Assert.assertEquals("mj", rocksDBBackend.get(CF_NAME, "name")); + rocksDBBackend.delete(CF_NAME, 4556); + rocksDBBackend.delete(CF_NAME, "name"); + Assert.assertNull(rocksDBBackend.get(CF_NAME, 4556)); + Assert.assertNull(rocksDBBackend.get(CF_NAME, "name")); + rocksDBBackend.put(CF_NAME, 2, "mj"); + Assert.assertEquals("mj", rocksDBBackend.get(CF_NAME, 2)); + rocksDBBackend.put(CF_NAME, "name", "mj"); + Assert.assertEquals("mj", rocksDBBackend.get(CF_NAME, "name")); + rocksDBBackend.dropColumnFamily(CF_NAME); + try { + String value = rocksDBBackend.get(CF_NAME, "name"); + Assert.assertTrue(false); + } catch (Throwable t) { + Assert.assertTrue(t instanceof IllegalArgumentException); + } + } + + + @Test + public void testIterator() { + RocksDBBackend rocksDBBackend = RocksDBBackend.getOrCreateInstance(); + rocksDBBackend.addColumnFamily(CF_NAME); + List expect = Arrays.asList(new String[] { + "mj", + "zjs", + "zyx" + }); + rocksDBBackend.put(CF_NAME, "name", expect.get(0)); + rocksDBBackend.put(CF_NAME, 2, expect.get(1)); + rocksDBBackend.put(CF_NAME, 4556, expect.get(2)); + Iterator values = rocksDBBackend.iterator(CF_NAME); + List valueList = new ArrayList<>(); + for ( ; values.hasNext(); ) { + valueList.add(values.next()); + } + Collections.sort(expect); + Collections.sort(valueList); + Assert.assertEquals(expect.size(), valueList.size()); + Assert.assertArrayEquals(expect.toArray(), valueList.toArray()); + + rocksDBBackend.delete(CF_NAME, "name"); + valueList = new ArrayList<>(); + values = rocksDBBackend.iterator(CF_NAME); + for ( ; values.hasNext(); ) { + valueList.add(values.next()); + } + Assert.assertEquals(2, valueList.size()); + rocksDBBackend.dropColumnFamily(CF_NAME); + } + + @Test + public void testClose() { + RocksDBBackend rocksDBBackend = RocksDBBackend.getOrCreateInstance(); + File baseFile = new File(rocksDBBackend.getRocksDBBasePath()); + Assert.assertTrue(baseFile.exists()); + Assert.assertTrue(baseFile.isDirectory()); + rocksDBBackend.close(); + Assert.assertTrue(!baseFile.exists()); + } +} diff --git a/core/src/test/java/com/netease/arctic/utils/map/TestSimpleSpillableMap.java b/core/src/test/java/com/netease/arctic/utils/map/TestSimpleSpillableMap.java new file mode 100644 index 0000000000..d4ed962759 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/utils/map/TestSimpleSpillableMap.java @@ -0,0 +1,108 @@ +package com.netease.arctic.utils.map; + +import com.google.common.collect.Maps; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.openjdk.jol.info.GraphLayout; + +import java.io.Serializable; +import java.util.Map; +import java.util.Random; +import java.util.UUID; + +public class TestSimpleSpillableMap { + + private static final Random random = new Random(100000); + + private SimpleSpillableMap map; + private long keySize; + private long valueSize; + + @Before + public void initSizes() { + keySize = GraphLayout.parseInstance(new Key()).totalSize(); + valueSize = GraphLayout.parseInstance(new Value()).totalSize(); + } + + @Test + public void testMemoryMap() { + SimpleSpillableMap map = testMap(10, 10); + Assert.assertTrue(map.getSizeOfFileOnDiskInBytes() == 0); + map.close(); + } + + @Test + public void testSpilledMap() { + SimpleSpillableMap map = testMap(0, 20); + Assert.assertTrue(map.getSizeOfFileOnDiskInBytes() > 0); + map.close(); + } + + @Test + public void testSpillableMap() { + SimpleSpillableMap map = testMap(10, 20); + Assert.assertTrue(map.getSizeOfFileOnDiskInBytes() > 0); + map.close(); + } + + private SimpleSpillableMap testMap(long expectMemorySize, int expectKeyCount) { + SimpleSpillableMap actualMap = + new SimpleSpillableMap(expectMemorySize * (keySize + valueSize), + TestRocksDBBackend.CF_NAME); + Assert.assertTrue(actualMap.getSizeOfFileOnDiskInBytes() == 0); + Map expectedMap = Maps.newHashMap(); + for (int i = 0; i < expectKeyCount; i++) { + Key key = new Key(); + Value value = new Value(); + expectedMap.put(key, value); + actualMap.put(key, value); + } + for (Key key : expectedMap.keySet()) { + Assert.assertEquals(expectedMap.get(key), actualMap.get(key)); + } + Assert.assertEquals(expectMemorySize, actualMap.getMemoryMapSize()); + Assert.assertEquals(expectMemorySize * (keySize + valueSize), + actualMap.getMemoryMapSpaceSize()); + return actualMap; + } + + + private static class Key implements Serializable { + String id = UUID.randomUUID().toString(); + + Long num = random.nextLong(); + + @Override + public int hashCode() { + return id.hashCode(); + } + + @Override + public boolean equals(Object obj) { + return id.equals(((Key) obj).id); + } + } + + private static class Value implements Serializable { + Long value = random.nextLong(); + String[] values = new String[10]; + + Value() { + for (int i = 0; i < values.length; i++) + values[i] = UUID.randomUUID().toString(); + } + + @Override + public boolean equals(Object obj) { + return value == (long) ((Value) obj).value; + } + + @Override + public String toString() { + return Long.toString(value); + } + } +} + diff --git a/core/src/test/java/com/netease/arctic/utils/map/TestSimpleSpilledMap.java b/core/src/test/java/com/netease/arctic/utils/map/TestSimpleSpilledMap.java new file mode 100644 index 0000000000..930917a619 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/utils/map/TestSimpleSpilledMap.java @@ -0,0 +1,79 @@ +package com.netease.arctic.utils.map; + +import com.netease.arctic.ArcticIOException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.Serializable; + +public class TestSimpleSpilledMap { + + private static class Key implements Serializable { + String k = "Key"; + + @Override + public boolean equals(Object obj) { + return ((Key) obj).k == k; + } + } + + private class Value implements Serializable { + int value = 666; + + @Override + public boolean equals(Object obj) { + return ((Value) obj).value == value; + } + } + + private SimpleSpillableMap.SimpleSpilledMap map; + @Before + public void createMap() { + SimpleSpillableMap spillableMap = new SimpleSpillableMap(100L, + TestRocksDBBackend.CF_NAME); + map = spillableMap.new SimpleSpilledMap(); + } + + @After + public void disposeMap() { + map.close(); + map = null; + } + @Test + public void testPutGetRemove() { + Key key = new Key(); + Value value = new Value(); + map.put( "name", 555); + map.put(2, "zjs"); + map.put(4556, "zyx"); + map.put(key, value); + Assert.assertEquals(555, map.get("name")); + Assert.assertEquals("zjs", map.get(2)); + Assert.assertEquals("zyx", map.get(4556)); + Assert.assertEquals(value, map.get(key)); + map.delete(4556); + Assert.assertNull(map.get(4556)); + map.put(4556, value); + Assert.assertEquals(value, map.get(4556)); + } + + @Test + public void testPutNull() { + Key key = new Key(); + Value value = new Value(); + map.put(key, value); + Assert.assertEquals(value, map.get(key)); + try { + map.put(key, null); + } catch (Throwable t) { + Assert.assertTrue(t instanceof ArcticIOException); + } + try { + map.put(null, value); + } catch (Throwable t) { + Assert.assertTrue(t instanceof ArcticIOException); + } + } +} diff --git a/core/src/test/java/com/netease/arctic/utils/map/TestStructLikeMap.java b/core/src/test/java/com/netease/arctic/utils/map/TestStructLikeMap.java new file mode 100644 index 0000000000..cc134e01f0 --- /dev/null +++ b/core/src/test/java/com/netease/arctic/utils/map/TestStructLikeMap.java @@ -0,0 +1,79 @@ +package com.netease.arctic.utils.map; + +import com.google.common.collect.Maps; +import com.netease.arctic.data.ChangedLsn; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; + +import java.io.UnsupportedEncodingException; +import java.util.Arrays; +import java.util.Map; +import java.util.Random; +import java.util.UUID; + +public class TestStructLikeMap { + private static final Random RANDOM = new Random(100000); + + private static Types.StructType TYPE = Types.StructType.of( + Arrays.asList(new Types.NestedField[] { + Types.NestedField.of(1, false,"c1", Types.DoubleType.get()), + Types.NestedField.of(2, false,"c2", Types.IntegerType.get()), + Types.NestedField.of(3, false,"c3", Types.BooleanType.get()), + Types.NestedField.of(4, false,"c4", Types.StringType.get()), + Types.NestedField.of(5, false,"c5", Types.BinaryType.get()) } + ) + ); + private static class StructLikeImpl implements StructLike { + + private Object[] values = new Object[]{ + RANDOM.nextDouble(), + RANDOM.nextInt(), + RANDOM.nextBoolean(), + UUID.randomUUID().toString(), + UUID.randomUUID().toString().getBytes("utf8") + }; + + StructLikeImpl() throws UnsupportedEncodingException { + } + @Override + public int size() { + return 5; + } + + @Override + public T get(int pos, Class javaClass) { + return javaClass.cast(values[pos]); + } + + @Override + public void set(int pos, T value) { + throw new UnsupportedOperationException(); + } + } + + @Test + public void testMemoryMap() throws UnsupportedEncodingException { + testMap(StructLikeMemoryMap.create(TYPE)); + } + + @Test + public void testSpillableMap() throws UnsupportedEncodingException { + testMap(StructLikeSpillableMap.create(TYPE, + 50L, TestRocksDBBackend.CF_NAME)); + } + + private void testMap(StructLikeBaseMap actualMap) throws UnsupportedEncodingException { + Map expectedMap = Maps.newHashMap(); + for (long i = 0; i < 100; i++) { + StructLikeImpl key = new StructLikeImpl(); + expectedMap.put(key, ChangedLsn.of(i, i)); + actualMap.put(key, ChangedLsn.of(i, i)); + } + for (StructLike key : expectedMap.keySet()) { + Assert.assertEquals(expectedMap.get(key), actualMap.get(key)); + } + actualMap.close(); + } +} diff --git a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/DynamicTableSourceTestBase.java b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/DynamicTableSourceTestBase.java index 02c6643a93..397a11a793 100644 --- a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/DynamicTableSourceTestBase.java +++ b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/DynamicTableSourceTestBase.java @@ -24,7 +24,6 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.connector.source.ScanTableSource; import org.apache.flink.table.connector.source.SourceFunctionProvider; import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown; diff --git a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TemporalJoinSplitsThreadSafeTest.java b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TemporalJoinSplitsThreadSafeTest.java index 4284f3af9f..6b8e28408f 100644 --- a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TemporalJoinSplitsThreadSafeTest.java +++ b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TemporalJoinSplitsThreadSafeTest.java @@ -18,8 +18,8 @@ package com.netease.arctic.flink.read.hybrid.enumerator; -import com.netease.arctic.flink.read.hybrid.split.TemporalJoinSplits; import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; +import com.netease.arctic.flink.read.hybrid.split.TemporalJoinSplits; import org.junit.Assert; import org.junit.Test; diff --git a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunctionTest.java b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunctionTest.java index 7107617c84..d5147430e2 100644 --- a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunctionTest.java +++ b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/read/hybrid/reader/RowDataReaderFunctionTest.java @@ -26,7 +26,6 @@ import com.netease.arctic.flink.read.source.DataIterator; import com.netease.arctic.scan.ArcticFileScanTask; import com.netease.arctic.scan.BaseArcticFileScanTask; -import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.KeyedTable; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.GenericRowData; diff --git a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java index ead64c3e89..c037f9461e 100644 --- a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java +++ b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java @@ -18,7 +18,6 @@ package com.netease.arctic.flink.util; -import java.time.Instant; import org.apache.commons.lang3.ArrayUtils; import org.apache.flink.table.api.ApiExpression; import org.apache.flink.table.data.GenericRowData; @@ -34,6 +33,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.junit.Assert; +import java.time.Instant; import java.time.LocalDateTime; import java.util.Collection; import java.util.HashMap; diff --git a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/util/ProjectionTest.java b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/util/ProjectionTest.java index 692e6cbe10..f81ffb8e8e 100644 --- a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/util/ProjectionTest.java +++ b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/util/ProjectionTest.java @@ -19,7 +19,6 @@ package com.netease.arctic.flink.util; import org.apache.flink.table.types.DataType; - import org.junit.jupiter.api.Test; import static org.apache.flink.table.api.DataTypes.BIGINT; diff --git a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/write/AdaptHiveWriterTest.java b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/write/AdaptHiveWriterTest.java index de67e9e0f2..dddd0cf884 100644 --- a/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/write/AdaptHiveWriterTest.java +++ b/flink/v1.12/flink/src/test/java/com/netease/arctic/flink/write/AdaptHiveWriterTest.java @@ -20,27 +20,12 @@ import com.netease.arctic.flink.read.AdaptHiveFlinkParquetReaders; import com.netease.arctic.hive.HiveTableTestBase; -import com.netease.arctic.hive.io.HiveTestRecords; -import com.netease.arctic.hive.io.writer.AdaptHiveGenericTaskWriterBuilder; import com.netease.arctic.hive.table.HiveLocationKind; -import com.netease.arctic.io.writer.GenericBaseTaskWriter; -import com.netease.arctic.io.writer.GenericChangeTaskWriter; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.BaseLocationKind; import com.netease.arctic.table.ChangeLocationKind; import com.netease.arctic.table.LocationKind; import com.netease.arctic.table.WriteOperationKind; -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; @@ -48,8 +33,6 @@ import org.apache.flink.table.data.TimestampData; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.parquet.AdaptHiveGenericParquetReaders; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.TaskWriter; @@ -62,6 +45,17 @@ import org.junit.BeforeClass; import org.junit.Test; +import java.io.IOException; +import java.math.BigDecimal; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + public class AdaptHiveWriterTest extends HiveTableTestBase { @BeforeClass diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/ArcticSourceReader.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/ArcticSourceReader.java index 519e9fbbab..1040af0bf8 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/ArcticSourceReader.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/ArcticSourceReader.java @@ -25,6 +25,7 @@ import com.netease.arctic.flink.read.hybrid.split.SplitRequestEvent; import com.netease.arctic.flink.util.FlinkClassReflectionUtil; import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SourceOutput; @@ -32,6 +33,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; import org.apache.flink.core.io.InputStatus; +import org.apache.flink.streaming.api.operators.source.ProgressiveTimestampsAndWatermarks; import org.apache.flink.streaming.api.operators.source.SourceOutputWithWatermarks; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticDynamicSink.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticDynamicSink.java index 3ff1382df4..e178024594 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticDynamicSink.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticDynamicSink.java @@ -21,7 +21,6 @@ import com.netease.arctic.flink.util.ArcticUtils; import com.netease.arctic.flink.write.FlinkSink; import com.netease.arctic.table.ArcticTable; -import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.connector.ChangelogMode; @@ -29,7 +28,6 @@ import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; -import org.apache.flink.table.data.RowData; import org.apache.flink.types.RowKind; import org.apache.hadoop.security.UserGroupInformation; import org.slf4j.Logger; diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticTableLoader.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticTableLoader.java index b148a7c163..2d78eb08e8 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticTableLoader.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/table/ArcticTableLoader.java @@ -22,7 +22,6 @@ import com.netease.arctic.flink.InternalCatalogBuilder; import com.netease.arctic.flink.interceptor.FlinkTablePropertiesInvocationHandler; import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableIdentifier; import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; diff --git a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/write/hidden/kafka/HiddenKafkaFactory.java b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/write/hidden/kafka/HiddenKafkaFactory.java index a3e9b6c653..14982d9a40 100644 --- a/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/write/hidden/kafka/HiddenKafkaFactory.java +++ b/flink/v1.14/flink/src/main/java/com/netease/arctic/flink/write/hidden/kafka/HiddenKafkaFactory.java @@ -22,7 +22,6 @@ import com.netease.arctic.flink.write.hidden.ArcticLogPartitioner; import com.netease.arctic.flink.write.hidden.LogMsgFactory; import com.netease.arctic.log.LogDataJsonSerialization; -import com.netease.arctic.table.TableProperties; import java.util.Properties; diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/AdaptHiveFlinkParquetWriters.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/AdaptHiveFlinkParquetWriters.java index 1f668d58b4..a989bc24fc 100644 --- a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/AdaptHiveFlinkParquetWriters.java +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/data/AdaptHiveFlinkParquetWriters.java @@ -32,7 +32,6 @@ import org.apache.flink.table.types.logical.RowType.RowField; import org.apache.flink.table.types.logical.SmallIntType; import org.apache.flink.table.types.logical.TinyIntType; -import org.apache.iceberg.flink.data.ParquetWithFlinkSchemaVisitor; import org.apache.iceberg.parquet.AdaptHivePrimitiveWriter; import org.apache.iceberg.parquet.ParquetValueReaders; import org.apache.iceberg.parquet.ParquetValueWriter; diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssignerTest.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssignerTest.java index fc9750b4dc..916fe97081 100644 --- a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssignerTest.java +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/assigner/ShuffleSplitAssignerTest.java @@ -22,13 +22,6 @@ import com.netease.arctic.flink.read.FlinkSplitPlanner; import com.netease.arctic.flink.read.hybrid.reader.RowDataReaderFunctionTest; import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.Callable; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.BiConsumer; import org.apache.flink.api.connector.source.ReaderInfo; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumeratorContext; @@ -39,6 +32,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; + public class ShuffleSplitAssignerTest extends RowDataReaderFunctionTest { private static final Logger LOG = LoggerFactory.getLogger(ShuffleSplitAssignerTest.class); diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TemporalJoinSplitsThreadSafeTest.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TemporalJoinSplitsThreadSafeTest.java index 7894cdd903..766eb6d49d 100644 --- a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TemporalJoinSplitsThreadSafeTest.java +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/read/hybrid/enumerator/TemporalJoinSplitsThreadSafeTest.java @@ -18,8 +18,8 @@ package com.netease.arctic.flink.read.hybrid.enumerator; -import com.netease.arctic.flink.read.hybrid.split.TemporalJoinSplits; import com.netease.arctic.flink.read.hybrid.split.ArcticSplit; +import com.netease.arctic.flink.read.hybrid.split.TemporalJoinSplits; import org.junit.Assert; import org.junit.Test; diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java index 66bf7d7863..c037f9461e 100644 --- a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java @@ -18,7 +18,6 @@ package com.netease.arctic.flink.util; -import java.time.Instant; import org.apache.commons.lang3.ArrayUtils; import org.apache.flink.table.api.ApiExpression; import org.apache.flink.table.data.GenericRowData; @@ -34,6 +33,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.junit.Assert; +import java.time.Instant; import java.time.LocalDateTime; import java.util.Collection; import java.util.HashMap; @@ -45,7 +45,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.flink.table.api.Expressions.e; import static org.apache.flink.table.api.Expressions.row; public class DataUtil { diff --git a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/write/AdaptHiveWriterTest.java b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/write/AdaptHiveWriterTest.java index 8d20c37128..dddd0cf884 100644 --- a/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/write/AdaptHiveWriterTest.java +++ b/flink/v1.14/flink/src/test/java/com/netease/arctic/flink/write/AdaptHiveWriterTest.java @@ -26,17 +26,6 @@ import com.netease.arctic.table.ChangeLocationKind; import com.netease.arctic.table.LocationKind; import com.netease.arctic.table.WriteOperationKind; -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; @@ -56,6 +45,17 @@ import org.junit.BeforeClass; import org.junit.Test; +import java.io.IOException; +import java.math.BigDecimal; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + public class AdaptHiveWriterTest extends HiveTableTestBase { @BeforeClass diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/ArcticSourceReader.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/ArcticSourceReader.java index 519e9fbbab..1040af0bf8 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/ArcticSourceReader.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/read/hybrid/reader/ArcticSourceReader.java @@ -25,6 +25,7 @@ import com.netease.arctic.flink.read.hybrid.split.SplitRequestEvent; import com.netease.arctic.flink.util.FlinkClassReflectionUtil; import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.common.eventtime.WatermarkOutputMultiplexer; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SourceOutput; @@ -32,6 +33,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; import org.apache.flink.core.io.InputStatus; +import org.apache.flink.streaming.api.operators.source.ProgressiveTimestampsAndWatermarks; import org.apache.flink.streaming.api.operators.source.SourceOutputWithWatermarks; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java index 81ae89f635..4bf7379537 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticFileSource.java @@ -53,7 +53,6 @@ import java.util.Optional; import static com.netease.arctic.flink.FlinkSchemaUtil.addPrimaryKey; -import static com.netease.arctic.flink.table.descriptors.ArcticValidator.ARCTIC_LOG_CONSISTENCY_GUARANTEE_ENABLE; import static com.netease.arctic.flink.table.descriptors.ArcticValidator.DIM_TABLE_ENABLE; /** diff --git a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticTableLoader.java b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticTableLoader.java index b148a7c163..2d78eb08e8 100644 --- a/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticTableLoader.java +++ b/flink/v1.15/flink/src/main/java/com/netease/arctic/flink/table/ArcticTableLoader.java @@ -22,7 +22,6 @@ import com.netease.arctic.flink.InternalCatalogBuilder; import com.netease.arctic.flink.interceptor.FlinkTablePropertiesInvocationHandler; import com.netease.arctic.table.ArcticTable; -import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableIdentifier; import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/kafka/testutils/KafkaTestEnvironmentImpl.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/kafka/testutils/KafkaTestEnvironmentImpl.java index a74da536f8..81d851b83a 100644 --- a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/kafka/testutils/KafkaTestEnvironmentImpl.java +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/kafka/testutils/KafkaTestEnvironmentImpl.java @@ -16,7 +16,6 @@ */ package com.netease.arctic.flink.kafka.testutils; -import kafka.metrics.KafkaMetricsReporter; import kafka.server.KafkaConfig; import kafka.server.KafkaServer; import org.apache.commons.collections.list.UnmodifiableList; diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java index 64d767dd3f..c037f9461e 100644 --- a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/DataUtil.java @@ -45,7 +45,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.flink.table.api.Expressions.e; import static org.apache.flink.table.api.Expressions.row; public class DataUtil { diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/MockEnvironmentArctic.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/MockEnvironmentArctic.java index 74ff3c1610..0c4d098268 100644 --- a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/MockEnvironmentArctic.java +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/MockEnvironmentArctic.java @@ -31,7 +31,6 @@ import org.apache.flink.runtime.state.TaskStateManager; import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; -import org.apache.flink.runtime.throughput.ThroughputCalculator; import org.apache.flink.util.UserCodeClassLoader; public class MockEnvironmentArctic extends MockEnvironment { diff --git a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/MockEnvironmentBuilder.java b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/MockEnvironmentBuilder.java index c3ad3cafe5..d30666bc4c 100644 --- a/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/MockEnvironmentBuilder.java +++ b/flink/v1.15/flink/src/test/java/com/netease/arctic/flink/util/MockEnvironmentBuilder.java @@ -35,7 +35,6 @@ import org.apache.flink.runtime.state.TestTaskStateManager; import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager; import org.apache.flink.runtime.taskmanager.TaskManagerRuntimeInfo; -import org.apache.flink.runtime.throughput.ThroughputCalculator; import org.apache.flink.runtime.util.TestingTaskManagerRuntimeInfo; import org.apache.flink.runtime.util.TestingUserCodeClassLoader; import org.apache.flink.util.UserCodeClassLoader; diff --git a/hive/src/main/java/com/netease/arctic/hive/io/writer/AdaptHiveOutputFileFactory.java b/hive/src/main/java/com/netease/arctic/hive/io/writer/AdaptHiveOutputFileFactory.java index 114b389985..fa83df38b4 100644 --- a/hive/src/main/java/com/netease/arctic/hive/io/writer/AdaptHiveOutputFileFactory.java +++ b/hive/src/main/java/com/netease/arctic/hive/io/writer/AdaptHiveOutputFileFactory.java @@ -32,7 +32,7 @@ import java.util.concurrent.atomic.AtomicLong; -import static com.netease.arctic.utils.FileUtil.getFileName; +import static com.netease.arctic.utils.TableFileUtils.getFileName; /** * For adapt hive table with partitions the dir construct is : diff --git a/hive/src/main/java/com/netease/arctic/hive/op/HiveOperationTransaction.java b/hive/src/main/java/com/netease/arctic/hive/op/HiveOperationTransaction.java index 85f39c0750..0b34360cce 100644 --- a/hive/src/main/java/com/netease/arctic/hive/op/HiveOperationTransaction.java +++ b/hive/src/main/java/com/netease/arctic/hive/op/HiveOperationTransaction.java @@ -22,7 +22,6 @@ import com.netease.arctic.hive.HMSClient; import com.netease.arctic.hive.HMSClientPool; import com.netease.arctic.hive.table.UnkeyedHiveTable; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DeleteFiles; import org.apache.iceberg.ExpireSnapshots; diff --git a/hive/src/main/java/com/netease/arctic/hive/op/ReplaceHivePartitions.java b/hive/src/main/java/com/netease/arctic/hive/op/ReplaceHivePartitions.java index d14f619130..ad4e48085d 100644 --- a/hive/src/main/java/com/netease/arctic/hive/op/ReplaceHivePartitions.java +++ b/hive/src/main/java/com/netease/arctic/hive/op/ReplaceHivePartitions.java @@ -25,7 +25,7 @@ import com.netease.arctic.hive.utils.HivePartitionUtil; import com.netease.arctic.hive.utils.HiveTableUtil; import com.netease.arctic.op.UpdatePartitionProperties; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import com.netease.arctic.utils.TablePropertyUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; @@ -193,7 +193,7 @@ private void applyHivePartitions() { for (DataFile d : addFiles) { List partitionValues = HivePartitionUtil.partitionValuesAsList(d.partition(), partitionSchema); String value = Joiner.on("/").join(partitionValues); - String location = FileUtil.getFileDir(d.path().toString()); + String location = TableFileUtils.getFileDir(d.path().toString()); partitionLocationMap.put(value, location); if (!partitionDataFileMap.containsKey(value)) { partitionDataFileMap.put(value, Lists.newArrayList()); @@ -223,7 +223,7 @@ private void applyHivePartitions() { private void commitUnPartitionedTable() { if (!addFiles.isEmpty()) { - final String newDataLocation = FileUtil.getFileDir(addFiles.get(0).path().toString()); + final String newDataLocation = TableFileUtils.getFileDir(addFiles.get(0).path().toString()); try { transactionalHMSClient.run(c -> { Table tbl = c.getTable(db, tableName); @@ -264,7 +264,7 @@ private void commitPartitionedTable() { private void checkDataFileInSameLocation(String partitionLocation, List files) { Path partitionPath = new Path(partitionLocation); for (DataFile df : files) { - String fileDir = FileUtil.getFileDir(df.path().toString()); + String fileDir = TableFileUtils.getFileDir(df.path().toString()); Path dirPath = new Path(fileDir); if (!partitionPath.equals(dirPath)) { throw new CannotAlterHiveLocationException( @@ -276,6 +276,6 @@ private void checkDataFileInSameLocation(String partitionLocation, List getCreatePartition(Map partitionValues = HivePartitionUtil.partitionValuesAsList(d.partition(), partitionSchema); String value = Joiner.on("/").join(partitionValues); - String location = FileUtil.getFileDir(d.path().toString()); + String location = TableFileUtils.getFileDir(d.path().toString()); partitionLocationMap.put(value, location); if (!partitionDataFileMap.containsKey(value)) { partitionDataFileMap.put(value, Lists.newArrayList()); @@ -250,7 +250,7 @@ private void checkPartitionDelete(Set deleteFiles, Partition partition) { private void checkCreatePartitionDataFiles(List addFiles, String partitionLocation) { Path partitionPath = new Path(partitionLocation); for (DataFile df : addFiles) { - String fileDir = FileUtil.getFileDir(df.path().toString()); + String fileDir = TableFileUtils.getFileDir(df.path().toString()); Path dirPath = new Path(fileDir); if (!partitionPath.equals(dirPath)) { throw new CannotAlterHiveLocationException( @@ -368,7 +368,7 @@ private void generateUnpartitionTableLocation() { if (this.addFiles.isEmpty()) { unpartitionTableLocation = createUnpartitionEmptyLocationForHive(); } else { - unpartitionTableLocation = FileUtil.getFileDir(this.addFiles.get(0).path().toString()); + unpartitionTableLocation = TableFileUtils.getFileDir(this.addFiles.get(0).path().toString()); } } diff --git a/hive/src/test/java/com/netease/arctic/hive/HMSMockServer.java b/hive/src/test/java/com/netease/arctic/hive/HMSMockServer.java index fcdd54bb9c..21463d3dbf 100644 --- a/hive/src/test/java/com/netease/arctic/hive/HMSMockServer.java +++ b/hive/src/test/java/com/netease/arctic/hive/HMSMockServer.java @@ -18,15 +18,6 @@ package com.netease.arctic.hive; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.io.LineNumberReader; -import java.io.Reader; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.SQLException; -import java.sql.Statement; import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -51,6 +42,15 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.LineNumberReader; +import java.io.Reader; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; import java.util.Random; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; 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 a3bc86ad9f..ca6ac0f17e 100644 --- a/hive/src/test/java/com/netease/arctic/hive/HiveTableTestBase.java +++ b/hive/src/test/java/com/netease/arctic/hive/HiveTableTestBase.java @@ -33,7 +33,6 @@ import com.netease.arctic.utils.ArcticDataFiles; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.iceberg.DataFiles; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; diff --git a/hive/src/test/java/com/netease/arctic/hive/io/HiveTestRecords.java b/hive/src/test/java/com/netease/arctic/hive/io/HiveTestRecords.java index 6545a6511e..b25eb9b681 100644 --- a/hive/src/test/java/com/netease/arctic/hive/io/HiveTestRecords.java +++ b/hive/src/test/java/com/netease/arctic/hive/io/HiveTestRecords.java @@ -29,7 +29,12 @@ import java.time.ZoneOffset; import java.util.List; -import static com.netease.arctic.hive.HiveTableTestBase.*; +import static com.netease.arctic.hive.HiveTableTestBase.COLUMN_NAME_D; +import static com.netease.arctic.hive.HiveTableTestBase.COLUMN_NAME_ID; +import static com.netease.arctic.hive.HiveTableTestBase.COLUMN_NAME_NAME; +import static com.netease.arctic.hive.HiveTableTestBase.COLUMN_NAME_OP_TIME; +import static com.netease.arctic.hive.HiveTableTestBase.COLUMN_NAME_OP_TIME_WITH_ZONE; +import static com.netease.arctic.hive.HiveTableTestBase.HIVE_TABLE_SCHEMA; public class HiveTestRecords { diff --git a/hive/src/test/java/com/netease/arctic/hive/io/TestAdaptHiveReader.java b/hive/src/test/java/com/netease/arctic/hive/io/TestAdaptHiveReader.java index a754d778d1..cb9038f401 100644 --- a/hive/src/test/java/com/netease/arctic/hive/io/TestAdaptHiveReader.java +++ b/hive/src/test/java/com/netease/arctic/hive/io/TestAdaptHiveReader.java @@ -20,11 +20,12 @@ import com.netease.arctic.hive.HiveTableTestBase; import com.netease.arctic.hive.table.HiveLocationKind; -import java.io.IOException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.junit.Test; +import java.io.IOException; + import static com.netease.arctic.hive.io.TestIOUtils.testWrite; public class TestAdaptHiveReader extends HiveTableTestBase { diff --git a/hive/src/test/java/com/netease/arctic/hive/io/TestAdaptHiveWriter.java b/hive/src/test/java/com/netease/arctic/hive/io/TestAdaptHiveWriter.java index 0b85a7df39..337e26f97f 100644 --- a/hive/src/test/java/com/netease/arctic/hive/io/TestAdaptHiveWriter.java +++ b/hive/src/test/java/com/netease/arctic/hive/io/TestAdaptHiveWriter.java @@ -21,37 +21,15 @@ import com.netease.arctic.hive.HiveTableTestBase; import com.netease.arctic.hive.io.writer.AdaptHiveGenericTaskWriterBuilder; import com.netease.arctic.hive.table.HiveLocationKind; -import com.netease.arctic.io.TestRecords; import com.netease.arctic.io.writer.GenericBaseTaskWriter; import com.netease.arctic.io.writer.GenericChangeTaskWriter; -import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.BaseLocationKind; import com.netease.arctic.table.ChangeLocationKind; -import com.netease.arctic.table.LocationKind; import com.netease.arctic.table.WriteOperationKind; -import java.time.LocalDateTime; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.parquet.AdaptHiveGenericParquetReaders; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.parquet.AdaptHiveParquet; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterators; import org.junit.Assert; import org.junit.Test; import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; import static com.netease.arctic.hive.io.TestIOUtils.testWrite; diff --git a/hive/src/test/java/com/netease/arctic/hive/io/TestIOUtils.java b/hive/src/test/java/com/netease/arctic/hive/io/TestIOUtils.java index 35f1616964..a597ab36b3 100644 --- a/hive/src/test/java/com/netease/arctic/hive/io/TestIOUtils.java +++ b/hive/src/test/java/com/netease/arctic/hive/io/TestIOUtils.java @@ -19,7 +19,6 @@ package com.netease.arctic.hive.io; import com.netease.arctic.data.DefaultKeyedFile; -import com.netease.arctic.hive.io.reader.AdaptHiveBaseIcebergDataReader; import com.netease.arctic.hive.io.reader.AdaptHiveGenericArcticDataReader; import com.netease.arctic.hive.io.reader.GenericAdaptHiveIcebergDataReader; import com.netease.arctic.hive.io.writer.AdaptHiveGenericTaskWriterBuilder; @@ -31,13 +30,6 @@ import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.LocationKind; import com.netease.arctic.table.PrimaryKeySpec; -import java.io.IOException; -import java.util.Arrays; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; import org.apache.iceberg.DataFile; import org.apache.iceberg.Files; import org.apache.iceberg.PartitionSpec; @@ -55,6 +47,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterators; import org.junit.Assert; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + public class TestIOUtils { public static void testWrite(ArcticTable table, LocationKind locationKind, List records, String pathFeature) throws IOException { diff --git a/hive/src/test/java/com/netease/arctic/hive/op/AutoSyncHiveTest.java b/hive/src/test/java/com/netease/arctic/hive/op/AutoSyncHiveTest.java index e3cbac247b..d4a0ad2d28 100644 --- a/hive/src/test/java/com/netease/arctic/hive/op/AutoSyncHiveTest.java +++ b/hive/src/test/java/com/netease/arctic/hive/op/AutoSyncHiveTest.java @@ -29,7 +29,7 @@ import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.LocationKind; import com.netease.arctic.table.UnkeyedTable; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -129,7 +129,7 @@ private void testAutoSyncPartitionedTableHiveDataWrite(ArcticTable testTable) th writeRecords("p3")); dataFiles.addAll(newFiles); Partition newPartition = HivePartitionUtil.newPartition(hiveTable, Lists.newArrayList("p3"), - FileUtil.getFileDir(newFiles.get(0).path().toString()), newFiles, + TableFileUtils.getFileDir(newFiles.get(0).path().toString()), newFiles, (int) (System.currentTimeMillis() / 1000)); newPartition.getParameters().remove(HiveTableProperties.ARCTIC_TABLE_FLAG); hms.getClient().add_partition(newPartition); diff --git a/hive/src/test/java/com/netease/arctic/hive/op/TestOverwriteFiles.java b/hive/src/test/java/com/netease/arctic/hive/op/TestOverwriteFiles.java index d681677119..14d5c8f968 100644 --- a/hive/src/test/java/com/netease/arctic/hive/op/TestOverwriteFiles.java +++ b/hive/src/test/java/com/netease/arctic/hive/op/TestOverwriteFiles.java @@ -9,7 +9,7 @@ import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableIdentifier; import com.netease.arctic.table.UnkeyedTable; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import com.netease.arctic.utils.TablePropertyUtil; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.iceberg.DataFile; @@ -426,7 +426,7 @@ private void applyOverwrite( deleteLocations.forEach(partitionAndLocations::remove); addFiles.forEach(kv -> { - String partLocation = FileUtil.getFileDir(kv.getValue()); + String partLocation = TableFileUtils.getFileDir(kv.getValue()); partitionAndLocations.put( kv.getKey(), partLocation diff --git a/hive/src/test/java/com/netease/arctic/hive/op/TestRewriteFiles.java b/hive/src/test/java/com/netease/arctic/hive/op/TestRewriteFiles.java index 6589e3d0c3..98d83bb897 100644 --- a/hive/src/test/java/com/netease/arctic/hive/op/TestRewriteFiles.java +++ b/hive/src/test/java/com/netease/arctic/hive/op/TestRewriteFiles.java @@ -9,7 +9,7 @@ import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableIdentifier; import com.netease.arctic.table.UnkeyedTable; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import com.netease.arctic.utils.TablePropertyUtil; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.iceberg.DataFile; @@ -385,7 +385,7 @@ private void applyUpdateHiveFiles( deleteLocations.forEach(partitionAndLocations::remove); addFiles.forEach(kv -> { - String partLocation = FileUtil.getFileDir(kv.getValue()); + String partLocation = TableFileUtils.getFileDir(kv.getValue()); partitionAndLocations.put( kv.getKey(), partLocation diff --git a/hive/src/test/java/com/netease/arctic/hive/op/TestRewritePartitions.java b/hive/src/test/java/com/netease/arctic/hive/op/TestRewritePartitions.java index 972e02c0bb..20d6d5e59c 100644 --- a/hive/src/test/java/com/netease/arctic/hive/op/TestRewritePartitions.java +++ b/hive/src/test/java/com/netease/arctic/hive/op/TestRewritePartitions.java @@ -24,7 +24,7 @@ import com.netease.arctic.op.RewritePartitions; import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.UnkeyedTable; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.Transaction; @@ -178,7 +178,7 @@ private void applyRewritePartitions( Map partitionLocations, List> overwriteFiles) { overwriteFiles.forEach(kv -> { - String partLocation = FileUtil.getFileDir(kv.getValue()); + String partLocation = TableFileUtils.getFileDir(kv.getValue()); partitionLocations.put(kv.getKey(), partLocation); }); } diff --git a/hive/src/test/java/com/netease/arctic/hive/utils/HiveMetaSynchronizerTest.java b/hive/src/test/java/com/netease/arctic/hive/utils/HiveMetaSynchronizerTest.java index 603b28076f..fd4c2ef534 100644 --- a/hive/src/test/java/com/netease/arctic/hive/utils/HiveMetaSynchronizerTest.java +++ b/hive/src/test/java/com/netease/arctic/hive/utils/HiveMetaSynchronizerTest.java @@ -31,11 +31,10 @@ import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.LocationKind; import com.netease.arctic.table.UnkeyedTable; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; @@ -122,7 +121,7 @@ public void testSyncDataToHive() throws IOException, TException { Assert.assertEquals(1, newFiles.size()); partition3FilePath = newFiles.get(0).path().toString(); Partition newPartition = HivePartitionUtil.newPartition(hiveTable, Lists.newArrayList("p3"), - FileUtil.getFileDir(newFiles.get(0).path().toString()), newFiles, + TableFileUtils.getFileDir(newFiles.get(0).path().toString()), newFiles, (int) (System.currentTimeMillis() / 1000)); newPartition.getParameters().remove(HiveTableProperties.ARCTIC_TABLE_FLAG); hms.getClient().add_partition(newPartition); 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 c67133fb04..24cf9fe443 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 @@ -40,7 +40,7 @@ import com.netease.arctic.optimizer.operator.executor.TableIdentificationInfo; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; import com.netease.arctic.utils.TableTypeUtil; import org.apache.commons.beanutils.BeanUtils; import org.apache.commons.collections.CollectionUtils; @@ -229,29 +229,29 @@ private NodeTask constructTask(ArcticTable table, OptimizeTask task, int attempt if (TableTypeUtil.isIcebergTableFormat(table)) { for (ByteBuffer file : task.getBaseFiles()) { - nodeTask.addFile(SerializationUtil.toIcebergContentFile(file), DataFileType.BASE_FILE); + nodeTask.addFile(SerializationUtils.toIcebergContentFile(file), DataFileType.BASE_FILE); } for (ByteBuffer file : task.getInsertFiles()) { - nodeTask.addFile(SerializationUtil.toIcebergContentFile(file), DataFileType.INSERT_FILE); + nodeTask.addFile(SerializationUtils.toIcebergContentFile(file), DataFileType.INSERT_FILE); } for (ByteBuffer file : task.getDeleteFiles()) { - nodeTask.addFile(SerializationUtil.toIcebergContentFile(file), DataFileType.EQ_DELETE_FILE); + nodeTask.addFile(SerializationUtils.toIcebergContentFile(file), DataFileType.EQ_DELETE_FILE); } for (ByteBuffer file : task.getPosDeleteFiles()) { - nodeTask.addFile(SerializationUtil.toIcebergContentFile(file), DataFileType.POS_DELETE_FILE); + nodeTask.addFile(SerializationUtils.toIcebergContentFile(file), DataFileType.POS_DELETE_FILE); } } else { for (ByteBuffer file : task.getBaseFiles()) { - nodeTask.addFile(SerializationUtil.toInternalTableFile(file), DataFileType.BASE_FILE); + nodeTask.addFile(SerializationUtils.toInternalTableFile(file), DataFileType.BASE_FILE); } for (ByteBuffer file : task.getInsertFiles()) { - nodeTask.addFile(SerializationUtil.toInternalTableFile(file), DataFileType.INSERT_FILE); + nodeTask.addFile(SerializationUtils.toInternalTableFile(file), DataFileType.INSERT_FILE); } for (ByteBuffer file : task.getDeleteFiles()) { - nodeTask.addFile(SerializationUtil.toInternalTableFile(file), DataFileType.EQ_DELETE_FILE); + nodeTask.addFile(SerializationUtils.toInternalTableFile(file), DataFileType.EQ_DELETE_FILE); } for (ByteBuffer file : task.getPosDeleteFiles()) { - nodeTask.addFile(SerializationUtil.toInternalTableFile(file), DataFileType.POS_DELETE_FILE); + nodeTask.addFile(SerializationUtils.toInternalTableFile(file), DataFileType.POS_DELETE_FILE); } } diff --git a/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/BaseExecutor.java b/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/BaseExecutor.java index 8ae8431d04..69b60375ac 100644 --- a/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/BaseExecutor.java +++ b/optimizer/src/main/java/com/netease/arctic/optimizer/operator/executor/BaseExecutor.java @@ -27,8 +27,8 @@ import com.netease.arctic.optimizer.exception.TimeoutException; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.TableProperties; -import com.netease.arctic.utils.FileUtil; -import com.netease.arctic.utils.SerializationUtil; +import com.netease.arctic.utils.SerializationUtils; +import com.netease.arctic.utils.TableFileUtils; import org.apache.commons.beanutils.BeanUtils; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -65,17 +65,17 @@ public BaseExecutor(NodeTask task, ArcticTable table, long startTime, OptimizerC protected Map> groupDataFilesByNode(List dataFiles) { return new HashMap<>(dataFiles.stream().collect(Collectors.groupingBy(dataFile -> - FileUtil.parseFileNodeFromFileName(dataFile.path().toString())))); + TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())))); } protected Map> groupDeleteFilesByNode(List deleteFiles) { return new HashMap<>(deleteFiles.stream().collect(Collectors.groupingBy(deleteFile -> - FileUtil.parseFileNodeFromFileName(deleteFile.path().toString())))); + TableFileUtils.parseFileNodeFromFileName(deleteFile.path().toString())))); } protected long getMaxTransactionId(List dataFiles) { OptionalLong maxTransactionId = dataFiles.stream() - .mapToLong(file -> FileUtil.parseFileTidFromFileName(file.path().toString())).max(); + .mapToLong(file -> TableFileUtils.parseFileTidFromFileName(file.path().toString())).max(); if (maxTransactionId.isPresent()) { return maxTransactionId.getAsLong(); } @@ -89,7 +89,7 @@ protected OptimizeTaskResult buildOptimizeResult(Iterable baseFileBytesList = new ArrayList<>(); for (ContentFile targetFile : targetFiles) { totalFileSize += targetFile.fileSizeInBytes(); - baseFileBytesList.add(SerializationUtil.toByteBuffer(targetFile)); + baseFileBytesList.add(SerializationUtils.toByteBuffer(targetFile)); } OptimizeTaskStat optimizeTaskStat = new OptimizeTaskStat(); 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 index cad8bc9e80..feb12607f7 100644 --- 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 @@ -28,7 +28,7 @@ import com.netease.arctic.table.BaseLocationKind; import com.netease.arctic.table.UnkeyedTable; import com.netease.arctic.table.WriteOperationKind; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; @@ -112,7 +112,7 @@ default List insertBasePosDeleteFiles(ArcticTable arcticTable, List partitionFiles = dataFilePartitionMap.getValue(); Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() .collect(Collectors.groupingBy(dataFile -> - FileUtil.parseFileNodeFromFileName(dataFile.path().toString())))); + TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())))); for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { DataTreeNode key = nodeFilePartitionMap.getKey(); List nodeFiles = nodeFilePartitionMap.getValue(); @@ -153,7 +153,7 @@ default List insertOptimizeTargetDeleteFiles(ArcticTable arcticTable List partitionFiles = dataFilePartitionMap.getValue(); Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() .collect(Collectors.groupingBy(dataFile -> - FileUtil.parseFileNodeFromFileName(dataFile.path().toString())))); + TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())))); for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { DataTreeNode key = nodeFilePartitionMap.getKey(); List nodeFiles = nodeFilePartitionMap.getValue(); diff --git a/optimizer/src/test/java/com/netease/arctic/optimizer/util/DataFileInfoUtils.java b/optimizer/src/test/java/com/netease/arctic/optimizer/util/DataFileInfoUtils.java index fe775652ab..3e3bf5cce1 100644 --- a/optimizer/src/test/java/com/netease/arctic/optimizer/util/DataFileInfoUtils.java +++ b/optimizer/src/test/java/com/netease/arctic/optimizer/util/DataFileInfoUtils.java @@ -26,7 +26,7 @@ import com.netease.arctic.data.PrimaryKeyedFile; import com.netease.arctic.table.ArcticTable; import com.netease.arctic.table.KeyedTable; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.PartitionField; @@ -68,7 +68,7 @@ public static DataFileInfo convertToDatafileInfo(DeleteFile deleteFile, Snapshot dataFileInfo.setSpecId(keyedTable.spec().specId()); dataFileInfo.setRecordCount(deleteFile.recordCount()); dataFileInfo.setType(DataFileType.POS_DELETE_FILE.name()); - DataTreeNode node = FileUtil.parseFileNodeFromFileName(deleteFile.path().toString()); + DataTreeNode node = TableFileUtils.parseFileNodeFromFileName(deleteFile.path().toString()); dataFileInfo.setIndex(node.getIndex()); dataFileInfo.setMask(node.getMask()); dataFileInfo.setCommitTime(snapshot.timestampMillis()); diff --git a/spark/v2.3/spark/src/test/java/com/netease/arctic/spark/SparkTestContext.java b/spark/v2.3/spark/src/test/java/com/netease/arctic/spark/SparkTestContext.java index 458d9d5e07..6c92194762 100644 --- a/spark/v2.3/spark/src/test/java/com/netease/arctic/spark/SparkTestContext.java +++ b/spark/v2.3/spark/src/test/java/com/netease/arctic/spark/SparkTestContext.java @@ -38,9 +38,8 @@ import com.netease.arctic.table.LocationKind; import com.netease.arctic.table.TableIdentifier; import com.netease.arctic.table.UnkeyedTable; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import com.netease.arctic.utils.TablePropertyUtil; -import org.apache.commons.io.FileUtils; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.*; @@ -108,7 +107,7 @@ public static Map setUpTestDirAndArctic() throws IOException { } System.out.println("======================== start AMS ========================= "); - FileUtils.deleteQuietly(testBaseDir); + org.apache.commons.io.FileUtils.deleteQuietly(testBaseDir); testBaseDir.mkdirs(); AmsClientPools.cleanAll(); @@ -485,7 +484,7 @@ public void adaptHiveInsertPosDeleteFiles(Long transactionId, List dat List partitionFiles = dataFilePartitionMap.getValue(); Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() .collect(Collectors.groupingBy(dataFile -> - FileUtil.parseFileNodeFromFileName(dataFile.path().toString())))); + TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())))); for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { DataTreeNode key = nodeFilePartitionMap.getKey(); List nodeFiles = nodeFilePartitionMap.getValue(); diff --git a/spark/v3.1/spark/src/test/java/com/netease/arctic/spark/SparkTestContext.java b/spark/v3.1/spark/src/test/java/com/netease/arctic/spark/SparkTestContext.java index 311ef2677a..9dca61a46f 100644 --- a/spark/v3.1/spark/src/test/java/com/netease/arctic/spark/SparkTestContext.java +++ b/spark/v3.1/spark/src/test/java/com/netease/arctic/spark/SparkTestContext.java @@ -39,9 +39,8 @@ import com.netease.arctic.table.LocationKind; import com.netease.arctic.table.TableIdentifier; import com.netease.arctic.table.UnkeyedTable; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import com.netease.arctic.utils.TablePropertyUtil; -import org.apache.commons.io.FileUtils; import org.apache.commons.lang.StringUtils; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.AppendFiles; @@ -126,7 +125,7 @@ public static Map setUpTestDirAndArctic() throws IOException { } System.out.println("======================== start AMS ========================= "); - FileUtils.deleteQuietly(testBaseDir); + org.apache.commons.io.FileUtils.deleteQuietly(testBaseDir); testBaseDir.mkdirs(); AmsClientPools.cleanAll(); @@ -511,7 +510,7 @@ public void adaptHiveInsertPosDeleteFiles(Long transactionId, List dat List partitionFiles = dataFilePartitionMap.getValue(); Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() .collect(Collectors.groupingBy(dataFile -> - FileUtil.parseFileNodeFromFileName(dataFile.path().toString())))); + TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())))); for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { DataTreeNode key = nodeFilePartitionMap.getKey(); List nodeFiles = nodeFilePartitionMap.getValue(); diff --git a/spark/v3.1/spark/src/test/java/com/netease/arctic/spark/TestKeyedTableDML.java b/spark/v3.1/spark/src/test/java/com/netease/arctic/spark/TestKeyedTableDML.java index 7819308241..f66efe25f7 100644 --- a/spark/v3.1/spark/src/test/java/com/netease/arctic/spark/TestKeyedTableDML.java +++ b/spark/v3.1/spark/src/test/java/com/netease/arctic/spark/TestKeyedTableDML.java @@ -25,7 +25,7 @@ import com.netease.arctic.io.writer.SortedPosDeleteWriter; import com.netease.arctic.table.KeyedTable; import com.netease.arctic.table.TableIdentifier; -import com.netease.arctic.utils.FileUtil; +import com.netease.arctic.utils.TableFileUtils; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -147,7 +147,7 @@ protected void insertBasePosDeleteFiles(long transactionId, List dataF List partitionFiles = dataFilePartitionMap.getValue(); Map> nodeFilesPartitionMap = new HashMap<>(partitionFiles.stream() .collect(Collectors.groupingBy(dataFile -> - FileUtil.parseFileNodeFromFileName(dataFile.path().toString())))); + TableFileUtils.parseFileNodeFromFileName(dataFile.path().toString())))); for (Map.Entry> nodeFilePartitionMap : nodeFilesPartitionMap.entrySet()) { DataTreeNode key = nodeFilePartitionMap.getKey(); List nodeFiles = nodeFilePartitionMap.getValue();