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

@@ -155,7 +155,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id, that contains it.
int parallelism = autoComputeParallelism(recordsPerPartition, partitionToFileInfo,
partitionRecordKeyPairRDD);
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD,
return findMatchingFilesForRecordKeys(hoodieTable, partitionToFileInfo,
partitionRecordKeyPairRDD,
parallelism);
}
@@ -257,7 +258,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.mapToPair(ft -> {
try {
String[] minMaxKeys = ParquetUtils
.readMinMaxRecordKeys(ft._2().getFileStatus().getPath());
.readMinMaxRecordKeys(hoodieTable.getHadoopConf(),
ft._2().getFileStatus().getPath());
return new Tuple2<>(ft._1(),
new BloomIndexFileInfo(ft._2().getFileName(), minMaxKeys[0], minMaxKeys[1]));
} catch (MetadataNotFoundException me) {
@@ -358,7 +360,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Make sure the parallelism is atleast the groupby parallelism for tagging location
*/
@VisibleForTesting
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(
JavaPairRDD<String, String> findMatchingFilesForRecordKeys(HoodieTable hoodieTable,
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD,
int totalSubpartitions) {
@@ -372,7 +374,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.sortByKey(true, joinParallelism);
return fileSortedTripletRDD
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true)
.mapPartitionsWithIndex(
new HoodieBloomIndexCheckFunction(hoodieTable, config.getBasePath()), true)
.flatMap(indexLookupResults -> indexLookupResults.iterator())
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
.flatMapToPair(lookupResult -> {

View File

@@ -24,10 +24,12 @@ import com.uber.hoodie.common.util.ParquetUtils;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIndexException;
import com.uber.hoodie.func.LazyIterableIterator;
import com.uber.hoodie.table.HoodieTable;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -45,20 +47,24 @@ public class HoodieBloomIndexCheckFunction implements
private final String basePath;
public HoodieBloomIndexCheckFunction(String basePath) {
private final HoodieTable table;
public HoodieBloomIndexCheckFunction(HoodieTable table, String basePath) {
this.table = table;
this.basePath = basePath;
}
/**
* Given a list of row keys and one file, return only row keys existing in that file.
*/
public static List<String> checkCandidatesAgainstFile(List<String> candidateRecordKeys,
public static List<String> checkCandidatesAgainstFile(Configuration configuration,
List<String> candidateRecordKeys,
Path filePath) throws HoodieIndexException {
List<String> foundRecordKeys = new ArrayList<>();
try {
// Load all rowKeys from the file, to double-confirm
if (!candidateRecordKeys.isEmpty()) {
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(filePath);
Set<String> fileRowKeys = ParquetUtils.readRowKeysFromParquet(configuration, filePath);
logger.info("Loading " + fileRowKeys.size() + " row keys from " + filePath);
if (logger.isDebugEnabled()) {
logger.debug("Keys from " + filePath + " => " + fileRowKeys);
@@ -107,7 +113,8 @@ public class HoodieBloomIndexCheckFunction implements
private void initState(String fileName, String partitionPath) throws HoodieIndexException {
try {
Path filePath = new Path(basePath + "/" + partitionPath + "/" + fileName);
bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(filePath);
bloomFilter = ParquetUtils
.readBloomFilterFromParquetMetadata(table.getHadoopConf(), filePath);
candidateRecordKeys = new ArrayList<>();
currentFile = fileName;
currentParitionPath = partitionPath;
@@ -154,7 +161,7 @@ public class HoodieBloomIndexCheckFunction implements
.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
}
ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath)));
initState(fileName, partitionPath);
if (bloomFilter.mightContain(recordKey)) {
@@ -177,7 +184,7 @@ public class HoodieBloomIndexCheckFunction implements
logger.debug("#The candidate row keys for " + filePath + " => " + candidateRecordKeys);
}
ret.add(new IndexLookupResult(currentFile,
checkCandidatesAgainstFile(candidateRecordKeys, filePath)));
checkCandidatesAgainstFile(table.getHadoopConf(), candidateRecordKeys, filePath)));
}
} catch (Throwable e) {