1
0

[HUDI-1486] Remove inline inflight rollback in hoodie writer (#2359)

1. Refactor rollback and move cleaning failed commits logic into cleaner
2. Introduce hoodie heartbeat to ascertain failed commits
3. Fix test cases
This commit is contained in:
n3nash
2021-02-19 20:12:22 -08:00
committed by GitHub
parent c9fcf964b2
commit ffcfb58bac
64 changed files with 1541 additions and 306 deletions

View File

@@ -178,7 +178,7 @@ public class HoodieInputFormatUtils {
* @return
*/
public static HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
HoodieDefaultTimeline commitsAndCompactionTimeline = timeline.getCommitsAndCompactionTimeline();
HoodieDefaultTimeline commitsAndCompactionTimeline = timeline.getWriteTimeline();
Option<HoodieInstant> pendingCompactionInstant = commitsAndCompactionTimeline
.filterPendingCompactionTimeline().firstInstant();
if (pendingCompactionInstant.isPresent()) {

View File

@@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.SchemaTestUtil;
import org.apache.hudi.common.util.collection.Pair;
@@ -126,9 +127,11 @@ public class TestHoodieRealtimeRecordReader {
Schema schema = HoodieAvroUtils.addMetadataFields(SchemaTestUtil.getEvolvedSchema());
HoodieTestUtils.init(hadoopConf, basePath.toString(), HoodieTableType.MERGE_ON_READ);
String baseInstant = "100";
File partitionDir = partitioned ? InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant)
: InputFormatTestUtil.prepareNonPartitionedParquetTable(basePath, schema, 1, 100, baseInstant);
InputFormatTestUtil.commit(basePath, baseInstant);
File partitionDir = partitioned ? InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, 100, baseInstant,
HoodieTableType.MERGE_ON_READ)
: InputFormatTestUtil.prepareNonPartitionedParquetTable(basePath, schema, 1, 100, baseInstant,
HoodieTableType.MERGE_ON_READ);
FileCreateUtils.createDeltaCommit(basePath.toString(), baseInstant);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -163,6 +166,7 @@ public class TestHoodieRealtimeRecordReader {
long size = writer.getCurrentSize();
writer.close();
assertTrue(size > 0, "block - size should be > 0");
FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
// create a split with baseFile (parquet file written earlier) and new log file(s)
fileSlice.addLogFile(writer.getLogFile());
@@ -214,8 +218,9 @@ public class TestHoodieRealtimeRecordReader {
final int numRecords = 1000;
final int firstBatchLastRecordKey = numRecords - 1;
final int secondBatchLastRecordKey = 2 * numRecords - 1;
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numRecords, instantTime);
InputFormatTestUtil.commit(basePath, instantTime);
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numRecords, instantTime,
HoodieTableType.MERGE_ON_READ);
FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -227,6 +232,7 @@ public class TestHoodieRealtimeRecordReader {
long size = writer.getCurrentSize();
writer.close();
assertTrue(size > 0, "block - size should be > 0");
FileCreateUtils.createDeltaCommit(basePath.toString(), newCommitTime);
// create a split with baseFile (parquet file written earlier) and new log file(s)
String logFilePath = writer.getLogFile().getPath().toString();
@@ -291,7 +297,8 @@ public class TestHoodieRealtimeRecordReader {
String instantTime = "100";
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numberOfRecords, instantTime);
File partitionDir = InputFormatTestUtil.prepareParquetTable(basePath, schema, 1, numberOfRecords,
instantTime, HoodieTableType.MERGE_ON_READ);
InputFormatTestUtil.commit(basePath, instantTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
@@ -421,7 +428,8 @@ public class TestHoodieRealtimeRecordReader {
int numberOfRecords = 100;
int numberOfLogRecords = numberOfRecords / 2;
File partitionDir =
InputFormatTestUtil.prepareSimpleParquetTable(basePath, schema, 1, numberOfRecords, instantTime);
InputFormatTestUtil.prepareSimpleParquetTable(basePath, schema, 1, numberOfRecords,
instantTime, HoodieTableType.MERGE_ON_READ);
InputFormatTestUtil.commit(basePath, instantTime);
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());

View File

@@ -125,7 +125,12 @@ public class InputFormatTestUtil {
public static File prepareParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws IOException {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString());
return prepareParquetTable(basePath, schema, numberOfFiles, numberOfRecords, commitNumber, HoodieTableType.COPY_ON_WRITE);
}
public static File prepareParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber, HoodieTableType tableType) throws IOException {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType);
java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01"));
createData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber);
return partitionPath.toFile();
@@ -133,7 +138,12 @@ public class InputFormatTestUtil {
public static File prepareSimpleParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws Exception {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString());
return prepareSimpleParquetTable(basePath, schema, numberOfFiles, numberOfRecords, commitNumber, HoodieTableType.COPY_ON_WRITE);
}
public static File prepareSimpleParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber, HoodieTableType tableType) throws Exception {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType);
java.nio.file.Path partitionPath = basePath.resolve(Paths.get("2016", "05", "01"));
createSimpleData(schema, partitionPath, numberOfFiles, numberOfRecords, commitNumber);
return partitionPath.toFile();
@@ -141,7 +151,12 @@ public class InputFormatTestUtil {
public static File prepareNonPartitionedParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws IOException {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString());
return prepareNonPartitionedParquetTable(basePath, schema, numberOfFiles, numberOfRecords, commitNumber, HoodieTableType.COPY_ON_WRITE);
}
public static File prepareNonPartitionedParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber, HoodieTableType tableType) throws IOException {
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.toString(), tableType);
createData(schema, basePath, numberOfFiles, numberOfRecords, commitNumber);
return basePath.toFile();
}