1
0

[HUDI-2439] Replace RDD with HoodieData in HoodieSparkTable and commit executors (#4856)

- Adopt HoodieData in Spark action commit executors
- Make Spark independent DeleteHelper, WriteHelper, MergeHelper in hudi-client-common
- Make HoodieTable in WriteClient APIs have raw type to decouple with Client's generic types
This commit is contained in:
Raymond Xu
2022-03-17 19:17:56 +08:00
committed by GitHub
parent bf191f8d46
commit 7446ff95a7
69 changed files with 723 additions and 769 deletions

View File

@@ -0,0 +1,50 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.hudi.common.data;
import org.apache.hudi.common.util.collection.Pair;
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.Stream;
import static org.junit.jupiter.api.Assertions.assertEquals;
class TestHoodieList {
private static Stream<Arguments> distinctWithKey() {
return Stream.of(
Arguments.of(
Arrays.asList(Pair.of("k1", 1), Pair.of("k2", 2)),
Arrays.asList(Pair.of("k1", 1), Pair.of("k1", 10), Pair.of("k1", 100), Pair.of("k2", 2)))
);
}
@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();
assertEquals(expected, distinctList);
}
}

View File

@@ -25,6 +25,9 @@ import org.apache.hudi.common.util.collection.Pair;
import org.junit.jupiter.api.BeforeAll;
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.ArrayList;
import java.util.Arrays;
@@ -33,7 +36,10 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
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 {
@@ -117,6 +123,29 @@ public class TestHoodieMapPair {
})));
}
private static Stream<Arguments> testReduceByKey() {
return Stream.of(
Arguments.of(
createImmutableMap(
Pair.of(1, createImmutableList(1001)),
Pair.of(2, createImmutableList(2001)),
Pair.of(3, createImmutableList(3001)),
Pair.of(4, createImmutableList())),
createImmutableMap(
Pair.of(1, createImmutableList(1001, 1002, 1003)),
Pair.of(2, createImmutableList(2001, 2002)),
Pair.of(3, createImmutableList(3001)),
Pair.of(4, createImmutableList())))
);
}
@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));
}
@Test
public void testLeftOuterJoinSingleValuePerKey() {
HoodiePairData<String, String> pairData1 = constructTestMapPairData(Arrays.asList(