1
0

[HUDI-1439] Remove scala dependency from hudi-client-common (#2306)

This commit is contained in:
Shen Hong
2020-12-11 16:36:37 +08:00
committed by GitHub
parent 6cf25d5c8a
commit 236d1b0dec
16 changed files with 70 additions and 41 deletions

View File

@@ -24,10 +24,12 @@ import org.apache.hudi.client.common.function.SerializableConsumer;
import org.apache.hudi.client.common.function.SerializableFunction;
import org.apache.hudi.client.common.function.SerializablePairFunction;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.SQLContext;
import scala.Tuple2;
import java.util.List;
import java.util.Map;
@@ -82,9 +84,15 @@ public class HoodieSparkEngineContext extends HoodieEngineContext {
@Override
public <I, K, V> Map<K, V> mapToPair(List<I> data, SerializablePairFunction<I, K, V> func, Integer parallelism) {
if (Objects.nonNull(parallelism)) {
return javaSparkContext.parallelize(data, parallelism).mapToPair(func::call).collectAsMap();
return javaSparkContext.parallelize(data, parallelism).mapToPair(input -> {
Pair<K, V> pair = func.call(input);
return new Tuple2(pair.getLeft(), pair.getRight());
}).collectAsMap();
} else {
return javaSparkContext.parallelize(data).mapToPair(func::call).collectAsMap();
return javaSparkContext.parallelize(data).mapToPair(input -> {
Pair<K, V> pair = func.call(input);
return new Tuple2(pair.getLeft(), pair.getRight());
}).collectAsMap();
}
}

View File

@@ -147,6 +147,7 @@ public class SparkHoodieSimpleIndex<T extends HoodieRecordPayload> extends Spark
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
int fetchParallelism = Math.max(1, Math.max(baseFiles.size(), parallelism));
return jsc.parallelize(baseFiles, fetchParallelism)
.flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile).locations());
.flatMapToPair(partitionPathBaseFile -> new HoodieKeyLocationFetchHandle(config, hoodieTable, partitionPathBaseFile)
.locations().map(x -> Tuple2.apply(((Pair)x).getLeft(), ((Pair)x).getRight())).iterator());
}
}

View File

@@ -95,9 +95,9 @@ public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness {
for (Tuple2<String, HoodieBaseFile> entry : partitionPathFileIdPairs) {
HoodieKeyLocationFetchHandle fetcherHandle = new HoodieKeyLocationFetchHandle(config, hoodieTable, Pair.of(entry._1, entry._2));
Iterator<Tuple2<HoodieKey, HoodieRecordLocation>> result = fetcherHandle.locations();
Iterator<Pair<HoodieKey, HoodieRecordLocation>> result = fetcherHandle.locations().iterator();
List<Tuple2<HoodieKey, HoodieRecordLocation>> actualList = new ArrayList<>();
result.forEachRemaining(actualList::add);
result.forEachRemaining(x -> actualList.add(new Tuple2<>(x.getLeft(), x.getRight())));
assertEquals(expectedList.get(new Tuple2<>(entry._1, entry._2.getFileId())), actualList);
}
}