[HUDI-308] Avoid Renames for tracking state transitions of all actions on dataset
This commit is contained in:
committed by
Balaji Varadarajan
parent
8963a68e6a
commit
9a1f698eef
@@ -18,15 +18,18 @@
|
||||
|
||||
package org.apache.hudi.cli;
|
||||
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
|
||||
/**
|
||||
* This class is responsible to load table metadata and hoodie related configs.
|
||||
*/
|
||||
@@ -39,6 +42,7 @@ public class HoodieCLI {
|
||||
public static String basePath;
|
||||
public static HoodieTableMetaClient tableMetadata;
|
||||
public static HoodieTableMetaClient syncTableMetadata;
|
||||
public static TimelineLayoutVersion layoutVersion;
|
||||
|
||||
/**
|
||||
* Enum for CLI state.
|
||||
@@ -59,6 +63,11 @@ public class HoodieCLI {
|
||||
HoodieCLI.basePath = basePath;
|
||||
}
|
||||
|
||||
private static void setLayoutVersion(Integer layoutVersion) {
|
||||
HoodieCLI.layoutVersion = new TimelineLayoutVersion(
|
||||
(layoutVersion == null) ? TimelineLayoutVersion.CURR_VERSION : layoutVersion);
|
||||
}
|
||||
|
||||
public static boolean initConf() {
|
||||
if (HoodieCLI.conf == null) {
|
||||
HoodieCLI.conf = FSUtils.prepareHadoopConf(new Configuration());
|
||||
@@ -74,11 +83,13 @@ public class HoodieCLI {
|
||||
}
|
||||
|
||||
public static void refreshTableMetadata() {
|
||||
setTableMetaClient(new HoodieTableMetaClient(HoodieCLI.conf, basePath, false, HoodieCLI.consistencyGuardConfig));
|
||||
setTableMetaClient(new HoodieTableMetaClient(HoodieCLI.conf, basePath, false, HoodieCLI.consistencyGuardConfig,
|
||||
Option.of(layoutVersion)));
|
||||
}
|
||||
|
||||
public static void connectTo(String basePath) {
|
||||
public static void connectTo(String basePath, Integer layoutVersion) {
|
||||
setBasePath(basePath);
|
||||
setLayoutVersion(layoutVersion);
|
||||
refreshTableMetadata();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -97,32 +97,33 @@ public class CompactionCommand implements CommandMarker {
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
for (int i = 0; i < instants.size(); i++) {
|
||||
HoodieInstant instant = instants.get(i);
|
||||
HoodieCompactionPlan workload = null;
|
||||
HoodieCompactionPlan compactionPlan = null;
|
||||
if (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
try {
|
||||
// This could be a completed compaction. Assume a compaction request file is present but skip if fails
|
||||
workload = AvroUtils.deserializeCompactionPlan(activeTimeline
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
activeTimeline.readPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
} catch (HoodieIOException ioe) {
|
||||
// SKIP
|
||||
}
|
||||
} else {
|
||||
workload = AvroUtils.deserializeCompactionPlan(activeTimeline
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
compactionPlan = AvroUtils.deserializeCompactionPlan(activeTimeline.readPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
}
|
||||
|
||||
if (null != workload) {
|
||||
if (null != compactionPlan) {
|
||||
HoodieInstant.State state = instant.getState();
|
||||
if (committed.contains(instant.getTimestamp())) {
|
||||
state = State.COMPLETED;
|
||||
}
|
||||
if (includeExtraMetadata) {
|
||||
rows.add(new Comparable[] {instant.getTimestamp(), state.toString(),
|
||||
workload.getOperations() == null ? 0 : workload.getOperations().size(),
|
||||
workload.getExtraMetadata().toString()});
|
||||
compactionPlan.getOperations() == null ? 0 : compactionPlan.getOperations().size(),
|
||||
compactionPlan.getExtraMetadata().toString()});
|
||||
} else {
|
||||
rows.add(new Comparable[] {instant.getTimestamp(), state.toString(),
|
||||
workload.getOperations() == null ? 0 : workload.getOperations().size()});
|
||||
compactionPlan.getOperations() == null ? 0 : compactionPlan.getOperations().size()});
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -148,12 +149,13 @@ public class CompactionCommand implements CommandMarker {
|
||||
unspecifiedDefaultValue = "false") final boolean headerOnly)
|
||||
throws Exception {
|
||||
HoodieActiveTimeline activeTimeline = HoodieCLI.tableMetadata.getActiveTimeline();
|
||||
HoodieCompactionPlan workload = AvroUtils.deserializeCompactionPlan(activeTimeline
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
activeTimeline.readPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
if ((null != workload) && (null != workload.getOperations())) {
|
||||
for (HoodieCompactionOperation op : workload.getOperations()) {
|
||||
if ((null != compactionPlan) && (null != compactionPlan.getOperations())) {
|
||||
for (HoodieCompactionOperation op : compactionPlan.getOperations()) {
|
||||
rows.add(new Comparable[] {op.getPartitionPath(), op.getFileId(), op.getBaseInstantTime(), op.getDataFilePath(),
|
||||
op.getDeltaFilePaths().size(), op.getMetrics() == null ? "" : op.getMetrics().toString()});
|
||||
}
|
||||
@@ -173,7 +175,7 @@ public class CompactionCommand implements CommandMarker {
|
||||
HoodieCLI.initFS(initialized);
|
||||
|
||||
// First get a compaction instant time and pass it to spark launcher for scheduling compaction
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
if (HoodieCLI.tableMetadata.getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
String sparkPropertiesPath =
|
||||
|
||||
@@ -50,6 +50,7 @@ public class DatasetsCommand implements CommandMarker {
|
||||
@CliCommand(value = "connect", help = "Connect to a hoodie dataset")
|
||||
public String connect(
|
||||
@CliOption(key = {"path"}, mandatory = true, help = "Base Path of the dataset") final String path,
|
||||
@CliOption(key = {"layoutVersion"}, mandatory = false, help = "Timeline Layout version") Integer layoutVersion,
|
||||
@CliOption(key = {"eventuallyConsistent"}, mandatory = false, unspecifiedDefaultValue = "false",
|
||||
help = "Enable eventual consistency") final boolean eventuallyConsistent,
|
||||
@CliOption(key = {"initialCheckIntervalMs"}, mandatory = false, unspecifiedDefaultValue = "2000",
|
||||
@@ -65,7 +66,7 @@ public class DatasetsCommand implements CommandMarker {
|
||||
.withMaxConsistencyCheckIntervalMs(maxConsistencyIntervalMs).withMaxConsistencyChecks(maxConsistencyChecks)
|
||||
.build());
|
||||
HoodieCLI.initConf();
|
||||
HoodieCLI.connectTo(path);
|
||||
HoodieCLI.connectTo(path, layoutVersion);
|
||||
HoodieCLI.initFS(true);
|
||||
HoodieCLI.state = HoodieCLI.CLIState.DATASET;
|
||||
return "Metadata for table " + HoodieCLI.tableMetadata.getTableConfig().getTableName() + " loaded";
|
||||
@@ -85,6 +86,8 @@ public class DatasetsCommand implements CommandMarker {
|
||||
@CliOption(key = {"tableName"}, mandatory = true, help = "Hoodie Table Name") final String name,
|
||||
@CliOption(key = {"tableType"}, unspecifiedDefaultValue = "COPY_ON_WRITE",
|
||||
help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr,
|
||||
@CliOption(key = {"archiveLogFolder"}, help = "Folder Name for storing archived timeline") String archiveFolder,
|
||||
@CliOption(key = {"layoutVersion"}, help = "Specific Layout Version to use") Integer layoutVersion,
|
||||
@CliOption(key = {"payloadClass"}, unspecifiedDefaultValue = "org.apache.hudi.common.model.HoodieAvroPayload",
|
||||
help = "Payload Class") final String payloadClass)
|
||||
throws IOException {
|
||||
@@ -106,10 +109,11 @@ public class DatasetsCommand implements CommandMarker {
|
||||
}
|
||||
|
||||
final HoodieTableType tableType = HoodieTableType.valueOf(tableTypeStr);
|
||||
HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, payloadClass);
|
||||
HoodieTableMetaClient.initTableType(HoodieCLI.conf, path, tableType, name, archiveFolder,
|
||||
payloadClass, layoutVersion);
|
||||
|
||||
// Now connect to ensure loading works
|
||||
return connect(path, false, 0, 0, 0);
|
||||
return connect(path, layoutVersion, false, 0, 0, 0);
|
||||
}
|
||||
|
||||
@CliAvailabilityIndicator({"desc"})
|
||||
|
||||
@@ -179,9 +179,10 @@ public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
// revert if in inflight state
|
||||
metaClient.getActiveTimeline().revertCompactionInflightToRequested(inflight);
|
||||
}
|
||||
// Overwrite compaction plan with updated info
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionOperationWithInstant.getLeft()),
|
||||
AvroUtils.serializeCompactionPlan(newPlan));
|
||||
AvroUtils.serializeCompactionPlan(newPlan), true);
|
||||
}
|
||||
return res;
|
||||
}
|
||||
@@ -218,8 +219,9 @@ public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
*/
|
||||
private static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, String compactionInstant)
|
||||
throws IOException {
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(metaClient.getActiveTimeline()
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
metaClient.getActiveTimeline().readPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
|
||||
return compactionPlan;
|
||||
}
|
||||
|
||||
|
||||
@@ -67,7 +67,7 @@ public class HoodieCleanClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* cleaned)
|
||||
*/
|
||||
public void clean() throws HoodieIOException {
|
||||
String startCleanTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String startCleanTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
clean(startCleanTime);
|
||||
}
|
||||
|
||||
@@ -86,7 +86,7 @@ public class HoodieCleanClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
// If there are inflight(failed) or previously requested clean operation, first perform them
|
||||
table.getCleanTimeline().filterInflightsAndRequested().getInstants().forEach(hoodieInstant -> {
|
||||
LOG.info("There were previously unfinished cleaner operations. Finishing Instant=" + hoodieInstant);
|
||||
runClean(table, hoodieInstant.getTimestamp());
|
||||
runClean(table, hoodieInstant);
|
||||
});
|
||||
|
||||
Option<HoodieCleanerPlan> cleanerPlanOpt = scheduleClean(startCleanTime);
|
||||
@@ -96,7 +96,7 @@ public class HoodieCleanClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
if ((cleanerPlan.getFilesToBeDeletedPerPartition() != null)
|
||||
&& !cleanerPlan.getFilesToBeDeletedPerPartition().isEmpty()) {
|
||||
final HoodieTable<T> hoodieTable = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc);
|
||||
return runClean(hoodieTable, startCleanTime);
|
||||
return runClean(hoodieTable, HoodieTimeline.getCleanRequestedInstant(startCleanTime), cleanerPlan);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
@@ -136,13 +136,20 @@ public class HoodieCleanClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* Executes the Cleaner plan stored in the instant metadata.
|
||||
*
|
||||
* @param table Hoodie Table
|
||||
* @param cleanInstantTs Cleaner Instant Timestamp
|
||||
* @param cleanInstant Cleaner Instant
|
||||
*/
|
||||
@VisibleForTesting
|
||||
protected HoodieCleanMetadata runClean(HoodieTable<T> table, String cleanInstantTs) {
|
||||
HoodieInstant cleanInstant =
|
||||
table.getCleanTimeline().getInstants().filter(x -> x.getTimestamp().equals(cleanInstantTs)).findFirst().get();
|
||||
protected HoodieCleanMetadata runClean(HoodieTable<T> table, HoodieInstant cleanInstant) {
|
||||
try {
|
||||
HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(table.getMetaClient(), cleanInstant);
|
||||
return runClean(table, cleanInstant, cleanerPlan);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
private HoodieCleanMetadata runClean(HoodieTable<T> table, HoodieInstant cleanInstant,
|
||||
HoodieCleanerPlan cleanerPlan) {
|
||||
Preconditions.checkArgument(
|
||||
cleanInstant.getState().equals(State.REQUESTED) || cleanInstant.getState().equals(State.INFLIGHT));
|
||||
|
||||
@@ -152,10 +159,11 @@ public class HoodieCleanClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
|
||||
if (!cleanInstant.isInflight()) {
|
||||
// Mark as inflight first
|
||||
cleanInstant = table.getActiveTimeline().transitionCleanRequestedToInflight(cleanInstant);
|
||||
cleanInstant = table.getActiveTimeline().transitionCleanRequestedToInflight(cleanInstant,
|
||||
AvroUtils.serializeCleanerPlan(cleanerPlan));
|
||||
}
|
||||
|
||||
List<HoodieCleanStat> cleanStats = table.clean(jsc, cleanInstant);
|
||||
List<HoodieCleanStat> cleanStats = table.clean(jsc, cleanInstant, cleanerPlan);
|
||||
|
||||
if (cleanStats.isEmpty()) {
|
||||
return HoodieCleanMetadata.newBuilder().build();
|
||||
|
||||
@@ -80,8 +80,6 @@ import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.text.ParseException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -101,7 +99,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieWriteClient.class);
|
||||
private static final String UPDATE_STR = "update";
|
||||
private static final String LOOKUP_STR = "lookup";
|
||||
private final boolean rollbackInFlight;
|
||||
private final boolean rollbackPending;
|
||||
private final transient HoodieMetrics metrics;
|
||||
private final transient HoodieIndex<T> index;
|
||||
private final transient HoodieCleanClient<T> cleanClient;
|
||||
@@ -119,21 +117,21 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackInFlight) {
|
||||
this(jsc, clientConfig, rollbackInFlight, HoodieIndex.createIndex(clientConfig, jsc));
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackPending) {
|
||||
this(jsc, clientConfig, rollbackPending, HoodieIndex.createIndex(clientConfig, jsc));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackInFlight, HoodieIndex index) {
|
||||
this(jsc, clientConfig, rollbackInFlight, index, Option.empty());
|
||||
HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackPending, HoodieIndex index) {
|
||||
this(jsc, clientConfig, rollbackPending, index, Option.empty());
|
||||
}
|
||||
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackInFlight,
|
||||
public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackPending,
|
||||
HoodieIndex index, Option<EmbeddedTimelineService> timelineService) {
|
||||
super(jsc, clientConfig, timelineService);
|
||||
this.index = index;
|
||||
this.metrics = new HoodieMetrics(config, config.getTableName());
|
||||
this.rollbackInFlight = rollbackInFlight;
|
||||
this.rollbackPending = rollbackPending;
|
||||
this.cleanClient = new HoodieCleanClient<>(jsc, config, metrics, timelineService);
|
||||
}
|
||||
|
||||
@@ -356,7 +354,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
return upsertRecordsInternal(taggedValidRecords, commitTime, table, true);
|
||||
} else {
|
||||
// if entire set of keys are non existent
|
||||
JavaRDD<WriteStatus> writeStatusRDD = jsc.parallelize(Collections.EMPTY_LIST, 1);
|
||||
saveWorkloadProfileMetadataToInflight(new WorkloadProfile(jsc.emptyRDD()), table, commitTime);
|
||||
JavaRDD<WriteStatus> writeStatusRDD = jsc.emptyRDD();
|
||||
commitOnAutoCommit(commitTime, writeStatusRDD, table.getMetaClient().getCommitActionType());
|
||||
return writeStatusRDD;
|
||||
}
|
||||
@@ -388,6 +387,9 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
final List<String> fileIDPrefixes =
|
||||
IntStream.range(0, parallelism).mapToObj(i -> FSUtils.createNewFileIdPfx()).collect(Collectors.toList());
|
||||
|
||||
table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED,
|
||||
table.getMetaClient().getCommitActionType(), commitTime), Option.empty());
|
||||
|
||||
JavaRDD<WriteStatus> writeStatusRDD = repartitionedRecords
|
||||
.mapPartitionsWithIndex(new BulkInsertMapFunction<T>(commitTime, config, table, fileIDPrefixes), true)
|
||||
.flatMap(writeStatuses -> writeStatuses.iterator());
|
||||
@@ -435,9 +437,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
});
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
Option<HoodieInstant> instant =
|
||||
activeTimeline.getCommitsTimeline().filterInflightsExcludingCompaction().lastInstant();
|
||||
activeTimeline.saveToInflight(instant.get(), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
String commitActionType = table.getMetaClient().getCommitActionType();
|
||||
HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, commitTime);
|
||||
activeTimeline.transitionRequestedToInflight(requested,
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieCommitException("Failed to commit " + commitTime + " unable to save inflight metadata ", io);
|
||||
}
|
||||
@@ -664,7 +667,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
|
||||
HoodieSavepointMetadata metadata = AvroUtils.convertSavepointMetadata(user, comment, latestFilesMap);
|
||||
// Nothing to save in the savepoint
|
||||
table.getActiveTimeline().saveAsComplete(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime),
|
||||
table.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime));
|
||||
table.getActiveTimeline()
|
||||
.saveAsComplete(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, commitTime),
|
||||
AvroUtils.serializeSavepointMetadata(metadata));
|
||||
LOG.info("Savepoint " + commitTime + " created");
|
||||
return true;
|
||||
@@ -792,23 +798,25 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
.filter(instant -> HoodieActiveTimeline.GREATER.test(instant.getTimestamp(), instantTime))
|
||||
.collect(Collectors.toList());
|
||||
// Start a rollback instant for all commits to be rolled back
|
||||
String startRollbackInstant = startInstant();
|
||||
String startRollbackInstant = HoodieActiveTimeline.createNewInstantTime();
|
||||
// Start the timer
|
||||
final Timer.Context context = startContext();
|
||||
ImmutableMap.Builder<String, List<HoodieRollbackStat>> instantsToStats = ImmutableMap.builder();
|
||||
table.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, startRollbackInstant));
|
||||
instantsToRollback.stream().forEach(instant -> {
|
||||
try {
|
||||
switch (instant.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
List<HoodieRollbackStat> statsForInstant = doRollbackAndGetStats(instant.getTimestamp());
|
||||
List<HoodieRollbackStat> statsForInstant = doRollbackAndGetStats(instant);
|
||||
instantsToStats.put(instant.getTimestamp(), statsForInstant);
|
||||
break;
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
// TODO : Get file status and create a rollback stat and file
|
||||
// TODO : Delete the .aux files along with the instant file, okay for now since the archival process will
|
||||
// delete these files when it does not see a corresponding instant file under .hoodie
|
||||
List<HoodieRollbackStat> statsForCompaction = doRollbackAndGetStats(instant.getTimestamp());
|
||||
List<HoodieRollbackStat> statsForCompaction = doRollbackAndGetStats(instant);
|
||||
instantsToStats.put(instant.getTimestamp(), statsForCompaction);
|
||||
LOG.info("Deleted compaction instant " + instant);
|
||||
break;
|
||||
@@ -828,17 +836,16 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
}
|
||||
|
||||
private String startInstant() {
|
||||
return HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
|
||||
}
|
||||
|
||||
private Timer.Context startContext() {
|
||||
return metrics.getRollbackCtx();
|
||||
}
|
||||
|
||||
private List<HoodieRollbackStat> doRollbackAndGetStats(final String commitToRollback) throws IOException {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc);
|
||||
HoodieTimeline inflightCommitTimeline = table.getInflightCommitTimeline();
|
||||
private List<HoodieRollbackStat> doRollbackAndGetStats(final HoodieInstant instantToRollback) throws
|
||||
IOException {
|
||||
final String commitToRollback = instantToRollback.getTimestamp();
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
createMetaClient(true), config, jsc);
|
||||
HoodieTimeline inflightAndRequestedCommitTimeline = table.getPendingCommitTimeline();
|
||||
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
|
||||
// Check if any of the commits is a savepoint - do not allow rollback on those commits
|
||||
List<String> savepoints = table.getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
|
||||
@@ -850,7 +857,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
});
|
||||
|
||||
if (commitTimeline.empty() && inflightCommitTimeline.empty()) {
|
||||
if (commitTimeline.empty() && inflightAndRequestedCommitTimeline.empty()) {
|
||||
// nothing to rollback
|
||||
LOG.info("No commits to rollback " + commitToRollback);
|
||||
}
|
||||
@@ -865,14 +872,14 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
"Found commits after time :" + lastCommit + ", please rollback greater commits first");
|
||||
}
|
||||
|
||||
List<String> inflights =
|
||||
inflightCommitTimeline.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
List<String> inflights = inflightAndRequestedCommitTimeline.getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
if ((lastCommit != null) && !inflights.isEmpty() && (inflights.indexOf(lastCommit) != inflights.size() - 1)) {
|
||||
throw new HoodieRollbackException(
|
||||
"Found in-flight commits after time :" + lastCommit + ", please rollback greater commits first");
|
||||
}
|
||||
|
||||
List<HoodieRollbackStat> stats = table.rollback(jsc, commitToRollback, true);
|
||||
List<HoodieRollbackStat> stats = table.rollback(jsc, instantToRollback, true);
|
||||
|
||||
LOG.info("Deleted inflight commits " + commitToRollback);
|
||||
|
||||
@@ -893,9 +900,13 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
durationInMs = Option.of(metrics.getDurationInMs(context.stop()));
|
||||
metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted);
|
||||
}
|
||||
HoodieRollbackMetadata rollbackMetadata =
|
||||
AvroUtils.convertRollbackMetadata(startRollbackTime, durationInMs, commitsToRollback, rollbackStats);
|
||||
table.getActiveTimeline().saveAsComplete(new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime),
|
||||
HoodieRollbackMetadata rollbackMetadata = AvroUtils
|
||||
.convertRollbackMetadata(startRollbackTime, durationInMs, commitsToRollback, rollbackStats);
|
||||
//TODO: varadarb - This will be fixed in subsequent PR when Rollback and Clean transition mimics that of commit
|
||||
table.getActiveTimeline().createNewInstant(new HoodieInstant(State.INFLIGHT, HoodieTimeline.ROLLBACK_ACTION,
|
||||
startRollbackTime));
|
||||
table.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime),
|
||||
AvroUtils.serializeRollbackMetadata(rollbackMetadata));
|
||||
LOG.info("Commits " + commitsToRollback + " rollback is complete");
|
||||
|
||||
@@ -937,17 +948,22 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
|
||||
private void rollbackInternal(String commitToRollback) {
|
||||
if (commitToRollback.isEmpty()) {
|
||||
LOG.info("List of commits to rollback is empty");
|
||||
return;
|
||||
}
|
||||
final String startRollbackTime = startInstant();
|
||||
final String startRollbackTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
final Timer.Context context = startContext();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
try {
|
||||
List<HoodieRollbackStat> stats = doRollbackAndGetStats(commitToRollback);
|
||||
Map<String, List<HoodieRollbackStat>> statToCommit = new HashMap<>();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
createMetaClient(true), config, jsc);
|
||||
Option<HoodieInstant> rollbackInstantOpt =
|
||||
Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants()
|
||||
.filter(instant -> HoodieActiveTimeline.EQUAL.test(instant.getTimestamp(), commitToRollback))
|
||||
.findFirst());
|
||||
|
||||
if (rollbackInstantOpt.isPresent()) {
|
||||
List<HoodieRollbackStat> stats = doRollbackAndGetStats(rollbackInstantOpt.get());
|
||||
finishRollback(context, stats, Arrays.asList(commitToRollback), startRollbackTime);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRollbackException("Failed to rollback " + config.getBasePath() + " commits " + commitToRollback,
|
||||
e);
|
||||
@@ -992,20 +1008,20 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
*/
|
||||
public String startCommit() {
|
||||
// NOTE : Need to ensure that rollback is done before a new commit is started
|
||||
if (rollbackInFlight) {
|
||||
// Only rollback inflight commit/delta-commits. Do not touch compaction commits
|
||||
rollbackInflightCommits();
|
||||
if (rollbackPending) {
|
||||
// Only rollback pending commit/delta-commits. Do not touch compaction commits
|
||||
rollbackPendingCommits();
|
||||
}
|
||||
String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
startCommit(commitTime);
|
||||
return commitTime;
|
||||
}
|
||||
|
||||
public void startCommitWithTime(String instantTime) {
|
||||
// NOTE : Need to ensure that rollback is done before a new commit is started
|
||||
if (rollbackInFlight) {
|
||||
if (rollbackPending) {
|
||||
// Only rollback inflight commit/delta-commits. Do not touch compaction commits
|
||||
rollbackInflightCommits();
|
||||
rollbackPendingCommits();
|
||||
}
|
||||
startCommit(instantTime);
|
||||
}
|
||||
@@ -1023,14 +1039,14 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
String commitActionType = table.getMetaClient().getCommitActionType();
|
||||
activeTimeline.createInflight(new HoodieInstant(true, commitActionType, instantTime));
|
||||
activeTimeline.createNewInstant(new HoodieInstant(State.REQUESTED, commitActionType, instantTime));
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedules a new compaction instant.
|
||||
*/
|
||||
public Option<String> scheduleCompaction(Option<Map<String, String>> extraMetadata) throws IOException {
|
||||
String instantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
LOG.info("Generate a new instant time " + instantTime);
|
||||
boolean notEmpty = scheduleCompactionAtInstant(instantTime, extraMetadata);
|
||||
return notEmpty ? Option.of(instantTime) : Option.empty();
|
||||
@@ -1046,7 +1062,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
throws IOException {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
// if there are inflight writes, their instantTime must not be less than that of compaction instant time
|
||||
metaClient.getCommitsTimeline().filterInflightsExcludingCompaction().firstInstant().ifPresent(earliestInflight -> {
|
||||
metaClient.getCommitsTimeline().filterPendingExcludingCompaction().firstInstant().ifPresent(earliestInflight -> {
|
||||
Preconditions.checkArgument(
|
||||
HoodieTimeline.compareTimestamps(earliestInflight.getTimestamp(), instantTime, HoodieTimeline.GREATER),
|
||||
"Earliest write inflight instant time must be later than compaction time. Earliest :" + earliestInflight
|
||||
@@ -1091,7 +1107,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
timeline.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
|
||||
timeline.readPlanAsBytes(HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get());
|
||||
// Merge extra meta-data passed by user with the one already in inflight compaction
|
||||
Option<Map<String, String>> mergedMetaData = extraMetadata.map(m -> {
|
||||
Map<String, String> merged = new HashMap<>();
|
||||
@@ -1141,11 +1157,11 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanup all inflight commits.
|
||||
* Cleanup all pending commits.
|
||||
*/
|
||||
private void rollbackInflightCommits() {
|
||||
private void rollbackPendingCommits() {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(createMetaClient(true), config, jsc);
|
||||
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterInflightsExcludingCompaction();
|
||||
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
|
||||
List<String> commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
for (String commit : commits) {
|
||||
@@ -1238,7 +1254,6 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
HoodieCompactionPlan compactionPlan =
|
||||
CompactionUtils.getCompactionPlan(metaClient, compactionInstant.getTimestamp());
|
||||
|
||||
// Mark instant as compaction inflight
|
||||
activeTimeline.transitionCompactionRequestedToInflight(compactionInstant);
|
||||
compactionTimer = metrics.getCompactionCtx();
|
||||
@@ -1306,7 +1321,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
*/
|
||||
@VisibleForTesting
|
||||
void rollbackInflightCompaction(HoodieInstant inflightInstant, HoodieTable table) throws IOException {
|
||||
table.rollback(jsc, inflightInstant.getTimestamp(), false);
|
||||
table.rollback(jsc, inflightInstant, false);
|
||||
// Revert instant state file
|
||||
table.getActiveTimeline().revertCompactionInflightToRequested(inflightInstant);
|
||||
}
|
||||
|
||||
@@ -18,7 +18,9 @@
|
||||
|
||||
package org.apache.hudi.client.utils;
|
||||
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -35,6 +37,6 @@ public class ClientUtils {
|
||||
public static HoodieTableMetaClient createMetaClient(JavaSparkContext jsc, HoodieWriteConfig config,
|
||||
boolean loadActiveTimelineOnLoad) {
|
||||
return new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), loadActiveTimelineOnLoad,
|
||||
config.getConsistencyGuardConfig());
|
||||
config.getConsistencyGuardConfig(), Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion())));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.config;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
@@ -48,6 +49,7 @@ import java.util.Properties;
|
||||
public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String TABLE_NAME = "hoodie.table.name";
|
||||
private static final String TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version";
|
||||
private static final String BASE_PATH_PROP = "hoodie.base.path";
|
||||
private static final String AVRO_SCHEMA = "hoodie.avro.schema";
|
||||
private static final String DEFAULT_PARALLELISM = "1500";
|
||||
@@ -141,6 +143,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return Boolean.parseBoolean(props.getProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP));
|
||||
}
|
||||
|
||||
public Integer getTimelineLayoutVersion() {
|
||||
return Integer.parseInt(props.getProperty(TIMELINE_LAYOUT_VERSION));
|
||||
}
|
||||
|
||||
public int getBulkInsertShuffleParallelism() {
|
||||
return Integer.parseInt(props.getProperty(BULKINSERT_PARALLELISM));
|
||||
}
|
||||
@@ -587,6 +593,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withTimelineLayoutVersion(int version) {
|
||||
props.setProperty(TIMELINE_LAYOUT_VERSION, String.valueOf(version));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBulkInsertParallelism(int bulkInsertParallelism) {
|
||||
props.setProperty(BULKINSERT_PARALLELISM, String.valueOf(bulkInsertParallelism));
|
||||
return this;
|
||||
@@ -693,7 +704,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM, DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(DELETE_PARALLELISM), DELETE_PARALLELISM, DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(ROLLBACK_PARALLELISM), ROLLBACK_PARALLELISM, DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(ROLLBACK_PARALLELISM), ROLLBACK_PARALLELISM,
|
||||
DEFAULT_ROLLBACK_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP), COMBINE_BEFORE_INSERT_PROP,
|
||||
DEFAULT_COMBINE_BEFORE_INSERT);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_UPSERT_PROP), COMBINE_BEFORE_UPSERT_PROP,
|
||||
@@ -733,6 +745,12 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
setDefaultOnCondition(props, !isConsistencyGuardSet,
|
||||
ConsistencyGuardConfig.newBuilder().fromProperties(props).build());
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(TIMELINE_LAYOUT_VERSION), TIMELINE_LAYOUT_VERSION,
|
||||
String.valueOf(TimelineLayoutVersion.CURR_VERSION));
|
||||
String layoutVersion = props.getProperty(TIMELINE_LAYOUT_VERSION);
|
||||
// Ensure Layout Version is good
|
||||
new TimelineLayoutVersion(Integer.parseInt(layoutVersion));
|
||||
|
||||
// Build WriteConfig at the end
|
||||
HoodieWriteConfig config = new HoodieWriteConfig(props);
|
||||
Preconditions.checkArgument(config.getBasePath() != null);
|
||||
|
||||
@@ -19,7 +19,7 @@
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.common.model.ActionType;
|
||||
@@ -37,7 +37,10 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.CleanerUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
@@ -111,15 +114,18 @@ public class HoodieCommitArchiveLog {
|
||||
public boolean archiveIfRequired(final JavaSparkContext jsc) throws IOException {
|
||||
try {
|
||||
List<HoodieInstant> instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList());
|
||||
|
||||
boolean success = true;
|
||||
if (instantsToArchive.iterator().hasNext()) {
|
||||
if (!instantsToArchive.isEmpty()) {
|
||||
this.writer = openWriter();
|
||||
LOG.info("Archiving instants " + instantsToArchive);
|
||||
archive(instantsToArchive);
|
||||
LOG.info("Deleting archived instants " + instantsToArchive);
|
||||
success = deleteArchivedInstants(instantsToArchive);
|
||||
} else {
|
||||
LOG.info("No Instants to archive");
|
||||
}
|
||||
|
||||
return success;
|
||||
} finally {
|
||||
close();
|
||||
@@ -171,7 +177,15 @@ public class HoodieCommitArchiveLog {
|
||||
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
|
||||
}
|
||||
|
||||
return instants;
|
||||
// For archiving and cleaning instants, we need to include intermediate state files if they exist
|
||||
HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false);
|
||||
Map<Pair<String, String>, List<HoodieInstant>> groupByTsAction = rawActiveTimeline.getInstants()
|
||||
.collect(Collectors.groupingBy(i -> Pair.of(i.getTimestamp(),
|
||||
HoodieInstant.getComparableAction(i.getAction()))));
|
||||
|
||||
return instants.flatMap(hoodieInstant ->
|
||||
groupByTsAction.get(Pair.of(hoodieInstant.getTimestamp(),
|
||||
HoodieInstant.getComparableAction(hoodieInstant.getAction()))).stream());
|
||||
}
|
||||
|
||||
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) throws IOException {
|
||||
@@ -194,6 +208,7 @@ public class HoodieCommitArchiveLog {
|
||||
return i.isCompleted() && (i.getAction().equals(HoodieTimeline.COMMIT_ACTION)
|
||||
|| (i.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)));
|
||||
}).max(Comparator.comparing(HoodieInstant::getTimestamp)));
|
||||
LOG.info("Latest Committed Instant=" + latestCommitted);
|
||||
if (latestCommitted.isPresent()) {
|
||||
success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get());
|
||||
}
|
||||
@@ -208,8 +223,8 @@ public class HoodieCommitArchiveLog {
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
private boolean deleteAllInstantsOlderorEqualsInAuxMetaFolder(HoodieInstant thresholdInstant) throws IOException {
|
||||
List<HoodieInstant> instants = HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
|
||||
List<HoodieInstant> instants = metaClient.scanHoodieInstantsFromFileSystem(
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false);
|
||||
|
||||
List<HoodieInstant> instantsToBeDeleted =
|
||||
instants.stream().filter(instant1 -> HoodieTimeline.compareTimestamps(instant1.getTimestamp(),
|
||||
@@ -270,10 +285,14 @@ public class HoodieCommitArchiveLog {
|
||||
throws IOException {
|
||||
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
||||
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
||||
archivedMetaWrapper.setActionState(hoodieInstant.getState().name());
|
||||
switch (hoodieInstant.getAction()) {
|
||||
case HoodieTimeline.CLEAN_ACTION: {
|
||||
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils
|
||||
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCleanMetadata.class));
|
||||
if (hoodieInstant.isCompleted()) {
|
||||
archivedMetaWrapper.setHoodieCleanMetadata(CleanerUtils.getCleanerMetadata(metaClient, hoodieInstant));
|
||||
} else {
|
||||
archivedMetaWrapper.setHoodieCleanerPlan(CleanerUtils.getCleanerPlan(metaClient, hoodieInstant));
|
||||
}
|
||||
archivedMetaWrapper.setActionType(ActionType.clean.name());
|
||||
break;
|
||||
}
|
||||
@@ -303,9 +322,16 @@ public class HoodieCommitArchiveLog {
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
default:
|
||||
case HoodieTimeline.COMPACTION_ACTION: {
|
||||
HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, hoodieInstant.getTimestamp());
|
||||
archivedMetaWrapper.setHoodieCompactionPlan(plan);
|
||||
archivedMetaWrapper.setActionType(ActionType.compaction.name());
|
||||
break;
|
||||
}
|
||||
default: {
|
||||
throw new UnsupportedOperationException("Action not fully supported yet");
|
||||
}
|
||||
}
|
||||
return archivedMetaWrapper;
|
||||
}
|
||||
|
||||
|
||||
@@ -35,7 +35,7 @@ import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
@@ -109,14 +109,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
Tuple2<String, String> partitionDelFileTuple = iter.next();
|
||||
String partitionPath = partitionDelFileTuple._1();
|
||||
String delFileName = partitionDelFileTuple._2();
|
||||
String deletePathStr = new Path(new Path(basePath, partitionPath), delFileName).toString();
|
||||
Path deletePath = new Path(new Path(basePath, partitionPath), delFileName);
|
||||
String deletePathStr = deletePath.toString();
|
||||
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
|
||||
if (!partitionCleanStatMap.containsKey(partitionPath)) {
|
||||
partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
|
||||
}
|
||||
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePathStr);
|
||||
partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult);
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePath.getName());
|
||||
partitionCleanStat.addDeletedFileResult(deletePath.getName(), deletedFileResult);
|
||||
}
|
||||
return partitionCleanStatMap.entrySet().stream().map(e -> new Tuple2<>(e.getKey(), e.getValue()))
|
||||
.collect(Collectors.toList()).iterator();
|
||||
@@ -312,11 +313,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
* @throws IllegalArgumentException if unknown cleaning policy is provided
|
||||
*/
|
||||
@Override
|
||||
public List<HoodieCleanStat> clean(JavaSparkContext jsc, HoodieInstant cleanInstant) {
|
||||
try {
|
||||
HoodieCleanerPlan cleanerPlan = AvroUtils.deserializeCleanerPlan(getActiveTimeline()
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCleanRequestedInstant(cleanInstant.getTimestamp())).get());
|
||||
|
||||
public List<HoodieCleanStat> clean(JavaSparkContext jsc, HoodieInstant cleanInstant, HoodieCleanerPlan cleanerPlan) {
|
||||
int cleanerParallelism = Math.min(
|
||||
(int) (cleanerPlan.getFilesToBeDeletedPerPartition().values().stream().mapToInt(x -> x.size()).count()),
|
||||
config.getCleanerParallelism());
|
||||
@@ -339,55 +336,55 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath)
|
||||
.withEarliestCommitRetained(Option.ofNullable(
|
||||
actionInstant != null
|
||||
? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()),
|
||||
? new HoodieInstant(State.valueOf(actionInstant.getState()),
|
||||
actionInstant.getAction(), actionInstant.getTimestamp())
|
||||
: null))
|
||||
.withDeletePathPattern(partitionCleanStat.deletePathPatterns)
|
||||
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles)
|
||||
.withFailedDeletes(partitionCleanStat.failedDeleteFiles).build();
|
||||
}).collect(Collectors.toList());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to clean up after commit", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, HoodieInstant instant, boolean deleteInstants)
|
||||
throws IOException {
|
||||
Long startTime = System.currentTimeMillis();
|
||||
List<HoodieRollbackStat> stats = new ArrayList<>();
|
||||
String actionType = metaClient.getCommitActionType();
|
||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
|
||||
List<String> inflights =
|
||||
this.getInflightCommitTimeline().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
// Atomically unpublish the commits
|
||||
if (!inflights.contains(commit)) {
|
||||
LOG.info("Unpublishing " + commit);
|
||||
activeTimeline.revertToInflight(new HoodieInstant(false, actionType, commit));
|
||||
|
||||
if (instant.isCompleted()) {
|
||||
LOG.info("Unpublishing instant " + instant);
|
||||
instant = activeTimeline.revertToInflight(instant);
|
||||
}
|
||||
|
||||
HoodieInstant instantToRollback = new HoodieInstant(false, actionType, commit);
|
||||
Long startTime = System.currentTimeMillis();
|
||||
// For Requested State (like failure during index lookup), there is nothing to do rollback other than
|
||||
// deleting the timeline file
|
||||
if (!instant.isRequested()) {
|
||||
String commit = instant.getTimestamp();
|
||||
|
||||
// delete all the data files for this commit
|
||||
LOG.info("Clean out all parquet files generated for commit: " + commit);
|
||||
List<RollbackRequest> rollbackRequests = generateRollbackRequests(instantToRollback);
|
||||
List<RollbackRequest> rollbackRequests = generateRollbackRequests(instant);
|
||||
|
||||
//TODO: We need to persist this as rollback workload and use it in case of partial failures
|
||||
List<HoodieRollbackStat> stats =
|
||||
new RollbackExecutor(metaClient, config).performRollback(jsc, instantToRollback, rollbackRequests);
|
||||
|
||||
stats = new RollbackExecutor(metaClient, config).performRollback(jsc, instant, rollbackRequests);
|
||||
}
|
||||
// Delete Inflight instant if enabled
|
||||
deleteInflightInstant(deleteInstants, activeTimeline, new HoodieInstant(true, actionType, commit));
|
||||
deleteInflightAndRequestedInstant(deleteInstants, activeTimeline, instant);
|
||||
LOG.info("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
return stats;
|
||||
}
|
||||
|
||||
private List<RollbackRequest> generateRollbackRequests(HoodieInstant instantToRollback) throws IOException {
|
||||
private List<RollbackRequest> generateRollbackRequests(HoodieInstant instantToRollback)
|
||||
throws IOException {
|
||||
return FSUtils.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()).stream().map(partitionPath -> {
|
||||
return RollbackRequest.createRollbackRequestWithDeleteDataAndLogFilesAction(partitionPath, instantToRollback);
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Delete Inflight instant if enabled.
|
||||
*
|
||||
@@ -395,15 +392,22 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
* @param activeTimeline Hoodie active timeline
|
||||
* @param instantToBeDeleted Instant to be deleted
|
||||
*/
|
||||
protected void deleteInflightInstant(boolean deleteInstant, HoodieActiveTimeline activeTimeline,
|
||||
protected void deleteInflightAndRequestedInstant(boolean deleteInstant, HoodieActiveTimeline activeTimeline,
|
||||
HoodieInstant instantToBeDeleted) {
|
||||
// Remove marker files always on rollback
|
||||
deleteMarkerDir(instantToBeDeleted.getTimestamp());
|
||||
|
||||
// Remove the rolled back inflight commits
|
||||
if (deleteInstant) {
|
||||
activeTimeline.deleteInflight(instantToBeDeleted);
|
||||
LOG.info("Deleted inflight commit " + instantToBeDeleted);
|
||||
LOG.info("Deleting instant=" + instantToBeDeleted);
|
||||
activeTimeline.deletePending(instantToBeDeleted);
|
||||
if (instantToBeDeleted.isInflight() && !metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
// Delete corresponding requested instant
|
||||
instantToBeDeleted = new HoodieInstant(State.REQUESTED, instantToBeDeleted.getAction(),
|
||||
instantToBeDeleted.getTimestamp());
|
||||
activeTimeline.deletePending(instantToBeDeleted);
|
||||
}
|
||||
LOG.info("Deleted pending commit " + instantToBeDeleted);
|
||||
} else {
|
||||
LOG.warn("Rollback finished without deleting inflight instant file. Instant=" + instantToBeDeleted);
|
||||
}
|
||||
|
||||
@@ -44,7 +44,6 @@ import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
@@ -167,32 +166,39 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||
throws IOException {
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, HoodieInstant instant,
|
||||
boolean deleteInstants) throws IOException {
|
||||
Long startTime = System.currentTimeMillis();
|
||||
|
||||
String commit = instant.getTimestamp();
|
||||
LOG.error("Rolling back instant " + instant);
|
||||
|
||||
// Atomically un-publish all non-inflight commits
|
||||
if (instant.isCompleted()) {
|
||||
LOG.error("Un-publishing instant " + instant + ", deleteInstants=" + deleteInstants);
|
||||
instant = this.getActiveTimeline().revertToInflight(instant);
|
||||
}
|
||||
|
||||
List<HoodieRollbackStat> allRollbackStats = new ArrayList<>();
|
||||
|
||||
// At the moment, MOR table type does not support bulk nested rollbacks. Nested rollbacks is an experimental
|
||||
// feature that is expensive. To perform nested rollbacks, initiate multiple requests of client.rollback
|
||||
// (commitToRollback).
|
||||
// NOTE {@link HoodieCompactionConfig#withCompactionLazyBlockReadEnabled} needs to be set to TRUE. This is
|
||||
// required to avoid OOM when merging multiple LogBlocks performed during nested rollbacks.
|
||||
// Atomically un-publish all non-inflight commits
|
||||
Option<HoodieInstant> commitOrCompactionOption = Option.fromJavaOptional(this.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION))
|
||||
.getInstants().filter(i -> commit.equals(i.getTimestamp())).findFirst());
|
||||
HoodieInstant instantToRollback = commitOrCompactionOption.get();
|
||||
// Atomically un-publish all non-inflight commits
|
||||
if (!instantToRollback.isInflight()) {
|
||||
this.getActiveTimeline().revertToInflight(instantToRollback);
|
||||
}
|
||||
// For Requested State (like failure during index lookup), there is nothing to do rollback other than
|
||||
// deleting the timeline file
|
||||
if (!instant.isRequested()) {
|
||||
LOG.info("Unpublished " + commit);
|
||||
Long startTime = System.currentTimeMillis();
|
||||
List<RollbackRequest> rollbackRequests = generateRollbackRequests(jsc, instantToRollback);
|
||||
List<RollbackRequest> rollbackRequests = generateRollbackRequests(jsc, instant);
|
||||
// TODO: We need to persist this as rollback workload and use it in case of partial failures
|
||||
List<HoodieRollbackStat> allRollbackStats =
|
||||
new RollbackExecutor(metaClient, config).performRollback(jsc, instantToRollback, rollbackRequests);
|
||||
allRollbackStats = new RollbackExecutor(metaClient, config).performRollback(jsc, instant, rollbackRequests);
|
||||
}
|
||||
|
||||
// Delete Inflight instants if enabled
|
||||
deleteInflightInstant(deleteInstants, this.getActiveTimeline(),
|
||||
new HoodieInstant(true, instantToRollback.getAction(), instantToRollback.getTimestamp()));
|
||||
deleteInflightAndRequestedInstant(deleteInstants, this.getActiveTimeline(), instant);
|
||||
|
||||
LOG.info("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
|
||||
|
||||
@@ -182,8 +182,8 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
/**
|
||||
* Get only the inflights (no-completed) commit timeline.
|
||||
*/
|
||||
public HoodieTimeline getInflightCommitTimeline() {
|
||||
return metaClient.getCommitsTimeline().filterInflightsExcludingCompaction();
|
||||
public HoodieTimeline getPendingCommitTimeline() {
|
||||
return metaClient.getCommitsTimeline().filterPendingExcludingCompaction();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -287,16 +287,18 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
*
|
||||
* @param jsc Java Spark Context
|
||||
* @param cleanInstant Clean Instant
|
||||
* @param cleanerPlan Cleaner Plan
|
||||
* @return list of Clean Stats
|
||||
*/
|
||||
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc, HoodieInstant cleanInstant);
|
||||
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc, HoodieInstant cleanInstant,
|
||||
HoodieCleanerPlan cleanerPlan);
|
||||
|
||||
/**
|
||||
* Rollback the (inflight/committed) record changes with the given commit time. Four steps: (1) Atomically unpublish
|
||||
* this commit (2) clean indexing data (3) clean new generated parquet files / log blocks (4) Finally, delete
|
||||
* .<action>.commit or .<action>.inflight file if deleteInstants = true
|
||||
*/
|
||||
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, HoodieInstant instant, boolean deleteInstants)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
|
||||
@@ -116,6 +116,8 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
// Reload and rollback inflight compaction
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc);
|
||||
// hoodieTable.rollback(jsc,
|
||||
// new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), false);
|
||||
|
||||
client.rollbackInflightCompaction(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable);
|
||||
@@ -166,7 +168,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
assertEquals("Pending Compaction instant has expected instant time", pendingCompactionInstant.getTimestamp(),
|
||||
compactionInstantTime);
|
||||
HoodieInstant inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get();
|
||||
assertEquals("inflight instant has expected instant time", inflightInstant.getTimestamp(), inflightInstantTime);
|
||||
|
||||
// This should rollback
|
||||
@@ -174,10 +176,10 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
|
||||
// Validate
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
inflightInstant = metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
inflightInstant = metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get();
|
||||
assertEquals("inflight instant has expected instant time", inflightInstant.getTimestamp(), nextInflightInstantTime);
|
||||
assertEquals("Expect only one inflight instant", 1, metaClient.getActiveTimeline()
|
||||
.filterInflightsExcludingCompaction().getInstants().count());
|
||||
.filterPendingExcludingCompaction().getInstants().count());
|
||||
// Expect pending Compaction to be present
|
||||
pendingCompactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertEquals("Pending Compaction instant has expected instant time", pendingCompactionInstant.getTimestamp(),
|
||||
@@ -274,7 +276,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase {
|
||||
|
||||
metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath());
|
||||
HoodieInstant inflightInstant =
|
||||
metaClient.getActiveTimeline().filterInflightsExcludingCompaction().firstInstant().get();
|
||||
metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get();
|
||||
assertEquals("inflight instant has expected instant time", inflightInstant.getTimestamp(), inflightInstantTime);
|
||||
|
||||
boolean gotException = false;
|
||||
|
||||
@@ -434,7 +434,8 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
metaClient.reloadActiveTimeline()
|
||||
.revertToInflight(new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, cleanInstantTs));
|
||||
final HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
HoodieCleanMetadata cleanMetadata2 = writeClient.runClean(table, cleanInstantTs);
|
||||
HoodieCleanMetadata cleanMetadata2 = writeClient.runClean(table,
|
||||
HoodieTimeline.getCleanInflightInstant(cleanInstantTs));
|
||||
Assert.assertTrue(
|
||||
Objects.equals(cleanMetadata1.getEarliestCommitToRetain(), cleanMetadata2.getEarliestCommitToRetain()));
|
||||
Assert.assertEquals(new Integer(0), cleanMetadata2.getTotalFilesDeleted());
|
||||
@@ -646,8 +647,11 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
|
||||
HoodieCleanMetadata metadata =
|
||||
CleanerUtils.convertCleanMetadata(metaClient, commitTime, Option.of(0L), Arrays.asList(cleanStat1, cleanStat2));
|
||||
metadata.setVersion(CleanerUtils.CLEAN_METADATA_VERSION_1);
|
||||
|
||||
Assert.assertEquals(CleanerUtils.LATEST_CLEAN_METADATA_VERSION, metadata.getVersion());
|
||||
// NOw upgrade and check
|
||||
CleanMetadataMigrator metadataMigrator = new CleanMetadataMigrator(metaClient);
|
||||
metadata = metadataMigrator.upgradeToLatest(metadata, metadata.getVersion());
|
||||
testCleanMetadataPathEquality(metadata, newExpected);
|
||||
|
||||
CleanMetadataMigrator migrator = new CleanMetadataMigrator(metaClient);
|
||||
@@ -736,7 +740,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
.build();
|
||||
|
||||
// make 1 commit, with 1 file per partition
|
||||
HoodieTestUtils.createCommitFiles(basePath, "000");
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, "000");
|
||||
|
||||
String file1P0C0 =
|
||||
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "000");
|
||||
@@ -769,7 +773,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
file1P1C0));
|
||||
|
||||
// make next commit, with 1 insert & 1 update per partition
|
||||
HoodieTestUtils.createCommitFiles(basePath, "001");
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, "001");
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
@@ -809,7 +813,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
file1P1C0));
|
||||
|
||||
// make next commit, with 2 updates to existing files, and 1 insert
|
||||
HoodieTestUtils.createCommitFiles(basePath, "002");
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, "002");
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
@@ -837,7 +841,7 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
file1P0C0));
|
||||
|
||||
// make next commit, with 2 updates to existing files, and 1 insert
|
||||
HoodieTestUtils.createCommitFiles(basePath, "003");
|
||||
HoodieTestUtils.createInflightCommitFiles(basePath, "003");
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
@@ -882,8 +886,10 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
|
||||
new ImmutableList.Builder<>().add(file3P0C2).build())
|
||||
.build());
|
||||
metaClient.getActiveTimeline().saveToInflight(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "004"),
|
||||
metaClient.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "004"));
|
||||
metaClient.getActiveTimeline().transitionRequestedToInflight(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "004"),
|
||||
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
List<HoodieCleanStat> hoodieCleanStatsFive = runCleaner(config, simulateFailureRetry);
|
||||
HoodieCleanStat cleanStat = getCleanStat(hoodieCleanStatsFive, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH);
|
||||
@@ -900,7 +906,6 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
*/
|
||||
@Test
|
||||
public void testCleanMarkerDataFilesOnRollback() throws IOException {
|
||||
HoodieTestUtils.createCommitFiles(basePath, "000");
|
||||
List<String> markerFiles = createMarkerFiles("000", 10);
|
||||
assertEquals("Some marker files are created.", 10, markerFiles.size());
|
||||
assertEquals("Some marker files are created.", markerFiles.size(), getTotalTempFiles());
|
||||
@@ -908,8 +913,12 @@ public class TestCleaner extends TestHoodieClientBase {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
table.rollback(jsc, "000", true);
|
||||
table.getActiveTimeline().createNewInstant(new HoodieInstant(State.REQUESTED,
|
||||
HoodieTimeline.COMMIT_ACTION, "000"));
|
||||
table.getActiveTimeline().transitionRequestedToInflight(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "000"), Option.empty());
|
||||
metaClient.reloadActiveTimeline();
|
||||
table.rollback(jsc, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000"), true);
|
||||
assertEquals("All temp files are deleted.", 0, getTotalTempFiles());
|
||||
}
|
||||
|
||||
|
||||
@@ -263,6 +263,8 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
String commitTime1 = "20160501010101";
|
||||
String commitTime2 = "20160502020601";
|
||||
String commitTime3 = "20160506030611";
|
||||
String commitTime4 = "20160506030621";
|
||||
String commitTime5 = "20160506030631";
|
||||
new File(basePath + "/.hoodie").mkdirs();
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, new String[] {"2016/05/01", "2016/05/02", "2016/05/06"},
|
||||
basePath);
|
||||
@@ -292,7 +294,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(config, false);) {
|
||||
|
||||
client.startCommitWithTime(commitTime4);
|
||||
// Check results, nothing changed
|
||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertTrue(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
@@ -310,7 +312,7 @@ public class TestClientRollback extends TestHoodieClientBase {
|
||||
|
||||
// Turn auto rollback on
|
||||
try (HoodieWriteClient client = getHoodieWriteClient(config, true)) {
|
||||
client.startCommit();
|
||||
client.startCommitWithTime(commitTime5);
|
||||
assertTrue(HoodieTestUtils.doesCommitExist(basePath, commitTime1));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime2));
|
||||
assertFalse(HoodieTestUtils.doesInflightExist(basePath, commitTime3));
|
||||
|
||||
@@ -27,9 +27,11 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRollingStat;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.ReadOptimizedView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
@@ -259,13 +261,16 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
/**
|
||||
* Test one of HoodieWriteClient upsert(Prepped) APIs.
|
||||
*
|
||||
* @param hoodieWriteConfig Write Config
|
||||
* @param config Write Config
|
||||
* @param writeFn One of Hoodie Write Function API
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private void testUpsertsInternal(HoodieWriteConfig hoodieWriteConfig,
|
||||
private void testUpsertsInternal(HoodieWriteConfig config,
|
||||
Function3<JavaRDD<WriteStatus>, HoodieWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPrepped)
|
||||
throws Exception {
|
||||
// Force using older timeline layout
|
||||
HoodieWriteConfig hoodieWriteConfig = getConfigBuilder().withProps(config.getProps()).withTimelineLayoutVersion(
|
||||
TimelineLayoutVersion.VERSION_0).build();
|
||||
HoodieWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
|
||||
// Write 1 (only inserts)
|
||||
@@ -292,6 +297,44 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase {
|
||||
deleteBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
|
||||
initCommitTime, numRecords, HoodieWriteClient::delete, isPrepped, true,
|
||||
0, 150);
|
||||
|
||||
// Now simulate an upgrade and perform a restore operation
|
||||
HoodieWriteConfig newConfig = getConfigBuilder().withProps(config.getProps()).withTimelineLayoutVersion(
|
||||
TimelineLayoutVersion.CURR_VERSION).build();
|
||||
client = getHoodieWriteClient(newConfig, false);
|
||||
client.restoreToInstant("004");
|
||||
|
||||
// Check the entire dataset has all records still
|
||||
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
|
||||
for (int i = 0; i < fullPartitionPaths.length; i++) {
|
||||
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
|
||||
}
|
||||
assertEquals("Must contain " + 200 + " records", 200,
|
||||
HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count());
|
||||
|
||||
// Perform Delete again on upgraded dataset.
|
||||
prevCommitTime = newCommitTime;
|
||||
newCommitTime = "006";
|
||||
numRecords = 50;
|
||||
|
||||
deleteBatch(newConfig, client, newCommitTime, prevCommitTime,
|
||||
initCommitTime, numRecords, HoodieWriteClient::delete, isPrepped, true,
|
||||
0, 150);
|
||||
|
||||
HoodieActiveTimeline activeTimeline = new HoodieActiveTimeline(metaClient, false);
|
||||
List<HoodieInstant> instants = activeTimeline.getCommitTimeline().getInstants().collect(Collectors.toList());
|
||||
Assert.assertEquals(5, instants.size());
|
||||
Assert.assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001"),
|
||||
instants.get(0));
|
||||
Assert.assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "004"),
|
||||
instants.get(1));
|
||||
// New Format should have all states of instants
|
||||
Assert.assertEquals(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "006"),
|
||||
instants.get(2));
|
||||
Assert.assertEquals(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "006"),
|
||||
instants.get(3));
|
||||
Assert.assertEquals(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "006"),
|
||||
instants.get(4));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -145,20 +145,31 @@ public class HoodieTestDataGenerator {
|
||||
createCommitFile(basePath, commitTime, HoodieTestUtils.getDefaultHadoopConf());
|
||||
}
|
||||
|
||||
public static void createCommitFile(String basePath, String commitTime, Configuration configuration)
|
||||
throws IOException {
|
||||
public static void createCommitFile(String basePath, String commitTime, Configuration configuration) {
|
||||
Arrays.asList(HoodieTimeline.makeCommitFileName(commitTime), HoodieTimeline.makeInflightCommitFileName(commitTime),
|
||||
HoodieTimeline.makeRequestedCommitFileName(commitTime)).forEach(f -> {
|
||||
Path commitFile = new Path(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f);
|
||||
FSDataOutputStream os = null;
|
||||
try {
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
os = fs.create(commitFile, true);
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
// Write empty commit metadata
|
||||
os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
} finally {
|
||||
if (null != os) {
|
||||
try {
|
||||
os.close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static void createCompactionRequestedFile(String basePath, String commitTime, Configuration configuration)
|
||||
throws IOException {
|
||||
|
||||
@@ -43,7 +43,7 @@ import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
|
||||
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
private final String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
||||
@@ -59,7 +59,7 @@ import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
|
||||
private final String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
private final String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
||||
@@ -37,6 +37,7 @@ import org.apache.hudi.index.hbase.HBaseIndexQPSResourceAllocator;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
@@ -131,7 +132,6 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
try (HoodieWriteClient writeClient = getWriteClient(config);) {
|
||||
writeClient.startCommit();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
@@ -140,6 +140,7 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
|
||||
|
||||
// Insert 200 records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
@@ -171,13 +172,19 @@ public class TestHbaseIndex extends HoodieClientTestHarness {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
HBaseIndex index = new HBaseIndex(config);
|
||||
HoodieWriteClient writeClient = new HoodieWriteClient(jsc, config);
|
||||
writeClient.startCommit();
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = index.tagLocation(writeRecords, jsc, hoodieTable);
|
||||
|
||||
// Duplicate upsert and ensure correctness is maintained
|
||||
// We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not
|
||||
// recomputed. This includes the state transitions. We need to delete the inflight instance so that subsequent
|
||||
// upsert will not run into conflicts.
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(), "001.inflight"));
|
||||
|
||||
writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
|
||||
@@ -45,8 +45,8 @@ import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
@@ -142,13 +142,12 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "100", dfs.getConf());
|
||||
HoodieTestUtils.createInflightCleanFiles(basePath, dfs.getConf(), "101");
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "101", dfs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "102", dfs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "103", dfs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "104", dfs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "105", dfs.getConf());
|
||||
HoodieTestUtils.createInflightCleanFiles(basePath, dfs.getConf(), "106", "107");
|
||||
HoodieTestUtils.createPendingCleanFiles(metaClient, dfs.getConf(), "106", "107");
|
||||
|
||||
// reload the timeline and get all the commmits before archive
|
||||
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
|
||||
@@ -157,7 +156,7 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants());
|
||||
|
||||
// verify in-flight instants before archive
|
||||
verifyInflightInstants(metaClient, 3);
|
||||
verifyInflightInstants(metaClient, 2);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
@@ -169,8 +168,8 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
|
||||
|
||||
// Check compaction instants
|
||||
List<HoodieInstant> instants = HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
|
||||
List<HoodieInstant> instants = metaClient.scanHoodieInstantsFromFileSystem(
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false);
|
||||
assertEquals("Should delete all compaction instants < 104", 4, instants.size());
|
||||
assertFalse("Requested Compaction must be absent for 100",
|
||||
instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")));
|
||||
@@ -201,30 +200,31 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
Reader reader =
|
||||
HoodieLogFormat.newReader(dfs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1_1-0-1")),
|
||||
HoodieArchivedMetaEntry.getClassSchema());
|
||||
|
||||
int archivedRecordsCount = 0;
|
||||
List<IndexedRecord> readRecords = new ArrayList<>();
|
||||
// read the avro blocks and validate the number of records written in each avro block
|
||||
int numBlocks = 0;
|
||||
while (reader.hasNext()) {
|
||||
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
||||
List<IndexedRecord> records = blk.getRecords();
|
||||
readRecords.addAll(records);
|
||||
assertEquals("Archived and read records for each block are same", 8, records.size());
|
||||
archivedRecordsCount += records.size();
|
||||
numBlocks++;
|
||||
}
|
||||
assertEquals("Total archived records and total read records are the same count", 8, archivedRecordsCount);
|
||||
|
||||
System.out.println("Read Records :" + readRecords.stream().map(r -> (GenericRecord) r)
|
||||
.map(r -> r.get("actionType") + "_" + r.get("actionState") + "_" + r.get("commitTime")).collect(Collectors.toList()));
|
||||
assertEquals("Total archived records and total read records are the same count", 24, archivedRecordsCount);
|
||||
assertTrue("Average Archived records per block is greater than 1", archivedRecordsCount / numBlocks > 1);
|
||||
// make sure the archived commits are the same as the (originalcommits - commitsleft)
|
||||
List<String> readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> {
|
||||
Set<String> readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> {
|
||||
return r.get("commitTime").toString();
|
||||
}).collect(Collectors.toList());
|
||||
Collections.sort(readCommits);
|
||||
}).collect(Collectors.toSet());
|
||||
|
||||
assertEquals("Read commits map should match the originalCommits - commitsLoadedFromArchival",
|
||||
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), readCommits);
|
||||
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), readCommits);
|
||||
|
||||
// verify in-flight instants after archive
|
||||
verifyInflightInstants(metaClient, 3);
|
||||
verifyInflightInstants(metaClient, 2);
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@@ -272,8 +272,8 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals("Should not archive commits when maxCommitsToKeep is 5", 4, timeline.countInstants());
|
||||
|
||||
List<HoodieInstant> instants = HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
|
||||
List<HoodieInstant> instants = metaClient.scanHoodieInstantsFromFileSystem(
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false);
|
||||
assertEquals("Should not delete any aux compaction files when maxCommitsToKeep is 5", 8, instants.size());
|
||||
assertTrue("Requested Compaction must be present for 100",
|
||||
instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")));
|
||||
|
||||
@@ -101,7 +101,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
public void testCompactionOnCopyOnWriteFail() throws Exception {
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, getConfig(), jsc);
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
}
|
||||
|
||||
@@ -117,7 +117,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
JavaRDD<WriteStatus> result =
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
assertTrue("If there is nothing to compact, result will be empty", result.isEmpty());
|
||||
@@ -167,7 +167,7 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
JavaRDD<WriteStatus> result =
|
||||
table.compact(jsc, compactionInstantTime, table.scheduleCompaction(jsc, compactionInstantTime));
|
||||
|
||||
|
||||
@@ -196,7 +196,7 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
client.startCommit();
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
@@ -821,6 +821,8 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
}
|
||||
|
||||
// Mark 2nd delta-instant as completed
|
||||
metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(State.INFLIGHT,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty());
|
||||
|
||||
@@ -1009,6 +1011,8 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc);
|
||||
tableRTFileSystemView = table.getRTFileSystemView();
|
||||
((SyncableFileSystemView) tableRTFileSystemView).reset();
|
||||
Option<HoodieInstant> lastInstant = ((SyncableFileSystemView) tableRTFileSystemView).getLastInstant();
|
||||
System.out.println("Last Instant =" + lastInstant);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
Assert.assertTrue(tableRTFileSystemView.getLatestFileSlices(partitionPath)
|
||||
.filter(fileSlice -> fileSlice.getDataFile().isPresent()).count() == 0);
|
||||
@@ -1032,8 +1036,10 @@ public class TestMergeOnReadTable extends HoodieClientTestHarness {
|
||||
// Create a commit without rolling stats in metadata to test backwards compatibility
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
String commitActionType = table.getMetaClient().getCommitActionType();
|
||||
HoodieInstant instant = new HoodieInstant(true, commitActionType, "000");
|
||||
activeTimeline.createInflight(instant);
|
||||
HoodieInstant instant = new HoodieInstant(State.REQUESTED, commitActionType, "000");
|
||||
activeTimeline.createNewInstant(instant);
|
||||
activeTimeline.transitionRequestedToInflight(instant, Option.empty());
|
||||
instant = new HoodieInstant(State.INFLIGHT, commitActionType, "000");
|
||||
activeTimeline.saveAsComplete(instant, Option.empty());
|
||||
|
||||
String commitTime = "001";
|
||||
|
||||
@@ -71,6 +71,7 @@
|
||||
<import>${basedir}/src/main/avro/HoodieSavePointMetadata.avsc</import>
|
||||
<import>${basedir}/src/main/avro/HoodieCompactionMetadata.avsc</import>
|
||||
<import>${basedir}/src/main/avro/HoodieCleanMetadata.avsc</import>
|
||||
<import>${basedir}/src/main/avro/HoodieCleanerPlan.avsc</import>
|
||||
<import>${basedir}/src/main/avro/HoodieRollbackMetadata.avsc</import>
|
||||
<import>${basedir}/src/main/avro/HoodieRestoreMetadata.avsc</import>
|
||||
<import>${basedir}/src/main/avro/HoodieArchivedMetaEntry.avsc</import>
|
||||
|
||||
@@ -37,6 +37,7 @@
|
||||
"default": null
|
||||
},
|
||||
{
|
||||
/** DEPRECATED **/
|
||||
"name":"hoodieCompactionMetadata",
|
||||
"type":[
|
||||
"null",
|
||||
@@ -74,6 +75,27 @@
|
||||
"name":"version",
|
||||
"type":["int", "null"],
|
||||
"default": 1
|
||||
},
|
||||
{
|
||||
"name":"hoodieCompactionPlan",
|
||||
"type":[
|
||||
"null",
|
||||
"HoodieCompactionPlan"
|
||||
],
|
||||
"default": null
|
||||
},
|
||||
{
|
||||
"name":"hoodieCleanerPlan",
|
||||
"type":[
|
||||
"null",
|
||||
"HoodieCleanerPlan"
|
||||
],
|
||||
"default": null
|
||||
},
|
||||
{
|
||||
"name":"actionState",
|
||||
"type":["null","string"],
|
||||
"default": null
|
||||
}
|
||||
]
|
||||
}
|
||||
|
||||
@@ -0,0 +1,79 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.common.model;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Metadata Layout Version. Add new version when timeline format changes
|
||||
*/
|
||||
public class TimelineLayoutVersion implements Serializable, Comparable<TimelineLayoutVersion> {
|
||||
|
||||
public static final Integer VERSION_0 = 0; // pre 0.5.1 version format
|
||||
public static final Integer VERSION_1 = 1; // current version with no renames
|
||||
|
||||
public static final Integer CURR_VERSION = VERSION_1;
|
||||
public static final TimelineLayoutVersion CURR_LAYOUT_VERSION = new TimelineLayoutVersion(CURR_VERSION);
|
||||
|
||||
private Integer version;
|
||||
|
||||
public TimelineLayoutVersion(Integer version) {
|
||||
Preconditions.checkArgument(version <= CURR_VERSION);
|
||||
Preconditions.checkArgument(version >= VERSION_0);
|
||||
this.version = version;
|
||||
}
|
||||
|
||||
/**
|
||||
* For Pre 0.5.1 release, there was no metadata version. This method is used to detect
|
||||
* this case.
|
||||
* @return
|
||||
*/
|
||||
public boolean isNullVersion() {
|
||||
return Objects.equals(version, VERSION_0);
|
||||
}
|
||||
|
||||
public Integer getVersion() {
|
||||
return version;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
TimelineLayoutVersion that = (TimelineLayoutVersion) o;
|
||||
return Objects.equals(version, that.version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(version);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(TimelineLayoutVersion o) {
|
||||
return Integer.compare(version, o.version);
|
||||
}
|
||||
}
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.common.table;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
@@ -54,12 +55,15 @@ public class HoodieTableConfig implements Serializable {
|
||||
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
|
||||
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME = "hoodie.table.ro.file.format";
|
||||
public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME = "hoodie.table.rt.file.format";
|
||||
public static final String HOODIE_TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version";
|
||||
public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class";
|
||||
public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = "hoodie.archivelog.folder";
|
||||
|
||||
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
|
||||
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
|
||||
public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
|
||||
public static final Integer DEFAULT_TIMELINE_LAYOUT_VERSION = TimelineLayoutVersion.VERSION_0;
|
||||
|
||||
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
|
||||
public static final String DEFAULT_ARCHIVELOG_FOLDER = "";
|
||||
private Properties props;
|
||||
@@ -112,6 +116,10 @@ public class HoodieTableConfig implements Serializable {
|
||||
if (!properties.containsKey(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) {
|
||||
properties.setProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);
|
||||
}
|
||||
if (!properties.containsKey(HOODIE_TIMELINE_LAYOUT_VERSION)) {
|
||||
// Use latest Version as default unless forced by client
|
||||
properties.setProperty(HOODIE_TIMELINE_LAYOUT_VERSION, TimelineLayoutVersion.CURR_VERSION.toString());
|
||||
}
|
||||
properties.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
}
|
||||
}
|
||||
@@ -126,6 +134,12 @@ public class HoodieTableConfig implements Serializable {
|
||||
return DEFAULT_TABLE_TYPE;
|
||||
}
|
||||
|
||||
public TimelineLayoutVersion getTimelineLayoutVersion() {
|
||||
return new TimelineLayoutVersion(Integer.valueOf(props.getProperty(HOODIE_TIMELINE_LAYOUT_VERSION,
|
||||
String.valueOf(DEFAULT_TIMELINE_LAYOUT_VERSION))));
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Read the payload class for HoodieRecords from the table properties.
|
||||
*/
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.common.table;
|
||||
import org.apache.hudi.common.SerializableConfiguration;
|
||||
import org.apache.hudi.common.io.storage.HoodieWrapperFileSystem;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
@@ -28,6 +29,7 @@ import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FailSafeConsistencyGuard;
|
||||
import org.apache.hudi.common.util.NoOpConsistencyGuard;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.DatasetNotFoundException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
@@ -44,12 +46,12 @@ import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* <code>HoodieTableMetaClient</code> allows to access meta-data about a hoodie table It returns meta-data about
|
||||
@@ -76,6 +78,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
private boolean loadActiveTimelineOnLoad;
|
||||
private SerializableConfiguration hadoopConf;
|
||||
private HoodieTableType tableType;
|
||||
private TimelineLayoutVersion timelineLayoutVersion;
|
||||
private HoodieTableConfig tableConfig;
|
||||
private HoodieActiveTimeline activeTimeline;
|
||||
private HoodieArchivedTimeline archivedTimeline;
|
||||
@@ -87,11 +90,13 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad) {
|
||||
this(conf, basePath, loadActiveTimelineOnLoad, ConsistencyGuardConfig.newBuilder().build());
|
||||
this(conf, basePath, loadActiveTimelineOnLoad, ConsistencyGuardConfig.newBuilder().build(),
|
||||
Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION));
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad,
|
||||
ConsistencyGuardConfig consistencyGuardConfig) throws DatasetNotFoundException {
|
||||
ConsistencyGuardConfig consistencyGuardConfig, Option<TimelineLayoutVersion> layoutVersion)
|
||||
throws DatasetNotFoundException {
|
||||
LOG.info("Loading HoodieTableMetaClient from " + basePath);
|
||||
this.basePath = basePath;
|
||||
this.consistencyGuardConfig = consistencyGuardConfig;
|
||||
@@ -103,8 +108,9 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
|
||||
this.tableConfig = new HoodieTableConfig(fs, metaPath);
|
||||
this.tableType = tableConfig.getTableType();
|
||||
LOG.info("Finished Loading Table of type " + tableType + " from " + basePath);
|
||||
this.timelineLayoutVersion = layoutVersion.orElse(tableConfig.getTimelineLayoutVersion());
|
||||
this.loadActiveTimelineOnLoad = loadActiveTimelineOnLoad;
|
||||
LOG.info("Finished Loading Table of type " + tableType + "(version=" + timelineLayoutVersion + ") from " + basePath);
|
||||
if (loadActiveTimelineOnLoad) {
|
||||
LOG.info("Loading Active commit timeline for " + basePath);
|
||||
getActiveTimeline();
|
||||
@@ -120,7 +126,8 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
|
||||
public static HoodieTableMetaClient reload(HoodieTableMetaClient oldMetaClient) {
|
||||
return new HoodieTableMetaClient(oldMetaClient.hadoopConf.get(), oldMetaClient.basePath,
|
||||
oldMetaClient.loadActiveTimelineOnLoad, oldMetaClient.consistencyGuardConfig);
|
||||
oldMetaClient.loadActiveTimelineOnLoad, oldMetaClient.consistencyGuardConfig,
|
||||
Option.of(oldMetaClient.timelineLayoutVersion));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -201,6 +208,10 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
return tableConfig;
|
||||
}
|
||||
|
||||
public TimelineLayoutVersion getTimelineLayoutVersion() {
|
||||
return timelineLayoutVersion;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the FS implementation for this table.
|
||||
*/
|
||||
@@ -274,12 +285,8 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
*/
|
||||
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath, String tableType,
|
||||
String tableName, String archiveLogFolder) throws IOException {
|
||||
HoodieTableType type = HoodieTableType.valueOf(tableType);
|
||||
Properties properties = new Properties();
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, type.name());
|
||||
properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder);
|
||||
return HoodieTableMetaClient.initDatasetAndGetMetaClient(hadoopConf, basePath, properties);
|
||||
return initTableType(hadoopConf, basePath, HoodieTableType.valueOf(tableType), tableName,
|
||||
archiveLogFolder, null, null);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -287,12 +294,26 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
*/
|
||||
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
|
||||
HoodieTableType tableType, String tableName, String payloadClassName) throws IOException {
|
||||
return initTableType(hadoopConf, basePath, tableType, tableName, null, payloadClassName, null);
|
||||
}
|
||||
|
||||
public static HoodieTableMetaClient initTableType(Configuration hadoopConf, String basePath,
|
||||
HoodieTableType tableType, String tableName, String archiveLogFolder, String payloadClassName,
|
||||
Integer timelineLayoutVersion) throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
|
||||
if (tableType == HoodieTableType.MERGE_ON_READ) {
|
||||
if (tableType == HoodieTableType.MERGE_ON_READ && payloadClassName != null) {
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
|
||||
}
|
||||
|
||||
if (null != archiveLogFolder) {
|
||||
properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder);
|
||||
}
|
||||
|
||||
if (null != timelineLayoutVersion) {
|
||||
properties.put(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION, String.valueOf(timelineLayoutVersion));
|
||||
}
|
||||
return HoodieTableMetaClient.initDatasetAndGetMetaClient(hadoopConf, basePath, properties);
|
||||
}
|
||||
|
||||
@@ -414,23 +435,41 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
/**
|
||||
* Helper method to scan all hoodie-instant metafiles and construct HoodieInstant objects.
|
||||
*
|
||||
* @param fs FileSystem
|
||||
* @param metaPath Meta Path where hoodie instants are present
|
||||
* @param includedExtensions Included hoodie extensions
|
||||
* @param applyLayoutVersionFilters Depending on Timeline layout version, if there are multiple states for the same
|
||||
* action instant, only include the highest state
|
||||
* @return List of Hoodie Instants generated
|
||||
* @throws IOException in case of failure
|
||||
*/
|
||||
public static List<HoodieInstant> scanHoodieInstantsFromFileSystem(FileSystem fs, Path metaPath,
|
||||
Set<String> includedExtensions) throws IOException {
|
||||
return Arrays.stream(HoodieTableMetaClient.scanFiles(fs, metaPath, path -> {
|
||||
public List<HoodieInstant> scanHoodieInstantsFromFileSystem(Set<String> includedExtensions,
|
||||
boolean applyLayoutVersionFilters) throws IOException {
|
||||
return scanHoodieInstantsFromFileSystem(new Path(metaPath), includedExtensions, applyLayoutVersionFilters);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to scan all hoodie-instant metafiles and construct HoodieInstant objects.
|
||||
*
|
||||
* @param timelinePath MetaPath where instant files are stored
|
||||
* @param includedExtensions Included hoodie extensions
|
||||
* @param applyLayoutVersionFilters Depending on Timeline layout version, if there are multiple states for the same
|
||||
* action instant, only include the highest state
|
||||
* @return List of Hoodie Instants generated
|
||||
* @throws IOException in case of failure
|
||||
*/
|
||||
public List<HoodieInstant> scanHoodieInstantsFromFileSystem(Path timelinePath, Set<String> includedExtensions,
|
||||
boolean applyLayoutVersionFilters) throws IOException {
|
||||
Stream<HoodieInstant> instantStream = Arrays.stream(
|
||||
HoodieTableMetaClient
|
||||
.scanFiles(getFs(), timelinePath, path -> {
|
||||
// Include only the meta files with extensions that needs to be included
|
||||
String extension = FSUtils.getFileExtension(path.getName());
|
||||
return includedExtensions.contains(extension);
|
||||
})).sorted(Comparator.comparing(
|
||||
// Sort the meta-data by the instant time (first part of the file name)
|
||||
fileStatus -> FSUtils.getInstantTime(fileStatus.getPath().getName())))
|
||||
// create HoodieInstantMarkers from FileStatus, which extracts properties
|
||||
.map(HoodieInstant::new).collect(Collectors.toList());
|
||||
})).map(HoodieInstant::new);
|
||||
|
||||
if (applyLayoutVersionFilters) {
|
||||
instantStream = TimelineLayout.getLayout(getTimelineLayoutVersion()).filterHoodieInstants(instantStream);
|
||||
}
|
||||
return instantStream.sorted().collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.common.table;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
@@ -62,6 +63,8 @@ public interface HoodieTimeline extends Serializable {
|
||||
String SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION;
|
||||
// this is to preserve backwards compatibility on commit in-flight filenames
|
||||
String INFLIGHT_COMMIT_EXTENSION = INFLIGHT_EXTENSION;
|
||||
String REQUESTED_COMMIT_EXTENSION = "." + COMMIT_ACTION + REQUESTED_EXTENSION;
|
||||
String REQUESTED_DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION + REQUESTED_EXTENSION;
|
||||
String INFLIGHT_DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION + INFLIGHT_EXTENSION;
|
||||
String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION;
|
||||
String REQUESTED_CLEAN_EXTENSION = "." + CLEAN_ACTION + REQUESTED_EXTENSION;
|
||||
@@ -94,7 +97,7 @@ public interface HoodieTimeline extends Serializable {
|
||||
*
|
||||
* @return New instance of HoodieTimeline with just in-flights excluding compaction inflights
|
||||
*/
|
||||
HoodieTimeline filterInflightsExcludingCompaction();
|
||||
HoodieTimeline filterPendingExcludingCompaction();
|
||||
|
||||
/**
|
||||
* Filter this timeline to just include the completed instants.
|
||||
@@ -251,7 +254,17 @@ public interface HoodieTimeline extends Serializable {
|
||||
return new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, timestamp);
|
||||
}
|
||||
|
||||
static HoodieInstant getInflightInstant(final HoodieInstant instant) {
|
||||
/**
|
||||
* Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names
|
||||
* between inflight and completed instants (compaction <=> commit).
|
||||
* @param instant Hoodie Instant
|
||||
* @param tableType Hoodie Table Type
|
||||
* @return Inflight Hoodie Instant
|
||||
*/
|
||||
static HoodieInstant getInflightInstant(final HoodieInstant instant, final HoodieTableType tableType) {
|
||||
if ((tableType == HoodieTableType.MERGE_ON_READ) && instant.getAction().equals(COMMIT_ACTION)) {
|
||||
return new HoodieInstant(true, COMPACTION_ACTION, instant.getTimestamp());
|
||||
}
|
||||
return new HoodieInstant(true, instant.getAction(), instant.getTimestamp());
|
||||
}
|
||||
|
||||
@@ -263,6 +276,10 @@ public interface HoodieTimeline extends Serializable {
|
||||
return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_COMMIT_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeRequestedCommitFileName(String commitTime) {
|
||||
return StringUtils.join(commitTime, HoodieTimeline.REQUESTED_COMMIT_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeCleanerFileName(String instant) {
|
||||
return StringUtils.join(instant, HoodieTimeline.CLEAN_EXTENSION);
|
||||
}
|
||||
@@ -295,6 +312,10 @@ public interface HoodieTimeline extends Serializable {
|
||||
return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeRequestedDeltaFileName(String commitTime) {
|
||||
return StringUtils.join(commitTime, HoodieTimeline.REQUESTED_DELTA_COMMIT_EXTENSION);
|
||||
}
|
||||
|
||||
static String makeInflightCompactionFileName(String commitTime) {
|
||||
return StringUtils.join(commitTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION);
|
||||
}
|
||||
|
||||
@@ -0,0 +1,79 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.common.table;
|
||||
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
|
||||
/**
|
||||
* Timeline Layout responsible for applying specific filters when generating timeline instants.
|
||||
*/
|
||||
public abstract class TimelineLayout implements Serializable {
|
||||
|
||||
private static final Map<TimelineLayoutVersion, TimelineLayout> LAYOUT_MAP = new HashMap<>();
|
||||
|
||||
static {
|
||||
LAYOUT_MAP.put(new TimelineLayoutVersion(TimelineLayoutVersion.VERSION_0), new TimelineLayoutV0());
|
||||
LAYOUT_MAP.put(new TimelineLayoutVersion(TimelineLayoutVersion.VERSION_1), new TimelineLayoutV1());
|
||||
}
|
||||
|
||||
public static TimelineLayout getLayout(TimelineLayoutVersion version) {
|
||||
return LAYOUT_MAP.get(version);
|
||||
}
|
||||
|
||||
public abstract Stream<HoodieInstant> filterHoodieInstants(Stream<HoodieInstant> instantStream);
|
||||
|
||||
/**
|
||||
* Table Layout where state transitions are managed by renaming files.
|
||||
*/
|
||||
private static class TimelineLayoutV0 extends TimelineLayout {
|
||||
|
||||
@Override
|
||||
public Stream<HoodieInstant> filterHoodieInstants(Stream<HoodieInstant> instantStream) {
|
||||
return instantStream;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Table Layout where state transitions are managed by creating new files.
|
||||
*/
|
||||
private static class TimelineLayoutV1 extends TimelineLayout {
|
||||
|
||||
@Override
|
||||
public Stream<HoodieInstant> filterHoodieInstants(Stream<HoodieInstant> instantStream) {
|
||||
return instantStream.collect(Collectors.groupingBy(instant -> Pair.of(instant.getTimestamp(),
|
||||
HoodieInstant.getComparableAction(instant.getAction())))).entrySet().stream()
|
||||
.map(e -> e.getValue().stream().reduce((x, y) -> {
|
||||
// Pick the one with the highest state
|
||||
if (x.getState().compareTo(y.getState()) >= 0) {
|
||||
return x;
|
||||
}
|
||||
return y;
|
||||
}).get());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -43,6 +43,7 @@ import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
@@ -61,21 +62,21 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
|
||||
|
||||
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE =
|
||||
new HashSet<>(Arrays.asList(new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
|
||||
INFLIGHT_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION,
|
||||
INFLIGHT_CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_COMPACTION_EXTENSION,
|
||||
REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
|
||||
public static final Set<String> VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList(
|
||||
new String[]{COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION,
|
||||
INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION,
|
||||
INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION,
|
||||
INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class);
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
private static AtomicReference<String> lastInstantTime = new AtomicReference<>(String.valueOf(Integer.MIN_VALUE));
|
||||
|
||||
/**
|
||||
* Returns next commit time in the {@link #COMMIT_FORMATTER} format.
|
||||
* Ensures each commit time is atleast 1 second apart since we create COMMIT times at second granularity
|
||||
* Returns next instant time in the {@link #COMMIT_FORMATTER} format.
|
||||
* Ensures each instant time is atleast 1 second apart since we create instant times at second granularity
|
||||
*/
|
||||
public static String createNewCommitTime() {
|
||||
public static String createNewInstantTime() {
|
||||
lastInstantTime.updateAndGet((oldVal) -> {
|
||||
String newCommitTime = null;
|
||||
do {
|
||||
@@ -87,12 +88,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, Set<String> includedExtensions) {
|
||||
this(metaClient, includedExtensions, true);
|
||||
}
|
||||
|
||||
protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, Set<String> includedExtensions,
|
||||
boolean applyLayoutFilters) {
|
||||
// Filter all the filter in the metapath and include only the extensions passed and
|
||||
// convert them into HoodieInstant
|
||||
try {
|
||||
this.setInstants(HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
|
||||
new Path(metaClient.getMetaPath()), includedExtensions));
|
||||
LOG.info("Loaded instants " + getInstants());
|
||||
this.setInstants(metaClient.scanHoodieInstantsFromFileSystem(includedExtensions, applyLayoutFilters));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to scan metadata", e);
|
||||
}
|
||||
@@ -100,12 +104,19 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
// multiple casts will make this lambda serializable -
|
||||
// http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
|
||||
this.details = (Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails;
|
||||
LOG.info("Loaded instants " + getInstants().collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
public HoodieActiveTimeline(HoodieTableMetaClient metaClient) {
|
||||
this(metaClient, new ImmutableSet.Builder<String>().addAll(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE).build());
|
||||
}
|
||||
|
||||
public HoodieActiveTimeline(HoodieTableMetaClient metaClient, boolean applyLayoutFilter) {
|
||||
this(metaClient,
|
||||
new ImmutableSet.Builder<String>()
|
||||
.addAll(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE).build(), applyLayoutFilter);
|
||||
}
|
||||
|
||||
/**
|
||||
* For serialization and de-serialization only.
|
||||
*
|
||||
@@ -209,10 +220,10 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
return getInstants().filter(s -> s.getAction().equals(action));
|
||||
}
|
||||
|
||||
public void createInflight(HoodieInstant instant) {
|
||||
LOG.info("Creating a new in-flight instant " + instant);
|
||||
public void createNewInstant(HoodieInstant instant) {
|
||||
LOG.info("Creating a new instant " + instant);
|
||||
// Create the in-flight file
|
||||
createFileInMetaPath(instant.getFileName(), Option.empty());
|
||||
createFileInMetaPath(instant.getFileName(), Option.empty(), false);
|
||||
}
|
||||
|
||||
public void saveAsComplete(HoodieInstant instant, Option<byte[]> data) {
|
||||
@@ -223,18 +234,12 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
LOG.info("Completed " + instant);
|
||||
}
|
||||
|
||||
public void revertToInflight(HoodieInstant instant) {
|
||||
LOG.info("Reverting " + instant + " to inflight ");
|
||||
revertStateTransition(instant, HoodieTimeline.getInflightInstant(instant));
|
||||
LOG.info("Reverted " + instant + " to inflight");
|
||||
}
|
||||
|
||||
public HoodieInstant revertToRequested(HoodieInstant instant) {
|
||||
LOG.warn("Reverting " + instant + " to requested ");
|
||||
HoodieInstant requestedInstant = HoodieTimeline.getRequestedInstant(instant);
|
||||
revertStateTransition(instant, HoodieTimeline.getRequestedInstant(instant));
|
||||
LOG.warn("Reverted " + instant + " to requested");
|
||||
return requestedInstant;
|
||||
public HoodieInstant revertToInflight(HoodieInstant instant) {
|
||||
LOG.info("Reverting instant to inflight " + instant);
|
||||
HoodieInstant inflight = HoodieTimeline.getInflightInstant(instant, metaClient.getTableType());
|
||||
revertCompleteToInflight(instant, inflight);
|
||||
LOG.info("Reverted " + instant + " to inflight " + inflight);
|
||||
return inflight;
|
||||
}
|
||||
|
||||
public void deleteInflight(HoodieInstant instant) {
|
||||
@@ -242,6 +247,11 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
deleteInstantFile(instant);
|
||||
}
|
||||
|
||||
public void deletePending(HoodieInstant instant) {
|
||||
Preconditions.checkArgument(!instant.isCompleted());
|
||||
deleteInstantFile(instant);
|
||||
}
|
||||
|
||||
public void deleteCompactionRequested(HoodieInstant instant) {
|
||||
Preconditions.checkArgument(instant.isRequested());
|
||||
Preconditions.checkArgument(instant.getAction() == HoodieTimeline.COMPACTION_ACTION);
|
||||
@@ -254,9 +264,9 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
try {
|
||||
boolean result = metaClient.getFs().delete(inFlightCommitFilePath, false);
|
||||
if (result) {
|
||||
LOG.info("Removed in-flight " + instant);
|
||||
LOG.info("Removed instant " + instant);
|
||||
} else {
|
||||
throw new HoodieIOException("Could not delete in-flight instant " + instant);
|
||||
throw new HoodieIOException("Could not delete instant " + instant);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not remove inflight commit " + inFlightCommitFilePath, e);
|
||||
@@ -273,8 +283,13 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
// BEGIN - COMPACTION RELATED META-DATA MANAGEMENT.
|
||||
//-----------------------------------------------------------------
|
||||
|
||||
public Option<byte[]> getInstantAuxiliaryDetails(HoodieInstant instant) {
|
||||
Path detailPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
|
||||
public Option<byte[]> readPlanAsBytes(HoodieInstant instant) {
|
||||
Path detailPath = null;
|
||||
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
detailPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
|
||||
} else {
|
||||
detailPath = new Path(metaClient.getMetaPath(), instant.getFileName());
|
||||
}
|
||||
return readDataFromPath(detailPath);
|
||||
}
|
||||
|
||||
@@ -289,8 +304,12 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
Preconditions.checkArgument(inflightInstant.isInflight());
|
||||
HoodieInstant requestedInstant =
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, inflightInstant.getTimestamp());
|
||||
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
// Pass empty data since it is read from the corresponding .aux/.compaction instant file
|
||||
transitionState(inflightInstant, requestedInstant, Option.empty());
|
||||
} else {
|
||||
deleteInflight(inflightInstant);
|
||||
}
|
||||
return requestedInstant;
|
||||
}
|
||||
|
||||
@@ -325,9 +344,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
private void createFileInAuxiliaryFolder(HoodieInstant instant, Option<byte[]> data) {
|
||||
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
/**
|
||||
* For latest version, since we write immutable files directly in timeline directory, there is no need to write
|
||||
* additional immutable files in .aux folder
|
||||
*/
|
||||
Path fullPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
|
||||
createFileInPath(fullPath, data);
|
||||
}
|
||||
}
|
||||
|
||||
//-----------------------------------------------------------------
|
||||
// END - COMPACTION RELATED META-DATA MANAGEMENT
|
||||
@@ -355,59 +380,92 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
* Transition Clean State from requested to inflight.
|
||||
*
|
||||
* @param requestedInstant requested instant
|
||||
* @param data Optional data to be stored
|
||||
* @return commit instant
|
||||
*/
|
||||
public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant) {
|
||||
public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant, Option<byte[]> data) {
|
||||
Preconditions.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION));
|
||||
Preconditions.checkArgument(requestedInstant.isRequested());
|
||||
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, CLEAN_ACTION, requestedInstant.getTimestamp());
|
||||
transitionState(requestedInstant, inflight, Option.empty());
|
||||
transitionState(requestedInstant, inflight, data);
|
||||
return inflight;
|
||||
}
|
||||
|
||||
private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option<byte[]> data) {
|
||||
Preconditions.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp()));
|
||||
Path commitFilePath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
|
||||
try {
|
||||
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
// Re-create the .inflight file by opening a new file and write the commit metadata in
|
||||
Path inflightCommitFile = new Path(metaClient.getMetaPath(), fromInstant.getFileName());
|
||||
createFileInMetaPath(fromInstant.getFileName(), data);
|
||||
boolean success = metaClient.getFs().rename(inflightCommitFile, commitFilePath);
|
||||
createFileInMetaPath(fromInstant.getFileName(), data, false);
|
||||
Path fromInstantPath = new Path(metaClient.getMetaPath(), fromInstant.getFileName());
|
||||
Path toInstantPath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
|
||||
boolean success = metaClient.getFs().rename(fromInstantPath, toInstantPath);
|
||||
if (!success) {
|
||||
throw new HoodieIOException("Could not rename " + inflightCommitFile + " to " + commitFilePath);
|
||||
throw new HoodieIOException("Could not rename " + fromInstantPath + " to " + toInstantPath);
|
||||
}
|
||||
} else {
|
||||
// Ensures old state exists in timeline
|
||||
LOG.info("Checking for file exists ?" + new Path(metaClient.getMetaPath(), fromInstant.getFileName()));
|
||||
Preconditions.checkArgument(metaClient.getFs().exists(new Path(metaClient.getMetaPath(),
|
||||
fromInstant.getFileName())));
|
||||
// Use Write Once to create Target File
|
||||
createImmutableFileInPath(new Path(metaClient.getMetaPath(), toInstant.getFileName()), data);
|
||||
LOG.info("Create new file for toInstant ?" + new Path(metaClient.getMetaPath(), toInstant.getFileName()));
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not complete " + fromInstant, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void revertStateTransition(HoodieInstant curr, HoodieInstant revert) {
|
||||
Preconditions.checkArgument(curr.getTimestamp().equals(revert.getTimestamp()));
|
||||
Path revertFilePath = new Path(metaClient.getMetaPath(), revert.getFileName());
|
||||
private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
|
||||
Preconditions.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp()));
|
||||
Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), inflight.getFileName());
|
||||
Path commitFilePath = new Path(metaClient.getMetaPath(), completed.getFileName());
|
||||
try {
|
||||
if (!metaClient.getFs().exists(revertFilePath)) {
|
||||
Path currFilePath = new Path(metaClient.getMetaPath(), curr.getFileName());
|
||||
boolean success = metaClient.getFs().rename(currFilePath, revertFilePath);
|
||||
if (metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
|
||||
boolean success = metaClient.getFs().rename(commitFilePath, inFlightCommitFilePath);
|
||||
if (!success) {
|
||||
throw new HoodieIOException("Could not rename " + currFilePath + " to " + revertFilePath);
|
||||
throw new HoodieIOException(
|
||||
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
|
||||
}
|
||||
LOG.info("Renamed " + currFilePath + " to " + revertFilePath);
|
||||
}
|
||||
} else {
|
||||
Path requestedInstantFilePath = new Path(metaClient.getMetaPath(),
|
||||
new HoodieInstant(State.REQUESTED, inflight.getAction(), inflight.getTimestamp()).getFileName());
|
||||
|
||||
// If inflight and requested files do not exist, create one
|
||||
if (!metaClient.getFs().exists(requestedInstantFilePath)) {
|
||||
metaClient.getFs().create(requestedInstantFilePath, false).close();
|
||||
}
|
||||
|
||||
if (!metaClient.getFs().exists(inFlightCommitFilePath)) {
|
||||
metaClient.getFs().create(inFlightCommitFilePath, false).close();
|
||||
}
|
||||
|
||||
boolean success = metaClient.getFs().delete(commitFilePath, false);
|
||||
Preconditions.checkArgument(success, "State Reverting failed");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not complete revert " + curr, e);
|
||||
throw new HoodieIOException("Could not complete revert " + completed, e);
|
||||
}
|
||||
}
|
||||
|
||||
public void saveToInflight(HoodieInstant instant, Option<byte[]> content) {
|
||||
Preconditions.checkArgument(instant.isInflight());
|
||||
createFileInMetaPath(instant.getFileName(), content);
|
||||
public void transitionRequestedToInflight(HoodieInstant requested, Option<byte[]> content) {
|
||||
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, requested.getAction(), requested.getTimestamp());
|
||||
Preconditions.checkArgument(requested.isRequested(), "Instant " + requested + " in wrong state");
|
||||
transitionState(requested, inflight, content);
|
||||
}
|
||||
|
||||
public void saveToCompactionRequested(HoodieInstant instant, Option<byte[]> content) {
|
||||
saveToCompactionRequested(instant, content, false);
|
||||
}
|
||||
|
||||
public void saveToCompactionRequested(HoodieInstant instant, Option<byte[]> content, boolean overwrite) {
|
||||
Preconditions.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
|
||||
// Write workload to auxiliary folder
|
||||
createFileInAuxiliaryFolder(instant, content);
|
||||
createFileInMetaPath(instant.getFileName(), content);
|
||||
createFileInMetaPath(instant.getFileName(), content, overwrite);
|
||||
}
|
||||
|
||||
public void saveToCleanRequested(HoodieInstant instant, Option<byte[]> content) {
|
||||
@@ -416,12 +474,16 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
// Write workload to auxiliary folder
|
||||
createFileInAuxiliaryFolder(instant, content);
|
||||
// Plan is only stored in auxiliary folder
|
||||
createFileInMetaPath(instant.getFileName(), Option.empty());
|
||||
createFileInMetaPath(instant.getFileName(), Option.empty(), false);
|
||||
}
|
||||
|
||||
private void createFileInMetaPath(String filename, Option<byte[]> content) {
|
||||
private void createFileInMetaPath(String filename, Option<byte[]> content, boolean allowOverwrite) {
|
||||
Path fullPath = new Path(metaClient.getMetaPath(), filename);
|
||||
if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) {
|
||||
createFileInPath(fullPath, content);
|
||||
} else {
|
||||
createImmutableFileInPath(fullPath, content);
|
||||
}
|
||||
}
|
||||
|
||||
private void createFileInPath(Path fullPath, Option<byte[]> content) {
|
||||
@@ -445,6 +507,32 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new file in timeline with overwrite set to false. This ensures
|
||||
* files are created only once and never rewritten
|
||||
* @param fullPath File Path
|
||||
* @param content Content to be stored
|
||||
*/
|
||||
private void createImmutableFileInPath(Path fullPath, Option<byte[]> content) {
|
||||
FSDataOutputStream fsout = null;
|
||||
try {
|
||||
fsout = metaClient.getFs().create(fullPath, false);
|
||||
if (content.isPresent()) {
|
||||
fsout.write(content.get());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to create file " + fullPath, e);
|
||||
} finally {
|
||||
try {
|
||||
if (null != fsout) {
|
||||
fsout.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to close file " + fullPath, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Option<byte[]> readDataFromPath(Path detailPath) {
|
||||
try (FSDataInputStream is = metaClient.getFs().open(detailPath)) {
|
||||
return Option.of(FileIOUtils.readAsByteArray(is));
|
||||
|
||||
@@ -70,7 +70,6 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
} catch (NoSuchAlgorithmException nse) {
|
||||
throw new HoodieException(nse);
|
||||
}
|
||||
|
||||
this.timelineHash = StringUtils.toHexString(md.digest());
|
||||
}
|
||||
|
||||
@@ -94,15 +93,15 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline filterInflightsExcludingCompaction() {
|
||||
public HoodieTimeline filterPendingExcludingCompaction() {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(instant -> {
|
||||
return instant.isInflight() && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
|
||||
return (!instant.isCompleted()) && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
|
||||
}), details);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline filterCompletedInstants() {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(s -> !s.isInflight()), details);
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(HoodieInstant::isCompleted), details);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -223,5 +222,4 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
public String toString() {
|
||||
return this.getClass().getName() + ": " + instants.stream().map(Object::toString).collect(Collectors.joining(","));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -21,9 +21,12 @@ package org.apache.hudi.common.table.timeline;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Comparator;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
@@ -32,7 +35,24 @@ import java.util.Objects;
|
||||
*
|
||||
* @see HoodieTimeline
|
||||
*/
|
||||
public class HoodieInstant implements Serializable {
|
||||
public class HoodieInstant implements Serializable, Comparable<HoodieInstant> {
|
||||
|
||||
/**
|
||||
* A COMPACTION action eventually becomes COMMIT when completed. So, when grouping instants
|
||||
* for state transitions, this needs to be taken into account
|
||||
*/
|
||||
private static final Map<String, String> COMPARABLE_ACTIONS = new ImmutableMap.Builder<String, String>()
|
||||
.put(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION).build();
|
||||
|
||||
public static final Comparator<HoodieInstant> ACTION_COMPARATOR =
|
||||
Comparator.<HoodieInstant, String>comparing(instant -> getComparableAction(instant.getAction()));
|
||||
|
||||
public static final Comparator<HoodieInstant> COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp)
|
||||
.thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState);
|
||||
|
||||
public static final String getComparableAction(String action) {
|
||||
return COMPARABLE_ACTIONS.containsKey(action) ? COMPARABLE_ACTIONS.get(action) : action;
|
||||
}
|
||||
|
||||
/**
|
||||
* Instant State.
|
||||
@@ -116,6 +136,7 @@ public class HoodieInstant implements Serializable {
|
||||
public String getFileName() {
|
||||
if (HoodieTimeline.COMMIT_ACTION.equals(action)) {
|
||||
return isInflight() ? HoodieTimeline.makeInflightCommitFileName(timestamp)
|
||||
: isRequested() ? HoodieTimeline.makeRequestedCommitFileName(timestamp)
|
||||
: HoodieTimeline.makeCommitFileName(timestamp);
|
||||
} else if (HoodieTimeline.CLEAN_ACTION.equals(action)) {
|
||||
return isInflight() ? HoodieTimeline.makeInflightCleanerFileName(timestamp)
|
||||
@@ -129,6 +150,7 @@ public class HoodieInstant implements Serializable {
|
||||
: HoodieTimeline.makeSavePointFileName(timestamp);
|
||||
} else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) {
|
||||
return isInflight() ? HoodieTimeline.makeInflightDeltaFileName(timestamp)
|
||||
: isRequested() ? HoodieTimeline.makeRequestedDeltaFileName(timestamp)
|
||||
: HoodieTimeline.makeDeltaFileName(timestamp);
|
||||
} else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) {
|
||||
if (isInflight()) {
|
||||
@@ -166,6 +188,11 @@ public class HoodieInstant implements Serializable {
|
||||
return Objects.hash(state, action, timestamp);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(HoodieInstant o) {
|
||||
return COMPARATOR.compare(this, o);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "[" + ((isInflight() || isRequested()) ? "==>" : "") + timestamp + "__" + action + "__" + state + "]";
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.CleanerUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
@@ -259,8 +260,7 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
|
||||
*/
|
||||
private void addCleanInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
|
||||
LOG.info("Syncing cleaner instant (" + instant + ")");
|
||||
HoodieCleanMetadata cleanMetadata =
|
||||
AvroUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(instant).get());
|
||||
HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(metaClient, instant);
|
||||
cleanMetadata.getPartitionMetadata().entrySet().stream().forEach(entry -> {
|
||||
final String basePath = metaClient.getBasePath();
|
||||
final String partitionPath = entry.getValue().getPartitionPath();
|
||||
|
||||
@@ -20,14 +20,17 @@ package org.apache.hudi.common.util;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.versioning.clean.CleanMetadataMigrator;
|
||||
import org.apache.hudi.common.versioning.clean.CleanV1MigrationHandler;
|
||||
import org.apache.hudi.common.versioning.clean.CleanV2MigrationHandler;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
public class CleanerUtils {
|
||||
@@ -54,9 +57,35 @@ public class CleanerUtils {
|
||||
|
||||
HoodieCleanMetadata metadata = new HoodieCleanMetadata(startCleanTime,
|
||||
durationInMs.orElseGet(() -> -1L), totalDeleted, earliestCommitToRetain,
|
||||
partitionMetadataBuilder.build(), CLEAN_METADATA_VERSION_1);
|
||||
partitionMetadataBuilder.build(), CLEAN_METADATA_VERSION_2);
|
||||
return metadata;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Latest Version of Hoodie Cleaner Metadata - Output of cleaner operation.
|
||||
* @param metaClient Hoodie Table Meta Client
|
||||
* @param cleanInstant Instant referring to clean action
|
||||
* @return Latest version of Clean metadata corresponding to clean instant
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HoodieCleanMetadata getCleanerMetadata(HoodieTableMetaClient metaClient, HoodieInstant cleanInstant)
|
||||
throws IOException {
|
||||
CleanMetadataMigrator metadataMigrator = new CleanMetadataMigrator(metaClient);
|
||||
return metadataMigrator.upgradeToLatest(metadata, metadata.getVersion());
|
||||
HoodieCleanMetadata cleanMetadata = AvroUtils.deserializeHoodieCleanMetadata(
|
||||
metaClient.getActiveTimeline().readPlanAsBytes(cleanInstant).get());
|
||||
return metadataMigrator.upgradeToLatest(cleanMetadata, cleanMetadata.getVersion());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Cleaner Plan corresponding to a clean instant.
|
||||
* @param metaClient Hoodie Table Meta Client
|
||||
* @param cleanInstant Instant referring to clean action
|
||||
* @return Cleaner plan corresponding to clean instant
|
||||
* @throws IOException
|
||||
*/
|
||||
public static HoodieCleanerPlan getCleanerPlan(HoodieTableMetaClient metaClient, HoodieInstant cleanInstant)
|
||||
throws IOException {
|
||||
return AvroUtils.deserializeAvroMetadata(metaClient.getActiveTimeline().readPlanAsBytes(cleanInstant).get(),
|
||||
HoodieCleanerPlan.class);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -139,8 +139,9 @@ public class CompactionUtils {
|
||||
public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, String compactionInstant)
|
||||
throws IOException {
|
||||
CompactionPlanMigrator migrator = new CompactionPlanMigrator(metaClient);
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(metaClient.getActiveTimeline()
|
||||
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
metaClient.getActiveTimeline().readPlanAsBytes(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
|
||||
return migrator.upgradeToLatest(compactionPlan, compactionPlan.getVersion());
|
||||
}
|
||||
|
||||
|
||||
@@ -18,7 +18,9 @@
|
||||
|
||||
package org.apache.hudi.common.model;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieActionInstant;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieCleanerPlan;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
@@ -39,6 +41,7 @@ import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import com.esotericsoftware.kryo.Kryo;
|
||||
import com.esotericsoftware.kryo.io.Input;
|
||||
@@ -68,6 +71,7 @@ import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Calendar;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -123,6 +127,12 @@ public class HoodieTestUtils {
|
||||
|
||||
public static final void createCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeRequestedCommitFileName(commitTime)).createNewFile();
|
||||
new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeInflightCommitFileName(commitTime)).createNewFile();
|
||||
new File(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(commitTime))
|
||||
.createNewFile();
|
||||
@@ -142,24 +152,41 @@ public class HoodieTestUtils {
|
||||
}
|
||||
|
||||
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
|
||||
|
||||
for (String commitTime : commitTimes) {
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeInflightCommitFileName(commitTime)).createNewFile();
|
||||
+ HoodieTimeline.makeRequestedCommitFileName(commitTime)).createNewFile();
|
||||
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeInflightCommitFileName(
|
||||
commitTime)).createNewFile();
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createInflightCleanFiles(String basePath, Configuration configuration, String... commitTimes)
|
||||
throws IOException {
|
||||
public static final void createPendingCleanFiles(HoodieTableMetaClient metaClient, Configuration configuration,
|
||||
String... commitTimes) throws IOException {
|
||||
for (String commitTime : commitTimes) {
|
||||
Path commitFile = new Path((basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeInflightCleanerFileName(commitTime)));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
FSDataOutputStream os = fs.create(commitFile, true);
|
||||
Arrays.asList(HoodieTimeline.makeRequestedCleanerFileName(commitTime),
|
||||
HoodieTimeline.makeInflightCleanerFileName(commitTime)).forEach(f -> {
|
||||
FSDataOutputStream os = null;
|
||||
try {
|
||||
Path commitFile = new Path(
|
||||
metaClient.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f);
|
||||
os = metaClient.getFs().create(commitFile, true);
|
||||
// Write empty clean metadata
|
||||
os.write(AvroUtils.serializeCleanerPlan(
|
||||
new HoodieCleanerPlan(new HoodieActionInstant("", "", ""), "", new HashMap<>(), 1)).get());
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
} finally {
|
||||
if (null != os) {
|
||||
try {
|
||||
os.close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
public static final void createInflightCleanFiles(String basePath, String... commitTimes) throws IOException {
|
||||
createInflightCleanFiles(basePath, HoodieTestUtils.getDefaultHadoopConf(), commitTimes);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
public static final String createNewDataFile(String basePath, String partitionPath, String commitTime)
|
||||
@@ -276,6 +303,7 @@ public class HoodieTestUtils {
|
||||
public static void createCleanFiles(HoodieTableMetaClient metaClient, String basePath,
|
||||
String commitTime, Configuration configuration)
|
||||
throws IOException {
|
||||
createPendingCleanFiles(metaClient, configuration, commitTime);
|
||||
Path commitFile = new Path(
|
||||
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime));
|
||||
FileSystem fs = FSUtils.getFs(basePath, configuration);
|
||||
|
||||
@@ -68,7 +68,7 @@ public class TestHoodieTableMetaClient extends HoodieCommonTestHarness {
|
||||
assertNotNull(deseralizedMetaClient);
|
||||
HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline();
|
||||
HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
commitTimeline.createInflight(instant);
|
||||
commitTimeline.createNewInstant(instant);
|
||||
commitTimeline.saveAsComplete(instant, Option.of("test-detail".getBytes()));
|
||||
commitTimeline = commitTimeline.reload();
|
||||
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
|
||||
@@ -84,7 +84,7 @@ public class TestHoodieTableMetaClient extends HoodieCommonTestHarness {
|
||||
assertTrue("Should be empty commit timeline", activeCommitTimeline.empty());
|
||||
|
||||
HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
activeTimeline.createInflight(instant);
|
||||
activeTimeline.createNewInstant(instant);
|
||||
activeTimeline.saveAsComplete(instant, Option.of("test-detail".getBytes()));
|
||||
|
||||
// Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached
|
||||
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.common.table;
|
||||
|
||||
import org.apache.hudi.common.model.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class TestTimelineLayout {
|
||||
|
||||
@Test
|
||||
public void testTimelineLayoutFilter() {
|
||||
List<HoodieInstant> rawInstants = Arrays.asList(
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "001"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "001"),
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, "001"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "002"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002"),
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "002"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "003"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "003"),
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "004"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "004"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "005"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "005"),
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "005"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "006"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "006"),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "007"),
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "007"));
|
||||
|
||||
List<HoodieInstant> layout0Instants = TimelineLayout.getLayout(new TimelineLayoutVersion(0))
|
||||
.filterHoodieInstants(rawInstants.stream()).collect(Collectors.toList());
|
||||
Assert.assertEquals(rawInstants, layout0Instants);
|
||||
List<HoodieInstant> layout1Instants = TimelineLayout.getLayout(TimelineLayoutVersion.CURR_LAYOUT_VERSION)
|
||||
.filterHoodieInstants(rawInstants.stream()).collect(Collectors.toList());
|
||||
Assert.assertEquals(7, layout1Instants.size());
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "007")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "006")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "005")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "004")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "002")));
|
||||
Assert.assertTrue(layout1Instants.contains(
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, "001")));
|
||||
}
|
||||
}
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.junit.Before;
|
||||
@@ -53,10 +54,10 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
|
||||
@Test
|
||||
public void testLoadingInstantsFromFiles() throws IOException {
|
||||
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
HoodieInstant instant3 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5");
|
||||
HoodieInstant instant4 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "8");
|
||||
HoodieInstant instant1 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant instant2 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
HoodieInstant instant3 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "5");
|
||||
HoodieInstant instant4 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "8");
|
||||
HoodieInstant instant1Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieInstant instant2Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
HoodieInstant instant3Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5");
|
||||
@@ -65,11 +66,23 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9");
|
||||
|
||||
timeline = new HoodieActiveTimeline(metaClient);
|
||||
timeline.saveAsComplete(instant1, Option.empty());
|
||||
timeline.saveAsComplete(instant2, Option.empty());
|
||||
timeline.saveAsComplete(instant3, Option.empty());
|
||||
timeline.saveAsComplete(instant4, Option.empty());
|
||||
timeline.createInflight(instant5);
|
||||
timeline.createNewInstant(instant1);
|
||||
timeline.transitionRequestedToInflight(instant1, Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant1.getAction(), instant1.getTimestamp()),
|
||||
Option.empty());
|
||||
timeline.createNewInstant(instant2);
|
||||
timeline.transitionRequestedToInflight(instant2, Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant2.getAction(), instant2.getTimestamp()),
|
||||
Option.empty());
|
||||
timeline.createNewInstant(instant3);
|
||||
timeline.transitionRequestedToInflight(instant3, Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant3.getAction(), instant3.getTimestamp()),
|
||||
Option.empty());
|
||||
timeline.createNewInstant(instant4);
|
||||
timeline.transitionRequestedToInflight(instant4, Option.empty());
|
||||
timeline.saveAsComplete(new HoodieInstant(true, instant4.getAction(), instant4.getTimestamp()),
|
||||
Option.empty());
|
||||
timeline.createNewInstant(instant5);
|
||||
timeline = timeline.reload();
|
||||
|
||||
assertEquals("Total instants should be 5", 5, timeline.countInstants());
|
||||
@@ -83,7 +96,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
Stream.of(instant1Complete, instant2Complete, instant3Complete, instant4Complete),
|
||||
timeline.getCommitTimeline().filterCompletedInstants().getInstants());
|
||||
HoodieTestUtils.assertStreamEquals("Check the instants stream", Stream.of(instant5),
|
||||
timeline.getCommitTimeline().filterInflightsExcludingCompaction().getInstants());
|
||||
timeline.getCommitTimeline().filterPendingExcludingCompaction().getInstants());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -107,7 +120,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
|
||||
HoodieTestUtils.assertStreamEquals("", Stream.of("09", "11"), timeline.getCommitTimeline().filterCompletedInstants()
|
||||
.findInstantsAfter("07", 2).getInstants().map(HoodieInstant::getTimestamp));
|
||||
assertFalse(timeline.empty());
|
||||
assertFalse(timeline.getCommitTimeline().filterInflightsExcludingCompaction().empty());
|
||||
assertFalse(timeline.getCommitTimeline().filterPendingExcludingCompaction().empty());
|
||||
assertEquals("", 12, timeline.countInstants());
|
||||
HoodieTimeline activeCommitTimeline = timeline.getCommitTimeline().filterCompletedInstants();
|
||||
assertEquals("", 10, activeCommitTimeline.countInstants());
|
||||
|
||||
@@ -129,9 +129,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
|
||||
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
|
||||
|
||||
commitTimeline.saveAsComplete(instant1, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant2, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant3, Option.empty());
|
||||
saveAsComplete(commitTimeline, instant1, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant2, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant3, Option.empty());
|
||||
|
||||
refreshFsView();
|
||||
|
||||
@@ -262,9 +262,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
|
||||
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
|
||||
|
||||
commitTimeline.saveAsComplete(instant1, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant2, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant3, Option.empty());
|
||||
saveAsComplete(commitTimeline, instant1, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant2, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant3, Option.empty());
|
||||
|
||||
refreshFsView();
|
||||
List<FileSlice> fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
@@ -309,8 +309,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName4).createNewFile();
|
||||
HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4);
|
||||
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
|
||||
commitTimeline.saveAsComplete(deltaInstant4, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant5, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant4, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant5, Option.empty());
|
||||
refreshFsView();
|
||||
|
||||
List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath).collect(Collectors.toList());
|
||||
@@ -415,9 +415,10 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
inflightDeltaInstantTime, 0, TEST_WRITE_TOKEN);
|
||||
new File(basePath + "/" + partitionPath + "/" + inflightLogFileName).createNewFile();
|
||||
// Mark instant as inflight
|
||||
commitTimeline.saveToInflight(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, inflightDeltaInstantTime),
|
||||
Option.empty());
|
||||
commitTimeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION,
|
||||
inflightDeltaInstantTime));
|
||||
commitTimeline.transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION,
|
||||
inflightDeltaInstantTime), Option.empty());
|
||||
refreshFsView();
|
||||
|
||||
List<FileSlice> allRawFileSlices = getAllRawFileSlices(partitionPath).collect(Collectors.toList());
|
||||
@@ -480,18 +481,16 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
assertEquals("Inflight File Slice with log-file check data-file", inflightLogFileName,
|
||||
logFiles.get(0).getFileName());
|
||||
}
|
||||
|
||||
compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime);
|
||||
// Now simulate Compaction completing - Check the view
|
||||
if (!isCompactionInFlight) {
|
||||
// For inflight compaction, we already create a data-file to test concurrent inflight case.
|
||||
// If we skipped creating data file corresponding to compaction commit, create it now
|
||||
new File(basePath + "/" + partitionPath + "/" + compactDataFileName).createNewFile();
|
||||
commitTimeline.createNewInstant(compactionInstant);
|
||||
}
|
||||
if (isCompactionInFlight) {
|
||||
commitTimeline.deleteInflight(compactionInstant);
|
||||
} else {
|
||||
commitTimeline.deleteCompactionRequested(compactionInstant);
|
||||
}
|
||||
compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime);
|
||||
|
||||
commitTimeline.saveAsComplete(compactionInstant, Option.empty());
|
||||
refreshFsView();
|
||||
// populate the cache
|
||||
@@ -566,7 +565,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
// Make this commit safe
|
||||
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
|
||||
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
|
||||
commitTimeline.saveAsComplete(instant1, Option.empty());
|
||||
saveAsComplete(commitTimeline, instant1, Option.empty());
|
||||
refreshFsView();
|
||||
assertEquals("", fileName1, roView.getLatestDataFiles(partitionPath)
|
||||
.filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName());
|
||||
@@ -581,7 +580,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
|
||||
// Make it safe
|
||||
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2);
|
||||
commitTimeline.saveAsComplete(instant2, Option.empty());
|
||||
saveAsComplete(commitTimeline, instant2, Option.empty());
|
||||
refreshFsView();
|
||||
assertEquals("", fileName2, roView.getLatestDataFiles(partitionPath)
|
||||
.filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName());
|
||||
@@ -1043,9 +1042,9 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
|
||||
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
|
||||
|
||||
commitTimeline.saveAsComplete(instant1, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant2, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant3, Option.empty());
|
||||
saveAsComplete(commitTimeline, instant1, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant2, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant3, Option.empty());
|
||||
|
||||
// Now we list all partitions
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(
|
||||
@@ -1090,7 +1089,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(requested);
|
||||
|
||||
// Fake delta-ingestion after compaction-requested
|
||||
String deltaInstantTime4 = "3";
|
||||
String deltaInstantTime4 = "4";
|
||||
String deltaInstantTime5 = "6";
|
||||
List<String> allInstantTimes = Arrays.asList(instantTime1, deltaInstantTime1, deltaInstantTime2,
|
||||
compactionRequestedTime, deltaInstantTime4, deltaInstantTime5);
|
||||
@@ -1107,8 +1106,8 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
|
||||
HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4);
|
||||
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
|
||||
commitTimeline.saveAsComplete(deltaInstant4, Option.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant5, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant4, Option.empty());
|
||||
saveAsComplete(commitTimeline, deltaInstant5, Option.empty());
|
||||
refreshFsView();
|
||||
|
||||
// Test Data Files
|
||||
@@ -1168,6 +1167,17 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
|
||||
Assert.assertTrue(fileIdsInCompaction.contains(fileId));
|
||||
}
|
||||
|
||||
private static void saveAsComplete(HoodieActiveTimeline timeline, HoodieInstant inflight, Option<byte[]> data) {
|
||||
if (inflight.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
timeline.transitionCompactionInflightToComplete(inflight, data);
|
||||
} else {
|
||||
HoodieInstant requested = new HoodieInstant(State.REQUESTED, inflight.getAction(), inflight.getTimestamp());
|
||||
timeline.createNewInstant(requested);
|
||||
timeline.transitionRequestedToInflight(requested, Option.empty());
|
||||
timeline.saveAsComplete(inflight, data);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
|
||||
@@ -158,6 +158,8 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
// Add an empty ingestion
|
||||
String firstEmptyInstantTs = "11";
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
metaClient.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs),
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
@@ -421,9 +423,11 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
new ArrayList<>(), Integer.toString(Integer.parseInt(instant) + 1));
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
HoodieInstant cleanInflightInstant = new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant);
|
||||
metaClient.getActiveTimeline().createNewInstant(cleanInflightInstant);
|
||||
HoodieCleanMetadata cleanMetadata = CleanerUtils
|
||||
.convertCleanMetadata(metaClient, cleanInstant, Option.empty(), cleanStats);
|
||||
metaClient.getActiveTimeline().saveAsComplete(new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant),
|
||||
metaClient.getActiveTimeline().saveAsComplete(cleanInflightInstant,
|
||||
AvroUtils.serializeCleanMetadata(cleanMetadata));
|
||||
}
|
||||
|
||||
@@ -458,10 +462,12 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
metadata.setInstantsToRollback(rollbackInstants);
|
||||
metadata.setStartRestoreTime(rollbackInstant);
|
||||
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant),
|
||||
AvroUtils.serializeRestoreMetadata(metadata));
|
||||
HoodieInstant restoreInstant = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant);
|
||||
metaClient.getActiveTimeline().createNewInstant(restoreInstant);
|
||||
metaClient.getActiveTimeline().saveAsComplete(restoreInstant, AvroUtils.serializeRestoreMetadata(metadata));
|
||||
} else {
|
||||
metaClient.getActiveTimeline().createNewInstant(
|
||||
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant),
|
||||
AvroUtils.serializeRollbackMetadata(rollbackMetadata));
|
||||
@@ -741,13 +747,16 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness {
|
||||
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue()));
|
||||
metaClient.getActiveTimeline()
|
||||
.saveAsComplete(new HoodieInstant(true,
|
||||
deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, instant),
|
||||
HoodieInstant inflightInstant = new HoodieInstant(true,
|
||||
deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, instant);
|
||||
metaClient.getActiveTimeline().createNewInstant(inflightInstant);
|
||||
metaClient.getActiveTimeline().saveAsComplete(inflightInstant,
|
||||
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
/**
|
||||
// Delete pending compaction if present
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant).getFileName()));
|
||||
*/
|
||||
return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
|
||||
@@ -129,8 +129,11 @@ public class CompactionTestUtils {
|
||||
}
|
||||
|
||||
public static void createDeltaCommit(HoodieTableMetaClient metaClient, String instantTime) throws IOException {
|
||||
metaClient.getActiveTimeline().saveAsComplete(new HoodieInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime),
|
||||
Option.empty());
|
||||
HoodieInstant requested = new HoodieInstant(State.REQUESTED, DELTA_COMMIT_ACTION, instantTime);
|
||||
metaClient.getActiveTimeline().createNewInstant(requested);
|
||||
metaClient.getActiveTimeline().transitionRequestedToInflight(requested, Option.empty());
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime), Option.empty());
|
||||
}
|
||||
|
||||
public static void scheduleInflightCompaction(HoodieTableMetaClient metaClient, String instantTime,
|
||||
|
||||
@@ -100,7 +100,7 @@ public class SchemaTestUtil {
|
||||
public static List<IndexedRecord> generateHoodieTestRecords(int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
List<IndexedRecord> records = generateTestRecords(from, limit);
|
||||
String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
|
||||
return records.stream().map(s -> HoodieAvroUtils.rewriteRecord((GenericRecord) s, hoodieFieldsSchema)).map(p -> {
|
||||
p.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, UUID.randomUUID().toString());
|
||||
|
||||
@@ -118,7 +118,7 @@ public class TestDiskBasedMap extends HoodieCommonTestHarness {
|
||||
|
||||
// generate updates from inserts
|
||||
List<IndexedRecord> updatedRecords = SchemaTestUtil.updateHoodieTestRecords(recordKeys,
|
||||
SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewCommitTime());
|
||||
SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewInstantTime());
|
||||
String newCommitTime =
|
||||
((GenericRecord) updatedRecords.get(0)).get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString();
|
||||
|
||||
|
||||
@@ -101,7 +101,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
assert recordKeys.contains(rec.getRecordKey());
|
||||
}
|
||||
List<IndexedRecord> updatedRecords = SchemaTestUtil.updateHoodieTestRecords(recordKeys,
|
||||
SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewCommitTime());
|
||||
SchemaTestUtil.generateHoodieTestRecords(0, 100), HoodieActiveTimeline.createNewInstantTime());
|
||||
|
||||
// update records already inserted
|
||||
SpillableMapTestUtils.upsertRecords(updatedRecords, records);
|
||||
@@ -214,7 +214,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
List<IndexedRecord> recordsToUpdate = new ArrayList<>();
|
||||
recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get());
|
||||
|
||||
String newCommitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String newCommitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
List<String> keysToBeUpdated = new ArrayList<>();
|
||||
keysToBeUpdated.add(key);
|
||||
// Update the commitTime for this record
|
||||
@@ -232,7 +232,7 @@ public class TestExternalSpillableMap extends HoodieCommonTestHarness {
|
||||
recordsToUpdate = new ArrayList<>();
|
||||
recordsToUpdate.add((IndexedRecord) record.getData().getInsertValue(schema).get());
|
||||
|
||||
newCommitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
newCommitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
keysToBeUpdated = new ArrayList<>();
|
||||
keysToBeUpdated.add(key);
|
||||
// Update the commitTime for this record
|
||||
|
||||
@@ -159,7 +159,7 @@ public class InputFormatTestUtil {
|
||||
parquetWriter = new AvroParquetWriter(new Path(dataFile.getAbsolutePath()), schema);
|
||||
try {
|
||||
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, numberOfRecords);
|
||||
String commitTime = HoodieActiveTimeline.createNewCommitTime();
|
||||
String commitTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
Schema hoodieFieldsSchema = HoodieAvroUtils.addMetadataFields(schema);
|
||||
for (IndexedRecord record : records) {
|
||||
GenericRecord p = HoodieAvroUtils.rewriteRecord((GenericRecord) record, hoodieFieldsSchema);
|
||||
|
||||
Reference in New Issue
Block a user