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

@@ -67,10 +67,10 @@ public class HoodieReadClient implements Serializable {
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
this.jsc = jsc;
this.fs = FSUtils.getFs();
this.fs = FSUtils.getFs(basePath, jsc.hadoopConfiguration());
// Create a Hoodie table which encapsulated the commits and files visible
this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true), null);
this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants();
this.index =
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
@@ -129,8 +129,8 @@ public class HoodieReadClient implements Serializable {
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD()
.mapToPair(row -> {
HoodieKey key = new HoodieKey(
row.<String>getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
row.<String>getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
return new Tuple2<>(key, row);
});

View File

@@ -53,18 +53,6 @@ import com.uber.hoodie.table.HoodieTable;
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
import com.uber.hoodie.table.WorkloadProfile;
import com.uber.hoodie.table.WorkloadStat;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.storage.StorageLevel;
import scala.Option;
import scala.Tuple2;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
@@ -76,6 +64,17 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.PairFunction;
import org.apache.spark.storage.StorageLevel;
import scala.Option;
import scala.Tuple2;
/**
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient
@@ -112,7 +111,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
boolean rollbackInFlight) {
this.fs = FSUtils.getFs();
this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
this.jsc = jsc;
this.config = clientConfig;
this.index = HoodieIndex.createIndex(config, jsc);
@@ -133,8 +132,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, table);
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
@@ -146,8 +146,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
writeContext = metrics.getCommitCtx();
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
try {
// De-dupe/merge if needed
@@ -180,8 +181,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
writeContext = metrics.getCommitCtx();
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
try {
// De-dupe/merge if needed
JavaRDD<HoodieRecord<T>> dedupedRecords =
@@ -237,8 +239,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
writeContext = metrics.getCommitCtx();
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
try {
// De-dupe/merge if needed
@@ -306,8 +309,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* file instead of using HoodieCommitMetadata
*/
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile,
HoodieTable<T> table,
String commitTime) throws HoodieCommitException {
HoodieTable<T> table, String commitTime) throws HoodieCommitException {
try {
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
profile.getPartitionPaths().stream().forEach(path -> {
@@ -409,8 +411,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
logger.info("Commiting " + commitTime);
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
@@ -485,8 +488,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @return true if the savepoint was created successfully
*/
public boolean savepoint(String user, String comment) {
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
if (table.getCompletedCommitTimeline().empty()) {
throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty");
}
@@ -512,8 +516,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @return true if the savepoint was created successfully
*/
public boolean savepoint(String commitTime, String user, String comment) {
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
@@ -575,8 +580,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @return true if the savepoint was deleted successfully
*/
public void deleteSavepoint(String savepointTime) {
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieInstant savePoint =
@@ -602,8 +608,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @return true if the savepoint was rollecback to successfully
*/
public boolean rollbackToSavepoint(String savepointTime) {
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieTimeline commitTimeline = table.getCommitsTimeline();
@@ -653,8 +660,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
String startRollbackTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieTimeline inflightTimeline = table.getInflightCommitTimeline();
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
@@ -761,8 +769,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
final Timer.Context context = metrics.getCleanCtx();
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(),
true), config);
List<HoodieCleanStat> cleanStats = table.clean(jsc);
if (cleanStats.isEmpty()) {
@@ -810,8 +819,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public void startCommitWithTime(String commitTime) {
logger.info("Generate a new commit time " + commitTime);
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
String commitActionType = table.getCommitActionType();
activeTimeline.createInflight(
@@ -827,8 +837,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/
private void compact(String compactionCommitTime) throws IOException {
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
Optional<HoodieCommitMetadata> compactionMetadata = table.compact(jsc, compactionCommitTime);
if (compactionMetadata.isPresent()) {
logger.info("Compacted successfully on commit " + compactionCommitTime);
@@ -876,8 +887,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* Cleanup all inflight commits
*/
private void rollbackInflightCommits() {
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTable<T> table = HoodieTable.getHoodieTable(
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true),
config);
HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights();
List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());

View File

@@ -72,7 +72,8 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends
HoodieIOHandle.cleanupTmpFilesFromCurrentCommit(hoodieConfig,
commitTime,
record.getPartitionPath(),
TaskContext.getPartitionId());
TaskContext.getPartitionId(),
hoodieTable);
partitionsCleaned.add(record.getPartitionPath());
}

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) {

View File

@@ -33,7 +33,6 @@ import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -51,14 +50,12 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
private final HoodieTimeline commitTimeline;
private HoodieTable<T> hoodieTable;
private HoodieWriteConfig config;
private FileSystem fs;
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
this.hoodieTable = hoodieTable;
this.fileSystemView = hoodieTable.getCompletedFileSystemView();
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
this.config = config;
this.fs = hoodieTable.getFs();
}

View File

@@ -126,7 +126,8 @@ public class HoodieCommitArchiveLog {
int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
.getHoodieTable(new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true),
config);
// GroupBy each action and limit each action timeline to maxCommitsToKeep
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
@@ -165,7 +166,7 @@ public class HoodieCommitArchiveLog {
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
log.info("Deleting instants " + archivedInstants);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true);
boolean success = true;
for (HoodieInstant archivedInstant : archivedInstants) {
@@ -188,7 +189,7 @@ public class HoodieCommitArchiveLog {
try {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
new HoodieTableMetaClient(fs.getConf(), config.getBasePath(), true);
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();

View File

@@ -47,7 +47,7 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
HoodieTable<T> hoodieTable) {
this.commitTime = commitTime;
this.config = config;
this.fs = FSUtils.getFs();
this.fs = hoodieTable.getMetaClient().getFs();
this.hoodieTable = hoodieTable;
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
this.fileSystemView = hoodieTable.getROFileSystemView();
@@ -73,8 +73,9 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
String commitTime,
String partitionPath,
int taskPartitionId) {
FileSystem fs = FSUtils.getFs();
int taskPartitionId,
HoodieTable hoodieTable) {
FileSystem fs = hoodieTable.getMetaClient().getFs();
try {
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
.format("%s/%s/%s", config.getBasePath(), partitionPath,

View File

@@ -16,6 +16,8 @@
package com.uber.hoodie.io.compact;
import static java.util.stream.Collectors.toList;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
@@ -34,13 +36,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FlatMapFunction;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
@@ -49,8 +44,12 @@ import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import static java.util.stream.Collectors.toList;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FlatMapFunction;
/**
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
@@ -73,7 +72,6 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
.getTableType().name());
// TODO - rollback any compactions in flight
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
List<String> partitionPaths =
@@ -102,15 +100,9 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
log.info("After filtering, Compacting " + operations + " files");
List<HoodieWriteStat> updateStatusMap =
jsc.parallelize(operations, operations.size())
.map(s -> executeCompaction(metaClient, config, s, compactionCommitTime))
.flatMap(new FlatMapFunction<List<HoodieWriteStat>, HoodieWriteStat>() {
@Override
public Iterator<HoodieWriteStat> call(
List<HoodieWriteStat> hoodieWriteStats)
throws Exception {
return hoodieWriteStats.iterator();
}
}).collect();
.map(s -> executeCompaction(hoodieTable, config, s, compactionCommitTime))
.flatMap(writeStatList -> writeStatList.iterator())
.collect();
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
for (HoodieWriteStat stat : updateStatusMap) {
@@ -134,10 +126,11 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
return true;
}
private List<HoodieWriteStat> executeCompaction(HoodieTableMetaClient metaClient,
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
private List<HoodieWriteStat> executeCompaction(HoodieTable hoodieTable,
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
throws IOException {
FileSystem fs = FSUtils.getFs();
FileSystem fs = hoodieTable.getMetaClient().getFs();
Schema readerSchema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
@@ -147,7 +140,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// Reads the entire avro file. Always only specific blocks should be read from the avro file (failure recover).
// Load all the delta commits since the last compaction commit and get all the blocks to be loaded and load it using CompositeAvroLogReader
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
String maxInstantTime = metaClient.getActiveTimeline()
.getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
@@ -162,8 +155,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
}
// Compacting is very similar to applying updates to existing file
HoodieCopyOnWriteTable table =
new HoodieCopyOnWriteTable(config, metaClient);
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, metaClient);
Iterator<List<WriteStatus>> result = table
.handleUpdate(commitTime, operation.getFileId(), scanner.iterator());
Iterable<List<WriteStatus>> resultIterable = () -> result;

View File

@@ -19,6 +19,7 @@ package com.uber.hoodie.io.storage;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
@@ -26,7 +27,6 @@ import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.hadoop.ParquetFileWriter;
import org.apache.parquet.hadoop.ParquetWriter;
@@ -51,9 +51,9 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
private final Schema schema;
private static Configuration registerFileSystem(Configuration conf) {
private static Configuration registerFileSystem(Path file, Configuration conf) {
Configuration returnConf = new Configuration(conf);
String scheme = FileSystem.getDefaultUri(conf).getScheme();
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
HoodieWrapperFileSystem.class.getName());
return returnConf;
@@ -67,11 +67,11 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED,
ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED, ParquetWriter.DEFAULT_WRITER_VERSION,
registerFileSystem(parquetConfig.getHadoopConf()));
registerFileSystem(file, parquetConfig.getHadoopConf()));
this.file =
HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
this.fs = (HoodieWrapperFileSystem) this.file
.getFileSystem(registerFileSystem(parquetConfig.getHadoopConf()));
.getFileSystem(registerFileSystem(file, parquetConfig.getHadoopConf()));
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
this.maxFileSize = parquetConfig.getMaxFileSize() + Math

View File

@@ -19,7 +19,6 @@ package com.uber.hoodie.io.storage;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
@@ -37,11 +36,12 @@ public class HoodieStorageWriterFactory {
throws IOException {
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
// Currently only parquet is supported
return newParquetStorageWriter(commitTime, path, config, schema);
return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable);
}
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
String commitTime, Path path, HoodieWriteConfig config, Schema schema) throws IOException {
String commitTime, Path path, HoodieWriteConfig config, Schema schema,
HoodieTable hoodieTable) throws IOException {
BloomFilter filter =
new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
HoodieAvroWriteSupport writeSupport =
@@ -50,7 +50,7 @@ public class HoodieStorageWriterFactory {
HoodieParquetConfig parquetConfig =
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
config.getParquetBlockSize(), config.getParquetPageSize(),
config.getParquetMaxFileSize(), FSUtils.getFs().getConf());
config.getParquetMaxFileSize(), hoodieTable.getHadoopConf());
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
}

View File

@@ -16,7 +16,8 @@
package com.uber.hoodie.io.storage;
import java.io.FileNotFoundException;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
@@ -33,7 +34,6 @@ import org.apache.hadoop.fs.ContentSummary;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileAlreadyExistsException;
import org.apache.hadoop.fs.FileChecksum;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -41,17 +41,14 @@ import org.apache.hadoop.fs.FsServerDefaults;
import org.apache.hadoop.fs.FsStatus;
import org.apache.hadoop.fs.LocatedFileStatus;
import org.apache.hadoop.fs.Options;
import org.apache.hadoop.fs.ParentNotDirectoryException;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.fs.XAttrSetFlag;
import org.apache.hadoop.fs.permission.AclEntry;
import org.apache.hadoop.fs.permission.AclStatus;
import org.apache.hadoop.fs.permission.FsAction;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Progressable;
@@ -70,6 +67,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
SUPPORT_SCHEMES.add("file");
SUPPORT_SCHEMES.add("hdfs");
SUPPORT_SCHEMES.add("s3");
SUPPORT_SCHEMES.add("s3a");
// Hoodie currently relies on underlying object store being fully
// consistent so only regional buckets should be used.
@@ -85,7 +84,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public void initialize(URI uri, Configuration conf) throws IOException {
// Get the default filesystem to decorate
fileSystem = FileSystem.get(conf);
Path path = new Path(uri);
// Remove 'hoodie-' prefix from path
if (path.toString().startsWith(HOODIE_SCHEME_PREFIX)) {
path = new Path(path.toString().replace(HOODIE_SCHEME_PREFIX, ""));
}
this.fileSystem = FSUtils.getFs(path.toString(), conf);
// Do not need to explicitly initialize the default filesystem, its done already in the above FileSystem.get
// fileSystem.initialize(FileSystem.getDefaultUri(conf), conf);
// fileSystem.setConf(conf);
@@ -219,7 +223,7 @@ public class HoodieWrapperFileSystem extends FileSystem {
}
@Override
public FileStatus[] listStatus(Path f) throws FileNotFoundException, IOException {
public FileStatus[] listStatus(Path f) throws IOException {
return fileSystem.listStatus(convertToDefaultPath(f));
}
@@ -415,19 +419,19 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public FileStatus[] listStatus(Path f, PathFilter filter)
throws FileNotFoundException, IOException {
throws IOException {
return fileSystem.listStatus(convertToDefaultPath(f), filter);
}
@Override
public FileStatus[] listStatus(Path[] files)
throws FileNotFoundException, IOException {
throws IOException {
return fileSystem.listStatus(convertDefaults(files));
}
@Override
public FileStatus[] listStatus(Path[] files, PathFilter filter)
throws FileNotFoundException, IOException {
throws IOException {
return fileSystem.listStatus(convertDefaults(files), filter);
}
@@ -444,13 +448,13 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public RemoteIterator<LocatedFileStatus> listLocatedStatus(Path f)
throws FileNotFoundException, IOException {
throws IOException {
return fileSystem.listLocatedStatus(convertToDefaultPath(f));
}
@Override
public RemoteIterator<LocatedFileStatus> listFiles(Path f, boolean recursive)
throws FileNotFoundException, IOException {
throws IOException {
return fileSystem.listFiles(convertToDefaultPath(f), recursive);
}
@@ -571,21 +575,21 @@ public class HoodieWrapperFileSystem extends FileSystem {
@Override
public void access(Path path, FsAction mode)
throws AccessControlException, FileNotFoundException, IOException {
throws IOException {
fileSystem.access(convertToDefaultPath(path), mode);
}
@Override
public void createSymlink(Path target, Path link, boolean createParent)
throws AccessControlException, FileAlreadyExistsException, FileNotFoundException,
ParentNotDirectoryException, UnsupportedFileSystemException, IOException {
throws
IOException {
fileSystem
.createSymlink(convertToDefaultPath(target), convertToDefaultPath(link), createParent);
}
@Override
public FileStatus getFileLinkStatus(Path f)
throws AccessControlException, FileNotFoundException, UnsupportedFileSystemException,
throws
IOException {
return fileSystem.getFileLinkStatus(convertToDefaultPath(f));
}
@@ -759,8 +763,12 @@ public class HoodieWrapperFileSystem extends FileSystem {
}
public static Path convertToHoodiePath(Path file, Configuration conf) {
String scheme = FileSystem.getDefaultUri(conf).getScheme();
return convertPathWithScheme(file, getHoodieScheme(scheme));
try {
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
return convertPathWithScheme(file, getHoodieScheme(scheme));
} catch (HoodieIOException e) {
throw e;
}
}
private Path convertToDefaultPath(Path oldPath) {

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

View File

@@ -32,19 +32,11 @@ import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Arrays;
@@ -55,6 +47,12 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
/**
@@ -74,8 +72,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
public HoodieMergeOnReadTable(HoodieWriteConfig config,
HoodieTableMetaClient metaClient) {
public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
super(config, metaClient);
}
@@ -195,7 +192,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.onParentPath(
new Path(this.getMetaClient().getBasePath(), partitionPath))
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
.withFs(FSUtils.getFs())
.withFs(getMetaClient().getFs())
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
Long numRollbackBlocks = 0L;
// generate metadata
@@ -215,7 +212,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
numRollbackBlocks++;
}
filesToNumBlocksRollback
.put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()),
.put(getMetaClient().getFs()
.getFileStatus(writer.getLogFile().getPath()),
numRollbackBlocks);
} catch (IOException | InterruptedException io) {
throw new HoodieRollbackException(

View File

@@ -34,12 +34,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieSavepointException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.io.Serializable;
import java.util.Iterator;
@@ -47,6 +41,11 @@ import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.conf.Configuration;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
import org.apache.spark.api.java.JavaSparkContext;
/**
* Abstract implementation of a HoodieTable
@@ -55,6 +54,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
protected final HoodieWriteConfig config;
protected final HoodieTableMetaClient metaClient;
private static Logger logger = LogManager.getLogger(HoodieTable.class);
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
@@ -87,8 +87,8 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
return metaClient;
}
public FileSystem getFs() {
return metaClient.getFs();
public Configuration getHadoopConf() {
return metaClient.getHadoopConf();
}
/**