1
0

[HUDI-4249] Fixing in-memory HoodieData implementation to operate lazily (#5855)

This commit is contained in:
Alexey Kudinkin
2022-07-16 16:26:48 -07:00
committed by GitHub
parent 80368a049d
commit 4bda6afe0b
35 changed files with 868 additions and 555 deletions

View File

@@ -0,0 +1,72 @@
/*
* 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.Either;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public abstract class HoodieBaseListData<T> {
protected final Either<Stream<T>, List<T>> data;
protected final boolean lazy;
protected HoodieBaseListData(List<T> data, boolean lazy) {
this.data = lazy ? Either.left(data.stream().parallel()) : Either.right(data);
this.lazy = lazy;
}
protected HoodieBaseListData(Stream<T> dataStream, boolean lazy) {
// NOTE: In case this container is being instantiated by an eager parent, we have to
// pre-materialize the stream
this.data = lazy ? Either.left(dataStream) : Either.right(dataStream.collect(Collectors.toList()));
this.lazy = lazy;
}
protected Stream<T> asStream() {
return lazy ? data.asLeft() : data.asRight().parallelStream();
}
protected boolean isEmpty() {
if (lazy) {
return data.asLeft().findAny().isPresent();
} else {
return data.asRight().isEmpty();
}
}
protected long count() {
if (lazy) {
return data.asLeft().count();
} else {
return data.asRight().size();
}
}
protected List<T> collectAsList() {
if (lazy) {
return data.asLeft().collect(Collectors.toList());
} else {
return data.asRight();
}
}
}

View File

@@ -21,108 +21,163 @@ 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.io.Serializable;
import java.util.Iterator;
import java.util.List;
/**
* An abstraction for a data collection of objects in type T to store the reference
* and do transformation.
* An interface abstracting a container holding a collection of objects of type {@code T}
* allowing to perform common transformation on it.
*
* @param <T> type of object.
* This abstraction provides common API implemented by
* <ol>
* <li>In-memory implementation ({@code HoodieListData}, {@code HoodieListPairData}), where all objects
* are held in-memory by the executing process</li>
* <li>RDD-based implementation ({@code HoodieJavaRDD}, etc)</li>, where underlying collection is held
* by an RDD allowing to execute transformations using Spark engine on the cluster
* </ol>
*
* All implementations provide for consistent semantic, where
* <ul>
* <li>All non-terminal* operations are executed lazily (for ex, {@code map}, {@code filter}, etc)</li>
* <li>All terminal operations are executed eagerly, executing all previously accumulated transformations.
* Note that, collection could not be re-used after invoking terminal operation on it.</li>
* </ul>
*
* @param <T> type of object
*/
public abstract class HoodieData<T> implements Serializable {
/**
* @return the collection of objects.
*/
public abstract Object get();
public interface HoodieData<T> extends Serializable {
/**
* Caches the data.
* Persists the data w/ provided {@code level} (if applicable)
*/
void persist(String level);
/**
* Un-persists the data (if previously persisted)
*/
void unpersist();
/**
* Returns whether the collection is empty.
*/
boolean isEmpty();
/**
* Returns number of objects held in the collection
*
* @param cacheConfig config value for caching.
* NOTE: This is a terminal operation
*/
public abstract void persist(String cacheConfig);
long count();
/**
* Removes the cached data.
* Maps every element in the collection using provided mapping {@code func}.
*
* This is an intermediate operation
*
* @param func serializable map function
* @param <O> output object type
* @return {@link HoodieData<O>} holding mapped elements
*/
public abstract void unpersist();
<O> HoodieData<O> map(SerializableFunction<T, O> func);
/**
* @return whether the collection is empty.
* Maps every element in the collection's partition (if applicable) by applying provided
* mapping {@code func} to every collection's partition
*
* This is an intermediate operation
*
* @param func serializable map function accepting {@link Iterator} of a single
* partition's elements and returning a new {@link Iterator} mapping
* every element of the partition into a new one
* @param preservesPartitioning whether to preserve partitioning in the resulting collection
* @param <O> output object type
* @return {@link HoodieData<O>} holding mapped elements
*/
public abstract boolean isEmpty();
<O> HoodieData<O> mapPartitions(SerializableFunction<Iterator<T>,
Iterator<O>> func, boolean preservesPartitioning);
/**
* @return the number of objects.
* Maps every element in the collection into a collection of the new elements (provided by
* {@link Iterator}) using provided mapping {@code func}, subsequently flattening the result
* (by concatenating) into a single collection
*
* This is an intermediate operation
*
* @param func serializable function mapping every element {@link T} into {@code Iterator<O>}
* @param <O> output object type
* @return {@link HoodieData<O>} holding mapped elements
*/
public abstract long count();
<O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func);
/**
* @param func serializable map function.
* @param <O> output object type.
* @return {@link HoodieData<O>} containing the result. Actual execution may be deferred.
* Maps every element in the collection using provided mapping {@code func} into a {@link Pair<K, V>}
* of elements {@code K} and {@code V}
* <p>
* This is an intermediate operation
*
* @param func serializable map function
* @param <K> key type of the pair
* @param <V> value type of the pair
* @return {@link HoodiePairData<K, V>} holding mapped elements
*/
public abstract <O> HoodieData<O> map(SerializableFunction<T, O> func);
<K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> 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.
* Returns new {@link HoodieData} collection holding only distinct objects of the original one
*
* This is a stateful intermediate operation
*/
public abstract <O> HoodieData<O> mapPartitions(
SerializableFunction<Iterator<T>, Iterator<O>> func, boolean preservesPartitioning);
HoodieData<T> distinct();
/**
* @param func serializable map function by taking a partition of objects
* and generating an iterator.
* @param <O> output object type.
* @return {@link HoodieData<O>} containing the result. Actual execution may be deferred.
* Returns new {@link HoodieData} collection holding only distinct objects of the original one
*
* This is a stateful intermediate operation
*/
public abstract <O> HoodieData<O> mapPartitions(
SerializableFunction<Iterator<T>, Iterator<O>> func);
HoodieData<T> distinct(int parallelism);
/**
* @param func serializable flatmap function.
* @param <O> output object type.
* @return {@link HoodieData<O>} containing the result. Actual execution may be deferred.
* Returns new instance of {@link HoodieData} collection only containing elements matching provided
* {@code filterFunc} (ie ones it returns true on)
*
* @param filterFunc filtering func either accepting or rejecting the elements
* @return {@link HoodieData<T>} holding filtered elements
*/
public abstract <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func);
HoodieData<T> filter(SerializableFunction<T, Boolean> filterFunc);
/**
* @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.
* Unions {@link HoodieData} with another instance of {@link HoodieData}.
* Note that, it's only able to union same underlying collection implementations.
*
* This is a stateful intermediate operation
*
* @param other {@link HoodieData} collection
* @return {@link HoodieData<T>} holding superset of elements of this and {@code other} collections
*/
public abstract <K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> mapToPairFunc);
HoodieData<T> union(HoodieData<T> other);
/**
* @return distinct objects in {@link HoodieData}.
* Collects results of the underlying collection into a {@link List<T>}
*
* This is a terminal operation
*/
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);
List<T> collectAsList();
/**
* Unions this {@link HoodieData} with other {@link HoodieData}.
* @param other {@link HoodieData} of interest.
* @return the union of two as as instance of {@link HoodieData}.
* Re-partitions underlying collection (if applicable) making sure new {@link HoodieData} has
* exactly {@code parallelism} partitions
*
* @param parallelism target number of partitions in the underlying collection
* @return {@link HoodieData<T>} holding re-partitioned collection
*/
public abstract HoodieData<T> union(HoodieData<T> other);
HoodieData<T> repartition(int parallelism);
/**
* @return collected results in {@link List<T>}.
*/
public abstract List<T> collectAsList();
public abstract HoodieData<T> repartition(int parallelism);
default <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();
}
}

