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

@@ -97,6 +97,12 @@ public abstract class HoodieData<T> implements Serializable {
*/
public abstract HoodieData<T> distinct();
public abstract HoodieData<T> distinct(int parallelism);
public abstract <O> HoodieData<T> distinctWithKey(SerializableFunction<T, O> keyGetter, int parallelism);
public abstract HoodieData<T> filter(SerializableFunction<T, Boolean> filterFunc);
/**
* Unions this {@link HoodieData} with other {@link HoodieData}.
* @param other {@link HoodieData} of interest.
@@ -108,4 +114,6 @@ public abstract class HoodieData<T> implements Serializable {
* @return collected results in {@link List<T>}.
*/
public abstract List<T> collectAsList();
public abstract HoodieData<T> repartition(int parallelism);
}

View File

@@ -132,6 +132,26 @@ public class HoodieList<T> extends HoodieData<T> {
return HoodieList.of(new ArrayList<>(new HashSet<>(listData)));
}
@Override
public HoodieData<T> distinct(int parallelism) {
return distinct();
}
@Override
public <O> HoodieData<T> distinctWithKey(SerializableFunction<T, O> keyGetter, int parallelism) {
return mapToPair(i -> Pair.of(keyGetter.apply(i), i))
.reduceByKey((value1, value2) -> value1, parallelism)
.values();
}
@Override
public HoodieData<T> filter(SerializableFunction<T, Boolean> filterFunc) {
return HoodieList.of(listData
.stream()
.filter(i -> throwingMapWrapper(filterFunc).apply(i))
.collect(Collectors.toList()));
}
@Override
public HoodieData<T> union(HoodieData<T> other) {
List<T> unionResult = new ArrayList<>();
@@ -144,4 +164,10 @@ public class HoodieList<T> extends HoodieData<T> {
public List<T> collectAsList() {
return listData;
}
@Override
public HoodieData<T> repartition(int parallelism) {
// no op
return this;
}
}

View File

@@ -20,6 +20,7 @@
package org.apache.hudi.common.data;
import org.apache.hudi.common.function.FunctionWrapper;
import org.apache.hudi.common.function.SerializableBiFunction;
import org.apache.hudi.common.function.SerializableFunction;
import org.apache.hudi.common.function.SerializablePairFunction;
import org.apache.hudi.common.util.Option;
@@ -27,6 +28,7 @@ import org.apache.hudi.common.util.collection.ImmutablePair;
import org.apache.hudi.common.util.collection.Pair;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -110,6 +112,15 @@ public class HoodieMapPair<K, V> extends HoodiePairData<K, V> {
Collectors.toMap(Map.Entry::getKey, entry -> (long) entry.getValue().size()));
}
@Override
public HoodiePairData<K, V> reduceByKey(SerializableBiFunction<V, V, V> func, int parallelism) {
return HoodieMapPair.of(mapPairData.entrySet().stream()
.collect(Collectors.toMap(Map.Entry::getKey, e -> {
Option<V> reducedValue = Option.fromJavaOptional(e.getValue().stream().reduce(func::apply));
return reducedValue.isPresent() ? Collections.singletonList(reducedValue.get()) : Collections.emptyList();
})));
}
@Override
public <O> HoodieData<O> map(SerializableFunction<Pair<K, V>, O> func) {
Function<Pair<K, V>, O> throwableFunc = throwingMapWrapper(func);

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.common.data;
import org.apache.hudi.common.function.SerializableBiFunction;
import org.apache.hudi.common.function.SerializableFunction;
import org.apache.hudi.common.function.SerializablePairFunction;
import org.apache.hudi.common.util.Option;
@@ -72,6 +73,8 @@ public abstract class HoodiePairData<K, V> implements Serializable {
*/
public abstract Map<K, Long> countByKey();
public abstract HoodiePairData<K, V> reduceByKey(SerializableBiFunction<V, V, V> func, int parallelism);
/**
* @param func serializable map function.
* @param <O> output object type.