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

@@ -52,7 +52,6 @@ import java.util.Set;
import java.util.stream.Collectors;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -413,10 +412,10 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
throw new HoodieUpsertException("Error in finding the old file path at commit " +
commitTime + " at fileLoc: " + fileLoc);
} else {
Configuration conf = FSUtils.getFs().getConf();
AvroReadSupport.setAvroReadSchema(conf, upsertHandle.getSchema());
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
ParquetReader<IndexedRecord> reader =
AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(conf).build();
AvroParquetReader.builder(upsertHandle.getOldFilePath()).withConf(getHadoopConf())
.build();
try {
IndexedRecord record;
while ((record = reader.read()) != null) {
@@ -500,8 +499,9 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
@Override
public List<HoodieCleanStat> clean(JavaSparkContext jsc) {
try {
FileSystem fs = getMetaClient().getFs();
List<String> partitionsToClean =
FSUtils.getAllPartitionPaths(getFs(), getMetaClient().getBasePath(),
FSUtils.getAllPartitionPaths(fs, getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning());
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config
.getCleanerPolicy());
@@ -522,7 +522,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits)
throws IOException {
logger.info("Cleaning path " + partitionPath);
FileSystem fs = FSUtils.getFs();
FileSystem fs = getMetaClient().getFs();
FileStatus[] toBeDeleted =
fs.listStatus(new Path(config.getBasePath(), partitionPath), path -> {
if (!path.toString().contains(".parquet")) {
@@ -558,7 +558,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
// delete all the data files for all these commits
logger.info("Clean out all parquet files generated for commits: " + commits);
List<HoodieRollbackStat> stats = jsc.parallelize(
FSUtils.getAllPartitionPaths(FSUtils.getFs(), this.getMetaClient().getBasePath(),
FSUtils.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning()))
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
// Scan all partitions files with this commit time
@@ -618,7 +618,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
.parallelize(partitionsToClean, cleanerParallelism)
.flatMapToPair(getFilesToDeleteFunc(this, config))
.repartition(cleanerParallelism) // repartition to remove skews
.mapPartitionsToPair(deleteFilesFunc(this, config))
.mapPartitionsToPair(deleteFilesFunc(this))
.reduceByKey(
// merge partition level clean stats below
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1
@@ -646,17 +646,17 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}).collect(Collectors.toList());
}
private PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat> deleteFilesFunc(
HoodieTable table, HoodieWriteConfig config) {
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat> deleteFilesFunc(
HoodieTable table) {
return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>) iter -> {
HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config);
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
FileSystem fs = table.getMetaClient().getFs();
while (iter.hasNext()) {
Tuple2<String, String> partitionDelFileTuple = iter.next();
String partitionPath = partitionDelFileTuple._1();
String deletePathStr = partitionDelFileTuple._2();
Boolean deletedFileResult = deleteFileAndGetResult(deletePathStr);
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
if (!partitionCleanStatMap.containsKey(partitionPath)) {
partitionCleanStatMap.put(partitionPath,
new PartitionCleanStat(partitionPath));
@@ -682,10 +682,11 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
};
}
private Boolean deleteFileAndGetResult(String deletePathStr) throws IOException {
private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr)
throws IOException {
Path deletePath = new Path(deletePathStr);
logger.debug("Working on delete path :" + deletePath);
boolean deleteResult = getFs().delete(deletePath, false);
boolean deleteResult = fs.delete(deletePath, false);
if (deleteResult) {
logger.debug("Cleaned file at path :" + deletePath);
}