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

@@ -41,6 +41,7 @@ import java.util.Map;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat;
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
@@ -64,10 +65,13 @@ import org.junit.rules.TemporaryFolder;
public class HoodieRealtimeRecordReaderTest {
private JobConf jobConf;
private FileSystem fs;
@Before
public void setUp() {
jobConf = new JobConf();
fs = FSUtils
.getFs(basePath.getRoot().getAbsolutePath(), HoodieTestUtils.getDefaultHadoopConf());
}
@Rule
@@ -79,7 +83,9 @@ public class HoodieRealtimeRecordReaderTest {
HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(partitionDir.getPath()))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId)
.overBaseCommit(baseCommit).withFs(FSUtils.getFs()).build();
.overBaseCommit(baseCommit)
.withFs(fs)
.build();
List<IndexedRecord> records = new ArrayList<>();
for (int i = 0; i < numberOfRecords; i++) {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
@@ -123,8 +129,8 @@ public class HoodieRealtimeRecordReaderTest {
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<Void, ArrayWritable> reader =
new MapredParquetInputFormat().
getRecordReader(new FileSplit(split.getPath(), 0,
FSUtils.getFs().getLength(split.getPath()), (String[]) null), jobConf, null);
getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()),
(String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();
String names = fields.stream().map(f -> f.name().toString()).collect(Collectors.joining(","));
@@ -183,8 +189,8 @@ public class HoodieRealtimeRecordReaderTest {
//create a RecordReader to be used by HoodieRealtimeRecordReader
RecordReader<Void, ArrayWritable> reader =
new MapredParquetInputFormat().
getRecordReader(new FileSplit(split.getPath(), 0,
FSUtils.getFs().getLength(split.getPath()), (String[]) null), jobConf, null);
getRecordReader(new FileSplit(split.getPath(), 0, fs.getLength(split.getPath()),
(String[]) null), jobConf, null);
JobConf jobConf = new JobConf();
List<Schema.Field> fields = schema.getFields();