[HUDI-687] Stop incremental reader on RO table when there is a pending compaction (#1396)
This commit is contained in:
@@ -38,6 +38,23 @@ public class HoodieHiveUtil {
|
||||
public static final String HOODIE_CONSUME_MODE_PATTERN = "hoodie.%s.consume.mode";
|
||||
public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp";
|
||||
public static final String HOODIE_MAX_COMMIT_PATTERN = "hoodie.%s.consume.max.commits";
|
||||
/*
|
||||
* Boolean property to stop incremental reader when there is a pending compaction.
|
||||
* This is needed to prevent certain race conditions with RO views of MOR tables. only applicable for RO views.
|
||||
*
|
||||
* example timeline:
|
||||
*
|
||||
* t0 -> create bucket1.parquet
|
||||
* t1 -> create and append updates bucket1.log
|
||||
* t2 -> request compaction
|
||||
* t3 -> create bucket2.parquet
|
||||
*
|
||||
* if compaction at t2 takes a long time, incremental readers on RO tables can move to t3 and would skip updates in t1
|
||||
*
|
||||
* To workaround this problem, we want to stop returning data belonging to commits > t2.
|
||||
* After compaction is complete, incremental reader would see updates in t2, t3, so on.
|
||||
*/
|
||||
public static final String HOODIE_STOP_AT_COMPACTION_PATTERN = "hoodie.%s.ro.stop.at.compaction";
|
||||
public static final String INCREMENTAL_SCAN_MODE = "INCREMENTAL";
|
||||
public static final String SNAPSHOT_SCAN_MODE = "SNAPSHOT";
|
||||
public static final String DEFAULT_SCAN_MODE = SNAPSHOT_SCAN_MODE;
|
||||
@@ -46,6 +63,13 @@ public class HoodieHiveUtil {
|
||||
public static final int DEFAULT_LEVELS_TO_BASEPATH = 3;
|
||||
public static final Pattern HOODIE_CONSUME_MODE_PATTERN_STRING = Pattern.compile("hoodie\\.(.*)\\.consume\\.mode");
|
||||
|
||||
public static boolean stopAtCompaction(JobContext job, String tableName) {
|
||||
String compactionPropName = String.format(HOODIE_STOP_AT_COMPACTION_PATTERN, tableName);
|
||||
boolean stopAtCompaction = job.getConfiguration().getBoolean(compactionPropName, true);
|
||||
LOG.info("Read stop at compaction - " + stopAtCompaction);
|
||||
return stopAtCompaction;
|
||||
}
|
||||
|
||||
public static Integer readMaxCommits(JobContext job, String tableName) {
|
||||
String maxCommitName = String.format(HOODIE_MAX_COMMIT_PATTERN, tableName);
|
||||
int maxCommits = job.getConfiguration().getInt(maxCommitName, DEFAULT_MAX_COMMITS);
|
||||
|
||||
@@ -18,17 +18,6 @@
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
@@ -42,6 +31,18 @@ import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hadoop.mapred.Reporter;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -118,6 +119,36 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
return returns.toArray(new FileStatus[returns.size()]);
|
||||
}
|
||||
|
||||
/**
|
||||
* Filter any specific instants that we do not want to process.
|
||||
* example timeline:
|
||||
*
|
||||
* t0 -> create bucket1.parquet
|
||||
* t1 -> create and append updates bucket1.log
|
||||
* t2 -> request compaction
|
||||
* t3 -> create bucket2.parquet
|
||||
*
|
||||
* if compaction at t2 takes a long time, incremental readers on RO tables can move to t3 and would skip updates in t1
|
||||
*
|
||||
* To workaround this problem, we want to stop returning data belonging to commits > t2.
|
||||
* After compaction is complete, incremental reader would see updates in t2, t3, so on.
|
||||
*/
|
||||
protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
|
||||
HoodieDefaultTimeline commitsAndCompactionTimeline = timeline.getCommitsAndCompactionTimeline();
|
||||
Option<HoodieInstant> pendingCompactionInstant = commitsAndCompactionTimeline.filterPendingCompactionTimeline().firstInstant();
|
||||
if (pendingCompactionInstant.isPresent()) {
|
||||
HoodieDefaultTimeline instantsTimeline = commitsAndCompactionTimeline.findInstantsBefore(pendingCompactionInstant.get().getTimestamp());
|
||||
int numCommitsFilteredByCompaction = commitsAndCompactionTimeline.getCommitsTimeline().countInstants()
|
||||
- instantsTimeline.getCommitsTimeline().countInstants();
|
||||
LOG.info("Earliest pending compaction instant is: " + pendingCompactionInstant.get().getTimestamp()
|
||||
+ " skipping " + numCommitsFilteredByCompaction + " commits");
|
||||
|
||||
return instantsTimeline;
|
||||
} else {
|
||||
return timeline;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Achieves listStatus functionality for an incrementally queried table. Instead of listing all
|
||||
* partitions and then filtering based on the commits of interest, this logic first extracts the
|
||||
@@ -126,10 +157,18 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
private List<FileStatus> listStatusForIncrementalMode(
|
||||
JobConf job, HoodieTableMetaClient tableMetaClient, List<Path> inputPaths) throws IOException {
|
||||
String tableName = tableMetaClient.getTableConfig().getTableName();
|
||||
HoodieTimeline timeline = tableMetaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName);
|
||||
Job jobContext = Job.getInstance(job);
|
||||
HoodieDefaultTimeline baseTimeline;
|
||||
if (HoodieHiveUtil.stopAtCompaction(jobContext, tableName)) {
|
||||
baseTimeline = filterInstantsTimeline(tableMetaClient.getActiveTimeline());
|
||||
} else {
|
||||
baseTimeline = tableMetaClient.getActiveTimeline();
|
||||
}
|
||||
|
||||
HoodieTimeline timeline = baseTimeline.getCommitsTimeline().filterCompletedInstants();
|
||||
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(jobContext, tableName);
|
||||
// Total number of commits to return in this batch. Set this to -1 to get all the commits.
|
||||
Integer maxCommits = HoodieHiveUtil.readMaxCommits(Job.getInstance(job), tableName);
|
||||
Integer maxCommits = HoodieHiveUtil.readMaxCommits(jobContext, tableName);
|
||||
LOG.info("Last Incremental timestamp was set as " + lastIncrementalTs);
|
||||
List<HoodieInstant> commitsToCheck = timeline.findInstantsAfter(lastIncrementalTs, maxCommits)
|
||||
.getInstants().collect(Collectors.toList());
|
||||
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
@@ -159,6 +160,12 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
|
||||
return super.listStatus(job);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) {
|
||||
// no specific filtering for Realtime format
|
||||
return timeline;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a field to the existing fields projected.
|
||||
*/
|
||||
|
||||
@@ -50,8 +50,13 @@ public class InputFormatTestUtil {
|
||||
basePath.create();
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath.getRoot().toString());
|
||||
File partitionPath = basePath.newFolder("2016", "05", "01");
|
||||
return simulateInserts(partitionPath, "fileId1", numberOfFiles, commitNumber);
|
||||
}
|
||||
|
||||
public static File simulateInserts(File partitionPath, String fileId, int numberOfFiles, String commitNumber)
|
||||
throws IOException {
|
||||
for (int i = 0; i < numberOfFiles; i++) {
|
||||
File dataFile = new File(partitionPath, FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, "fileid" + i));
|
||||
File dataFile = new File(partitionPath, FSUtils.makeDataFileName(commitNumber, TEST_WRITE_TOKEN, fileId + i));
|
||||
dataFile.createNewFile();
|
||||
}
|
||||
return partitionPath;
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.hadoop;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
@@ -32,6 +33,11 @@ import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
import org.apache.hadoop.mapreduce.Job;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
@@ -41,9 +47,11 @@ import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
public class TestHoodieParquetInputFormat {
|
||||
@@ -61,6 +69,67 @@ public class TestHoodieParquetInputFormat {
|
||||
@Rule
|
||||
public TemporaryFolder basePath = new TemporaryFolder();
|
||||
|
||||
// Verify that HoodieParquetInputFormat does not return instants after pending compaction
|
||||
@Test
|
||||
public void testPendingCompactionWithActiveCommits() throws IOException {
|
||||
// setup 4 sample instants in timeline
|
||||
List<HoodieInstant> instants = new ArrayList<>();
|
||||
HoodieInstant t1 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant t2 = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "2");
|
||||
HoodieInstant t3 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3");
|
||||
HoodieInstant t4 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "4");
|
||||
HoodieInstant t5 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5");
|
||||
HoodieInstant t6 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "6");
|
||||
|
||||
instants.add(t1);
|
||||
instants.add(t2);
|
||||
instants.add(t3);
|
||||
instants.add(t4);
|
||||
instants.add(t5);
|
||||
instants.add(t6);
|
||||
HoodieTableMetaClient metaClient = HoodieTestUtils.init(basePath.getRoot().getAbsolutePath().toString());
|
||||
HoodieActiveTimeline timeline = new HoodieActiveTimeline(metaClient);
|
||||
timeline.setInstants(instants);
|
||||
|
||||
// Verify getCommitsTimelineBeforePendingCompaction does not return instants after first compaction instant
|
||||
HoodieTimeline filteredTimeline = new HoodieParquetInputFormat().filterInstantsTimeline(timeline);
|
||||
assertTrue(filteredTimeline.containsInstant(t1));
|
||||
assertTrue(filteredTimeline.containsInstant(t2));
|
||||
assertFalse(filteredTimeline.containsInstant(t3));
|
||||
assertFalse(filteredTimeline.containsInstant(t4));
|
||||
assertFalse(filteredTimeline.containsInstant(t5));
|
||||
assertFalse(filteredTimeline.containsInstant(t6));
|
||||
|
||||
|
||||
// remove compaction instant and setup timeline again
|
||||
instants.remove(t3);
|
||||
timeline = new HoodieActiveTimeline(metaClient);
|
||||
timeline.setInstants(instants);
|
||||
filteredTimeline = new HoodieParquetInputFormat().filterInstantsTimeline(timeline);
|
||||
|
||||
// verify all remaining instants are returned.
|
||||
assertTrue(filteredTimeline.containsInstant(t1));
|
||||
assertTrue(filteredTimeline.containsInstant(t2));
|
||||
assertFalse(filteredTimeline.containsInstant(t3));
|
||||
assertTrue(filteredTimeline.containsInstant(t4));
|
||||
assertFalse(filteredTimeline.containsInstant(t5));
|
||||
assertFalse(filteredTimeline.containsInstant(t6));
|
||||
|
||||
// remove remaining compaction instant and setup timeline again
|
||||
instants.remove(t5);
|
||||
timeline = new HoodieActiveTimeline(metaClient);
|
||||
timeline.setInstants(instants);
|
||||
filteredTimeline = new HoodieParquetInputFormat().filterInstantsTimeline(timeline);
|
||||
|
||||
// verify all remaining instants are returned.
|
||||
assertTrue(filteredTimeline.containsInstant(t1));
|
||||
assertTrue(filteredTimeline.containsInstant(t2));
|
||||
assertFalse(filteredTimeline.containsInstant(t3));
|
||||
assertTrue(filteredTimeline.containsInstant(t4));
|
||||
assertFalse(filteredTimeline.containsInstant(t5));
|
||||
assertTrue(filteredTimeline.containsInstant(t6));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInputFormatLoad() throws IOException {
|
||||
// initial commit
|
||||
@@ -133,6 +202,22 @@ public class TestHoodieParquetInputFormat {
|
||||
fileOutputStream.close();
|
||||
}
|
||||
|
||||
private File createCompactionFile(TemporaryFolder basePath, String commitTime)
|
||||
throws IOException {
|
||||
File file = new File(basePath.getRoot().toString() + "/.hoodie/",
|
||||
HoodieTimeline.makeRequestedCompactionFileName(commitTime));
|
||||
assertTrue(file.createNewFile());
|
||||
FileOutputStream os = new FileOutputStream(file);
|
||||
try {
|
||||
HoodieCompactionPlan compactionPlan = HoodieCompactionPlan.newBuilder().setVersion(2).build();
|
||||
// Write empty commit metadata
|
||||
os.write(TimelineMetadataUtils.serializeCompactionPlan(compactionPlan).get());
|
||||
return file;
|
||||
} finally {
|
||||
os.close();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIncrementalWithMultipleCommits() throws IOException {
|
||||
// initial commit
|
||||
@@ -228,6 +313,43 @@ public class TestHoodieParquetInputFormat {
|
||||
}
|
||||
}
|
||||
|
||||
// test incremental read does not go past compaction instant for RO views
|
||||
@Test
|
||||
public void testIncrementalWithPendingCompaction() throws IOException {
|
||||
// initial commit
|
||||
File partitionDir = InputFormatTestUtil.prepareTable(basePath, 10, "100");
|
||||
createCommitFile(basePath, "100", "2016/05/01");
|
||||
|
||||
// simulate compaction requested at 300
|
||||
File compactionFile = createCompactionFile(basePath, "300");
|
||||
|
||||
// write inserts into new bucket
|
||||
InputFormatTestUtil.simulateInserts(partitionDir, "fileId2", 10, "400");
|
||||
createCommitFile(basePath, "400", "2016/05/01");
|
||||
|
||||
// Add the paths
|
||||
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
|
||||
InputFormatTestUtil.setupIncremental(jobConf, "0", -1);
|
||||
FileStatus[] files = inputFormat.listStatus(jobConf);
|
||||
assertEquals("Pulling all commit from beginning, should not return instants after begin compaction",
|
||||
10, files.length);
|
||||
ensureFilesInCommit("Pulling all commit from beginning, should not return instants after begin compaction",
|
||||
files, "100", 10);
|
||||
|
||||
// delete compaction and verify inserts show up
|
||||
compactionFile.delete();
|
||||
InputFormatTestUtil.setupIncremental(jobConf, "0", -1);
|
||||
files = inputFormat.listStatus(jobConf);
|
||||
assertEquals("after deleting compaction, should get all inserted files",
|
||||
20, files.length);
|
||||
|
||||
ensureFilesInCommit("Pulling all commit from beginning, should return instants before requested compaction",
|
||||
files, "100", 10);
|
||||
ensureFilesInCommit("Pulling all commit from beginning, should return instants after requested compaction",
|
||||
files, "400", 10);
|
||||
|
||||
}
|
||||
|
||||
private void ensureRecordsInCommit(String msg, String commit, int expectedNumberOfRecordsInCommit,
|
||||
int totalExpected) throws IOException {
|
||||
int actualCount = 0;
|
||||
|
||||
Reference in New Issue
Block a user