1
0

[HUDI-4249] Fixing in-memory HoodieData implementation to operate lazily (#5855)

This commit is contained in:
Alexey Kudinkin
2022-07-16 16:26:48 -07:00
committed by GitHub
parent 80368a049d
commit 4bda6afe0b
35 changed files with 868 additions and 555 deletions

View File

@@ -21,17 +21,19 @@ package org.apache.hudi.common.data;
import org.apache.hudi.common.util.collection.Pair;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.jupiter.api.Assertions.assertEquals;
class TestHoodieList {
class TestHoodieListData {
private static Stream<Arguments> distinctWithKey() {
return Stream.of(
@@ -44,7 +46,22 @@ class TestHoodieList {
@ParameterizedTest
@MethodSource
void distinctWithKey(List<Pair<String, Integer>> expected, List<Pair<String, Integer>> originalList) {
List<Pair<String, Integer>> distinctList = HoodieList.of(originalList).distinctWithKey(Pair::getLeft, 1).collectAsList();
List<Pair<String, Integer>> distinctList = HoodieListData.eager(originalList).distinctWithKey(Pair::getLeft, 1).collectAsList();
assertEquals(expected, distinctList);
}
@Test
void testEagerSemantic() {
List<String> sourceList = Arrays.asList("quick", "brown", "fox");
HoodieListData<String> originalListData = HoodieListData.eager(sourceList);
HoodieData<Integer> lengthsListData = originalListData.map(String::length);
List<Integer> expectedLengths = sourceList.stream().map(String::length).collect(Collectors.toList());
assertEquals(expectedLengths, lengthsListData.collectAsList());
// Here we assert that even though we already de-referenced derivative container,
// we still can dereference its parent (multiple times)
assertEquals(3, originalListData.count());
assertEquals(sourceList, originalListData.collectAsList());
}
}

View File

@@ -22,8 +22,7 @@ package org.apache.hudi.common.data;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.ImmutablePair;
import org.apache.hudi.common.util.collection.Pair;
import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
@@ -37,12 +36,13 @@ import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import static org.apache.hudi.common.util.CollectionUtils.createImmutableList;
import static org.apache.hudi.common.util.CollectionUtils.createImmutableMap;
import static org.junit.jupiter.api.Assertions.assertEquals;
public class TestHoodieMapPair {
public class TestHoodieListDataPairData {
private static final String KEY1 = "key1";
private static final String KEY2 = "key2";
@@ -63,30 +63,30 @@ public class TestHoodieMapPair {
private static final int INTEGER_VALUE4 = 4;
private static final int INTEGER_VALUE5 = 5;
private static List<Pair<String, String>> TEST_PAIRS;
private static HoodiePairData<String, String> TEST_HOODIE_MAP_PAIR;
private List<Pair<String, String>> testPairs;
private HoodiePairData<String, String> testHoodiePairData;
@BeforeAll
public static void setup() {
TEST_PAIRS = constructPairs();
TEST_HOODIE_MAP_PAIR = constructTestMapPairData(TEST_PAIRS);
@BeforeEach
public void setup() {
testPairs = constructPairs();
testHoodiePairData = HoodieListPairData.lazy(testPairs);
}
@Test
public void testKeys() {
assertHoodieDataEquals(Arrays.asList(KEY1, KEY2, KEY3, KEY4), TEST_HOODIE_MAP_PAIR.keys());
assertHoodieDataEquals(Arrays.asList(KEY1, KEY1, KEY2, KEY2, KEY3, KEY4), testHoodiePairData.keys());
}
@Test
public void testValues() {
assertHoodieDataEquals(Arrays.asList(
STRING_VALUE1, STRING_VALUE2, STRING_VALUE3, STRING_VALUE4, STRING_VALUE5, STRING_VALUE6),
TEST_HOODIE_MAP_PAIR.values());
testHoodiePairData.values());
}
@Test
public void testCount() {
assertEquals(6, TEST_HOODIE_MAP_PAIR.count());
assertEquals(6, testHoodiePairData.count());
}
@Test
@@ -97,14 +97,14 @@ public class TestHoodieMapPair {
expectedResultMap.put(KEY3, 1L);
expectedResultMap.put(KEY4, 1L);
assertEquals(expectedResultMap, TEST_HOODIE_MAP_PAIR.countByKey());
assertEquals(expectedResultMap, testHoodiePairData.countByKey());
}
@Test
public void testMap() {
assertHoodieDataEquals(Arrays.asList(
"key1,value1", "key1,value2", "key2,value3", "key2,value4", "key3,value5", "key4,value6"),
TEST_HOODIE_MAP_PAIR.map(pair -> pair.getKey() + "," + pair.getValue()));
testHoodiePairData.map(pair -> pair.getKey() + "," + pair.getValue()));
}
@Test
@@ -114,8 +114,8 @@ public class TestHoodieMapPair {
expectedResultMap.put("key20", Arrays.asList(3, 4));
expectedResultMap.put("key30", Arrays.asList(5));
expectedResultMap.put("key40", Arrays.asList(6));
assertEquals(expectedResultMap, HoodieMapPair.getMapPair(
TEST_HOODIE_MAP_PAIR.mapToPair(
assertEquals(expectedResultMap, toMap(
testHoodiePairData.mapToPair(
pair -> {
String value = pair.getValue();
return new ImmutablePair<>(pair.getKey() + "0",
@@ -129,8 +129,7 @@ public class TestHoodieMapPair {
createImmutableMap(
Pair.of(1, createImmutableList(1001)),
Pair.of(2, createImmutableList(2001)),
Pair.of(3, createImmutableList(3001)),
Pair.of(4, createImmutableList())),
Pair.of(3, createImmutableList(3001))),
createImmutableMap(
Pair.of(1, createImmutableList(1001, 1002, 1003)),
Pair.of(2, createImmutableList(2001, 2002)),
@@ -142,20 +141,20 @@ public class TestHoodieMapPair {
@ParameterizedTest
@MethodSource
public void testReduceByKey(Map<Integer, List<Integer>> expected, Map<Integer, List<Integer>> original) {
HoodiePairData<Integer, Integer> reduced = HoodieMapPair.of(original).reduceByKey((a, b) -> a, 1);
assertEquals(expected, HoodieMapPair.getMapPair(reduced));
HoodiePairData<Integer, Integer> reduced = HoodieListPairData.lazy(original).reduceByKey((a, b) -> a, 1);
assertEquals(expected, toMap(reduced));
}
@Test
public void testLeftOuterJoinSingleValuePerKey() {
HoodiePairData<String, String> pairData1 = constructTestMapPairData(Arrays.asList(
HoodiePairData<String, String> pairData1 = HoodieListPairData.lazy(Arrays.asList(
ImmutablePair.of(KEY1, STRING_VALUE1),
ImmutablePair.of(KEY2, STRING_VALUE2),
ImmutablePair.of(KEY3, STRING_VALUE3),
ImmutablePair.of(KEY4, STRING_VALUE4)
));
HoodiePairData<String, Integer> pairData2 = constructTestMapPairData(Arrays.asList(
HoodiePairData<String, Integer> pairData2 = HoodieListPairData.lazy(Arrays.asList(
ImmutablePair.of(KEY1, INTEGER_VALUE1),
ImmutablePair.of(KEY2, INTEGER_VALUE2),
ImmutablePair.of(KEY5, INTEGER_VALUE3)
@@ -172,12 +171,12 @@ public class TestHoodieMapPair {
ImmutablePair.of(STRING_VALUE4, Option.empty())));
assertEquals(expectedResultMap,
HoodieMapPair.getMapPair(pairData1.leftOuterJoin(pairData2)));
toMap(pairData1.leftOuterJoin(pairData2)));
}
@Test
public void testLeftOuterJoinMultipleValuesPerKey() {
HoodiePairData<String, Integer> otherPairData = constructTestMapPairData(Arrays.asList(
HoodiePairData<String, Integer> otherPairData = HoodieListPairData.lazy(Arrays.asList(
ImmutablePair.of(KEY1, INTEGER_VALUE1),
ImmutablePair.of(KEY2, INTEGER_VALUE2),
ImmutablePair.of(KEY2, INTEGER_VALUE3),
@@ -200,7 +199,25 @@ public class TestHoodieMapPair {
ImmutablePair.of(STRING_VALUE6, Option.empty())));
assertEquals(expectedResultMap,
HoodieMapPair.getMapPair(TEST_HOODIE_MAP_PAIR.leftOuterJoin(otherPairData)));
toMap(testHoodiePairData.leftOuterJoin(otherPairData)));
}
@Test
void testEagerSemantic() {
List<Pair<String, Integer>> sourceList =
Stream.of("quick", "brown", "fox")
.map(s -> Pair.of(s, s.length()))
.collect(Collectors.toList());
HoodieListPairData<String, Integer> originalListData = HoodieListPairData.eager(sourceList);
HoodieData<Integer> lengthsListData = originalListData.values();
List<Integer> expectedLengths = sourceList.stream().map(Pair::getValue).collect(Collectors.toList());
assertEquals(expectedLengths, lengthsListData.collectAsList());
// Here we assert that even though we already de-referenced derivative container,
// we still can dereference its parent (multiple times)
assertEquals(3, originalListData.count());
assertEquals(sourceList, originalListData.collectAsList());
}
private static List<Pair<String, String>> constructPairs() {
@@ -214,11 +231,14 @@ public class TestHoodieMapPair {
);
}
private static <V> HoodiePairData<String, V> constructTestMapPairData(
final List<Pair<String, V>> pairs) {
Map<String, List<V>> map = new HashMap<>();
addPairsToMap(map, pairs);
return HoodieMapPair.of(map);
private static <K,V> Map<K, List<V>> toMap(HoodiePairData<K, V> pairData) {
return ((List<Pair<K, Iterable<V>>>) pairData.groupByKey().get()).stream()
.collect(
Collectors.toMap(
p -> p.getKey(),
p -> StreamSupport.stream(p.getValue().spliterator(), false).collect(Collectors.toList())
)
);
}
private static <V> void addPairsToMap(