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

@@ -30,6 +30,7 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.api.Partition;
@@ -183,7 +184,7 @@ public class HiveSyncTool {
cmd.usage();
System.exit(1);
}
FileSystem fs = FSUtils.getFs();
FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
HiveConf hiveConf = new HiveConf();
hiveConf.addResource(fs.getConf());
new HiveSyncTool(cfg, hiveConf, fs).syncHoodieTable();

View File

@@ -91,7 +91,7 @@ public class HoodieHiveClient {
HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
this.syncConfig = cfg;
this.fs = fs;
this.metaClient = new HoodieTableMetaClient(fs, cfg.basePath, true);
this.metaClient = new HoodieTableMetaClient(fs.getConf(), cfg.basePath, true);
this.tableType = metaClient.getTableType();
LOG.info("Creating hive connection " + cfg.jdbcUrl);