[HUDI-571] Add 'commits show archived' command to CLI
This commit is contained in:
@@ -19,24 +19,18 @@
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import org.apache.hudi.HoodieClientTestHarness;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.common.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTestUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import com.google.common.collect.Sets;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.After;
|
||||
@@ -44,7 +38,8 @@ import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
@@ -197,35 +192,18 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105")));
|
||||
|
||||
// read the file
|
||||
Reader reader =
|
||||
HoodieLogFormat.newReader(dfs, new HoodieLogFile(new Path(basePath + "/.hoodie/.commits_.archive.1_1-0-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
|
||||
int numBlocks = 0;
|
||||
while (reader.hasNext()) {
|
||||
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
|
||||
List<IndexedRecord> records = blk.getRecords();
|
||||
readRecords.addAll(records);
|
||||
archivedRecordsCount += records.size();
|
||||
numBlocks++;
|
||||
}
|
||||
System.out.println("Read Records :" + readRecords.stream().map(r -> (GenericRecord) r)
|
||||
.map(r -> r.get("actionType") + "_" + r.get("actionState") + "_" + r.get("commitTime")).collect(Collectors.toList()));
|
||||
assertEquals("Total archived records and total read records are the same count", 24, archivedRecordsCount);
|
||||
assertTrue("Average Archived records per block is greater than 1", archivedRecordsCount / numBlocks > 1);
|
||||
// make sure the archived commits are the same as the (originalcommits - commitsleft)
|
||||
Set<String> readCommits = readRecords.stream().map(r -> (GenericRecord) r).map(r -> {
|
||||
return r.get("commitTime").toString();
|
||||
}).collect(Collectors.toSet());
|
||||
HoodieArchivedTimeline archivedTimeline = new HoodieArchivedTimeline(metaClient);
|
||||
assertEquals("Total archived records and total read records are the same count",
|
||||
24, archivedTimeline.countInstants());
|
||||
|
||||
//make sure the archived commits are the same as the (originalcommits - commitsleft)
|
||||
Set<String> readCommits =
|
||||
archivedTimeline.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
|
||||
assertEquals("Read commits map should match the originalCommits - commitsLoadedFromArchival",
|
||||
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), readCommits);
|
||||
originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), readCommits);
|
||||
|
||||
// verify in-flight instants after archive
|
||||
verifyInflightInstants(metaClient, 2);
|
||||
reader.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -397,6 +375,37 @@ public class TestHoodieCommitArchiveLog extends HoodieClientTestHarness {
|
||||
timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void checkArchiveCommitTimeline() throws IOException, InterruptedException {
|
||||
HoodieWriteConfig cfg =
|
||||
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2).forTable("test-trip-table")
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieCommitArchiveLog archiveLog = new HoodieCommitArchiveLog(cfg, metaClient);
|
||||
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "1", dfs.getConf());
|
||||
HoodieInstant instant1 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "2", dfs.getConf());
|
||||
HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "3", dfs.getConf());
|
||||
HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
|
||||
//add 2 more instants to pass filter criteria set in compaction config above
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "4", dfs.getConf());
|
||||
HoodieInstant instant4 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "4");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "5", dfs.getConf());
|
||||
HoodieInstant instant5 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5");
|
||||
|
||||
boolean result = archiveLog.archiveIfRequired(jsc);
|
||||
assertTrue(result);
|
||||
|
||||
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
|
||||
List<HoodieInstant> archivedInstants = Arrays.asList(instant1, instant2, instant3);
|
||||
assertEquals(new HashSet(archivedInstants), archivedTimeline.getInstants().collect(Collectors.toSet()));
|
||||
}
|
||||
|
||||
private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) {
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().reload()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterInflights();
|
||||
|
||||
Reference in New Issue
Block a user