|
|
|
|
@@ -24,6 +24,7 @@ import org.apache.hudi.client.WriteStatus;
|
|
|
|
|
import org.apache.hudi.common.model.FileSlice;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieBaseFile;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieFileFormat;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieFileGroup;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieKey;
|
|
|
|
|
import org.apache.hudi.common.model.HoodieRecord;
|
|
|
|
|
@@ -46,7 +47,9 @@ import org.apache.hudi.config.HoodieCompactionConfig;
|
|
|
|
|
import org.apache.hudi.config.HoodieIndexConfig;
|
|
|
|
|
import org.apache.hudi.config.HoodieStorageConfig;
|
|
|
|
|
import org.apache.hudi.config.HoodieWriteConfig;
|
|
|
|
|
import org.apache.hudi.exception.HoodieIOException;
|
|
|
|
|
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
|
|
|
|
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
|
|
|
|
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
|
|
|
|
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
|
|
|
|
|
import org.apache.hudi.index.HoodieIndex;
|
|
|
|
|
@@ -66,9 +69,9 @@ import org.apache.hadoop.mapred.FileInputFormat;
|
|
|
|
|
import org.apache.hadoop.mapred.JobConf;
|
|
|
|
|
import org.apache.spark.api.java.JavaRDD;
|
|
|
|
|
import org.junit.jupiter.api.AfterEach;
|
|
|
|
|
import org.junit.jupiter.api.BeforeEach;
|
|
|
|
|
import org.junit.jupiter.api.Test;
|
|
|
|
|
import org.junit.jupiter.api.io.TempDir;
|
|
|
|
|
import org.junit.jupiter.params.ParameterizedTest;
|
|
|
|
|
import org.junit.jupiter.params.provider.MethodSource;
|
|
|
|
|
|
|
|
|
|
import java.io.File;
|
|
|
|
|
import java.io.IOException;
|
|
|
|
|
@@ -90,37 +93,32 @@ import static org.junit.jupiter.api.Assertions.assertFalse;
|
|
|
|
|
import static org.junit.jupiter.api.Assertions.assertTrue;
|
|
|
|
|
|
|
|
|
|
public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
private HoodieParquetInputFormat roSnapshotInputFormat;
|
|
|
|
|
private JobConf roSnapshotJobConf;
|
|
|
|
|
|
|
|
|
|
private HoodieParquetInputFormat roInputFormat;
|
|
|
|
|
private JobConf roJobConf;
|
|
|
|
|
|
|
|
|
|
private HoodieParquetRealtimeInputFormat rtInputFormat;
|
|
|
|
|
private JobConf rtJobConf;
|
|
|
|
|
|
|
|
|
|
@BeforeEach
|
|
|
|
|
public void init() throws IOException {
|
|
|
|
|
@TempDir
|
|
|
|
|
public java.nio.file.Path tempFolder;
|
|
|
|
|
private HoodieFileFormat baseFileFormat;
|
|
|
|
|
|
|
|
|
|
static Stream<HoodieFileFormat> argumentsProvider() {
|
|
|
|
|
return Stream.of(HoodieFileFormat.PARQUET);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
public void init(HoodieFileFormat baseFileFormat) throws IOException {
|
|
|
|
|
this.baseFileFormat = baseFileFormat;
|
|
|
|
|
|
|
|
|
|
initDFS();
|
|
|
|
|
initSparkContexts("TestHoodieMergeOnReadTable");
|
|
|
|
|
hadoopConf.addResource(dfs.getConf());
|
|
|
|
|
initPath();
|
|
|
|
|
dfs.mkdirs(new Path(basePath));
|
|
|
|
|
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
|
|
|
|
|
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, baseFileFormat);
|
|
|
|
|
initTestDataGenerator();
|
|
|
|
|
|
|
|
|
|
// initialize parquet input format
|
|
|
|
|
roSnapshotInputFormat = new HoodieParquetInputFormat();
|
|
|
|
|
roSnapshotJobConf = new JobConf(jsc.hadoopConfiguration());
|
|
|
|
|
roSnapshotInputFormat.setConf(roSnapshotJobConf);
|
|
|
|
|
|
|
|
|
|
roInputFormat = new HoodieParquetInputFormat();
|
|
|
|
|
roSnapshotJobConf = new JobConf(hadoopConf);
|
|
|
|
|
roJobConf = new JobConf(hadoopConf);
|
|
|
|
|
roInputFormat.setConf(roJobConf);
|
|
|
|
|
|
|
|
|
|
rtInputFormat = new HoodieParquetRealtimeInputFormat();
|
|
|
|
|
rtJobConf = new JobConf(hadoopConf);
|
|
|
|
|
rtInputFormat.setConf(rtJobConf);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@AfterEach
|
|
|
|
|
@@ -128,8 +126,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
cleanupResources();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testSimpleInsertAndUpdate() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testSimpleInsertAndUpdate(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
HoodieWriteConfig cfg = getConfig(true);
|
|
|
|
|
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
|
|
|
|
|
|
|
|
|
@@ -153,9 +154,10 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString();
|
|
|
|
|
client.compact(compactionCommitTime);
|
|
|
|
|
|
|
|
|
|
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath());
|
|
|
|
|
hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
|
|
|
|
|
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
|
|
|
|
|
FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
|
|
|
|
HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
|
|
|
|
Stream<HoodieBaseFile> dataFilesToRead = tableView.getLatestBaseFiles();
|
|
|
|
|
assertTrue(dataFilesToRead.findAny().isPresent());
|
|
|
|
|
|
|
|
|
|
@@ -174,8 +176,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
|
|
|
|
|
// test incremental read does not go past compaction instant for RO views
|
|
|
|
|
// For RT views, incremental read can go past compaction
|
|
|
|
|
@Test
|
|
|
|
|
public void testIncrementalReadsWithCompaction() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testIncrementalReadsWithCompaction(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
String partitionPath = "2020/02/20"; // use only one partition for this test
|
|
|
|
|
dataGen = new HoodieTestDataGenerator(new String[] { partitionPath });
|
|
|
|
|
HoodieWriteConfig cfg = getConfig(true);
|
|
|
|
|
@@ -190,19 +195,17 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
List<HoodieRecord> records001 = dataGen.generateInserts(commitTime1, 200);
|
|
|
|
|
insertAndGetFilePaths(records001, client, cfg, commitTime1);
|
|
|
|
|
|
|
|
|
|
// verify only one parquet file shows up with commit time 001
|
|
|
|
|
// verify only one base file shows up with commit time 001
|
|
|
|
|
FileStatus[] snapshotROFiles = getROSnapshotFiles(partitionPath);
|
|
|
|
|
validateFiles(partitionPath,1, snapshotROFiles, roSnapshotInputFormat,
|
|
|
|
|
roSnapshotJobConf,200, commitTime1);
|
|
|
|
|
validateFiles(partitionPath, 1, snapshotROFiles, false, roSnapshotJobConf, 200, commitTime1);
|
|
|
|
|
|
|
|
|
|
FileStatus[] incrementalROFiles = getROIncrementalFiles(partitionPath, true);
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalROFiles, roInputFormat,
|
|
|
|
|
roJobConf,200, commitTime1);
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalROFiles, false, roJobConf, 200, commitTime1);
|
|
|
|
|
Path firstFilePath = incrementalROFiles[0].getPath();
|
|
|
|
|
|
|
|
|
|
FileStatus[] incrementalRTFiles = getRTIncrementalFiles(partitionPath);
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalRTFiles, rtInputFormat,
|
|
|
|
|
rtJobConf,200, commitTime1);
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalRTFiles, true, rtJobConf,200, commitTime1);
|
|
|
|
|
|
|
|
|
|
assertEquals(firstFilePath, incrementalRTFiles[0].getPath());
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
@@ -215,14 +218,12 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
|
|
|
|
|
// verify RO incremental reads - only one parquet file shows up because updates to into log files
|
|
|
|
|
incrementalROFiles = getROIncrementalFiles(partitionPath, false);
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalROFiles, roInputFormat,
|
|
|
|
|
roJobConf, 200, commitTime1);
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalROFiles, false, roJobConf, 200, commitTime1);
|
|
|
|
|
assertEquals(firstFilePath, incrementalROFiles[0].getPath());
|
|
|
|
|
|
|
|
|
|
// verify RT incremental reads includes updates also
|
|
|
|
|
incrementalRTFiles = getRTIncrementalFiles(partitionPath);
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalRTFiles, rtInputFormat,
|
|
|
|
|
rtJobConf, 200, commitTime1, updateTime);
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalRTFiles, true, rtJobConf, 200, commitTime1, updateTime);
|
|
|
|
|
|
|
|
|
|
// request compaction, but do not perform compaction
|
|
|
|
|
String compactionCommitTime = "005";
|
|
|
|
|
@@ -230,13 +231,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
|
|
|
|
|
// verify RO incremental reads - only one parquet file shows up because updates go into log files
|
|
|
|
|
incrementalROFiles = getROIncrementalFiles(partitionPath, true);
|
|
|
|
|
validateFiles(partitionPath,1, incrementalROFiles, roInputFormat,
|
|
|
|
|
roJobConf, 200, commitTime1);
|
|
|
|
|
validateFiles(partitionPath,1, incrementalROFiles, false, roJobConf, 200, commitTime1);
|
|
|
|
|
|
|
|
|
|
// verify RT incremental reads includes updates also
|
|
|
|
|
incrementalRTFiles = getRTIncrementalFiles(partitionPath);
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalRTFiles, rtInputFormat,
|
|
|
|
|
rtJobConf, 200, commitTime1, updateTime);
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalRTFiles, true, rtJobConf, 200, commitTime1, updateTime);
|
|
|
|
|
|
|
|
|
|
// write 3 - more inserts
|
|
|
|
|
String insertsTime = "006";
|
|
|
|
|
@@ -246,44 +245,44 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
|
|
|
|
|
// verify new write shows up in snapshot mode even though there is pending compaction
|
|
|
|
|
snapshotROFiles = getROSnapshotFiles(partitionPath);
|
|
|
|
|
validateFiles(partitionPath, 2, snapshotROFiles, roSnapshotInputFormat,
|
|
|
|
|
roSnapshotJobConf,400, commitTime1, insertsTime);
|
|
|
|
|
validateFiles(partitionPath, 2, snapshotROFiles, false, roSnapshotJobConf,400, commitTime1, insertsTime);
|
|
|
|
|
|
|
|
|
|
incrementalROFiles = getROIncrementalFiles(partitionPath, true);
|
|
|
|
|
assertEquals(firstFilePath, incrementalROFiles[0].getPath());
|
|
|
|
|
// verify 006 does not show up in RO mode because of pending compaction
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalROFiles, roInputFormat,
|
|
|
|
|
roJobConf, 200, commitTime1);
|
|
|
|
|
|
|
|
|
|
validateFiles(partitionPath, 1, incrementalROFiles, false, roJobConf, 200, commitTime1);
|
|
|
|
|
|
|
|
|
|
// verify that if stopAtCompaction is disabled, inserts from "insertsTime" show up
|
|
|
|
|
incrementalROFiles = getROIncrementalFiles(partitionPath, false);
|
|
|
|
|
validateFiles(partitionPath,2, incrementalROFiles, roInputFormat,
|
|
|
|
|
roJobConf, 400, commitTime1, insertsTime);
|
|
|
|
|
validateFiles(partitionPath,2, incrementalROFiles, false, roJobConf, 400, commitTime1, insertsTime);
|
|
|
|
|
|
|
|
|
|
// verify 006 shows up in RT views
|
|
|
|
|
incrementalRTFiles = getRTIncrementalFiles(partitionPath);
|
|
|
|
|
validateFiles(partitionPath, 2, incrementalRTFiles, rtInputFormat,
|
|
|
|
|
rtJobConf, 400, commitTime1, updateTime, insertsTime);
|
|
|
|
|
validateFiles(partitionPath, 2, incrementalRTFiles, true, rtJobConf, 400, commitTime1, updateTime, insertsTime);
|
|
|
|
|
|
|
|
|
|
// perform the scheduled compaction
|
|
|
|
|
client.compact(compactionCommitTime);
|
|
|
|
|
|
|
|
|
|
// verify new write shows up in snapshot mode after compaction is complete
|
|
|
|
|
snapshotROFiles = getROSnapshotFiles(partitionPath);
|
|
|
|
|
validateFiles(partitionPath,2, snapshotROFiles, roSnapshotInputFormat,
|
|
|
|
|
roSnapshotJobConf,400, commitTime1, compactionCommitTime, insertsTime);
|
|
|
|
|
validateFiles(partitionPath,2, snapshotROFiles, false, roSnapshotJobConf,400, commitTime1, compactionCommitTime,
|
|
|
|
|
insertsTime);
|
|
|
|
|
|
|
|
|
|
incrementalROFiles = getROIncrementalFiles(partitionPath, "002", -1, true);
|
|
|
|
|
assertTrue(incrementalROFiles.length == 2);
|
|
|
|
|
// verify 006 shows up because of pending compaction
|
|
|
|
|
validateFiles(partitionPath, 2, incrementalROFiles, roInputFormat,
|
|
|
|
|
roJobConf, 400, commitTime1, compactionCommitTime, insertsTime);
|
|
|
|
|
validateFiles(partitionPath, 2, incrementalROFiles, false, roJobConf, 400, commitTime1, compactionCommitTime,
|
|
|
|
|
insertsTime);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Check if record level metadata is aggregated properly at the end of write.
|
|
|
|
|
@Test
|
|
|
|
|
public void testMetadataAggregateFromWriteStatus() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testMetadataAggregateFromWriteStatus(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
HoodieWriteConfig cfg = getConfigBuilder(false).withWriteStatusClass(MetadataMergeWriteStatus.class).build();
|
|
|
|
|
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
|
|
|
|
|
|
|
|
|
@@ -305,8 +304,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testSimpleInsertUpdateAndDelete() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testSimpleInsertUpdateAndDelete(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
HoodieWriteConfig cfg = getConfig(true);
|
|
|
|
|
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
|
|
|
|
|
|
|
|
|
@@ -332,7 +334,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
|
|
|
|
assertFalse(commit.isPresent());
|
|
|
|
|
|
|
|
|
|
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
tableView = getHoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
|
|
|
|
|
Stream<HoodieBaseFile> dataFilesToRead = tableView.getLatestBaseFiles();
|
|
|
|
|
assertFalse(dataFilesToRead.findAny().isPresent());
|
|
|
|
|
@@ -373,23 +375,25 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
|
|
|
|
assertFalse(commit.isPresent());
|
|
|
|
|
|
|
|
|
|
allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath());
|
|
|
|
|
allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
|
|
|
|
dataFilesToRead = tableView.getLatestBaseFiles();
|
|
|
|
|
assertTrue(dataFilesToRead.findAny().isPresent());
|
|
|
|
|
|
|
|
|
|
List<String> dataFiles = tableView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
|
|
|
|
|
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
|
|
|
|
|
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles, basePath);
|
|
|
|
|
// Wrote 20 records and deleted 20 records, so remaining 20-20 = 0
|
|
|
|
|
assertEquals(0, recordsRead.size(), "Must contain 0 records");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testCOWToMORConvertedTableRollback() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testCOWToMORConvertedTableRollback(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
// Set TableType to COW
|
|
|
|
|
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
|
|
|
|
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE, baseFileFormat);
|
|
|
|
|
|
|
|
|
|
HoodieWriteConfig cfg = getConfig(true);
|
|
|
|
|
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
|
|
|
|
@@ -425,14 +429,14 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
assertNoWriteErrors(statuses);
|
|
|
|
|
|
|
|
|
|
// Set TableType to MOR
|
|
|
|
|
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
|
|
|
|
|
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ, baseFileFormat);
|
|
|
|
|
|
|
|
|
|
// rollback a COW commit when TableType is MOR
|
|
|
|
|
client.rollback(newCommitTime);
|
|
|
|
|
|
|
|
|
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
|
|
|
|
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
|
|
|
|
|
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
|
|
|
|
|
|
|
|
|
final String absentCommit = newCommitTime;
|
|
|
|
|
@@ -440,8 +444,10 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testRollbackWithDeltaAndCompactionCommit() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testRollbackWithDeltaAndCompactionCommit(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
HoodieWriteConfig cfg = getConfig(false);
|
|
|
|
|
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
|
|
|
|
@@ -471,7 +477,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
|
|
|
|
assertFalse(commit.isPresent());
|
|
|
|
|
|
|
|
|
|
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
tableView =
|
|
|
|
|
getHoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
|
|
|
|
|
Stream<HoodieBaseFile> dataFilesToRead = tableView.getLatestBaseFiles();
|
|
|
|
|
@@ -495,7 +501,8 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
copyOfRecords.addAll(dataGen.generateInserts(commitTime1, 200));
|
|
|
|
|
|
|
|
|
|
List<String> dataFiles = tableView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
|
|
|
|
|
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
|
|
|
|
|
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles,
|
|
|
|
|
basePath);
|
|
|
|
|
assertEquals(recordsRead.size(), 200);
|
|
|
|
|
|
|
|
|
|
statuses = secondClient.upsert(jsc.parallelize(copyOfRecords, 1), commitTime1).collect();
|
|
|
|
|
@@ -504,12 +511,12 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
|
|
|
|
|
// Test failed delta commit rollback
|
|
|
|
|
secondClient.rollback(commitTime1);
|
|
|
|
|
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
// After rollback, there should be no parquet file with the failed commit time
|
|
|
|
|
allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
// After rollback, there should be no base file with the failed commit time
|
|
|
|
|
assertEquals(0, Arrays.stream(allFiles)
|
|
|
|
|
.filter(file -> file.getPath().getName().contains(commitTime1)).count());
|
|
|
|
|
dataFiles = tableView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
|
|
|
|
|
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
|
|
|
|
|
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles, basePath);
|
|
|
|
|
assertEquals(200, recordsRead.size());
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@@ -525,7 +532,8 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
copyOfRecords.addAll(dataGen.generateInserts(commitTime2, 200));
|
|
|
|
|
|
|
|
|
|
List<String> dataFiles = tableView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
|
|
|
|
|
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
|
|
|
|
|
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles,
|
|
|
|
|
basePath);
|
|
|
|
|
assertEquals(200, recordsRead.size());
|
|
|
|
|
|
|
|
|
|
writeRecords = jsc.parallelize(copyOfRecords, 1);
|
|
|
|
|
@@ -537,7 +545,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
|
|
|
|
|
// Test successful delta commit rollback
|
|
|
|
|
thirdClient.rollback(commitTime2);
|
|
|
|
|
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
// After rollback, there should be no parquet file with the failed commit time
|
|
|
|
|
assertEquals(0, Arrays.stream(allFiles)
|
|
|
|
|
.filter(file -> file.getPath().getName().contains(commitTime2)).count());
|
|
|
|
|
@@ -546,7 +554,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
|
|
|
|
|
tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
|
|
|
|
dataFiles = tableView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
|
|
|
|
|
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
|
|
|
|
|
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles, basePath);
|
|
|
|
|
// check that the number of records read is still correct after rollback operation
|
|
|
|
|
assertEquals(200, recordsRead.size());
|
|
|
|
|
|
|
|
|
|
@@ -569,7 +577,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
JavaRDD<WriteStatus> ws = thirdClient.compact(compactionInstantTime);
|
|
|
|
|
thirdClient.commitCompaction(compactionInstantTime, ws, Option.empty());
|
|
|
|
|
|
|
|
|
|
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
|
|
|
|
tableView = getHoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
|
|
|
|
|
|
|
|
|
|
@@ -580,7 +588,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
|
|
|
|
|
thirdClient.rollback(compactedCommitTime);
|
|
|
|
|
|
|
|
|
|
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
|
|
|
|
tableView = getHoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
|
|
|
|
|
|
|
|
|
|
@@ -589,8 +597,10 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testMultiRollbackWithDeltaAndCompactionCommit(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
HoodieWriteConfig cfg = getConfig(false);
|
|
|
|
|
try (final HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
|
|
|
|
@@ -618,7 +628,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
|
|
|
|
assertFalse(commit.isPresent());
|
|
|
|
|
|
|
|
|
|
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
tableView = getHoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
|
|
|
|
|
Stream<HoodieBaseFile> dataFilesToRead = tableView.getLatestBaseFiles();
|
|
|
|
|
assertFalse(dataFilesToRead.findAny().isPresent());
|
|
|
|
|
@@ -641,7 +651,8 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
copyOfRecords.addAll(dataGen.generateInserts(newCommitTime, 200));
|
|
|
|
|
|
|
|
|
|
List<String> dataFiles = tableView.getLatestBaseFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
|
|
|
|
|
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
|
|
|
|
|
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles,
|
|
|
|
|
basePath);
|
|
|
|
|
assertEquals(200, recordsRead.size());
|
|
|
|
|
|
|
|
|
|
statuses = nClient.upsert(jsc.parallelize(copyOfRecords, 1), newCommitTime).collect();
|
|
|
|
|
@@ -696,7 +707,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
JavaRDD<WriteStatus> ws = client.compact(compactionInstantTime);
|
|
|
|
|
client.commitCompaction(compactionInstantTime, ws, Option.empty());
|
|
|
|
|
|
|
|
|
|
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
|
|
|
|
tableView = getHoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles);
|
|
|
|
|
|
|
|
|
|
@@ -724,7 +735,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
client.restoreToInstant("000");
|
|
|
|
|
|
|
|
|
|
metaClient = HoodieTableMetaClient.reload(metaClient);
|
|
|
|
|
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
tableView = getHoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
|
|
|
|
|
dataFilesToRead = tableView.getLatestBaseFiles();
|
|
|
|
|
assertFalse(dataFilesToRead.findAny().isPresent());
|
|
|
|
|
@@ -751,8 +762,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
.build();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testUpsertPartitioner() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testUpsertPartitioner(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
HoodieWriteConfig cfg = getConfig(true);
|
|
|
|
|
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
|
|
|
|
|
|
|
|
|
@@ -778,7 +792,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
|
|
|
|
assertFalse(commit.isPresent());
|
|
|
|
|
|
|
|
|
|
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
BaseFileOnlyView roView = getHoodieTableFileSystemView(metaClient,
|
|
|
|
|
metaClient.getCommitsTimeline().filterCompletedInstants(), allFiles);
|
|
|
|
|
Stream<HoodieBaseFile> dataFilesToRead = roView.getLatestBaseFiles();
|
|
|
|
|
@@ -812,7 +826,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
|
|
|
|
assertFalse(commit.isPresent());
|
|
|
|
|
|
|
|
|
|
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
roView = getHoodieTableFileSystemView(metaClient,
|
|
|
|
|
hoodieTable.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(), allFiles);
|
|
|
|
|
dataFilesToRead = roView.getLatestBaseFiles();
|
|
|
|
|
@@ -823,14 +837,18 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
assertTrue(parquetFileIdToNewSize.entrySet().stream().anyMatch(entry -> parquetFileIdToSize.get(entry.getKey()) < entry.getValue()));
|
|
|
|
|
|
|
|
|
|
List<String> dataFiles = roView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
|
|
|
|
|
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
|
|
|
|
|
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles,
|
|
|
|
|
basePath);
|
|
|
|
|
// Wrote 20 records in 2 batches
|
|
|
|
|
assertEquals(40, recordsRead.size(), "Must contain 40 records");
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testLogFileCountsAfterCompaction() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testLogFileCountsAfterCompaction(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
// insert 100 records
|
|
|
|
|
HoodieWriteConfig config = getConfig(true);
|
|
|
|
|
try (HoodieWriteClient writeClient = getHoodieWriteClient(config);) {
|
|
|
|
|
@@ -902,8 +920,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testSimpleInsertsGeneratedIntoLogFiles() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testSimpleInsertsGeneratedIntoLogFiles(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
// insert 100 records
|
|
|
|
|
// Setting IndexType to be InMemory to simulate Global Index nature
|
|
|
|
|
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
|
|
|
|
|
@@ -939,8 +960,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testInsertsGeneratedIntoLogFilesRollback(@TempDir java.nio.file.Path tempFolder) throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testInsertsGeneratedIntoLogFilesRollback(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
// insert 100 records
|
|
|
|
|
// Setting IndexType to be InMemory to simulate Global Index nature
|
|
|
|
|
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
|
|
|
|
|
@@ -1010,8 +1034,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
@Test
|
|
|
|
|
public void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
// insert 100 records
|
|
|
|
|
// Setting IndexType to be InMemory to simulate Global Index nature
|
|
|
|
|
HoodieWriteConfig config = getConfigBuilder(false, IndexType.INMEMORY).build();
|
|
|
|
|
@@ -1063,8 +1090,10 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
/**
|
|
|
|
|
* Test to ensure rolling stats are correctly written to metadata file.
|
|
|
|
|
*/
|
|
|
|
|
@Test
|
|
|
|
|
public void testRollingStatsInMetadata() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testRollingStatsInMetadata(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
|
|
|
|
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
|
|
|
|
@@ -1163,8 +1192,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
/**
|
|
|
|
|
* Test to ensure rolling stats are correctly written to the metadata file, identifies small files and corrects them.
|
|
|
|
|
*/
|
|
|
|
|
@Test
|
|
|
|
|
public void testRollingStatsWithSmallFileHandling() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testRollingStatsWithSmallFileHandling(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
HoodieWriteConfig cfg = getConfigBuilder(false, IndexType.INMEMORY).withAutoCommit(false).build();
|
|
|
|
|
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
|
|
|
|
Map<String, Long> fileIdToInsertsMap = new HashMap<>();
|
|
|
|
|
@@ -1296,8 +1328,11 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
/**
|
|
|
|
|
* Test to validate invoking table.handleUpdate() with input records from multiple partitions will fail.
|
|
|
|
|
*/
|
|
|
|
|
@Test
|
|
|
|
|
public void testHandleUpdateWithMultiplePartitions() throws Exception {
|
|
|
|
|
@ParameterizedTest
|
|
|
|
|
@MethodSource("argumentsProvider")
|
|
|
|
|
public void testHandleUpdateWithMultiplePartitions(HoodieFileFormat baseFileFormat) throws Exception {
|
|
|
|
|
init(baseFileFormat);
|
|
|
|
|
|
|
|
|
|
HoodieWriteConfig cfg = getConfig(true);
|
|
|
|
|
try (HoodieWriteClient client = getHoodieWriteClient(cfg);) {
|
|
|
|
|
|
|
|
|
|
@@ -1323,7 +1358,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
|
|
|
|
assertFalse(commit.isPresent());
|
|
|
|
|
|
|
|
|
|
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
BaseFileOnlyView roView =
|
|
|
|
|
getHoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
|
|
|
|
|
Stream<HoodieBaseFile> dataFilesToRead = roView.getLatestBaseFiles();
|
|
|
|
|
@@ -1401,7 +1436,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
assertFalse(status.hasErrors(), "Errors found in write of " + status.getFileId());
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
private FileStatus[] insertAndGetFilePaths(List<HoodieRecord> records, HoodieWriteClient client,
|
|
|
|
|
HoodieWriteConfig cfg, String commitTime) throws IOException {
|
|
|
|
|
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
|
|
|
|
@@ -1419,7 +1454,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().lastInstant();
|
|
|
|
|
assertFalse(commit.isPresent());
|
|
|
|
|
|
|
|
|
|
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
FileStatus[] allFiles = listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
BaseFileOnlyView roView =
|
|
|
|
|
getHoodieTableFileSystemView(metaClient, metaClient.getCommitTimeline().filterCompletedInstants(), allFiles);
|
|
|
|
|
Stream<HoodieBaseFile> dataFilesToRead = roView.getLatestBaseFiles();
|
|
|
|
|
@@ -1452,14 +1487,14 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
|
|
|
|
|
Option<HoodieInstant> commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
|
|
|
|
|
assertFalse(commit.isPresent());
|
|
|
|
|
return HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
|
|
|
|
|
HoodieTable hoodieTable = HoodieTable.create(metaClient, cfg, hadoopConf);
|
|
|
|
|
return listAllDataFilesInPath(hoodieTable, cfg.getBasePath());
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private FileStatus[] getROSnapshotFiles(String partitionPath)
|
|
|
|
|
throws Exception {
|
|
|
|
|
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
|
|
|
|
|
FileInputFormat.setInputPaths(roSnapshotJobConf, basePath + "/" + partitionPath);
|
|
|
|
|
return roSnapshotInputFormat.listStatus(roSnapshotJobConf);
|
|
|
|
|
return listStatus(roSnapshotJobConf, false);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private FileStatus[] getROIncrementalFiles(String partitionPath, boolean stopAtCompaction)
|
|
|
|
|
@@ -1469,10 +1504,9 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
|
|
|
|
|
private FileStatus[] getROIncrementalFiles(String partitionPath, String startCommitTime, int numCommitsToPull, boolean stopAtCompaction)
|
|
|
|
|
throws Exception {
|
|
|
|
|
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
|
|
|
|
|
setupIncremental(roJobConf, startCommitTime, numCommitsToPull, stopAtCompaction);
|
|
|
|
|
FileInputFormat.setInputPaths(roJobConf, Paths.get(basePath, partitionPath).toString());
|
|
|
|
|
return roInputFormat.listStatus(roJobConf);
|
|
|
|
|
return listStatus(roJobConf, false);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private FileStatus[] getRTIncrementalFiles(String partitionPath)
|
|
|
|
|
@@ -1482,10 +1516,9 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
|
|
|
|
|
private FileStatus[] getRTIncrementalFiles(String partitionPath, String startCommitTime, int numCommitsToPull)
|
|
|
|
|
throws Exception {
|
|
|
|
|
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
|
|
|
|
|
setupIncremental(rtJobConf, startCommitTime, numCommitsToPull, false);
|
|
|
|
|
FileInputFormat.setInputPaths(rtJobConf, Paths.get(basePath, partitionPath).toString());
|
|
|
|
|
return rtInputFormat.listStatus(rtJobConf);
|
|
|
|
|
return listStatus(rtJobConf, true);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void setupIncremental(JobConf jobConf, String startCommit, int numberOfCommitsToPull, boolean stopAtCompaction) {
|
|
|
|
|
@@ -1507,16 +1540,37 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private void validateFiles(String partitionPath, int expectedNumFiles,
|
|
|
|
|
FileStatus[] files, HoodieParquetInputFormat inputFormat,
|
|
|
|
|
JobConf jobConf, int expectedRecords, String... expectedCommits) {
|
|
|
|
|
FileStatus[] files, boolean realtime, JobConf jobConf,
|
|
|
|
|
int expectedRecords, String... expectedCommits) {
|
|
|
|
|
|
|
|
|
|
assertEquals(expectedNumFiles, files.length);
|
|
|
|
|
Set<String> expectedCommitsSet = Arrays.stream(expectedCommits).collect(Collectors.toSet());
|
|
|
|
|
List<GenericRecord> records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
|
|
|
|
Collections.singletonList(Paths.get(basePath, partitionPath).toString()), basePath, jobConf, inputFormat);
|
|
|
|
|
List<GenericRecord> records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf,
|
|
|
|
|
Collections.singletonList(Paths.get(basePath, partitionPath).toString()), basePath, jobConf, realtime);
|
|
|
|
|
assertEquals(expectedRecords, records.size());
|
|
|
|
|
Set<String> actualCommits = records.stream().map(r ->
|
|
|
|
|
r.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()).collect(Collectors.toSet());
|
|
|
|
|
assertEquals(expectedCommitsSet, actualCommits);
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private FileStatus[] listAllDataFilesInPath(HoodieTable table, String basePath) throws IOException {
|
|
|
|
|
return HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), basePath, table.getBaseFileExtension());
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private FileStatus[] listStatus(JobConf jobConf, boolean realtime) throws IOException {
|
|
|
|
|
// This is required as Hoodie InputFormats do not extend a common base class and FileInputFormat's
|
|
|
|
|
// listStatus() is protected.
|
|
|
|
|
FileInputFormat inputFormat = HoodieInputFormatUtils.getInputFormat(baseFileFormat, realtime, jobConf);
|
|
|
|
|
switch (baseFileFormat) {
|
|
|
|
|
case PARQUET:
|
|
|
|
|
if (realtime) {
|
|
|
|
|
return ((HoodieParquetRealtimeInputFormat)inputFormat).listStatus(jobConf);
|
|
|
|
|
} else {
|
|
|
|
|
return ((HoodieParquetInputFormat)inputFormat).listStatus(jobConf);
|
|
|
|
|
}
|
|
|
|
|
default:
|
|
|
|
|
throw new HoodieIOException("Hoodie InputFormat not implemented for base file format " + baseFileFormat);
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|