1
0

[HUDI-839] Introducing support for rollbacks using marker files (#1756)

* [HUDI-839] Introducing rollback strategy using marker files

 - Adds a new mechanism for rollbacks where it's based on the marker files generated during the write
 - Consequently, marker file/dir deletion now happens post commit, instead of during finalize 
 - Marker files are also generated for AppendHandle, making it consistent throughout the write path 
 - Until upgrade-downgrade mechanism can upgrade non-marker based inflight writes to marker based, this should only be turned on for new datasets.
 - Added marker dir deletion after successful commit/rollback, individual files are not deleted during finalize
 - Fail safe for deleting marker directories, now during timeline archival process
 - Added check to ensure completed instants are not rolled back using marker based strategy. This will be incorrect
 - Reworked tests to rollback inflight instants, instead of completed instants whenever necessary
 - Added an unit test for MarkerBasedRollbackStrategy


Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
lw0090
2020-07-21 13:41:42 +08:00
committed by GitHub
parent b71f25f210
commit 1ec89e9a94
43 changed files with 1947 additions and 512 deletions

View File

@@ -18,6 +18,10 @@
package org.apache.hudi.common.testutils;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hudi.common.fs.inline.InLineFSUtils;
import org.apache.hudi.common.fs.inline.InLineFileSystem;
import org.apache.hudi.common.fs.inline.InMemoryFileSystem;
@@ -26,6 +30,8 @@ import org.apache.hadoop.fs.Path;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
import java.util.UUID;
@@ -67,4 +73,13 @@ public class FileSystemTestUtils {
throw new IOException(message);
}
}
public static List<FileStatus> listRecursive(FileSystem fs, Path path) throws IOException {
RemoteIterator<LocatedFileStatus> itr = fs.listFiles(path, true);
List<FileStatus> statuses = new ArrayList<>();
while (itr.hasNext()) {
statuses.add(itr.next());
}
return statuses;
}
}

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.SyncableFileSystemView;
import org.apache.hudi.exception.HoodieIOException;
import org.junit.jupiter.api.io.TempDir;
import java.io.IOException;
@@ -77,6 +78,17 @@ public class HoodieCommonTestHarness {
return getFileSystemView(timeline, true);
}
protected SyncableFileSystemView getFileSystemViewWithUnCommittedSlices(HoodieTableMetaClient metaClient) {
try {
return new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline(),
HoodieTestUtils.listAllDataFilesAndLogFilesInPath(metaClient.getFs(), metaClient.getBasePath())
);
} catch (IOException ioe) {
throw new HoodieIOException("Error getting file system view", ioe);
}
}
/**
* Gets a default {@link HoodieTableType#COPY_ON_WRITE} table type. Sub-classes can override this method to specify a
* new table type.

View File

@@ -18,6 +18,12 @@
package org.apache.hudi.common.testutils;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieActionInstant;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
@@ -60,12 +66,6 @@ import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.util.StringUtils;
import java.io.ByteArrayInputStream;
@@ -268,12 +268,6 @@ public class HoodieTestUtils {
return createDataFileFixLength(basePath, partitionPath, instantTime, fileID, length);
}
public static String createNewMarkerFile(String basePath, String partitionPath, String instantTime)
throws IOException {
String fileID = UUID.randomUUID().toString();
return createMarkerFile(basePath, partitionPath, instantTime, fileID);
}
public static String createDataFile(String basePath, String partitionPath, String instantTime, String fileID)
throws IOException {
String folderPath = basePath + "/" + partitionPath + "/";
@@ -294,16 +288,6 @@ public class HoodieTestUtils {
return fileID;
}
public static String createMarkerFile(String basePath, String partitionPath, String instantTime, String fileID)
throws IOException {
String folderPath =
basePath + "/" + HoodieTableMetaClient.TEMPFOLDER_NAME + "/" + instantTime + "/" + partitionPath + "/";
new File(folderPath).mkdirs();
File f = new File(folderPath + FSUtils.makeMarkerFile(instantTime, DEFAULT_WRITE_TOKEN, fileID));
f.createNewFile();
return f.getAbsolutePath();
}
public static String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String instantTime,
String fileID, Option<Integer> version) throws IOException {
String folderPath = basePath + "/" + partitionPath + "/";
@@ -465,7 +449,7 @@ public class HoodieTestUtils {
// TODO: should be removed
public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath) throws IOException {
return listAllDataFilesInPath(fs, basePath, ".parquet");
return listAllDataFilesInPath(fs, basePath, HoodieFileFormat.PARQUET.getFileExtension());
}
public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath, String datafileExtension)
@@ -474,26 +458,31 @@ public class HoodieTestUtils {
List<FileStatus> returns = new ArrayList<>();
while (itr.hasNext()) {
LocatedFileStatus status = itr.next();
if (status.getPath().getName().contains(datafileExtension)) {
if (status.getPath().getName().contains(datafileExtension) && !status.getPath().getName().contains(".marker")) {
returns.add(status);
}
}
return returns.toArray(new FileStatus[returns.size()]);
}
public static FileStatus[] listAllLogFilesInPath(FileSystem fs, String basePath, String logfileExtension)
public static FileStatus[] listAllLogFilesInPath(FileSystem fs, String basePath)
throws IOException {
RemoteIterator<LocatedFileStatus> itr = fs.listFiles(new Path(basePath), true);
List<FileStatus> returns = new ArrayList<>();
while (itr.hasNext()) {
LocatedFileStatus status = itr.next();
if (status.getPath().getName().contains(logfileExtension)) {
if (status.getPath().getName().contains(HoodieFileFormat.HOODIE_LOG.getFileExtension())) {
returns.add(status);
}
}
return returns.toArray(new FileStatus[returns.size()]);
}
public static FileStatus[] listAllDataFilesAndLogFilesInPath(FileSystem fs, String basePath) throws IOException {
return Stream.concat(Arrays.stream(listAllDataFilesInPath(fs, basePath)), Arrays.stream(listAllLogFilesInPath(fs, basePath)))
.toArray(FileStatus[]::new);
}
public static List<String> monotonicIncreasingCommitTimestamps(int numTimestamps, int startSecsDelta) {
Calendar cal = Calendar.getInstance();
cal.add(Calendar.SECOND, startSecsDelta);