1. Use HoodieLogFormat to archive commits and other actions 2. Introduced avro schema for commits and compactions and an avro wrapper schema
This commit is contained in:
committed by
vinoth chandar
parent
616c9a68c3
commit
19c22b231e
@@ -53,17 +53,6 @@ import com.uber.hoodie.io.HoodieCommitArchiveLog;
|
|||||||
import com.uber.hoodie.metrics.HoodieMetrics;
|
import com.uber.hoodie.metrics.HoodieMetrics;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
import com.uber.hoodie.table.WorkloadProfile;
|
import com.uber.hoodie.table.WorkloadProfile;
|
||||||
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.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
@@ -81,6 +70,18 @@ import org.apache.spark.util.LongAccumulator;
|
|||||||
import scala.Option;
|
import scala.Option;
|
||||||
import scala.Tuple2;
|
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
|
* Hoodie Write Client helps you build datasets on HDFS [insert()] and then
|
||||||
* perform efficient mutations on a HDFS dataset [upsert()]
|
* perform efficient mutations on a HDFS dataset [upsert()]
|
||||||
|
|||||||
@@ -16,33 +16,44 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.io;
|
package com.uber.hoodie.io;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
|
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
|
||||||
|
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
||||||
|
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
|
||||||
|
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.HoodieCommitMetadata;
|
||||||
|
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
|
import com.uber.hoodie.common.table.log.HoodieLogFormat;
|
||||||
|
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||||
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
|
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
|
||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
|
import com.uber.hoodie.common.util.AvroUtils;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
import com.uber.hoodie.common.file.HoodieAppendLog;
|
|
||||||
import com.uber.hoodie.exception.HoodieCommitException;
|
import com.uber.hoodie.exception.HoodieCommitException;
|
||||||
|
import com.uber.hoodie.exception.HoodieException;
|
||||||
import com.uber.hoodie.exception.HoodieIOException;
|
import com.uber.hoodie.exception.HoodieIOException;
|
||||||
import com.uber.hoodie.table.HoodieTable;
|
import com.uber.hoodie.table.HoodieTable;
|
||||||
import java.util.Optional;
|
import org.apache.avro.Schema;
|
||||||
import java.util.function.Function;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.apache.hadoop.io.compress.BZip2Codec;
|
|
||||||
import org.apache.log4j.LogManager;
|
import org.apache.log4j.LogManager;
|
||||||
import org.apache.log4j.Logger;
|
import org.apache.log4j.Logger;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Log to hold older historical commits, to bound the growth of .commit files
|
* Archiver to bound the growth of <action>.commit files
|
||||||
*/
|
*/
|
||||||
public class HoodieCommitArchiveLog {
|
public class HoodieCommitArchiveLog {
|
||||||
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
||||||
@@ -50,6 +61,7 @@ public class HoodieCommitArchiveLog {
|
|||||||
private final Path archiveFilePath;
|
private final Path archiveFilePath;
|
||||||
private final FileSystem fs;
|
private final FileSystem fs;
|
||||||
private final HoodieWriteConfig config;
|
private final HoodieWriteConfig config;
|
||||||
|
private HoodieLogFormat.Writer writer;
|
||||||
|
|
||||||
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
|
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
|
||||||
this.fs = fs;
|
this.fs = fs;
|
||||||
@@ -58,132 +70,200 @@ public class HoodieCommitArchiveLog {
|
|||||||
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
|
.getArchiveLogPath(config.getBasePath() + "/" + HoodieTableMetaClient.METAFOLDER_NAME);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private HoodieLogFormat.Writer openWriter() {
|
||||||
|
try {
|
||||||
|
if(this.writer == null) {
|
||||||
|
return HoodieLogFormat.newWriterBuilder()
|
||||||
|
.onParentPath(archiveFilePath.getParent())
|
||||||
|
.withFileId(archiveFilePath.getName())
|
||||||
|
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
|
||||||
|
.withFs(fs)
|
||||||
|
.overBaseCommit("").build();
|
||||||
|
} else {
|
||||||
|
return this.writer;
|
||||||
|
}
|
||||||
|
} catch(InterruptedException | IOException e) {
|
||||||
|
throw new HoodieException("Unable to initialize HoodieLogFormat writer", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void close() {
|
||||||
|
try {
|
||||||
|
if(this.writer != null) {
|
||||||
|
this.writer.close();
|
||||||
|
}
|
||||||
|
} catch(IOException e) {
|
||||||
|
throw new HoodieException("Unable to close HoodieLogFormat writer", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Check if commits need to be archived. If yes, archive commits.
|
* Check if commits need to be archived. If yes, archive commits.
|
||||||
*/
|
*/
|
||||||
public boolean archiveIfRequired() {
|
public boolean archiveIfRequired() {
|
||||||
List<HoodieInstant> commitsToArchive = getCommitsToArchive().collect(Collectors.toList());
|
try {
|
||||||
|
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
|
||||||
boolean success = true;
|
boolean success = true;
|
||||||
if (commitsToArchive.iterator().hasNext()) {
|
if (instantsToArchive.iterator().hasNext()) {
|
||||||
log.info("Archiving commits " + commitsToArchive);
|
this.writer = openWriter();
|
||||||
archive(commitsToArchive);
|
log.info("Archiving instants " + instantsToArchive);
|
||||||
success = deleteInstants(commitsToArchive);
|
archive(instantsToArchive);
|
||||||
|
success = deleteArchivedInstants(instantsToArchive);
|
||||||
} else {
|
} else {
|
||||||
log.info("No Commits to archive");
|
log.info("No Instants to archive");
|
||||||
|
}
|
||||||
|
return success;
|
||||||
|
} finally {
|
||||||
|
close();
|
||||||
}
|
}
|
||||||
return success & deleteOtherInstants();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean deleteOtherInstants() {
|
private Stream<HoodieInstant> getInstantsToArchive() {
|
||||||
// Delete clean and rollback files
|
|
||||||
List<HoodieInstant> toDelete = getInstantsToDelete().collect(Collectors.toList());
|
|
||||||
if(!toDelete.isEmpty()) {
|
|
||||||
log.info("Deleting actions " + toDelete);
|
|
||||||
return deleteInstants(toDelete);
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
private Stream<HoodieInstant> getInstantsToDelete() {
|
|
||||||
|
|
||||||
|
// TODO : rename to max/minInstantsToKeep
|
||||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||||
|
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
||||||
HoodieTimeline cleanTimeline = table.getActiveTimeline().getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
|
|
||||||
if (!cleanTimeline.empty() && cleanTimeline.countInstants() > maxCommitsToKeep) {
|
// GroupBy each action and limit each action timeline to maxCommitsToKeep
|
||||||
// Actually do the commits
|
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline().getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION,
|
||||||
return cleanTimeline.getInstants().limit(cleanTimeline.countInstants() - minCommitsToKeep);
|
HoodieTimeline.ROLLBACK_ACTION));
|
||||||
}
|
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
|
||||||
return Stream.empty();
|
.collect(Collectors.groupingBy(s -> s.getAction()))
|
||||||
|
.entrySet()
|
||||||
|
.stream()
|
||||||
|
.map(i -> {
|
||||||
|
if (i.getValue().size() > maxCommitsToKeep) {
|
||||||
|
return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep);
|
||||||
|
} else {
|
||||||
|
return new ArrayList<HoodieInstant>();
|
||||||
}
|
}
|
||||||
|
})
|
||||||
|
.flatMap(i -> i.stream());
|
||||||
|
|
||||||
private Stream<HoodieInstant> getCommitsToArchive() {
|
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats
|
||||||
|
|
||||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
|
||||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
|
||||||
|
|
||||||
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
|
|
||||||
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
|
HoodieTimeline commitTimeline = table.getCompletedCommitTimeline();
|
||||||
// We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present.
|
// We cannot have any holes in the commit timeline. We cannot archive any commits which are made after the first savepoint present.
|
||||||
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
|
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
|
||||||
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
|
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
|
||||||
// Actually do the commits
|
// Actually do the commits
|
||||||
return commitTimeline.getInstants().filter(s -> {
|
instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> {
|
||||||
// if no savepoint present, then dont filter
|
// if no savepoint present, then dont filter
|
||||||
return !(firstSavepoint.isPresent() && HoodieTimeline
|
return !(firstSavepoint.isPresent() && HoodieTimeline
|
||||||
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
|
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
|
||||||
HoodieTimeline.LESSER_OR_EQUAL));
|
HoodieTimeline.LESSER_OR_EQUAL));
|
||||||
}).limit(commitTimeline.countInstants() - minCommitsToKeep);
|
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
|
||||||
}
|
|
||||||
return Stream.empty();
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private boolean deleteInstants(List<HoodieInstant> commitsToArchive) {
|
return instants;
|
||||||
log.info("Deleting instant " + commitsToArchive);
|
}
|
||||||
|
|
||||||
|
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
|
||||||
|
log.info("Deleting instants " + archivedInstants);
|
||||||
HoodieTableMetaClient metaClient =
|
HoodieTableMetaClient metaClient =
|
||||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||||
|
|
||||||
boolean success = true;
|
boolean success = true;
|
||||||
for (HoodieInstant commitToArchive : commitsToArchive) {
|
for (HoodieInstant archivedInstant : archivedInstants) {
|
||||||
Path commitFile =
|
Path commitFile =
|
||||||
new Path(metaClient.getMetaPath(), commitToArchive.getFileName());
|
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||||
try {
|
try {
|
||||||
if (fs.exists(commitFile)) {
|
if (fs.exists(commitFile)) {
|
||||||
success &= fs.delete(commitFile, false);
|
success &= fs.delete(commitFile, false);
|
||||||
log.info("Archived and deleted instant file " + commitFile);
|
log.info("Archived and deleted instant file " + commitFile);
|
||||||
}
|
}
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new HoodieIOException("Failed to delete archived instant " + commitToArchive,
|
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant,
|
||||||
e);
|
e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return success;
|
return success;
|
||||||
}
|
}
|
||||||
|
|
||||||
private HoodieAppendLog.Writer openWriter() throws IOException {
|
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
|
||||||
log.info("Opening archive file at path: " + archiveFilePath);
|
|
||||||
return HoodieAppendLog
|
|
||||||
.createWriter(fs.getConf(), HoodieAppendLog.Writer.file(archiveFilePath),
|
|
||||||
HoodieAppendLog.Writer.keyClass(Text.class),
|
|
||||||
HoodieAppendLog.Writer.appendIfExists(true),
|
|
||||||
HoodieAppendLog.Writer.valueClass(Text.class), HoodieAppendLog.Writer
|
|
||||||
.compression(HoodieAppendLog.CompressionType.RECORD, new BZip2Codec()));
|
|
||||||
}
|
|
||||||
|
|
||||||
private void archive(List<HoodieInstant> commits) throws HoodieCommitException {
|
try {
|
||||||
HoodieTableMetaClient metaClient =
|
HoodieTableMetaClient metaClient =
|
||||||
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
new HoodieTableMetaClient(fs, config.getBasePath(), true);
|
||||||
HoodieTimeline commitTimeline =
|
HoodieTimeline commitTimeline =
|
||||||
metaClient.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
|
||||||
|
|
||||||
HoodieAppendLog.Writer writer = null;
|
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
|
||||||
try {
|
log.info("Wrapper schema " + wrapperSchema.toString());
|
||||||
writer = openWriter();
|
List<IndexedRecord> records = new ArrayList<>();
|
||||||
for (HoodieInstant commitTime : commits) {
|
for (HoodieInstant hoodieInstant : instants) {
|
||||||
Text k = new Text(commitTime.getTimestamp());
|
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
|
||||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
|
||||||
.fromBytes(commitTimeline.getInstantDetails(commitTime).get());
|
|
||||||
Text v = new Text(commitMetadata.toJsonString());
|
|
||||||
writer.append(k, v);
|
|
||||||
log.info("Wrote " + k);
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieCommitException("Could not archive commits " + commits, e);
|
|
||||||
} finally {
|
|
||||||
if (writer != null) {
|
|
||||||
try {
|
|
||||||
writer.hsync();
|
|
||||||
writer.close();
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new HoodieCommitException(
|
|
||||||
"Could not close the archive commits writer " + commits, e);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, wrapperSchema);
|
||||||
|
this.writer = writer.appendBlock(block);
|
||||||
|
} catch(Exception e) {
|
||||||
|
throw new HoodieCommitException("Failed to archive commits", e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public Path getArchiveFilePath() {
|
public Path getArchiveFilePath() {
|
||||||
return archiveFilePath;
|
return archiveFilePath;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, HoodieInstant hoodieInstant) throws IOException {
|
||||||
|
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
||||||
|
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
||||||
|
switch(hoodieInstant.getAction()) {
|
||||||
|
case HoodieTimeline.CLEAN_ACTION:{
|
||||||
|
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCleanMetadata.class));
|
||||||
|
archivedMetaWrapper.setActionType(ActionType.clean.name());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case HoodieTimeline.COMMIT_ACTION:{
|
||||||
|
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||||
|
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||||
|
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||||
|
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(), HoodieRollbackMetadata.class));
|
||||||
|
archivedMetaWrapper.setActionType(ActionType.rollback.name());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case HoodieTimeline.SAVEPOINT_ACTION:{
|
||||||
|
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class));
|
||||||
|
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
case HoodieTimeline.DELTA_COMMIT_ACTION:{
|
||||||
|
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||||
|
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get());
|
||||||
|
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||||
|
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return archivedMetaWrapper;
|
||||||
|
}
|
||||||
|
|
||||||
|
private com.uber.hoodie.avro.model.HoodieCommitMetadata commitMetadataConverter(HoodieCommitMetadata hoodieCommitMetadata) {
|
||||||
|
ObjectMapper mapper = new ObjectMapper();
|
||||||
|
//Need this to ignore other public get() methods
|
||||||
|
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||||
|
com.uber.hoodie.avro.model.HoodieCommitMetadata avroMetaData =
|
||||||
|
mapper.convertValue(hoodieCommitMetadata, 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,16 +16,17 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.common;
|
package com.uber.hoodie.common;
|
||||||
|
|
||||||
|
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
||||||
|
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
|
||||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||||
import com.uber.hoodie.common.model.HoodieKey;
|
import com.uber.hoodie.common.model.HoodieKey;
|
||||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||||
import com.uber.hoodie.common.model.HoodieRecord;
|
import com.uber.hoodie.common.model.HoodieRecord;
|
||||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
|
import com.uber.hoodie.common.util.AvroUtils;
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||||
|
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.GenericData;
|
import org.apache.avro.generic.GenericData;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
@@ -35,7 +36,12 @@ import org.apache.hadoop.fs.Path;
|
|||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.util.*;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Random;
|
||||||
|
import java.util.UUID;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Class to be used in tests to keep generating test inserts and updates against a corpus.
|
* Class to be used in tests to keep generating test inserts and updates against a corpus.
|
||||||
@@ -209,8 +215,6 @@ public class HoodieTestDataGenerator {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
public String[] getPartitionPaths() {
|
public String[] getPartitionPaths() {
|
||||||
return partitionPaths;
|
return partitionPaths;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -16,28 +16,31 @@
|
|||||||
|
|
||||||
package com.uber.hoodie.io;
|
package com.uber.hoodie.io;
|
||||||
|
|
||||||
import com.google.common.collect.Lists;
|
import com.uber.hoodie.avro.model.HoodieArchivedMetaEntry;
|
||||||
|
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
||||||
|
import com.uber.hoodie.common.model.HoodieArchivedLogFile;
|
||||||
|
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||||
|
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
|
import com.uber.hoodie.common.table.log.HoodieLogFormat;
|
||||||
|
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
|
||||||
import com.uber.hoodie.common.HoodieTestDataGenerator;
|
|
||||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
|
||||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||||
|
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||||
|
import org.apache.avro.generic.GenericRecord;
|
||||||
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.io.SequenceFile;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.io.Text;
|
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.rules.TemporaryFolder;
|
import org.junit.rules.TemporaryFolder;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.SortedMap;
|
|
||||||
import java.util.TreeMap;
|
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
@@ -66,6 +69,75 @@ public class TestHoodieCommitArchiveLog {
|
|||||||
assertTrue(result);
|
assertTrue(result);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testArchiveDatasetWithArchival() throws IOException {
|
||||||
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||||
|
.withCompactionConfig(HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 4).build())
|
||||||
|
.forTable("test-trip-table").build();
|
||||||
|
HoodieTestUtils.init(basePath);
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
||||||
|
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
||||||
|
|
||||||
|
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
||||||
|
HoodieTimeline timeline =
|
||||||
|
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
||||||
|
|
||||||
|
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
||||||
|
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "100");
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "101");
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "102");
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "103");
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "104");
|
||||||
|
HoodieTestUtils.createCleanFiles(basePath, "105");
|
||||||
|
|
||||||
|
//reload the timeline and get all the commmits before archive
|
||||||
|
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
|
||||||
|
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
|
||||||
|
|
||||||
|
assertEquals("Loaded 6 commits and the count should match", 12, timeline.countInstants());
|
||||||
|
|
||||||
|
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
||||||
|
|
||||||
|
assertTrue(archiveLog.archiveIfRequired());
|
||||||
|
|
||||||
|
//reload the timeline and remove the remaining commits
|
||||||
|
timeline = metadata.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
|
||||||
|
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
|
||||||
|
|
||||||
|
//read the file
|
||||||
|
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(),
|
||||||
|
new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.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
|
||||||
|
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();
|
||||||
|
}
|
||||||
|
assertEquals("Total archived records and total read records are the same count", 8, archivedRecordsCount);
|
||||||
|
|
||||||
|
//make sure the archived commits are the same as the (originalcommits - commitsleft)
|
||||||
|
List<String> readCommits = readRecords.stream().map(r -> (GenericRecord)r).map(r -> {
|
||||||
|
return r.get("commitTime").toString();
|
||||||
|
}).collect(Collectors.toList());
|
||||||
|
Collections.sort(readCommits);
|
||||||
|
|
||||||
|
assertEquals(
|
||||||
|
"Read commits map should match the originalCommits - commitsLoadedFromArchival",
|
||||||
|
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
|
||||||
|
readCommits);
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testArchiveDatasetWithNoArchival() throws IOException {
|
public void testArchiveDatasetWithNoArchival() throws IOException {
|
||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
@@ -91,58 +163,6 @@ public class TestHoodieCommitArchiveLog {
|
|||||||
timeline.countInstants());
|
timeline.countInstants());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testArchiveDatasetWithArchival() throws IOException {
|
|
||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
|
||||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
|
||||||
.forTable("test-trip-table").withCompactionConfig(
|
|
||||||
HoodieCompactionConfig.newBuilder().archiveCommitsWith(2, 5).build()).build();
|
|
||||||
HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs, basePath);
|
|
||||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, fs);
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "100");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "101");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "102");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "103");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "104");
|
|
||||||
HoodieTestDataGenerator.createCommitFile(basePath, "105");
|
|
||||||
|
|
||||||
HoodieTimeline timeline =
|
|
||||||
metadata.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
|
|
||||||
|
|
||||||
assertEquals("Loaded 6 commits and the count should match", 6, timeline.countInstants());
|
|
||||||
boolean result = archiveLog.archiveIfRequired();
|
|
||||||
assertTrue(result);
|
|
||||||
timeline =
|
|
||||||
metadata.getActiveTimeline().reload().getCommitsAndCompactionsTimeline().filterCompletedInstants();
|
|
||||||
assertEquals(
|
|
||||||
"Should archive commits when maxCommitsToKeep is 5 and now the commits length should be minCommitsToKeep which is 2",
|
|
||||||
2, timeline.countInstants());
|
|
||||||
assertEquals("Archive should not archive the last 2 commits",
|
|
||||||
Lists.newArrayList("104", "105"),
|
|
||||||
timeline.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList()));
|
|
||||||
|
|
||||||
// Remove all the commits from the original commits, make it ready to be checked against the read map
|
|
||||||
timeline.getInstants().forEach(originalCommits::remove);
|
|
||||||
|
|
||||||
// Read back the commits to make sure
|
|
||||||
SequenceFile.Reader reader = new SequenceFile.Reader(fs.getConf(),
|
|
||||||
SequenceFile.Reader.file(archiveLog.getArchiveFilePath()));
|
|
||||||
Text key = new Text();
|
|
||||||
Text val = new Text();
|
|
||||||
SortedMap<String, HoodieCommitMetadata> readCommits = new TreeMap<>();
|
|
||||||
while (reader.next(key, val)) {
|
|
||||||
HoodieCommitMetadata meta = HoodieCommitMetadata.fromJsonString(val.toString());
|
|
||||||
readCommits.put(key.toString(), meta);
|
|
||||||
}
|
|
||||||
|
|
||||||
assertEquals(
|
|
||||||
"Read commits map should match the originalCommits - commitsLoadedAfterArchival",
|
|
||||||
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
|
|
||||||
new ArrayList<>(readCommits.keySet()));
|
|
||||||
reader.close();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testArchiveCommitSafety() throws IOException {
|
public void testArchiveCommitSafety() throws IOException {
|
||||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||||
|
|||||||
@@ -50,6 +50,16 @@
|
|||||||
<plugin>
|
<plugin>
|
||||||
<groupId>org.apache.avro</groupId>
|
<groupId>org.apache.avro</groupId>
|
||||||
<artifactId>avro-maven-plugin</artifactId>
|
<artifactId>avro-maven-plugin</artifactId>
|
||||||
|
<configuration>
|
||||||
|
<imports>
|
||||||
|
<!-- import avro files -->
|
||||||
|
<import>${basedir}/src/main/avro/HoodieCommitMetadata.avsc</import>
|
||||||
|
<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/HoodieRollbackMetadata.avsc</import>
|
||||||
|
</imports>
|
||||||
|
</configuration>
|
||||||
</plugin>
|
</plugin>
|
||||||
</plugins>
|
</plugins>
|
||||||
</build>
|
</build>
|
||||||
|
|||||||
55
hoodie-common/src/main/avro/HoodieArchivedMetaEntry.avsc
Normal file
55
hoodie-common/src/main/avro/HoodieArchivedMetaEntry.avsc
Normal file
@@ -0,0 +1,55 @@
|
|||||||
|
{
|
||||||
|
"type":"record",
|
||||||
|
"name":"HoodieArchivedMetaEntry",
|
||||||
|
"namespace":"com.uber.hoodie.avro.model",
|
||||||
|
"fields":[
|
||||||
|
{
|
||||||
|
"name":"hoodieCommitMetadata",
|
||||||
|
"type":[
|
||||||
|
"null",
|
||||||
|
"HoodieCommitMetadata"
|
||||||
|
],
|
||||||
|
"default": "null"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"hoodieCleanMetadata",
|
||||||
|
"type":[
|
||||||
|
"null",
|
||||||
|
"HoodieCleanMetadata"
|
||||||
|
],
|
||||||
|
"default": "null"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"hoodieCompactionMetadata",
|
||||||
|
"type":[
|
||||||
|
"null",
|
||||||
|
"HoodieCompactionMetadata"
|
||||||
|
],
|
||||||
|
"default": "null"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"hoodieRollbackMetadata",
|
||||||
|
"type":[
|
||||||
|
"null",
|
||||||
|
"HoodieRollbackMetadata"
|
||||||
|
],
|
||||||
|
"default": "null"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"hoodieSavePointMetadata",
|
||||||
|
"type":[
|
||||||
|
"null",
|
||||||
|
"HoodieSavepointMetadata"
|
||||||
|
],
|
||||||
|
"default": "null"
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"commitTime",
|
||||||
|
"type":["null","string"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"actionType",
|
||||||
|
"type":["null","string"]
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
61
hoodie-common/src/main/avro/HoodieCommitMetadata.avsc
Normal file
61
hoodie-common/src/main/avro/HoodieCommitMetadata.avsc
Normal file
@@ -0,0 +1,61 @@
|
|||||||
|
{
|
||||||
|
"namespace":"com.uber.hoodie.avro.model",
|
||||||
|
"type":"record",
|
||||||
|
"name":"HoodieCommitMetadata",
|
||||||
|
"fields":[
|
||||||
|
{
|
||||||
|
"name":"partitionToWriteStats",
|
||||||
|
"type":["null", {
|
||||||
|
"type":"map",
|
||||||
|
"values":{
|
||||||
|
"type":"array",
|
||||||
|
"items":{
|
||||||
|
"name":"HoodieWriteStat",
|
||||||
|
"type":"record",
|
||||||
|
"fields":[
|
||||||
|
{
|
||||||
|
"name":"fileId",
|
||||||
|
"type":["null","string"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"path",
|
||||||
|
"type":["null","string"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"prevCommit",
|
||||||
|
"type":["null","string"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"numWrites",
|
||||||
|
"type":["null","long"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"numDeletes",
|
||||||
|
"type":["null","long"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"numUpdateWrites",
|
||||||
|
"type":["null","long"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"totalWriteBytes",
|
||||||
|
"type":["null","long"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"totalWriteErrors",
|
||||||
|
"type":["null","long"]
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"extraMetadata",
|
||||||
|
"type":["null", {
|
||||||
|
"type":"map",
|
||||||
|
"values":"string"
|
||||||
|
}]
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
42
hoodie-common/src/main/avro/HoodieCompactionMetadata.avsc
Normal file
42
hoodie-common/src/main/avro/HoodieCompactionMetadata.avsc
Normal file
@@ -0,0 +1,42 @@
|
|||||||
|
{
|
||||||
|
"namespace":"com.uber.hoodie.avro.model",
|
||||||
|
"type":"record",
|
||||||
|
"name":"HoodieCompactionMetadata",
|
||||||
|
"fields":[
|
||||||
|
{
|
||||||
|
"name":"partitionToCompactionWriteStats",
|
||||||
|
"type": ["null", {
|
||||||
|
"type":"map",
|
||||||
|
"values":{
|
||||||
|
"type":"array",
|
||||||
|
"items":{
|
||||||
|
"name":"HoodieCompactionWriteStat",
|
||||||
|
"type":"record",
|
||||||
|
"fields":[
|
||||||
|
{
|
||||||
|
"name":"partitionPath",
|
||||||
|
"type":["null","string"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"totalLogRecords",
|
||||||
|
"type":["null","long"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"totalLogFiles",
|
||||||
|
"type":["null","long"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"totalRecordsToBeUpdate",
|
||||||
|
"type":["null","long"]
|
||||||
|
},
|
||||||
|
{
|
||||||
|
"name":"hoodieWriteStat",
|
||||||
|
"type":["null","HoodieWriteStat"]
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}]
|
||||||
|
}
|
||||||
|
]
|
||||||
|
}
|
||||||
File diff suppressed because it is too large
Load Diff
@@ -0,0 +1,21 @@
|
|||||||
|
/*
|
||||||
|
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||||
|
*
|
||||||
|
* Licensed 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 com.uber.hoodie.common.model;
|
||||||
|
|
||||||
|
public enum ActionType {
|
||||||
|
commit, savepoint, compaction, clean, rollback;
|
||||||
|
}
|
||||||
@@ -0,0 +1,40 @@
|
|||||||
|
/*
|
||||||
|
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||||
|
*
|
||||||
|
* Licensed 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 com.uber.hoodie.common.model;
|
||||||
|
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
|
||||||
|
public class HoodieArchivedLogFile extends HoodieLogFile {
|
||||||
|
|
||||||
|
public static final String ARCHIVE_EXTENSION = ".archive";
|
||||||
|
|
||||||
|
public HoodieArchivedLogFile(FileStatus fileStatus) {
|
||||||
|
super(fileStatus);
|
||||||
|
}
|
||||||
|
|
||||||
|
public HoodieArchivedLogFile(Path logPath) {
|
||||||
|
super(logPath);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
return "HoodieArchivedLogFile {" + super.getPath() + '}';
|
||||||
|
}
|
||||||
|
}
|
||||||
@@ -41,9 +41,9 @@ import java.util.Map;
|
|||||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||||
public class HoodieCommitMetadata implements Serializable {
|
public class HoodieCommitMetadata implements Serializable {
|
||||||
private static volatile Logger log = LogManager.getLogger(HoodieCommitMetadata.class);
|
private static volatile Logger log = LogManager.getLogger(HoodieCommitMetadata.class);
|
||||||
protected HashMap<String, List<HoodieWriteStat>> partitionToWriteStats;
|
protected Map<String, List<HoodieWriteStat>> partitionToWriteStats;
|
||||||
|
|
||||||
private HashMap<String, String> extraMetadataMap;
|
private Map<String, String> extraMetadataMap;
|
||||||
|
|
||||||
public HoodieCommitMetadata() {
|
public HoodieCommitMetadata() {
|
||||||
extraMetadataMap = new HashMap<>();
|
extraMetadataMap = new HashMap<>();
|
||||||
@@ -65,7 +65,9 @@ public class HoodieCommitMetadata implements Serializable {
|
|||||||
return partitionToWriteStats.get(partitionPath);
|
return partitionToWriteStats.get(partitionPath);
|
||||||
}
|
}
|
||||||
|
|
||||||
public HashMap<String, List<HoodieWriteStat>> getPartitionToWriteStats() {
|
public Map<String, String> getExtraMetadata() { return extraMetadataMap; }
|
||||||
|
|
||||||
|
public Map<String, List<HoodieWriteStat>> getPartitionToWriteStats() {
|
||||||
return partitionToWriteStats;
|
return partitionToWriteStats;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|||||||
@@ -83,11 +83,12 @@ public class HoodieLogFile {
|
|||||||
public HoodieLogFile rollOver(FileSystem fs) throws IOException {
|
public HoodieLogFile rollOver(FileSystem fs) throws IOException {
|
||||||
String fileId = getFileId();
|
String fileId = getFileId();
|
||||||
String baseCommitTime = getBaseCommitTime();
|
String baseCommitTime = getBaseCommitTime();
|
||||||
|
String extension = "." + FSUtils.getFileExtensionFromLog(path);
|
||||||
int newVersion = FSUtils
|
int newVersion = FSUtils
|
||||||
.computeNextLogVersion(fs, path.getParent(), fileId,
|
.computeNextLogVersion(fs, path.getParent(), fileId,
|
||||||
DELTA_EXTENSION, baseCommitTime);
|
extension, baseCommitTime);
|
||||||
return new HoodieLogFile(new Path(path.getParent(),
|
return new HoodieLogFile(new Path(path.getParent(),
|
||||||
FSUtils.makeLogFileName(fileId, DELTA_EXTENSION, baseCommitTime, newVersion)));
|
FSUtils.makeLogFileName(fileId, extension, baseCommitTime, newVersion)));
|
||||||
}
|
}
|
||||||
|
|
||||||
public static Comparator<HoodieLogFile> getLogVersionComparator() {
|
public static Comparator<HoodieLogFile> getLogVersionComparator() {
|
||||||
|
|||||||
@@ -129,6 +129,15 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
|||||||
return getTimelineOfActions(Sets.newHashSet(COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION));
|
return getTimelineOfActions(Sets.newHashSet(COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Get all instants (commits, delta commits, compactions, clean, savepoint, rollback) that result in actions, in the active timeline
|
||||||
|
**
|
||||||
|
* @return
|
||||||
|
*/
|
||||||
|
public HoodieTimeline getAllCommitsTimeline() {
|
||||||
|
return getTimelineOfActions(Sets.newHashSet(COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION));
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get only pure commits (inflight and completed) in the active timeline
|
* Get only pure commits (inflight and completed) in the active timeline
|
||||||
*
|
*
|
||||||
|
|||||||
@@ -45,7 +45,7 @@ import java.util.stream.Collectors;
|
|||||||
* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
|
* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
|
||||||
*/
|
*/
|
||||||
public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||||
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits.archived";
|
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits";
|
||||||
private transient FileSystem fs;
|
private transient FileSystem fs;
|
||||||
private String metaPath;
|
private String metaPath;
|
||||||
private Map<String, byte[]> readCommits = new HashMap<>();
|
private Map<String, byte[]> readCommits = new HashMap<>();
|
||||||
|
|||||||
@@ -22,17 +22,21 @@ import com.esotericsoftware.kryo.Kryo;
|
|||||||
import com.esotericsoftware.kryo.io.Input;
|
import com.esotericsoftware.kryo.io.Input;
|
||||||
import com.esotericsoftware.kryo.io.Output;
|
import com.esotericsoftware.kryo.io.Output;
|
||||||
import com.esotericsoftware.kryo.serializers.JavaSerializer;
|
import com.esotericsoftware.kryo.serializers.JavaSerializer;
|
||||||
|
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
||||||
|
import com.uber.hoodie.common.HoodieCleanStat;
|
||||||
import com.uber.hoodie.common.table.HoodieTableConfig;
|
import com.uber.hoodie.common.table.HoodieTableConfig;
|
||||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||||
import com.uber.hoodie.common.table.log.HoodieLogFormat;
|
import com.uber.hoodie.common.table.log.HoodieLogFormat;
|
||||||
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
|
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
|
||||||
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||||
|
import com.uber.hoodie.common.util.AvroUtils;
|
||||||
import com.uber.hoodie.common.util.FSUtils;
|
import com.uber.hoodie.common.util.FSUtils;
|
||||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||||
import org.apache.avro.Schema;
|
import org.apache.avro.Schema;
|
||||||
import org.apache.avro.generic.GenericRecord;
|
import org.apache.avro.generic.GenericRecord;
|
||||||
import org.apache.avro.generic.IndexedRecord;
|
import org.apache.avro.generic.IndexedRecord;
|
||||||
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||||
@@ -46,11 +50,15 @@ import java.io.File;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.Serializable;
|
import java.io.Serializable;
|
||||||
import java.text.SimpleDateFormat;
|
import java.text.SimpleDateFormat;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Date;
|
import java.util.Date;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
|
import java.util.Random;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
@@ -63,6 +71,8 @@ public class HoodieTestUtils {
|
|||||||
public static final String TEST_EXTENSION = ".test";
|
public static final String TEST_EXTENSION = ".test";
|
||||||
public static final String RAW_TRIPS_TEST_NAME = "raw_trips";
|
public static final String RAW_TRIPS_TEST_NAME = "raw_trips";
|
||||||
public static final int DEFAULT_TASK_PARTITIONID = 1;
|
public static final int DEFAULT_TASK_PARTITIONID = 1;
|
||||||
|
public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"};
|
||||||
|
private static Random rand = new Random(46474747);
|
||||||
|
|
||||||
public static void resetFS() {
|
public static void resetFS() {
|
||||||
HoodieTestUtils.fs = FSUtils.getFs();
|
HoodieTestUtils.fs = FSUtils.getFs();
|
||||||
@@ -139,6 +149,26 @@ public class HoodieTestUtils {
|
|||||||
return instant + TEST_EXTENSION + HoodieTimeline.INFLIGHT_EXTENSION;
|
return instant + TEST_EXTENSION + HoodieTimeline.INFLIGHT_EXTENSION;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public static void createCleanFiles(String basePath, String commitTime) throws IOException {
|
||||||
|
Path commitFile =
|
||||||
|
new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCleanerFileName(commitTime));
|
||||||
|
FileSystem fs = FSUtils.getFs();
|
||||||
|
FSDataOutputStream os = fs.create(commitFile, true);
|
||||||
|
try {
|
||||||
|
HoodieCleanStat cleanStats = new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS,
|
||||||
|
DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)],
|
||||||
|
new ArrayList<>(), new ArrayList<>(),
|
||||||
|
new ArrayList<>(), commitTime);
|
||||||
|
// Create the clean metadata
|
||||||
|
HoodieCleanMetadata cleanMetadata =
|
||||||
|
AvroUtils.convertCleanMetadata(commitTime, Optional.of(0L), Arrays.asList(cleanStats));
|
||||||
|
// Write empty clean metadata
|
||||||
|
os.write(AvroUtils.serializeCleanMetadata(cleanMetadata).get());
|
||||||
|
} finally {
|
||||||
|
os.close();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public static String makeTestFileName(String instant) {
|
public static String makeTestFileName(String instant) {
|
||||||
return instant + TEST_EXTENSION;
|
return instant + TEST_EXTENSION;
|
||||||
}
|
}
|
||||||
|
|||||||
@@ -47,6 +47,7 @@ import java.net.URISyntaxException;
|
|||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.UUID;
|
import java.util.UUID;
|
||||||
@@ -213,7 +214,7 @@ public class TestUtil {
|
|||||||
}
|
}
|
||||||
|
|
||||||
private static HoodieCommitMetadata createLogFiles(
|
private static HoodieCommitMetadata createLogFiles(
|
||||||
HashMap<String, List<HoodieWriteStat>> partitionWriteStats, boolean isLogSchemaSimple)
|
Map<String, List<HoodieWriteStat>> partitionWriteStats, boolean isLogSchemaSimple)
|
||||||
throws InterruptedException, IOException, URISyntaxException {
|
throws InterruptedException, IOException, URISyntaxException {
|
||||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||||
for (Entry<String, List<HoodieWriteStat>> wEntry : partitionWriteStats.entrySet()) {
|
for (Entry<String, List<HoodieWriteStat>> wEntry : partitionWriteStats.entrySet()) {
|
||||||
|
|||||||
Reference in New Issue
Block a user