View File

@@ -1,178 +0,0 @@
/*
* 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.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;
/**
* Holds a {@link List} of objects.
*
* @param <T> type of object.
*/
public class HoodieList<T> extends HoodieData<T> {
private final List<T> listData;
private HoodieList(List<T> listData) {
this.listData = listData;
}
/**
* @param listData a {@link List} of objects in type T.
* @param <T> type of object.
* @return a new instance containing the {@link List<T>} reference.
*/
public static <T> HoodieList<T> of(List<T> listData) {
return new HoodieList<>(listData);
}
/**
* @param hoodieData {@link HoodieList <T>} instance containing the {@link List} of objects.
* @param <T> type of object.
* @return the a {@link List} of objects in type T.
*/
public static <T> List<T> getList(HoodieData<T> hoodieData) {
return ((HoodieList<T>) hoodieData).get();
}
@Override
public List<T> get() {
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) {
return mapPartitions(func);
}
@Override
public <O> HoodieData<O> mapPartitions(SerializableFunction<Iterator<T>, Iterator<O>> func) {
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);
return HoodieList.of(listData.stream().flatMap(e -> {
List<O> result = new ArrayList<>();
Iterator<O> iterator = throwableFunc.apply(e);
iterator.forEachRemaining(result::add);
return result.stream();
}).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 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<>();
unionResult.addAll(listData);
unionResult.addAll(other.collectAsList());
return HoodieList.of(unionResult);
}
@Override
public List<T> collectAsList() {
return listData;
}
@Override
public HoodieData<T> repartition(int parallelism) {
// no op
return this;
}
}

