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

@@ -295,6 +295,6 @@ public class HoodieInputFormat extends MapredParquetInputFormat
}
Path baseDir = HoodieHiveUtil.getNthParent(dataPath, levels);
LOG.info("Reading hoodie metadata from path " + baseDir.toString());
return new HoodieTableMetaClient(fs, baseDir.toString());
return new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
}
}

View File

@@ -19,6 +19,7 @@ import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.DatasetNotFoundException;
import com.uber.hoodie.exception.HoodieException;
import java.io.Serializable;
@@ -86,7 +87,7 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
}
Path folder = null;
try {
FileSystem fs = path.getFileSystem(new Configuration());
FileSystem fs = path.getFileSystem(FSUtils.prepareHadoopConf(new Configuration()));
if (fs.isDirectory(path)) {
return true;
}
@@ -123,7 +124,7 @@ public class HoodieROTablePathFilter implements PathFilter, Serializable {
if (baseDir != null) {
try {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, baseDir.toString());
new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitTimeline()
.filterCompletedInstants(),

View File

@@ -84,7 +84,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
LOG.info("cfg ==> " + job.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR));
try {
baseFileSchema = readSchema(jobConf, split.getPath());
readAndCompactLog();
readAndCompactLog(jobConf);
} catch (IOException e) {
throw new HoodieIOException(
"Could not create HoodieRealtimeRecordReader on path " + this.split.getPath(), e);
@@ -110,7 +110,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
* Goes through the log files and populates a map with latest version of each key logged, since
* the base split was written.
*/
private void readAndCompactLog() throws IOException {
private void readAndCompactLog(JobConf jobConf) throws IOException {
Schema writerSchema = new AvroSchemaConverter().convert(baseFileSchema);
List<String> projectionFields = orderFields(
jobConf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR),
@@ -123,7 +123,8 @@ public class HoodieRealtimeRecordReader implements RecordReader<Void, ArrayWrita
String.format("About to read compacted logs %s for base split %s, projecting cols %s",
split.getDeltaFilePaths(), split.getPath(), projectionFields));
HoodieCompactedLogRecordScanner compactedLogRecordScanner =
new HoodieCompactedLogRecordScanner(FSUtils.getFs(), split.getBasePath(),
new HoodieCompactedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf),
split.getBasePath(),
split.getDeltaFilePaths(),
readerSchema, split.getMaxCommitTime());
// NOTE: HoodieCompactedLogRecordScanner will not return records for an in-flight commit

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();