[HUDI-1055] Remove hardcoded parquet in tests (#2740)
* Remove hardcoded parquet in tests * Use DataFileUtils.getInstance * Renaming DataFileUtils to BaseFileUtils Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -47,7 +47,7 @@ public class HoodieSparkBootstrapSchemaProvider extends HoodieBootstrapSchemaPro
|
||||
MessageType parquetSchema = partitions.stream().flatMap(p -> p.getValue().stream()).map(fs -> {
|
||||
try {
|
||||
Path filePath = FileStatusUtils.toPath(fs.getPath());
|
||||
return ParquetUtils.readSchema(context.getHadoopConf().get(), filePath);
|
||||
return new ParquetUtils().readSchema(context.getHadoopConf().get(), filePath);
|
||||
} catch (Exception ex) {
|
||||
return null;
|
||||
}
|
||||
|
||||
@@ -163,7 +163,7 @@ public class TestClientRollback extends HoodieClientTestBase {
|
||||
*/
|
||||
@Test
|
||||
public void testRollbackCommit() throws Exception {
|
||||
// Let's create some commit files and parquet files
|
||||
// Let's create some commit files and base files
|
||||
final String p1 = "2016/05/01";
|
||||
final String p2 = "2016/05/02";
|
||||
final String p3 = "2016/05/06";
|
||||
@@ -251,7 +251,7 @@ public class TestClientRollback extends HoodieClientTestBase {
|
||||
*/
|
||||
@Test
|
||||
public void testAutoRollbackInflightCommit() throws Exception {
|
||||
// Let's create some commit files and parquet files
|
||||
// Let's create some commit files and base files
|
||||
final String p1 = "2016/05/01";
|
||||
final String p2 = "2016/05/02";
|
||||
final String p3 = "2016/05/06";
|
||||
|
||||
@@ -49,9 +49,9 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.ClusteringUtils;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.BaseFileUtils;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieClusteringConfig;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
@@ -115,7 +115,6 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.NULL_SCHE
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys;
|
||||
import static org.apache.hudi.common.testutils.Transformations.recordsToRecordKeySet;
|
||||
import static org.apache.hudi.common.util.ParquetUtils.readRowKeysFromParquet;
|
||||
import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE_OPT_KEY;
|
||||
import static org.apache.hudi.config.HoodieClusteringConfig.DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS;
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
@@ -424,23 +423,24 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(jsc.hadoopConfiguration()).setBasePath(basePath).build();
|
||||
String basePathStr = basePath;
|
||||
HoodieTable table = getHoodieTable(metaClient, cfg);
|
||||
String extension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
|
||||
jsc.parallelize(Arrays.asList(1)).map(e -> {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(metaClient.getActiveTimeline().getInstantDetails(
|
||||
metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get()).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
String filePath = commitMetadata.getPartitionToWriteStats().values().stream()
|
||||
.flatMap(w -> w.stream()).filter(s -> s.getPath().endsWith(".parquet")).findAny()
|
||||
.flatMap(w -> w.stream()).filter(s -> s.getPath().endsWith(extension)).findAny()
|
||||
.map(ee -> ee.getPath()).orElse(null);
|
||||
String partitionPath = commitMetadata.getPartitionToWriteStats().values().stream()
|
||||
.flatMap(w -> w.stream()).filter(s -> s.getPath().endsWith(".parquet")).findAny()
|
||||
.flatMap(w -> w.stream()).filter(s -> s.getPath().endsWith(extension)).findAny()
|
||||
.map(ee -> ee.getPartitionPath()).orElse(null);
|
||||
Path parquetFilePath = new Path(basePathStr, filePath);
|
||||
HoodieBaseFile baseFile = new HoodieBaseFile(parquetFilePath.toString());
|
||||
Path baseFilePath = new Path(basePathStr, filePath);
|
||||
HoodieBaseFile baseFile = new HoodieBaseFile(baseFilePath.toString());
|
||||
|
||||
try {
|
||||
HoodieMergeHandle handle = new HoodieMergeHandle(cfg, instantTime, table, new HashMap<>(),
|
||||
partitionPath, FSUtils.getFileId(parquetFilePath.getName()), baseFile, new SparkTaskContextSupplier());
|
||||
partitionPath, FSUtils.getFileId(baseFilePath.getName()), baseFile, new SparkTaskContextSupplier());
|
||||
WriteStatus writeStatus = new WriteStatus(false, 0.0);
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
writeStatus.getStat().setNumWrites(0);
|
||||
@@ -454,7 +454,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
cfg.getProps().setProperty("hoodie.merge.data.validation.enabled", "true");
|
||||
HoodieWriteConfig cfg2 = HoodieWriteConfig.newBuilder().withProps(cfg.getProps()).build();
|
||||
HoodieMergeHandle handle = new HoodieMergeHandle(cfg2, newInstantTime, table, new HashMap<>(),
|
||||
partitionPath, FSUtils.getFileId(parquetFilePath.getName()), baseFile, new SparkTaskContextSupplier());
|
||||
partitionPath, FSUtils.getFileId(baseFilePath.getName()), baseFile, new SparkTaskContextSupplier());
|
||||
WriteStatus writeStatus = new WriteStatus(false, 0.0);
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
writeStatus.getStat().setNumWrites(0);
|
||||
@@ -850,6 +850,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit); // hold upto 200 records max
|
||||
dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath});
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(config);
|
||||
BaseFileUtils fileUtils = BaseFileUtils.getInstance(metaClient);
|
||||
|
||||
// Inserts => will write file1
|
||||
String commitTime1 = "001";
|
||||
@@ -865,7 +866,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
assertEquals(1, statuses.size(), "Just 1 file needs to be added.");
|
||||
String file1 = statuses.get(0).getFileId();
|
||||
assertEquals(100,
|
||||
readRowKeysFromParquet(hadoopConf, new Path(basePath, statuses.get(0).getStat().getPath()))
|
||||
fileUtils.readRowKeys(hadoopConf, new Path(basePath, statuses.get(0).getStat().getPath()))
|
||||
.size(), "file should contain 100 records");
|
||||
|
||||
// Update + Inserts such that they just expand file1
|
||||
@@ -885,10 +886,10 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
assertEquals(file1, statuses.get(0).getFileId(), "Existing file should be expanded");
|
||||
assertEquals(commitTime1, statuses.get(0).getStat().getPrevCommit(), "Existing file should be expanded");
|
||||
Path newFile = new Path(basePath, statuses.get(0).getStat().getPath());
|
||||
assertEquals(140, readRowKeysFromParquet(hadoopConf, newFile).size(),
|
||||
assertEquals(140, fileUtils.readRowKeys(hadoopConf, newFile).size(),
|
||||
"file should contain 140 records");
|
||||
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(hadoopConf, newFile);
|
||||
List<GenericRecord> records = fileUtils.readAvroRecords(hadoopConf, newFile);
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertEquals(commitTime2, record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(), "only expect commit2");
|
||||
@@ -919,7 +920,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
for (HoodieBaseFile file : files) {
|
||||
if (file.getFileName().contains(file1)) {
|
||||
assertEquals(commitTime3, file.getCommitTime(), "Existing file should be expanded");
|
||||
records = ParquetUtils.readAvroRecords(hadoopConf, new Path(file.getPath()));
|
||||
records = fileUtils.readAvroRecords(hadoopConf, new Path(file.getPath()));
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String recordCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
@@ -935,7 +936,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
assertEquals(0, keys2.size(), "All keys added in commit 2 must be updated in commit3 correctly");
|
||||
} else {
|
||||
assertEquals(commitTime3, file.getCommitTime(), "New file must be written for commit 3");
|
||||
records = ParquetUtils.readAvroRecords(hadoopConf, new Path(file.getPath()));
|
||||
records = fileUtils.readAvroRecords(hadoopConf, new Path(file.getPath()));
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertEquals(commitTime3, record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString(),
|
||||
@@ -961,6 +962,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
HoodieWriteConfig config = getSmallInsertWriteConfig(insertSplitLimit, false, mergeAllowDuplicateInserts); // hold upto 200 records max
|
||||
dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath});
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(config);
|
||||
BaseFileUtils fileUtils = BaseFileUtils.getInstance(metaClient);
|
||||
|
||||
// Inserts => will write file1
|
||||
String commitTime1 = "001";
|
||||
@@ -974,7 +976,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
assertEquals(1, statuses.size(), "Just 1 file needs to be added.");
|
||||
String file1 = statuses.get(0).getFileId();
|
||||
assertEquals(100,
|
||||
readRowKeysFromParquet(hadoopConf, new Path(basePath, statuses.get(0).getStat().getPath()))
|
||||
fileUtils.readRowKeys(hadoopConf, new Path(basePath, statuses.get(0).getStat().getPath()))
|
||||
.size(), "file should contain 100 records");
|
||||
|
||||
// Second, set of Inserts should just expand file1
|
||||
@@ -990,9 +992,9 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
assertEquals(commitTime1, statuses.get(0).getStat().getPrevCommit(), "Existing file should be expanded");
|
||||
|
||||
Path newFile = new Path(basePath, statuses.get(0).getStat().getPath());
|
||||
assertEquals(140, readRowKeysFromParquet(hadoopConf, newFile).size(),
|
||||
assertEquals(140, fileUtils.readRowKeys(hadoopConf, newFile).size(),
|
||||
"file should contain 140 records");
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(hadoopConf, newFile);
|
||||
List<GenericRecord> records = fileUtils.readAvroRecords(hadoopConf, newFile);
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String recCommitTime = record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
@@ -1011,8 +1013,8 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
assertNoWriteErrors(statuses);
|
||||
assertEquals(2, statuses.size(), "2 files needs to be committed.");
|
||||
assertEquals(340,
|
||||
readRowKeysFromParquet(hadoopConf, new Path(basePath, statuses.get(0).getStat().getPath())).size()
|
||||
+ readRowKeysFromParquet(hadoopConf, new Path(basePath, statuses.get(1).getStat().getPath())).size(),
|
||||
fileUtils.readRowKeys(hadoopConf, new Path(basePath, statuses.get(0).getStat().getPath())).size()
|
||||
+ fileUtils.readRowKeys(hadoopConf, new Path(basePath, statuses.get(1).getStat().getPath())).size(),
|
||||
"file should contain 340 records");
|
||||
|
||||
HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(basePath).build();
|
||||
@@ -1024,7 +1026,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
int totalInserts = 0;
|
||||
for (HoodieBaseFile file : files) {
|
||||
assertEquals(commitTime3, file.getCommitTime(), "All files must be at commit 3");
|
||||
totalInserts += ParquetUtils.readAvroRecords(hadoopConf, new Path(file.getPath())).size();
|
||||
totalInserts += fileUtils.readAvroRecords(hadoopConf, new Path(file.getPath())).size();
|
||||
}
|
||||
assertEquals(totalInserts, inserts1.size() + inserts2.size() + inserts3.size(), "Total number of records must add up");
|
||||
}
|
||||
@@ -1056,7 +1058,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
assertEquals(1, statuses.size(), "Just 1 file needs to be added.");
|
||||
String file1 = statuses.get(0).getFileId();
|
||||
assertEquals(100,
|
||||
readRowKeysFromParquet(hadoopConf, new Path(basePath, statuses.get(0).getStat().getPath()))
|
||||
BaseFileUtils.getInstance(metaClient).readRowKeys(hadoopConf, new Path(basePath, statuses.get(0).getStat().getPath()))
|
||||
.size(), "file should contain 100 records");
|
||||
|
||||
// Delete 20 among 100 inserted
|
||||
@@ -1356,13 +1358,13 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
}
|
||||
|
||||
/**
|
||||
* Verify data in parquet files matches expected records and commit time.
|
||||
* Verify data in base files matches expected records and commit time.
|
||||
*/
|
||||
private void verifyRecordsWritten(String commitTime, List<HoodieRecord> expectedRecords, List<WriteStatus> allStatus) {
|
||||
List<GenericRecord> records = new ArrayList<>();
|
||||
for (WriteStatus status : allStatus) {
|
||||
Path filePath = new Path(basePath, status.getStat().getPath());
|
||||
records.addAll(ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), filePath));
|
||||
records.addAll(BaseFileUtils.getInstance(metaClient).readAvroRecords(jsc.hadoopConfiguration(), filePath));
|
||||
}
|
||||
|
||||
Set<String> expectedKeys = recordsToRecordKeySet(expectedRecords);
|
||||
@@ -1410,7 +1412,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
}
|
||||
|
||||
private void testDeletes(SparkRDDWriteClient client, List<HoodieRecord> previousRecords, int sizeToDelete,
|
||||
String existingFile, String instantTime, int exepctedRecords, List<String> keys) {
|
||||
String existingFile, String instantTime, int expectedRecords, List<String> keys) {
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
List<HoodieKey> hoodieKeysToDelete = randomSelectAsHoodieKeys(previousRecords, sizeToDelete);
|
||||
@@ -1427,16 +1429,16 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
for (int i = 0; i < fullPartitionPaths.length; i++) {
|
||||
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
|
||||
}
|
||||
assertEquals(exepctedRecords,
|
||||
assertEquals(expectedRecords,
|
||||
HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count(),
|
||||
"Must contain " + exepctedRecords + " records");
|
||||
"Must contain " + expectedRecords + " records");
|
||||
|
||||
Path newFile = new Path(basePath, statuses.get(0).getStat().getPath());
|
||||
assertEquals(exepctedRecords,
|
||||
readRowKeysFromParquet(hadoopConf, newFile).size(),
|
||||
assertEquals(expectedRecords,
|
||||
BaseFileUtils.getInstance(metaClient).readRowKeys(hadoopConf, newFile).size(),
|
||||
"file should contain 110 records");
|
||||
|
||||
List<GenericRecord> records = ParquetUtils.readAvroRecords(hadoopConf, newFile);
|
||||
List<GenericRecord> records = BaseFileUtils.getInstance(metaClient).readAvroRecords(hadoopConf, newFile);
|
||||
for (GenericRecord record : records) {
|
||||
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
assertTrue(keys.contains(recordKey), "key expected to be part of " + instantTime);
|
||||
@@ -1491,7 +1493,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
|
||||
assertTrue(testTable.commitExists(instantTime),
|
||||
"After explicit commit, commit file should be created");
|
||||
|
||||
// Get parquet file paths from commit metadata
|
||||
// Get base file paths from commit metadata
|
||||
String actionType = metaClient.getCommitActionType();
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, actionType, instantTime);
|
||||
HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
|
||||
@@ -113,7 +113,7 @@ public class TestHoodieReadClient extends HoodieClientTestBase {
|
||||
assertEquals(100, filteredRDD.collect().size());
|
||||
|
||||
JavaRDD<HoodieRecord> smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1);
|
||||
// We create three parquet file, each having one record. (3 different partitions)
|
||||
// We create three base file, each having one record. (3 different partitions)
|
||||
List<WriteStatus> statuses = writeFn.apply(writeClient, smallRecordsRDD, newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
@@ -25,7 +25,7 @@ import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.BaseFileUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
@@ -36,7 +36,6 @@ import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.io.InvalidRecordException;
|
||||
import org.apache.parquet.io.ParquetDecodingException;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
@@ -123,9 +122,10 @@ public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
Executable executable = () -> {
|
||||
HoodieMergeHandle mergeHandle = new HoodieMergeHandle(updateTable.getConfig(), "101", updateTable,
|
||||
updateRecords.iterator(), updateRecords.get(0).getPartitionPath(), insertResult.getFileId(), supplier);
|
||||
AvroReadSupport.setAvroReadSchema(updateTable.getHadoopConf(), mergeHandle.getWriterSchemaWithMetafields());
|
||||
List<GenericRecord> oldRecords = ParquetUtils.readAvroRecords(updateTable.getHadoopConf(),
|
||||
new Path(updateTable.getConfig().getBasePath() + "/" + insertResult.getStat().getPath()));
|
||||
List<GenericRecord> oldRecords = BaseFileUtils.getInstance(updateTable.getBaseFileFormat())
|
||||
.readAvroRecords(updateTable.getHadoopConf(),
|
||||
new Path(updateTable.getConfig().getBasePath() + "/" + insertResult.getStat().getPath()),
|
||||
mergeHandle.getWriterSchemaWithMetafields());
|
||||
for (GenericRecord rec : oldRecords) {
|
||||
mergeHandle.write(rec);
|
||||
}
|
||||
|
||||
@@ -754,7 +754,7 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
List<HoodieCleanStat> hoodieCleanStats = runCleaner(config);
|
||||
assertEquals(3,
|
||||
getCleanStat(hoodieCleanStats, p0).getSuccessDeleteFiles()
|
||||
.size(), "Must clean three files, one parquet and 2 log files");
|
||||
.size(), "Must clean three files, one base and 2 log files");
|
||||
assertFalse(testTable.baseFileExists(p0, "000", file1P0));
|
||||
assertFalse(testTable.logFilesExist(p0, "000", file1P0, 1, 2));
|
||||
assertTrue(testTable.baseFileExists(p0, "001", file1P0));
|
||||
@@ -797,7 +797,7 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
List<HoodieCleanStat> hoodieCleanStats = runCleaner(config);
|
||||
assertEquals(3,
|
||||
getCleanStat(hoodieCleanStats, p0).getSuccessDeleteFiles()
|
||||
.size(), "Must clean three files, one parquet and 2 log files");
|
||||
.size(), "Must clean three files, one base and 2 log files");
|
||||
assertFalse(testTable.baseFileExists(p0, "000", file1P0));
|
||||
assertFalse(testTable.logFilesExist(p0, "000", file1P0, 1, 2));
|
||||
assertTrue(testTable.baseFileExists(p0, "001", file1P0));
|
||||
@@ -935,8 +935,9 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
String partition1 = DEFAULT_PARTITION_PATHS[0];
|
||||
String partition2 = DEFAULT_PARTITION_PATHS[1];
|
||||
|
||||
String fileName1 = "data1_1_000.parquet";
|
||||
String fileName2 = "data2_1_000.parquet";
|
||||
String extension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
|
||||
String fileName1 = "data1_1_000" + extension;
|
||||
String fileName2 = "data2_1_000" + extension;
|
||||
|
||||
String filePath1 = metaClient.getBasePath() + "/" + partition1 + "/" + fileName1;
|
||||
String filePath2 = metaClient.getBasePath() + "/" + partition1 + "/" + fileName2;
|
||||
@@ -1025,8 +1026,9 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
String partition1 = DEFAULT_PARTITION_PATHS[0];
|
||||
String partition2 = DEFAULT_PARTITION_PATHS[1];
|
||||
|
||||
String fileName1 = "data1_1_000.parquet";
|
||||
String fileName2 = "data2_1_000.parquet";
|
||||
String extension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension();
|
||||
String fileName1 = "data1_1_000" + extension;
|
||||
String fileName2 = "data2_1_000" + extension;
|
||||
|
||||
Map<String, List<String>> filesToBeCleanedPerPartition = new HashMap<>();
|
||||
filesToBeCleanedPerPartition.put(partition1, Arrays.asList(fileName1));
|
||||
@@ -1314,7 +1316,7 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build())
|
||||
.build();
|
||||
// Deletions:
|
||||
// . FileId Parquet Logs Total Retained Commits
|
||||
// . FileId Base Logs Total Retained Commits
|
||||
// FileId7 5 10 15 009, 011
|
||||
// FileId6 5 10 15 009
|
||||
// FileId5 3 6 9 005
|
||||
@@ -1338,7 +1340,7 @@ public class TestCleaner extends HoodieClientTestBase {
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(2).build())
|
||||
.build();
|
||||
// Deletions:
|
||||
// . FileId Parquet Logs Total Retained Commits
|
||||
// . FileId Base Logs Total Retained Commits
|
||||
// FileId7 5 10 15 009, 011
|
||||
// FileId6 4 8 12 007, 009
|
||||
// FileId5 2 4 6 003 005
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.common.fs.ConsistencyGuard;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.fs.FailSafeConsistencyGuard;
|
||||
import org.apache.hudi.common.fs.OptimisticConsistencyGuard;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.testutils.FileCreateUtils;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
@@ -44,6 +45,8 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
*/
|
||||
public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
|
||||
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
|
||||
|
||||
// multiple parameters, uses Collection<Object[]>
|
||||
public static List<Arguments> consistencyGuardType() {
|
||||
return Arrays.asList(
|
||||
@@ -73,17 +76,19 @@ public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
ConsistencyGuardConfig config = getConsistencyGuardConfig(1, 1000, 1000);
|
||||
ConsistencyGuard passing = consistencyGuardType.equals(FailSafeConsistencyGuard.class.getName())
|
||||
? new FailSafeConsistencyGuard(fs, config) : new OptimisticConsistencyGuard(fs, config);
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-1_000" + BASE_FILE_EXTENSION));
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f2_1-0-1_000" + BASE_FILE_EXTENSION));
|
||||
passing.waitTillAllFilesAppear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000.parquet", basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000" + BASE_FILE_EXTENSION,
|
||||
basePath + "/partition/path/f2_1-0-1_000" + BASE_FILE_EXTENSION));
|
||||
|
||||
fs.delete(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"), false);
|
||||
fs.delete(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"), false);
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
fs.delete(new Path(basePath + "/partition/path/f1_1-0-1_000" + BASE_FILE_EXTENSION), false);
|
||||
fs.delete(new Path(basePath + "/partition/path/f2_1-0-1_000" + BASE_FILE_EXTENSION), false);
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000" + BASE_FILE_EXTENSION));
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f2_1-0-1_000" + BASE_FILE_EXTENSION));
|
||||
passing.waitTillAllFilesDisappear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000.parquet", basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000" + BASE_FILE_EXTENSION,
|
||||
basePath + "/partition/path/f2_1-0-1_000" + BASE_FILE_EXTENSION));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -92,7 +97,8 @@ public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
assertThrows(TimeoutException.class, () -> {
|
||||
passing.waitTillAllFilesAppear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-2_000.parquet", basePath + "/partition/path/f2_1-0-2_000.parquet"));
|
||||
.asList(basePath + "/partition/path/f1_1-0-2_000" + BASE_FILE_EXTENSION,
|
||||
basePath + "/partition/path/f2_1-0-2_000" + BASE_FILE_EXTENSION));
|
||||
});
|
||||
}
|
||||
|
||||
@@ -101,7 +107,8 @@ public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new OptimisticConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillAllFilesAppear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-2_000.parquet", basePath + "/partition/path/f2_1-0-2_000.parquet"));
|
||||
.asList(basePath + "/partition/path/f1_1-0-2_000" + BASE_FILE_EXTENSION,
|
||||
basePath + "/partition/path/f2_1-0-2_000" + BASE_FILE_EXTENSION));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -109,7 +116,7 @@ public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
assertThrows(TimeoutException.class, () -> {
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-2_000.parquet"));
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-2_000" + BASE_FILE_EXTENSION));
|
||||
});
|
||||
}
|
||||
|
||||
@@ -117,7 +124,7 @@ public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
public void testCheckFailingAppearsTimedWait() throws Exception {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new OptimisticConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-2_000.parquet"));
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-2_000" + BASE_FILE_EXTENSION));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -126,7 +133,8 @@ public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
assertThrows(TimeoutException.class, () -> {
|
||||
passing.waitTillAllFilesDisappear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000.parquet", basePath + "/partition/path/f2_1-0-2_000.parquet"));
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000" + BASE_FILE_EXTENSION,
|
||||
basePath + "/partition/path/f2_1-0-2_000" + BASE_FILE_EXTENSION));
|
||||
});
|
||||
}
|
||||
|
||||
@@ -135,7 +143,8 @@ public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new OptimisticConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillAllFilesDisappear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000.parquet", basePath + "/partition/path/f2_1-0-2_000.parquet"));
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000" + BASE_FILE_EXTENSION,
|
||||
basePath + "/partition/path/f2_1-0-2_000" + BASE_FILE_EXTENSION));
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -144,7 +153,7 @@ public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
assertThrows(TimeoutException.class, () -> {
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000" + BASE_FILE_EXTENSION));
|
||||
});
|
||||
}
|
||||
|
||||
@@ -153,7 +162,7 @@ public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new OptimisticConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000" + BASE_FILE_EXTENSION));
|
||||
}
|
||||
|
||||
private ConsistencyGuardConfig getConsistencyGuardConfig() {
|
||||
|
||||
@@ -33,6 +33,7 @@ import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
@@ -131,7 +132,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
|
||||
@BeforeEach
|
||||
public void init() throws IOException {
|
||||
init(HoodieFileFormat.PARQUET);
|
||||
init(HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
@@ -345,7 +346,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
List<HoodieRecord> records004 = dataGen.generateUpdates(updateTime, 100);
|
||||
updateRecords(records004, client, cfg, updateTime);
|
||||
|
||||
// verify RO incremental reads - only one parquet file shows up because updates to into log files
|
||||
// verify RO incremental reads - only one base file shows up because updates to into log files
|
||||
incrementalROFiles = getROIncrementalFiles(partitionPath, false);
|
||||
validateFiles(partitionPath, 1, incrementalROFiles, false, roJobConf, 200, commitTime1);
|
||||
assertEquals(firstFilePath, incrementalROFiles[0].getPath());
|
||||
@@ -358,7 +359,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
String compactionCommitTime = "005";
|
||||
client.scheduleCompactionAtInstant("005", Option.empty());
|
||||
|
||||
// verify RO incremental reads - only one parquet file shows up because updates go into log files
|
||||
// verify RO incremental reads - only one base file shows up because updates go into log files
|
||||
incrementalROFiles = getROIncrementalFiles(partitionPath, true);
|
||||
validateFiles(partitionPath,1, incrementalROFiles, false, roJobConf, 200, commitTime1);
|
||||
|
||||
@@ -436,7 +437,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts, written as parquet file)
|
||||
* Write 1 (only inserts, written as base file)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
@@ -465,7 +466,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
dataFilesToRead = tableView.getLatestBaseFiles();
|
||||
assertTrue(dataFilesToRead.findAny().isPresent(),
|
||||
"should list the parquet files we wrote in the delta commit");
|
||||
"should list the base files we wrote in the delta commit");
|
||||
|
||||
/**
|
||||
* Write 2 (only updates, written to .log file)
|
||||
@@ -613,7 +614,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
dataFilesToRead = tableView.getLatestBaseFiles();
|
||||
assertTrue(dataFilesToRead.findAny().isPresent(),
|
||||
"should list the parquet files we wrote in the delta commit");
|
||||
"should list the base files we wrote in the delta commit");
|
||||
|
||||
/**
|
||||
* Write 2 (inserts + updates - testing failed delta commit)
|
||||
@@ -630,7 +631,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
List<String> dataFiles = tableView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
|
||||
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles,
|
||||
basePath);
|
||||
assertEquals(recordsRead.size(), 200);
|
||||
assertEquals(200, recordsRead.size());
|
||||
|
||||
statuses = secondClient.upsert(jsc.parallelize(copyOfRecords, 1), commitTime1).collect();
|
||||
// Verify there are no errors
|
||||
@@ -674,7 +675,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
// Test successful delta commit rollback
|
||||
thirdClient.rollback(commitTime2);
|
||||
allFiles = listAllBaseFilesInPath(hoodieTable);
|
||||
// After rollback, there should be no parquet file with the failed commit time
|
||||
// After rollback, there should be no base file with the failed commit time
|
||||
assertEquals(0, Arrays.stream(allFiles)
|
||||
.filter(file -> file.getPath().getName().contains(commitTime2)).count());
|
||||
|
||||
@@ -768,7 +769,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
dataFilesToRead = tableView.getLatestBaseFiles();
|
||||
assertTrue(dataFilesToRead.findAny().isPresent(),
|
||||
"Should list the parquet files we wrote in the delta commit");
|
||||
"Should list the base files we wrote in the delta commit");
|
||||
|
||||
/**
|
||||
* Write 2 (inserts + updates)
|
||||
@@ -901,7 +902,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts, written as parquet file)
|
||||
* Write 1 (only inserts, written as base file)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
@@ -926,17 +927,17 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
BaseFileOnlyView roView = getHoodieTableFileSystemView(metaClient,
|
||||
metaClient.getCommitsTimeline().filterCompletedInstants(), allFiles);
|
||||
Stream<HoodieBaseFile> dataFilesToRead = roView.getLatestBaseFiles();
|
||||
Map<String, Long> parquetFileIdToSize =
|
||||
Map<String, Long> fileIdToSize =
|
||||
dataFilesToRead.collect(Collectors.toMap(HoodieBaseFile::getFileId, HoodieBaseFile::getFileSize));
|
||||
|
||||
roView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
dataFilesToRead = roView.getLatestBaseFiles();
|
||||
List<HoodieBaseFile> dataFilesList = dataFilesToRead.collect(Collectors.toList());
|
||||
assertTrue(dataFilesList.size() > 0,
|
||||
"Should list the parquet files we wrote in the delta commit");
|
||||
"Should list the base files we wrote in the delta commit");
|
||||
|
||||
/**
|
||||
* Write 2 (only updates + inserts, written to .log file + correction of existing parquet file size)
|
||||
* Write 2 (only updates + inserts, written to .log file + correction of existing base file size)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
@@ -961,10 +962,10 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
hoodieTable.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(), allFiles);
|
||||
dataFilesToRead = roView.getLatestBaseFiles();
|
||||
List<HoodieBaseFile> newDataFilesList = dataFilesToRead.collect(Collectors.toList());
|
||||
Map<String, Long> parquetFileIdToNewSize =
|
||||
Map<String, Long> fileIdToNewSize =
|
||||
newDataFilesList.stream().collect(Collectors.toMap(HoodieBaseFile::getFileId, HoodieBaseFile::getFileSize));
|
||||
|
||||
assertTrue(parquetFileIdToNewSize.entrySet().stream().anyMatch(entry -> parquetFileIdToSize.get(entry.getKey()) < entry.getValue()));
|
||||
assertTrue(fileIdToNewSize.entrySet().stream().anyMatch(entry -> fileIdToSize.get(entry.getKey()) < entry.getValue()));
|
||||
|
||||
List<String> dataFiles = roView.getLatestBaseFiles().map(HoodieBaseFile::getPath).collect(Collectors.toList());
|
||||
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(hadoopConf, dataFiles,
|
||||
@@ -1082,8 +1083,9 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
// Do a compaction
|
||||
String instantTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
statuses = (JavaRDD<WriteStatus>) writeClient.compact(instantTime);
|
||||
assertEquals(statuses.map(status -> status.getStat().getPath().contains("parquet")).count(), numLogFiles);
|
||||
assertEquals(statuses.count(), numLogFiles);
|
||||
String extension = table.getBaseFileExtension();
|
||||
assertEquals(numLogFiles, statuses.map(status -> status.getStat().getPath().contains(extension)).count());
|
||||
assertEquals(numLogFiles, statuses.count());
|
||||
writeClient.commitCompaction(instantTime, statuses, Option.empty());
|
||||
}
|
||||
}
|
||||
@@ -1215,9 +1217,10 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
// Do a compaction
|
||||
newCommitTime = writeClient.scheduleCompaction(Option.empty()).get().toString();
|
||||
statuses = (JavaRDD<WriteStatus>) writeClient.compact(newCommitTime);
|
||||
// Ensure all log files have been compacted into parquet files
|
||||
assertEquals(statuses.map(status -> status.getStat().getPath().contains("parquet")).count(), numLogFiles);
|
||||
assertEquals(statuses.count(), numLogFiles);
|
||||
// Ensure all log files have been compacted into base files
|
||||
String extension = table.getBaseFileExtension();
|
||||
assertEquals(numLogFiles, statuses.map(status -> status.getStat().getPath().contains(extension)).count());
|
||||
assertEquals(numLogFiles, statuses.count());
|
||||
//writeClient.commitCompaction(newCommitTime, statuses, Option.empty());
|
||||
// Trigger a rollback of compaction
|
||||
table.getActiveTimeline().reload();
|
||||
@@ -1463,7 +1466,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts, written as parquet file)
|
||||
* Write 1 (only inserts, written as base file)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
@@ -1493,7 +1496,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
roView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
dataFilesToRead = roView.getLatestBaseFiles();
|
||||
assertTrue(dataFilesToRead.findAny().isPresent(),
|
||||
"should list the parquet files we wrote in the delta commit");
|
||||
"should list the base files we wrote in the delta commit");
|
||||
|
||||
/**
|
||||
* Write 2 (only updates, written to .log file)
|
||||
@@ -1603,7 +1606,7 @@ public class TestHoodieMergeOnReadTable extends HoodieClientTestHarness {
|
||||
roView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
|
||||
dataFilesToRead = roView.getLatestBaseFiles();
|
||||
assertTrue(dataFilesToRead.findAny().isPresent(),
|
||||
"should list the parquet files we wrote in the delta commit");
|
||||
"should list the base files we wrote in the delta commit");
|
||||
}
|
||||
|
||||
private void updateRecords(List<HoodieRecord> records, SparkRDDWriteClient client,
|
||||
|
||||
@@ -18,6 +18,8 @@
|
||||
|
||||
package org.apache.hudi.table.action.bootstrap;
|
||||
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
@@ -47,13 +49,15 @@ public class TestBootstrapUtils extends HoodieClientTestBase {
|
||||
});
|
||||
|
||||
// Files inside partitions and marker directories
|
||||
List<String> files = Arrays.asList(
|
||||
"2016/04/15/1_1-0-1_20190528120000.parquet",
|
||||
"2016/04/15/2_1-0-1_20190528120000.parquet",
|
||||
"2016/05/16/3_1-0-1_20190528120000.parquet",
|
||||
"2016/05/16/4_1-0-1_20190528120000.parquet",
|
||||
"2016/04/17/5_1-0-1_20190528120000.parquet",
|
||||
"2016/04/17/6_1-0-1_20190528120000.parquet");
|
||||
List<String> files = Stream.of(
|
||||
"2016/04/15/1_1-0-1_20190528120000",
|
||||
"2016/04/15/2_1-0-1_20190528120000",
|
||||
"2016/05/16/3_1-0-1_20190528120000",
|
||||
"2016/05/16/4_1-0-1_20190528120000",
|
||||
"2016/04/17/5_1-0-1_20190528120000",
|
||||
"2016/04/17/6_1-0-1_20190528120000")
|
||||
.map(file -> file + metaClient.getTableConfig().getBaseFileFormat().getFileExtension())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
files.forEach(f -> {
|
||||
try {
|
||||
|
||||
@@ -29,8 +29,8 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.testutils.Transformations;
|
||||
import org.apache.hudi.common.util.BaseFileUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -154,14 +154,14 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
assertEquals(1, allFiles.length);
|
||||
|
||||
// Read out the bloom filter and make sure filter can answer record exist or not
|
||||
Path parquetFilePath = allFiles[0].getPath();
|
||||
BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(hadoopConf, parquetFilePath);
|
||||
Path filePath = allFiles[0].getPath();
|
||||
BloomFilter filter = BaseFileUtils.getInstance(table.getBaseFileFormat()).readBloomFilterFromMetadata(hadoopConf, filePath);
|
||||
for (HoodieRecord record : records) {
|
||||
assertTrue(filter.mightContain(record.getRecordKey()));
|
||||
}
|
||||
|
||||
// Read the parquet file, check the record content
|
||||
List<GenericRecord> fileRecords = ParquetUtils.readAvroRecords(hadoopConf, parquetFilePath);
|
||||
// Read the base file, check the record content
|
||||
List<GenericRecord> fileRecords = BaseFileUtils.getInstance(table.getBaseFileFormat()).readAvroRecords(hadoopConf, filePath);
|
||||
GenericRecord newRecord;
|
||||
int index = 0;
|
||||
for (GenericRecord record : fileRecords) {
|
||||
@@ -192,12 +192,12 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
allFiles = getIncrementalFiles(partitionPath, firstCommitTime, -1);
|
||||
assertEquals(1, allFiles.length);
|
||||
// verify new incremental file group is same as the previous one
|
||||
assertEquals(FSUtils.getFileId(parquetFilePath.getName()), FSUtils.getFileId(allFiles[0].getPath().getName()));
|
||||
assertEquals(FSUtils.getFileId(filePath.getName()), FSUtils.getFileId(allFiles[0].getPath().getName()));
|
||||
|
||||
// Check whether the record has been updated
|
||||
Path updatedParquetFilePath = allFiles[0].getPath();
|
||||
Path updatedFilePath = allFiles[0].getPath();
|
||||
BloomFilter updatedFilter =
|
||||
ParquetUtils.readBloomFilterFromParquetMetadata(hadoopConf, updatedParquetFilePath);
|
||||
BaseFileUtils.getInstance(metaClient).readBloomFilterFromMetadata(hadoopConf, updatedFilePath);
|
||||
for (HoodieRecord record : records) {
|
||||
// No change to the _row_key
|
||||
assertTrue(updatedFilter.mightContain(record.getRecordKey()));
|
||||
@@ -206,7 +206,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
assertTrue(updatedFilter.mightContain(insertedRecord1.getRecordKey()));
|
||||
records.add(insertedRecord1);// add this so it can further check below
|
||||
|
||||
ParquetReader updatedReader = ParquetReader.builder(new AvroReadSupport<>(), updatedParquetFilePath).build();
|
||||
ParquetReader updatedReader = ParquetReader.builder(new AvroReadSupport<>(), updatedFilePath).build();
|
||||
index = 0;
|
||||
while ((newRecord = (GenericRecord) updatedReader.read()) != null) {
|
||||
assertEquals(newRecord.get("_row_key").toString(), records.get(index).getRecordKey());
|
||||
@@ -393,7 +393,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
// Check the updated file
|
||||
int counts = 0;
|
||||
for (File file : Paths.get(basePath, "2016/01/31").toFile().listFiles()) {
|
||||
if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()).equals(instantTime)) {
|
||||
if (file.getName().endsWith(table.getBaseFileExtension()) && FSUtils.getCommitTime(file.getName()).equals(instantTime)) {
|
||||
LOG.info(file.getName() + "-" + file.length());
|
||||
counts++;
|
||||
}
|
||||
|
||||
@@ -135,7 +135,7 @@ public class CompactionTestBase extends HoodieClientTestBase {
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
List<HoodieBaseFile> dataFilesToRead = getCurrentLatestBaseFiles(hoodieTable);
|
||||
assertTrue(dataFilesToRead.stream().findAny().isPresent(),
|
||||
"should list the parquet files we wrote in the delta commit");
|
||||
"should list the base files we wrote in the delta commit");
|
||||
validateDeltaCommit(firstInstant, fgIdToCompactionOperation, cfg);
|
||||
}
|
||||
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -274,7 +275,7 @@ public class TestHoodieCompactionStrategy {
|
||||
private final long size;
|
||||
|
||||
public TestHoodieBaseFile(long size) {
|
||||
super("/tmp/XYXYXYXYXYYX_11_20180918020003.parquet");
|
||||
super("/tmp/XYXYXYXYXYYX_11_20180918020003" + HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension());
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
|
||||
@@ -67,7 +67,7 @@ public class TestCopyOnWriteRollbackActionExecutor extends HoodieClientRollbackT
|
||||
final String p1 = "2015/03/16";
|
||||
final String p2 = "2015/03/17";
|
||||
final String p3 = "2016/03/15";
|
||||
// Let's create some commit files and parquet files
|
||||
// Let's create some commit files and base files
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient)
|
||||
.withPartitionMetaFiles(p1, p2, p3)
|
||||
.addCommit("001")
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
@@ -37,6 +38,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertIterableEquals;
|
||||
|
||||
public class TestRollbackUtils {
|
||||
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension();
|
||||
|
||||
private FileStatus generateFileStatus(String filePath) {
|
||||
Path dataFile1Path = new Path(filePath);
|
||||
@@ -62,15 +64,15 @@ public class TestRollbackUtils {
|
||||
String partitionPath2 = "/partitionPath2/";
|
||||
//prepare HoodieRollbackStat for different partition
|
||||
Map<FileStatus, Boolean> dataFilesOnlyStat1Files = new HashMap<>();
|
||||
dataFilesOnlyStat1Files.put(generateFileStatus(partitionPath1 + "dataFile1.parquet"), true);
|
||||
dataFilesOnlyStat1Files.put(generateFileStatus(partitionPath1 + "dataFile2.parquet"), true);
|
||||
dataFilesOnlyStat1Files.put(generateFileStatus(partitionPath1 + "dataFile1" + BASE_FILE_EXTENSION), true);
|
||||
dataFilesOnlyStat1Files.put(generateFileStatus(partitionPath1 + "dataFile2" + BASE_FILE_EXTENSION), true);
|
||||
HoodieRollbackStat dataFilesOnlyStat1 = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath1)
|
||||
.withDeletedFileResults(dataFilesOnlyStat1Files).build();
|
||||
|
||||
Map<FileStatus, Boolean> dataFilesOnlyStat2Files = new HashMap<>();
|
||||
dataFilesOnlyStat2Files.put(generateFileStatus(partitionPath2 + "dataFile1.parquet"), true);
|
||||
dataFilesOnlyStat2Files.put(generateFileStatus(partitionPath2 + "dataFile2.parquet"), true);
|
||||
dataFilesOnlyStat2Files.put(generateFileStatus(partitionPath2 + "dataFile1" + BASE_FILE_EXTENSION), true);
|
||||
dataFilesOnlyStat2Files.put(generateFileStatus(partitionPath2 + "dataFile2" + BASE_FILE_EXTENSION), true);
|
||||
HoodieRollbackStat dataFilesOnlyStat2 = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath2)
|
||||
.withDeletedFileResults(dataFilesOnlyStat1Files).build();
|
||||
@@ -83,7 +85,7 @@ public class TestRollbackUtils {
|
||||
//prepare HoodieRollbackStat for failed and block append
|
||||
Map<FileStatus, Boolean> dataFilesOnlyStat3Files = new HashMap<>();
|
||||
dataFilesOnlyStat3Files.put(generateFileStatus(partitionPath1 + "dataFile1.log"), true);
|
||||
dataFilesOnlyStat3Files.put(generateFileStatus(partitionPath1 + "dataFile3.parquet"), false);
|
||||
dataFilesOnlyStat3Files.put(generateFileStatus(partitionPath1 + "dataFile3" + BASE_FILE_EXTENSION), false);
|
||||
HoodieRollbackStat dataFilesOnlyStat3 = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath1)
|
||||
.withDeletedFileResults(dataFilesOnlyStat3Files).build();
|
||||
@@ -98,10 +100,10 @@ public class TestRollbackUtils {
|
||||
HoodieRollbackStat dataFilesOnlyStatMerge1 =
|
||||
RollbackUtils.mergeRollbackStat(dataFilesOnlyStat1, dataFilesOnlyStat3);
|
||||
assertEquals(partitionPath1, dataFilesOnlyStatMerge1.getPartitionPath());
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile3.parquet"),
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile3" + BASE_FILE_EXTENSION),
|
||||
dataFilesOnlyStatMerge1.getFailedDeleteFiles());
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile1.parquet",
|
||||
partitionPath1 + "dataFile2.parquet", partitionPath1 + "dataFile1.log").stream().sorted().collect(Collectors.toList()),
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile1" + BASE_FILE_EXTENSION,
|
||||
partitionPath1 + "dataFile2" + BASE_FILE_EXTENSION, partitionPath1 + "dataFile1.log").stream().sorted().collect(Collectors.toList()),
|
||||
dataFilesOnlyStatMerge1.getSuccessDeleteFiles().stream().sorted().collect(Collectors.toList()));
|
||||
assertEquals(0, dataFilesOnlyStatMerge1.getCommandBlocksCount().size());
|
||||
|
||||
@@ -109,10 +111,10 @@ public class TestRollbackUtils {
|
||||
HoodieRollbackStat dataFilesOnlyStatMerge2 =
|
||||
RollbackUtils.mergeRollbackStat(dataFilesOnlyStatMerge1, dataFilesOnlyStat4);
|
||||
assertEquals(partitionPath1, dataFilesOnlyStatMerge1.getPartitionPath());
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile3.parquet").stream().sorted().collect(Collectors.toList()),
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile3" + BASE_FILE_EXTENSION).stream().sorted().collect(Collectors.toList()),
|
||||
dataFilesOnlyStatMerge2.getFailedDeleteFiles().stream().sorted().collect(Collectors.toList()));
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile1.parquet",
|
||||
partitionPath1 + "dataFile2.parquet", partitionPath1 + "dataFile1.log").stream().sorted().collect(Collectors.toList()),
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile1" + BASE_FILE_EXTENSION,
|
||||
partitionPath1 + "dataFile2" + BASE_FILE_EXTENSION, partitionPath1 + "dataFile1.log").stream().sorted().collect(Collectors.toList()),
|
||||
dataFilesOnlyStatMerge2.getSuccessDeleteFiles().stream().sorted().collect(Collectors.toList()));
|
||||
assertEquals(CollectionUtils.createImmutableMap(generateFileStatus(partitionPath1 + "dataFile1.log"), 10L),
|
||||
dataFilesOnlyStatMerge2.getCommandBlocksCount());
|
||||
|
||||
@@ -49,7 +49,6 @@ import org.junit.jupiter.api.io.TempDir;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
|
||||
import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestUtils.RAW_TRIPS_TEST_NAME;
|
||||
|
||||
@@ -120,7 +119,6 @@ public class FunctionalTestHarness implements SparkProvider, DFSProvider, Hoodie
|
||||
.setTableName(RAW_TRIPS_TEST_NAME)
|
||||
.setTableType(COPY_ON_WRITE)
|
||||
.setPayloadClass(HoodieAvroPayload.class)
|
||||
.setBaseFileFormat(PARQUET.toString())
|
||||
.fromProperties(props)
|
||||
.build();
|
||||
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, props);
|
||||
|
||||
Reference in New Issue
Block a user