Removing compaction action type and associated compaction timeline operations, replace with commit action type
This commit is contained in:
committed by
vinoth chandar
parent
a1c0d0dbad
commit
44839b88c6
@@ -22,7 +22,6 @@ import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieCleanStat;
|
||||
import com.uber.hoodie.common.HoodieRollbackStat;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
@@ -486,7 +485,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc, String commitCompactionTime) {
|
||||
public Optional<HoodieCommitMetadata> compact(JavaSparkContext jsc, String commitCompactionTime) {
|
||||
logger.info("Nothing to compact in COW storage format");
|
||||
return Optional.empty();
|
||||
}
|
||||
@@ -544,7 +543,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
|
||||
throws IOException {
|
||||
String actionType = this.getCompactedCommitActionType();
|
||||
String actionType = this.getCommitActionType();
|
||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
|
||||
List<String> inflights = this.getInflightCommitTimeline().getInstants()
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
|
||||
@@ -21,7 +21,6 @@ import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.HoodieRollbackStat;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
@@ -93,9 +92,9 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc, String compactionCommitTime) {
|
||||
public Optional<HoodieCommitMetadata> compact(JavaSparkContext jsc, String compactionCommitTime) {
|
||||
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||
Optional<HoodieInstant> lastCompaction = getActiveTimeline().getCompactionTimeline()
|
||||
Optional<HoodieInstant> lastCompaction = getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
String deltaCommitsSinceTs = "0";
|
||||
if (lastCompaction.isPresent()) {
|
||||
@@ -130,8 +129,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
Map<String, HoodieInstant> commitsAndCompactions =
|
||||
this.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.COMPACTION_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
||||
.getInstants()
|
||||
.filter(i -> commits.contains(i.getTimestamp()))
|
||||
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
||||
@@ -149,11 +147,10 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
List<HoodieRollbackStat> stats = null;
|
||||
switch (instant.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
try {
|
||||
logger.info("Starting to rollback Commit/Compaction " + instant);
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitTimeline().getInstantDetails(
|
||||
.fromBytes(this.getCommitsTimeline().getInstantDetails(
|
||||
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream()
|
||||
@@ -174,7 +171,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
logger.info("Starting to rollback delta commit " + instant);
|
||||
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitTimeline().getInstantDetails(
|
||||
.fromBytes(this.getCommitsTimeline().getInstantDetails(
|
||||
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream()
|
||||
|
||||
@@ -16,12 +16,11 @@
|
||||
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
|
||||
import com.uber.hoodie.common.HoodieCleanStat;
|
||||
import com.uber.hoodie.common.HoodieRollbackStat;
|
||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
@@ -35,6 +34,12 @@ import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieSavepointException;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Iterator;
|
||||
@@ -42,11 +47,6 @@ import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Abstract implementation of a HoodieTable
|
||||
@@ -116,21 +116,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
* Get the completed (commit + compaction) view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView getCompletedFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCommitTimeline());
|
||||
return new HoodieTableFileSystemView(metaClient, getCommitsTimeline());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) commit timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedCommitTimeline() {
|
||||
return getCommitTimeline().filterCompletedInstants();
|
||||
return getCommitsTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the inflights (no-completed) commit timeline
|
||||
*/
|
||||
public HoodieTimeline getInflightCommitTimeline() {
|
||||
return getCommitTimeline().filterInflights();
|
||||
return getCommitsTimeline().filterInflights();
|
||||
}
|
||||
|
||||
|
||||
@@ -185,38 +185,28 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
/**
|
||||
* Get the commit timeline visible for this table
|
||||
*/
|
||||
public HoodieTimeline getCommitTimeline() {
|
||||
public HoodieTimeline getCommitsTimeline() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return getActiveTimeline().getCommitTimeline();
|
||||
case MERGE_ON_READ:
|
||||
// We need to include the parquet files written out in delta commits
|
||||
// Include commit action to be able to start doing a MOR over a COW dataset - no migration required
|
||||
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||
return getActiveTimeline().getCommitsTimeline();
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) compaction commit timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedCompactionCommitTimeline() {
|
||||
return getCompactionCommitTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the compacted commit timeline visible for this table
|
||||
*/
|
||||
public HoodieTimeline getCompactionCommitTimeline() {
|
||||
public HoodieTimeline getCommitTimeline() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||
case MERGE_ON_READ:
|
||||
// We need to include the parquet files written out in delta commits in tagging
|
||||
return getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION));
|
||||
return getActiveTimeline().getCommitTimeline();
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
@@ -236,20 +226,6 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
"Could not commit on unknown storage type " + metaClient.getTableType());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the action type for a compaction commit
|
||||
*/
|
||||
public String getCompactedCommitActionType() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return HoodieTimeline.COMMIT_ACTION;
|
||||
case MERGE_ON_READ:
|
||||
return HoodieTimeline.COMPACTION_ACTION;
|
||||
}
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Perform the ultimate IO for a given upserted (RDD) partition
|
||||
*/
|
||||
@@ -279,8 +255,8 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
|
||||
* Run Compaction on the table. Compaction arranges the data so that it is optimized for data
|
||||
* access
|
||||
*/
|
||||
public abstract Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc,
|
||||
String commitCompactionTime);
|
||||
public abstract Optional<HoodieCommitMetadata> compact(JavaSparkContext jsc,
|
||||
String commitCompactionTime);
|
||||
|
||||
/**
|
||||
* Clean partition paths according to cleaning policy and returns the number of files cleaned.
|
||||
|
||||
Reference in New Issue
Block a user