[HUDI-583] Code Cleanup, remove redundant code, and other changes (#1237)
This commit is contained in:
@@ -21,6 +21,7 @@ package org.apache.hudi.hadoop;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
import java.util.stream.Collectors;
|
||||
@@ -77,8 +78,7 @@ public class HoodieHiveUtil {
|
||||
return (!matcher.find() ? null : matcher.group(1));
|
||||
}
|
||||
return null;
|
||||
}).filter(s -> s != null)
|
||||
.collect(Collectors.toList());
|
||||
}).filter(Objects::nonNull).collect(Collectors.toList());
|
||||
if (result == null) {
|
||||
// Returns an empty list instead of null.
|
||||
result = new ArrayList<>();
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.hadoop;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
@@ -48,6 +49,7 @@ import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -93,9 +95,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
if (nonHoodiePaths.size() > 0) {
|
||||
setInputPaths(job, nonHoodiePaths.toArray(new Path[nonHoodiePaths.size()]));
|
||||
FileStatus[] fileStatuses = super.listStatus(job);
|
||||
for (FileStatus fileStatus: fileStatuses) {
|
||||
returns.add(fileStatus);
|
||||
}
|
||||
returns.addAll(Arrays.asList(fileStatuses));
|
||||
}
|
||||
|
||||
// process snapshot queries next.
|
||||
@@ -133,8 +133,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
.getInstants().collect(Collectors.toList());
|
||||
// Extract partitions touched by the commitsToCheck
|
||||
Set<String> partitionsToList = new HashSet<>();
|
||||
for (int i = 0; i < commitsToCheck.size(); i++) {
|
||||
HoodieInstant commit = commitsToCheck.get(i);
|
||||
for (HoodieInstant commit : commitsToCheck) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
partitionsToList.addAll(commitMetadata.getPartitionToWriteStats().keySet());
|
||||
@@ -171,15 +170,14 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
return false;
|
||||
})
|
||||
.collect(Collectors.joining(","));
|
||||
if (incrementalInputPaths == null || incrementalInputPaths.isEmpty()) {
|
||||
if (StringUtils.isNullOrEmpty(incrementalInputPaths)) {
|
||||
return null;
|
||||
}
|
||||
// Mutate the JobConf to set the input paths to only partitions touched by incremental pull.
|
||||
setInputPaths(job, incrementalInputPaths);
|
||||
FileStatus[] fileStatuses = super.listStatus(job);
|
||||
BaseFileOnlyView roView = new HoodieTableFileSystemView(tableMetaClient, timeline,
|
||||
fileStatuses);
|
||||
List<String> commitsList = commitsToCheck.stream().map(s -> s.getTimestamp()).collect(Collectors.toList());
|
||||
BaseFileOnlyView roView = new HoodieTableFileSystemView(tableMetaClient, timeline, fileStatuses);
|
||||
List<String> commitsList = commitsToCheck.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
List<HoodieBaseFile> filteredFiles = roView.getLatestBaseFilesInRange(commitsList).collect(Collectors.toList());
|
||||
List<FileStatus> returns = new ArrayList<>();
|
||||
for (HoodieBaseFile filteredFile : filteredFiles) {
|
||||
@@ -200,7 +198,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
* @throws IOException
|
||||
*/
|
||||
private Map<HoodieTableMetaClient, List<FileStatus>> groupFileStatusForSnapshotPaths(
|
||||
FileStatus[] fileStatuses, Collection<HoodieTableMetaClient> metaClientList) throws IOException {
|
||||
FileStatus[] fileStatuses, Collection<HoodieTableMetaClient> metaClientList) {
|
||||
// This assumes the paths for different tables are grouped together
|
||||
Map<HoodieTableMetaClient, List<FileStatus>> grouped = new HashMap<>();
|
||||
HoodieTableMetaClient metadata = null;
|
||||
@@ -231,8 +229,8 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
* Filters data files for a snapshot queried table.
|
||||
*/
|
||||
private List<FileStatus> filterFileStatusForSnapshotMode(
|
||||
HoodieTableMetaClient metadata, List<FileStatus> fileStatuses) throws IOException {
|
||||
FileStatus[] statuses = fileStatuses.toArray(new FileStatus[fileStatuses.size()]);
|
||||
HoodieTableMetaClient metadata, List<FileStatus> fileStatuses) {
|
||||
FileStatus[] statuses = fileStatuses.toArray(new FileStatus[0]);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metadata);
|
||||
}
|
||||
@@ -258,7 +256,7 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
|
||||
* super.listStatus() and gets back a FileStatus[] 2. Then it creates the HoodieTableMetaClient for the paths listed.
|
||||
* 3. Generation of splits looks at FileStatus size to create splits, which skips this file
|
||||
*/
|
||||
private HoodieBaseFile checkFileStatus(HoodieBaseFile dataFile) throws IOException {
|
||||
private HoodieBaseFile checkFileStatus(HoodieBaseFile dataFile) {
|
||||
Path dataPath = dataFile.getFileStatus().getPath();
|
||||
try {
|
||||
if (dataFile.getFileSize() == 0) {
|
||||
|
||||
@@ -65,11 +65,12 @@ import java.io.DataOutput;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
@@ -115,7 +116,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
|
||||
@Override
|
||||
public Set<Integer> call() throws Exception {
|
||||
Set<Integer> nonCombinablePathIndices = new HashSet<Integer>();
|
||||
Set<Integer> nonCombinablePathIndices = new HashSet<>();
|
||||
for (int i = 0; i < length; i++) {
|
||||
PartitionDesc part = HiveFileFormatUtils.getPartitionDescFromPathRecursively(pathToPartitionInfo,
|
||||
paths[i + start], IOPrepareCache.get().allocatePartitionDescMap());
|
||||
@@ -356,25 +357,21 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim combine =
|
||||
new HoodieCombineHiveInputFormat.HoodieCombineFileInputFormatShim();
|
||||
|
||||
InputSplit[] splits = null;
|
||||
if (combine == null) {
|
||||
splits = super.getSplits(job, numSplits);
|
||||
return splits;
|
||||
}
|
||||
InputSplit[] splits;
|
||||
|
||||
if (combine.getInputPathsShim(job).length == 0) {
|
||||
throw new IOException("No input paths specified in job");
|
||||
}
|
||||
ArrayList<InputSplit> result = new ArrayList<InputSplit>();
|
||||
ArrayList<InputSplit> result = new ArrayList<>();
|
||||
|
||||
// combine splits only from same tables and same partitions. Do not combine splits from multiple
|
||||
// tables or multiple partitions.
|
||||
Path[] paths = StringInternUtils.internUriStringsInPathArray(combine.getInputPathsShim(job));
|
||||
|
||||
List<Path> inpDirs = new ArrayList<Path>();
|
||||
List<Path> inpFiles = new ArrayList<Path>();
|
||||
Map<CombinePathInputFormat, CombineFilter> poolMap = new HashMap<CombinePathInputFormat, CombineFilter>();
|
||||
Set<Path> poolSet = new HashSet<Path>();
|
||||
List<Path> inpDirs = new ArrayList<>();
|
||||
List<Path> inpFiles = new ArrayList<>();
|
||||
Map<CombinePathInputFormat, CombineFilter> poolMap = new HashMap<>();
|
||||
Set<Path> poolSet = new HashSet<>();
|
||||
|
||||
for (Path path : paths) {
|
||||
PartitionDesc part = HiveFileFormatUtils.getPartitionDescFromPathRecursively(pathToPartitionInfo, path,
|
||||
@@ -414,8 +411,8 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
Path filterPath = path;
|
||||
|
||||
// Does a pool exist for this path already
|
||||
CombineFilter f = null;
|
||||
List<Operator<? extends OperatorDesc>> opList = null;
|
||||
CombineFilter f;
|
||||
List<Operator<? extends OperatorDesc>> opList;
|
||||
|
||||
if (!mrwork.isMapperCannotSpanPartns()) {
|
||||
// if mapper can span partitions, make sure a splits does not contain multiple
|
||||
@@ -441,7 +438,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
// parent directory will be grouped into one pool but not files from different parent
|
||||
// directories. This guarantees that a split will combine all files in the same partition
|
||||
// but won't cross multiple partitions if the user has asked so.
|
||||
if (!path.getFileSystem(job).getFileStatus(path).isDir()) { // path is not directory
|
||||
if (!path.getFileSystem(job).getFileStatus(path).isDirectory()) { // path is not directory
|
||||
filterPath = path.getParent();
|
||||
inpFiles.add(path);
|
||||
poolSet.add(filterPath);
|
||||
@@ -452,7 +449,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
}
|
||||
|
||||
// Processing directories
|
||||
List<CombineFileSplit> iss = new ArrayList<CombineFileSplit>();
|
||||
List<CombineFileSplit> iss = new ArrayList<>();
|
||||
if (!mrwork.isMapperCannotSpanPartns()) {
|
||||
// mapper can span partitions
|
||||
// combine into as few as one split, subject to the PathFilters set
|
||||
@@ -496,14 +493,14 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
int numPathPerThread = (int) Math.ceil((double) paths.length / numThreads);
|
||||
|
||||
ExecutorService executor = Executors.newFixedThreadPool(numThreads);
|
||||
List<Future<Set<Integer>>> futureList = new ArrayList<Future<Set<Integer>>>(numThreads);
|
||||
List<Future<Set<Integer>>> futureList = new ArrayList<>(numThreads);
|
||||
try {
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
int start = i * numPathPerThread;
|
||||
int length = i != numThreads - 1 ? numPathPerThread : paths.length - start;
|
||||
futureList.add(executor.submit(new CheckNonCombinablePathCallable(paths, start, length, job)));
|
||||
}
|
||||
Set<Integer> nonCombinablePathIndices = new HashSet<Integer>();
|
||||
Set<Integer> nonCombinablePathIndices = new HashSet<>();
|
||||
for (Future<Set<Integer>> future : futureList) {
|
||||
nonCombinablePathIndices.addAll(future.get());
|
||||
}
|
||||
@@ -522,12 +519,12 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.GET_SPLITS);
|
||||
init(job);
|
||||
|
||||
ArrayList<InputSplit> result = new ArrayList<InputSplit>();
|
||||
List<InputSplit> result = new ArrayList<>();
|
||||
|
||||
Path[] paths = getInputPaths(job);
|
||||
|
||||
List<Path> nonCombinablePaths = new ArrayList<Path>(paths.length / 2);
|
||||
List<Path> combinablePaths = new ArrayList<Path>(paths.length / 2);
|
||||
List<Path> nonCombinablePaths = new ArrayList<>(paths.length / 2);
|
||||
List<Path> combinablePaths = new ArrayList<>(paths.length / 2);
|
||||
|
||||
int numThreads = Math.min(MAX_CHECK_NONCOMBINABLE_THREAD_NUM,
|
||||
(int) Math.ceil((double) paths.length / DEFAULT_NUM_PATH_PER_THREAD));
|
||||
@@ -561,22 +558,18 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
|
||||
// Process the normal splits
|
||||
if (nonCombinablePaths.size() > 0) {
|
||||
FileInputFormat.setInputPaths(job, nonCombinablePaths.toArray(new Path[nonCombinablePaths.size()]));
|
||||
FileInputFormat.setInputPaths(job, nonCombinablePaths.toArray(new Path[0]));
|
||||
InputSplit[] splits = super.getSplits(job, numSplits);
|
||||
for (InputSplit split : splits) {
|
||||
result.add(split);
|
||||
}
|
||||
Collections.addAll(result, splits);
|
||||
}
|
||||
|
||||
// Process the combine splits
|
||||
if (combinablePaths.size() > 0) {
|
||||
FileInputFormat.setInputPaths(job, combinablePaths.toArray(new Path[combinablePaths.size()]));
|
||||
FileInputFormat.setInputPaths(job, combinablePaths.toArray(new Path[0]));
|
||||
Map<Path, PartitionDesc> pathToPartitionInfo = this.pathToPartitionInfo != null ? this.pathToPartitionInfo
|
||||
: Utilities.getMapWork(job).getPathToPartitionInfo();
|
||||
InputSplit[] splits = getCombineSplits(job, numSplits, pathToPartitionInfo);
|
||||
for (InputSplit split : splits) {
|
||||
result.add(split);
|
||||
}
|
||||
Collections.addAll(result, splits);
|
||||
}
|
||||
|
||||
// Restore the old path information back
|
||||
@@ -634,8 +627,8 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
*/
|
||||
private List<CombineFileSplit> sampleSplits(List<CombineFileSplit> splits) {
|
||||
HashMap<String, SplitSample> nameToSamples = mrwork.getNameToSplitSample();
|
||||
List<CombineFileSplit> retLists = new ArrayList<CombineFileSplit>();
|
||||
Map<String, ArrayList<CombineFileSplit>> aliasToSplitList = new HashMap<String, ArrayList<CombineFileSplit>>();
|
||||
List<CombineFileSplit> retLists = new ArrayList<>();
|
||||
Map<String, ArrayList<CombineFileSplit>> aliasToSplitList = new HashMap<>();
|
||||
Map<Path, ArrayList<String>> pathToAliases = mrwork.getPathToAliases();
|
||||
Map<Path, ArrayList<String>> pathToAliasesNoScheme = removeScheme(pathToAliases);
|
||||
|
||||
@@ -651,7 +644,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
// 1. it serves more than one alias
|
||||
// 2. the alias it serves is not sampled
|
||||
// 3. it serves different alias than another path for the same split
|
||||
if (l.size() != 1 || !nameToSamples.containsKey(l.get(0)) || (alias != null && l.get(0) != alias)) {
|
||||
if (l.size() != 1 || !nameToSamples.containsKey(l.get(0)) || (alias != null && !Objects.equals(l.get(0), alias))) {
|
||||
alias = null;
|
||||
break;
|
||||
}
|
||||
@@ -662,7 +655,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
// split exclusively serves alias, which needs to be sampled
|
||||
// add it to the split list of the alias.
|
||||
if (!aliasToSplitList.containsKey(alias)) {
|
||||
aliasToSplitList.put(alias, new ArrayList<CombineFileSplit>());
|
||||
aliasToSplitList.put(alias, new ArrayList<>());
|
||||
}
|
||||
aliasToSplitList.get(alias).add(split);
|
||||
} else {
|
||||
@@ -727,7 +720,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
CombineHiveInputSplit hsplit = (CombineHiveInputSplit) split;
|
||||
|
||||
String inputFormatClassName = null;
|
||||
Class inputFormatClass = null;
|
||||
Class inputFormatClass;
|
||||
try {
|
||||
inputFormatClassName = hsplit.inputFormatClassName();
|
||||
inputFormatClass = job.getClassByName(inputFormatClassName);
|
||||
@@ -743,7 +736,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
|
||||
static class CombineFilter implements PathFilter {
|
||||
|
||||
private final Set<String> pStrings = new HashSet<String>();
|
||||
private final Set<String> pStrings = new HashSet<>();
|
||||
|
||||
// store a path prefix in this TestFilter
|
||||
// PRECONDITION: p should always be a directory
|
||||
@@ -764,7 +757,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
@Override
|
||||
public boolean accept(Path path) {
|
||||
boolean find = false;
|
||||
while (path != null && !find) {
|
||||
while (path != null) {
|
||||
if (pStrings.contains(path.toUri().getPath())) {
|
||||
find = true;
|
||||
break;
|
||||
@@ -838,19 +831,12 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
input = new HoodieParquetInputFormat();
|
||||
}
|
||||
input.setConf(job.getConfiguration());
|
||||
result = new ArrayList<FileStatus>(Arrays.asList(input.listStatus(new JobConf(job.getConfiguration()))));
|
||||
result = new ArrayList<>(Arrays.asList(input.listStatus(new JobConf(job.getConfiguration()))));
|
||||
} else {
|
||||
result = super.listStatus(job);
|
||||
}
|
||||
|
||||
Iterator it = result.iterator();
|
||||
|
||||
while (it.hasNext()) {
|
||||
FileStatus stat = (FileStatus) it.next();
|
||||
if (!stat.isFile()) {
|
||||
it.remove();
|
||||
}
|
||||
}
|
||||
result.removeIf(stat -> !stat.isFile());
|
||||
return result;
|
||||
}
|
||||
|
||||
@@ -870,12 +856,12 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
}
|
||||
|
||||
InputSplit[] splits = super.getSplits(job, numSplits);
|
||||
ArrayList inputSplitShims = new ArrayList();
|
||||
List<InputSplitShim> inputSplitShims = new ArrayList<>();
|
||||
|
||||
for (int pos = 0; pos < splits.length; ++pos) {
|
||||
CombineFileSplit split = (CombineFileSplit) splits[pos];
|
||||
for (InputSplit inputSplit : splits) {
|
||||
CombineFileSplit split = (CombineFileSplit) inputSplit;
|
||||
if (split.getPaths().length > 0) {
|
||||
inputSplitShims.add(new HadoopShimsSecure.InputSplitShim(job, split.getPaths(), split.getStartOffsets(),
|
||||
inputSplitShims.add(new InputSplitShim(job, split.getPaths(), split.getStartOffsets(),
|
||||
split.getLengths(), split.getLocations()));
|
||||
}
|
||||
}
|
||||
@@ -884,7 +870,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
|
||||
}
|
||||
|
||||
@Override
|
||||
public HadoopShimsSecure.InputSplitShim getInputSplitShim() throws IOException {
|
||||
public HadoopShimsSecure.InputSplitShim getInputSplitShim() {
|
||||
return new HadoopShimsSecure.InputSplitShim();
|
||||
}
|
||||
|
||||
|
||||
@@ -78,7 +78,7 @@ public abstract class AbstractRealtimeRecordReader {
|
||||
// Property to set the max memory for dfs inputstream buffer size
|
||||
public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size";
|
||||
// Setting this to lower value of 1 MB since no control over how many RecordReaders will be started in a mapper
|
||||
public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 1 * 1024 * 1024; // 1 MB
|
||||
public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 1024 * 1024; // 1 MB
|
||||
// Property to set file path prefix for spillable file
|
||||
public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path";
|
||||
// Default file path prefix for spillable file
|
||||
@@ -170,18 +170,12 @@ public abstract class AbstractRealtimeRecordReader {
|
||||
// /org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java#L188}
|
||||
// Field Names -> {@link https://github.com/apache/hive/blob/f37c5de6c32b9395d1b34fa3c02ed06d1bfbf6eb/serde/src/java
|
||||
// /org/apache/hadoop/hive/serde2/ColumnProjectionUtils.java#L229}
|
||||
Set<String> fieldOrdersSet = new LinkedHashSet<>();
|
||||
String[] fieldOrdersWithDups = fieldOrderCsv.split(",");
|
||||
for (String fieldOrder : fieldOrdersWithDups) {
|
||||
fieldOrdersSet.add(fieldOrder);
|
||||
}
|
||||
String[] fieldOrders = fieldOrdersSet.toArray(new String[fieldOrdersSet.size()]);
|
||||
Set<String> fieldOrdersSet = new LinkedHashSet<>(Arrays.asList(fieldOrdersWithDups));
|
||||
String[] fieldOrders = fieldOrdersSet.toArray(new String[0]);
|
||||
List<String> fieldNames = Arrays.stream(fieldNameCsv.split(","))
|
||||
.filter(fn -> !partitioningFields.contains(fn)).collect(Collectors.toList());
|
||||
Set<String> fieldNamesSet = new LinkedHashSet<>();
|
||||
for (String fieldName : fieldNames) {
|
||||
fieldNamesSet.add(fieldName);
|
||||
}
|
||||
Set<String> fieldNamesSet = new LinkedHashSet<>(fieldNames);
|
||||
// Hive does not provide ids for partitioning fields, so check for lengths excluding that.
|
||||
if (fieldNamesSet.size() != fieldOrders.length) {
|
||||
throw new HoodieException(String
|
||||
@@ -189,7 +183,7 @@ public abstract class AbstractRealtimeRecordReader {
|
||||
fieldNames.size(), fieldOrders.length));
|
||||
}
|
||||
TreeMap<Integer, String> orderedFieldMap = new TreeMap<>();
|
||||
String[] fieldNamesArray = fieldNamesSet.toArray(new String[fieldNamesSet.size()]);
|
||||
String[] fieldNamesArray = fieldNamesSet.toArray(new String[0]);
|
||||
for (int ox = 0; ox < fieldOrders.length; ox++) {
|
||||
orderedFieldMap.put(Integer.parseInt(fieldOrders[ox]), fieldNamesArray[ox]);
|
||||
}
|
||||
@@ -402,7 +396,7 @@ public abstract class AbstractRealtimeRecordReader {
|
||||
public long getMaxCompactionMemoryInBytes() {
|
||||
// jobConf.getMemoryForMapTask() returns in MB
|
||||
return (long) Math
|
||||
.ceil(Double.valueOf(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
|
||||
.ceil(Double.parseDouble(jobConf.get(COMPACTION_MEMORY_FRACTION_PROP, DEFAULT_COMPACTION_MEMORY_FRACTION))
|
||||
* jobConf.getMemoryForMapTask() * 1024 * 1024L);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -108,7 +108,7 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
|
||||
// for all unique split parents, obtain all delta files based on delta commit timeline,
|
||||
// grouped on file id
|
||||
List<HoodieRealtimeFileSplit> rtSplits = new ArrayList<>();
|
||||
partitionsToParquetSplits.keySet().stream().forEach(partitionPath -> {
|
||||
partitionsToParquetSplits.keySet().forEach(partitionPath -> {
|
||||
// for each partition path obtain the data & log file groupings, then map back to inputsplits
|
||||
HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath);
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline());
|
||||
@@ -149,7 +149,7 @@ public class HoodieParquetRealtimeInputFormat extends HoodieParquetInputFormat i
|
||||
}
|
||||
});
|
||||
LOG.info("Returning a total splits of " + rtSplits.size());
|
||||
return rtSplits.toArray(new InputSplit[rtSplits.size()]);
|
||||
return rtSplits.toArray(new InputSplit[0]);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -48,7 +48,7 @@ public class HoodieRealtimeRecordReader implements RecordReader<NullWritable, Ar
|
||||
}
|
||||
|
||||
public static boolean canSkipMerging(JobConf jobConf) {
|
||||
return Boolean.valueOf(jobConf.get(REALTIME_SKIP_MERGE_PROP, DEFAULT_REALTIME_SKIP_MERGE));
|
||||
return Boolean.parseBoolean(jobConf.get(REALTIME_SKIP_MERGE_PROP, DEFAULT_REALTIME_SKIP_MERGE));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -64,7 +64,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader
|
||||
split.getDeltaLogPaths(), usesCustomPayload ? getWriterSchema() : getReaderSchema(), split.getMaxCommitTime(),
|
||||
getMaxCompactionMemoryInBytes(),
|
||||
Boolean
|
||||
.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
.parseBoolean(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE),
|
||||
jobConf.get(SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH));
|
||||
}
|
||||
|
||||
@@ -79,7 +79,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
|
||||
this.logRecordScanner = new HoodieUnMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf),
|
||||
split.getBasePath(), split.getDeltaLogPaths(), getReaderSchema(), split.getMaxCommitTime(),
|
||||
Boolean
|
||||
.valueOf(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
.parseBoolean(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
|
||||
false, jobConf.getInt(MAX_DFS_STREAM_BUFFER_SIZE_PROP, DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), record -> {
|
||||
// convert Hoodie log record to Hadoop AvroWritable and buffer
|
||||
GenericRecord rec = (GenericRecord) record.getData().getInsertValue(getReaderSchema()).get();
|
||||
@@ -93,7 +93,6 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
|
||||
/**
|
||||
* Setup log and parquet reading in parallel. Both write to central buffer.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
private List<BoundedInMemoryQueueProducer<ArrayWritable>> getParallelProducers() {
|
||||
List<BoundedInMemoryQueueProducer<ArrayWritable>> producers = new ArrayList<>();
|
||||
producers.add(new FunctionBasedQueueProducer<>(buffer -> {
|
||||
@@ -105,7 +104,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean next(NullWritable key, ArrayWritable value) throws IOException {
|
||||
public boolean next(NullWritable key, ArrayWritable value) {
|
||||
if (!iterator.hasNext()) {
|
||||
return false;
|
||||
}
|
||||
@@ -125,7 +124,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPos() throws IOException {
|
||||
public long getPos() {
|
||||
// TODO: vb - No logical way to represent parallel stream pos in a single long.
|
||||
// Should we just return invalid (-1). Where is it used ?
|
||||
return 0;
|
||||
|
||||
Reference in New Issue
Block a user