[HUDI-1439] Remove scala dependency from hudi-client-common (#2306)
This commit is contained in:
@@ -33,7 +33,7 @@ import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import scala.Tuple2;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import static org.apache.hudi.client.common.function.FunctionWrapper.throwingFlatMapWrapper;
|
||||
import static org.apache.hudi.client.common.function.FunctionWrapper.throwingForeachWrapper;
|
||||
@@ -76,7 +76,7 @@ public class HoodieFlinkEngineContext extends HoodieEngineContext {
|
||||
|
||||
@Override
|
||||
public <I, K, V> Map<K, V> mapToPair(List<I> data, SerializablePairFunction<I, K, V> func, Integer parallelism) {
|
||||
return data.stream().map(throwingMapToPairWrapper(func)).collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
|
||||
return data.stream().map(throwingMapToPairWrapper(func)).collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -29,6 +29,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.NumericUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
@@ -210,7 +211,8 @@ public class UpsertPartitioner<T extends HoodieRecordPayload<T>> implements Part
|
||||
Map<String, List<SmallFile>> partitionSmallFilesMap = new HashMap<>();
|
||||
if (partitionPaths != null && partitionPaths.size() > 0) {
|
||||
context.setJobStatus(this.getClass().getSimpleName(), "Getting small files from partitions");
|
||||
partitionSmallFilesMap = context.mapToPair(partitionPaths, partitionPath -> new Tuple2<>(partitionPath, getSmallFiles(partitionPath)), 0);
|
||||
partitionSmallFilesMap = context.mapToPair(partitionPaths,
|
||||
partitionPath -> new ImmutablePair<>(partitionPath, getSmallFiles(partitionPath)), 0);
|
||||
}
|
||||
return partitionSmallFilesMap;
|
||||
}
|
||||
|
||||
@@ -27,6 +27,7 @@ import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
@@ -48,8 +49,6 @@ import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Performs Rollback of Hoodie Tables.
|
||||
*/
|
||||
@@ -106,13 +105,13 @@ public class ListingBasedRollbackHelper implements Serializable {
|
||||
case DELETE_DATA_FILES_ONLY: {
|
||||
final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseFiles(metaClient, config, instantToRollback.getTimestamp(),
|
||||
rollbackRequest.getPartitionPath(), doDelete);
|
||||
return new Tuple2<>(rollbackRequest.getPartitionPath(),
|
||||
return new ImmutablePair<>(rollbackRequest.getPartitionPath(),
|
||||
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
|
||||
.withDeletedFileResults(filesToDeletedStatus).build());
|
||||
}
|
||||
case DELETE_DATA_AND_LOG_FILES: {
|
||||
final Map<FileStatus, Boolean> filesToDeletedStatus = deleteBaseAndLogFiles(metaClient, config, instantToRollback.getTimestamp(), rollbackRequest.getPartitionPath(), doDelete);
|
||||
return new Tuple2<>(rollbackRequest.getPartitionPath(),
|
||||
return new ImmutablePair<>(rollbackRequest.getPartitionPath(),
|
||||
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
|
||||
.withDeletedFileResults(filesToDeletedStatus).build());
|
||||
}
|
||||
@@ -150,7 +149,7 @@ public class ListingBasedRollbackHelper implements Serializable {
|
||||
metaClient.getFs().getFileStatus(Objects.requireNonNull(writer).getLogFile().getPath()),
|
||||
1L
|
||||
);
|
||||
return new Tuple2<>(rollbackRequest.getPartitionPath(),
|
||||
return new ImmutablePair<>(rollbackRequest.getPartitionPath(),
|
||||
HoodieRollbackStat.newBuilder().withPartitionPath(rollbackRequest.getPartitionPath())
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build());
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.client.common;
|
||||
|
||||
import org.apache.hudi.client.FlinkTaskContextSupplier;
|
||||
|
||||
import org.apache.hudi.common.util.collection.ImmutablePair;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
@@ -30,8 +31,6 @@ import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Unit test against HoodieFlinkEngineContext.
|
||||
*/
|
||||
@@ -85,7 +84,7 @@ public class TestHoodieFlinkEngineContext {
|
||||
|
||||
Map<String, String> resultMap = context.mapToPair(mapList, x -> {
|
||||
String[] splits = x.split("_");
|
||||
return Tuple2.apply(splits[0], splits[1]);
|
||||
return new ImmutablePair<>(splits[0], splits[1]);
|
||||
}, 2);
|
||||
|
||||
Assertions.assertEquals(resultMap.get("spark"), resultMap.get("flink"));
|
||||
|
||||
Reference in New Issue
Block a user