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
@@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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(
|
||||
|
||||
Reference in New Issue
Block a user