1
0

fix for cleaning log files(mor)

This commit is contained in:
Nishith Agarwal
2017-06-29 00:32:11 -07:00
committed by prazanna
parent 19c22b231e
commit 0b26b60a5c
4 changed files with 103 additions and 40 deletions

View File

@@ -17,7 +17,6 @@
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;
@@ -28,6 +27,7 @@ import com.uber.hoodie.common.model.HoodieFileGroup;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
@@ -43,9 +43,6 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.table.HoodieTable;
import java.util.Collection;
import java.util.Map;
import org.apache.avro.generic.GenericRecord;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FileSystem;
@@ -61,21 +58,23 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import scala.collection.Iterator;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors;
import scala.collection.Iterator;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
@@ -1134,6 +1133,42 @@ public class TestHoodieClient implements Serializable {
assertTrue(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "002", file3P0C2));
}
@Test
public void testKeepLatestFileVersionsMOR() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withAssumeDatePartitioning(true)
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS)
.retainFileVersions(1).build()).build();
HoodieTableMetaClient metaClient = HoodieTestUtils.initTableType(basePath, HoodieTableType.MERGE_ON_READ);
// Make 3 files, one base file and 2 log files associated with base file
String file1P0 = HoodieTestUtils.createNewDataFile(basePath, partitionPaths[0], "000");
String file2P0L0 = HoodieTestUtils.createNewLogFile(basePath, partitionPaths[0], "000", file1P0, Optional.empty());
String file2P0L1 = HoodieTestUtils.createNewLogFile(basePath, partitionPaths[0], "000", file1P0, Optional.of(2));
// make 1 compaction commit
HoodieTestUtils.createCompactionCommitFiles(basePath, "000");
// Make 4 files, one base file and 3 log files associated with base file
HoodieTestUtils.createDataFile(basePath, partitionPaths[0], "001", file1P0);
file2P0L0 = HoodieTestUtils.createNewLogFile(basePath, partitionPaths[0], "001", file1P0, Optional.empty());
file2P0L0 = HoodieTestUtils.createNewLogFile(basePath, partitionPaths[0], "001", file1P0, Optional.of(2));
file2P0L0 = HoodieTestUtils.createNewLogFile(basePath, partitionPaths[0], "001", file1P0, Optional.of(3));
// make 1 compaction commit
HoodieTestUtils.createCompactionCommitFiles(basePath, "001");
HoodieTable table = HoodieTable
.getHoodieTable(metaClient, config);
List<HoodieCleanStat> hoodieCleanStats = table.clean(jsc);
assertEquals("Must clean three files, one parquet and 2 log files" , 3, getCleanStat(hoodieCleanStats, partitionPaths[0]).getSuccessDeleteFiles().size());
assertFalse(HoodieTestUtils.doesDataFileExist(basePath, partitionPaths[0], "000", file1P0));
assertFalse(HoodieTestUtils.doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.empty()));
assertFalse(HoodieTestUtils.doesLogFileExist(basePath, partitionPaths[0], "000", file2P0L0, Optional.of(2)));
}
@Test
public void testKeepLatestCommits() throws IOException {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)