[HUDI-687] Stop incremental reader on RO table when there is a pending compaction (#1396)
This commit is contained in:
@@ -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