1
0

[HUDI-2502] Refactor index in hudi-client module (#3778)

- Refactor Index to reduce Line of Code and re-use across engines.
This commit is contained in:
Y Ethan Guo
2021-10-28 01:16:00 -07:00
committed by GitHub
parent e5b6b8602c
commit 0223c442ec
70 changed files with 2196 additions and 1567 deletions

View File

@@ -20,6 +20,7 @@
package org.apache.hudi.common.data;
import org.apache.hudi.common.function.SerializableFunction;
import org.apache.hudi.common.function.SerializablePairFunction;
import java.io.Serializable;
import java.util.Iterator;
@@ -37,11 +38,28 @@ public abstract class HoodieData<T> implements Serializable {
*/
public abstract Object get();
/**
* Caches the data.
*
* @param cacheConfig config value for caching.
*/
public abstract void persist(String cacheConfig);
/**
* Removes the cached data.
*/
public abstract void unpersist();
/**
* @return whether the collection is empty.
*/
public abstract boolean isEmpty();
/**
* @return the number of objects.
*/
public abstract long count();
/**
* @param func serializable map function.
* @param <O> output object type.
@@ -49,6 +67,16 @@ public abstract class HoodieData<T> implements Serializable {
*/
public abstract <O> HoodieData<O> map(SerializableFunction<T, O> func);
/**
* @param func serializable map function by taking a partition of objects
* and generating an iterator.
* @param preservesPartitioning whether to preserve partitions in the result.
* @param <O> output object type.
* @return {@link HoodieData<O>} containing the result. Actual execution may be deferred.
*/
public abstract <O> HoodieData<O> mapPartitions(
SerializableFunction<Iterator<T>, Iterator<O>> func, boolean preservesPartitioning);
/**
* @param func serializable flatmap function.
* @param <O> output object type.
@@ -56,6 +84,19 @@ public abstract class HoodieData<T> implements Serializable {
*/
public abstract <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func);
/**
* @param mapToPairFunc serializable map function to generate a pair.
* @param <K> key type of the pair.
* @param <V> value type of the pair.
* @return {@link HoodiePairData<K, V>} containing the result. Actual execution may be deferred.
*/
public abstract <K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> mapToPairFunc);
/**
* @return distinct objects in {@link HoodieData}.
*/
public abstract HoodieData<T> distinct();
/**
* @return collected results in {@link List<T>}.
*/

View File