View File

@@ -0,0 +1,182 @@
/*
* 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.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import java.util.Iterator;
import java.util.List;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.function.Function;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper;
import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper;
/**
* In-memory implementation of {@link HoodieData} holding internally a {@link Stream} of objects.
*
* {@link HoodieListData} can have either of the 2 execution semantics:
*
* <ol>
* <li>Eager: with every operation being executed right away</li>
* <li>Lazy: with every operation being "stacked up", with it execution postponed until
* "terminal" operation is invoked</li>
* </ol>
*
* NOTE: This is an in-memory counterpart for {@code HoodieJavaRDD}, and it strives to provide
* similar semantic as RDD container -- all intermediate (non-terminal, not de-referencing
* the stream like "collect", "groupBy", etc) operations are executed *lazily*.
* This allows to make sure that compute/memory churn is minimal since only necessary
* computations will ultimately be performed.
*
* Please note, however, that while RDD container allows the same collection to be
* de-referenced more than once (ie terminal operation invoked more than once),
* {@link HoodieListData} allows that only when instantiated w/ an eager execution semantic.
*
* @param <T> type of object.
*/
public class HoodieListData<T> extends HoodieBaseListData<T> implements HoodieData<T> {
private HoodieListData(List<T> data, boolean lazy) {
super(data, lazy);
}
HoodieListData(Stream<T> dataStream, boolean lazy) {
super(dataStream, lazy);
}
/**
* Creates instance of {@link HoodieListData} bearing *eager* execution semantic
*
* @param listData a {@link List} of objects in type T
* @param <T> type of object
* @return a new instance containing the {@link List<T>} reference
*/
public static <T> HoodieListData<T> eager(List<T> listData) {
return new HoodieListData<>(listData, false);
}
/**
* Creates instance of {@link HoodieListData} bearing *lazy* execution semantic
*
* @param listData a {@link List} of objects in type T
* @param <T> type of object
* @return a new instance containing the {@link List<T>} reference
*/
public static <T> HoodieListData<T> lazy(List<T> listData) {
return new HoodieListData<>(listData, true);
}
@Override
public void persist(String level) {
// No OP
}
@Override
public void unpersist() {
// No OP
}
@Override
public <O> HoodieData<O> map(SerializableFunction<T, O> func) {
return new HoodieListData<>(asStream().map(throwingMapWrapper(func)), lazy);
}
@Override
public <O> HoodieData<O> mapPartitions(SerializableFunction<Iterator<T>, Iterator<O>> func, boolean preservesPartitioning) {
Function<Iterator<T>, Iterator<O>> mapper = throwingMapWrapper(func);
return new HoodieListData<>(
StreamSupport.stream(
Spliterators.spliteratorUnknownSize(
mapper.apply(asStream().iterator()), Spliterator.ORDERED), true),
lazy
);
}
@Override
public <O> HoodieData<O> flatMap(SerializableFunction<T, Iterator<O>> func) {
Function<T, Iterator<O>> mapper = throwingMapWrapper(func);
Stream<O> mappedStream = asStream().flatMap(e ->
StreamSupport.stream(
Spliterators.spliteratorUnknownSize(mapper.apply(e), Spliterator.ORDERED), true));
return new HoodieListData<>(mappedStream, lazy);
}
@Override
public <K, V> HoodiePairData<K, V> mapToPair(SerializablePairFunction<T, K, V> func) {
Function<T, Pair<K, V>> throwableMapToPairFunc = throwingMapToPairWrapper(func);
return new HoodieListPairData<>(asStream().map(throwableMapToPairFunc), lazy);
}
@Override
public HoodieData<T> distinct() {
return new HoodieListData<>(asStream().distinct(), lazy);
}
@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 new HoodieListData<>(asStream().filter(r -> throwingMapWrapper(filterFunc).apply(r)), lazy);
}
@Override
public HoodieData<T> union(HoodieData<T> other) {
ValidationUtils.checkArgument(other instanceof HoodieListData);
return new HoodieListData<>(Stream.concat(asStream(), ((HoodieListData<T>)other).asStream()), lazy);
}
@Override
public HoodieData<T> repartition(int parallelism) {
// no op
return this;
}
@Override
public boolean isEmpty() {
return super.isEmpty();
}
@Override
public long count() {
return super.count();
}
@Override
public List<T> collectAsList() {
return super.collectAsList();
}
}

