1
0

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:
Nishith Agarwal
2017-06-20 23:50:23 -07:00
committed by vinoth chandar
parent 616c9a68c3
commit 19c22b231e
17 changed files with 546 additions and 3854 deletions

View File

@@ -53,17 +53,6 @@ import com.uber.hoodie.io.HoodieCommitArchiveLog;
import com.uber.hoodie.metrics.HoodieMetrics;
import com.uber.hoodie.table.HoodieTable;
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.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -81,6 +70,18 @@ import org.apache.spark.util.LongAccumulator;
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()]

View File

@@ -16,33 +16,44 @@
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.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.HoodieCompactionMetadata;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
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.HoodieInstant;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.file.HoodieAppendLog;
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.util.Optional;
import java.util.function.Function;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
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.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;
/**
* 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 {
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
@@ -50,6 +61,7 @@ public class HoodieCommitArchiveLog {
private final Path archiveFilePath;
private final FileSystem fs;
private final HoodieWriteConfig config;
private HoodieLogFormat.Writer writer;
public HoodieCommitArchiveLog(HoodieWriteConfig config, FileSystem fs) {
this.fs = fs;
@@ -58,132 +70,200 @@ public class HoodieCommitArchiveLog {
.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.
*/
public boolean archiveIfRequired() {
List<HoodieInstant> commitsToArchive = getCommitsToArchive().collect(Collectors.toList());
boolean success = true;
if (commitsToArchive.iterator().hasNext()) {
log.info("Archiving commits " + commitsToArchive);
archive(commitsToArchive);
success = deleteInstants(commitsToArchive);
} else {
log.info("No Commits to archive");
try {
List<HoodieInstant> instantsToArchive = getInstantsToArchive().collect(Collectors.toList());
boolean success = true;
if (instantsToArchive.iterator().hasNext()) {
this.writer = openWriter();
log.info("Archiving instants " + instantsToArchive);
archive(instantsToArchive);
success = deleteArchivedInstants(instantsToArchive);
} else {
log.info("No Instants to archive");
}
return success;
} finally {
close();
}
return success & deleteOtherInstants();
}
private boolean deleteOtherInstants() {
// 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() {
private Stream<HoodieInstant> getInstantsToArchive() {
// TODO : rename to max/minInstantsToKeep
int maxCommitsToKeep = config.getMaxCommitsToKeep();
int minCommitsToKeep = config.getMinCommitsToKeep();
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) {
// Actually do the commits
return cleanTimeline.getInstants().limit(cleanTimeline.countInstants() - minCommitsToKeep);
}
return Stream.empty();
}
private Stream<HoodieInstant> getCommitsToArchive() {
// GroupBy each action and limit each action timeline to maxCommitsToKeep
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline().getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION,
HoodieTimeline.ROLLBACK_ACTION));
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
.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());
int maxCommitsToKeep = config.getMaxCommitsToKeep();
int minCommitsToKeep = config.getMinCommitsToKeep();
HoodieTable table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify with logic above to avoid Stream.concats
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.
Optional<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
// Actually do the commits
return commitTimeline.getInstants().filter(s -> {
instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> {
// if no savepoint present, then dont filter
return !(firstSavepoint.isPresent() && HoodieTimeline
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
HoodieTimeline.LESSER_OR_EQUAL));
}).limit(commitTimeline.countInstants() - minCommitsToKeep);
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
}
return Stream.empty();
return instants;
}
private boolean deleteInstants(List<HoodieInstant> commitsToArchive) {
log.info("Deleting instant " + commitsToArchive);
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) {
log.info("Deleting instants " + archivedInstants);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
new HoodieTableMetaClient(fs, config.getBasePath(), true);
boolean success = true;
for (HoodieInstant commitToArchive : commitsToArchive) {
for (HoodieInstant archivedInstant : archivedInstants) {
Path commitFile =
new Path(metaClient.getMetaPath(), commitToArchive.getFileName());
new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
try {
if (fs.exists(commitFile)) {
success &= fs.delete(commitFile, false);
log.info("Archived and deleted instant file " + commitFile);
}
} catch (IOException e) {
throw new HoodieIOException("Failed to delete archived instant " + commitToArchive,
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant,
e);
}
}
return success;
}
private HoodieAppendLog.Writer openWriter() throws IOException {
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()));
}
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
private void archive(List<HoodieInstant> commits) throws HoodieCommitException {
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getCommitsAndCompactionsTimeline().filterCompletedInstants();
HoodieAppendLog.Writer writer = null;
try {
writer = openWriter();
for (HoodieInstant commitTime : commits) {
Text k = new Text(commitTime.getTimestamp());
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);
}
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(fs, config.getBasePath(), true);
HoodieTimeline commitTimeline =
metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
log.info("Wrapper schema " + wrapperSchema.toString());
List<IndexedRecord> records = new ArrayList<>();
for (HoodieInstant hoodieInstant : instants) {
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
}
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() {
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;
}
}

View File

@@ -16,16 +16,17 @@
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.HoodieKey;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
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.HoodieTimeline;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericRecord;
@@ -35,7 +36,12 @@ import org.apache.hadoop.fs.Path;
import java.io.IOException;
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.
@@ -209,9 +215,7 @@ public class HoodieTestDataGenerator {
}
}
public String[] getPartitionPaths() {
public String[] getPartitionPaths() {
return partitionPaths;
}
}

View File

@@ -16,28 +16,31 @@
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.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.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.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.io.SequenceFile;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.fs.Path;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.stream.Collectors;
import static org.junit.Assert.assertEquals;
@@ -66,6 +69,75 @@ public class TestHoodieCommitArchiveLog {
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
public void testArchiveDatasetWithNoArchival() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
@@ -91,58 +163,6 @@ public class TestHoodieCommitArchiveLog {
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
public void testArchiveCommitSafety() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)