1
0

Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0

The following is the gist of changes done

- All low-level operation of creating a commit code was in HoodieClient which made it hard to share code if there was a compaction commit.
- HoodieTableMetadata contained a mix of metadata and filtering files. (Also few operations required FileSystem to be passed in because those were called from TaskExecutors and others had FileSystem as a global variable). Since merge-on-read requires a lot of that code, but will have to change slightly on how it operates on the metadata and how it filters the files. The two set of operation are split into HoodieTableMetaClient and TableFileSystemView.
- Everything (active commits, archived commits, cleaner log, save point log and in future delta and compaction commits) in HoodieTableMetaClient is a HoodieTimeline. Timeline is a series of instants, which has an in-built concept of inflight and completed commit markers.
- A timeline can be queries for ranges, contains and also use to create new datapoint (create a new commit etc). Commit (and all the above metadata) creation/deletion is streamlined in a timeline
- Multiple timelines can be merged into a single timeline, giving us an audit timeline to whatever happened in a hoodie dataset. This also helps with #55.
- Move to java 8 and introduce java 8 succinct syntax in refactored code
This commit is contained in:
Prasanna Rajaperumal
2017-01-04 14:40:26 -08:00
parent 283269e57f
commit 8ee777a9bb
76 changed files with 3480 additions and 1932 deletions

View File

