(1) Define CompactionWorkload in avro to allow storing them in instant files.
(2) Split APIs in HoodieRealtimeCompactor to separate generating compaction workload from running compaction
This commit is contained in:
committed by
vinoth chandar
parent
6d01ae8ca0
commit
1b61f04e05
@@ -20,19 +20,24 @@ import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import com.beust.jcommander.internal.Lists;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionOperation;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.config.HoodieCompactionConfig;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.io.compact.CompactionOperation;
|
||||
import com.uber.hoodie.io.compact.strategy.BoundedIOCompactionStrategy;
|
||||
import com.uber.hoodie.io.compact.strategy.DayBasedCompactionStrategy;
|
||||
import com.uber.hoodie.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
|
||||
import com.uber.hoodie.io.compact.strategy.UnBoundedCompactionStrategy;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Random;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
@@ -51,8 +56,8 @@ public class TestHoodieCompactionStrategy {
|
||||
UnBoundedCompactionStrategy strategy = new UnBoundedCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).build()).build();
|
||||
List<CompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
|
||||
List<CompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations);
|
||||
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
|
||||
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
|
||||
assertEquals("UnBounded should not re-order or filter", operations, returned);
|
||||
}
|
||||
|
||||
@@ -67,14 +72,14 @@ public class TestHoodieCompactionStrategy {
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build())
|
||||
.build();
|
||||
List<CompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
|
||||
List<CompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations);
|
||||
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
|
||||
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
|
||||
|
||||
assertTrue("BoundedIOCompaction should have resulted in fewer compactions", returned.size() < operations.size());
|
||||
assertEquals("BoundedIOCompaction should have resulted in 2 compactions being chosen", 2, returned.size());
|
||||
// Total size of all the log files
|
||||
Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB))
|
||||
.map(s -> (Long) s).reduce((size1, size2) -> size1 + size2).orElse(0L);
|
||||
.map(s -> s.longValue()).reduce((size1, size2) -> size1 + size2).orElse(0L);
|
||||
assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 610,
|
||||
(long) returnedSize);
|
||||
}
|
||||
@@ -90,15 +95,15 @@ public class TestHoodieCompactionStrategy {
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build())
|
||||
.build();
|
||||
List<CompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
|
||||
List<CompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations);
|
||||
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
|
||||
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
|
||||
|
||||
assertTrue("LogFileSizeBasedCompactionStrategy should have resulted in fewer compactions",
|
||||
returned.size() < operations.size());
|
||||
assertEquals("LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction", 1, returned.size());
|
||||
// Total size of all the log files
|
||||
Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB))
|
||||
.map(s -> (Long) s).reduce((size1, size2) -> size1 + size2).orElse(0L);
|
||||
.map(s -> s.longValue()).reduce((size1, size2) -> size1 + size2).orElse(0L);
|
||||
assertEquals("Should chose the first 2 compactions which should result in a total IO of 690 MB", 1204,
|
||||
(long) returnedSize);
|
||||
}
|
||||
@@ -110,25 +115,20 @@ public class TestHoodieCompactionStrategy {
|
||||
sizesMap.put(110 * MB, Lists.newArrayList());
|
||||
sizesMap.put(100 * MB, Lists.newArrayList(MB));
|
||||
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
|
||||
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
|
||||
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
|
||||
sizesMap.put(90 * MB, Lists.newArrayList(1024 * MB));
|
||||
|
||||
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder().put(120 * MB, partitionPaths[2])
|
||||
.put(110 * MB, partitionPaths[2])
|
||||
.put(100 * MB, partitionPaths[1])
|
||||
.put(90 * MB, partitionPaths[0])
|
||||
.build();
|
||||
|
||||
DayBasedCompactionStrategy strategy = new DayBasedCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy)
|
||||
.withTargetPartitionsPerDayBasedCompaction(1)
|
||||
.build()).build();
|
||||
|
||||
List<CompactionOperation> operations = Lists.newArrayList(sizesMap.size());
|
||||
int partitionPathIndex = 0;
|
||||
for (Map.Entry<Long, List<Long>> entry : sizesMap.entrySet()) {
|
||||
operations.add(new CompactionOperation(Optional.of(TestHoodieDataFile.newDataFile(entry.getKey())),
|
||||
partitionPaths[(partitionPathIndex % (partitionPaths.length - 1))],
|
||||
entry.getValue().stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), writeConfig));
|
||||
partitionPathIndex++;
|
||||
}
|
||||
List<CompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations);
|
||||
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap, keyToPartitionMap);
|
||||
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
|
||||
|
||||
assertTrue("DayBasedCompactionStrategy should have resulted in fewer compactions",
|
||||
returned.size() < operations.size());
|
||||
@@ -141,13 +141,28 @@ public class TestHoodieCompactionStrategy {
|
||||
assertTrue("DayBasedCompactionStrategy should sort partitions in descending order", comparision >= 0);
|
||||
}
|
||||
|
||||
private List<CompactionOperation> createCompactionOperations(HoodieWriteConfig config,
|
||||
private List<HoodieCompactionOperation> createCompactionOperations(HoodieWriteConfig config,
|
||||
Map<Long, List<Long>> sizesMap) {
|
||||
List<CompactionOperation> operations = Lists.newArrayList(sizesMap.size());
|
||||
Map<Long, String> keyToParitionMap = sizesMap.entrySet().stream().map(e ->
|
||||
Pair.of(e.getKey(), partitionPaths[new Random().nextInt(partitionPaths.length - 1)]))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
return createCompactionOperations(config, sizesMap, keyToParitionMap);
|
||||
}
|
||||
|
||||
private List<HoodieCompactionOperation> createCompactionOperations(HoodieWriteConfig config,
|
||||
Map<Long, List<Long>> sizesMap, Map<Long, String> keyToPartitionMap) {
|
||||
List<HoodieCompactionOperation> operations = Lists.newArrayList(sizesMap.size());
|
||||
|
||||
sizesMap.forEach((k, v) -> {
|
||||
operations.add(new CompactionOperation(Optional.of(TestHoodieDataFile.newDataFile(k)),
|
||||
partitionPaths[new Random().nextInt(partitionPaths.length - 1)],
|
||||
v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList()), config));
|
||||
HoodieDataFile df = TestHoodieDataFile.newDataFile(k);
|
||||
String partitionPath = keyToPartitionMap.get(k);
|
||||
List<HoodieLogFile> logFiles = v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList());
|
||||
operations.add(new HoodieCompactionOperation(df.getCommitTime(),
|
||||
logFiles.stream().map(s -> s.getPath().toString()).collect(Collectors.toList()),
|
||||
df.getPath(),
|
||||
df.getFileId(),
|
||||
partitionPath,
|
||||
config.getCompactionStrategy().captureMetrics(config, Optional.of(df), partitionPath, logFiles)));
|
||||
});
|
||||
return operations;
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user