1
0

FileSystem View must treat same fileIds present in different partitions as different file-groups and handle pending compaction correctly

This commit is contained in:
Balaji Varadarajan
2019-02-12 21:29:14 -08:00
committed by vinoth chandar
parent 363df2c12e
commit 3ae6cb4ed5
20 changed files with 388 additions and 120 deletions

View File

@@ -26,6 +26,7 @@ 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.HoodieFileGroup;
import com.uber.hoodie.common.model.HoodieFileGroupId;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
@@ -51,6 +52,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
@@ -676,7 +678,7 @@ public class HoodieTableFileSystemViewTest {
assertEquals(3, fileGroups.size());
for (HoodieFileGroup fileGroup : fileGroups) {
String fileId = fileGroup.getId();
String fileId = fileGroup.getFileGroupId().getFileId();
Set<String> filenames = Sets.newHashSet();
fileGroup.getAllDataFiles().forEach(dataFile -> {
assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId());
@@ -850,16 +852,17 @@ public class HoodieTableFileSystemViewTest {
assertEquals(3, fileGroups.size());
for (HoodieFileGroup fileGroup : fileGroups) {
List<FileSlice> slices = fileGroup.getAllFileSlices().collect(Collectors.toList());
if (fileGroup.getId().equals(fileId1)) {
String fileId = fileGroup.getFileGroupId().getFileId();
if (fileId.equals(fileId1)) {
assertEquals(2, slices.size());
assertEquals(commitTime4, slices.get(0).getBaseInstantTime());
assertEquals(commitTime1, slices.get(1).getBaseInstantTime());
} else if (fileGroup.getId().equals(fileId2)) {
} else if (fileId.equals(fileId2)) {
assertEquals(3, slices.size());
assertEquals(commitTime3, slices.get(0).getBaseInstantTime());
assertEquals(commitTime2, slices.get(1).getBaseInstantTime());
assertEquals(commitTime1, slices.get(2).getBaseInstantTime());
} else if (fileGroup.getId().equals(fileId3)) {
} else if (fileId.equals(fileId3)) {
assertEquals(2, slices.size());
assertEquals(commitTime4, slices.get(0).getBaseInstantTime());
assertEquals(commitTime3, slices.get(1).getBaseInstantTime());
@@ -876,4 +879,159 @@ public class HoodieTableFileSystemViewTest {
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
}
@Test
public void testPendingCompactionWithDuplicateFileIdsAcrossPartitions() throws Exception {
// Put some files in the partition
String partitionPath1 = "2016/05/01";
String partitionPath2 = "2016/05/02";
String partitionPath3 = "2016/05/03";
String fullPartitionPath1 = basePath + "/" + partitionPath1 + "/";
new File(fullPartitionPath1).mkdirs();
String fullPartitionPath2 = basePath + "/" + partitionPath2 + "/";
new File(fullPartitionPath2).mkdirs();
String fullPartitionPath3 = basePath + "/" + partitionPath3 + "/";
new File(fullPartitionPath3).mkdirs();
String instantTime1 = "1";
String deltaInstantTime1 = "2";
String deltaInstantTime2 = "3";
String fileId = UUID.randomUUID().toString();
String dataFileName = FSUtils.makeDataFileName(instantTime1, 1, fileId);
new File(fullPartitionPath1 + dataFileName).createNewFile();
String fileName1 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, instantTime1, 0);
new File(fullPartitionPath1 + fileName1)
.createNewFile();
new File(fullPartitionPath2 + FSUtils.makeDataFileName(instantTime1, 1, fileId)).createNewFile();
new File(fullPartitionPath2 + fileName1)
.createNewFile();
new File(fullPartitionPath3 + FSUtils.makeDataFileName(instantTime1, 1, fileId)).createNewFile();
new File(fullPartitionPath3 + fileName1)
.createNewFile();
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1);
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
commitTimeline.saveAsComplete(instant1, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant2, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant3, Optional.empty());
// Now we list all partitions
FileStatus[] statuses = metaClient.getFs().listStatus(new Path[] {
new Path(fullPartitionPath1), new Path(fullPartitionPath2), new Path(fullPartitionPath3)
});
assertEquals(6, statuses.length);
refreshFsView(statuses);
List<HoodieFileGroup> groups = fsView.getAllFileGroups().collect(Collectors.toList());
Assert.assertEquals("Expected number of file-groups", 3, groups.size());
Assert.assertEquals("Partitions must be different for file-groups", 3,
groups.stream().map(HoodieFileGroup::getPartitionPath).collect(Collectors.toSet()).size());
Set<String> fileIds = groups.stream().map(HoodieFileGroup::getFileGroupId)
.map(HoodieFileGroupId::getFileId).collect(Collectors.toSet());
Assert.assertEquals("File Id must be same", 1, fileIds.size());
Assert.assertTrue("Expected FileId", fileIds.contains(fileId));
// Setup Pending compaction for all of these fileIds.
List<Pair<String, FileSlice>> partitionFileSlicesPairs = new ArrayList<>();
List<FileSlice> fileSlices = rtView.getLatestFileSlices(partitionPath1).collect(Collectors.toList());
partitionFileSlicesPairs.add(Pair.of(partitionPath1, fileSlices.get(0)));
fileSlices = rtView.getLatestFileSlices(partitionPath2).collect(Collectors.toList());
partitionFileSlicesPairs.add(Pair.of(partitionPath2, fileSlices.get(0)));
fileSlices = rtView.getLatestFileSlices(partitionPath3).collect(Collectors.toList());
partitionFileSlicesPairs.add(Pair.of(partitionPath3, fileSlices.get(0)));
String compactionRequestedTime = "2";
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, 1, fileId);
HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs,
Optional.empty(), Optional.empty());
// Create a Data-file for some of the partitions but this should be skipped by view
new File(basePath + "/" + partitionPath1 + "/" + compactDataFileName).createNewFile();
new File(basePath + "/" + partitionPath2 + "/" + compactDataFileName).createNewFile();
HoodieInstant compactionInstant =
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime);
HoodieInstant requested = HoodieTimeline.getCompactionRequestedInstant(compactionInstant.getTimestamp());
metaClient.getActiveTimeline().saveToCompactionRequested(requested,
AvroUtils.serializeCompactionPlan(compactionPlan));
metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(requested);
// Fake delta-ingestion after compaction-requested
String deltaInstantTime4 = "3";
String deltaInstantTime5 = "6";
List<String> allInstantTimes = Arrays.asList(instantTime1, deltaInstantTime1, deltaInstantTime2,
compactionRequestedTime, deltaInstantTime4, deltaInstantTime5);
String fileName3 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 0);
String fileName4 = FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, compactionRequestedTime, 1);
new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile();
new File(basePath + "/" + partitionPath1 + "/" + fileName4).createNewFile();
new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile();
new File(basePath + "/" + partitionPath2 + "/" + fileName4).createNewFile();
new File(basePath + "/" + partitionPath3 + "/" + fileName3).createNewFile();
new File(basePath + "/" + partitionPath3 + "/" + fileName4).createNewFile();
HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4);
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
commitTimeline.saveAsComplete(deltaInstant4, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant5, Optional.empty());
refreshFsView(null);
// Test Data Files
List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath1).collect(Collectors.toList());
assertEquals("One data-file is expected as there is only one file-group", 1, dataFiles.size());
assertEquals("Expect only valid commit", "1", dataFiles.get(0).getCommitTime());
dataFiles = roView.getAllDataFiles(partitionPath2).collect(Collectors.toList());
assertEquals("One data-file is expected as there is only one file-group", 1, dataFiles.size());
assertEquals("Expect only valid commit", "1", dataFiles.get(0).getCommitTime());
/** Merge API Tests **/
Arrays.asList(partitionPath1, partitionPath2, partitionPath3).stream().forEach(partitionPath -> {
List<FileSlice> fileSliceList = rtView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5)
.collect(Collectors.toList());
assertEquals("Expect file-slice to be merged", 1, fileSliceList.size());
FileSlice fileSlice = fileSliceList.get(0);
assertEquals(fileId, fileSlice.getFileId());
assertEquals("Data file must be present", dataFileName, fileSlice.getDataFile().get().getFileName());
assertEquals("Base Instant of penultimate file-slice must be base instant", instantTime1,
fileSlice.getBaseInstantTime());
List<HoodieLogFile> logFiles = fileSlice.getLogFiles().collect(Collectors.toList());
assertEquals("Log files must include those after compaction request", 3, logFiles.size());
assertEquals("Log File Order check", fileName4, logFiles.get(0).getFileName());
assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName());
assertEquals("Log File Order check", fileName1, logFiles.get(2).getFileName());
fileSliceList = rtView.getLatestFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5)
.collect(Collectors.toList());
assertEquals("Expect only one file-id", 1, fileSliceList.size());
fileSlice = fileSliceList.get(0);
assertEquals(fileId, fileSlice.getFileId());
assertFalse("No data-file expected in latest file-slice", fileSlice.getDataFile().isPresent());
assertEquals("Compaction requested instant must be base instant", compactionRequestedTime,
fileSlice.getBaseInstantTime());
logFiles = fileSlice.getLogFiles().collect(Collectors.toList());
assertEquals("Log files must include only those after compaction request", 2, logFiles.size());
assertEquals("Log File Order check", fileName4, logFiles.get(0).getFileName());
assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName());
});
Assert.assertEquals(3, fsView.getFgIdToPendingCompaction().size());
Set<String> partitionsInCompaction =
fsView.getFgIdToPendingCompaction().keySet().stream().map(HoodieFileGroupId::getPartitionPath)
.collect(Collectors.toSet());
Assert.assertEquals(3, partitionsInCompaction.size());
Assert.assertTrue(partitionsInCompaction.contains(partitionPath1));
Assert.assertTrue(partitionsInCompaction.contains(partitionPath2));
Assert.assertTrue(partitionsInCompaction.contains(partitionPath3));
Set<String> fileIdsInCompaction =
fsView.getFgIdToPendingCompaction().keySet().stream().map(HoodieFileGroupId::getFileId)
.collect(Collectors.toSet());
Assert.assertEquals(1, fileIdsInCompaction.size());
Assert.assertTrue(fileIdsInCompaction.contains(fileId));
}
}

