[HUDI-2497] Refactor clean and restore actions in hudi-client module (#3734)
This commit is contained in:
@@ -22,10 +22,13 @@ import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.function.SerializableBiFunction;
|
||||
import org.apache.hudi.common.function.SerializableConsumer;
|
||||
import org.apache.hudi.common.function.SerializableFunction;
|
||||
import org.apache.hudi.common.function.SerializablePairFlatMapFunction;
|
||||
import org.apache.hudi.common.function.SerializablePairFunction;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
@@ -61,6 +64,10 @@ public abstract class HoodieEngineContext {
|
||||
public abstract <I, K, V> List<V> mapToPairAndReduceByKey(
|
||||
List<I> data, SerializablePairFunction<I, K, V> mapToPairFunc, SerializableBiFunction<V, V, V> reduceFunc, int parallelism);
|
||||
|
||||
public abstract <I, K, V> Stream<ImmutablePair<K, V>> mapPartitionsToPairAndReduceByKey(
|
||||
Stream<I> data, SerializablePairFlatMapFunction<Iterator<I>, K, V> flatMapToPairFunc,
|
||||
SerializableBiFunction<V, V, V> reduceFunc, int parallelism);
|
||||
|
||||
public abstract <I, K, V> List<V> reduceByKey(
|
||||
List<Pair<K, V>> data, SerializableBiFunction<V, V, V> reduceFunc, int parallelism);
|
||||
|
||||
|
||||
@@ -19,15 +19,19 @@
|
||||
package org.apache.hudi.common.engine;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.function.SerializableBiFunction;
|
||||
import org.apache.hudi.common.function.SerializableConsumer;
|
||||
import org.apache.hudi.common.function.SerializableFunction;
|
||||
import org.apache.hudi.common.function.SerializablePairFlatMapFunction;
|
||||
import org.apache.hudi.common.function.SerializablePairFunction;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
@@ -35,6 +39,7 @@ import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
import static org.apache.hudi.common.function.FunctionWrapper.throwingFlatMapToPairWrapper;
|
||||
import static org.apache.hudi.common.function.FunctionWrapper.throwingFlatMapWrapper;
|
||||
import static org.apache.hudi.common.function.FunctionWrapper.throwingForeachWrapper;
|
||||
import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper;
|
||||
@@ -68,6 +73,17 @@ public final class HoodieLocalEngineContext extends HoodieEngineContext {
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, K, V> Stream<ImmutablePair<K, V>> mapPartitionsToPairAndReduceByKey(
|
||||
Stream<I> data, SerializablePairFlatMapFunction<Iterator<I>, K, V> flatMapToPairFunc,
|
||||
SerializableBiFunction<V, V, V> reduceFunc, int parallelism) {
|
||||
return throwingFlatMapToPairWrapper(flatMapToPairFunc).apply(data.parallel().iterator())
|
||||
.collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream()
|
||||
.map(entry -> new ImmutablePair<>(entry.getKey(), entry.getValue().stream().map(
|
||||
Pair::getValue).reduce(throwingReduceWrapper(reduceFunc)).orElse(null)))
|
||||
.filter(Objects::nonNull);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <I, K, V> List<V> reduceByKey(
|
||||
List<Pair<K, V>> data, SerializableBiFunction<V, V, V> reduceFunc, int parallelism) {
|
||||
|
||||
@@ -72,6 +72,17 @@ public class FunctionWrapper {
|
||||
};
|
||||
}
|
||||
|
||||
public static <I, K, V> Function<I, Stream<Pair<K, V>>> throwingFlatMapToPairWrapper(
|
||||
SerializablePairFlatMapFunction<I, K, V> throwingPairFlatMapFunction) {
|
||||
return v1 -> {
|
||||
try {
|
||||
return throwingPairFlatMapFunction.call(v1);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error occurs when executing mapToPair", e);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static <V> BinaryOperator<V> throwingReduceWrapper(SerializableBiFunction<V, V, V> throwingReduceFunction) {
|
||||
return (v1, v2) -> {
|
||||
try {
|
||||
|
||||
@@ -0,0 +1,33 @@
|
||||
/*
|
||||
* 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.function;
|
||||
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* A function that returns a stream of key-value pairs (Pair<K, V>).
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface SerializablePairFlatMapFunction<I, K, V> extends Serializable {
|
||||
Stream<Pair<K, V>> call(I t) throws Exception;
|
||||
}
|
||||
Reference in New Issue
Block a user