Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[ARCTIC-1105][AMS] Fix snapshot expire and orphan clean misdelete the DataFiles #1122

Merged
merged 5 commits into from
Feb 22, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,19 +19,17 @@
package com.netease.arctic.ams.server.service.impl;

import com.netease.arctic.ams.api.Constants;
import com.netease.arctic.ams.api.DataFileInfo;
import com.netease.arctic.ams.server.service.IOrphanFilesCleanService;
import com.netease.arctic.ams.server.service.ServiceContainer;
import com.netease.arctic.ams.server.utils.CatalogUtil;
import com.netease.arctic.ams.server.utils.HiveLocationUtils;
import com.netease.arctic.ams.server.utils.ScheduledTasks;
import com.netease.arctic.ams.server.utils.ThreadPool;
import com.netease.arctic.ams.server.utils.UnKeyedTableUtil;
import com.netease.arctic.catalog.ArcticCatalog;
import com.netease.arctic.catalog.CatalogLoader;
import com.netease.arctic.io.ArcticFileIO;
import com.netease.arctic.table.ArcticTable;
import com.netease.arctic.table.BaseTable;
import com.netease.arctic.table.ChangeTable;
import com.netease.arctic.table.KeyedTable;
import com.netease.arctic.table.TableIdentifier;
import com.netease.arctic.table.TableProperties;
Expand All @@ -40,11 +38,9 @@
import com.netease.arctic.utils.TableFileUtils;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.iceberg.FileScanTask;
import org.apache.iceberg.ManifestFile;
import org.apache.iceberg.ReachableFileUtil;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -56,7 +52,6 @@
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;