View File

@@ -26,6 +26,7 @@ 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.HoodieFileGroupId;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
@@ -48,7 +49,7 @@ import org.junit.Assert;
public class CompactionTestUtils {
public static Map<String, Pair<String, HoodieCompactionOperation>> setupAndValidateCompactionOperations(
public static Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> setupAndValidateCompactionOperations(
HoodieTableMetaClient metaClient, boolean inflight,
int numEntriesInPlan1, int numEntriesInPlan2,
int numEntriesInPlan3, int numEntriesInPlan4) throws IOException {
@@ -91,10 +92,10 @@ public class CompactionTestUtils {
});
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath(), true);
Map<String, Pair<String, HoodieCompactionOperation>> pendingCompactionMap =
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> pendingCompactionMap =
CompactionUtils.getAllPendingCompactionOperations(metaClient);
Map<String, Pair<String, HoodieCompactionOperation>> expPendingCompactionMap =
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> expPendingCompactionMap =
generateExpectedCompactionOperations(Arrays.asList(plan1, plan2, plan3, plan4), baseInstantsToCompaction);
// Ensure Compaction operations are fine.
@@ -102,12 +103,13 @@ public class CompactionTestUtils {
return expPendingCompactionMap;
}
public static Map<String, Pair<String, HoodieCompactionOperation>> generateExpectedCompactionOperations(
public static Map<HoodieFileGroupId, 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(),
return plan.getOperations().stream().map(op -> Pair.of(
new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()),
Pair.of(baseInstantsToCompaction.get(op.getBaseInstantTime()), op)));
}
return Stream.empty();
@@ -146,7 +148,7 @@ public class CompactionTestUtils {
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);
FileSlice slice = new FileSlice(DEFAULT_PARTITION_PATHS[0], instantId, fileId);
if (createDataFile) {
slice.setDataFile(new TestHoodieDataFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0]
+ "/" + FSUtils.makeDataFileName(instantId, 1, fileId)));

