1
0

[HUDI-1646] Provide mechanism to read uncommitted data through InputFormat (#2611)

This commit is contained in:
satishkotha
2021-03-04 17:43:31 -08:00
committed by GitHub
parent 89003bc780
commit 7cc75e0be2
7 changed files with 109 additions and 6 deletions

View File

@@ -28,7 +28,9 @@ 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.apache.hudi.common.testutils.FileCreateUtils;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.hadoop.testutils.InputFormatTestUtil;
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
@@ -57,6 +59,7 @@ import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResou
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
public class TestHoodieParquetInputFormat {
@@ -429,6 +432,44 @@ public class TestHoodieParquetInputFormat {
}
@Test
public void testSnapshotPreCommitValidate() throws IOException {
// initial commit
File partitionDir = InputFormatTestUtil.prepareTable(basePath, baseFileFormat, 10, "100");
createCommitFile(basePath, "100", "2016/05/01");
// Add the paths
FileInputFormat.setInputPaths(jobConf, partitionDir.getPath());
FileStatus[] files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length, "Snapshot read must return all files in partition");
// add more files
InputFormatTestUtil.simulateInserts(partitionDir, baseFileExtension, "fileId2-", 5, "200");
FileCreateUtils.createInflightCommit(basePath.toString(), "200");
// Verify that validate mode reads uncommitted files
InputFormatTestUtil.setupSnapshotIncludePendingCommits(jobConf, "200");
files = inputFormat.listStatus(jobConf);
assertEquals(15, files.length, "Must return uncommitted files");
ensureFilesInCommit("Pulling 1 commit from 100, should get us the 5 files committed at 200", files, "200", 5);
ensureFilesInCommit("Pulling 1 commit from 100, should get us the 10 files committed at 100", files, "100", 10);
try {
// Verify that Validate mode throws error with invalid commit time
InputFormatTestUtil.setupSnapshotIncludePendingCommits(jobConf, "300");
inputFormat.listStatus(jobConf);
fail("Expected list status to fail when validate is called with unknown timestamp");
} catch (HoodieIOException e) {
// expected because validate is called with invalid instantTime
}
// verify that snapshot mode skips uncommitted files
InputFormatTestUtil.setupSnapshotScanMode(jobConf);
files = inputFormat.listStatus(jobConf);
assertEquals(10, files.length, "snapshot scan mode must not return uncommitted files");
ensureFilesInCommit("Pulling 1 commit from 100, should get us the 10 files committed at 100", files, "100", 10);
}
private void ensureRecordsInCommit(String msg, String commit, int expectedNumberOfRecordsInCommit,
int totalExpected) throws IOException {
int actualCount = 0;

View File

@@ -122,6 +122,26 @@ public class InputFormatTestUtil {
String.format(HoodieHiveUtils.HOODIE_MAX_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.setInt(maxCommitPulls, numberOfCommitsToPull);
}
public static void setupSnapshotIncludePendingCommits(JobConf jobConf, String instantTime) {
setupSnapshotScanMode(jobConf, true);
String validateTimestampName =
String.format(HoodieHiveUtils.HOODIE_CONSUME_COMMIT, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.set(validateTimestampName, instantTime);
}
public static void setupSnapshotScanMode(JobConf jobConf) {
setupSnapshotScanMode(jobConf, false);
}
private static void setupSnapshotScanMode(JobConf jobConf, boolean includePending) {
String modePropertyName =
String.format(HoodieHiveUtils.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.set(modePropertyName, HoodieHiveUtils.SNAPSHOT_SCAN_MODE);
String includePendingCommitsName =
String.format(HoodieHiveUtils.HOODIE_CONSUME_PENDING_COMMITS, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
jobConf.setBoolean(includePendingCommitsName, includePending);
}
public static File prepareParquetTable(java.nio.file.Path basePath, Schema schema, int numberOfFiles,
int numberOfRecords, String commitNumber) throws IOException {