Reformatting code per Google Code Style all over
This commit is contained in:
committed by
vinoth chandar
parent
5a62480a92
commit
e45679f5e2
@@ -50,10 +50,21 @@ import com.uber.hoodie.func.BulkInsertMapFunction;
|
||||
import com.uber.hoodie.index.HoodieIndex;
|
||||
import com.uber.hoodie.io.HoodieCommitArchiveLog;
|
||||
import com.uber.hoodie.metrics.HoodieMetrics;
|
||||
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
|
||||
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 java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.ParseException;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
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;
|
||||
@@ -66,25 +77,12 @@ 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;
|
||||
import java.text.ParseException;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
* 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.
|
||||
*/
|
||||
public class HoodieWriteClient<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
@@ -102,7 +100,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @param clientConfig
|
||||
* @throws Exception
|
||||
*/
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) throws Exception {
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig)
|
||||
throws Exception {
|
||||
this(jsc, clientConfig, false);
|
||||
}
|
||||
|
||||
@@ -111,7 +110,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* @param clientConfig
|
||||
* @param rollbackInFlight
|
||||
*/
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackInFlight) {
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
|
||||
boolean rollbackInFlight) {
|
||||
this.fs = FSUtils.getFs();
|
||||
this.jsc = jsc;
|
||||
this.config = clientConfig;
|
||||
@@ -121,7 +121,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
if (rollbackInFlight) {
|
||||
rollbackInflightCommits();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -163,17 +163,17 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
throw (HoodieUpsertException) e;
|
||||
}
|
||||
throw new HoodieUpsertException("Failed to upsert for commit time " + commitTime, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
* 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 records HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
@@ -194,7 +194,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieInsertException("Failed to insert for commit time " + commitTime, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -206,11 +206,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* 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) {
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records,
|
||||
final String commitTime) {
|
||||
return bulkInsert(records, commitTime, Option.empty());
|
||||
}
|
||||
|
||||
@@ -221,16 +222,18 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
*
|
||||
* 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)}. Optionally it allows users to specify their own partitioner. If
|
||||
* specified then it will be used for repartitioning records. See {@link UserDefinedBulkInsertPartitioner}.
|
||||
* HoodieWriteClient#insert(JavaRDD, String)}. Optionally it allows users to specify their own
|
||||
* partitioner. If specified then it will be used for repartitioning records. See {@link
|
||||
* UserDefinedBulkInsertPartitioner}.
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param records HoodieRecords to insert
|
||||
* @param commitTime Commit Time handle
|
||||
* @param bulkInsertPartitioner If specified then it will be used to partition input records before they are
|
||||
* inserted into hoodie.
|
||||
* @param bulkInsertPartitioner If specified then it will be used to partition input records
|
||||
* before they are inserted into hoodie.
|
||||
* @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records, final String commitTime,
|
||||
public JavaRDD<WriteStatus> bulkInsert(JavaRDD<HoodieRecord<T>> records,
|
||||
final String commitTime,
|
||||
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
@@ -240,7 +243,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
try {
|
||||
// De-dupe/merge if needed
|
||||
JavaRDD<HoodieRecord<T>> dedupedRecords =
|
||||
combineOnCondition(config.shouldCombineBeforeInsert(), records, config.getInsertShuffleParallelism());
|
||||
combineOnCondition(config.shouldCombineBeforeInsert(), records,
|
||||
config.getInsertShuffleParallelism());
|
||||
|
||||
final JavaRDD<HoodieRecord<T>> repartitionedRecords;
|
||||
if (bulkInsertPartitioner.isDefined()) {
|
||||
@@ -259,20 +263,22 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}, true, config.getBulkInsertShuffleParallelism());
|
||||
}
|
||||
JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
|
||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table), true)
|
||||
.flatMap(writeStatuses -> writeStatuses.iterator());
|
||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table),
|
||||
true)
|
||||
.flatMap(writeStatuses -> writeStatuses.iterator());
|
||||
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, table, commitTime);
|
||||
} 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 bulk 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) {
|
||||
@@ -280,30 +286,28 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
} else {
|
||||
logger.info("Auto commit disabled for " + commitTime);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
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;
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* Save the workload profile in an intermediate file (here re-using commit files)
|
||||
* This is useful when performing rollback for MOR datasets. Only updates are recorded
|
||||
* in the workload profile metadata since updates to log blocks are unknown across batches
|
||||
* Inserts (which are new parquet files) are rolled back based on commit time.
|
||||
* // TODO : Create a new WorkloadProfile metadata file instead of using HoodieCommitMetadata
|
||||
* @param profile
|
||||
* @param commitTime
|
||||
* @throws HoodieCommitException
|
||||
* Save the workload profile in an intermediate file (here re-using commit files) This is useful
|
||||
* when performing rollback for MOR datasets. Only updates are recorded in the workload profile
|
||||
* metadata since updates to log blocks are unknown across batches Inserts (which are new parquet
|
||||
* files) are rolled back based on commit time. // TODO : Create a new WorkloadProfile metadata
|
||||
* file instead of using HoodieCommitMetadata
|
||||
*/
|
||||
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, HoodieTable<T> table, String commitTime) throws HoodieCommitException {
|
||||
private void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile,
|
||||
HoodieTable<T> table,
|
||||
String commitTime) throws HoodieCommitException {
|
||||
try {
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
profile.getPartitionPaths().stream().forEach(path -> {
|
||||
@@ -319,16 +323,17 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
Optional<HoodieInstant> instant = activeTimeline.filterInflights().lastInstant();
|
||||
activeTimeline.saveToInflight(instant.get(),
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} catch(IOException io) {
|
||||
throw new HoodieCommitException("Failed to commit " + commitTime + " unable to save inflight metadata ", io);
|
||||
}
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieCommitException(
|
||||
"Failed to commit " + commitTime + " unable to save inflight metadata ", io);
|
||||
}
|
||||
}
|
||||
|
||||
private JavaRDD<WriteStatus> upsertRecordsInternal(JavaRDD<HoodieRecord<T>> preppedRecords,
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
final boolean isUpsert) {
|
||||
String commitTime,
|
||||
HoodieTable<T> hoodieTable,
|
||||
final boolean isUpsert) {
|
||||
|
||||
// Cache the tagged records, so we don't end up computing both
|
||||
preppedRecords.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||
@@ -344,29 +349,31 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
final Partitioner partitioner = getPartitioner(hoodieTable, isUpsert, profile);
|
||||
JavaRDD<HoodieRecord<T>> partitionedRecords = partition(preppedRecords, partitioner);
|
||||
JavaRDD<WriteStatus> writeStatusRDD = partitionedRecords
|
||||
.mapPartitionsWithIndex((partition, recordItr) -> {
|
||||
if (isUpsert) {
|
||||
return hoodieTable
|
||||
.handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
} else {
|
||||
return hoodieTable
|
||||
.handleInsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
}
|
||||
}, true)
|
||||
.flatMap(writeStatuses -> writeStatuses.iterator());
|
||||
.mapPartitionsWithIndex((partition, recordItr) -> {
|
||||
if (isUpsert) {
|
||||
return hoodieTable
|
||||
.handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
} else {
|
||||
return hoodieTable
|
||||
.handleInsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
}
|
||||
}, true)
|
||||
.flatMap(writeStatuses -> writeStatuses.iterator());
|
||||
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime);
|
||||
}
|
||||
|
||||
private Partitioner getPartitioner(HoodieTable table, boolean isUpsert, WorkloadProfile profile) {
|
||||
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, HoodieTable<T> table, String commitTime) {
|
||||
private JavaRDD<WriteStatus> updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieTable<T> table, String commitTime) {
|
||||
// Update the index back
|
||||
JavaRDD<WriteStatus> statuses = index.updateLocation(writeStatusRDD, table);
|
||||
// Trigger the insert and collect statuses
|
||||
@@ -375,12 +382,15 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
return statuses;
|
||||
}
|
||||
|
||||
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords, Partitioner partitioner) {
|
||||
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> dedupedRecords,
|
||||
Partitioner partitioner) {
|
||||
return dedupedRecords
|
||||
.mapToPair(record ->
|
||||
new Tuple2<>(new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record))
|
||||
.partitionBy(partitioner)
|
||||
.map(tuple -> tuple._2());
|
||||
.mapToPair(record ->
|
||||
new Tuple2<>(
|
||||
new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())),
|
||||
record))
|
||||
.partitionBy(partitioner)
|
||||
.map(tuple -> tuple._2());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -394,8 +404,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
* Commit changes performed at the given commitTime marker
|
||||
*/
|
||||
public boolean commit(String commitTime,
|
||||
JavaRDD<WriteStatus> writeStatuses,
|
||||
Optional<HashMap<String, String>> extraMetadata) {
|
||||
JavaRDD<WriteStatus> writeStatuses,
|
||||
Optional<HashMap<String, String>> extraMetadata) {
|
||||
|
||||
logger.info("Commiting " + commitTime);
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
@@ -405,9 +415,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
|
||||
List<Tuple2<String, HoodieWriteStat>> stats = writeStatuses
|
||||
.mapToPair((PairFunction<WriteStatus, String, HoodieWriteStat>) writeStatus ->
|
||||
new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat()))
|
||||
.collect();
|
||||
.mapToPair((PairFunction<WriteStatus, String, HoodieWriteStat>) writeStatus ->
|
||||
new Tuple2<>(writeStatus.getPartitionPath(), writeStatus.getStat()))
|
||||
.collect();
|
||||
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
for (Tuple2<String, HoodieWriteStat> stat : stats) {
|
||||
@@ -438,7 +448,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
// We cannot have unbounded commit files. Archive commits if we have to archive
|
||||
archiveLog.archiveIfRequired();
|
||||
if(config.isAutoClean()) {
|
||||
if (config.isAutoClean()) {
|
||||
// Call clean to cleanup if there is anything to cleanup after the commit,
|
||||
logger.info("Auto cleaning is enabled. Running cleaner now");
|
||||
clean(commitTime);
|
||||
@@ -465,12 +475,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
/**
|
||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime
|
||||
* will be referenced in the savepoint and will never be cleaned. The savepointed commit
|
||||
* will never be rolledback or archived.
|
||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime will
|
||||
* be referenced in the savepoint and will never be cleaned. The savepointed commit will never be
|
||||
* rolledback or archived.
|
||||
*
|
||||
* This gives an option to rollback the state to the savepoint anytime.
|
||||
* Savepoint needs to be manually created and deleted.
|
||||
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be
|
||||
* manually created and deleted.
|
||||
*
|
||||
* Savepoint should be on a commit that could not have been cleaned.
|
||||
*
|
||||
@@ -491,12 +501,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
/**
|
||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime
|
||||
* will be referenced in the savepoint and will never be cleaned. The savepointed commit
|
||||
* will never be rolledback or archived.
|
||||
* Savepoint a specific commit. Latest version of data files as of the passed in commitTime will
|
||||
* be referenced in the savepoint and will never be cleaned. The savepointed commit will never be
|
||||
* rolledback or archived.
|
||||
*
|
||||
* This gives an option to rollback the state to the savepoint anytime.
|
||||
* Savepoint needs to be manually created and deleted.
|
||||
* This gives an option to rollback the state to the savepoint anytime. Savepoint needs to be
|
||||
* manually created and deleted.
|
||||
*
|
||||
* Savepoint should be on a commit that could not have been cleaned.
|
||||
*
|
||||
@@ -510,9 +520,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
|
||||
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
if(!table.getCompletedCommitTimeline().containsInstant(commitInstant)) {
|
||||
throw new HoodieSavepointException("Could not savepoint non-existing commit " + commitInstant);
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
|
||||
commitTime);
|
||||
if (!table.getCompletedCommitTimeline().containsInstant(commitInstant)) {
|
||||
throw new HoodieSavepointException(
|
||||
"Could not savepoint non-existing commit " + commitInstant);
|
||||
}
|
||||
|
||||
try {
|
||||
@@ -534,7 +546,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
+ lastCommitRetained);
|
||||
|
||||
Map<String, List<String>> latestFilesMap = jsc.parallelize(
|
||||
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()))
|
||||
FSUtils.getAllPartitionPaths(fs, table.getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()))
|
||||
.mapToPair((PairFunction<String, String, List<String>>) partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
logger.info("Collecting latest files in partition path " + partitionPath);
|
||||
@@ -555,12 +568,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
return true;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieSavepointException("Failed to savepoint " + commitTime, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete a savepoint that was created. Once the savepoint is deleted, the commit can be rolledback
|
||||
* and cleaner may clean up data files.
|
||||
* Delete a savepoint that was created. Once the savepoint is deleted, the commit can be
|
||||
* rolledback and cleaner may clean up data files.
|
||||
*
|
||||
* @param savepointTime - delete the savepoint
|
||||
* @return true if the savepoint was deleted successfully
|
||||
@@ -586,9 +599,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
/**
|
||||
* Rollback the state to the savepoint.
|
||||
* WARNING: This rollsback recent commits and deleted data files. Queries accessing the files
|
||||
* will mostly fail. This should be done during a downtime.
|
||||
* Rollback the state to the savepoint. WARNING: This rollsback recent commits and deleted data
|
||||
* files. Queries accessing the files will mostly fail. This should be done during a downtime.
|
||||
*
|
||||
* @param savepointTime - savepoint time to rollback to
|
||||
* @return true if the savepoint was rollecback to successfully
|
||||
@@ -616,7 +628,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
// Make sure the rollback was successful
|
||||
Optional<HoodieInstant> lastInstant =
|
||||
activeTimeline.reload().getCommitsAndCompactionsTimeline().filterCompletedInstants().lastInstant();
|
||||
activeTimeline.reload().getCommitsAndCompactionsTimeline().filterCompletedInstants()
|
||||
.lastInstant();
|
||||
Preconditions.checkArgument(lastInstant.isPresent());
|
||||
Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime),
|
||||
savepointTime + "is not the last commit after rolling back " + commitsToRollback
|
||||
@@ -625,12 +638,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
/**
|
||||
* 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,
|
||||
* 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 {
|
||||
rollback(Lists.newArrayList(commitTime));
|
||||
@@ -638,7 +648,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
private void rollback(List<String> commits) {
|
||||
if(commits.isEmpty()) {
|
||||
if (commits.isEmpty()) {
|
||||
logger.info("List of commits to rollback is empty");
|
||||
return;
|
||||
}
|
||||
@@ -702,7 +712,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
Optional<Long> durationInMs = Optional.empty();
|
||||
if (context != null) {
|
||||
durationInMs = Optional.of(metrics.getDurationInMs(context.stop()));
|
||||
Long numFilesDeleted = stats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size()).sum();
|
||||
Long numFilesDeleted = stats.stream()
|
||||
.mapToLong(stat -> stat.getSuccessDeleteFiles().size())
|
||||
.sum();
|
||||
metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted);
|
||||
}
|
||||
HoodieRollbackMetadata rollbackMetadata =
|
||||
@@ -722,7 +734,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRollbackException("Failed to rollback " +
|
||||
config.getBasePath() + " commits " + commits, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -733,9 +745,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean up any stale/old files/data lying around (either on file storage or index storage)
|
||||
* based on the configurations and CleaningPolicy used. (typically files that no longer can be used
|
||||
* by a running query can be cleaned)
|
||||
* Clean up any stale/old files/data lying around (either on file storage or index storage) based
|
||||
* on the configurations and CleaningPolicy used. (typically files that no longer can be used by a
|
||||
* running query can be cleaned)
|
||||
*/
|
||||
public void clean() throws HoodieIOException {
|
||||
String startCleanTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
@@ -743,11 +755,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean up any stale/old files/data lying around (either on file storage or index storage)
|
||||
* based on the configurations and CleaningPolicy used. (typically files that no longer can be used
|
||||
* by a running query can be cleaned)
|
||||
* Clean up any stale/old files/data lying around (either on file storage or index storage) based
|
||||
* on the configurations and CleaningPolicy used. (typically files that no longer can be used by a
|
||||
* running query can be cleaned)
|
||||
*/
|
||||
private void clean(String startCleanTime) throws HoodieIOException {
|
||||
private void clean(String startCleanTime) throws HoodieIOException {
|
||||
try {
|
||||
logger.info("Cleaner started");
|
||||
final Timer.Context context = metrics.getCleanCtx();
|
||||
@@ -788,7 +800,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to clean up after commit", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -811,30 +823,30 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
public static SparkConf registerClasses(SparkConf conf) {
|
||||
conf.registerKryoClasses(new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
|
||||
conf.registerKryoClasses(
|
||||
new Class[]{HoodieWriteConfig.class, HoodieRecord.class, HoodieKey.class});
|
||||
return conf;
|
||||
}
|
||||
|
||||
/**
|
||||
* Deduplicate Hoodie records, using the given deduplication funciton.
|
||||
*/
|
||||
private JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records, int parallelism) {
|
||||
private JavaRDD<HoodieRecord<T>> deduplicateRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
int parallelism) {
|
||||
return records
|
||||
.mapToPair(record -> new Tuple2<>(record.getKey(), record))
|
||||
.reduceByKey((rec1, rec2) -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
|
||||
// we cannot allow the user to change the key or partitionPath, since that will affect everything
|
||||
// so pick it from one of the records.
|
||||
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
||||
}, parallelism)
|
||||
.map(recordTuple -> recordTuple._2());
|
||||
.mapToPair(record -> new Tuple2<>(record.getKey(), record))
|
||||
.reduceByKey((rec1, rec2) -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
T reducedData = (T) rec1.getData().preCombine(rec2.getData());
|
||||
// we cannot allow the user to change the key or partitionPath, since that will affect everything
|
||||
// so pick it from one of the records.
|
||||
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
||||
}, parallelism)
|
||||
.map(recordTuple -> recordTuple._2());
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanup all inflight commits
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
private void rollbackInflightCommits() {
|
||||
HoodieTable<T> table = HoodieTable
|
||||
|
||||
Reference in New Issue
Block a user