1
0

Removing compaction action type and associated compaction timeline operations, replace with commit action type

This commit is contained in:
Nishith Agarwal
2017-12-05 00:58:53 -08:00
committed by vinoth chandar
parent a1c0d0dbad
commit 44839b88c6
34 changed files with 265 additions and 450 deletions

View File

@@ -71,7 +71,7 @@ public class HoodieReadClient implements Serializable {
// Create a Hoodie table which encapsulated the commits and files visible
this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
this.commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline();
this.commitTimeline = hoodieTable.getCommitTimeline().filterCompletedInstants();
this.index =
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
this.sqlContextOpt = Optional.absent();

View File

@@ -25,7 +25,6 @@ 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.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;
@@ -54,17 +53,6 @@ import com.uber.hoodie.table.HoodieTable;
import com.uber.hoodie.table.UserDefinedBulkInsertPartitioner;
import com.uber.hoodie.table.WorkloadProfile;
import com.uber.hoodie.table.WorkloadStat;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -77,6 +65,18 @@ import org.apache.spark.storage.StorageLevel;
import scala.Option;
import scala.Tuple2;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.text.ParseException;
import java.util.Collections;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
/**
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then perform efficient
* mutations on a HDFS dataset [upsert()]
@@ -605,7 +605,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieTimeline commitTimeline = table.getCommitTimeline();
HoodieTimeline commitTimeline = table.getCommitsTimeline();
HoodieInstant savePoint =
new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
@@ -624,7 +624,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// Make sure the rollback was successful
Optional<HoodieInstant> lastInstant =
activeTimeline.reload().getCommitsAndCompactionsTimeline().filterCompletedInstants()
activeTimeline.reload().getCommitsTimeline().filterCompletedInstants()
.lastInstant();
Preconditions.checkArgument(lastInstant.isPresent());
Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime),
@@ -829,7 +829,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
Optional<HoodieCompactionMetadata> compactionMetadata = table.compact(jsc, compactionCommitTime);
Optional<HoodieCommitMetadata> compactionMetadata = table.compact(jsc, compactionCommitTime);
if (compactionMetadata.isPresent()) {
logger.info("Compacted successfully on commit " + compactionCommitTime);
} else {
@@ -878,7 +878,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
private void rollbackInflightCommits() {
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
HoodieTimeline inflightTimeline = table.getCommitTimeline().filterInflights();
HoodieTimeline inflightTimeline = table.getCommitsTimeline().filterInflights();
List<String> commits = inflightTimeline.getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
Collections.reverse(commits);

View File

@@ -239,7 +239,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
.parallelize(partitions, Math.max(partitions.size(), 1))
.flatMapToPair(partitionPath -> {
java.util.Optional<HoodieInstant> latestCommitTime =
hoodieTable.getCommitTimeline().filterCompletedInstants().lastInstant();
hoodieTable.getCommitsTimeline().filterCompletedInstants().lastInstant();
List<Tuple2<String, HoodieDataFile>> filteredFiles = new ArrayList<>();
if (latestCommitTime.isPresent()) {
filteredFiles =

View File

@@ -26,7 +26,6 @@ import com.uber.hoodie.avro.model.HoodieSavepointMetadata;
import com.uber.hoodie.common.model.ActionType;
import com.uber.hoodie.common.model.HoodieArchivedLogFile;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
@@ -39,12 +38,6 @@ import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
@@ -52,6 +45,13 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Archiver to bound the growth of <action>.commit files
*/
@@ -228,14 +228,6 @@ public class HoodieCommitArchiveLog {
archivedMetaWrapper.setActionType(ActionType.commit.name());
break;
}
case HoodieTimeline.COMPACTION_ACTION: {
com.uber.hoodie.common.model.HoodieCompactionMetadata compactionMetadata = com.uber.hoodie.common.model.HoodieCompactionMetadata
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
archivedMetaWrapper
.setHoodieCompactionMetadata(compactionMetadataConverter(compactionMetadata));
archivedMetaWrapper.setActionType(ActionType.compaction.name());
break;
}
case HoodieTimeline.ROLLBACK_ACTION: {
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
@@ -271,14 +263,4 @@ public class HoodieCommitArchiveLog {
com.uber.hoodie.avro.model.HoodieCommitMetadata.class);
return avroMetaData;
}
private com.uber.hoodie.avro.model.HoodieCompactionMetadata compactionMetadataConverter(
HoodieCompactionMetadata hoodieCompactionMetadata) {
ObjectMapper mapper = new ObjectMapper();
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
com.uber.hoodie.avro.model.HoodieCompactionMetadata avroMetaData = mapper
.convertValue(hoodieCompactionMetadata,
com.uber.hoodie.avro.model.HoodieCompactionMetadata.class);
return avroMetaData;
}
}

View File