View File

@@ -0,0 +1,201 @@
/*
* 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.SerializableBiFunction;
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.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collector;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.function.FunctionWrapper.throwingMapToPairWrapper;
import static org.apache.hudi.common.function.FunctionWrapper.throwingMapWrapper;
/**
* In-memory implementation of {@link HoodiePairData} holding internally a {@link Stream} of {@link Pair}s.
*
* {@link HoodieListData} can have either of the 2 execution semantics:
*
* <ol>
* <li>Eager: with every operation being executed right away</li>
* <li>Lazy: with every operation being "stacked up", with it execution postponed until
* "terminal" operation is invoked</li>
* </ol>
*
*
* NOTE: This is an in-memory counterpart for {@code HoodieJavaPairRDD}, and it strives to provide
* similar semantic as RDD container -- all intermediate (non-terminal, not de-referencing
* the stream like "collect", "groupBy", etc) operations are executed *lazily*.
* This allows to make sure that compute/memory churn is minimal since only necessary
* computations will ultimately be performed.
*
* Please note, however, that while RDD container allows the same collection to be
* de-referenced more than once (ie terminal operation invoked more than once),
* {@link HoodieListData} allows that only when instantiated w/ an eager execution semantic.
*
* @param <K> type of the key in the pair
* @param <V> type of the value in the pair
*/
public class HoodieListPairData<K, V> extends HoodieBaseListData<Pair<K, V>> implements HoodiePairData<K, V> {
private HoodieListPairData(List<Pair<K, V>> data, boolean lazy) {
super(data, lazy);
}
HoodieListPairData(Stream<Pair<K, V>> dataStream, boolean lazy) {
super(dataStream, lazy);
}
@Override
public List<Pair<K, V>> get() {
return collectAsList();
}
@Override
public void persist(String cacheConfig) {
// no-op
}
@Override
public void unpersist() {
// no-op
}
@Override
public HoodieData<K> keys() {
return new HoodieListData<>(asStream().map(Pair::getKey), lazy);
}
@Override
public HoodieData<V> values() {
return new HoodieListData<>(asStream().map(Pair::getValue), lazy);
}
@Override
public Map<K, Long> countByKey() {
return asStream().collect(Collectors.groupingBy(Pair::getKey, Collectors.counting()));
}
@Override
public HoodiePairData<K, Iterable<V>> groupByKey() {
Collector<Pair<K, V>, ?, List<V>> mappingCollector = Collectors.mapping(Pair::getValue, Collectors.toList());
Collector<Pair<K, V>, ?, Map<K, List<V>>> groupingCollector =
Collectors.groupingBy(Pair::getKey, mappingCollector);
Map<K, List<V>> groupedByKey = asStream().collect(groupingCollector);
return new HoodieListPairData<>(
groupedByKey.entrySet().stream().map(e -> Pair.of(e.getKey(), e.getValue())),
lazy
);
}
@Override
public HoodiePairData<K, V> reduceByKey(SerializableBiFunction<V, V, V> combiner, int parallelism) {
Map<K, java.util.Optional<V>> reducedMap = asStream().collect(
Collectors.groupingBy(
Pair::getKey,
HashMap::new,
Collectors.mapping(Pair::getValue, Collectors.reducing(combiner::apply))));
return new HoodieListPairData<>(
reducedMap.entrySet()
.stream()
.map(e -> Pair.of(e.getKey(), e.getValue().orElse(null))),
lazy
);
}
@Override
public <O> HoodieData<O> map(SerializableFunction<Pair<K, V>, O> func) {
Function<Pair<K, V>, O> uncheckedMapper = throwingMapWrapper(func);
return new HoodieListData<>(asStream().map(uncheckedMapper), lazy);
}
@Override
public <L, W> HoodiePairData<L, W> mapToPair(SerializablePairFunction<Pair<K, V>, L, W> mapToPairFunc) {
return new HoodieListPairData<>(asStream().map(p -> throwingMapToPairWrapper(mapToPairFunc).apply(p)), lazy);
}
@Override
public <W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> other) {
ValidationUtils.checkArgument(other instanceof HoodieListPairData);
// Transform right-side container to a multi-map of [[K]] to [[List<W>]] values
HashMap<K, List<W>> rightStreamMap = ((HoodieListPairData<K, W>) other).asStream().collect(
Collectors.groupingBy(
Pair::getKey,
HashMap::new,
Collectors.mapping(Pair::getValue, Collectors.toList())));
Stream<Pair<K, Pair<V, Option<W>>>> leftOuterJoined = asStream().flatMap(pair -> {
K key = pair.getKey();
V leftValue = pair.getValue();
List<W> rightValues = rightStreamMap.get(key);
if (rightValues == null) {
return Stream.of(Pair.of(key, Pair.of(leftValue, Option.empty())));
} else {
return rightValues.stream().map(rightValue ->
Pair.of(key, Pair.of(leftValue, Option.of(rightValue))));
}
});
return new HoodieListPairData<>(leftOuterJoined, lazy);
}
@Override
public long count() {
return super.count();
}
@Override
public List<Pair<K, V>> collectAsList() {
return super.collectAsList();
}
public static <K, V> HoodieListPairData<K, V> lazy(List<Pair<K, V>> data) {
return new HoodieListPairData<>(data, true);
}
public static <K, V> HoodieListPairData<K, V> eager(List<Pair<K, V>> data) {
return new HoodieListPairData<>(data, false);
}
public static <K, V> HoodieListPairData<K, V> lazy(Map<K, List<V>> data) {
return new HoodieListPairData<>(explode(data), true);
}
public static <K, V> HoodieListPairData<K, V> eager(Map<K, List<V>> data) {
return new HoodieListPairData<>(explode(data), false);
}
private static <K, V> Stream<Pair<K, V>> explode(Map<K, List<V>> data) {
return data.entrySet().stream()
.flatMap(e -> e.getValue().stream().map(v -> Pair.of(e.getKey(), v)));
}
}