@@ -20,13 +20,19 @@
package org.apache.hudi.common.data;
import org.apache.hudi.common.function.SerializableFunction;
import org.apache.hudi.common.function.SerializablePairFunction;
import org.apache.hudi.common.util.collection.Pair;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper;
import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper;
/**
@@ -65,17 +71,39 @@ public class HoodieList<T> extends HoodieData<T> {
return listData;
}
@Override
public void persist(String cacheConfig) {
// No OP
}
@Override
public void unpersist() {
// No OP
}
@Override
public boolean isEmpty() {
return listData.isEmpty();
}
@Override
public long count() {
return listData.size();
}
@Override
public <O> HoodieData<O> map(SerializableFunction<T, O> func) {
return HoodieList.of(listData.stream().parallel()
.map(throwingMapWrapper(func)).collect(Collectors.toList()));
}
@Override
public <O> HoodieData<O> mapPartitions(SerializableFunction<Iterator<T>, Iterator<O>> func, boolean preservesPartitioning) {
List<O> result = new ArrayList<>();
throwingMapWrapper(func).apply(listData.iterator()).forEachRemaining(result::add);
return HoodieList.of(result);
}
@Override
public <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func) {
Function<T, Iterator<O>> throwableFunc = throwingMapWrapper(func);
@@ -87,6 +115,23 @@ public class HoodieList<T> extends HoodieData<T> {
}).collect(Collectors.toList()));
}
@Override
public <K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> mapToPairFunc) {
Map<K, List<V>> mapOfPairs = new HashMap<>();
Function<T, Pair<K, V>> throwableMapToPairFunc = throwingMapToPairWrapper(mapToPairFunc);
listData.forEach(data -> {
Pair<K, V> pair = throwableMapToPairFunc.apply(data);
List<V> list = mapOfPairs.computeIfAbsent(pair.getKey(), k -> new ArrayList<>());
list.add(pair.getValue());
});
return HoodieMapPair.of(mapOfPairs);
}
@Override
public HoodieData<T> distinct() {
return HoodieList.of(new ArrayList<>(new HashSet<>(listData)));
}
@Override
public List<T> collectAsList() {
return listData;

View File

@@ -0,0 +1,157 @@
/*
* 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.function.FunctionWrapper;
import org.apache.hudi.common.function.SerializableFunction;
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.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper;
/**
* Implementation of {@link HoodiePairData} using Java {@link Map}.
* The pairs are organized by the key in the Map and values for the same key
* are stored in a list as the value corresponding to the key in the Map.
*
* @param <K> type of key.
* @param <V> type of value.
*/
public class HoodieMapPair<K, V> extends HoodiePairData<K, V> {
private final Map<K, List<V>> mapPairData;
private HoodieMapPair(Map<K, List<V>> mapPairData) {
this.mapPairData = mapPairData;
}
/**
* @param mapPairData a {@link Map} of pairs.
* @param <K> type of key.
* @param <V> type of value.
* @return a new instance containing the {@link Map<K, List<V>>} reference.
*/
public static <K, V> HoodieMapPair<K, V> of(Map<K, List<V>> mapPairData) {
return new HoodieMapPair<>(mapPairData);
}
/**
* @param hoodiePairData {@link HoodieMapPair <K, V>} instance containing the {@link Map} of pairs.
* @param <K> type of key.
* @param <V> type of value.
* @return the {@link Map} of pairs.
*/
public static <K, V> Map<K, List<V>> getMapPair(HoodiePairData<K, V> hoodiePairData) {
return ((HoodieMapPair<K, V>) hoodiePairData).get();
}
@Override
public Map<K, List<V>> get() {
return mapPairData;
}
@Override
public void persist(String cacheConfig) {
// No OP
}
@Override
public void unpersist() {
// No OP
}
@Override
public HoodieData<K> keys() {
return HoodieList.of(new ArrayList<>(mapPairData.keySet()));
}
@Override
public HoodieData<V> values() {
return HoodieList.of(
mapPairData.values().stream().flatMap(List::stream).collect(Collectors.toList()));
}
@Override
public long count() {
return mapPairData.values().stream().map(
list -> (long) list.size()).reduce(Long::sum).orElse(0L);
}
@Override
public Map<K, Long> countByKey() {
return mapPairData.entrySet().stream().collect(
Collectors.toMap(Map.Entry::getKey, entry -> (long) entry.getValue().size()));
}
@Override
public <O> HoodieData<O> map(SerializableFunction<Pair<K, V>, O> func) {
Function<Pair<K, V>, O> throwableFunc = throwingMapWrapper(func);
return HoodieList.of(
streamAllPairs().map(throwableFunc).collect(Collectors.toList()));
}
@Override
public <L, W> HoodiePairData<L, W> mapToPair(SerializablePairFunction<Pair<K, V>, L, W> mapToPairFunc) {
Map<L, List<W>> newMap = new HashMap<>();
Function<Pair<K, V>, Pair<L, W>> throwableMapToPairFunc =
FunctionWrapper.throwingMapToPairWrapper(mapToPairFunc);
streamAllPairs().map(pair -> throwableMapToPairFunc.apply(pair)).forEach(newPair -> {
List<W> list = newMap.computeIfAbsent(newPair.getKey(), k -> new ArrayList<>());
list.add(newPair.getValue());
});
return HoodieMapPair.of(newMap);
}
@Override
public <W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> other) {
Map<K, List<W>> otherMapPairData = HoodieMapPair.getMapPair(other);
Stream<ImmutablePair<K, ImmutablePair<V, Option<List<W>>>>> pairs = streamAllPairs()
.map(pair -> new ImmutablePair<>(pair.getKey(), new ImmutablePair<>(
pair.getValue(), Option.ofNullable(otherMapPairData.get(pair.getKey())))));
Map<K, List<Pair<V, Option<W>>>> resultMap = new HashMap<>();
pairs.forEach(pair -> {
K key = pair.getKey();
ImmutablePair<V, Option<List<W>>> valuePair = pair.getValue();
List<Pair<V, Option<W>>> resultList = resultMap.computeIfAbsent(key, k -> new ArrayList<>());
if (!valuePair.getRight().isPresent()) {
resultList.add(new ImmutablePair<>(valuePair.getLeft(), Option.empty()));
} else {
resultList.addAll(valuePair.getRight().get().stream().map(
w -> new ImmutablePair<>(valuePair.getLeft(), Option.of(w))).collect(Collectors.toList()));
}
});
return HoodieMapPair.of(resultMap);
}
private Stream<ImmutablePair<K, V>> streamAllPairs() {
return mapPairData.entrySet().stream().flatMap(
entry -> entry.getValue().stream().map(e -> new ImmutablePair<>(entry.getKey(), e)));
}
}

