1
0

[MINOR] Code Cleanup, remove redundant code (#1337)

This commit is contained in:
Suneel Marthi
2020-02-15 09:03:29 -05:00
committed by GitHub
parent aaa6cf9a98
commit 24e73816b2
18 changed files with 148 additions and 170 deletions

View File

@@ -21,7 +21,6 @@ package org.apache.hudi.hadoop.hive;
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -381,7 +380,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
}
// Use HiveInputFormat if any of the paths is not splittable
Class inputFormatClass = part.getInputFileFormatClass();
Class<?> inputFormatClass = part.getInputFileFormatClass();
String inputFormatClassName = inputFormatClass.getName();
InputFormat inputFormat = getInputFormatFromCache(inputFormatClass, job);
LOG.info("Input Format => " + inputFormatClass.getName());
@@ -484,7 +483,6 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
/**
* Gets all the path indices that should not be combined.
*/
@VisibleForTesting
public Set<Integer> getNonCombinablePathIndices(JobConf job, Path[] paths, int numThreads)
throws ExecutionException, InterruptedException {
LOG.info("Total number of paths: " + paths.length + ", launching " + numThreads
@@ -719,7 +717,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
CombineHiveInputSplit hsplit = (CombineHiveInputSplit) split;
String inputFormatClassName = null;
Class inputFormatClass;
Class<?> inputFormatClass;
try {
inputFormatClassName = hsplit.inputFormatClassName();
inputFormatClass = job.getClassByName(inputFormatClassName);
@@ -865,7 +863,7 @@ public class HoodieCombineHiveInputFormat<K extends WritableComparable, V extend
}
}
return (CombineFileSplit[]) inputSplitShims.toArray(new HadoopShimsSecure.InputSplitShim[inputSplitShims.size()]);
return inputSplitShims.toArray(new HadoopShimsSecure.InputSplitShim[inputSplitShims.size()]);
}
@Override

View File

@@ -78,8 +78,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader
this.iterator = this.executor.getQueue().iterator();
this.logRecordScanner = new HoodieUnMergedLogRecordScanner(FSUtils.getFs(split.getPath().toString(), jobConf),
split.getBasePath(), split.getDeltaLogPaths(), getReaderSchema(), split.getMaxCommitTime(),
Boolean
.parseBoolean(jobConf.get(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED)),
Boolean.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();

View File

@@ -37,7 +37,6 @@ import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hadoop.InputFormatTestUtil;
import com.google.common.collect.Maps;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.generic.IndexedRecord;
@@ -69,6 +68,7 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -109,7 +109,7 @@ public class TestHoodieRealtimeRecordReader {
.overBaseCommit(baseCommit).withFs(fs).withLogVersion(logVersion).withLogWriteToken("1-0-1")
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
// generate metadata
Map<HeaderMetadataType, String> header = Maps.newHashMap();
Map<HeaderMetadataType, String> header = new HashMap<>();
header.put(HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, rolledBackInstant);
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
@@ -130,7 +130,7 @@ public class TestHoodieRealtimeRecordReader {
records.add(SchemaTestUtil.generateAvroRecordFromJson(schema, i, newCommit, "fileid0"));
}
Schema writeSchema = records.get(0).getSchema();
Map<HeaderMetadataType, String> header = Maps.newHashMap();
Map<HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writeSchema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
@@ -144,7 +144,7 @@ public class TestHoodieRealtimeRecordReader {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(fileId).overBaseCommit(baseCommit)
.withLogVersion(logVersion).withFs(fs).build();
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, newCommit);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, oldCommit);