View File

@@ -1,168 +0,0 @@
/*
* 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.SerializableBiFunction;
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.Collections;
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 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);
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

@@ -26,6 +26,7 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
/**
@@ -35,71 +36,92 @@ import java.util.Map;
* @param <K> type of key.
* @param <V> type of value.
*/
public abstract class HoodiePairData<K, V> implements Serializable {
public interface HoodiePairData<K, V> extends Serializable {
/**
* @return the collection of pairs.
*/
public abstract Object get();
Object get();
/**
* Caches the data.
* Persists the data (if applicable)
*
* @param cacheConfig config value for caching.
*/
public abstract void persist(String cacheConfig);
void persist(String cacheConfig);
/**
* Removes the cached data.
* Un-persists the data (if applicable)
*/
public abstract void unpersist();
void unpersist();
/**
* @return all keys in {@link HoodieData}.
* Returns a {@link HoodieData} holding the key from every corresponding pair
*/
public abstract HoodieData<K> keys();
HoodieData<K> keys();
/**
* @return all values in {@link HoodieData}.
* Returns a {@link HoodieData} holding the value from every corresponding pair
*/
public abstract HoodieData<V> values();
HoodieData<V> values();
/**
* @return the number of pairs.
* Returns number of held pairs
*/
public abstract long count();
long count();
/**
* @return the number of pairs per key in a {@link Map}.
* Counts the number of pairs grouping them by key
*/
public abstract Map<K, Long> countByKey();
Map<K, Long> countByKey();
public abstract HoodiePairData<K, V> reduceByKey(SerializableBiFunction<V, V, V> func, int parallelism);
/**
* Groups the values for each key in the dataset into a single sequence
*/
HoodiePairData<K, Iterable<V>> groupByKey();
/**
* Reduces original sequence by de-duplicating the pairs w/ the same key, using provided
* binary operator {@code combiner}. Returns an instance of {@link HoodiePairData} holding
* the "de-duplicated" pairs, ie only pairs with unique keys.
*
* @param combiner method to combine values of the pairs with the same key
* @param parallelism target parallelism (if applicable)
*/
HoodiePairData<K, V> reduceByKey(SerializableBiFunction<V, V, V> combiner, int parallelism);
/**
* @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);
<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.
* @return containing the result. Actual execution may be deferred.
*/
public abstract <L, W> HoodiePairData<L, W> mapToPair(
<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.
* Performs a left outer join of this dataset against {@code other}.
*
* For each element (k, v) in this, the resulting {@link HoodiePairData} will either contain all
* pairs {@code (k, (v, Some(w)))} for every {@code w} in the {@code other}, or the pair {@code (k, (v, None))}
* if no elements in {@code other} have the pair w/ a key {@code 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.
* @return containing the result of the left outer join
*/
public abstract <W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> other);
<W> HoodiePairData<K, Pair<V, Option<W>>> leftOuterJoin(HoodiePairData<K, W> other);
/**
* Collects results of the underlying collection into a {@link List<Pair<K, V>>}
*
* This is a terminal operation
*/
List<Pair<K, V>> collectAsList();
}

View File

@@ -24,7 +24,7 @@ import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.data.HoodieAccumulator;
import org.apache.hudi.common.data.HoodieAtomicLongAccumulator;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.data.HoodieList;
import org.apache.hudi.common.data.HoodieListData;
import org.apache.hudi.common.function.SerializableBiFunction;
import org.apache.hudi.common.function.SerializableConsumer;
import org.apache.hudi.common.function.SerializableFunction;
@@ -71,12 +71,12 @@ public final class HoodieLocalEngineContext extends HoodieEngineContext {
@Override
public <T> HoodieData<T> emptyHoodieData() {
return HoodieList.of(Collections.emptyList());
return HoodieListData.eager(Collections.emptyList());
}
@Override
public <T> HoodieData<T> parallelize(List<T> data, int parallelism) {
return HoodieList.of(data);
return HoodieListData.eager(data);
}
@Override

View File

@@ -34,6 +34,7 @@ import java.util.Properties;
import java.util.Set;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.function.BiFunction;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
@@ -107,6 +108,19 @@ public class CollectionUtils {
return combined;
}
/**
* Combines provided {@link Map}s into one, returning new instance of {@link HashMap}.
*
* NOTE: That values associated with overlapping keys from the second map, will override
* values from the first one
*/
public static <K, V> HashMap<K, V> combine(Map<K, V> one, Map<K, V> another, BiFunction<V, V, V> merge) {
HashMap<K, V> combined = new HashMap<>(one.size() + another.size());
combined.putAll(one);
another.forEach((k, v) -> combined.merge(k, v, merge));
return combined;
}
/**
* Returns difference b/w {@code one} {@link Set} of elements and {@code another}
*/

View File

@@ -0,0 +1,93 @@
/*
* 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.util;
import javax.annotation.Nonnull;
import static org.apache.hudi.TypeUtils.unsafeCast;
/**
* Utility that could hold exclusively only either of (hence the name):
* <ul>
* <li>Non-null value of type {@link L}</li>
* <li>Non-null value of type {@link R}</li>
* </ul>
*
* @param <L> type of the "left" potential element
* @param <R> type of the "right" potential element
*/
public abstract class Either<L, R> {
@Nonnull
protected abstract Object getValue();
public final boolean isLeft() {
return this instanceof EitherLeft;
}
public final boolean isRight() {
return this instanceof EitherRight;
}
public R asRight() {
ValidationUtils.checkArgument(isRight(), "Trying to access non-existent value of Either");
EitherRight<L, R> right = unsafeCast(this);
return right.getValue();
}
public L asLeft() {
ValidationUtils.checkArgument(isLeft(), "Trying to access non-existent value of Either");
EitherLeft<L, R> left = unsafeCast(this);
return left.getValue();
}
public static <L, R> Either<L, R> right(R right) {
return new EitherRight<>(right);
}
public static <L, R> Either<L, R> left(L left) {
return new EitherLeft<>(left);
}
public static class EitherRight<L, R> extends Either<L, R> {
private final R value;
private EitherRight(@Nonnull R right) {
this.value = right;
}
@Nonnull
@Override
protected R getValue() {
return value;
}
}
public static class EitherLeft<L, R> extends Either<L, R> {
private final L value;
private EitherLeft(@Nonnull L value) {
this.value = value;
}
@Nonnull
@Override
protected L getValue() {
return value;
}
}
}