View File

@@ -26,6 +26,7 @@ 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.HoodieFileGroupId;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.model.HoodieTestUtils;
@@ -69,20 +70,20 @@ public class TestCompactionUtils {
@Test
public void testBuildFromFileSlice() {
// Empty File-Slice with no data and log files
FileSlice emptyFileSlice = new FileSlice("000", "empty1");
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "empty1");
HoodieCompactionOperation op = CompactionUtils.buildFromFileSlice(
DEFAULT_PARTITION_PATHS[0], emptyFileSlice, Optional.of(metricsCaptureFn));
testFileSliceCompactionOpEquality(emptyFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
// File Slice with data-file but no log files
FileSlice noLogFileSlice = new FileSlice("000", "noLog1");
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1");
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
op = CompactionUtils.buildFromFileSlice(
DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Optional.of(metricsCaptureFn));
testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
//File Slice with no data-file but log files present
FileSlice noDataFileSlice = new FileSlice("000", "noData1");
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
@@ -92,7 +93,7 @@ public class TestCompactionUtils {
testFileSliceCompactionOpEquality(noDataFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
//File Slice with data-file and log files present
FileSlice fileSlice = new FileSlice("000", "noData1");
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
fileSlice.addLogFile(new HoodieLogFile(new Path(
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
@@ -107,16 +108,16 @@ public class TestCompactionUtils {
* Generate input for compaction plan tests
*/
private Pair<List<Pair<String, FileSlice>>, HoodieCompactionPlan> buildCompactionPlan() {
FileSlice emptyFileSlice = new FileSlice("000", "empty1");
FileSlice fileSlice = new FileSlice("000", "noData1");
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "empty1");
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
fileSlice.addLogFile(new HoodieLogFile(new Path(
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
fileSlice.addLogFile(new HoodieLogFile(new Path(
FSUtils.makeLogFileName("noData1", ".log", "000", 2))));
FileSlice noLogFileSlice = new FileSlice("000", "noLog1");
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1");
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
FileSlice noDataFileSlice = new FileSlice("000", "noData1");
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
@@ -161,7 +162,7 @@ public class TestCompactionUtils {
// schedule same plan again so that there will be duplicates
scheduleCompaction(metaClient, "005", plan1);
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
Map<String, Pair<String, HoodieCompactionOperation>> res =
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> res =
CompactionUtils.getAllPendingCompactionOperations(metaClient);
}