yihua commented on a change in pull request #3778:
URL: https://github.com/apache/hudi/pull/3778#discussion_r736980261



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPairData.java
##########
@@ -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 HoodieMapPairData<K, V> extends HoodiePairData<K, V> {
+
+  private final Map<K, List<V>> mapPairData;
+
+  private HoodieMapPairData(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> HoodieMapPairData<K, V> of(Map<K, List<V>> mapPairData) 
{
+    return new HoodieMapPairData<>(mapPairData);
+  }
+
+  /**
+   * @param hoodiePairData {@link HoodieMapPairData<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 ((HoodieMapPairData<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 HoodieListData.of(new ArrayList<>(mapPairData.keySet()));
+  }
+
+  @Override
+  public HoodieData<V> values() {
+    return HoodieListData.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 HoodieListData.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 HoodieMapPairData.of(newMap);
+  }
+
+  @Override
+  public <W> HoodiePairData<K, Pair<V, Option<W>>> 
leftOuterJoin(HoodiePairData<K, W> other) {

Review comment:
       I added unit tests for `HoodieMapPair` which has more complex 
transformations.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to