View File

@@ -0,0 +1,102 @@
/*
* 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.function.SerializableFunction;
import org.apache.hudi.common.function.SerializablePairFunction;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import java.io.Serializable;
import java.util.Map;
/**
* An abstraction for pairs of key in type K and value in type V to store the reference
* and do transformation.
*
* @param <K> type of key.
* @param <V> type of value.
*/
public abstract class HoodiePairData<K, V> implements Serializable {
/**
* @return the collection of pairs.
*/
public abstract Object get();
/**
* Caches the data.
*
* @param cacheConfig config value for caching.
*/
public abstract void persist(String cacheConfig);
/**
* Removes the cached data.
*/
public abstract void unpersist();
/**
* @return all keys in {@link HoodieData}.
*/
public abstract HoodieData<K> keys();
/**
* @return all values in {@link HoodieData}.
*/
public abstract HoodieData<V> values();
/**
* @return the number of pairs.
*/
public abstract long count();
/**
* @return the number of pairs per key in a {@link Map}.
*/
public abstract Map<K, Long> countByKey();
/**
* @param func serializable map function.
* @param <O> output object type.
* @return {@link HoodieData<O>} containing the result. Actual execution may be deferred.
*/
public abstract <O> HoodieData<O> map(SerializableFunction<Pair<K, V>, O> func);
/**
* @param mapToPairFunc serializable map function to generate another pair.
* @param <L> new key type.
* @param <W> new value type.
* @return {@link HoodiePairData<L, W>} containing the result. Actual execution may be deferred.
*/
public abstract <L, W> HoodiePairData<L, W> mapToPair(
SerializablePairFunction<Pair<K, V>, L, W> mapToPairFunc);
/**
* Performs a left outer join of this and other. For each element (k, v) in this,
* the resulting HoodiePairData will either contain all pairs (k, (v, Some(w))) for w in other,
* or the pair (k, (v, None)) if no elements in other have key k.
*
* @param other the other {@link HoodiePairData}
* @param <W> value type of the other {@link HoodiePairData}
* @return {@link HoodiePairData<K, Pair<V, Option<W>>>} containing the left outer join result.
* Actual execution may be deferred.
*/
public abstract <W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> other);
}

View File

@@ -65,7 +65,11 @@ public abstract class HoodieEngineContext {
public abstract <T> HoodieData<T> emptyHoodieData();
public abstract <T> HoodieData<T> parallelize(List<T> data);
public <T> HoodieData<T> parallelize(List<T> data) {
return parallelize(data, data.size());
}
public abstract <T> HoodieData<T> parallelize(List<T> data, int parallelism);
public abstract <I, O> List<O> map(List<I> data, SerializableFunction<I, O> func, int parallelism);

View File

@@ -75,7 +75,7 @@ public final class HoodieLocalEngineContext extends HoodieEngineContext {
}
@Override
public <T> HoodieData<T> parallelize(List<T> data) {
public <T> HoodieData<T> parallelize(List<T> data, int parallelism) {
return HoodieList.of(data);
}