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

@@ -47,8 +47,6 @@ import com.esotericsoftware.kryo.Kryo;
import com.esotericsoftware.kryo.io.Input;
import com.esotericsoftware.kryo.io.Output;
import com.esotericsoftware.kryo.serializers.JavaSerializer;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -351,7 +349,7 @@ public class HoodieTestUtils {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId(location.getFileId())
.overBaseCommit(location.getInstantTime()).withFs(fs).build();
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, location.getInstantTime());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
logWriter.appendBlock(new HoodieAvroDataBlock(value.stream().map(r -> {
@@ -372,7 +370,7 @@ public class HoodieTestUtils {
public static FileStatus[] listAllDataFilesInPath(FileSystem fs, String basePath) throws IOException {
RemoteIterator<LocatedFileStatus> itr = fs.listFiles(new Path(basePath), true);
List<FileStatus> returns = Lists.newArrayList();
List<FileStatus> returns = new ArrayList<>();
while (itr.hasNext()) {
LocatedFileStatus status = itr.next();
if (status.getPath().getName().contains(".parquet")) {

View File

@@ -39,7 +39,6 @@ import org.apache.hudi.common.util.HoodieAvroUtils;
import org.apache.hudi.common.util.SchemaTestUtil;
import org.apache.hudi.exception.CorruptedLogFileException;
import com.google.common.collect.Maps;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -58,10 +57,11 @@ import org.junit.runners.Parameterized;
import java.io.IOException;
import java.io.UncheckedIOException;
import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
@@ -139,7 +139,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
@@ -157,7 +157,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
@@ -223,7 +223,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFile logFile1 = writer.getLogFile();
HoodieLogFile logFile2 = writer2.getLogFile();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
@@ -241,7 +241,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
@@ -321,7 +321,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
// Some data & append two times.
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
@@ -346,7 +346,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords = records.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
@@ -374,7 +374,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
Schema schema = getSimpleSchema();
List<IndexedRecord> copyOfRecords1 = records1.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
@@ -438,7 +438,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withSizeThreshold(1024).withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
Schema schema = HoodieAvroUtils.addMetadataFields(getSimpleSchema());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
@@ -478,7 +478,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
HoodieLogFormat.newWriterBuilder().onParentPath(partitionPath).withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, getSimpleSchema().toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
@@ -564,7 +564,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> copyOfRecords1 = records1.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
@@ -609,7 +609,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords1 = records1.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
@@ -670,7 +670,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords1 = records1.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
@@ -749,7 +749,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords1 = records1.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
@@ -834,7 +834,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords1 = records1.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
@@ -901,7 +901,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords1 = records1.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
@@ -943,7 +943,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
@@ -983,7 +983,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
List<IndexedRecord> copyOfRecords1 = records1.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
@@ -1030,7 +1030,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
// Write 1
List<IndexedRecord> records1 = SchemaTestUtil.generateHoodieTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
@@ -1131,7 +1131,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).build();
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records.subList(0, numRecordsInLog1), header);
@@ -1145,7 +1145,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).withFileId("test-fileid1")
.overBaseCommit("100").withFs(fs).withSizeThreshold(size - 1).build();
Map<HoodieLogBlock.HeaderMetadataType, String> header2 = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header2 = new HashMap<>();
header2.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header2.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock2 = new HoodieAvroDataBlock(records2.subList(0, numRecordsInLog2), header2);
@@ -1204,7 +1204,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords1 = records1.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);
@@ -1271,7 +1271,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
.withFileId("test-fileid1").overBaseCommit("100").withFs(fs).build();
Schema schema = getSimpleSchema();
List<IndexedRecord> records = SchemaTestUtil.generateTestRecords(0, 100);
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
@@ -1331,7 +1331,7 @@ public class TestHoodieLogFormat extends HoodieCommonTestHarness {
List<IndexedRecord> records1 = SchemaTestUtil.generateTestRecords(0, 100);
List<IndexedRecord> copyOfRecords1 = records1.stream()
.map(record -> HoodieAvroUtils.rewriteRecord((GenericRecord) record, schema)).collect(Collectors.toList());
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "100");
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records1, header);

View File

@@ -28,16 +28,17 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.util.Option;
import com.google.common.collect.Sets;
import org.apache.hadoop.fs.Path;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import com.google.common.collect.Sets;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Random;
import java.util.Set;
@@ -177,11 +178,11 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
@Test
public void testTimelineGetOperations() {
List<HoodieInstant> allInstants = getAllInstants();
Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
Supplier<Stream<HoodieInstant>> sup = allInstants::stream;
timeline = new HoodieActiveTimeline(metaClient, true);
timeline.setInstants(allInstants);
/**
/*
* Helper function to check HoodieTimeline only contains some type of Instant actions.
* @param timeline The HoodieTimeline to check
* @param actions The actions that should be present in the timeline being checked
@@ -197,13 +198,13 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION));
checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(),
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION));
checkTimeline.accept(timeline.getCommitTimeline(), Sets.newHashSet(HoodieTimeline.COMMIT_ACTION));
checkTimeline.accept(timeline.getCommitTimeline(), Collections.singleton(HoodieTimeline.COMMIT_ACTION));
checkTimeline.accept(timeline.getDeltaCommitTimeline(), Sets.newHashSet(HoodieTimeline.DELTA_COMMIT_ACTION));
checkTimeline.accept(timeline.getCleanerTimeline(), Sets.newHashSet(HoodieTimeline.CLEAN_ACTION));
checkTimeline.accept(timeline.getRollbackTimeline(), Sets.newHashSet(HoodieTimeline.ROLLBACK_ACTION));
checkTimeline.accept(timeline.getRestoreTimeline(), Sets.newHashSet(HoodieTimeline.RESTORE_ACTION));
checkTimeline.accept(timeline.getSavePointTimeline(), Sets.newHashSet(HoodieTimeline.SAVEPOINT_ACTION));
checkTimeline.accept(timeline.getDeltaCommitTimeline(), Collections.singleton(HoodieTimeline.DELTA_COMMIT_ACTION));
checkTimeline.accept(timeline.getCleanerTimeline(), Collections.singleton(HoodieTimeline.CLEAN_ACTION));
checkTimeline.accept(timeline.getRollbackTimeline(), Collections.singleton(HoodieTimeline.ROLLBACK_ACTION));
checkTimeline.accept(timeline.getRestoreTimeline(), Collections.singleton(HoodieTimeline.RESTORE_ACTION));
checkTimeline.accept(timeline.getSavePointTimeline(), Collections.singleton(HoodieTimeline.SAVEPOINT_ACTION));
checkTimeline.accept(timeline.getAllCommitsTimeline(),
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION,
HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION,
@@ -212,8 +213,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
// Get some random Instants
Random rand = new Random();
Set<String> randomInstants = sup.get().filter(i -> rand.nextBoolean())
.map(i -> i.getAction())
.collect(Collectors.toSet());
.map(HoodieInstant::getAction).collect(Collectors.toSet());
checkTimeline.accept(timeline.getTimelineOfActions(randomInstants), randomInstants);
}
@@ -318,7 +318,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
timeline.setInstants(allInstants);
timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2"));
allInstants.stream().map(i -> i.getTimestamp()).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
allInstants.stream().map(HoodieInstant::getTimestamp).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s)));
assertTrue(timeline.containsOrBeforeTimelineStarts("0"));
assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000)));
assertFalse(timeline.getTimelineHash().isEmpty());
@@ -356,7 +356,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
@Test
public void testFiltering() {
List<HoodieInstant> allInstants = getAllInstants();
Supplier<Stream<HoodieInstant>> sup = () -> allInstants.stream();
Supplier<Stream<HoodieInstant>> sup = allInstants::stream;
timeline = new HoodieActiveTimeline(metaClient);
timeline.setInstants(allInstants);
@@ -368,7 +368,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
Collections.reverse(v2);
assertEquals(v1, v2);
/**
/*
* Helper function to check HoodieTimeline only contains some type of Instant states.
* @param timeline The HoodieTimeline to check
* @param states The states that should be present in the timeline being checked
@@ -378,8 +378,8 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
sup.get().filter(i -> !states.contains(i.getState())).forEach(i -> assertFalse(timeline.containsInstant(i)));
};
checkFilter.accept(timeline.filter(i -> false), Sets.newHashSet());
checkFilter.accept(timeline.filterInflights(), Sets.newHashSet(State.INFLIGHT));
checkFilter.accept(timeline.filter(i -> false), new HashSet<>());
checkFilter.accept(timeline.filterInflights(), Collections.singleton(State.INFLIGHT));
checkFilter.accept(timeline.filterInflightsAndRequested(),
Sets.newHashSet(State.INFLIGHT, State.REQUESTED));
@@ -387,7 +387,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
// This cannot be done using checkFilter as it involves both states and actions
final HoodieTimeline t1 = timeline.filterCompletedAndCompactionInstants();
final Set<State> states = Sets.newHashSet(State.REQUESTED, State.COMPLETED);
final Set<String> actions = Sets.newHashSet(HoodieTimeline.COMPACTION_ACTION);
final Set<String> actions = Collections.singleton(HoodieTimeline.COMPACTION_ACTION);
sup.get().filter(i -> states.contains(i.getState()) || actions.contains(i.getAction()))
.forEach(i -> assertTrue(t1.containsInstant(i)));
sup.get().filter(i -> !(states.contains(i.getState()) || actions.contains(i.getAction())))
@@ -395,9 +395,9 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
// filterPendingCompactionTimeline
final HoodieTimeline t2 = timeline.filterPendingCompactionTimeline();
sup.get().filter(i -> i.getAction() == HoodieTimeline.COMPACTION_ACTION)
sup.get().filter(i -> i.getAction().equals(HoodieTimeline.COMPACTION_ACTION))
.forEach(i -> assertTrue(t2.containsInstant(i)));
sup.get().filter(i -> i.getAction() != HoodieTimeline.COMPACTION_ACTION)
sup.get().filter(i -> !i.getAction().equals(HoodieTimeline.COMPACTION_ACTION))
.forEach(i -> assertFalse(t2.containsInstant(i)));
}
@@ -407,7 +407,7 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
*/
private List<HoodieInstant> getAllInstants() {
timeline = new HoodieActiveTimeline(metaClient);
List<HoodieInstant> allInstants = new ArrayList<HoodieInstant>();
List<HoodieInstant> allInstants = new ArrayList<>();
long commitTime = 1;
for (State state : State.values()) {
if (state == State.INVALID) {
@@ -417,19 +417,19 @@ public class TestHoodieActiveTimeline extends HoodieCommonTestHarness {
// Following are not valid combinations of actions and state so we should
// not be generating them.
if (state == State.REQUESTED) {
if (action == HoodieTimeline.SAVEPOINT_ACTION || action == HoodieTimeline.RESTORE_ACTION
|| action == HoodieTimeline.ROLLBACK_ACTION) {
if (action.equals(HoodieTimeline.SAVEPOINT_ACTION) || action.equals(HoodieTimeline.RESTORE_ACTION)
|| action.equals(HoodieTimeline.ROLLBACK_ACTION)) {
continue;
}
}
if (state == State.INFLIGHT && action == HoodieTimeline.ROLLBACK_ACTION) {
if (state == State.INFLIGHT && action.equals(HoodieTimeline.ROLLBACK_ACTION)) {
continue;
}
if (state == State.COMPLETED && action == HoodieTimeline.ROLLBACK_ACTION) {
if (state == State.COMPLETED && action.equals(HoodieTimeline.ROLLBACK_ACTION)) {
continue;
}
// Compaction complete is called commit complete
if (state == State.COMPLETED && action == HoodieTimeline.COMPACTION_ACTION) {
if (state == State.COMPLETED && action.equals(HoodieTimeline.COMPACTION_ACTION)) {
action = HoodieTimeline.COMMIT_ACTION;
}

View File

@@ -41,7 +41,6 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -234,7 +233,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
* @param expTotalFileSlices Total number of file-slices across file-groups in the partition path
* @param expTotalDataFiles Total number of data-files across file-groups in the partition path
* @param includeInvalidAndInflight Whether view includes inflight and invalid file-groups.
* @throws Exception
* @throws Exception -
*/
protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFile, boolean isCompactionInFlight,
int expTotalFileSlices, int expTotalDataFiles, boolean includeInvalidAndInflight) throws Exception {
@@ -652,7 +651,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
List<HoodieBaseFile> dataFileList =
roView.getLatestBaseFilesBeforeOrOn("2016/05/01", commitTime4).collect(Collectors.toList());
assertEquals(3, dataFileList.size());
Set<String> filenames = Sets.newHashSet();
Set<String> filenames = new HashSet<>();
for (HoodieBaseFile status : dataFileList) {
filenames.add(status.getFileName());
}
@@ -660,7 +659,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, TEST_WRITE_TOKEN, fileId2)));
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime4, TEST_WRITE_TOKEN, fileId3)));
filenames = Sets.newHashSet();
filenames = new HashSet<>();
List<HoodieLogFile> logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime4, true)
.map(FileSlice::getLogFiles).flatMap(logFileList -> logFileList).collect(Collectors.toList());
assertEquals(logFilesList.size(), 4);
@@ -679,7 +678,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
// Reset the max commit time
List<HoodieBaseFile> dataFiles =
roView.getLatestBaseFilesBeforeOrOn("2016/05/01", commitTime3).collect(Collectors.toList());
filenames = Sets.newHashSet();
filenames = new HashSet<>();
for (HoodieBaseFile status : dataFiles) {
filenames.add(status.getFileName());
}
@@ -739,7 +738,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
for (HoodieFileGroup fileGroup : fileGroups) {
String fileId = fileGroup.getFileGroupId().getFileId();
Set<String> filenames = Sets.newHashSet();
Set<String> filenames = new HashSet<>();
fileGroup.getAllBaseFiles().forEach(dataFile -> {
assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId());
filenames.add(dataFile.getFileName());
@@ -817,7 +816,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
List<HoodieBaseFile> dataFiles =
roView.getLatestBaseFilesInRange(Lists.newArrayList(commitTime2, commitTime3)).collect(Collectors.toList());
assertEquals(isLatestFileSliceOnly ? 2 : 3, dataFiles.size());
Set<String> filenames = Sets.newHashSet();
Set<String> filenames = new HashSet<>();
for (HoodieBaseFile status : dataFiles) {
filenames.add(status.getFileName());
}
@@ -888,7 +887,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
roView.getLatestBaseFilesBeforeOrOn(partitionPath, commitTime2).collect(Collectors.toList());
if (!isLatestFileSliceOnly) {
assertEquals(2, dataFiles.size());
Set<String> filenames = Sets.newHashSet();
Set<String> filenames = new HashSet<>();
for (HoodieBaseFile status : dataFiles) {
filenames.add(status.getFileName());
}
@@ -983,7 +982,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
List<HoodieBaseFile> statuses1 = roView.getLatestBaseFiles().collect(Collectors.toList());
assertEquals(3, statuses1.size());
Set<String> filenames = Sets.newHashSet();
Set<String> filenames = new HashSet<>();
for (HoodieBaseFile status : statuses1) {
filenames.add(status.getFileName());
}
@@ -1100,7 +1099,7 @@ public class TestHoodieTableFileSystemView extends HoodieCommonTestHarness {
assertEquals("One data-file is expected as there is only one file-group", 1, dataFiles.size());
assertEquals("Expect only valid commit", "1", dataFiles.get(0).getCommitTime());
/** Merge API Tests **/
// Merge API Tests
Arrays.asList(partitionPath1, partitionPath2, partitionPath3).forEach(partitionPath -> {
List<FileSlice> fileSliceList =
rtView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, deltaInstantTime5).collect(Collectors.toList());