1
0

Multi FS Support

- Reviving PR 191, to make FileSystem creation off actual path
 - Streamline all filesystem access to HoodieTableMetaClient
 - Hadoop Conf from Spark Context serialized & passed to executor code too
 - Pick up env vars prefixed with HOODIE_ENV_ into Configuration object
 - Cleanup usage of FSUtils.getFS, piggybacking off HoodieTableMetaClient.getFS
 - Adding s3a to supported schemes & support escaping "." in env vars
 - Tests use HoodieTestUtils.getDefaultHadoopConf
This commit is contained in:
Vinoth Chandar
2017-12-10 23:31:54 -08:00
committed by vinoth chandar
parent 44839b88c6
commit 0cd186c899
78 changed files with 851 additions and 535 deletions

View File

@@ -84,6 +84,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
private transient JavaSparkContext jsc = null;
private transient SQLContext sqlContext;
private transient FileSystem fs;
private String basePath = null;
private transient HoodieTestDataGenerator dataGen = null;
private String[] partitionPaths = {"2016/01/01", "2016/02/02", "2016/06/02"};
@@ -100,6 +101,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
fs = FSUtils.getFs(basePath.toString(), jsc.hadoopConfiguration());
HoodieTestUtils.init(basePath);
dataGen = new HoodieTestDataGenerator();
}
@@ -212,7 +214,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieWriteConfig cfg = getConfig();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
FileSystem fs = FSUtils.getFs();
/**
* Write 1 (only inserts)
@@ -230,8 +231,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertPartitionMetadata(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, fs);
// verify that there is a commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath())
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
.getCommitTimeline();
assertEquals("Expecting a single commit.", 1,
@@ -242,7 +244,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
records.size(),
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
// Should have 100 records in table (check using Index), all in locations marked at commit
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
HoodieTable table = HoodieTable
.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), table)
.collect();
@@ -268,13 +271,13 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses);
// verify there are now 2 commits
timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline();
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
assertEquals("Expecting two commits.",
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2);
assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(),
newCommitTime);
metaClient = new HoodieTableMetaClient(fs, basePath);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
// Index should be able to locate all updates in correct locations.
@@ -305,7 +308,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieWriteConfig cfg = getConfig();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
FileSystem fs = FSUtils.getFs();
/**
* Write 1 (inserts and deletes)
@@ -326,8 +328,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses);
// verify that there is a commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath())
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
.getCommitTimeline();
assertEquals("Expecting a single commit.", 1,
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
@@ -336,7 +339,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertEquals("Must contain 200 records", fewRecordsForInsert.size(),
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count());
// Should have 100 records in table (check using Index), all in locations marked at commit
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
HoodieTable table = HoodieTable
.getHoodieTable(metaClient, getConfig());
List<HoodieRecord> taggedRecords = index
.tagLocation(jsc.parallelize(fewRecordsForInsert, 1), table).collect();
@@ -359,7 +363,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses);
// verify there are now 2 commits
timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath()).getCommitTimeline();
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
assertEquals("Expecting two commits.",
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), 2);
assertEquals("Latest commit should be 004", timeline.lastInstant().get().getTimestamp(),
@@ -393,7 +397,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
.build()).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
FileSystem fs = FSUtils.getFs();
HoodieTestDataGenerator
.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
@@ -451,8 +454,10 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
List<String> partitionPaths = FSUtils
.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTable table = HoodieTable
.getHoodieTable(metaClient, getConfig());
final TableFileSystemView.ReadOptimizedView view = table.getROFileSystemView();
List<HoodieDataFile> dataFiles = partitionPaths.stream().flatMap(s -> {
return view.getAllDataFiles(s).filter(f -> f.getCommitTime().equals("002"));
@@ -472,7 +477,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, basePath);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
@@ -490,7 +495,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1)
.build()).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
FileSystem fs = FSUtils.getFs();
HoodieTestDataGenerator
.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
@@ -531,7 +535,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses);
List<String> partitionPaths = FSUtils
.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView.ReadOptimizedView view1 = table.getROFileSystemView();
@@ -551,7 +556,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, basePath);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView.ReadOptimizedView view2 = table.getROFileSystemView();
@@ -573,7 +578,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
table.getCompletedSavepointTimeline().getInstants().findFirst().get();
client.rollbackToSavepoint(savepoint.getTimestamp());
metaClient = new HoodieTableMetaClient(fs, basePath);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = HoodieTable.getHoodieTable(metaClient, getConfig());
final TableFileSystemView.ReadOptimizedView view3 = table.getROFileSystemView();
dataFiles = partitionPaths.stream().flatMap(s -> {
@@ -602,7 +607,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
.retainFileVersions(maxVersions).build()).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
FileSystem fs = FSUtils.getFs();
/**
* do a big insert
@@ -618,8 +622,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses);
// verify that there is a commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath())
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
.getCommitTimeline();
assertEquals("Expecting a single commit.", 1,
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
@@ -647,7 +652,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Verify there are no errors
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
table = HoodieTable.getHoodieTable(metadata, getConfig());
timeline = table.getCommitsTimeline();
@@ -702,7 +708,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
.retainCommits(maxCommits).build()).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
HoodieIndex index = HoodieIndex.createIndex(cfg, jsc);
FileSystem fs = FSUtils.getFs();
/**
* do a big insert
@@ -718,13 +723,15 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses);
// verify that there is a commit
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(fs, metaClient.getMetaPath())
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient)
.getCommitTimeline();
assertEquals("Expecting a single commit.", 1,
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
// Should have 100 records in table (check using Index), all in locations marked at commit
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig());
HoodieTable table = HoodieTable
.getHoodieTable(metaClient, getConfig());
assertFalse(table.getCompletedCommitTimeline().empty());
String commitTime =
@@ -747,7 +754,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// Verify there are no errors
assertNoWriteErrors(statuses);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg);
HoodieTimeline activeTimeline = table1.getCompletedCommitTimeline();
Optional<HoodieInstant>
@@ -788,8 +796,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs();
HoodieTestDataGenerator.writePartitionMetadata(FSUtils.getFs(),
new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
HoodieTestDataGenerator
.writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
basePath);
// Only first two have commit files
@@ -878,9 +886,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
String commitTime2 = "20160502020601";
String commitTime3 = "20160506030611";
new File(basePath + "/.hoodie").mkdirs();
HoodieTestDataGenerator.writePartitionMetadata(FSUtils.getFs(),
new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
basePath);
HoodieTestDataGenerator
.writePartitionMetadata(fs, new String[]{"2016/05/01", "2016/05/02", "2016/05/06"},
basePath);
// One good commit
HoodieTestUtils.createCommitFiles(basePath, commitTime1);
@@ -956,8 +964,6 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
@Test
public void testSmallInsertHandlingForUpserts() throws Exception {
FileSystem fs = FSUtils.getFs();
final String TEST_PARTITION_PATH = "2016/09/26";
final int INSERT_SPLIT_LIMIT = 100;
// setup the small file handling params
@@ -982,7 +988,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
String file1 = statuses.get(0).getFileId();
assertEquals("file should contain 100 records",
ParquetUtils.readRowKeysFromParquet(new Path(basePath,
ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), new Path(basePath,
TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(),
100);
@@ -1006,9 +1012,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
Path newFile = new Path(basePath,
TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1));
assertEquals("file should contain 140 records",
ParquetUtils.readRowKeysFromParquet(newFile).size(), 140);
ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
List<GenericRecord> records = ParquetUtils.readAvroRecords(newFile);
List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), newFile);
for (GenericRecord record : records) {
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
assertEquals("only expect commit2", commitTime2,
@@ -1030,7 +1036,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses);
assertEquals("2 files needs to be committed.", 2, statuses.size());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieTable table = HoodieTable.getHoodieTable(metadata, config);
TableFileSystemView.ReadOptimizedView fileSystemView = table.getROFileSystemView();
List<HoodieDataFile> files = fileSystemView
@@ -1040,7 +1046,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
for (HoodieDataFile file : files) {
if (file.getFileName().contains(file1)) {
assertEquals("Existing file should be expanded", commitTime3, file.getCommitTime());
records = ParquetUtils.readAvroRecords(new Path(file.getPath()));
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), 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();
@@ -1057,7 +1063,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
keys2.size());
} else {
assertEquals("New file must be written for commit 3", commitTime3, file.getCommitTime());
records = ParquetUtils.readAvroRecords(new Path(file.getPath()));
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
for (GenericRecord record : records) {
String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
assertEquals("only expect commit3", commitTime3,
@@ -1091,12 +1097,12 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
List<WriteStatus> statuses = client.insert(insertRecordsRDD1, commitTime1).collect();
assertNoWriteErrors(statuses);
assertPartitionMetadata(new String[]{TEST_PARTITION_PATH}, FSUtils.getFs());
assertPartitionMetadata(new String[]{TEST_PARTITION_PATH}, fs);
assertEquals("Just 1 file needs to be added.", 1, statuses.size());
String file1 = statuses.get(0).getFileId();
assertEquals("file should contain 100 records",
ParquetUtils.readRowKeysFromParquet(new Path(basePath,
ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), new Path(basePath,
TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime1, 0, file1))).size(),
100);
@@ -1116,9 +1122,9 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
Path newFile = new Path(basePath,
TEST_PARTITION_PATH + "/" + FSUtils.makeDataFileName(commitTime2, 0, file1));
assertEquals("file should contain 140 records",
ParquetUtils.readRowKeysFromParquet(newFile).size(), 140);
ParquetUtils.readRowKeysFromParquet(jsc.hadoopConfiguration(), newFile).size(), 140);
List<GenericRecord> records = ParquetUtils.readAvroRecords(newFile);
List<GenericRecord> records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), 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();
@@ -1137,8 +1143,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
assertNoWriteErrors(statuses);
assertEquals("2 files needs to be committed.", 2, statuses.size());
FileSystem fs = FSUtils.getFs();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
List<HoodieDataFile> files =
table.getROFileSystemView().getLatestDataFilesBeforeOrOn(TEST_PARTITION_PATH, commitTime3)
@@ -1148,7 +1154,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
int totalInserts = 0;
for (HoodieDataFile file : files) {
assertEquals("All files must be at commit 3", commitTime3, file.getCommitTime());
records = ParquetUtils.readAvroRecords(new Path(file.getPath()));
records = ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), new Path(file.getPath()));
totalInserts += records.size();
}
assertEquals("Total number of records must add up", totalInserts,
@@ -1169,7 +1175,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
@@ -1183,7 +1190,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make next commit, with 1 insert & 1 update per partition
HoodieTestUtils.createCommitFiles(basePath, "001");
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true),
config);
String file2P0C1 = HoodieTestUtils
@@ -1206,7 +1214,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "002");
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
@@ -1260,8 +1269,7 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make 1 compaction commit
HoodieTestUtils.createCompactionCommitFiles(basePath, "001");
HoodieTable table = HoodieTable
.getHoodieTable(metaClient, config);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
assertEquals("Must clean three files, one parquet and 2 log files", 3,
getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size());
@@ -1286,9 +1294,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
config);
HoodieTable table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
assertEquals("Must not clean any files", 0,
@@ -1300,9 +1307,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make next commit, with 1 insert & 1 update per partition
HoodieTestUtils.createCommitFiles(basePath, "001");
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
config);
table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config);
String file2P0C1 = HoodieTestUtils
.createNewDataFile(basePath, partitionPaths[0], "001"); // insert
@@ -1324,7 +1330,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "002");
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
@@ -1341,7 +1348,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
// make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "003");
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update
@@ -1384,7 +1392,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieTestUtils.createCommitFiles(basePath, "000");
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
@@ -1449,7 +1458,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
updateAllFilesInPartition(filesP2C0, partitionPaths[2], "003");
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true),
.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
@@ -1476,8 +1486,8 @@ public class TestHoodieClientOnCopyOnWriteStorage implements Serializable {
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).build();
HoodieWriteClient client = new HoodieWriteClient(jsc, cfg);
FileSystem fs = FSUtils.getFs();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, cfg);
String commitTime = "000";