1
0

Parallelize file version deletes during clean and related tests

This commit is contained in:
Kaushik Devarajaiah
2017-05-29 15:36:12 -07:00
committed by prazanna
parent dda28c0b4b
commit 521555c576
7 changed files with 459 additions and 299 deletions

View File

@@ -18,6 +18,7 @@ package com.uber.hoodie;
import com.google.common.collect.Iterables;
import com.uber.hoodie.common.HoodieCleanStat;
import com.uber.hoodie.common.HoodieClientTestUtils;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
@@ -42,13 +43,17 @@ import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.util.Map;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.scheduler.SparkListener;
import org.apache.spark.scheduler.SparkListenerTaskEnd;
import org.apache.spark.sql.SQLContext;
import org.apache.spark.util.AccumulatorV2;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@@ -68,6 +73,7 @@ import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import scala.collection.Iterator;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@@ -78,8 +84,8 @@ public class TestHoodieClient implements Serializable {
private transient JavaSparkContext jsc = null;
private transient SQLContext sqlContext;
private String basePath = null;
private transient HoodieTestDataGenerator
dataGen = null;
private transient HoodieTestDataGenerator dataGen = null;
private String[] partitionPaths = {"2016/01/01", "2016/02/02", "2016/06/02"};
@Before
public void init() throws IOException {
@@ -1070,8 +1076,249 @@ public class TestHoodieClient implements Serializable {
assertEquals("Total number of records must add up", totalInserts, inserts1.size() + inserts2.size() + insert3.size());
}
@Test
public void testKeepLatestFileVersions() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withAssumeDatePartitioning(true)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
.retainFileVersions(1).build()).build();
// make 1 commit, with 1 file per partition
HoodieTestUtils.createCommitFiles(basePath, "000");
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), config);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
assertEquals("Must not clean any files" , 0, getCleanStat(hoodieCleanStatsOne, partitionPaths[0]).getSuccessDeleteFiles().size());
assertEquals("Must not clean any files" , 0, getCleanStat(hoodieCleanStatsOne, partitionPaths[1]).getSuccessDeleteFiles().size());
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "000", file1P1C0));
// make next commit, with 1 insert & 1 update per partition
HoodieTestUtils.createCommitFiles(basePath, "001");
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), config);
String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "001"); // insert
String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update
List<HoodieCleanStat> hoodieCleanStatsTwo = table.clean(jsc);
assertEquals("Must clean 1 file" , 1, getCleanStat(hoodieCleanStatsTwo, partitionPaths[0]).getSuccessDeleteFiles().size());
assertEquals("Must clean 1 file" , 1, getCleanStat(hoodieCleanStatsTwo, partitionPaths[1]).getSuccessDeleteFiles().size());
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "001", file2P1C1));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "000", file1P1C0));
// make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "002");
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), config);
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update
String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002");
List<HoodieCleanStat> hoodieCleanStatsThree = table.clean(jsc);
assertEquals("Must clean two files" , 2, getCleanStat(hoodieCleanStatsThree, partitionPaths[0]).getSuccessDeleteFiles().size());
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0));
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2));
// No cleaning on partially written file, with no commit.
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file3P0C2); // update
List<HoodieCleanStat> hoodieCleanStatsFour = table.clean(jsc);
assertEquals("Must not clean any files" , 0, getCleanStat(hoodieCleanStatsFour, partitionPaths[0]).getSuccessDeleteFiles().size());
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2));
}
@Test
public void testKeepLatestCommits() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withAssumeDatePartitioning(true)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS)
.retainCommits(2).build()).build();
// make 1 commit, with 1 file per partition
HoodieTestUtils.createCommitFiles(basePath, "000");
String file1P0C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
String file1P1C0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "000");
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), config);
List<HoodieCleanStat> hoodieCleanStatsOne = table.clean(jsc);
assertEquals("Must not clean any files" , 0, getCleanStat(hoodieCleanStatsOne, partitionPaths[0]).getSuccessDeleteFiles().size());
assertEquals("Must not clean any files" , 0, getCleanStat(hoodieCleanStatsOne, partitionPaths[1]).getSuccessDeleteFiles().size());
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "000", file1P1C0));
// make next commit, with 1 insert & 1 update per partition
HoodieTestUtils.createCommitFiles(basePath, "001");
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), config);
String file2P0C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "001"); // insert
String file2P1C1 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[1], "001"); // insert
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[1], "001", file1P1C0); // update
List<HoodieCleanStat> hoodieCleanStatsTwo = table.clean(jsc);
assertEquals("Must not clean any files" , 0, getCleanStat(hoodieCleanStatsTwo, partitionPaths[0]).getSuccessDeleteFiles().size());
assertEquals("Must not clean any files" , 0, getCleanStat(hoodieCleanStatsTwo, partitionPaths[1]).getSuccessDeleteFiles().size());
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "001", file2P1C1));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[1], "000", file1P1C0));
// make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "002");
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), config);
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "002", file2P0C1); // update
String file3P0C2 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "002");
List<HoodieCleanStat> hoodieCleanStatsThree = table.clean(jsc);
assertEquals(
"Must not clean any file. We have to keep 1 version before the latest commit time to keep",
0, getCleanStat(hoodieCleanStatsThree, partitionPaths[0]).getSuccessDeleteFiles().size());
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
// make next commit, with 2 updates to existing files, and 1 insert
HoodieTestUtils.createCommitFiles(basePath, "003");
table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), config);
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file1P0C0); // update
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "003", file2P0C1); // update
String file4P0C3 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "003");
List<HoodieCleanStat> hoodieCleanStatsFour = table.clean(jsc);
assertEquals(
"Must not clean one old file", 1, getCleanStat(hoodieCleanStatsFour, partitionPaths[0]).getSuccessDeleteFiles().size());
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0C0));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file1P0C0));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file2P0C1));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "003", file4P0C3));
// No cleaning on partially written file, with no commit.
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "004", file3P0C2); // update
List<HoodieCleanStat> hoodieCleanStatsFive = table.clean(jsc);
assertEquals("Must not clean any files" , 0, getCleanStat(hoodieCleanStatsFive, partitionPaths[0]).getSuccessDeleteFiles().size());
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file1P0C0));
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "001", file2P0C1));
}
@Test
public void testCleaningSkewedPartitons() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withAssumeDatePartitioning(true)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS)
.retainCommits(2).build()).build();
Map<Long, Long> stageOneShuffleReadTaskRecordsCountMap = new HashMap<>();
// Since clean involves repartition in order to uniformly distribute data,
// we can inspect the number of records read by various tasks in stage 1.
// There should not be skew in the number of records read in the task.
// SparkListener below listens to the stage end events and captures number of
// records read by various tasks in stage-1.
jsc.sc().addSparkListener(new SparkListener() {
@Override
public void onTaskEnd(SparkListenerTaskEnd taskEnd) {
Iterator<AccumulatorV2<?, ?>> iterator = taskEnd.taskMetrics().accumulators()
.iterator();
while(iterator.hasNext()) {
AccumulatorV2 accumulator = iterator.next();
if (taskEnd.stageId() == 1 &&
accumulator.isRegistered() &&
accumulator.name().isDefined() &&
accumulator.name().get().equals("internal.metrics.shuffle.read.recordsRead")) {
stageOneShuffleReadTaskRecordsCountMap.put(taskEnd.taskInfo().taskId(), (Long) accumulator.value());
}
}
}
});
// make 1 commit, with 100 files in one partition and 10 in other two
HoodieTestUtils.createCommitFiles(basePath, "000");
List<String> filesP0C0 = createFilesInPartition(partitionPaths[0], "000", 100);
List<String> filesP1C0 = createFilesInPartition(partitionPaths[1], "000", 10);
List<String> filesP2C0 = createFilesInPartition(partitionPaths[2], "000", 10);
HoodieTestUtils.createCommitFiles(basePath, "001");
updateAllFilesInPartition(filesP0C0, partitionPaths[0], "001");
updateAllFilesInPartition(filesP1C0, partitionPaths[1], "001");
updateAllFilesInPartition(filesP2C0, partitionPaths[2], "001");
HoodieTestUtils.createCommitFiles(basePath, "002");
updateAllFilesInPartition(filesP0C0, partitionPaths[0], "002");
updateAllFilesInPartition(filesP1C0, partitionPaths[1], "002");
updateAllFilesInPartition(filesP2C0, partitionPaths[2], "002");
HoodieTestUtils.createCommitFiles(basePath, "003");
updateAllFilesInPartition(filesP0C0, partitionPaths[0], "003");
updateAllFilesInPartition(filesP1C0, partitionPaths[1], "003");
updateAllFilesInPartition(filesP2C0, partitionPaths[2], "003");
HoodieTable table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(FSUtils.getFs(), config.getBasePath(), true), config);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
assertEquals(100, getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size());
assertEquals(10, getCleanStat(hoodieCleanStats, partitionPaths[1]).getSuccessDeleteFiles().size());
assertEquals(10, getCleanStat(hoodieCleanStats, partitionPaths[2]).getSuccessDeleteFiles().size());
// 3 tasks are expected since the number of partitions is 3
assertEquals(3, stageOneShuffleReadTaskRecordsCountMap.keySet().size());
// Sum of all records processed = total number of files to clean
assertEquals(120, stageOneShuffleReadTaskRecordsCountMap
.values().stream().reduce((a,b) -> a + b).get().intValue());
assertTrue("The skew in handling files to clean is not removed. "
+ "Each task should handle more records than the partitionPath with least files "
+ "and less records than the partitionPath with most files.",
stageOneShuffleReadTaskRecordsCountMap.values().stream().filter(a -> a > 10 && a < 100).count() == 3);
}
private HoodieCleanStat getCleanStat(List<HoodieCleanStat> hoodieCleanStatsTwo,
String partitionPath) {
return hoodieCleanStatsTwo.stream()
.filter(e -> e.getPartitionPath().equals(partitionPath))
.findFirst().get();
}
private void updateAllFilesInPartition(List<String> files, String partitionPath,
String commitTime) throws IOException {
for (String fileId : files) {
HoodieTestUtils.createDataFile(basePath, partitionPath, commitTime, fileId);
}
}
private List<String> createFilesInPartition(String partitionPath, String commitTime, int numFiles) throws IOException {
List<String> files = new ArrayList<>();
for (int i = 0; i < numFiles; i++) {
files.add(HoodieTestUtils.createNewDataFile(basePath, partitionPath, commitTime));
}
return files;
}
@After
public void clean() {