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

@@ -18,14 +18,13 @@
package org.apache.hudi.client.common.function;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Stream;
import scala.Tuple2;
/**
* Function wrapper util class, which catches the exception thrown by input function and return a similar function
* with no exception thrown.
@@ -62,7 +61,7 @@ public class FunctionWrapper {
};
}
public static <I, K, V> Function<I, Tuple2<K, V>> throwingMapToPairWrapper(SerializablePairFunction<I, K, V> throwingPairFunction) {
public static <I, K, V> Function<I, Pair<K, V>> throwingMapToPairWrapper(SerializablePairFunction<I, K, V> throwingPairFunction) {
return v1 -> {
try {
return throwingPairFunction.call(v1);

View File

@@ -18,7 +18,7 @@
package org.apache.hudi.client.common.function;
import scala.Tuple2;
import org.apache.hudi.common.util.collection.Pair;
import java.io.Serializable;
@@ -27,5 +27,5 @@ import java.io.Serializable;
*/
@FunctionalInterface
public interface SerializablePairFunction<I, K, V> extends Serializable {
Tuple2<K, V> call(I t) throws Exception;
Pair<K, V> call(I t) throws Exception;
}

View File

@@ -29,9 +29,7 @@ import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.fs.Path;
import java.util.Iterator;
import scala.Tuple2;
import java.util.stream.Stream;
/**
* {@link HoodieRecordLocation} fetch handle for all records from {@link HoodieBaseFile} of interest.
@@ -48,10 +46,10 @@ public class HoodieKeyLocationFetchHandle<T extends HoodieRecordPayload, I, K, O
this.partitionPathBaseFilePair = partitionPathBaseFilePair;
}
public Iterator<Tuple2<HoodieKey, HoodieRecordLocation>> locations() {
public Stream<Pair<HoodieKey, HoodieRecordLocation>> locations() {
HoodieBaseFile baseFile = partitionPathBaseFilePair.getRight();
return ParquetUtils.fetchRecordKeyPartitionPathFromParquet(hoodieTable.getHadoopConf(), new Path(baseFile.getPath())).stream()
.map(entry -> new Tuple2<>(entry,
new HoodieRecordLocation(baseFile.getCommitTime(), baseFile.getFileId()))).iterator();
.map(entry -> Pair.of(entry,
new HoodieRecordLocation(baseFile.getCommitTime(), baseFile.getFileId())));
}
}

View File

@@ -31,13 +31,13 @@ import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.ImmutablePair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieSavepointException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.BaseActionExecutor;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import scala.Tuple2;
import java.io.IOException;
import java.util.List;
@@ -96,7 +96,7 @@ public class SavepointActionExecutor<T extends HoodieRecordPayload, I, K, O> ext
TableFileSystemView.BaseFileOnlyView view = table.getBaseFileOnlyView();
List<String> latestFiles = view.getLatestBaseFilesBeforeOrOn(partitionPath, instantTime)
.map(HoodieBaseFile::getFileName).collect(Collectors.toList());
return new Tuple2<>(partitionPath, latestFiles);
return new ImmutablePair<>(partitionPath, latestFiles);
}, null);
HoodieSavepointMetadata metadata = TimelineMetadataUtils.convertSavepointMetadata(user, comment, latestFilesMap);
// Nothing to save in the savepoint