1
0

[HUDI-699] Fix CompactionCommand and add unit test for CompactionCommand (#2325)

This commit is contained in:
hongdd
2021-04-08 15:35:33 +08:00
committed by GitHub
parent 18459d4045
commit ecdbd2517f
11 changed files with 725 additions and 84 deletions

View File

@@ -18,12 +18,14 @@
package org.apache.hudi.common.table.timeline;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
@@ -43,6 +45,7 @@ import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@@ -66,6 +69,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE_PREFIX = "commits";
private static final String ACTION_TYPE_KEY = "actionType";
private static final String ACTION_STATE = "actionState";
private HoodieTableMetaClient metaClient;
private Map<String, byte[]> readCommits = new HashMap<>();
@@ -108,6 +112,22 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
loadInstants(startTs, endTs);
}
public void loadCompactionDetailsInMemory(String compactionInstantTime) {
loadCompactionDetailsInMemory(compactionInstantTime, compactionInstantTime);
}
public void loadCompactionDetailsInMemory(String startTs, String endTs) {
// load compactionPlan
loadInstants(new TimeRangeFilter(startTs, endTs), true, record ->
record.get(ACTION_TYPE_KEY).toString().equals(HoodieTimeline.COMPACTION_ACTION)
&& HoodieInstant.State.INFLIGHT.toString().equals(record.get(ACTION_STATE).toString())
);
}
public void clearInstantDetailsFromMemory(String instantTime) {
this.readCommits.remove(instantTime);
}
public void clearInstantDetailsFromMemory(String startTs, String endTs) {
this.findInstantsInRange(startTs, endTs).getInstants().forEach(instant ->
this.readCommits.remove(instant.getTimestamp()));
@@ -126,11 +146,16 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString();
final String action = record.get(ACTION_TYPE_KEY).toString();
if (loadDetails) {
Option.ofNullable(record.get(getMetadataKey(action))).map(actionData ->
this.readCommits.put(instantTime, actionData.toString().getBytes(StandardCharsets.UTF_8))
);
Option.ofNullable(record.get(getMetadataKey(action))).map(actionData -> {
if (action.equals(HoodieTimeline.COMPACTION_ACTION)) {
this.readCommits.put(instantTime, HoodieAvroUtils.indexedRecordToBytes((IndexedRecord)actionData));
} else {
this.readCommits.put(instantTime, actionData.toString().getBytes(StandardCharsets.UTF_8));
}
return null;
});
}
return new HoodieInstant(false, action, instantTime);
return new HoodieInstant(HoodieInstant.State.valueOf(record.get(ACTION_STATE).toString()), action, instantTime);
}
private String getMetadataKey(String action) {
@@ -145,6 +170,8 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
return "hoodieRollbackMetadata";
case HoodieTimeline.SAVEPOINT_ACTION:
return "hoodieSavePointMetadata";
case HoodieTimeline.COMPACTION_ACTION:
return "hoodieCompactionPlan";
default:
throw new HoodieIOException("Unknown action in metadata " + action);
}
@@ -158,12 +185,18 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
return loadInstants(new TimeRangeFilter(startTs, endTs), true);
}
private List<HoodieInstant> loadInstants(TimeRangeFilter filter, boolean loadInstantDetails) {
return loadInstants(filter, loadInstantDetails, record -> true);
}
/**
* This is method to read selected instants. Do NOT use this directly use one of the helper methods above
* If loadInstantDetails is set to true, this would also update 'readCommits' map with commit details
* If filter is specified, only the filtered instants are loaded
* If commitsFilter is specified, only the filtered records are loaded
*/
private List<HoodieInstant> loadInstants(TimeRangeFilter filter, boolean loadInstantDetails) {
private List<HoodieInstant> loadInstants(TimeRangeFilter filter, boolean loadInstantDetails,
Function<GenericRecord, Boolean> commitsFilter) {
try {
// list all files
FileStatus[] fsStatuses = metaClient.getFs().globStatus(
@@ -187,6 +220,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
List<IndexedRecord> records = blk.getRecords();
// filter blocks in desired time window
Stream<HoodieInstant> instantsInBlkStream = records.stream()
.filter(r -> commitsFilter.apply((GenericRecord) r))
.map(r -> readCommit((GenericRecord) r, loadInstantDetails));
if (filter != null) {
@@ -254,4 +288,13 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
return 0;
}
}
@Override
public HoodieDefaultTimeline getWriteTimeline() {
// filter in-memory instants
Set<String> validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION);
return new HoodieDefaultTimeline(getInstants().filter(i ->
readCommits.keySet().contains(i.getTimestamp()))
.filter(s -> validActions.contains(s.getAction())), details);
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.common.table.timeline;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
@@ -33,12 +34,14 @@ import org.apache.hudi.common.HoodieRollbackStat;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.avro.Schema;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileWriter;
import org.apache.avro.file.FileReader;
import org.apache.avro.file.SeekableByteArrayInput;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.DatumWriter;
import org.apache.avro.specific.SpecificData;
import org.apache.avro.specific.SpecificDatumReader;
import org.apache.avro.specific.SpecificDatumWriter;
import org.apache.avro.specific.SpecificRecordBase;
@@ -176,4 +179,13 @@ public class TimelineMetadataUtils {
ValidationUtils.checkArgument(fileReader.hasNext(), "Could not deserialize metadata of type " + clazz);
return fileReader.next();
}
public static <T extends SpecificRecordBase> T deserializeAvroRecordMetadata(byte[] bytes, Schema schema)
throws IOException {
return deserializeAvroRecordMetadata(HoodieAvroUtils.bytesToAvro(bytes, schema), schema);
}
public static <T extends SpecificRecordBase> T deserializeAvroRecordMetadata(Object object, Schema schema) {
return (T) SpecificData.get().deepCopy(schema, object);
}
}