Compaction validate, unschedule and repair
This commit is contained in:
committed by
vinoth chandar
parent
d904fe69ca
commit
07324e7a20
@@ -33,12 +33,16 @@ import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Helper class to generate compaction plan from FileGroup/FileSlice abstraction
|
||||
*/
|
||||
public class CompactionUtils {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(CompactionUtils.class);
|
||||
|
||||
/**
|
||||
* Generate compaction operation from file-slice
|
||||
*
|
||||
@@ -47,7 +51,7 @@ public class CompactionUtils {
|
||||
* @param metricsCaptureFunction Metrics Capture function
|
||||
* @return Compaction Operation
|
||||
*/
|
||||
public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, FileSlice fileSlice,
|
||||
public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, FileSlice fileSlice,
|
||||
Optional<Function<Pair<String, FileSlice>, Map<String, Double>>> metricsCaptureFunction) {
|
||||
HoodieCompactionOperation.Builder builder = HoodieCompactionOperation.newBuilder();
|
||||
builder.setPartitionPath(partitionPath);
|
||||
@@ -114,16 +118,21 @@ public class CompactionUtils {
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
|
||||
return pendingCompactionInstants.stream().map(instant -> {
|
||||
try {
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
metaClient.getActiveTimeline().getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get());
|
||||
return Pair.of(instant, compactionPlan);
|
||||
return Pair.of(instant, getCompactionPlan(metaClient, instant.getTimestamp()));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient,
|
||||
String compactionInstant) throws IOException {
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
metaClient.getActiveTimeline().getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
|
||||
return compactionPlan;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get all file-ids with pending Compaction operations and their target compaction instant time
|
||||
*
|
||||
|
||||
@@ -0,0 +1,207 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed under the Apache License, Version 2.0 (the "License");
|
||||
* you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_PARTITION_PATHS;
|
||||
import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION;
|
||||
import static com.uber.hoodie.common.table.HoodieTimeline.DELTA_COMMIT_ACTION;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionOperation;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
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.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Assert;
|
||||
|
||||
public class CompactionTestUtils {
|
||||
|
||||
public static Map<String, Pair<String, HoodieCompactionOperation>> setupAndValidateCompactionOperations(
|
||||
HoodieTableMetaClient metaClient, boolean inflight,
|
||||
int numEntriesInPlan1, int numEntriesInPlan2,
|
||||
int numEntriesInPlan3, int numEntriesInPlan4) throws IOException {
|
||||
HoodieCompactionPlan plan1 = createCompactionPlan(metaClient, "000", "001", numEntriesInPlan1, true, true);
|
||||
HoodieCompactionPlan plan2 = createCompactionPlan(metaClient, "002", "003", numEntriesInPlan2, false, true);
|
||||
HoodieCompactionPlan plan3 = createCompactionPlan(metaClient, "004", "005", numEntriesInPlan3, true, false);
|
||||
HoodieCompactionPlan plan4 = createCompactionPlan(metaClient, "006", "007", numEntriesInPlan4, false, false);
|
||||
|
||||
if (inflight) {
|
||||
scheduleInflightCompaction(metaClient, "001", plan1);
|
||||
scheduleInflightCompaction(metaClient, "003", plan2);
|
||||
scheduleInflightCompaction(metaClient, "005", plan3);
|
||||
scheduleInflightCompaction(metaClient, "007", plan4);
|
||||
} else {
|
||||
scheduleCompaction(metaClient, "001", plan1);
|
||||
scheduleCompaction(metaClient, "003", plan2);
|
||||
scheduleCompaction(metaClient, "005", plan3);
|
||||
scheduleCompaction(metaClient, "007", plan4);
|
||||
}
|
||||
|
||||
createDeltaCommit(metaClient, "000");
|
||||
createDeltaCommit(metaClient, "002");
|
||||
createDeltaCommit(metaClient, "004");
|
||||
createDeltaCommit(metaClient, "006");
|
||||
|
||||
Map<String, String> baseInstantsToCompaction =
|
||||
new ImmutableMap.Builder<String, String>().put("000", "001").put("002", "003")
|
||||
.put("004", "005").put("006", "007").build();
|
||||
List<Integer> expectedNumEntries =
|
||||
Arrays.asList(numEntriesInPlan1, numEntriesInPlan2, numEntriesInPlan3, numEntriesInPlan4);
|
||||
List<HoodieCompactionPlan> plans = new ImmutableList.Builder<HoodieCompactionPlan>()
|
||||
.add(plan1, plan2, plan3, plan4).build();
|
||||
IntStream.range(0, 4).boxed().forEach(idx -> {
|
||||
if (expectedNumEntries.get(idx) > 0) {
|
||||
Assert.assertEquals("check if plan " + idx + " has exp entries",
|
||||
expectedNumEntries.get(idx).longValue(), plans.get(idx).getOperations().size());
|
||||
} else {
|
||||
Assert.assertNull("Plan " + idx + " has null ops", plans.get(idx).getOperations());
|
||||
}
|
||||
});
|
||||
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath(), true);
|
||||
Map<String, Pair<String, HoodieCompactionOperation>> pendingCompactionMap =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
|
||||
Map<String, Pair<String, HoodieCompactionOperation>> expPendingCompactionMap =
|
||||
generateExpectedCompactionOperations(Arrays.asList(plan1, plan2, plan3, plan4), baseInstantsToCompaction);
|
||||
|
||||
// Ensure Compaction operations are fine.
|
||||
Assert.assertEquals(expPendingCompactionMap, pendingCompactionMap);
|
||||
return expPendingCompactionMap;
|
||||
}
|
||||
|
||||
public static Map<String, Pair<String, HoodieCompactionOperation>> generateExpectedCompactionOperations(
|
||||
List<HoodieCompactionPlan> plans, Map<String, String> baseInstantsToCompaction) {
|
||||
return plans.stream()
|
||||
.flatMap(plan -> {
|
||||
if (plan.getOperations() != null) {
|
||||
return plan.getOperations().stream().map(op -> Pair.of(op.getFileId(),
|
||||
Pair.of(baseInstantsToCompaction.get(op.getBaseInstantTime()), op)));
|
||||
}
|
||||
return Stream.empty();
|
||||
}).collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
}
|
||||
|
||||
public static void scheduleCompaction(HoodieTableMetaClient metaClient,
|
||||
String instantTime, HoodieCompactionPlan compactionPlan) throws IOException {
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime),
|
||||
AvroUtils.serializeCompactionPlan(compactionPlan));
|
||||
}
|
||||
|
||||
public static void createDeltaCommit(HoodieTableMetaClient metaClient, String instantTime) throws IOException {
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime), Optional.empty());
|
||||
}
|
||||
|
||||
public static void scheduleInflightCompaction(HoodieTableMetaClient metaClient, String instantTime,
|
||||
HoodieCompactionPlan compactionPlan) throws IOException {
|
||||
scheduleCompaction(metaClient, instantTime, compactionPlan);
|
||||
metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime));
|
||||
}
|
||||
|
||||
public static HoodieCompactionPlan createCompactionPlan(HoodieTableMetaClient metaClient, String instantId,
|
||||
String compactionInstantId, int numFileIds, boolean createDataFile,
|
||||
boolean deltaCommitsAfterCompactionRequests) {
|
||||
List<HoodieCompactionOperation> ops = IntStream.range(0, numFileIds).boxed().map(idx -> {
|
||||
try {
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
if (createDataFile) {
|
||||
HoodieTestUtils.createDataFile(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId);
|
||||
}
|
||||
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
|
||||
instantId, fileId, Optional.of(1));
|
||||
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
|
||||
instantId, fileId, Optional.of(2));
|
||||
FileSlice slice = new FileSlice(instantId, fileId);
|
||||
if (createDataFile) {
|
||||
slice.setDataFile(new TestHoodieDataFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0]
|
||||
+ "/" + FSUtils.makeDataFileName(instantId, 1, fileId)));
|
||||
}
|
||||
String logFilePath1 = HoodieTestUtils
|
||||
.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId,
|
||||
Optional.of(1));
|
||||
String logFilePath2 = HoodieTestUtils
|
||||
.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId,
|
||||
Optional.of(2));
|
||||
slice.addLogFile(new HoodieLogFile(new Path(logFilePath1)));
|
||||
slice.addLogFile(new HoodieLogFile(new Path(logFilePath2)));
|
||||
HoodieCompactionOperation op =
|
||||
CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], slice, Optional.empty());
|
||||
if (deltaCommitsAfterCompactionRequests) {
|
||||
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
|
||||
compactionInstantId, fileId, Optional.of(1));
|
||||
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
|
||||
compactionInstantId, fileId, Optional.of(2));
|
||||
}
|
||||
return op;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
return new HoodieCompactionPlan(ops.isEmpty() ? null : ops, new HashMap<>());
|
||||
}
|
||||
|
||||
public static class TestHoodieDataFile extends HoodieDataFile {
|
||||
|
||||
private final String path;
|
||||
|
||||
public TestHoodieDataFile(String path) {
|
||||
super(null);
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getFileId() {
|
||||
return UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getCommitTime() {
|
||||
return "100";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFileSize() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -17,32 +17,29 @@
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_PARTITION_PATHS;
|
||||
import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION;
|
||||
import static com.uber.hoodie.common.model.HoodieTestUtils.getDefaultHadoopConf;
|
||||
import static com.uber.hoodie.common.util.CompactionTestUtils.createCompactionPlan;
|
||||
import static com.uber.hoodie.common.util.CompactionTestUtils.scheduleCompaction;
|
||||
import static com.uber.hoodie.common.util.CompactionTestUtils.setupAndValidateCompactionOperations;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionOperation;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
|
||||
import com.uber.hoodie.common.util.CompactionTestUtils.TestHoodieDataFile;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.UUID;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
@@ -64,7 +61,8 @@ public class TestCompactionUtils {
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath());
|
||||
metaClient = HoodieTestUtils.initTableType(getDefaultHadoopConf(),
|
||||
tmpFolder.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
|
||||
basePath = metaClient.getBasePath();
|
||||
}
|
||||
|
||||
@@ -156,12 +154,12 @@ public class TestCompactionUtils {
|
||||
@Test(expected = IllegalStateException.class)
|
||||
public void testGetAllPendingCompactionOperationsWithDupFileId() throws IOException {
|
||||
// Case where there is duplicate fileIds in compaction requests
|
||||
HoodieCompactionPlan plan1 = createCompactionPlan("000", 10);
|
||||
HoodieCompactionPlan plan2 = createCompactionPlan("001", 10);
|
||||
scheduleCompaction("000", plan1);
|
||||
scheduleCompaction("001", plan2);
|
||||
HoodieCompactionPlan plan1 = createCompactionPlan(metaClient, "000", "001", 10, true, true);
|
||||
HoodieCompactionPlan plan2 = createCompactionPlan(metaClient, "002", "003", 0, false, false);
|
||||
scheduleCompaction(metaClient, "001", plan1);
|
||||
scheduleCompaction(metaClient, "003", plan2);
|
||||
// schedule same plan again so that there will be duplicates
|
||||
scheduleCompaction("003", plan1);
|
||||
scheduleCompaction(metaClient, "005", plan1);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
Map<String, Pair<String, HoodieCompactionOperation>> res =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
@@ -170,114 +168,19 @@ public class TestCompactionUtils {
|
||||
@Test
|
||||
public void testGetAllPendingCompactionOperations() throws IOException {
|
||||
// Case where there are 4 compaction requests where 1 is empty.
|
||||
testGetAllPendingCompactionOperations(false, 10, 10, 10, 0);
|
||||
setupAndValidateCompactionOperations(metaClient, false, 10, 10, 10, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAllPendingInflightCompactionOperations() throws IOException {
|
||||
// Case where there are 4 compaction requests where 1 is empty. All of them are marked inflight
|
||||
testGetAllPendingCompactionOperations(true, 10, 10, 10, 0);
|
||||
setupAndValidateCompactionOperations(metaClient, true, 10, 10, 10, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAllPendingCompactionOperationsForEmptyCompactions() throws IOException {
|
||||
// Case where there are 4 compaction requests and all are empty.
|
||||
testGetAllPendingCompactionOperations(false, 0, 0, 0, 0);
|
||||
}
|
||||
|
||||
private void testGetAllPendingCompactionOperations(boolean inflight, int numEntriesInPlan1, int numEntriesInPlan2,
|
||||
int numEntriesInPlan3, int numEntriesInPlan4) throws IOException {
|
||||
HoodieCompactionPlan plan1 = createCompactionPlan("000", numEntriesInPlan1);
|
||||
HoodieCompactionPlan plan2 = createCompactionPlan("001", numEntriesInPlan2);
|
||||
HoodieCompactionPlan plan3 = createCompactionPlan("002", numEntriesInPlan3);
|
||||
HoodieCompactionPlan plan4 = createCompactionPlan("003", numEntriesInPlan4);
|
||||
|
||||
if (inflight) {
|
||||
scheduleInflightCompaction("000", plan1);
|
||||
scheduleInflightCompaction("001", plan2);
|
||||
scheduleInflightCompaction("002", plan3);
|
||||
scheduleInflightCompaction("003", plan4);
|
||||
} else {
|
||||
scheduleCompaction("000", plan1);
|
||||
scheduleCompaction("001", plan2);
|
||||
scheduleCompaction("002", plan3);
|
||||
scheduleCompaction("003", plan4);
|
||||
}
|
||||
|
||||
List<Integer> expectedNumEntries =
|
||||
Arrays.asList(numEntriesInPlan1, numEntriesInPlan2, numEntriesInPlan3, numEntriesInPlan4);
|
||||
List<HoodieCompactionPlan> plans = new ImmutableList.Builder<HoodieCompactionPlan>()
|
||||
.add(plan1, plan2, plan3, plan4).build();
|
||||
IntStream.range(0, 4).boxed().forEach(idx -> {
|
||||
if (expectedNumEntries.get(idx) > 0) {
|
||||
Assert.assertEquals("check if plan " + idx + " has exp entries",
|
||||
expectedNumEntries.get(idx).longValue(), plans.get(idx).getOperations().size());
|
||||
} else {
|
||||
Assert.assertNull("Plan " + idx + " has null ops", plans.get(idx).getOperations());
|
||||
}
|
||||
});
|
||||
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
Map<String, Pair<String, HoodieCompactionOperation>> pendingCompactionMap =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
|
||||
Map<String, Pair<String, HoodieCompactionOperation>> expPendingCompactionMap =
|
||||
generateExpectedCompactionOperations(Arrays.asList(plan1, plan2, plan3, plan4));
|
||||
|
||||
// Ensure all the
|
||||
Assert.assertEquals(expPendingCompactionMap, pendingCompactionMap);
|
||||
}
|
||||
|
||||
private Map<String, Pair<String, HoodieCompactionOperation>> generateExpectedCompactionOperations(
|
||||
List<HoodieCompactionPlan> plans) {
|
||||
return plans.stream()
|
||||
.flatMap(plan -> {
|
||||
if (plan.getOperations() != null) {
|
||||
return plan.getOperations().stream().map(op -> Pair.of(op.getFileId(),
|
||||
Pair.of(op.getBaseInstantTime(), op)));
|
||||
}
|
||||
return Stream.empty();
|
||||
}).collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
}
|
||||
|
||||
private void scheduleCompaction(String instantTime, HoodieCompactionPlan compactionPlan) throws IOException {
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime),
|
||||
AvroUtils.serializeCompactionPlan(compactionPlan));
|
||||
}
|
||||
|
||||
private void scheduleInflightCompaction(String instantTime, HoodieCompactionPlan compactionPlan) throws IOException {
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime),
|
||||
AvroUtils.serializeCompactionPlan(compactionPlan));
|
||||
metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime));
|
||||
}
|
||||
|
||||
private HoodieCompactionPlan createCompactionPlan(String instantId, int numFileIds) {
|
||||
List<HoodieCompactionOperation> ops = IntStream.range(0, numFileIds).boxed().map(idx -> {
|
||||
try {
|
||||
String fileId =
|
||||
HoodieTestUtils.createNewDataFile(basePath, DEFAULT_PARTITION_PATHS[0], instantId);
|
||||
HoodieTestUtils.createNewLogFile(metaClient.getFs(), basePath, DEFAULT_PARTITION_PATHS[0],
|
||||
instantId, fileId, Optional.of(1));
|
||||
HoodieTestUtils.createNewLogFile(metaClient.getFs(), basePath, DEFAULT_PARTITION_PATHS[0],
|
||||
instantId, fileId, Optional.of(2));
|
||||
FileSlice slice = new FileSlice(instantId, fileId);
|
||||
slice.setDataFile(new TestHoodieDataFile(HoodieTestUtils.createDataFile(basePath, DEFAULT_PARTITION_PATHS[0],
|
||||
instantId, fileId)));
|
||||
String logFilePath1 = HoodieTestUtils.getLogFilePath(basePath, DEFAULT_PARTITION_PATHS[0], instantId, fileId,
|
||||
Optional.of(1));
|
||||
String logFilePath2 = HoodieTestUtils.getLogFilePath(basePath, DEFAULT_PARTITION_PATHS[0], instantId, fileId,
|
||||
Optional.of(2));
|
||||
slice.addLogFile(new HoodieLogFile(new Path(logFilePath1)));
|
||||
slice.addLogFile(new HoodieLogFile(new Path(logFilePath2)));
|
||||
return CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], slice, Optional.empty());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
return new HoodieCompactionPlan(ops.isEmpty() ? null : ops, new HashMap<>());
|
||||
setupAndValidateCompactionOperations(metaClient, false, 0, 0, 0, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -315,35 +218,4 @@ public class TestCompactionUtils {
|
||||
});
|
||||
Assert.assertEquals("Metrics set", metrics, op.getMetrics());
|
||||
}
|
||||
|
||||
|
||||
private static class TestHoodieDataFile extends HoodieDataFile {
|
||||
|
||||
private final String path;
|
||||
|
||||
public TestHoodieDataFile(String path) {
|
||||
super(null);
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPath() {
|
||||
return path;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getFileId() {
|
||||
return UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getCommitTime() {
|
||||
return "100";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFileSize() {
|
||||
return 0;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user