[HUDI-4249] Fixing in-memory HoodieData implementation to operate lazily (#5855)
This commit is contained in:
@@ -31,6 +31,7 @@ import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import scala.Tuple2;
|
||||
@@ -41,7 +42,7 @@ import scala.Tuple2;
|
||||
* @param <K> type of key.
|
||||
* @param <V> type of value.
|
||||
*/
|
||||
public class HoodieJavaPairRDD<K, V> extends HoodiePairData<K, V> {
|
||||
public class HoodieJavaPairRDD<K, V> implements HoodiePairData<K, V> {
|
||||
|
||||
private final JavaPairRDD<K, V> pairRDDData;
|
||||
|
||||
@@ -105,8 +106,13 @@ public class HoodieJavaPairRDD<K, V> extends HoodiePairData<K, V> {
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodiePairData<K, V> reduceByKey(SerializableBiFunction<V, V, V> func, int parallelism) {
|
||||
return HoodieJavaPairRDD.of(pairRDDData.reduceByKey(func::apply, parallelism));
|
||||
public HoodiePairData<K, Iterable<V>> groupByKey() {
|
||||
return new HoodieJavaPairRDD<>(pairRDDData.groupByKey());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodiePairData<K, V> reduceByKey(SerializableBiFunction<V, V, V> combiner, int parallelism) {
|
||||
return HoodieJavaPairRDD.of(pairRDDData.reduceByKey(combiner::apply, parallelism));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -130,4 +136,9 @@ public class HoodieJavaPairRDD<K, V> extends HoodiePairData<K, V> {
|
||||
.map(tuple -> new Tuple2<>(tuple._1,
|
||||
new ImmutablePair<>(tuple._2._1, Option.ofNullable(tuple._2._2.orElse(null)))))));
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<Pair<K, V>> collectAsList() {
|
||||
return pairRDDData.map(t -> Pair.of(t._1, t._2)).collect();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -26,6 +26,7 @@ import org.apache.hudi.common.function.SerializableFunction;
|
||||
import org.apache.hudi.common.function.SerializablePairFunction;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
@@ -39,7 +40,7 @@ import scala.Tuple2;
|
||||
*
|
||||
* @param <T> type of object.
|
||||
*/
|
||||
public class HoodieJavaRDD<T> extends HoodieData<T> {
|
||||
public class HoodieJavaRDD<T> implements HoodieData<T> {
|
||||
|
||||
private final JavaRDD<T> rddData;
|
||||
|
||||
@@ -74,17 +75,16 @@ public class HoodieJavaRDD<T> extends HoodieData<T> {
|
||||
* @return the a {@link JavaRDD} of objects in type T.
|
||||
*/
|
||||
public static <T> JavaRDD<T> getJavaRDD(HoodieData<T> hoodieData) {
|
||||
return ((HoodieJavaRDD<T>) hoodieData).get();
|
||||
return ((HoodieJavaRDD<T>) hoodieData).rddData;
|
||||
}
|
||||
|
||||
public static <K, V> JavaPairRDD<K, V> getJavaRDD(HoodiePairData<K, V> hoodieData) {
|
||||
return ((HoodieJavaPairRDD<K, V>) hoodieData).get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<T> get() {
|
||||
return rddData;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void persist(String cacheConfig) {
|
||||
rddData.persist(StorageLevel.fromString(cacheConfig));
|
||||
public void persist(String level) {
|
||||
rddData.persist(StorageLevel.fromString(level));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -112,20 +112,15 @@ public class HoodieJavaRDD<T> extends HoodieData<T> {
|
||||
return HoodieJavaRDD.of(rddData.mapPartitions(func::apply, preservesPartitioning));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> mapPartitions(SerializableFunction<Iterator<T>, Iterator<O>> func) {
|
||||
return HoodieJavaRDD.of(rddData.mapPartitions(func::apply));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func) {
|
||||
return HoodieJavaRDD.of(rddData.flatMap(e -> func.apply(e)));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> mapToPairFunc) {
|
||||
public <K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> func) {
|
||||
return HoodieJavaPairRDD.of(rddData.mapToPair(input -> {
|
||||
Pair<K, V> pair = mapToPairFunc.call(input);
|
||||
Pair<K, V> pair = func.call(input);
|
||||
return new Tuple2<>(pair.getLeft(), pair.getRight());
|
||||
}));
|
||||
}
|
||||
@@ -140,13 +135,6 @@ public class HoodieJavaRDD<T> extends HoodieData<T> {
|
||||
return HoodieJavaRDD.of(rddData.distinct(parallelism));
|
||||
}
|
||||
|
||||
@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 HoodieJavaRDD.of(rddData.filter(filterFunc::apply));
|
||||
@@ -154,7 +142,7 @@ public class HoodieJavaRDD<T> extends HoodieData<T> {
|
||||
|
||||
@Override
|
||||
public HoodieData<T> union(HoodieData<T> other) {
|
||||
return HoodieJavaRDD.of(rddData.union((JavaRDD<T>) other.get()));
|
||||
return HoodieJavaRDD.of(rddData.union(((HoodieJavaRDD<T>) other).rddData));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
Reference in New Issue
Block a user