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

@@ -89,7 +89,8 @@ public class TestCopyOnWriteTable {
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieWriteConfig config = makeHoodieClientConfig();
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
basePath);
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config);
HoodieCreateHandle io = new HoodieCreateHandle(config, commitTime, table, partitionPath);
@@ -115,7 +116,7 @@ public class TestCopyOnWriteTable {
// Prepare the AvroParquetIO
HoodieWriteConfig config = makeHoodieClientConfig();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
String partitionPath = "/2016/01/31";
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
@@ -154,7 +155,8 @@ public class TestCopyOnWriteTable {
// Read out the bloom filter and make sure filter can answer record exist or not
Path parquetFilePath = new Path(parquetFile.getAbsolutePath());
BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(parquetFilePath);
BloomFilter filter = ParquetUtils
.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), parquetFilePath);
for (HoodieRecord record : records) {
assertTrue(filter.mightContain(record.getRecordKey()));
}
@@ -163,7 +165,8 @@ public class TestCopyOnWriteTable {
+ FSUtils.getCommitTime(parquetFile.getName()) + ".commit").createNewFile();
// Read the parquet file, check the record content
List<GenericRecord> fileRecords = ParquetUtils.readAvroRecords(parquetFilePath);
List<GenericRecord> fileRecords = ParquetUtils
.readAvroRecords(jsc.hadoopConfiguration(), parquetFilePath);
GenericRecord newRecord;
int index = 0;
for (GenericRecord record : fileRecords) {
@@ -188,7 +191,7 @@ public class TestCopyOnWriteTable {
Thread.sleep(1000);
String newCommitTime = HoodieTestUtils.makeNewCommitTime();
metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
table = new HoodieCopyOnWriteTable(config, metadata);
Iterator<List<WriteStatus>> iter = table
.handleUpdate(newCommitTime, updatedRecord1.getCurrentLocation().getFileId(),
@@ -211,7 +214,7 @@ public class TestCopyOnWriteTable {
// Check whether the record has been updated
Path updatedParquetFilePath = new Path(updatedParquetFile.getAbsolutePath());
BloomFilter updatedFilter = ParquetUtils
.readBloomFilterFromParquetMetadata(updatedParquetFilePath);
.readBloomFilterFromParquetMetadata(jsc.hadoopConfiguration(), updatedParquetFilePath);
for (HoodieRecord record : records) {
// No change to the _row_key
assertTrue(updatedFilter.mightContain(record.getRecordKey()));
@@ -261,7 +264,7 @@ public class TestCopyOnWriteTable {
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
.withWriteStatusClass(MetadataMergeWriteStatus.class).build();
String firstCommitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
@@ -298,8 +301,8 @@ public class TestCopyOnWriteTable {
public void testInsertWithPartialFailures() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
FileSystem fs = FSUtils.getFs();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
FileSystem fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Write a few records, and get atleast one file
@@ -340,7 +343,7 @@ public class TestCopyOnWriteTable {
public void testInsertRecords() throws Exception {
HoodieWriteConfig config = makeHoodieClientConfig();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
// Case 1:
@@ -389,7 +392,7 @@ public class TestCopyOnWriteTable {
HoodieStorageConfig.newBuilder().limitFileSize(64 * 1024).parquetBlockSize(64 * 1024)
.parquetPageSize(64 * 1024).build()).build();
String commitTime = HoodieTestUtils.makeNewCommitTime();
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
List<HoodieRecord> records = new ArrayList<>();
@@ -437,7 +440,7 @@ public class TestCopyOnWriteTable {
HoodieClientTestUtils.fakeCommitFile(basePath, "001");
HoodieClientTestUtils.fakeDataFile(basePath, TEST_PARTITION_PATH, "001", "file1", fileSize);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(FSUtils.getFs(), basePath);
HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metadata);
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(

View File

@@ -19,6 +19,11 @@
package com.uber.hoodie.table;
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.HoodieClientTestUtils;
@@ -45,6 +50,14 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.compact.HoodieCompactor;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import java.io.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -61,25 +74,11 @@ import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.File;
import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static com.uber.hoodie.common.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class TestMergeOnReadTable {
private transient JavaSparkContext jsc = null;
private transient SQLContext sqlContext;
private String basePath = null;
private static String basePath = null;
private HoodieCompactor compactor;
private FileSystem fs;
@@ -94,12 +93,11 @@ public class TestMergeOnReadTable {
if (hdfsTestService != null) {
hdfsTestService.stop();
dfsCluster.shutdown();
;
}
FSUtils.setFs(null);
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the same JVM
FileSystem.closeAll();
HoodieTestUtils.resetFS();
HoodieTestUtils.resetFS(basePath);
}
@BeforeClass
@@ -113,30 +111,28 @@ public class TestMergeOnReadTable {
dfs = dfsCluster.getFileSystem();
}
FSUtils.setFs(dfs);
HoodieTestUtils.resetFS();
HoodieTestUtils.resetFS(basePath);
}
@Before
public void init() throws IOException {
this.fs = FSUtils.getFs();
// Initialize a local spark env
jsc = new JavaSparkContext(
HoodieClientTestUtils.getSparkConfForTest("TestHoodieMergeOnReadTable"));
jsc.hadoopConfiguration().addResource(FSUtils.getFs().getConf());
// Create a temp folder as the base path
TemporaryFolder folder = new TemporaryFolder();
folder.create();
basePath = folder.getRoot().getAbsolutePath();
fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
jsc.hadoopConfiguration().addResource(fs.getConf());
dfs.mkdirs(new Path(basePath));
FSUtils.setFs(dfs);
HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ);
sqlContext = new SQLContext(jsc); // SQLContext stuff
compactor = new HoodieRealtimeTableCompactor();
//SQLContext stuff
sqlContext = new SQLContext(jsc);
}
@After
@@ -167,7 +163,8 @@ public class TestMergeOnReadTable {
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
Optional<HoodieInstant> deltaCommit =
@@ -209,7 +206,7 @@ public class TestMergeOnReadTable {
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp());
@@ -229,8 +226,9 @@ public class TestMergeOnReadTable {
assertTrue(dataFilesToRead.findAny().isPresent());
// verify that there is a commit
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, cfg.getBasePath(), true), getConfig(false));
table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath(), true),
getConfig(false));
HoodieTimeline timeline = table.getCommitTimeline().filterCompletedInstants();
assertEquals("Expecting a single commit.", 1,
timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants());
@@ -284,7 +282,8 @@ public class TestMergeOnReadTable {
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
Optional<HoodieInstant> deltaCommit =
@@ -332,7 +331,7 @@ public class TestMergeOnReadTable {
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("Latest Delta commit should be 004", "004", deltaCommit.get().getTimestamp());
@@ -349,7 +348,7 @@ public class TestMergeOnReadTable {
List<String> dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath())
.collect(Collectors.toList());
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils
.getRecordsUsingInputFormat(dataFiles);
.getRecordsUsingInputFormat(dataFiles, basePath);
//Wrote 40 records and deleted 20 records, so remaining 40-20 = 20
assertEquals("Must contain 20 records", 20, recordsRead.size());
}
@@ -377,9 +376,8 @@ public class TestMergeOnReadTable {
//verify there are no errors
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
cfg.getBasePath());
Optional<HoodieInstant> commit =
metaClient.getActiveTimeline().getCommitTimeline().firstInstant();
assertTrue(commit.isPresent());
@@ -403,8 +401,9 @@ public class TestMergeOnReadTable {
//rollback a COW commit when TableType is MOR
client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable
.getHoodieTable(metaClient, cfg);
FileStatus[] allFiles = HoodieTestUtils
.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
HoodieTableFileSystemView roView = new HoodieTableFileSystemView(metaClient,
@@ -440,7 +439,8 @@ public class TestMergeOnReadTable {
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
assertNoWriteErrors(statuses);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
cfg.getBasePath());
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
Optional<HoodieInstant> deltaCommit =
@@ -476,7 +476,7 @@ public class TestMergeOnReadTable {
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
// Verify there are no errors
assertNoWriteErrors(statuses);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant();
assertTrue(deltaCommit.isPresent());
assertEquals("Latest Delta commit should be 002", "002", deltaCommit.get().getTimestamp());
@@ -487,19 +487,19 @@ public class TestMergeOnReadTable {
List<String> dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath())
.collect(Collectors.toList());
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils
.getRecordsUsingInputFormat(dataFiles);
.getRecordsUsingInputFormat(dataFiles, basePath);
assertEquals(recordsRead.size(), 200);
// Test delta commit rollback
client.rollback(newCommitTime);
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitTimeline(),
allFiles);
dataFiles = roView.getLatestDataFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles);
recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath);
assertEquals(recordsRead.size(), 200);
@@ -516,13 +516,14 @@ public class TestMergeOnReadTable {
assertNoWriteErrors(statuses);
HoodieCompactor compactor = new HoodieRealtimeTableCompactor();
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(true));
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
HoodieTable table = HoodieTable
.getHoodieTable(metaClient, getConfig(true));
compactor.compact(jsc, getConfig(true), table, HoodieActiveTimeline.createNewCommitTime());
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(),
allFiles);
@@ -541,7 +542,7 @@ public class TestMergeOnReadTable {
client.rollback(compactedCommitTime);
allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath());
metaClient = new HoodieTableMetaClient(fs, cfg.getBasePath());
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg);
roView = new HoodieTableFileSystemView(metaClient, hoodieTable.getCommitsTimeline(),
allFiles);