public class OrphanFilesCleanService implements IOrphanFilesCleanService {
private static final Logger LOG = LoggerFactory.getLogger(OrphanFilesCleanService.class);
Expand Down Expand Up @@ -125,61 +120,50 @@ public void run() {

public static void clean(ArcticTable arcticTable, long lastTime, boolean execute,
String mode, boolean metadata) {
Set<String> validFiles = new HashSet<>();
// For clean data files, should get valid files in the base store and the change store, so acquire in advance
// to prevent repeated acquisition
if (!metadata) {
validFiles = getValidDataFiles(arcticTable);
}
if (arcticTable.isKeyedTable()) {
KeyedTable keyedArcticTable = arcticTable.asKeyedTable();
if (Constants.INNER_TABLE_BASE.equals(mode)) {
clearInternalTable(keyedArcticTable, keyedArcticTable.baseTable(), lastTime, execute, metadata);
clearInternalTable(keyedArcticTable, keyedArcticTable.baseTable(), lastTime, execute, metadata, validFiles);
} else if (Constants.INNER_TABLE_CHANGE.equals(mode)) {
if (keyedArcticTable.primaryKeySpec().primaryKeyExisted()) {
clearInternalTable(keyedArcticTable, keyedArcticTable.changeTable(), lastTime, execute, metadata);
clearInternalTable(keyedArcticTable, keyedArcticTable.changeTable(), lastTime, execute, metadata, validFiles);
} else {
throw new IllegalStateException("no pk table, only support mode=all/base");
}
} else if ("all".equals(mode)) {
clearInternalTable(keyedArcticTable, keyedArcticTable.baseTable(), lastTime, execute, metadata);
clearInternalTable(keyedArcticTable, keyedArcticTable.changeTable(), lastTime, execute, metadata);
clearInternalTable(keyedArcticTable, keyedArcticTable.baseTable(), lastTime, execute, metadata, validFiles);
clearInternalTable(keyedArcticTable, keyedArcticTable.changeTable(), lastTime, execute, metadata, validFiles);
} else {
throw new IllegalStateException("only support mode=all/base/change");
}
} else {
clearInternalTable(arcticTable, arcticTable.asUnkeyedTable(), lastTime, execute, metadata);
clearInternalTable(arcticTable, arcticTable.asUnkeyedTable(), lastTime, execute, metadata, validFiles);
}
}

private static void clearInternalTable(ArcticTable table, UnkeyedTable internalTable, long lastTime,
boolean execute, boolean metadata) {
boolean execute, boolean metadata, Set<String> exclude) {
if (metadata) {
clearInternalTableMetadata(table, internalTable, lastTime, execute);
} else {
clearInternalTableDataFiles(table, internalTable, lastTime, execute);
clearInternalTableDataFiles(table, internalTable, lastTime, execute, exclude);
}
}

private static void clearInternalTableDataFiles(ArcticTable table, UnkeyedTable internalTable, long lastTime,
boolean execute) {
Set<String> validFiles = getValidDataFiles(table.id(), table.io(), internalTable);
LOG.info("{} table get {} valid files", table.id(), validFiles.size());
boolean execute, Set<String> exclude) {
int deleteFilesCnt = 0;
Set<String> exclude = new HashSet<>();
if (internalTable instanceof BaseTable) {
List<DataFileInfo> dataFilesInfo = ServiceContainer.getFileInfoCacheService()
.getOptimizeDatafiles(table.id().buildTableIdentifier(), Constants.INNER_TABLE_CHANGE);
exclude = dataFilesInfo.stream().map(DataFileInfo::getPath).collect(Collectors.toSet());
} else if (internalTable instanceof ChangeTable) {
List<DataFileInfo> dataFilesInfo = ServiceContainer.getFileInfoCacheService()
.getOptimizeDatafiles(table.id().buildTableIdentifier(), Constants.INNER_TABLE_BASE);
exclude = dataFilesInfo.stream().map(DataFileInfo::getPath).collect(Collectors.toSet());
}

// add hive location to exclude
exclude.addAll(HiveLocationUtils.getHiveLocation(table));

String dataLocation = internalTable.location() + File.separator + DATA_FOLDER_NAME;
if (table.io().exists(dataLocation)) {
for (FileStatus fileStatus : table.io().list(dataLocation)) {
deleteFilesCnt += deleteInvalidDataFiles(table.io(),
fileStatus,
validFiles,
lastTime,
exclude,
execute);
Expand Down Expand Up @@ -212,7 +196,6 @@ private static String formatTime(long timestamp) {

private static int deleteInvalidDataFiles(ArcticFileIO io,
FileStatus fileStatus,
Set<String> validFiles,
Long lastTime,
Set<String> exclude,
boolean execute) {
Expand All @@ -222,7 +205,7 @@ private static int deleteInvalidDataFiles(ArcticFileIO io,
LOG.info("start orphan files clean in {}", location);
int deleteFileCnt = 0;
for (FileStatus file : io.list(location)) {
deleteFileCnt += deleteInvalidDataFiles(io, file, validFiles, lastTime, exclude, execute);
deleteFileCnt += deleteInvalidDataFiles(io, file, lastTime, exclude, execute);
}
LOG.info("delete[{}] {} files in {}", execute, deleteFileCnt, location);

Expand All @@ -247,17 +230,15 @@ private static int deleteInvalidDataFiles(ArcticFileIO io,
return 0;
}
} else {
if (!validFiles.contains(location) &&
if (execute &&
!exclude.contains(location) &&
!exclude.contains(TableFileUtils.getUriPath(new Path(location).getParent().toString())) &&
fileStatus.getModificationTime() < lastTime) {
if (execute &&
!exclude.contains(location) &&
!exclude.contains(new Path(location).getParent().toString())) {
io.deleteFile(location);
}
io.deleteFile(location);
return 1;
} else {
return 0;
}

return 0;
}
}

Expand Down Expand Up @@ -313,34 +294,28 @@ private static Set<String> getValidMetadataFiles(TableIdentifier tableIdentifier
return validFiles;
}

private static Set<String> getValidDataFiles(TableIdentifier tableIdentifier, ArcticFileIO io,
UnkeyedTable internalTable) {
private static Set<String> getValidDataFiles(ArcticTable arcticTable) {
Set<String> validFiles = new HashSet<>();
Iterable<Snapshot> snapshots = internalTable.snapshots();
int size = Iterables.size(snapshots);
LOG.info("{} get {} snapshots to scan", tableIdentifier, size);
int cnt = 0;
for (Snapshot snapshot : snapshots) {
cnt++;
int before = validFiles.size();
io.doAs(() -> {
// valid data files

try (CloseableIterable<FileScanTask> fileScanTasks =
internalTable.newScan().useSnapshot(snapshot.snapshotId()).planFiles()) {
for (FileScanTask scanTask : fileScanTasks) {
if (scanTask.file() != null) {
validFiles.add(TableFileUtils.getUriPath(scanTask.file().path().toString()));
scanTask.deletes().forEach(file -> validFiles.add(TableFileUtils.getUriPath(file.path().toString())));
}
}
return null;
}
});
LOG.info("{} scan snapshot {}: {} and get {} files, complete {}/{}", tableIdentifier, snapshot.snapshotId(),
formatTime(snapshot.timestampMillis()), validFiles.size() - before, cnt, size);
if (arcticTable.isKeyedTable()) {
Set<String> baseValidFiles = UnKeyedTableUtil.getAllContentFilePath(arcticTable.asKeyedTable().baseTable());
LOG.info("{} get {} valid files in the base store", arcticTable.id(), baseValidFiles.size());
Set<String> changeValidFiles = UnKeyedTableUtil.getAllContentFilePath(arcticTable.asKeyedTable().changeTable());
LOG.info("{} get {} valid files in the change store", arcticTable.id(), baseValidFiles.size());
validFiles.addAll(baseValidFiles);
validFiles.addAll(changeValidFiles);
} else {
Set<String> baseValidFiles = UnKeyedTableUtil.getAllContentFilePath(arcticTable.asUnkeyedTable());
LOG.info("{} get {} valid files in the base store", arcticTable.id(), baseValidFiles.size());
validFiles.addAll(baseValidFiles);
}

LOG.info("{} get {} valid files", arcticTable.id(), validFiles.size());

// add hive location to exclude
Set<String> hiveValidLocations = HiveLocationUtils.getHiveLocation(arcticTable);
LOG.info("{} get {} valid locations in the Hive", arcticTable.id(), hiveValidLocations.size());
validFiles.addAll(hiveValidLocations);

return validFiles;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@

package com.netease.arctic.ams.server.service.impl;

import com.netease.arctic.ams.api.Constants;
import com.netease.arctic.ams.api.DataFileInfo;
import com.netease.arctic.ams.server.service.ITableExpireService;
import com.netease.arctic.ams.server.service.ServiceContainer;
Expand All @@ -28,10 +27,9 @@
import com.netease.arctic.ams.server.utils.HiveLocationUtils;
import com.netease.arctic.ams.server.utils.ScheduledTasks;
import com.netease.arctic.ams.server.utils.ThreadPool;
import com.netease.arctic.ams.server.utils.UnKeyedTableUtil;
import com.netease.arctic.catalog.ArcticCatalog;
import com.netease.arctic.catalog.CatalogLoader;
import com.netease.arctic.data.DefaultKeyedFile;
import com.netease.arctic.data.PrimaryKeyedFile;
import com.netease.arctic.hive.utils.TableTypeUtil;
import com.netease.arctic.table.ArcticTable;
import com.netease.arctic.table.KeyedTable;
Expand Down Expand Up @@ -129,27 +127,29 @@ public void run() {
LOG.warn("[{}] Base table is null: {} ", traceId, tableIdentifier);
return null;
}
List<DataFileInfo> changeFilesInfo = ServiceContainer.getFileInfoCacheService()
.getOptimizeDatafiles(tableIdentifier.buildTableIdentifier(), Constants.INNER_TABLE_CHANGE);
Set<String> baseExclude = changeFilesInfo.stream().map(DataFileInfo::getPath).collect(Collectors.toSet());
baseExclude.addAll(finalHiveLocation);
expireSnapshots(baseTable, startTime - baseSnapshotsKeepTime, baseExclude);
long baseCleanedTime = System.currentTimeMillis();
LOG.info("[{}] {} base expire cost {} ms", traceId, arcticTable.id(), baseCleanedTime - startTime);

UnkeyedTable changeTable = keyedArcticTable.changeTable();
if (changeTable == null) {
LOG.warn("[{}] Change table is null: {}", traceId, tableIdentifier);
return null;
}

// get valid files in the change store which shouldn't physically delete when expire the snapshot
// in the base store
Set<String> baseExclude = UnKeyedTableUtil.getAllContentFilePath(changeTable);
baseExclude.addAll(finalHiveLocation);
expireSnapshots(baseTable, startTime - baseSnapshotsKeepTime, baseExclude);
long baseCleanedTime = System.currentTimeMillis();
LOG.info("[{}] {} base expire cost {} ms", traceId, arcticTable.id(), baseCleanedTime - startTime);

// delete ttl files
List<DataFileInfo> changeDataFiles = ServiceContainer.getFileInfoCacheService()
.getChangeTableTTLDataFiles(keyedArcticTable.id().buildTableIdentifier(),
System.currentTimeMillis() - changeDataTTL);
deleteChangeFile(keyedArcticTable, changeDataFiles);
List<DataFileInfo> baseFilesInfo = ServiceContainer.getFileInfoCacheService()
.getOptimizeDatafiles(tableIdentifier.buildTableIdentifier(), Constants.INNER_TABLE_BASE);
Set<String> changeExclude = baseFilesInfo.stream().map(DataFileInfo::getPath).collect(Collectors.toSet());

// get valid files in the base store which shouldn't physically delete when expire the snapshot
// in the change store
Set<String> changeExclude = UnKeyedTableUtil.getAllContentFilePath(baseTable);
changeExclude.addAll(finalHiveLocation);
expireSnapshots(changeTable, startTime - changeSnapshotsKeepTime, changeExclude);
return null;
Expand Down Expand Up @@ -237,7 +237,8 @@ public static void expireSnapshots(UnkeyedTable arcticInternalTable,
.retainLast(1).expireOlderThan(olderThan)
.deleteWith(file -> {
try {
if (!exclude.contains(file) && !exclude.contains(new Path(file).getParent().toString())) {
String filePath = TableFileUtils.getUriPath(file);
if (!exclude.contains(filePath) && !exclude.contains(new Path(filePath).getParent().toString())) {
arcticInternalTable.io().deleteFile(file);
}
parentDirectory.add(new Path(file).getParent().toString());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,9 @@
import org.apache.iceberg.FileMetadata;
import org.apache.iceberg.PartitionSpec;

/**
* Tools for handling the ContentFile which in Iceberg
*/
public class ContentFileUtil {

public static ContentFileWithSequence<?> buildContentFile(DataFileInfo dataFileInfo,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import com.netease.arctic.hive.table.SupportHive;
import com.netease.arctic.hive.utils.TableTypeUtil;
import com.netease.arctic.table.ArcticTable;
import com.netease.arctic.utils.TableFileUtils;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.hadoop.hive.metastore.api.Table;
import org.slf4j.Logger;
Expand All @@ -45,7 +46,7 @@ public static Set<String> getHiveLocation(ArcticTable table) {
try {
Table hiveTable = ((SupportHive) table).getHMSClient().run(client ->
client.getTable(table.id().getDatabase(), table.id().getTableName()));
hiveLocations.add(hiveTable.getSd().getLocation());
hiveLocations.add(TableFileUtils.getUriPath(hiveTable.getSd().getLocation()));
} catch (Exception e) {
LOG.error("{} get hive table error", table.id(), e);
throw new IllegalStateException("get hive table error", e);
Expand All @@ -55,7 +56,7 @@ public static Set<String> getHiveLocation(ArcticTable table) {
List<Partition> partitions = ((SupportHive) table).getHMSClient().run(client ->
client.listPartitions(table.id().getDatabase(), table.id().getTableName(), Short.MAX_VALUE));
for (Partition partition : partitions) {
hiveLocations.add(partition.getSd().getLocation());
hiveLocations.add(TableFileUtils.getUriPath(partition.getSd().getLocation()));
}
} catch (Exception e) {
LOG.error("{} get hive partitions error", table.id(), e);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,28 @@

import com.netease.arctic.ams.server.model.TableOptimizeRuntime;
import com.netease.arctic.table.UnkeyedTable;
import com.netease.arctic.utils.ManifestEntryFields;
import com.netease.arctic.utils.TableFileUtils;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.HasTableOperations;
import org.apache.iceberg.MetadataTableType;
import org.apache.iceberg.MetadataTableUtils;
import org.apache.iceberg.Snapshot;
import org.apache.iceberg.Table;
import org.apache.iceberg.data.GenericRecord;
import org.apache.iceberg.data.IcebergGenerics;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.io.CloseableIterable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.HashSet;
import java.util.Set;

public class UnKeyedTableUtil {
private static final Logger LOG = LoggerFactory.getLogger(UnKeyedTableUtil.class);

public static long getSnapshotId(UnkeyedTable internalTable) {
internalTable.refresh();
Snapshot currentSnapshot = internalTable.currentSnapshot();
Expand All @@ -37,4 +56,32 @@ public static Snapshot getCurrentSnapshot(UnkeyedTable internalTable) {
internalTable.refresh();
return internalTable.currentSnapshot();
}

public static Set<String> getAllContentFilePath(UnkeyedTable internalTable) {
Set<String> validFilesPath = new HashSet<>();

Table manifestTable =
MetadataTableUtils.createMetadataTableInstance(((HasTableOperations) internalTable).operations(),
internalTable.name(), metadataTableName(internalTable.name(), MetadataTableType.ALL_ENTRIES),
MetadataTableType.ALL_ENTRIES);
try (CloseableIterable<Record> entries = IcebergGenerics.read(manifestTable).build()) {
for (Record entry : entries) {
ManifestEntryFields.Status status =
ManifestEntryFields.Status.of((int) entry.get(ManifestEntryFields.STATUS.fieldId()));
if (status == ManifestEntryFields.Status.ADDED || status == ManifestEntryFields.Status.EXISTING) {
GenericRecord dataFile = (GenericRecord) entry.get(ManifestEntryFields.DATA_FILE_ID);
String filePath = (String) dataFile.getField(DataFile.FILE_PATH.name());
validFilesPath.add(TableFileUtils.getUriPath(filePath));
}
}
} catch (IOException e) {
LOG.error("close manifest file error", e);
}

return validFilesPath;
}

private static String metadataTableName(String tableName, MetadataTableType type) {
return tableName + (tableName.contains("/") ? "#" : ".") + type;
}
}
Loading