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

@@ -34,12 +34,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieSavepointException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.io.Serializable;
import java.util.Iterator;
@@ -47,6 +41,11 @@ import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
import org.apache.spark.api.java.JavaSparkContext;
/**
* Abstract implementation of a HoodieTable
@@ -55,6 +54,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
protected final HoodieWriteConfig config;
protected final HoodieTableMetaClient metaClient;
private static Logger logger = LogManager.getLogger(HoodieTable.class);
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
@@ -87,8 +87,8 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
return metaClient;
}
public FileSystem getFs() {
return metaClient.getFs();
public Configuration getHadoopConf() {
return metaClient.getHadoopConf();
}
/**