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:
committed by
vinoth chandar
parent
44839b88c6
commit
0cd186c899
@@ -67,7 +67,7 @@ public class HoodieDataSourceHelpers {
|
||||
*/
|
||||
public static HoodieTimeline allCompletedCommitsCompactions(FileSystem fs, String basePath) {
|
||||
HoodieTable table = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), basePath, true), null);
|
||||
if (table.getMetaClient().getTableType().equals(HoodieTableType.MERGE_ON_READ)) {
|
||||
return table.getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
|
||||
@@ -47,7 +47,7 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
private val log = LogManager.getLogger(classOf[IncrementalRelation])
|
||||
|
||||
val fs = new Path(basePath).getFileSystem(sqlContext.sparkContext.hadoopConfiguration)
|
||||
val metaClient = new HoodieTableMetaClient(fs, basePath, true)
|
||||
val metaClient = new HoodieTableMetaClient(sqlContext.sparkContext.hadoopConfiguration, basePath, true)
|
||||
// MOR datasets not supported yet
|
||||
if (metaClient.getTableType.equals(HoodieTableType.MERGE_ON_READ)) {
|
||||
throw new HoodieException("Incremental view not implemented yet, for merge-on-read datasets")
|
||||
@@ -72,7 +72,8 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
val latestMeta = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(commitsToReturn.last).get)
|
||||
val metaFilePath = latestMeta.getFileIdAndFullPaths(basePath).values().iterator().next()
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema(new Path(metaFilePath)))
|
||||
AvroConversionUtils.convertAvroSchemaToStructType(ParquetUtils.readAvroSchema(
|
||||
sqlContext.sparkContext.hadoopConfiguration, new Path(metaFilePath)))
|
||||
}
|
||||
|
||||
override def schema: StructType = latestSchema
|
||||
|
||||
Reference in New Issue
Block a user