1
0

HUDI-479: Eliminate or Minimize use of Guava if possible (#1159)

This commit is contained in:
Suneel Marthi
2020-03-28 03:11:32 -04:00
committed by GitHub
parent 1713f686f8
commit 8c3001363d
46 changed files with 429 additions and 217 deletions

View File

@@ -249,17 +249,13 @@ public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5));
String filename0 =
HoodieClientTestUtils.writeParquetFile(basePath, "2016/04/01", Collections.singletonList(record1),
schema, null, false);
HoodieClientTestUtils.writeParquetFile(basePath, "2016/04/01", Collections.singletonList(record1), schema, null, false);
String filename1 =
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", new ArrayList<>(),
schema, null, false);
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", new ArrayList<>(), schema, null, false);
String filename2 =
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record2),
schema, null, false);
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record2), schema, null, false);
String filename3 =
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record4),
schema, null, false);
HoodieClientTestUtils.writeParquetFile(basePath, "2015/03/12", Collections.singletonList(record4), schema, null, false);
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
metaClient = HoodieTableMetaClient.reload(metaClient);

View File

@@ -45,6 +45,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.AvroUtils;
import org.apache.hudi.common.util.CleanerUtils;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.ConsistencyGuardConfig;
import org.apache.hudi.common.util.FSUtils;
@@ -55,9 +56,6 @@ import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.log4j.LogManager;
@@ -290,7 +288,7 @@ public class TestCleaner extends TestHoodieClientBase {
List<String> commitedVersions = new ArrayList<>(fileIdToVersions.get(fileId));
for (int i = 0; i < dataFiles.size(); i++) {
assertEquals("File " + fileId + " does not have latest versions on commits" + commitedVersions,
Iterables.get(dataFiles, i).getCommitTime(),
(dataFiles.get(i)).getCommitTime(),
commitedVersions.get(commitedVersions.size() - 1 - i));
}
}
@@ -740,12 +738,14 @@ public class TestCleaner extends TestHoodieClientBase {
String file1P1C0 =
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "000");
HoodieCommitMetadata commitMetadata = generateCommitMetadata(new ImmutableMap.Builder()
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
new ImmutableList.Builder<>().add(file1P0C0).build())
.put(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
new ImmutableList.Builder<>().add(file1P1C0).build())
.build());
HoodieCommitMetadata commitMetadata = generateCommitMetadata(
Collections.unmodifiableMap(new HashMap<String, List<String>>() {
{
put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, CollectionUtils.createImmutableList(file1P0C0));
put(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, CollectionUtils.createImmutableList(file1P1C0));
}
})
);
metaClient.getActiveTimeline().saveAsComplete(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "000"),
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
@@ -778,12 +778,12 @@ public class TestCleaner extends TestHoodieClientBase {
.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "001", file1P0C0); // update
HoodieTestUtils
.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, "001", file1P1C0); // update
commitMetadata = generateCommitMetadata(new ImmutableMap.Builder()
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
new ImmutableList.Builder<>().add(file1P0C0).add(file2P0C1).build())
.put(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
new ImmutableList.Builder<>().add(file1P1C0).add(file2P1C1).build())
.build());
commitMetadata = generateCommitMetadata(new HashMap<String, List<String>>() {
{
put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, CollectionUtils.createImmutableList(file1P0C0, file2P0C1));
put(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, CollectionUtils.createImmutableList(file1P1C0, file2P1C1));
}
});
metaClient.getActiveTimeline().saveAsComplete(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001"),
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
@@ -814,10 +814,9 @@ public class TestCleaner extends TestHoodieClientBase {
String file3P0C2 =
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "002");
commitMetadata = generateCommitMetadata(new ImmutableMap.Builder()
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
new ImmutableList.Builder<>().add(file1P0C0).add(file2P0C1).add(file3P0C2).build())
.build());
commitMetadata = generateCommitMetadata(CollectionUtils
.createImmutableMap(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file3P0C2)));
metaClient.getActiveTimeline().saveAsComplete(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "002"),
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
@@ -840,10 +839,8 @@ public class TestCleaner extends TestHoodieClientBase {
.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "003", file2P0C1); // update
String file4P0C3 =
HoodieTestUtils.createNewDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "003");
commitMetadata = generateCommitMetadata(new ImmutableMap.Builder()
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
new ImmutableList.Builder<>().add(file1P0C0).add(file2P0C1).add(file4P0C3).build())
.build());
commitMetadata = generateCommitMetadata(CollectionUtils.createImmutableMap(
HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, CollectionUtils.createImmutableList(file1P0C0, file2P0C1, file4P0C3)));
metaClient.getActiveTimeline().saveAsComplete(
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "003"),
Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
@@ -871,10 +868,8 @@ public class TestCleaner extends TestHoodieClientBase {
// No cleaning on partially written file, with no commit.
HoodieTestUtils
.createDataFile(basePath, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, "004", file3P0C2); // update
commitMetadata = generateCommitMetadata(new ImmutableMap.Builder()
.put(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
new ImmutableList.Builder<>().add(file3P0C2).build())
.build());
commitMetadata = generateCommitMetadata(CollectionUtils.createImmutableMap(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH,
CollectionUtils.createImmutableList(file3P0C2)));
metaClient.getActiveTimeline().createNewInstant(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "004"));
metaClient.getActiveTimeline().transitionRequestedToInflight(

View File

@@ -27,7 +27,6 @@ import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import com.google.common.collect.ImmutableMap;
import org.junit.Assert;
import org.junit.Test;
@@ -122,8 +121,14 @@ public class TestHoodieCompactionStrategy {
sizesMap.put(100 * MB, Collections.singletonList(MB));
sizesMap.put(90 * MB, Collections.singletonList(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();
Map<Long, String> keyToPartitionMap = Collections.unmodifiableMap(new HashMap<Long,String>() {
{
put(120 * MB, partitionPaths[2]);
put(110 * MB, partitionPaths[2]);
put(100 * MB, partitionPaths[1]);
put(90 * MB, partitionPaths[0]);
}
});
DayBasedCompactionStrategy strategy = new DayBasedCompactionStrategy();
HoodieWriteConfig writeConfig =
@@ -162,9 +167,16 @@ public class TestHoodieCompactionStrategy {
String currentDayPlus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(1));
String currentDayPlus5 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(5));
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder().put(120 * MB, currentDay)
.put(110 * MB, currentDayMinus1).put(100 * MB, currentDayMinus2).put(80 * MB, currentDayMinus3)
.put(90 * MB, currentDayPlus1).put(70 * MB, currentDayPlus5).build();
Map<Long, String> keyToPartitionMap = Collections.unmodifiableMap(new HashMap<Long,String>() {
{
put(120 * MB, currentDay);
put(110 * MB, currentDayMinus1);
put(100 * MB, currentDayMinus2);
put(80 * MB, currentDayMinus3);
put(90 * MB, currentDayPlus1);
put(70 * MB, currentDayPlus5);
}
});
BoundedPartitionAwareCompactionStrategy strategy = new BoundedPartitionAwareCompactionStrategy();
HoodieWriteConfig writeConfig =
@@ -204,9 +216,16 @@ public class TestHoodieCompactionStrategy {
String currentDayPlus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(1));
String currentDayPlus5 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(5));
Map<Long, String> keyToPartitionMap = new ImmutableMap.Builder().put(120 * MB, currentDay)
.put(110 * MB, currentDayMinus1).put(100 * MB, currentDayMinus2).put(80 * MB, currentDayMinus3)
.put(90 * MB, currentDayPlus1).put(70 * MB, currentDayPlus5).build();
Map<Long, String> keyToPartitionMap = Collections.unmodifiableMap(new HashMap<Long,String>() {
{
put(120 * MB, currentDay);
put(110 * MB, currentDayMinus1);
put(100 * MB, currentDayMinus2);
put(80 * MB, currentDayMinus3);
put(90 * MB, currentDayPlus1);
put(70 * MB, currentDayPlus5);
}
});
UnBoundedPartitionAwareCompactionStrategy strategy = new UnBoundedPartitionAwareCompactionStrategy();
HoodieWriteConfig writeConfig =