[HUDI-2906] Add a repair util to clean up dangling data and log files (#4278)
This commit is contained in:
@@ -18,10 +18,6 @@
|
||||
|
||||
package org.apache.hudi.common.table.timeline;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
@@ -32,10 +28,16 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
@@ -65,7 +67,7 @@ import java.util.stream.Collectors;
|
||||
*/
|
||||
public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
private static final Pattern ARCHIVE_FILE_PATTERN =
|
||||
Pattern.compile("^\\.commits_\\.archive\\.([0-9]*)$");
|
||||
Pattern.compile("^\\.commits_\\.archive\\.([0-9]*)$");
|
||||
|
||||
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE_PREFIX = "commits";
|
||||
private static final String ACTION_TYPE_KEY = "actionType";
|
||||
@@ -112,6 +114,11 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
loadInstants(startTs, endTs);
|
||||
}
|
||||
|
||||
public void loadCompletedInstantDetailsInMemory() {
|
||||
loadInstants(null, true,
|
||||
record -> HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STATE).toString()));
|
||||
}
|
||||
|
||||
public void loadCompactionDetailsInMemory(String compactionInstantTime) {
|
||||
loadCompactionDetailsInMemory(compactionInstantTime, compactionInstantTime);
|
||||
}
|
||||
@@ -143,15 +150,17 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
private HoodieInstant readCommit(GenericRecord record, boolean loadDetails) {
|
||||
final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString();
|
||||
final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString();
|
||||
final String action = record.get(ACTION_TYPE_KEY).toString();
|
||||
if (loadDetails) {
|
||||
getMetadataKey(action).map(key -> {
|
||||
Object actionData = record.get(key);
|
||||
if (action.equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
this.readCommits.put(instantTime, HoodieAvroUtils.indexedRecordToBytes((IndexedRecord)actionData));
|
||||
} else {
|
||||
this.readCommits.put(instantTime, actionData.toString().getBytes(StandardCharsets.UTF_8));
|
||||
if (actionData != null) {
|
||||
if (action.equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
this.readCommits.put(instantTime, HoodieAvroUtils.indexedRecordToBytes((IndexedRecord) actionData));
|
||||
} else {
|
||||
this.readCommits.put(instantTime, actionData.toString().getBytes(StandardCharsets.UTF_8));
|
||||
}
|
||||
}
|
||||
return null;
|
||||
});
|
||||
|
||||
@@ -18,6 +18,11 @@
|
||||
|
||||
package org.apache.hudi.common.util;
|
||||
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -98,6 +103,31 @@ public class FileIOUtils {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies the file content from source path to destination path.
|
||||
*
|
||||
* @param fileSystem {@link FileSystem} instance.
|
||||
* @param sourceFilePath Source file path.
|
||||
* @param destFilePath Destination file path.
|
||||
*/
|
||||
public static void copy(
|
||||
FileSystem fileSystem, org.apache.hadoop.fs.Path sourceFilePath,
|
||||
org.apache.hadoop.fs.Path destFilePath) {
|
||||
FSDataInputStream fsDataInputStream = null;
|
||||
FSDataOutputStream fsDataOutputStream = null;
|
||||
try {
|
||||
fsDataInputStream = fileSystem.open(sourceFilePath);
|
||||
fsDataOutputStream = fileSystem.create(destFilePath, false);
|
||||
copy(fsDataInputStream, fsDataOutputStream);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(String.format("Cannot copy from %s to %s",
|
||||
sourceFilePath.toString(), destFilePath.toString()), e);
|
||||
} finally {
|
||||
closeQuietly(fsDataInputStream);
|
||||
closeQuietly(fsDataOutputStream);
|
||||
}
|
||||
}
|
||||
|
||||
public static byte[] readAsByteArray(InputStream input) throws IOException {
|
||||
return readAsByteArray(input, 128);
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user