1
0

Ensure Cleaner and Archiver do not delete file-slices and workload marked for compaction

This commit is contained in:
Balaji Varadarajan
2018-05-31 14:16:19 -07:00
committed by vinoth chandar
parent 0a0451a765
commit 9b78523d62
10 changed files with 666 additions and 76 deletions

View File

@@ -26,6 +26,7 @@ import com.esotericsoftware.kryo.serializers.JavaSerializer;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats;
import com.uber.hoodie.common.table.HoodieTableConfig;
@@ -36,7 +37,10 @@ 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.HoodieLogBlock;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.CompactionUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import java.io.ByteArrayInputStream;
@@ -61,6 +65,7 @@ import java.util.stream.Stream;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
@@ -113,6 +118,14 @@ public class HoodieTestUtils {
}
}
public static final void createDeltaCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime : commitTimes) {
new File(
basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeDeltaFileName(commitTime))
.createNewFile();
}
}
public static final void createInflightCommitFiles(String basePath, String... commitTimes) throws IOException {
for (String commitTime : commitTimes) {
new File(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeInflightCommitFileName(
@@ -177,6 +190,15 @@ public class HoodieTestUtils {
}
}
public static final void createCompactionRequest(HoodieTableMetaClient metaClient, String instant,
List<Pair<String, FileSlice>> fileSliceList) throws IOException {
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(fileSliceList, Optional.empty(), Optional.empty());
HoodieInstant compactionInstant =
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant);
metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant,
AvroUtils.serializeCompactionPlan(plan));
}
public static final String getDataFilePath(String basePath, String partitionPath, String commitTime, String fileID)
throws IOException {
return basePath + "/" + partitionPath + "/" + FSUtils