@@ -19,17 +19,18 @@ package com.uber.hoodie;
import com.google.common.base.Optional;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieCommits;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.index.HoodieBloomIndex;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -53,6 +54,7 @@ import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import scala.Tuple2;
@@ -75,7 +77,8 @@ public class HoodieReadClient implements Serializable {
* BloomIndex
*/
private transient final HoodieBloomIndex index;
private HoodieTableMetadata metadata;
private final HoodieTimeline commitTimeline;
private HoodieTableMetaClient metaClient;
private transient Optional<SQLContext> sqlContextOpt;
@@ -85,7 +88,8 @@ public class HoodieReadClient implements Serializable {
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
this.jsc = jsc;
this.fs = FSUtils.getFs();
this.metadata = new HoodieTableMetadata(fs, basePath);
this.metaClient = new HoodieTableMetaClient(fs, basePath, true);
this.commitTimeline = metaClient.getActiveCommitTimeline();
this.index = new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
this.sqlContextOpt = Optional.absent();
}
@@ -127,7 +131,7 @@ public class HoodieReadClient implements Serializable {
assertSqlContext();
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD =
index.fetchRecordLocation(hoodieKeys, metadata);
index.fetchRecordLocation(hoodieKeys, metaClient);
List<String> paths = keyToFileRDD
.filter(new Function<Tuple2<HoodieKey, Optional<String>>, Boolean>() {
@Override
@@ -177,17 +181,20 @@ public class HoodieReadClient implements Serializable {
public Dataset<Row> read(String... paths) {
assertSqlContext();
List<String> filteredPaths = new ArrayList<>();
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metaClient);
try {
for (String path : paths) {
if (!path.contains(metadata.getBasePath())) {
if (!path.contains(metaClient.getBasePath())) {
throw new HoodieException("Path " + path
+ " does not seem to be a part of a Hoodie dataset at base path "
+ metadata.getBasePath());
+ metaClient.getBasePath());
}
FileStatus[] latestFiles = metadata.getLatestVersions(fs.globStatus(new Path(path)));
for (FileStatus file : latestFiles) {
filteredPaths.add(file.getPath().toString());
List<HoodieDataFile> latestFiles = fileSystemView.streamLatestVersions(fs.globStatus(new Path(path))).collect(
Collectors.toList());
for (HoodieDataFile file : latestFiles) {
filteredPaths.add(file.getPath());
}
}
return sqlContextOpt.get().read()
@@ -205,15 +212,19 @@ public class HoodieReadClient implements Serializable {
*/
public Dataset<Row> readSince(String lastCommitTimestamp) {
List<String> commitsToReturn = metadata.findCommitsAfter(lastCommitTimestamp, Integer.MAX_VALUE);
List<String> commitsToReturn =
commitTimeline.findInstantsAfter(lastCommitTimestamp, Integer.MAX_VALUE)
.collect(Collectors.toList());
//TODO: we can potentially trim this down to only affected partitions, using CommitMetadata
try {
// Go over the commit metadata, and obtain the new files that need to be read.
HashMap<String, String> fileIdToFullPath = new HashMap<>();
for (String commit: commitsToReturn) {
HoodieCommitMetadata metadata =
HoodieCommitMetadata.fromBytes(commitTimeline.readInstantDetails(commit).get());
// get files from each commit, and replace any previous versions
fileIdToFullPath.putAll(metadata.getCommitMetadata(commit).getFileIdAndFullPaths());
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths());
}
return sqlContextOpt.get().read()
@@ -229,13 +240,13 @@ public class HoodieReadClient implements Serializable {
*/
public Dataset<Row> readCommit(String commitTime) {
assertSqlContext();
HoodieCommits commits = metadata.getAllCommits();
if (!commits.contains(commitTime)) {
if (!commitTimeline.containsInstant(commitTime)) {
new HoodieException("No commit exists at " + commitTime);
}
try {
HoodieCommitMetadata commitMetdata = metadata.getCommitMetadata(commitTime);
HoodieCommitMetadata commitMetdata =
HoodieCommitMetadata.fromBytes(commitTimeline.readInstantDetails(commitTime).get());
Collection<String> paths = commitMetdata.getFileIdAndFullPaths().values();
return sqlContextOpt.get().read()
.parquet(paths.toArray(new String[paths.size()]))
@@ -253,7 +264,7 @@ public class HoodieReadClient implements Serializable {
*/
public JavaPairRDD<HoodieKey, Optional<String>> checkExists(
JavaRDD<HoodieKey> hoodieKeys) {
return index.fetchRecordLocation(hoodieKeys, metadata);
return index.fetchRecordLocation(hoodieKeys, metaClient);
}
/**
@@ -264,7 +275,7 @@ public class HoodieReadClient implements Serializable {
* @return A subset of hoodieRecords RDD, with existing records filtered out.
*/
public JavaRDD<HoodieRecord> filterExists(JavaRDD<HoodieRecord> hoodieRecords) {
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, metadata);
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, metaClient);
return recordsWithLocation.filter(new Function<HoodieRecord, Boolean>() {
@Override
public Boolean call(HoodieRecord v1) throws Exception {
@@ -287,13 +298,13 @@ public class HoodieReadClient implements Serializable {
* @return
*/
public List<String> listCommitsSince(String commitTimestamp) {
return metadata.getAllCommits().findCommitsAfter(commitTimestamp, Integer.MAX_VALUE);
return commitTimeline.findInstantsAfter(commitTimestamp, Integer.MAX_VALUE).collect(Collectors.toList());
}
/**
* Returns the last successful commit (a successful write operation) into a Hoodie table.
*/
public String latestCommit() {
return metadata.getAllCommits().lastCommit();
return commitTimeline.lastInstant().get();
}
}

View File

@@ -22,8 +22,9 @@ import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
@@ -31,7 +32,7 @@ import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieInsertException;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.BulkInsertMapFunction;
import com.uber.hoodie.func.InsertMapFunction;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieCleaner;
import com.uber.hoodie.io.HoodieCommitArchiveLog;
@@ -39,7 +40,6 @@ import com.uber.hoodie.metrics.HoodieMetrics;
import com.uber.hoodie.table.HoodieTable;
import com.uber.hoodie.table.WorkloadProfile;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -67,16 +67,19 @@ import java.util.Collections;
import java.util.Date;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import scala.Option;
import scala.Tuple2;
/**
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient
* mutations on a HDFS dataset [upsert()]
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then
* perform efficient mutations on a HDFS dataset [upsert()]
*
* Note that, at any given time, there can only be one Spark job performing
* these operatons on a Hoodie dataset.
*
* Note that, at any given time, there can only be one Spark job performing these operatons on a
* Hoodie dataset.
*/
public class HoodieWriteClient<T extends HoodieRecordPayload> implements Serializable {
@@ -111,7 +114,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
this.config = clientConfig;
this.index = HoodieIndex.createIndex(config, jsc);
this.metrics = new HoodieMetrics(config, config.getTableName());
this.archiveLog = new HoodieCommitArchiveLog(clientConfig);
this.archiveLog = new HoodieCommitArchiveLog(clientConfig, fs);
if (rollbackInFlight) {
rollbackInflightCommits();
}
@@ -125,9 +129,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* @return A subset of hoodieRecords RDD, with existing records filtered out.
*/
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
final HoodieTableMetadata metadata =
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, metadata);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
JavaRDD<HoodieRecord<T>> recordsWithLocation = index.tagLocation(hoodieRecords, metaClient);
return recordsWithLocation.filter(new Function<HoodieRecord<T>, Boolean>() {
@Override
public Boolean call(HoodieRecord<T> v1) throws Exception {
@@ -140,19 +144,74 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
* Upserts a bunch of new records into the Hoodie table, at the supplied commitTime
*/
public JavaRDD<WriteStatus> upsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
final HoodieTableMetadata metadata =
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
writeContext = metrics.getCommitCtx();
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
final HoodieTable table =
HoodieTable.getHoodieTable(metaClient, commitTime, config);
try {
// De-dupe/merge if needed
JavaRDD<HoodieRecord<T>> dedupedRecords =
combineOnCondition(config.shouldCombineBeforeUpsert(), records,
config.getUpsertShuffleParallelism());
combineOnCondition(config.shouldCombineBeforeUpsert(), records,
config.getUpsertShuffleParallelism());
// perform index loop up to get existing location of records
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, metadata);
return upsertRecordsInternal(taggedRecords, commitTime, metadata, true);
JavaRDD<HoodieRecord<T>> taggedRecords = index.tagLocation(dedupedRecords, metaClient);
// Cache the tagged records, so we don't end up computing both
taggedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER());
WorkloadProfile profile = null;
if (table.isWorkloadProfileNeeded()) {
profile = new WorkloadProfile(taggedRecords);
logger.info("Workload profile :" + profile);
}
// obtain the upsert partitioner, and the run the tagger records through that & get a partitioned RDD.
final Partitioner upsertPartitioner = table.getUpsertPartitioner(profile);
JavaRDD<HoodieRecord<T>> partitionedRecords = taggedRecords.mapToPair(
new PairFunction<HoodieRecord<T>, Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>() {
@Override
public Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> call(
HoodieRecord<T> record) throws Exception {
return new Tuple2<>(new Tuple2<>(record.getKey(),
Option.apply(record.getCurrentLocation())), record);
}
}).partitionBy(upsertPartitioner).map(
new Function<Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>, HoodieRecord<T>>() {
@Override
public HoodieRecord<T> call(
Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> tuple)
throws Exception {
return tuple._2();
}
});
// Perform the actual writing.
JavaRDD<WriteStatus> upsertStatusRDD = partitionedRecords.mapPartitionsWithIndex(
new Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>>() {
@Override
public Iterator<List<WriteStatus>> call(Integer partition,
Iterator<HoodieRecord<T>> recordItr) throws Exception {
return table.handleUpsertPartition(partition, recordItr, upsertPartitioner);
}
}, true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
@Override
public Iterable<WriteStatus> call(List<WriteStatus> writeStatuses)
throws Exception {
return writeStatuses;
}
});
// Update the index back.
JavaRDD<WriteStatus> resultRDD = index.updateLocation(upsertStatusRDD, metaClient);
resultRDD = resultRDD.persist(config.getWriteStatusStorageLevel());
commitOnAutoCommit(commitTime, resultRDD);
return resultRDD;
} catch (Throwable e) {
if (e instanceof HoodieUpsertException) {
throw (HoodieUpsertException) e;
@@ -161,38 +220,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
}
/**
* Inserts the given HoodieRecords, into the table. This API is intended to be used for normal
* writes.
*
* This implementation skips the index check and is able to leverage benefits such as
* small file handling/blocking alignment, as with upsert(), by profiling the workload
*
* @param records HoodieRecords to insert
* @param commitTime Commit Time handle
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
*/
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
final HoodieTableMetadata metadata =
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
writeContext = metrics.getCommitCtx();
try {
// De-dupe/merge if needed
JavaRDD<HoodieRecord<T>> dedupedRecords =
combineOnCondition(config.shouldCombineBeforeInsert(), records,
config.getInsertShuffleParallelism());
return upsertRecordsInternal(dedupedRecords, commitTime, metadata, false);
} catch (Throwable e) {
if (e instanceof HoodieInsertException) {
throw e;
}
throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e);
}
}
private void commitOnAutoCommit(String commitTime, JavaRDD<WriteStatus> resultRDD) {
if (config.shouldAutoCommit()) {
if(config.shouldAutoCommit()) {
logger.info("Auto commit enabled: Committing " + commitTime);
boolean commitResult = commit(commitTime, resultRDD);
if (!commitResult) {
@@ -204,146 +233,65 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
private JavaRDD<HoodieRecord<T>> combineOnCondition(boolean condition,
JavaRDD<HoodieRecord<T>> records, int parallelism) {
if (condition) {
JavaRDD<HoodieRecord<T>> records, int parallelism) {
if(condition) {
return deduplicateRecords(records, parallelism);
}
return records;
}
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
return dedupedRecords.mapToPair(
new PairFunction<HoodieRecord<T>, Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>() {
@Override
public Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> call(
HoodieRecord<T> record) throws Exception {
return new Tuple2<>(new Tuple2<>(record.getKey(),
Option.apply(record.getCurrentLocation())), record);
}
}).partitionBy(partitioner).map(
new Function<Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>>, HoodieRecord<T>>() {
@Override
public HoodieRecord<T> call(
Tuple2<Tuple2<HoodieKey, Option<HoodieRecordLocation>>, HoodieRecord<T>> tuple)
throws Exception {
return tuple._2();
}
});
}
private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, WorkloadProfile profile) {
if (isUpsert) {
return table.getUpsertPartitioner(profile);
} else {
return table.getInsertPartitioner(profile);
}
}
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTableMetadata metadata,
String commitTime) {
// Update the index back
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, metadata);
// Trigger the insert and collect statuses
statuses = statuses.persist(config.getWriteStatusStorageLevel());
commitOnAutoCommit(commitTime, statuses);
return statuses;
}
private JavaRDD<WriteStatus> upsertRecordsInternal(JavaRDD<HoodieRecord<T>> preppedRecords,
String commitTime,
HoodieTableMetadata metadata,
final boolean isUpsert) {
final HoodieTable table =
HoodieTable.getHoodieTable(metadata.getTableType(), commitTime, config, metadata);
// Cache the tagged records, so we don't end up computing both
preppedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER());
WorkloadProfile profile = null;
if (table.isWorkloadProfileNeeded()) {
profile = new WorkloadProfile(preppedRecords);
logger.info("Workload profile :" + profile);
}
// partition using the insert partitioner
final Partitioner partitioner = getPartitioner(table, isUpsert, profile);
JavaRDD<HoodieRecord<T>> partitionedRecords = partition(preppedRecords, partitioner);
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords.mapPartitionsWithIndex(
new Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>>() {
@Override
public Iterator<List<WriteStatus>> call(Integer partition,
Iterator<HoodieRecord<T>> recordItr) throws Exception {
if (isUpsert) {
return table.handleUpsertPartition(partition, recordItr, partitioner);
} else {
return table.handleInsertPartition(partition, recordItr, partitioner);
}
}
}, true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
@Override
public Iterator<WriteStatus> call(List<WriteStatus> writeStatuses)
throws Exception {
return writeStatuses.iterator();
}
});
return updateIndexAndCommitIfNeeded(writeStatusRDD, metadata, commitTime);
}
/**
* Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk
* loads into a Hoodie table for the very first time (e.g: converting an existing dataset to
* Hoodie).
* Loads the given HoodieRecords, as inserts into the table.
* (This implementation uses sortBy and attempts to control the numbers of files with less memory)
*
* This implementation uses sortBy (which does range partitioning based on reservoir sampling) and
* attempts to control the numbers of files with less memory compared to the {@link
* HoodieWriteClient#insert(JavaRDD, String)}
*
* @param records HoodieRecords to insert
* @param records HoodieRecords to insert
* @param commitTime Commit Time handle
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
*
*/
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
final HoodieTableMetadata metadata =
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
public JavaRDD<WriteStatus> insert(JavaRDD<HoodieRecord<T>> records, final String commitTime) {
writeContext = metrics.getCommitCtx();
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
try {
// De-dupe/merge if needed
JavaRDD<HoodieRecord<T>> dedupedRecords =
combineOnCondition(config.shouldCombineBeforeInsert(), records,
config.getInsertShuffleParallelism());
combineOnCondition(config.shouldCombineBeforeInsert(), records,
config.getInsertShuffleParallelism());
// Now, sort the records and line them up nicely for loading.
JavaRDD<HoodieRecord<T>> sortedRecords =
dedupedRecords.sortBy(new Function<HoodieRecord<T>, String>() {
@Override
public String call(HoodieRecord<T> record) {
// Let's use "partitionPath + key" as the sort key. Spark, will ensure
// the records split evenly across RDD partitions, such that small partitions fit
// into 1 RDD partition, while big ones spread evenly across multiple RDD partitions
return String
.format("%s+%s", record.getPartitionPath(), record.getRecordKey());
}
}, true, config.getInsertShuffleParallelism());
dedupedRecords.sortBy(new Function<HoodieRecord<T>, String>() {
@Override
public String call(HoodieRecord<T> record) {
// Let's use "partitionPath + key" as the sort key. Spark, will ensure
// the records split evenly across RDD partitions, such that small partitions fit
// into 1 RDD partition, while big ones spread evenly across multiple RDD partitions
return String
.format("%s+%s", record.getPartitionPath(), record.getRecordKey());
}
}, true, config.getInsertShuffleParallelism());
JavaRDD<WriteStatus> writeStatusRDD = sortedRecords
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, metadata),
true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
@Override
public Iterator<WriteStatus> call(List<WriteStatus> writeStatuses)
throws Exception {
return writeStatuses.iterator();
}
});
return updateIndexAndCommitIfNeeded(writeStatusRDD, metadata, commitTime);
.mapPartitionsWithIndex(new InsertMapFunction<T>(commitTime, config, metaClient),
true).flatMap(new FlatMapFunction<List<WriteStatus>, WriteStatus>() {
@Override
public Iterable<WriteStatus> call(List<WriteStatus> writeStatuses)
throws Exception {
return writeStatuses;
}
});
// Update the index back
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, metaClient);
// Trigger the insert and collect statuses
statuses = statuses.persist(config.getWriteStatusStorageLevel());
commitOnAutoCommit(commitTime, statuses);
return statuses;
} catch (Throwable e) {
if (e instanceof HoodieInsertException) {
throw e;
}
throw new HoodieInsertException("Failed to bulk insert for commit time " + commitTime, e);
throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e);
}
}
@@ -352,118 +300,110 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
*/
public boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses) {
logger.info("Comitting " + commitTime);
Path commitFile =
new Path(config.getBasePath() + "/.hoodie/" + FSUtils.makeCommitFileName(commitTime));
try {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
if (fs.exists(commitFile)) {
throw new HoodieCommitException("Duplicate commit found. " + commitTime);
}
List<Tuple2<String, HoodieWriteStat>> stats =
writeStatuses.mapToPair(new PairFunction<WriteStatus, String, HoodieWriteStat>() {
@Override
public Tuple2<String, HoodieWriteStat> call(WriteStatus writeStatus)
throws Exception {
return new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat());
}
}).collect();
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
for (Tuple2<String, HoodieWriteStat> stat : stats) {
metadata.addWriteStat(stat._1(), stat._2());
}
// open a new file and write the commit metadata in
Path inflightCommitFile = new Path(config.getBasePath() + "/.hoodie/" + FSUtils
.makeInflightCommitFileName(commitTime));
FSDataOutputStream fsout = fs.create(inflightCommitFile, true);
fsout.writeBytes(new String(metadata.toJsonString().getBytes(StandardCharsets.UTF_8),
StandardCharsets.UTF_8));
fsout.close();
boolean success = fs.rename(inflightCommitFile, commitFile);
if (success) {
// We cannot have unbounded commit files. Archive commits if we have to archive
archiveLog.archiveIfRequired();
// Call clean to cleanup if there is anything to cleanup after the commit,
clean();
if (writeContext != null) {
long durationInMs = metrics.getDurationInMs(writeContext.stop());
metrics.updateCommitMetrics(FORMATTER.parse(commitTime).getTime(), durationInMs,
metadata);
writeContext = null;
List<Tuple2<String, HoodieWriteStat>> stats =
writeStatuses.mapToPair(new PairFunction<WriteStatus, String, HoodieWriteStat>() {
@Override
public Tuple2<String, HoodieWriteStat> call(WriteStatus writeStatus)
throws Exception {
return new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat());
}
}).collect();
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
for (Tuple2<String, HoodieWriteStat> stat : stats) {
metadata.addWriteStat(stat._1(), stat._2());
}
try {
commitTimeline.saveInstantAsComplete(commitTime,
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
// Save was a success
// We cannot have unbounded commit files. Archive commits if we have to archive
archiveLog.archiveIfRequired();
// Call clean to cleanup if there is anything to cleanup after the commit,
clean();
if (writeContext != null) {
long durationInMs = metrics.getDurationInMs(writeContext.stop());
metrics.updateCommitMetrics(FORMATTER.parse(commitTime).getTime(), durationInMs,
metadata);
writeContext = null;
}
logger.info("Status of the commit " + commitTime + ": " + success);
return success;
logger.info("Status of the commit " + commitTime);
} catch (IOException e) {
throw new HoodieCommitException(
"Failed to commit " + config.getBasePath() + " at time " + commitTime, e);
"Failed to commit " + config.getBasePath() + " at time " + commitTime, e);
} catch (ParseException e) {
throw new HoodieCommitException(
"Commit time is not of valid format.Failed to commit " + config.getBasePath()
+ " at time " + commitTime, e);
"Commit time is not of valid format.Failed to commit " + config.getBasePath()
+ " at time " + commitTime, e);
}
return true;
}
/**
* Rollback the (inflight/committed) record changes with the given commit time. Three steps: (0)
* Obtain the commit or rollback file (1) clean indexing data, (2) clean new generated parquet
* files. (3) Finally delete .commit or .inflight file,
* Rollback the (inflight/committed) record changes with the given commit time.
* Three steps:
* (1) Atomically unpublish this commit
* (2) clean indexing data,
* (3) clean new generated parquet files.
* (4) Finally delete .commit or .inflight file,
*/
public boolean rollback(final String commitTime) throws HoodieRollbackException {
final Timer.Context context = metrics.getRollbackCtx();
final HoodieTableMetadata metadata =
new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
final String metaPath = config.getBasePath() + "/" + HoodieTableMetadata.METAFOLDER_NAME;
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
try {
// 0. Obtain the commit/.inflight file, to work on
FileStatus[] commitFiles =
fs.globStatus(new Path(metaPath + "/" + commitTime + ".*"));
if (commitFiles.length != 1) {
throw new HoodieRollbackException("Expected exactly one .commit or .inflight file for commitTime: " + commitTime);
if (commitTimeline.lastInstant().isPresent()
&& commitTimeline.findInstantsAfter(commitTime, Integer.MAX_VALUE).count() > 0) {
throw new HoodieRollbackException("Found commits after time :" + commitTime +
", please rollback greater commits first");
}
// we first need to unpublish the commit by making it .inflight again. (this will ensure no future queries see this data)
Path filePath = commitFiles[0].getPath();
if (filePath.getName().endsWith(HoodieTableMetadata.COMMIT_FILE_SUFFIX)) {
if (metadata.findCommitsAfter(commitTime, Integer.MAX_VALUE).size() > 0) {
throw new HoodieRollbackException("Found commits after time :" + commitTime +
List<String> inflights =
commitTimeline.getInflightInstants().collect(Collectors.toList());
if (!inflights.isEmpty() && inflights.indexOf(commitTime) != inflights.size() - 1) {
throw new HoodieRollbackException(
"Found in-flight commits after time :" + commitTime +
", please rollback greater commits first");
}
Path newInflightPath = new Path(metaPath + "/" + commitTime + HoodieTableMetadata.INFLIGHT_FILE_SUFFIX);
if (!fs.rename(filePath, newInflightPath)) {
throw new HoodieRollbackException("Unable to rename .commit file to .inflight for commitTime:" + commitTime);
}
filePath = newInflightPath;
}
// 1. Revert the index changes
logger.info("Clean out index changes at time: " + commitTime);
if (!index.rollbackCommit(commitTime)) {
throw new HoodieRollbackException("Clean out index changes failed, for time :" + commitTime);
}
if (inflights.contains(commitTime) || (commitTimeline.lastInstant().isPresent()
&& commitTimeline.lastInstant().get().equals(commitTime))) {
// 1. Atomically unpublish this commit
if(commitTimeline.containsInstant(commitTime)) {
commitTimeline.revertInstantToInflight(commitTime);
}
// 2. Revert the index changes
logger.info("Clean out index changes at time: " + commitTime);
if (!index.rollbackCommit(commitTime)) {
throw new HoodieRollbackException(
"Clean out index changes failed, for time :" + commitTime);
}
// 2. Delete the new generated parquet files
logger.info("Clean out all parquet files generated at time: " + commitTime);
final Accumulator<Integer> numFilesDeletedAccu = jsc.accumulator(0);
jsc.parallelize(FSUtils.getAllPartitionPaths(fs, metadata.getBasePath()))
// 3. Delete the new generated parquet files
logger.info("Clean out all parquet files generated at time: " + commitTime);
final Accumulator<Integer> numFilesDeletedAccu = jsc.accumulator(0);
jsc.parallelize(FSUtils.getAllPartitionPaths(fs, metaClient.getBasePath()))
.foreach(new VoidFunction<String>() {
@Override
public void call(String partitionPath) throws Exception {
// Scan all partitions files with this commit time
FileSystem fs = FSUtils.getFs();
FileStatus[] toBeDeleted =
fs.listStatus(new Path(config.getBasePath(), partitionPath),
new PathFilter() {
@Override
public boolean accept(Path path) {
return commitTime
.equals(FSUtils.getCommitTime(path.getName()));
}
});
fs.listStatus(new Path(config.getBasePath(), partitionPath),
new PathFilter() {
@Override
public boolean accept(Path path) {
return commitTime
.equals(FSUtils.getCommitTime(path.getName()));
}
});
for (FileStatus file : toBeDeleted) {
boolean success = fs.delete(file.getPath(), false);
logger.info("Delete file " + file.getPath() + "\t" + success);
@@ -473,24 +413,20 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
}
}
});
// 4. Remove commit
logger.info("Clean out metadata files at time: " + commitTime);
commitTimeline.removeInflightFromTimeline(commitTime);
// 3. Clean out metadata (.commit or .tmp)
logger.info("Clean out metadata files at time: " + commitTime);
if (!fs.delete(filePath, false)) {
logger.error("Deleting file " + filePath + " failed.");
throw new HoodieRollbackException("Delete file " + filePath + " failed.");
if (context != null) {
long durationInMs = metrics.getDurationInMs(context.stop());
int numFilesDeleted = numFilesDeletedAccu.value();
metrics.updateRollbackMetrics(durationInMs, numFilesDeleted);
}
}
if (context != null) {
long durationInMs = metrics.getDurationInMs(context.stop());
int numFilesDeleted = numFilesDeletedAccu.value();
metrics.updateRollbackMetrics(durationInMs, numFilesDeleted);
}
return true;
} catch (IOException e) {
throw new HoodieRollbackException("Failed to rollback " +
config.getBasePath() + " at commit time" + commitTime, e);
config.getBasePath() + " at commit time" + commitTime, e);
}
}
@@ -504,35 +440,38 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
/**
* Clean up any stale/old files/data lying around (either on file storage or index storage)
*/
private void clean() throws HoodieIOException {
private void clean() throws HoodieIOException {
try {
logger.info("Cleaner started");
final Timer.Context context = metrics.getCleanCtx();
final HoodieTableMetadata metadata = new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
List<String> partitionsToClean = FSUtils.getAllPartitionPaths(fs, metadata.getBasePath());
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
List<String> partitionsToClean = FSUtils.getAllPartitionPaths(fs, metaClient.getBasePath());
// shuffle to distribute cleaning work across partitions evenly
Collections.shuffle(partitionsToClean);
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config.getCleanerPolicy());
if (partitionsToClean.isEmpty()) {
if(partitionsToClean.isEmpty()) {
logger.info("Nothing to clean here mom. It is already clean");
return;
}
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
int numFilesDeleted = jsc.parallelize(partitionsToClean, cleanerParallelism)
.map(new Function<String, Integer>() {
@Override
public Integer call(String partitionPathToClean) throws Exception {
FileSystem fs = FSUtils.getFs();
HoodieCleaner cleaner = new HoodieCleaner(metadata, config, fs);
return cleaner.clean(partitionPathToClean);
}
}).reduce(new Function2<Integer, Integer, Integer>() {
@Override
public Integer call(Integer v1, Integer v2) throws Exception {
return v1 + v2;
}
});
.map(new Function<String, Integer>() {
@Override
public Integer call(String partitionPathToClean) throws Exception {
FileSystem fs = FSUtils.getFs();
HoodieCleaner cleaner = new HoodieCleaner(metaClient, config, fs);
return cleaner.clean(partitionPathToClean);
}
}).reduce(new Function2<Integer, Integer, Integer>() {
@Override
public Integer call(Integer v1, Integer v2) throws Exception {
return v1 + v2;
}
});
logger.info("Cleaned " + numFilesDeleted + " files");
// Emit metrics (duration, numFilesDeleted) if needed
if (context != null) {
@@ -556,21 +495,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
public void startCommitWithTime(String commitTime) {
logger.info("Generate a new commit time " + commitTime);
// Create the in-flight commit file
Path inflightCommitFilePath = new Path(
config.getBasePath() + "/.hoodie/" + FSUtils.makeInflightCommitFileName(commitTime));
try {
if (fs.createNewFile(inflightCommitFilePath)) {
logger.info("Create an inflight commit file " + inflightCommitFilePath);
return;
}
throw new HoodieCommitException(
"Failed to create the inflight commit file " + inflightCommitFilePath);
} catch (IOException e) {
// handled below
throw new HoodieCommitException(
"Failed to create the inflight commit file " + inflightCommitFilePath, e);
}
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
commitTimeline.saveInstantAsInflight(commitTime);
}
public static SparkConf registerClasses(SparkConf conf) {
@@ -606,10 +534,16 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
/**
* Cleanup all inflight commits
* @throws IOException
*/
private void rollbackInflightCommits() {
final HoodieTableMetadata metadata = new HoodieTableMetadata(fs, config.getBasePath(), config.getTableName());
for (String commit : metadata.getAllInflightCommits()) {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
List<String> commits = commitTimeline.getInflightInstants().collect(Collectors.toList());
Collections.reverse(commits);
for (String commit : commits) {
rollback(commit);
}
}

View File

@@ -16,11 +16,11 @@
package com.uber.hoodie.func;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import org.apache.spark.api.java.function.Function2;
import java.util.Iterator;
@@ -30,23 +30,23 @@ import java.util.List;
/**
* Map function that handles a sorted stream of HoodieRecords
*/
public class BulkInsertMapFunction<T extends HoodieRecordPayload>
public class InsertMapFunction<T extends HoodieRecordPayload>
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
private String commitTime;
private HoodieWriteConfig config;
private HoodieTableMetadata metadata;
private HoodieTableMetaClient metaClient;
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config,
HoodieTableMetadata metadata) {
public InsertMapFunction(String commitTime, HoodieWriteConfig config,
HoodieTableMetaClient metaClient) {
this.commitTime = commitTime;
this.config = config;
this.metadata = metadata;
this.metaClient = metaClient;
}
@Override
public Iterator<List<WriteStatus>> call(Integer partition, Iterator<HoodieRecord<T>> sortedRecordItr)
throws Exception {
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, metadata);
return new LazyInsertIterable<>(sortedRecordItr, config, commitTime, metaClient);
}
}

View File

@@ -16,11 +16,11 @@
package com.uber.hoodie.func;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.io.HoodieIOHandle;
import com.uber.hoodie.io.HoodieInsertHandle;
@@ -40,17 +40,17 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyItera
private final HoodieWriteConfig hoodieConfig;
private final String commitTime;
private final HoodieTableMetadata tableMetadata;
private final HoodieTableMetaClient metaClient;
private Set<String> partitionsCleaned;
private HoodieInsertHandle handle;
public LazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
String commitTime, HoodieTableMetadata metadata) {
String commitTime, HoodieTableMetaClient metaClient) {
super(sortedRecordItr);
this.partitionsCleaned = new HashSet<>();
this.hoodieConfig = config;
this.commitTime = commitTime;
this.tableMetadata = metadata;
this.metaClient = metaClient;
}
@Override protected void start() {
@@ -78,7 +78,7 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyItera
// lazily initialize the handle, for the first time
if (handle == null) {
handle =
new HoodieInsertHandle(hoodieConfig, commitTime, tableMetadata,
new HoodieInsertHandle(hoodieConfig, commitTime, metaClient,
record.getPartitionPath());
}
@@ -90,7 +90,7 @@ public class LazyInsertIterable<T extends HoodieRecordPayload> extends LazyItera
statuses.add(handle.close());
// Need to handle the rejected record & open new handle
handle =
new HoodieInsertHandle(hoodieConfig, commitTime, tableMetadata,
new HoodieInsertHandle(hoodieConfig, commitTime, metaClient,
record.getPartitionPath());
handle.write(record); // we should be able to write 1 record.
break;

View File

@@ -17,12 +17,13 @@
package com.uber.hoodie.index;
import com.google.common.base.Optional;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.config.HoodieIndexConfig;
@@ -65,7 +66,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, HoodieTableMetadata metadata) {
JavaRDD<HoodieKey> hoodieKeys, HoodieTableMetaClient metaClient) {
throw new UnsupportedOperationException("HBase index does not implement check exist yet");
}
@@ -91,10 +92,10 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
class LocationTagFunction
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
private final HoodieTableMetadata metadata;
private final HoodieTableMetaClient metaClient;
LocationTagFunction(HoodieTableMetadata metadata) {
this.metadata = metadata;
LocationTagFunction(HoodieTableMetaClient metaClient) {
this.metaClient = metaClient;
}
@Override
@@ -127,8 +128,9 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
String fileId =
Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
// if the last commit ts for this row is less than the system commit ts
if (!metadata.isCommitsEmpty() && metadata.isCommitTsSafe(commitTs)) {
if (commitTimeline.hasInstants() && commitTimeline.containsInstant(commitTs)) {
rec.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
}
}
@@ -155,8 +157,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTableMetadata metadata) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(metadata), true);
HoodieTableMetaClient metaClient) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(metaClient), true);
}
class UpdateLocationTask implements Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> {
@@ -217,7 +219,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTableMetadata metadata) {
HoodieTableMetaClient metaClient) {
return writeStatusRDD.mapPartitionsWithIndex(new UpdateLocationTask(), true);
}

View File

@@ -19,16 +19,18 @@ package com.uber.hoodie.index;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -43,6 +45,7 @@ import org.apache.spark.api.java.function.PairFunction;
import scala.Tuple2;
import java.util.*;
import java.util.stream.Collectors;
/**
* Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in
@@ -63,10 +66,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
@Override
/**
*
*/
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTableMetadata metadata) {
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, final HoodieTableMetaClient metaClient) {
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
@@ -79,7 +79,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
lookupIndex(partitionRecordKeyPairRDD, metadata);
lookupIndex(partitionRecordKeyPairRDD, metaClient);
// Cache the result, for subsequent stages.
rowKeyFilenamePairRDD.cache();
@@ -93,7 +93,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetadata metadata) {
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
hoodieKeys.mapToPair(new PairFunction<HoodieKey, String, String>() {
@Override
@@ -104,7 +104,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<String, String> rowKeyFilenamePairRDD =
lookupIndex(partitionRecordKeyPairRDD, metadata);
lookupIndex(partitionRecordKeyPairRDD, metaClient);
JavaPairRDD<String, HoodieKey> rowKeyHoodieKeyPairRDD =
hoodieKeys.mapToPair(new PairFunction<HoodieKey, String, HoodieKey>() {
@@ -115,17 +115,17 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
});
return rowKeyHoodieKeyPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).mapToPair(
new PairFunction<Tuple2<String, Tuple2<HoodieKey, org.apache.spark.api.java.Optional<String>>>, HoodieKey, Optional<String>>() {
new PairFunction<Tuple2<String, Tuple2<HoodieKey, Optional<String>>>, HoodieKey, Optional<String>>() {
@Override
public Tuple2<HoodieKey, Optional<String>> call(
Tuple2<String, Tuple2<HoodieKey, org.apache.spark.api.java.Optional<String>>> keyPathTuple)
Tuple2<String, Tuple2<HoodieKey, Optional<String>>> keyPathTuple)
throws Exception {
Optional<String> recordLocationPath;
if (keyPathTuple._2._2.isPresent()) {
String fileName = keyPathTuple._2._2.get();
String partitionPath = keyPathTuple._2._1.getPartitionPath();
recordLocationPath = Optional
.of(new Path(new Path(metadata.getBasePath(), partitionPath), fileName)
.of(new Path(new Path(metaClient.getBasePath(), partitionPath), fileName)
.toUri().getPath());
} else {
recordLocationPath = Optional.absent();
@@ -140,19 +140,19 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* record keys already present and drop the record keys if not present
*
* @param partitionRecordKeyPairRDD
* @param metadata
* @param metaClient
* @return
*/
private JavaPairRDD<String, String> lookupIndex(
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTableMetadata metadata) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final HoodieTableMetaClient metaClient) {
// Obtain records per partition, in the incoming records
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
Map<String, Object> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
// Step 2: Load all involved files as <Partition, filename> pairs
JavaPairRDD<String, String> partitionFilePairRDD =
loadInvolvedFiles(affectedPartitionPathList, metadata);
Map<String, Long> filesPerPartition = partitionFilePairRDD.countByKey();
loadInvolvedFiles(affectedPartitionPathList, metaClient);
Map<String, Object> filesPerPartition = partitionFilePairRDD.countByKey();
// Compute total subpartitions, to split partitions into.
Map<String, Long> subpartitionCountMap =
@@ -174,7 +174,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* @param filesPerPartition
* @return
*/
private Map<String, Long> computeSubPartitions(Map<String, Long> recordsPerPartition, Map<String, Long> filesPerPartition) {
private Map<String, Long> computeSubPartitions(Map<String, Object> recordsPerPartition, Map<String, Object> filesPerPartition) {
Map<String, Long> subpartitionCountMap = new HashMap<>();
long totalRecords = 0;
long totalFiles = 0;
@@ -210,21 +210,28 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
* Load all involved files as <Partition, filename> pair RDD.
*/
@VisibleForTesting
JavaPairRDD<String, String> loadInvolvedFiles(List<String> partitions, final HoodieTableMetadata metadata) {
JavaPairRDD<String, String> loadInvolvedFiles(List<String> partitions,
final HoodieTableMetaClient metaClient) {
return jsc.parallelize(partitions, Math.max(partitions.size(), 1))
.flatMapToPair(new PairFlatMapFunction<String, String, String>() {
@Override
public Iterator<Tuple2<String, String>> call(String partitionPath) {
FileSystem fs = FSUtils.getFs();
String latestCommitTime = metadata.getAllCommits().lastCommit();
FileStatus[] filteredStatus = metadata.getLatestVersionInPartition(fs, partitionPath, latestCommitTime);
List<Tuple2<String, String>> list = new ArrayList<>();
for (FileStatus fileStatus : filteredStatus) {
list.add(new Tuple2<>(partitionPath, fileStatus.getPath().getName()));
.flatMapToPair(new PairFlatMapFunction<String, String, String>() {
@Override
public Iterable<Tuple2<String, String>> call(String partitionPath) {
FileSystem fs = FSUtils.getFs();
TableFileSystemView view = new ReadOptimizedTableView(fs, metaClient);
java.util.Optional<String> latestCommitTime =
metaClient.getActiveCommitTimeline().lastInstant();
List<Tuple2<String, String>> list = new ArrayList<>();
if (latestCommitTime.isPresent()) {
List<HoodieDataFile> filteredFiles =
view.streamLatestVersionInPartition(partitionPath,
latestCommitTime.get()).collect(Collectors.toList());
for (HoodieDataFile file : filteredFiles) {
list.add(new Tuple2<>(partitionPath, file.getFileName()));
}
return list.iterator();
}
});
return list;
}
});
}
@Override
@@ -261,8 +268,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
})
.flatMapToPair(new PairFlatMapFunction<List<Tuple2<String, String>>, String, String>() {
@Override
public Iterator<Tuple2<String, String>> call(List<Tuple2<String, String>> exploded) throws Exception {
return exploded.iterator();
public Iterable<Tuple2<String, String>> call(List<Tuple2<String, String>> exploded) throws Exception {
return exploded;
}
});
@@ -323,7 +330,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
/**
* Find out <RowKey, filename> pair. All workload grouped by file-level.
*
* // Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) and then repartition such that
* // Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such that
// each RDD partition is a file, then for each file, we do (1) load bloom filter, (2) load rowKeys, (3) Tag rowKey
// Make sure the parallelism is atleast the groupby parallelism for tagging location
*/
@@ -362,9 +369,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(config.getBasePath()), true)
.flatMap(new FlatMapFunction<List<IndexLookupResult>, IndexLookupResult>() {
@Override
public Iterator<IndexLookupResult> call(List<IndexLookupResult> indexLookupResults)
public Iterable<IndexLookupResult> call(List<IndexLookupResult> indexLookupResults)
throws Exception {
return indexLookupResults.iterator();
return indexLookupResults;
}
}).filter(new Function<IndexLookupResult, Boolean>() {
@Override
@@ -373,13 +380,13 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
}).flatMapToPair(new PairFlatMapFunction<IndexLookupResult, String, String>() {
@Override
public Iterator<Tuple2<String, String>> call(IndexLookupResult lookupResult)
public Iterable<Tuple2<String, String>> call(IndexLookupResult lookupResult)
throws Exception {
List<Tuple2<String, String>> vals = new ArrayList<>();
for (String recordKey : lookupResult.getMatchingRecordKeys()) {
vals.add(new Tuple2<>(recordKey, lookupResult.getFileName()));
}
return vals.iterator();
return vals;
}
});
}
@@ -399,9 +406,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), so we do left outer join.
return rowKeyRecordPairRDD.leftOuterJoin(rowKeyFilenamePairRDD).values().map(
new Function<Tuple2<HoodieRecord<T>, org.apache.spark.api.java.Optional<String>>, HoodieRecord<T>>() {
new Function<Tuple2<HoodieRecord<T>, Optional<String>>, HoodieRecord<T>>() {
@Override
public HoodieRecord<T> call(Tuple2<HoodieRecord<T>, org.apache.spark.api.java.Optional<String>> v1) throws Exception {
public HoodieRecord<T> call(Tuple2<HoodieRecord<T>, Optional<String>> v1) throws Exception {
HoodieRecord<T> record = v1._1();
if (v1._2().isPresent()) {
String filename = v1._2().get();
@@ -416,7 +423,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTableMetadata metadata) {
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, HoodieTableMetaClient metaClient) {
return writeStatusRDD;
}
}

View File

@@ -17,11 +17,12 @@
package com.uber.hoodie.index;
import com.google.common.base.Optional;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.exception.HoodieIndexException;
@@ -58,18 +59,18 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
* value is present, it is the path component (without scheme) of the URI underlying file
*
* @param hoodieKeys
* @param metadata
* @param metaClient
* @return
*/
public abstract JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetadata metadata);
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient);
/**
* Looks up the index and tags each incoming record with a location of a file that contains the
* row (if it is actually present)
*/
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTableMetadata metadata) throws
HoodieTableMetaClient metaClient) throws
HoodieIndexException;
/**
@@ -78,8 +79,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
*/
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTableMetadata metadata) throws
HoodieIndexException;
HoodieTableMetaClient metaClient) throws HoodieIndexException;
/**
* Rollback the efffects of the commit made at commitTime.

View File

@@ -17,13 +17,13 @@
package com.uber.hoodie.index;
import com.google.common.base.Optional;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import org.apache.spark.api.java.JavaPairRDD;
import org.apache.spark.api.java.JavaRDD;
@@ -55,7 +55,7 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
@Override
public JavaPairRDD<HoodieKey, Optional<String>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetadata metadata) {
JavaRDD<HoodieKey> hoodieKeys, final HoodieTableMetaClient metaClient) {
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
}
@@ -81,13 +81,13 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
@Override
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
HoodieTableMetadata metadata) {
HoodieTableMetaClient metaClient) {
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
}
@Override
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
HoodieTableMetadata metadata) {
HoodieTableMetaClient metaClient) {
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
@Override
public WriteStatus call(WriteStatus writeStatus) {

View File

@@ -16,9 +16,12 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.model.HoodieCommits;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileStatus;
@@ -31,7 +34,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
@@ -45,26 +48,25 @@ import java.util.Map;
*
*/
public class HoodieCleaner {
private static Logger logger = LogManager.getLogger(HoodieCleaner.class);
public enum CleaningPolicy {
KEEP_LATEST_FILE_VERSIONS,
KEEP_LATEST_COMMITS
}
private static Logger logger = LogManager.getLogger(HoodieCleaner.class);
private HoodieTableMetadata metadata;
private final TableFileSystemView fileSystemView;
private final HoodieTimeline commitTimeline;
private HoodieTableMetaClient metaClient;
private HoodieWriteConfig config;
private FileSystem fs;
public HoodieCleaner(HoodieTableMetadata metadata,
public HoodieCleaner(HoodieTableMetaClient metaClient,
HoodieWriteConfig config,
FileSystem fs) {
this.metadata = metadata;
this.metaClient = metaClient;
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
this.commitTimeline = metaClient.getActiveCommitTimeline();
this.config = config;
this.fs = fs;
}
@@ -83,13 +85,13 @@ public class HoodieCleaner {
*/
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath) throws IOException {
logger.info("Cleaning "+ partitionPath+", retaining latest "+ config.getCleanerFileVersionsRetained()+" file versions. ");
Map<String, List<FileStatus>> fileVersions = metadata.getAllVersionsInPartition(fs, partitionPath);
List<List<HoodieDataFile>> fileVersions = fileSystemView.streamEveryVersionInPartition(partitionPath).collect(
Collectors.toList());
List<String> deletePaths = new ArrayList<>();
for (String file : fileVersions.keySet()) {
List<FileStatus> commitList = fileVersions.get(file);
for (List<HoodieDataFile> versionsForFileId : fileVersions) {
int keepVersions = config.getCleanerFileVersionsRetained();
Iterator<FileStatus> commitItr = commitList.iterator();
Iterator<HoodieDataFile> commitItr = versionsForFileId.iterator();
while (commitItr.hasNext() && keepVersions > 0) {
// Skip this most recent version
commitItr.next();
@@ -100,7 +102,7 @@ public class HoodieCleaner {
deletePaths.add(String.format("%s/%s/%s",
config.getBasePath(),
partitionPath,
commitItr.next().getPath().getName()));
commitItr.next().getFileName()));
}
}
return deletePaths;
@@ -133,22 +135,20 @@ public class HoodieCleaner {
List<String> deletePaths = new ArrayList<>();
// determine if we have enough commits, to start cleaning.
HoodieCommits commits = metadata.getAllCommits();
if (commits.getNumCommits() > commitsRetained) {
if (commitTimeline.getTotalInstants() > commitsRetained) {
String earliestCommitToRetain =
commits.nthCommit(commits.getNumCommits() - commitsRetained);
Map<String, List<FileStatus>> fileVersions =
metadata.getAllVersionsInPartition(fs, partitionPath);
for (String file : fileVersions.keySet()) {
List<FileStatus> fileList = fileVersions.get(file);
String lastVersion = FSUtils.getCommitTime(fileList.get(0).getPath().getName());
commitTimeline.nthInstant(commitTimeline.getTotalInstants() - commitsRetained).get();
List<List<HoodieDataFile>> fileVersions =
fileSystemView.streamEveryVersionInPartition(partitionPath).collect(Collectors.toList());
for (List<HoodieDataFile> fileList : fileVersions) {
String lastVersion = FSUtils.getCommitTime(fileList.get(0).getFileName());
String lastVersionBeforeEarliestCommitToRetain =
getLatestVersionBeforeCommit(fileList, earliestCommitToRetain);
// Ensure there are more than 1 version of the file (we only clean old files from updates)
// i.e always spare the last commit.
for (FileStatus afile : fileList) {
String fileCommitTime = FSUtils.getCommitTime(afile.getPath().getName());
for (HoodieDataFile afile : fileList) {
String fileCommitTime = afile.getCommitTime();
// Dont delete the latest commit and also the last commit before the earliest commit we are retaining
// The window of commit retain == max query run time. So a query could be running which still
// uses this file.
@@ -160,11 +160,12 @@ public class HoodieCleaner {
}
// Always keep the last commit
if (HoodieCommits.isCommit1After(earliestCommitToRetain, fileCommitTime)) {
if (commitTimeline.compareInstants(earliestCommitToRetain, fileCommitTime,
HoodieTimeline.GREATER)) {
// this is a commit, that should be cleaned.
deletePaths.add(String
.format("%s/%s/%s", config.getBasePath(), partitionPath,
FSUtils.maskWithoutTaskPartitionId(fileCommitTime, file)));
.format("%s/%s/%s", config.getBasePath(), partitionPath, FSUtils
.maskWithoutTaskPartitionId(fileCommitTime, afile.getFileId())));
}
}
}
@@ -176,10 +177,10 @@ public class HoodieCleaner {
/**
* Gets the latest version < commitTime. This version file could still be used by queries.
*/
private String getLatestVersionBeforeCommit(List<FileStatus> fileList, String commitTime) {
for (FileStatus file : fileList) {
String fileCommitTime = FSUtils.getCommitTime(file.getPath().getName());
if (HoodieCommits.isCommit1After(commitTime, fileCommitTime)) {
private String getLatestVersionBeforeCommit(List<HoodieDataFile> fileList, String commitTime) {
for (HoodieDataFile file : fileList) {
String fileCommitTime = FSUtils.getCommitTime(file.getFileName());
if (commitTimeline.compareInstants(commitTime, fileCommitTime, HoodieTimeline.GREATER)) {
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the one we want
return fileCommitTime;
}

View File

@@ -16,10 +16,13 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.file.HoodieAppendLog;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FileSystem;
@@ -30,65 +33,70 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Log to hold older historical commits, to bound the growth of .commit files
*/
public class HoodieCommitArchiveLog {
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits.archived";
private final Path archiveFilePath;
private final FileSystem fs;
private final HoodieWriteConfig config;
public HoodieCommitArchiveLog(HoodieWriteConfig config) {
this.archiveFilePath =
new Path(config.getBasePath(),
HoodieTableMetadata.METAFOLDER_NAME + "/" +HOODIE_COMMIT_ARCHIVE_LOG_FILE);
this.fs = FSUtils.getFs();
public HoodieCommitArchiveLog(HoodieWriteConfig config,
FileSystem fs) {
this.fs = fs;
this.config = config;
this.archiveFilePath = HoodieArchivedCommitTimeline
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
}
/**
* Check if commits need to be archived. If yes, archive commits.
*/
public boolean archiveIfRequired() {
HoodieTableMetadata metadata = new HoodieTableMetadata(fs, config.getBasePath());
List<String> commitsToArchive = getCommitsToArchive(metadata);
if (!commitsToArchive.isEmpty()) {
List<String> commitsToArchive = getCommitsToArchive().collect(Collectors.toList());
if (commitsToArchive.iterator().hasNext()) {
log.info("Archiving commits " + commitsToArchive);
archive(metadata, commitsToArchive);
return deleteCommits(metadata, commitsToArchive);
archive(commitsToArchive);
return deleteCommits(commitsToArchive);
} else {
log.info("No Commits to archive");
return true;
}
}
private List<String> getCommitsToArchive(HoodieTableMetadata metadata) {
private Stream<String> getCommitsToArchive() {
int maxCommitsToKeep = config.getMaxCommitsToKeep();
int minCommitsToKeep = config.getMinCommitsToKeep();
List<String> commits = metadata.getAllCommits().getCommitList();
List<String> commitsToArchive = new ArrayList<String>();
if (commits.size() > maxCommitsToKeep) {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
if (commitTimeline.hasInstants() && commitTimeline.getTotalInstants() > maxCommitsToKeep) {
// Actually do the commits
commitsToArchive = commits.subList(0, commits.size() - minCommitsToKeep);
return commitTimeline.getInstants()
.limit(commitTimeline.getTotalInstants() - minCommitsToKeep);
}
return commitsToArchive;
return Stream.empty();
}
private boolean deleteCommits(HoodieTableMetadata metadata, List<String> commitsToArchive) {
private boolean deleteCommits(List<String> commitsToArchive) {
log.info("Deleting commits " + commitsToArchive);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
boolean success = true;
for(String commitToArchive:commitsToArchive) {
Path commitFile =
new Path(metadata.getBasePath() + "/" +
HoodieTableMetadata.METAFOLDER_NAME + "/" +
FSUtils.makeCommitFileName(commitToArchive));
Path commitFile = new Path(metaClient.getMetaPath(),
((HoodieActiveCommitTimeline) commitTimeline)
.getCompletedFileName(commitToArchive));
try {
if (fs.exists(commitFile)) {
success &= fs.delete(commitFile, false);
@@ -112,14 +120,19 @@ public class HoodieCommitArchiveLog {
.compression(HoodieAppendLog.CompressionType.RECORD, new BZip2Codec()));
}
private void archive(HoodieTableMetadata metadata, List<String> commits)
throws HoodieCommitException {
private void archive(List<String> commits) throws HoodieCommitException {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
HoodieAppendLog.Writer writer = null;
try {
writer = openWriter();
for (String commitTime : commits) {
Text k = new Text(commitTime);
Text v = new Text(metadata.getCommitMetadata(commitTime).toJsonString());
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.readInstantDetails(commitTime).get());
Text v = new Text(commitMetadata.toJsonString());
writer.append(k, v);
log.info("Wrote " + k);
}

View File

@@ -16,9 +16,12 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.exception.HoodieIOException;
@@ -36,15 +39,19 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
protected final String commitTime;
protected final HoodieWriteConfig config;
protected final FileSystem fs;
protected final HoodieTableMetadata metadata;
protected final HoodieTableMetaClient metaClient;
protected final HoodieTimeline hoodieTimeline;
protected final TableFileSystemView fileSystemView;
protected final Schema schema;
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
HoodieTableMetadata metadata) {
HoodieTableMetaClient metaClient) {
this.commitTime = commitTime;
this.config = config;
this.fs = FSUtils.getFs();
this.metadata = metadata;
this.metaClient = metaClient;
this.hoodieTimeline = metaClient.getActiveCommitTimeline();
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
this.schema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
}

View File

@@ -16,12 +16,12 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieInsertException;
@@ -45,7 +45,7 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
private int recordsWritten = 0;
public HoodieInsertHandle(HoodieWriteConfig config, String commitTime,
HoodieTableMetadata metadata, String partitionPath) {
HoodieTableMetaClient metadata, String partitionPath) {
super(config, commitTime, metadata);
this.status = new WriteStatus();
status.setFileId(UUID.randomUUID().toString());

View File

@@ -16,12 +16,12 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieUpsertException;
@@ -52,10 +52,10 @@ import java.util.Iterator;
public HoodieUpdateHandle(HoodieWriteConfig config,
String commitTime,
HoodieTableMetadata metadata,
HoodieTableMetaClient metaClient,
Iterator<HoodieRecord<T>> recordItr,
String fileId) {
super(config, commitTime, metadata);
super(config, commitTime, metaClient);
WriteStatus writeStatus = new WriteStatus();
writeStatus.setStat(new HoodieWriteStat());
this.writeStatus = writeStatus;
@@ -74,7 +74,9 @@ import java.util.Iterator;
HoodieRecord<T> record = newRecordsItr.next();
// If the first record, we need to extract some info out
if (oldFilePath == null) {
String latestValidFilePath = metadata.getFilenameForRecord(fs, record, fileId);
String latestValidFilePath = fileSystemView
.getLatestDataFilesForFileId(record.getPartitionPath(), fileId).findFirst()
.get().getFileName();
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
oldFilePath = new Path(
config.getBasePath() + "/" + record.getPartitionPath() + "/"
@@ -102,14 +104,14 @@ import java.util.Iterator;
}
// Create the writer for writing the new version file
storageWriter = HoodieStorageWriterFactory
.getStorageWriter(commitTime, newFilePath, metadata, config, schema);
.getStorageWriter(commitTime, newFilePath, metaClient, config, schema);
} catch (Exception e) {
logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
+ commitTime + " on HDFS path " + metadata.getBasePath());
+ commitTime + " on HDFS path " + metaClient.getBasePath());
}
}

View File

@@ -16,11 +16,11 @@
package com.uber.hoodie.io.storage;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
@@ -32,7 +32,7 @@ import java.io.IOException;
public class HoodieStorageWriterFactory {
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
String commitTime, Path path, HoodieTableMetadata metadata, HoodieWriteConfig config, Schema schema)
String commitTime, Path path, HoodieTableMetaClient metaClient, HoodieWriteConfig config, Schema schema)
throws IOException {
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
// Currently only parquet is supported

View File

@@ -16,6 +16,11 @@
package com.uber.hoodie.table;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
@@ -23,7 +28,6 @@ import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieInsertException;
import com.uber.hoodie.exception.HoodieUpsertException;
@@ -33,7 +37,6 @@ import com.uber.hoodie.io.HoodieUpdateHandle;
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.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -52,6 +55,7 @@ import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.stream.Collectors;
import scala.Option;
import scala.Tuple2;
@@ -133,8 +137,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
public HoodieCopyOnWriteTable(String commitTime, HoodieWriteConfig config, HoodieTableMetadata metadata) {
super(commitTime, config, metadata);
public HoodieCopyOnWriteTable(String commitTime, HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
super(commitTime, config, metaClient);
}
/**
@@ -287,21 +291,22 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
FileSystem fs = FSUtils.getFs();
List<SmallFile> smallFileLocations = new ArrayList<>();
if (metadata.getAllCommits().getNumCommits() > 0) { // if we have some commits
String latestCommitTime = metadata.getAllCommits().lastCommit();
FileStatus[] allFiles = metadata.getLatestVersionInPartition(fs, partitionPath, latestCommitTime);
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
TableFileSystemView fileSystemView = new ReadOptimizedTableView(fs, metaClient);
if (allFiles != null && allFiles.length > 0) {
for (FileStatus fileStatus : allFiles) {
if (fileStatus.getLen() < config.getParquetSmallFileLimit()) {
String filename = fileStatus.getPath().getName();
SmallFile sf = new SmallFile();
sf.location = new HoodieRecordLocation(
FSUtils.getCommitTime(filename),
FSUtils.getFileId(filename));
sf.sizeBytes = fileStatus.getLen();
smallFileLocations.add(sf);
}
if (commitTimeline.hasInstants()) { // if we have some commits
String latestCommitTime = commitTimeline.lastInstant().get();
List<HoodieDataFile> allFiles = fileSystemView.streamLatestVersionInPartition(partitionPath, latestCommitTime).collect(
Collectors.toList());
for (HoodieDataFile file : allFiles) {
if (file.getFileSize() < config.getParquetSmallFileLimit()) {
String filename = file.getFileName();
SmallFile sf = new SmallFile();
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename),
FSUtils.getFileId(filename));
sf.sizeBytes = file.getFileSize();
smallFileLocations.add(sf);
}
}
}
@@ -317,11 +322,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
*/
private long averageBytesPerRecord() {
long avgSize = 0L;
HoodieTimeline commitTimeline = metaClient.getActiveCommitTimeline();
try {
if (metadata.getAllCommits().getNumCommits() > 0) {
String latestCommitTime = metadata.getAllCommits().lastCommit();
HoodieCommitMetadata commitMetadata = metadata.getCommitMetadata(latestCommitTime);
avgSize =(long) Math.ceil((1.0 * commitMetadata.fetchTotalBytesWritten())/commitMetadata.fetchTotalRecordsWritten());
if (commitTimeline.hasInstants()) {
String latestCommitTime = commitTimeline.lastInstant().get();
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(commitTimeline.readInstantDetails(latestCommitTime).get());
avgSize = (long) Math.ceil(
(1.0 * commitMetadata.fetchTotalBytesWritten()) / commitMetadata
.fetchTotalRecordsWritten());
}
} catch (Throwable t) {
// make this fail safe.
@@ -389,7 +398,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
public Iterator<List<WriteStatus>> handleUpdate(String fileLoc, Iterator<HoodieRecord<T>> recordItr) throws Exception {
// these are updates
HoodieUpdateHandle upsertHandle =
new HoodieUpdateHandle<>(config, commitTime, metadata, recordItr, fileLoc);
new HoodieUpdateHandle<>(config, commitTime, metaClient, recordItr, fileLoc);
if (upsertHandle.getOldFilePath() == null) {
throw new HoodieUpsertException("Error in finding the old file path at commit " +
commitTime +" at fileLoc: " + fileLoc);
@@ -424,7 +433,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
public Iterator<List<WriteStatus>> handleInsert(Iterator<HoodieRecord<T>> recordItr) throws Exception {
return new LazyInsertIterable<>(recordItr, config, commitTime, metadata);
return new LazyInsertIterable<>(recordItr, config, commitTime, metaClient);
}

View File

@@ -16,11 +16,11 @@
package com.uber.hoodie.table;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.exception.HoodieException;
@@ -39,12 +39,13 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
protected final HoodieWriteConfig config;
protected final HoodieTableMetadata metadata;
protected final HoodieTableMetaClient metaClient;
protected HoodieTable(String commitTime, HoodieWriteConfig config, HoodieTableMetadata metadata) {
protected HoodieTable(String commitTime, HoodieWriteConfig config,
HoodieTableMetaClient metaClient) {
this.commitTime = commitTime;
this.config = config;
this.metadata = metadata;
this.metaClient = metaClient;
}
/**
@@ -81,8 +82,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
* @param partitioner
*/
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(Integer partition,
Iterator<HoodieRecord<T>> recordIterator,
Partitioner partitioner);
Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
/**
* Perform the ultimate IO for a given inserted (RDD) partition
@@ -96,14 +96,13 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
Partitioner partitioner);
public static HoodieTable getHoodieTable(HoodieTableType type,
public static HoodieTable getHoodieTable(HoodieTableMetaClient metaClient,
String commitTime,
HoodieWriteConfig config,
HoodieTableMetadata metadata) {
if (type == HoodieTableType.COPY_ON_WRITE) {
return new HoodieCopyOnWriteTable(commitTime, config, metadata);
HoodieWriteConfig config) {
if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) {
return new HoodieCopyOnWriteTable(commitTime, config, metaClient);
} else {
throw new HoodieException("Unsupported table type :"+ type);
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
}
}
}