@@ -16,17 +16,16 @@
package com.uber.hoodie.io.compact;
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.Serializable;
import java.util.Date;
import java.util.Optional;
import org.apache.spark.api.java.JavaSparkContext;
/**
* A HoodieCompactor runs compaction on a hoodie table
@@ -36,8 +35,8 @@ public interface HoodieCompactor extends Serializable {
/**
* Compact the delta files with the data files
*/
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
HoodieTable hoodieTable, String compactionCommitTime) throws Exception;
HoodieCommitMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
HoodieTable hoodieTable, String compactionCommitTime) throws Exception;
// Helper methods
@@ -45,7 +44,7 @@ public interface HoodieCompactor extends Serializable {
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline();
activeTimeline
.createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime));
.createInflight(new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime));
return commitTime;
}
}

View File

@@ -16,15 +16,13 @@
package com.uber.hoodie.io.compact;
import static java.util.stream.Collectors.toList;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.CompactionWriteStat;
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
import com.uber.hoodie.common.model.HoodieCommitMetadata;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner;
@@ -36,6 +34,13 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FlatMapFunction;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.Collection;
@@ -44,12 +49,8 @@ import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.FlatMapFunction;
import static java.util.stream.Collectors.toList;
/**
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
@@ -63,8 +64,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
private static Logger log = LogManager.getLogger(HoodieRealtimeTableCompactor.class);
@Override
public HoodieCompactionMetadata compact(JavaSparkContext jsc, HoodieWriteConfig config,
HoodieTable hoodieTable, String compactionCommitTime) throws IOException {
public HoodieCommitMetadata compact(JavaSparkContext jsc, HoodieWriteConfig config,
HoodieTable hoodieTable, String compactionCommitTime) throws IOException {
Preconditions.checkArgument(
hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
"HoodieRealtimeTableCompactor can only compact table of type "
@@ -99,20 +100,20 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
}
log.info("After filtering, Compacting " + operations + " files");
List<CompactionWriteStat> updateStatusMap =
List<HoodieWriteStat> updateStatusMap =
jsc.parallelize(operations, operations.size())
.map(s -> executeCompaction(metaClient, config, s, compactionCommitTime))
.flatMap(new FlatMapFunction<List<CompactionWriteStat>, CompactionWriteStat>() {
.flatMap(new FlatMapFunction<List<HoodieWriteStat>, HoodieWriteStat>() {
@Override
public Iterator<CompactionWriteStat> call(
List<CompactionWriteStat> compactionWriteStats)
public Iterator<HoodieWriteStat> call(
List<HoodieWriteStat> hoodieWriteStats)
throws Exception {
return compactionWriteStats.iterator();
return hoodieWriteStats.iterator();
}
}).collect();
HoodieCompactionMetadata metadata = new HoodieCompactionMetadata();
for (CompactionWriteStat stat : updateStatusMap) {
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
for (HoodieWriteStat stat : updateStatusMap) {
metadata.addWriteStat(stat.getPartitionPath(), stat);
}
@@ -128,13 +129,13 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
return metadata;
}
private boolean isCompactionSucceeded(HoodieCompactionMetadata result) {
private boolean isCompactionSucceeded(HoodieCommitMetadata result) {
//TODO figure out a success factor for a compaction
return true;
}
private List<CompactionWriteStat> executeCompaction(HoodieTableMetaClient metaClient,
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
private List<HoodieWriteStat> executeCompaction(HoodieTableMetaClient metaClient,
HoodieWriteConfig config, CompactionOperation operation, String commitTime)
throws IOException {
FileSystem fs = FSUtils.getFs();
Schema readerSchema =
@@ -150,7 +151,6 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
String maxInstantTime = metaClient.getActiveTimeline()
.getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
HoodieTimeline.COMPACTION_ACTION,
HoodieTimeline.DELTA_COMMIT_ACTION))
.filterCompletedInstants().lastInstant().get().getTimestamp();
@@ -170,22 +170,23 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
return StreamSupport.stream(resultIterable.spliterator(), false)
.flatMap(Collection::stream)
.map(WriteStatus::getStat)
.map(s -> CompactionWriteStat.newBuilder().withHoodieWriteStat(s)
.setTotalRecordsToUpdate(scanner.getTotalRecordsToUpdate())
.setTotalLogFiles(scanner.getTotalLogFiles())
.setTotalLogRecords(scanner.getTotalLogRecords())
.onPartition(operation.getPartitionPath()).build())
.map(s -> {
s.setTotalRecordsToBeUpdate(scanner.getTotalRecordsToUpdate());
s.setTotalLogFiles(scanner.getTotalLogFiles());
s.setTotalLogRecords(scanner.getTotalLogRecords());
s.setPartitionPath(operation.getPartitionPath());
return s;})
.collect(toList());
}
public boolean commitCompaction(String commitTime, HoodieTableMetaClient metaClient,
HoodieCompactionMetadata metadata) {
HoodieCommitMetadata metadata) {
log.info("Committing Compaction " + commitTime);
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
try {
activeTimeline.saveAsComplete(
new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime),
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime),
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
} catch (IOException e) {
throw new HoodieCompactionException(

View File

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

View File

@@ -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()

View File

@@ -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.