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

@@ -24,6 +24,7 @@ import com.uber.hoodie.HoodieReadClient;
import com.uber.hoodie.HoodieWriteClient;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.minicluster.HdfsTestService;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.FSUtils;
@@ -38,7 +39,6 @@ import java.util.Map.Entry;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Path;
@@ -174,7 +174,7 @@ public class TestHDFSParquetImporter implements Serializable {
ParquetWriter<GenericRecord> writer = AvroParquetWriter
.<GenericRecord>builder(srcFile)
.withSchema(HoodieTestDataGenerator.avroSchema)
.withConf(new Configuration())
.withConf(HoodieTestUtils.getDefaultHadoopConf())
.build();
for (GenericRecord record : records) {
writer.write(record);

View File

@@ -44,17 +44,22 @@ public class TestHoodieSnapshotCopier {
@Before
public void init() throws IOException {
// Prepare directories
TemporaryFolder folder = new TemporaryFolder();
folder.create();
rootPath = folder.getRoot().getAbsolutePath();
basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME;
HoodieTestUtils.init(basePath);
outputPath = rootPath + "/output";
fs = FSUtils.getFs();
// Start a local Spark job
SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]");
jsc = new JavaSparkContext(conf);
try {
// Prepare directories
TemporaryFolder folder = new TemporaryFolder();
folder.create();
rootPath = "file://" + folder.getRoot().getAbsolutePath();
basePath = rootPath + "/" + HoodieTestUtils.RAW_TRIPS_TEST_NAME;
outputPath = rootPath + "/output";
fs = FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf());
HoodieTestUtils.init(basePath);
// Start a local Spark job
SparkConf conf = new SparkConf().setAppName("snapshot-test-job").setMaster("local[2]");
jsc = new JavaSparkContext(conf);
} catch (Exception e) {
e.printStackTrace();
}
}
@Test