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

@@ -24,7 +24,6 @@ import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import com.google.common.annotations.VisibleForTesting;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
@@ -220,7 +219,6 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
* HoodieLogFormat V1.
*/
@Deprecated
@VisibleForTesting
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema) {
super(new HashMap<>(), new HashMap<>(), Option.empty(), Option.empty(), null, false);
this.records = records;
@@ -264,7 +262,6 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
}
@Deprecated
@VisibleForTesting
public byte[] getBytes(Schema schema) throws IOException {
GenericDatumWriter<IndexedRecord> writer = new GenericDatumWriter<>(schema);

View File

@@ -25,7 +25,6 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import com.google.common.collect.Maps;
import org.apache.hadoop.fs.FSDataInputStream;
import javax.annotation.Nonnull;
@@ -35,9 +34,9 @@ import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
/**
* Abstract class defining a block in HoodieLogFile.
*/
@@ -192,7 +191,7 @@ public abstract class HoodieLogBlock {
*/
public static Map<HeaderMetadataType, String> getLogMetadata(DataInputStream dis) throws IOException {
Map<HeaderMetadataType, String> metadata = Maps.newHashMap();
Map<HeaderMetadataType, String> metadata = new HashMap<>();
// 1. Read the metadata written out
int metadataCount = dis.readInt();
try {
@@ -231,7 +230,7 @@ public abstract class HoodieLogBlock {
/**
* When lazyReading of blocks is turned on, inflate the content of a log block from disk.
*/
protected void inflate() throws IOException {
protected void inflate() throws HoodieIOException {
try {
content = Option.of(new byte[(int) this.getBlockContentLocation().get().getBlockSize()]);
@@ -239,13 +238,9 @@ public abstract class HoodieLogBlock {
inputStream.readFully(content.get(), 0, content.get().length);
safeSeek(inputStream, this.getBlockContentLocation().get().getBlockEndPos());
} catch (IOException e) {
try {
// TODO : fs.open() and return inputstream again, need to pass FS configuration
// because the inputstream might close/timeout for large number of log blocks to be merged
inflate();
} catch (IOException io) {
throw new HoodieIOException("unable to lazily read log block from disk", io);
}
// TODO : fs.open() and return inputstream again, need to pass FS configuration
// because the inputstream might close/timeout for large number of log blocks to be merged
inflate();
}
}
@@ -262,7 +257,7 @@ public abstract class HoodieLogBlock {
*
* @param inputStream Input Stream
* @param pos Position to seek
* @throws IOException
* @throws IOException -
*/
private static void safeSeek(FSDataInputStream inputStream, long pos) throws IOException {
try {

View File

@@ -31,6 +31,7 @@ import org.apache.log4j.Logger;
import java.io.Serializable;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.Collections;
import java.util.List;
import java.util.Set;
import java.util.function.Function;
@@ -95,9 +96,8 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
@Override
public HoodieTimeline filterPendingExcludingCompaction() {
return new HoodieDefaultTimeline(instants.stream().filter(instant -> {
return (!instant.isCompleted()) && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
}), details);
return new HoodieDefaultTimeline(instants.stream().filter(instant -> (!instant.isCompleted())
&& (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION))), details);
}
@Override
@@ -107,9 +107,8 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
@Override
public HoodieTimeline filterCompletedAndCompactionInstants() {
return new HoodieDefaultTimeline(instants.stream().filter(s -> {
return !s.isInflight() || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION);
}), details);
return new HoodieDefaultTimeline(instants.stream().filter(s -> !s.isInflight()
|| s.getAction().equals(HoodieTimeline.COMPACTION_ACTION)), details);
}
@Override
@@ -127,8 +126,7 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
@Override
public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) {
return new HoodieDefaultTimeline(
instants.stream().filter(s -> HoodieTimeline.isInRange(s.getTimestamp(), startTs, endTs)),
details);
instants.stream().filter(s -> HoodieTimeline.isInRange(s.getTimestamp(), startTs, endTs)), details);
}
@Override
@@ -163,7 +161,7 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
* Get only pure commits (inflight and completed) in the active timeline.
*/
public HoodieTimeline getCommitTimeline() {
return getTimelineOfActions(Sets.newHashSet(COMMIT_ACTION));
return getTimelineOfActions(Collections.singleton(COMMIT_ACTION));
}
/**

View File

@@ -188,7 +188,7 @@ public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSyste
}
@Override
/**
/*
* This is overridden to incrementally apply file-slices to rocks DB
*/
protected void applyDeltaFileSlicesToPartitionView(String partition, List<HoodieFileGroup> deltaFileGroups,