1
0

[HUDI-153] Use com.uber.hoodie.common.util.Option instead of Java and Guava Optional

This commit is contained in:
yanghua
2019-08-06 14:20:42 +08:00
committed by Balaji Varadarajan
parent d288e32833
commit 722b6be04a
128 changed files with 769 additions and 769 deletions

View File

@@ -20,9 +20,9 @@ package com.uber.hoodie.common;
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.Option;
import java.io.Serializable;
import java.util.List;
import java.util.Optional;
/**
* Collects stats about a single partition clean operation
@@ -114,7 +114,7 @@ public class HoodieCleanStat implements Serializable {
return this;
}
public Builder withEarliestCommitRetained(Optional<HoodieInstant> earliestCommitToRetain) {
public Builder withEarliestCommitRetained(Option<HoodieInstant> earliestCommitToRetain) {
this.earliestCommitToRetain = (earliestCommitToRetain.isPresent())
? earliestCommitToRetain.get().getTimestamp() : "-1";
return this;

View File

@@ -22,12 +22,12 @@ import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.uber.hoodie.avro.MercifulJsonConverter;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.exception.HoodieException;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.io.StringWriter;
import java.util.Optional;
import java.util.zip.Deflater;
import java.util.zip.DeflaterOutputStream;
import java.util.zip.InflaterInputStream;
@@ -51,15 +51,15 @@ public class HoodieJsonPayload implements HoodieRecordPayload<HoodieJsonPayload>
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema)
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema)
throws IOException {
return getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema);
return Optional.of(jsonConverter.convert(getJsonData()));
return Option.of(jsonConverter.convert(getJsonData()));
}
private String getJsonData() throws IOException {

View File

@@ -60,7 +60,7 @@ public class CompactionOperation implements Serializable {
this.metrics = metrics;
}
public CompactionOperation(java.util.Optional<HoodieDataFile> dataFile, String partitionPath,
public CompactionOperation(Option<HoodieDataFile> dataFile, String partitionPath,
List<HoodieLogFile> logFiles, Map<String, Double> metrics) {
if (dataFile.isPresent()) {
this.baseInstantTime = dataFile.get().getCommitTime();

View File

@@ -18,9 +18,9 @@
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.Option;
import java.io.Serializable;
import java.util.Objects;
import java.util.Optional;
import java.util.TreeSet;
import java.util.stream.Stream;
@@ -90,12 +90,12 @@ public class FileSlice implements Serializable {
return fileGroupId;
}
public Optional<HoodieDataFile> getDataFile() {
return Optional.ofNullable(dataFile);
public Option<HoodieDataFile> getDataFile() {
return Option.ofNullable(dataFile);
}
public Optional<HoodieLogFile> getLatestLogFile() {
return logFiles.stream().findFirst();
public Option<HoodieLogFile> getLatestLogFile() {
return Option.fromJavaOptional(logFiles.stream().findFirst());
}
/**

View File

@@ -19,9 +19,9 @@
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -36,7 +36,7 @@ public class HoodieAvroPayload implements HoodieRecordPayload<HoodieAvroPayload>
// java serializable
private final byte [] recordBytes;
public HoodieAvroPayload(Optional<GenericRecord> record) {
public HoodieAvroPayload(Option<GenericRecord> record) {
try {
if (record.isPresent()) {
this.recordBytes = HoodieAvroUtils.avroToBytes(record.get());
@@ -54,16 +54,16 @@ public class HoodieAvroPayload implements HoodieRecordPayload<HoodieAvroPayload>
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException {
return getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
if (recordBytes.length == 0) {
return Optional.empty();
return Option.empty();
}
return Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
return Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
}
}

View File

@@ -62,7 +62,7 @@ public class HoodieFileGroup implements Serializable {
this.fileGroupId = fileGroupId;
this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator());
this.timeline = timeline;
this.lastInstant = Option.fromJavaOptional(timeline.lastInstant());
this.lastInstant = timeline.lastInstant();
}
/**
@@ -126,8 +126,8 @@ public class HoodieFileGroup implements Serializable {
/**
* Get latest file slices including in-flight ones
*/
public Optional<FileSlice> getLatestFileSlicesIncludingInflight() {
return getAllFileSlicesIncludingInflight().findFirst();
public Option<FileSlice> getLatestFileSlicesIncludingInflight() {
return Option.fromJavaOptional(getAllFileSlicesIncludingInflight().findFirst());
}
/**
@@ -147,28 +147,28 @@ public class HoodieFileGroup implements Serializable {
* <p>
* - just the log files without data file - (or) data file with 0 or more log files
*/
public Optional<FileSlice> getLatestFileSlice() {
public Option<FileSlice> getLatestFileSlice() {
// there should always be one
return getAllFileSlices().findFirst();
return Option.fromJavaOptional(getAllFileSlices().findFirst());
}
/**
* Gets the latest data file
*/
public Optional<HoodieDataFile> getLatestDataFile() {
return getAllDataFiles().findFirst();
public Option<HoodieDataFile> getLatestDataFile() {
return Option.fromJavaOptional(getAllDataFiles().findFirst());
}
/**
* Obtain the latest file slice, upto a commitTime i.e <= maxCommitTime
*/
public Optional<FileSlice> getLatestFileSliceBeforeOrOn(String maxCommitTime) {
return getAllFileSlices()
public Option<FileSlice> getLatestFileSliceBeforeOrOn(String maxCommitTime) {
return Option.fromJavaOptional(getAllFileSlices()
.filter(slice ->
HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL))
.findFirst();
.findFirst());
}
/**
@@ -176,19 +176,19 @@ public class HoodieFileGroup implements Serializable {
* @param maxInstantTime Max Instant Time
* @return
*/
public Optional<FileSlice> getLatestFileSliceBefore(String maxInstantTime) {
return getAllFileSlices()
public Option<FileSlice> getLatestFileSliceBefore(String maxInstantTime) {
return Option.fromJavaOptional(getAllFileSlices()
.filter(slice ->
HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(),
maxInstantTime,
HoodieTimeline.LESSER))
.findFirst();
.findFirst());
}
public Optional<FileSlice> getLatestFileSliceInRange(List<String> commitRange) {
return getAllFileSlices()
public Option<FileSlice> getLatestFileSliceInRange(List<String> commitRange) {
return Option.fromJavaOptional(getAllFileSlices()
.filter(slice -> commitRange.contains(slice.getBaseInstantTime()))
.findFirst();
.findFirst());
}
/**

View File

@@ -20,9 +20,9 @@ package com.uber.hoodie.common.model;
import com.google.common.base.Objects;
import com.google.common.collect.ImmutableList;
import com.uber.hoodie.common.util.Option;
import java.io.Serializable;
import java.util.List;
import java.util.Optional;
/**
* A Single Record managed by Hoodie TODO - Make this generic
@@ -119,8 +119,8 @@ public class HoodieRecord<T extends HoodieRecordPayload> implements Serializable
return this;
}
public Optional<HoodieRecordLocation> getNewLocation() {
return Optional.ofNullable(this.newLocation);
public Option<HoodieRecordLocation> getNewLocation() {
return Option.ofNullable(this.newLocation);
}
public boolean isCurrentLocationKnown() {

View File

@@ -18,10 +18,10 @@
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.Option;
import java.io.IOException;
import java.io.Serializable;
import java.util.Map;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
@@ -50,7 +50,7 @@ public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Seri
* @return new combined/merged value to be written back to storage. EMPTY to skip writing this
* record.
*/
Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException;
/**
@@ -58,7 +58,7 @@ public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Seri
* Called when writing a new value for the given HoodieKey, wherein there is no existing record in
* storage to be combined against. (i.e insert) Return EMPTY to skip writing this record.
*/
Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException;
Option<IndexedRecord> getInsertValue(Schema schema) throws IOException;
/**
* This method can be used to extract some metadata from HoodieRecordPayload. The metadata is
@@ -66,7 +66,7 @@ public interface HoodieRecordPayload<T extends HoodieRecordPayload> extends Seri
* compute some aggregate metrics using the metadata in the context of a write success or
* failure.
*/
default Optional<Map<String, String>> getMetadata() {
return Optional.empty();
default Option<Map<String, String>> getMetadata() {
return Option.empty();
}
}

View File

@@ -21,9 +21,9 @@ package com.uber.hoodie.common.table;
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.StringUtils;
import java.io.Serializable;
import java.util.Optional;
import java.util.function.BiPredicate;
import java.util.function.Predicate;
import java.util.stream.Stream;
@@ -147,17 +147,17 @@ public interface HoodieTimeline extends Serializable {
/**
* @return first completed instant if available
*/
Optional<HoodieInstant> firstInstant();
Option<HoodieInstant> firstInstant();
/**
* @return nth completed instant from the first completed instant
*/
Optional<HoodieInstant> nthInstant(int n);
Option<HoodieInstant> nthInstant(int n);
/**
* @return last completed instant if available
*/
Optional<HoodieInstant> lastInstant();
Option<HoodieInstant> lastInstant();
/**
@@ -169,7 +169,7 @@ public interface HoodieTimeline extends Serializable {
/**
* @return nth completed instant going back from the last completed instant
*/
Optional<HoodieInstant> nthFromLastInstant(int n);
Option<HoodieInstant> nthFromLastInstant(int n);
/**
* @return true if the passed instant is present as a completed instant on the timeline
@@ -195,7 +195,7 @@ public interface HoodieTimeline extends Serializable {
/**
* Read the completed instant details
*/
Optional<byte[]> getInstantDetails(HoodieInstant instant);
Option<byte[]> getInstantDetails(HoodieInstant instant);
/**
* Helper methods to compare instants

View File

@@ -28,6 +28,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.exception.CorruptedLogFileException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
@@ -36,7 +37,6 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.BufferedFSInputStream;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -199,16 +199,16 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
return HoodieAvroDataBlock.getBlock(content, readerSchema);
} else {
return HoodieAvroDataBlock
.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily,
.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, readerSchema, header, footer);
}
case DELETE_BLOCK:
return HoodieDeleteBlock
.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily,
.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, header, footer);
case COMMAND_BLOCK:
return HoodieCommandBlock
.getBlock(logFile, inputStream, Optional.ofNullable(content), readBlockLazily,
.getBlock(logFile, inputStream, Option.ofNullable(content), readBlockLazily,
contentPosition, contentLength, blockEndPos, header, footer);
default:
throw new HoodieNotSupportedException("Unsupported Block " + blockType);
@@ -227,7 +227,7 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
byte[] corruptedBytes = HoodieLogBlock
.readOrSkipContent(inputStream, corruptedBlockSize, readBlockLazily);
return HoodieCorruptBlock
.getBlock(logFile, inputStream, Optional.ofNullable(corruptedBytes), readBlockLazily,
.getBlock(logFile, inputStream, Option.ofNullable(corruptedBytes), readBlockLazily,
contentPosition, corruptedBlockSize, corruptedBlockSize, new HashMap<>(),
new HashMap<>());
}

View File

@@ -21,11 +21,11 @@ package com.uber.hoodie.common.table.log;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.collection.Pair;
import java.io.Closeable;
import java.io.IOException;
import java.util.Iterator;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -209,7 +209,7 @@ public interface HoodieLogFormat {
if (logVersion == null) {
log.info("Computing the next log version for " + logFileId + " in " + parentPath);
Optional<Pair<Integer, String>> versionAndWriteToken =
Option<Pair<Integer, String>> versionAndWriteToken =
FSUtils.getLatestLogVersion(fs, parentPath, logFileId, fileExtension, commitTime);
if (versionAndWriteToken.isPresent()) {
logVersion = versionAndWriteToken.get().getKey();

View File

@@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
@@ -33,7 +34,6 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import javax.annotation.Nonnull;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericDatumReader;
@@ -62,7 +62,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
public HoodieAvroDataBlock(@Nonnull List<IndexedRecord> records,
@Nonnull Map<HeaderMetadataType, String> header,
@Nonnull Map<HeaderMetadataType, String> footer) {
super(header, footer, Optional.empty(), Optional.empty(), null, false);
super(header, footer, Option.empty(), Option.empty(), null, false);
this.records = records;
this.schema = Schema.parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
}
@@ -72,8 +72,8 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
this(records, header, new HashMap<>());
}
private HoodieAvroDataBlock(Optional<byte[]> content, @Nonnull FSDataInputStream inputStream,
boolean readBlockLazily, Optional<HoodieLogBlockContentLocation> blockContentLocation,
private HoodieAvroDataBlock(Option<byte[]> content, @Nonnull FSDataInputStream inputStream,
boolean readBlockLazily, Option<HoodieLogBlockContentLocation> blockContentLocation,
Schema readerSchema, @Nonnull Map<HeaderMetadataType, String> headers,
@Nonnull Map<HeaderMetadataType, String> footer) {
super(headers, footer, blockContentLocation, content, inputStream, readBlockLazily);
@@ -82,7 +82,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
public static HoodieLogBlock getBlock(HoodieLogFile logFile,
FSDataInputStream inputStream,
Optional<byte[]> content,
Option<byte[]> content,
boolean readBlockLazily,
long position,
long blockSize,
@@ -92,7 +92,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
Map<HeaderMetadataType, String> footer) {
return new HoodieAvroDataBlock(content, inputStream, readBlockLazily,
Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)),
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)),
readerSchema, header, footer);
}
@@ -230,7 +230,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock {
* which were written using HoodieLogFormat V1
*/
public HoodieAvroDataBlock(List<IndexedRecord> records, Schema schema) {
super(new HashMap<>(), new HashMap<>(), Optional.empty(), Optional.empty(), null, false);
super(new HashMap<>(), new HashMap<>(), Option.empty(), Option.empty(), null, false);
this.records = records;
this.schema = schema;
}

View File

@@ -19,9 +19,9 @@
package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.util.Option;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import org.apache.hadoop.fs.FSDataInputStream;
/**
@@ -36,11 +36,11 @@ public class HoodieCommandBlock extends HoodieLogBlock {
}
public HoodieCommandBlock(Map<HeaderMetadataType, String> header) {
this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>());
this(Option.empty(), null, false, Option.empty(), header, new HashMap<>());
}
private HoodieCommandBlock(Optional<byte[]> content, FSDataInputStream inputStream,
boolean readBlockLazily, Optional<HoodieLogBlockContentLocation> blockContentLocation,
private HoodieCommandBlock(Option<byte[]> content, FSDataInputStream inputStream,
boolean readBlockLazily, Option<HoodieLogBlockContentLocation> blockContentLocation,
Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer) {
super(header, footer, blockContentLocation, content, inputStream, readBlockLazily);
this.type = HoodieCommandBlockTypeEnum.values()[Integer
@@ -63,7 +63,7 @@ public class HoodieCommandBlock extends HoodieLogBlock {
public static HoodieLogBlock getBlock(HoodieLogFile logFile,
FSDataInputStream inputStream,
Optional<byte[]> content,
Option<byte[]> content,
boolean readBlockLazily,
long position,
long blockSize,
@@ -72,7 +72,7 @@ public class HoodieCommandBlock extends HoodieLogBlock {
Map<HeaderMetadataType, String> footer) {
return new HoodieCommandBlock(content, inputStream, readBlockLazily,
Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)),
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndpos)),
header, footer);
}
}

View File

@@ -19,9 +19,9 @@
package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.util.Option;
import java.io.IOException;
import java.util.Map;
import java.util.Optional;
import org.apache.hadoop.fs.FSDataInputStream;
/**
@@ -30,8 +30,8 @@ import org.apache.hadoop.fs.FSDataInputStream;
*/
public class HoodieCorruptBlock extends HoodieLogBlock {
private HoodieCorruptBlock(Optional<byte[]> corruptedBytes, FSDataInputStream inputStream,
boolean readBlockLazily, Optional<HoodieLogBlockContentLocation> blockContentLocation,
private HoodieCorruptBlock(Option<byte[]> corruptedBytes, FSDataInputStream inputStream,
boolean readBlockLazily, Option<HoodieLogBlockContentLocation> blockContentLocation,
Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer) {
super(header, footer, blockContentLocation, corruptedBytes, inputStream, readBlockLazily);
}
@@ -53,7 +53,7 @@ public class HoodieCorruptBlock extends HoodieLogBlock {
public static HoodieLogBlock getBlock(HoodieLogFile logFile,
FSDataInputStream inputStream,
Optional<byte[]> corruptedBytes,
Option<byte[]> corruptedBytes,
boolean readBlockLazily,
long position,
long blockSize,
@@ -62,7 +62,7 @@ public class HoodieCorruptBlock extends HoodieLogBlock {
Map<HeaderMetadataType, String> footer) {
return new HoodieCorruptBlock(corruptedBytes, inputStream, readBlockLazily,
Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)),
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)),
header, footer);
}
}

View File

@@ -21,6 +21,7 @@ package com.uber.hoodie.common.table.log.block;
import com.uber.hoodie.common.model.HoodieKey;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.storage.SizeAwareDataInputStream;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.SerializationUtils;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.ByteArrayInputStream;
@@ -30,7 +31,6 @@ import java.io.DataOutputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import org.apache.hadoop.fs.FSDataInputStream;
/**
@@ -42,13 +42,13 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
public HoodieDeleteBlock(HoodieKey[] keysToDelete,
Map<HeaderMetadataType, String> header) {
this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>());
this(Option.empty(), null, false, Option.empty(), header, new HashMap<>());
this.keysToDelete = keysToDelete;
}
private HoodieDeleteBlock(Optional<byte[]> content, FSDataInputStream inputStream,
boolean readBlockLazily, Optional<HoodieLogBlockContentLocation> blockContentLocation,
private HoodieDeleteBlock(Option<byte[]> content, FSDataInputStream inputStream,
boolean readBlockLazily, Option<HoodieLogBlockContentLocation> blockContentLocation,
Map<HeaderMetadataType, String> header, Map<HeaderMetadataType, String> footer) {
super(header, footer, blockContentLocation, content, inputStream, readBlockLazily);
}
@@ -103,7 +103,7 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
public static HoodieLogBlock getBlock(HoodieLogFile logFile,
FSDataInputStream inputStream,
Optional<byte[]> content,
Option<byte[]> content,
boolean readBlockLazily,
long position,
long blockSize,
@@ -112,7 +112,7 @@ public class HoodieDeleteBlock extends HoodieLogBlock {
Map<HeaderMetadataType, String> footer) throws IOException {
return new HoodieDeleteBlock(content, inputStream, readBlockLazily,
Optional.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)),
Option.of(new HoodieLogBlockContentLocation(logFile, position, blockSize, blockEndPos)),
header, footer);
}
}

View File

@@ -22,6 +22,7 @@ import com.google.common.collect.Maps;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.ByteArrayOutputStream;
@@ -30,7 +31,6 @@ import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.IOException;
import java.util.Map;
import java.util.Optional;
import javax.annotation.Nonnull;
import org.apache.hadoop.fs.FSDataInputStream;
@@ -51,9 +51,9 @@ public abstract class HoodieLogBlock {
// Footer for each log block
private final Map<HeaderMetadataType, String> logBlockFooter;
// Location of a log block on disk
private final Optional<HoodieLogBlockContentLocation> blockContentLocation;
private final Option<HoodieLogBlockContentLocation> blockContentLocation;
// data for a specific block
private Optional<byte[]> content;
private Option<byte[]> content;
// TODO : change this to just InputStream so this works for any FileSystem
// create handlers to return specific type of inputstream based on FS
// input stream corresponding to the log file where this logBlock belongs
@@ -63,8 +63,8 @@ public abstract class HoodieLogBlock {
public HoodieLogBlock(@Nonnull Map<HeaderMetadataType, String> logBlockHeader,
@Nonnull Map<HeaderMetadataType, String> logBlockFooter,
@Nonnull Optional<HoodieLogBlockContentLocation> blockContentLocation,
@Nonnull Optional<byte[]> content,
@Nonnull Option<HoodieLogBlockContentLocation> blockContentLocation,
@Nonnull Option<byte[]> content,
FSDataInputStream inputStream,
boolean readBlockLazily) {
this.logBlockHeader = logBlockHeader;
@@ -92,7 +92,7 @@ public abstract class HoodieLogBlock {
throw new HoodieException("No implementation was provided");
}
public Optional<HoodieLogBlockContentLocation> getBlockContentLocation() {
public Option<HoodieLogBlockContentLocation> getBlockContentLocation() {
return this.blockContentLocation;
}
@@ -104,7 +104,7 @@ public abstract class HoodieLogBlock {
return logBlockFooter;
}
public Optional<byte[]> getContent() {
public Option<byte[]> getContent() {
return content;
}
@@ -245,7 +245,7 @@ public abstract class HoodieLogBlock {
protected void inflate() throws IOException {
try {
content = Optional.of(new byte[(int) this.getBlockContentLocation().get().getBlockSize()]);
content = Option.of(new byte[(int) this.getBlockContentLocation().get().getBlockSize()]);
safeSeek(inputStream, this.getBlockContentLocation().get().getContentPositionInLogFile());
inputStream.readFully(content.get(), 0, content.get().length);
safeSeek(inputStream, this.getBlockContentLocation().get().getBlockEndPos());
@@ -266,7 +266,7 @@ public abstract class HoodieLogBlock {
* leaves the heap fragmented
*/
protected void deflate() {
content = Optional.empty();
content = Option.empty();
}
/**

View File

@@ -24,6 +24,7 @@ import com.google.common.collect.Sets;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.io.Serializable;
@@ -31,7 +32,6 @@ import java.text.SimpleDateFormat;
import java.util.Arrays;
import java.util.Date;
import java.util.HashSet;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import java.util.stream.Stream;
@@ -84,7 +84,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
// multiple casts will make this lambda serializable -
// http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details =
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
(Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails;
}
public HoodieActiveTimeline(HoodieTableMetaClient metaClient) {
@@ -153,7 +153,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
*/
public HoodieTimeline getDeltaCommitTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(DELTA_COMMIT_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
(Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails);
}
/**
@@ -164,7 +164,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
*/
public HoodieTimeline getTimelineOfActions(Set<String> actions) {
return new HoodieDefaultTimeline(getInstants().filter(s -> actions.contains(s.getAction())),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
(Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails);
}
@@ -173,7 +173,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
*/
public HoodieTimeline getCleanerTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(CLEAN_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
(Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails);
}
/**
@@ -181,7 +181,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
*/
public HoodieTimeline getRollbackTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(ROLLBACK_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
(Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails);
}
/**
@@ -189,7 +189,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
*/
public HoodieTimeline getSavePointTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(SAVEPOINT_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
(Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails);
}
/**
@@ -197,7 +197,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
*/
public HoodieTimeline getRestoreTimeline() {
return new HoodieDefaultTimeline(filterInstantsByAction(RESTORE_ACTION),
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
(Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails);
}
protected Stream<HoodieInstant> filterInstantsByAction(String action) {
@@ -207,10 +207,10 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
public void createInflight(HoodieInstant instant) {
log.info("Creating a new in-flight instant " + instant);
// Create the in-flight file
createFileInMetaPath(instant.getFileName(), Optional.empty());
createFileInMetaPath(instant.getFileName(), Option.empty());
}
public void saveAsComplete(HoodieInstant instant, Optional<byte[]> data) {
public void saveAsComplete(HoodieInstant instant, Option<byte[]> data) {
log.info("Marking instant complete " + instant);
Preconditions.checkArgument(instant.isInflight(),
"Could not mark an already completed instant as complete again " + instant);
@@ -252,14 +252,14 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
public Option<byte[]> getInstantDetails(HoodieInstant instant) {
Path detailPath = new Path(metaClient.getMetaPath(), instant.getFileName());
return readDataFromPath(detailPath);
}
/** BEGIN - COMPACTION RELATED META-DATA MANAGEMENT **/
public Optional<byte[]> getInstantAuxiliaryDetails(HoodieInstant instant) {
public Option<byte[]> getInstantAuxiliaryDetails(HoodieInstant instant) {
Path detailPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
return readDataFromPath(detailPath);
}
@@ -276,7 +276,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
HoodieInstant requestedInstant =
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, inflightInstant.getTimestamp());
// Pass empty data since it is read from the corresponding .aux/.compaction instant file
transitionState(inflightInstant, requestedInstant, Optional.empty());
transitionState(inflightInstant, requestedInstant, Option.empty());
return requestedInstant;
}
@@ -291,7 +291,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
Preconditions.checkArgument(requestedInstant.isRequested());
HoodieInstant inflightInstant =
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, requestedInstant.getTimestamp());
transitionState(requestedInstant, inflightInstant, Optional.empty());
transitionState(requestedInstant, inflightInstant, Option.empty());
return inflightInstant;
}
@@ -302,7 +302,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
* @param data Extra Metadata
* @return commit instant
*/
public HoodieInstant transitionCompactionInflightToComplete(HoodieInstant inflightInstant, Optional<byte[]> data) {
public HoodieInstant transitionCompactionInflightToComplete(HoodieInstant inflightInstant, Option<byte[]> data) {
Preconditions.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
Preconditions.checkArgument(inflightInstant.isInflight());
HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, COMMIT_ACTION, inflightInstant.getTimestamp());
@@ -310,7 +310,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
return commitInstant;
}
private void createFileInAuxiliaryFolder(HoodieInstant instant, Optional<byte[]> data) {
private void createFileInAuxiliaryFolder(HoodieInstant instant, Option<byte[]> data) {
Path fullPath = new Path(metaClient.getMetaAuxiliaryPath(), instant.getFileName());
createFileInPath(fullPath, data);
}
@@ -320,7 +320,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
**/
private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant,
Optional<byte[]> data) {
Option<byte[]> data) {
Preconditions.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp()));
Path commitFilePath = new Path(metaClient.getMetaPath(), toInstant.getFileName());
try {
@@ -354,24 +354,24 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
}
public void saveToInflight(HoodieInstant instant, Optional<byte[]> content) {
public void saveToInflight(HoodieInstant instant, Option<byte[]> content) {
Preconditions.checkArgument(instant.isInflight());
createFileInMetaPath(instant.getFileName(), content);
}
public void saveToCompactionRequested(HoodieInstant instant, Optional<byte[]> content) {
public void saveToCompactionRequested(HoodieInstant instant, Option<byte[]> content) {
Preconditions.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION));
// Write workload to auxiliary folder
createFileInAuxiliaryFolder(instant, content);
createFileInMetaPath(instant.getFileName(), content);
}
private void createFileInMetaPath(String filename, Optional<byte[]> content) {
private void createFileInMetaPath(String filename, Option<byte[]> content) {
Path fullPath = new Path(metaClient.getMetaPath(), filename);
createFileInPath(fullPath, content);
}
private void createFileInPath(Path fullPath, Optional<byte[]> content) {
private void createFileInPath(Path fullPath, Option<byte[]> content) {
try {
// If the path does not exist, create it first
if (!metaClient.getFs().exists(fullPath)) {
@@ -392,9 +392,9 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
}
}
private Optional<byte[]> readDataFromPath(Path detailPath) {
private Option<byte[]> readDataFromPath(Path detailPath) {
try (FSDataInputStream is = metaClient.getFs().open(detailPath)) {
return Optional.of(IOUtils.toByteArray(is));
return Option.of(IOUtils.toByteArray(is));
} catch (IOException e) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
}

View File

@@ -20,13 +20,13 @@ package com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.io.Serializable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.Path;
@@ -73,7 +73,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
// multiple casts will make this lambda serializable -
// http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16
this.details =
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails;
(Function<HoodieInstant, Option<byte[]>> & Serializable) this::getInstantDetails;
this.metaClient = metaClient;
}
@@ -101,8 +101,8 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
return Optional.ofNullable(readCommits.get(instant.getTimestamp()));
public Option<byte[]> getInstantDetails(HoodieInstant instant) {
return Option.ofNullable(readCommits.get(instant.getTimestamp()));
}
public HoodieArchivedTimeline reload() {

View File

@@ -20,12 +20,12 @@ package com.uber.hoodie.common.table.timeline;
import com.google.common.collect.Sets;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.StringUtils;
import com.uber.hoodie.exception.HoodieException;
import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import java.util.function.Predicate;
@@ -48,12 +48,12 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
private static final String HASHING_ALGORITHM = "SHA-256";
protected transient Function<HoodieInstant, Optional<byte[]>> details;
protected transient Function<HoodieInstant, Option<byte[]>> details;
private List<HoodieInstant> instants;
private String timelineHash;
public HoodieDefaultTimeline(Stream<HoodieInstant> instants,
Function<HoodieInstant, Optional<byte[]>> details) {
Function<HoodieInstant, Option<byte[]>> details) {
this.details = details;
setInstants(instants.collect(Collectors.toList()));
}
@@ -149,27 +149,27 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
}
@Override
public Optional<HoodieInstant> firstInstant() {
return instants.stream().findFirst();
public Option<HoodieInstant> firstInstant() {
return Option.fromJavaOptional(instants.stream().findFirst());
}
@Override
public Optional<HoodieInstant> nthInstant(int n) {
public Option<HoodieInstant> nthInstant(int n) {
if (empty() || n >= countInstants()) {
return Optional.empty();
return Option.empty();
}
return Optional.of(instants.get(n));
return Option.of(instants.get(n));
}
@Override
public Optional<HoodieInstant> lastInstant() {
return empty() ? Optional.empty() : nthInstant(countInstants() - 1);
public Option<HoodieInstant> lastInstant() {
return empty() ? Option.empty() : nthInstant(countInstants() - 1);
}
@Override
public Optional<HoodieInstant> nthFromLastInstant(int n) {
public Option<HoodieInstant> nthFromLastInstant(int n) {
if (countInstants() < n + 1) {
return Optional.empty();
return Option.empty();
}
return nthInstant(countInstants() - 1 - n);
}
@@ -197,13 +197,13 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
@Override
public boolean isBeforeTimelineStarts(String instant) {
Optional<HoodieInstant> firstCommit = firstInstant();
Option<HoodieInstant> firstCommit = firstInstant();
return firstCommit.isPresent()
&& HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER);
}
@Override
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
public Option<byte[]> getInstantDetails(HoodieInstant instant) {
return details.apply(instant);
}

View File

@@ -42,7 +42,6 @@ import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@@ -359,15 +358,15 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllStoredFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getAllDataFiles()
.map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllDataFiles()
.filter(dataFile ->
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL))
.filter(df -> !isDataFileDueToPendingCompaction(df))
.findFirst())
.filter(Optional::isPresent)
.map(Optional::get);
.findFirst()))
.filter(Option::isPresent)
.map(Option::get);
} finally {
readLock.unlock();
}
@@ -410,11 +409,11 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
try {
readLock.lock();
return fetchAllStoredFileGroups().map(fileGroup -> {
return fileGroup.getAllDataFiles()
return Option.fromJavaOptional(fileGroup.getAllDataFiles()
.filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime())
&& !isDataFileDueToPendingCompaction(dataFile))
.findFirst();
}).filter(Optional::isPresent).map(Optional::get);
.findFirst());
}).filter(Option::isPresent).map(Option::get);
} finally {
readLock.unlock();
}
@@ -477,9 +476,9 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
String compactionInstantTime = compactionWithInstantPair.get().getLeft();
return fileGroup.getLatestFileSliceBefore(compactionInstantTime);
}
return Optional.of(fileSlice);
return Option.of(fileSlice);
})
.map(Optional::get);
.map(Option::get);
} finally {
readLock.unlock();
}
@@ -512,15 +511,15 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
ensurePartitionLoadedCorrectly(partition);
return fetchAllStoredFileGroups(partition)
.map(fileGroup -> {
Optional<FileSlice> fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime);
Option<FileSlice> fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime);
// if the file-group is under construction, pick the latest before compaction instant time.
if (fileSlice.isPresent()) {
fileSlice = Optional.of(fetchMergedFileSlice(fileGroup, fileSlice.get()));
fileSlice = Option.of(fetchMergedFileSlice(fileGroup, fileSlice.get()));
}
return fileSlice;
})
.filter(Optional::isPresent)
.map(Optional::get);
.filter(Option::isPresent)
.map(Option::get);
} finally {
readLock.unlock();
}
@@ -656,7 +655,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
*/
Stream<FileSlice> fetchLatestFileSliceInRange(List<String> commitsToReturn) {
return fetchAllStoredFileGroups().map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
.map(Optional::get);
.map(Option::get);
}
/**
@@ -677,13 +676,14 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
Stream<HoodieDataFile> fetchLatestDataFiles(final String partitionPath) {
return fetchAllStoredFileGroups(partitionPath)
.map(this::getLatestDataFile)
.filter(Optional::isPresent)
.map(Optional::get);
.filter(Option::isPresent)
.map(Option::get);
}
protected Optional<HoodieDataFile> getLatestDataFile(HoodieFileGroup fileGroup) {
return fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst();
protected Option<HoodieDataFile> getLatestDataFile(HoodieFileGroup fileGroup) {
return Option.fromJavaOptional(
fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst());
}
/**
@@ -692,8 +692,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
Stream<HoodieDataFile> fetchLatestDataFiles() {
return fetchAllStoredFileGroups()
.map(this::getLatestDataFile)
.filter(Optional::isPresent)
.map(Optional::get);
.filter(Option::isPresent)
.map(Option::get);
}
/**
@@ -721,8 +721,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
Stream<FileSlice> fetchLatestFileSlices(String partitionPath) {
return fetchAllStoredFileGroups(partitionPath)
.map(HoodieFileGroup::getLatestFileSlice)
.filter(Optional::isPresent)
.map(Optional::get);
.filter(Option::isPresent)
.map(Option::get);
}
/**
@@ -735,8 +735,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
String maxCommitTime) {
return fetchAllStoredFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
.filter(Optional::isPresent)
.map(Optional::get);
.filter(Option::isPresent)
.map(Option::get);
}
/**
@@ -771,7 +771,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
if (compactionOpWithInstant.isPresent()) {
String compactionInstantTime = compactionOpWithInstant.get().getKey();
if (fileSlice.getBaseInstantTime().equals(compactionInstantTime)) {
Optional<FileSlice> prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime);
Option<FileSlice> prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime);
if (prevFileSlice.isPresent()) {
return mergeCompactionPendingFileSlices(fileSlice, prevFileSlice.get());
}
@@ -804,7 +804,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV
@Override
public Option<HoodieInstant> getLastInstant() {
return Option.fromJavaOptional(getTimeline().lastInstant());
return getTimeline().lastInstant();
}
@Override

View File

@@ -32,6 +32,7 @@ import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.CompactionUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.TimelineDiffHelper;
import com.uber.hoodie.common.util.TimelineDiffHelper.TimelineDiffResult;
import com.uber.hoodie.common.util.collection.Pair;
@@ -39,7 +40,6 @@ import com.uber.hoodie.exception.HoodieException;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
@@ -310,12 +310,12 @@ public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTabl
* the base-path,scheme and authority. Ensure the matching process takes care of this discrepancy.
*/
Map<String, HoodieDataFile> viewDataFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
.map(FileSlice::getDataFile).filter(Optional::isPresent).map(Optional::get)
.map(FileSlice::getDataFile).filter(Option::isPresent).map(Option::get)
.map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
//Note: Delta Log Files and Data FIles can be empty when adding/removing pending compactions
Map<String, HoodieDataFile> deltaDataFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
.map(FileSlice::getDataFile).filter(Optional::isPresent).map(Optional::get)
.map(FileSlice::getDataFile).filter(Option::isPresent).map(Option::get)
.map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));

View File

@@ -40,7 +40,6 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileWriter;
@@ -86,7 +85,7 @@ public class AvroUtils {
String partitionPath =
deltaRecord.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
loadedRecords.add(new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Optional.of(deltaRecord))));
new HoodieAvroPayload(Option.of(deltaRecord))));
}
fileReader.close(); // also closes underlying FsInput
} catch (IOException e) {
@@ -97,7 +96,7 @@ public class AvroUtils {
}
public static HoodieCleanMetadata convertCleanMetadata(String startCleanTime,
Optional<Long> durationInMs, List<HoodieCleanStat> cleanStats) {
Option<Long> durationInMs, List<HoodieCleanStat> cleanStats) {
ImmutableMap.Builder<String, HoodieCleanPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.builder();
int totalDeleted = 0;
@@ -119,7 +118,7 @@ public class AvroUtils {
}
public static HoodieRestoreMetadata convertRestoreMetadata(String startRestoreTime,
Optional<Long> durationInMs, List<String> commits, Map<String, List<HoodieRollbackStat>> commitToStats) {
Option<Long> durationInMs, List<String> commits, Map<String, List<HoodieRollbackStat>> commitToStats) {
ImmutableMap.Builder<String, List<HoodieRollbackMetadata>> commitToStatBuilder = ImmutableMap.builder();
for (Map.Entry<String, List<HoodieRollbackStat>> commitToStat : commitToStats.entrySet()) {
commitToStatBuilder.put(commitToStat.getKey(), Arrays.asList(convertRollbackMetadata(startRestoreTime,
@@ -130,7 +129,7 @@ public class AvroUtils {
}
public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime,
Optional<Long> durationInMs, List<String> commits, List<HoodieRollbackStat> rollbackStats) {
Option<Long> durationInMs, List<String> commits, List<HoodieRollbackStat> rollbackStats) {
ImmutableMap.Builder<String, HoodieRollbackPartitionMetadata> partitionMetadataBuilder =
ImmutableMap.builder();
int totalDeleted = 0;
@@ -159,32 +158,32 @@ public class AvroUtils {
partitionMetadataBuilder.build());
}
public static Optional<byte[]> serializeCompactionPlan(HoodieCompactionPlan compactionWorkload)
public static Option<byte[]> serializeCompactionPlan(HoodieCompactionPlan compactionWorkload)
throws IOException {
return serializeAvroMetadata(compactionWorkload, HoodieCompactionPlan.class);
}
public static Optional<byte[]> serializeCleanMetadata(HoodieCleanMetadata metadata)
public static Option<byte[]> serializeCleanMetadata(HoodieCleanMetadata metadata)
throws IOException {
return serializeAvroMetadata(metadata, HoodieCleanMetadata.class);
}
public static Optional<byte[]> serializeSavepointMetadata(HoodieSavepointMetadata metadata)
public static Option<byte[]> serializeSavepointMetadata(HoodieSavepointMetadata metadata)
throws IOException {
return serializeAvroMetadata(metadata, HoodieSavepointMetadata.class);
}
public static Optional<byte[]> serializeRollbackMetadata(
public static Option<byte[]> serializeRollbackMetadata(
HoodieRollbackMetadata rollbackMetadata) throws IOException {
return serializeAvroMetadata(rollbackMetadata, HoodieRollbackMetadata.class);
}
public static Optional<byte[]> serializeRestoreMetadata(
public static Option<byte[]> serializeRestoreMetadata(
HoodieRestoreMetadata restoreMetadata) throws IOException {
return serializeAvroMetadata(restoreMetadata, HoodieRestoreMetadata.class);
}
public static <T extends SpecificRecordBase> Optional<byte[]> serializeAvroMetadata(T metadata,
public static <T extends SpecificRecordBase> Option<byte[]> serializeAvroMetadata(T metadata,
Class<T> clazz) throws IOException {
DatumWriter<T> datumWriter = new SpecificDatumWriter<>(clazz);
DataFileWriter<T> fileWriter = new DataFileWriter<>(datumWriter);
@@ -192,7 +191,7 @@ public class AvroUtils {
fileWriter.create(metadata.getSchema(), baos);
fileWriter.append(metadata);
fileWriter.flush();
return Optional.of(baos.toByteArray());
return Option.of(baos.toByteArray());
}
public static HoodieCompactionPlan deserializeCompactionPlan(byte[] bytes)

View File

@@ -32,7 +32,6 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -55,7 +54,7 @@ public class CompactionUtils {
* @return Compaction Operation
*/
public static HoodieCompactionOperation buildFromFileSlice(String partitionPath, FileSlice fileSlice,
Optional<Function<Pair<String, FileSlice>, Map<String, Double>>> metricsCaptureFunction) {
Option<Function<Pair<String, FileSlice>, Map<String, Double>>> metricsCaptureFunction) {
HoodieCompactionOperation.Builder builder = HoodieCompactionOperation.newBuilder();
builder.setPartitionPath(partitionPath);
builder.setFileId(fileSlice.getFileId());
@@ -80,8 +79,8 @@ public class CompactionUtils {
*/
public static HoodieCompactionPlan buildFromFileSlices(
List<Pair<String, FileSlice>> partitionFileSlicePairs,
Optional<Map<String, String>> extraMetadata,
Optional<Function<Pair<String, FileSlice>, Map<String, Double>>> metricsCaptureFunction) {
Option<Map<String, String>> extraMetadata,
Option<Function<Pair<String, FileSlice>, Map<String, Double>>> metricsCaptureFunction) {
HoodieCompactionPlan.Builder builder = HoodieCompactionPlan.newBuilder();
extraMetadata.ifPresent(m -> builder.setExtraMetadata(m));
builder.setOperations(partitionFileSlicePairs.stream().map(pfPair ->

View File

@@ -38,7 +38,6 @@ import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import java.util.Map.Entry;
import java.util.Optional;
import java.util.UUID;
import java.util.function.Function;
import java.util.regex.Matcher;
@@ -416,8 +415,8 @@ public class FSUtils {
/**
* Get the latest log file written from the list of log files passed in
*/
public static Optional<HoodieLogFile> getLatestLogFile(Stream<HoodieLogFile> logFiles) {
return logFiles.sorted(HoodieLogFile.getReverseLogFileComparator()).findFirst();
public static Option<HoodieLogFile> getLatestLogFile(Stream<HoodieLogFile> logFiles) {
return Option.fromJavaOptional(logFiles.sorted(HoodieLogFile.getReverseLogFileComparator()).findFirst());
}
/**
@@ -435,17 +434,17 @@ public class FSUtils {
/**
* Get the latest log version for the fileId in the partition path
*/
public static Optional<Pair<Integer, String>> getLatestLogVersion(FileSystem fs, Path partitionPath,
public static Option<Pair<Integer, String>> getLatestLogVersion(FileSystem fs, Path partitionPath,
final String fileId, final String logFileExtension, final String baseCommitTime)
throws IOException {
Optional<HoodieLogFile> latestLogFile =
Option<HoodieLogFile> latestLogFile =
getLatestLogFile(
getAllLogFiles(fs, partitionPath, fileId, logFileExtension, baseCommitTime));
if (latestLogFile.isPresent()) {
return Optional.of(Pair.of(latestLogFile.get().getLogVersion(),
return Option.of(Pair.of(latestLogFile.get().getLogVersion(),
getWriteTokenFromLogPath(latestLogFile.get().getPath())));
}
return Optional.empty();
return Option.empty();
}
/**
@@ -453,7 +452,7 @@ public class FSUtils {
*/
public static int computeNextLogVersion(FileSystem fs, Path partitionPath, final String fileId,
final String logFileExtension, final String baseCommitTime) throws IOException {
Optional<Pair<Integer, String>> currentVersionWithWriteToken =
Option<Pair<Integer, String>> currentVersionWithWriteToken =
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
// handle potential overflow
return (currentVersionWithWriteToken.isPresent()) ? currentVersionWithWriteToken.get().getKey() + 1

View File

@@ -31,7 +31,6 @@ import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.function.Function;
import org.apache.avro.Schema;
@@ -73,9 +72,9 @@ public class ParquetUtils {
* @return Set Set of row keys matching candidateRecordKeys
*/
public static Set<String> filterParquetRowKeys(Configuration configuration, Path filePath, Set<String> filter) {
Optional<RecordKeysFilterFunction> filterFunction = Optional.empty();
Option<RecordKeysFilterFunction> filterFunction = Option.empty();
if (CollectionUtils.isNotEmpty(filter)) {
filterFunction = Optional.of(new RecordKeysFilterFunction(filter));
filterFunction = Option.of(new RecordKeysFilterFunction(filter));
}
Configuration conf = new Configuration(configuration);
conf.addResource(getFs(filePath.toString(), conf).getConf());

View File

@@ -26,7 +26,6 @@ import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStre
import com.uber.hoodie.exception.HoodieCorruptedDataException;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.util.Optional;
import java.util.zip.CRC32;
import org.apache.avro.generic.GenericRecord;
@@ -116,7 +115,7 @@ public class SpillableMapUtils {
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieRecord<>(
new HoodieKey(recKey, partitionPath),
ReflectionUtils
.loadPayload(payloadClazz, new Object[]{Optional.of(rec)}, Optional.class));
.loadPayload(payloadClazz, new Object[]{Option.of(rec)}, Option.class));
return (R) hoodieRecord;
}
@@ -127,7 +126,7 @@ public class SpillableMapUtils {
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = new HoodieRecord<>(
new HoodieKey(recKey, partitionPath),
ReflectionUtils
.loadPayload(payloadClazz, new Object[]{Optional.empty()}, Optional.class));
.loadPayload(payloadClazz, new Object[]{Option.empty()}, Option.class));
return (R) hoodieRecord;
}
}

View File

@@ -24,7 +24,6 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.collection.Pair;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
import org.apache.log4j.LogManager;
@@ -41,8 +40,8 @@ public class TimelineDiffHelper {
HoodieTimeline oldT = oldTimeline.filterCompletedAndCompactionInstants();
HoodieTimeline newT = newTimeline.filterCompletedAndCompactionInstants();
Optional<HoodieInstant> lastSeenInstant = oldT.lastInstant();
Optional<HoodieInstant> firstInstantInNewTimeline = newT.firstInstant();
Option<HoodieInstant> lastSeenInstant = oldT.lastInstant();
Option<HoodieInstant> firstInstantInNewTimeline = newT.firstInstant();
if (lastSeenInstant.isPresent() && firstInstantInNewTimeline.isPresent()) {
if (HoodieTimeline.compareTimestamps(lastSeenInstant.get().getTimestamp(),

View File

@@ -20,11 +20,11 @@
package com.uber.hoodie.common.util.queue;
import com.uber.hoodie.common.util.DefaultSizeEstimator;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.SizeEstimator;
import com.uber.hoodie.exception.HoodieException;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorCompletionService;
@@ -52,18 +52,18 @@ public class BoundedInMemoryExecutor<I, O, E> {
// Producers
private final List<BoundedInMemoryQueueProducer<I>> producers;
// Consumer
private final Optional<BoundedInMemoryQueueConsumer<O, E>> consumer;
private final Option<BoundedInMemoryQueueConsumer<O, E>> consumer;
public BoundedInMemoryExecutor(final long bufferLimitInBytes,
BoundedInMemoryQueueProducer<I> producer,
Optional<BoundedInMemoryQueueConsumer<O, E>> consumer,
Option<BoundedInMemoryQueueConsumer<O, E>> consumer,
final Function<I, O> transformFunction) {
this(bufferLimitInBytes, Arrays.asList(producer), consumer, transformFunction, new DefaultSizeEstimator<>());
}
public BoundedInMemoryExecutor(final long bufferLimitInBytes,
List<BoundedInMemoryQueueProducer<I>> producers,
Optional<BoundedInMemoryQueueConsumer<O, E>> consumer,
Option<BoundedInMemoryQueueConsumer<O, E>> consumer,
final Function<I, O> transformFunction,
final SizeEstimator<O> sizeEstimator) {
this.producers = producers;

View File

@@ -21,10 +21,10 @@ package com.uber.hoodie.common.util.queue;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.uber.hoodie.common.util.DefaultSizeEstimator;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.SizeEstimator;
import com.uber.hoodie.exception.HoodieException;
import java.util.Iterator;
import java.util.Optional;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
@@ -65,7 +65,7 @@ public class BoundedInMemoryQueue<I, O> implements Iterable<O> {
// used for sampling records with "RECORD_SAMPLING_RATE" frequency.
public final AtomicLong samplingRecordCounter = new AtomicLong(-1);
// internal queue for records.
private final LinkedBlockingQueue<Optional<O>> queue = new
private final LinkedBlockingQueue<Option<O>> queue = new
LinkedBlockingQueue<>();
// maximum amount of memory to be used for queueing records.
private final long memoryLimit;
@@ -176,7 +176,7 @@ public class BoundedInMemoryQueue<I, O> implements Iterable<O> {
// and record creation to it.
final O payload = transformFunction.apply(t);
adjustBufferSizeIfNeeded(payload);
queue.put(Optional.of(payload));
queue.put(Option.of(payload));
}
/**
@@ -190,13 +190,13 @@ public class BoundedInMemoryQueue<I, O> implements Iterable<O> {
* Reader interface but never exposed to outside world as this is a single consumer queue.
* Reading is done through a singleton iterator for this queue.
*/
private Optional<O> readNextRecord() {
private Option<O> readNextRecord() {
if (this.isReadDone.get()) {
return Optional.empty();
return Option.empty();
}
rateLimiter.release();
Optional<O> newRecord = Optional.empty();
Option<O> newRecord = Option.empty();
while (expectMoreRecords()) {
try {
throwExceptionIfFailed();
@@ -217,7 +217,7 @@ public class BoundedInMemoryQueue<I, O> implements Iterable<O> {
} else {
// We are done reading all the records from internal iterator.
this.isReadDone.set(true);
return Optional.empty();
return Option.empty();
}
}
@@ -261,7 +261,7 @@ public class BoundedInMemoryQueue<I, O> implements Iterable<O> {
@Override
public boolean hasNext() {
if (this.nextRecord == null) {
Optional<O> res = readNextRecord();
Option<O> res = readNextRecord();
this.nextRecord = res.orElse(null);
}
return this.nextRecord != null;

View File

@@ -19,9 +19,9 @@
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.exception.HoodieIOException;
import java.io.IOException;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -30,7 +30,7 @@ public class AvroBinaryTestPayload implements HoodieRecordPayload {
private final byte[] recordBytes;
public AvroBinaryTestPayload(Optional<GenericRecord> record) {
public AvroBinaryTestPayload(Option<GenericRecord> record) {
try {
if (record.isPresent()) {
@@ -49,13 +49,13 @@ public class AvroBinaryTestPayload implements HoodieRecordPayload {
}
@Override
public Optional<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
public Option<IndexedRecord> combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
throws IOException {
return getInsertValue(schema);
}
@Override
public Optional<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
public Option<IndexedRecord> getInsertValue(Schema schema) throws IOException {
return Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema));
}
}

View File

@@ -45,6 +45,7 @@ import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.CompactionUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.collection.Pair;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
@@ -59,7 +60,6 @@ import java.util.Date;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Properties;
import java.util.Random;
import java.util.UUID;
@@ -190,7 +190,7 @@ public class HoodieTestUtils {
}
public static final String createNewLogFile(FileSystem fs, String basePath, String partitionPath, String commitTime,
String fileID, Optional<Integer> version) throws IOException {
String fileID, Option<Integer> version) throws IOException {
String folderPath = basePath + "/" + partitionPath + "/";
boolean makeDir = fs.mkdirs(new Path(folderPath));
if (!makeDir) {
@@ -221,7 +221,7 @@ public class HoodieTestUtils {
public static final void createCompactionRequest(HoodieTableMetaClient metaClient, String instant,
List<Pair<String, FileSlice>> fileSliceList) throws IOException {
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(fileSliceList, Optional.empty(), Optional.empty());
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(fileSliceList, Option.empty(), Option.empty());
HoodieInstant compactionInstant =
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant);
metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant,
@@ -234,7 +234,7 @@ public class HoodieTestUtils {
}
public static final String getLogFilePath(String basePath, String partitionPath, String commitTime, String fileID,
Optional<Integer> version) {
Option<Integer> version) {
return basePath + "/" + partitionPath + "/" + FSUtils.makeLogFileName(fileID, ".log", commitTime,
version.orElse(DEFAULT_LOG_VERSION), HoodieLogFormat.UNKNOWN_WRITE_TOKEN);
}
@@ -259,7 +259,7 @@ public class HoodieTestUtils {
}
public static final boolean doesLogFileExist(String basePath, String partitionPath, String commitTime, String fileID,
Optional<Integer> version) {
Option<Integer> version) {
return new File(getLogFilePath(basePath, partitionPath, commitTime, fileID, version)).exists();
}
@@ -286,7 +286,7 @@ public class HoodieTestUtils {
DEFAULT_PARTITION_PATHS[rand.nextInt(DEFAULT_PARTITION_PATHS.length)], new ArrayList<>(), new ArrayList<>(),
new ArrayList<>(), commitTime);
// Create the clean metadata
HoodieCleanMetadata cleanMetadata = AvroUtils.convertCleanMetadata(commitTime, Optional.of(0L),
HoodieCleanMetadata cleanMetadata = AvroUtils.convertCleanMetadata(commitTime, Option.of(0L),
Arrays.asList(cleanStats));
// Write empty clean metadata
os.write(AvroUtils.serializeCleanMetadata(cleanMetadata).get());

View File

@@ -29,8 +29,8 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.Option;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.IOUtils;
@@ -72,7 +72,7 @@ public class HoodieTableMetaClientTest {
HoodieActiveTimeline commitTimeline = deseralizedMetaClient.getActiveTimeline();
HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
commitTimeline.createInflight(instant);
commitTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes()));
commitTimeline.saveAsComplete(instant, Option.of("test-detail".getBytes()));
commitTimeline = commitTimeline.reload();
HoodieInstant completedInstant = HoodieTimeline.getCompletedInstant(instant);
assertEquals("Commit should be 1 and completed", completedInstant, commitTimeline.getInstants().findFirst().get());
@@ -88,7 +88,7 @@ public class HoodieTableMetaClientTest {
HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1");
activeTimeline.createInflight(instant);
activeTimeline.saveAsComplete(instant, Optional.of("test-detail".getBytes()));
activeTimeline.saveAsComplete(instant, Option.of("test-detail".getBytes()));
// Commit timeline should not auto-reload every time getActiveCommitTimeline(), it should be cached
activeTimeline = metaClient.getActiveTimeline();

View File

@@ -27,8 +27,8 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.Option;
import java.io.IOException;
import java.util.Optional;
import java.util.stream.Stream;
import org.junit.Before;
import org.junit.Rule;
@@ -64,10 +64,10 @@ public class HoodieActiveTimelineTest {
HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9");
timeline = new HoodieActiveTimeline(metaClient);
timeline.saveAsComplete(instant1, Optional.empty());
timeline.saveAsComplete(instant2, Optional.empty());
timeline.saveAsComplete(instant3, Optional.empty());
timeline.saveAsComplete(instant4, Optional.empty());
timeline.saveAsComplete(instant1, Option.empty());
timeline.saveAsComplete(instant2, Option.empty());
timeline.saveAsComplete(instant3, Option.empty());
timeline.saveAsComplete(instant4, Option.empty());
timeline.createInflight(instant5);
timeline = timeline.reload();
@@ -90,10 +90,10 @@ public class HoodieActiveTimelineTest {
timeline = new HoodieActiveTimeline(metaClient);
assertTrue(timeline.empty());
assertEquals("", 0, timeline.countInstants());
assertEquals("", Optional.empty(), timeline.firstInstant());
assertEquals("", Optional.empty(), timeline.nthInstant(5));
assertEquals("", Optional.empty(), timeline.nthInstant(-1));
assertEquals("", Optional.empty(), timeline.lastInstant());
assertEquals("", Option.empty(), timeline.firstInstant());
assertEquals("", Option.empty(), timeline.nthInstant(5));
assertEquals("", Option.empty(), timeline.nthInstant(-1));
assertEquals("", Option.empty(), timeline.lastInstant());
assertFalse("", timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01")));
}

View File

@@ -43,6 +43,7 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.CompactionUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.collection.Pair;
import java.io.File;
import java.io.IOException;
@@ -51,7 +52,6 @@ import java.util.Arrays;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
@@ -137,9 +137,9 @@ public class HoodieTableFileSystemViewTest {
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
commitTimeline.saveAsComplete(instant1, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant2, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant3, Optional.empty());
commitTimeline.saveAsComplete(instant1, Option.empty());
commitTimeline.saveAsComplete(deltaInstant2, Option.empty());
commitTimeline.saveAsComplete(deltaInstant3, Option.empty());
refreshFsView();
@@ -229,7 +229,7 @@ public class HoodieTableFileSystemViewTest {
return fsView.getAllFileGroups(partitionPath)
.map(fileGroup -> fileGroup.getLatestFileSlicesIncludingInflight())
.filter(fileSliceOpt -> fileSliceOpt.isPresent())
.map(Optional::get);
.map(Option::get);
}
/**
@@ -271,9 +271,9 @@ public class HoodieTableFileSystemViewTest {
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
commitTimeline.saveAsComplete(instant1, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant2, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant3, Optional.empty());
commitTimeline.saveAsComplete(instant1, Option.empty());
commitTimeline.saveAsComplete(deltaInstant2, Option.empty());
commitTimeline.saveAsComplete(deltaInstant3, Option.empty());
refreshFsView();
List<FileSlice> fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
@@ -282,7 +282,7 @@ public class HoodieTableFileSystemViewTest {
List<Pair<String, FileSlice>> partitionFileSlicesPairs = new ArrayList<>();
partitionFileSlicesPairs.add(Pair.of(partitionPath, fileSlices.get(0)));
HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs,
Optional.empty(), Optional.empty());
Option.empty(), Option.empty());
HoodieInstant compactionInstant = null;
if (isCompactionInFlight) {
// Create a Data-file but this should be skipped by view
@@ -318,8 +318,8 @@ public class HoodieTableFileSystemViewTest {
new File(basePath + "/" + partitionPath + "/" + fileName4).createNewFile();
HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4);
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
commitTimeline.saveAsComplete(deltaInstant4, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant5, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant4, Option.empty());
commitTimeline.saveAsComplete(deltaInstant5, Option.empty());
refreshFsView();
List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath).collect(Collectors.toList());
@@ -425,7 +425,7 @@ public class HoodieTableFileSystemViewTest {
new File(basePath + "/" + partitionPath + "/" + inflightLogFileName).createNewFile();
// Mark instant as inflight
commitTimeline.saveToInflight(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION,
inflightDeltaInstantTime), Optional.empty());
inflightDeltaInstantTime), Option.empty());
refreshFsView();
List<FileSlice> allRawFileSlices = getAllRawFileSlices(partitionPath).collect(Collectors.toList());
@@ -501,7 +501,7 @@ public class HoodieTableFileSystemViewTest {
commitTimeline.deleteCompactionRequested(compactionInstant);
}
compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime);
commitTimeline.saveAsComplete(compactionInstant, Optional.empty());
commitTimeline.saveAsComplete(compactionInstant, Option.empty());
refreshFsView();
// populate the cache
roView.getAllDataFiles(partitionPath);
@@ -509,7 +509,7 @@ public class HoodieTableFileSystemViewTest {
fileSliceList = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
log.info("FILESLICE LIST=" + fileSliceList);
dataFiles = fileSliceList.stream().map(FileSlice::getDataFile)
.filter(Optional::isPresent).map(Optional::get).collect(Collectors.toList());
.filter(Option::isPresent).map(Option::get).collect(Collectors.toList());
assertEquals("Expect only one data-files in latest view as there is only one file-group", 1, dataFiles.size());
assertEquals("Data Filename must match", compactDataFileName, dataFiles.get(0).getFileName());
assertEquals("Only one latest file-slice in the partition", 1, fileSliceList.size());
@@ -581,7 +581,7 @@ public class HoodieTableFileSystemViewTest {
// Make this commit safe
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
commitTimeline.saveAsComplete(instant1, Optional.empty());
commitTimeline.saveAsComplete(instant1, Option.empty());
refreshFsView();
assertEquals("", fileName1,
roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
@@ -598,7 +598,7 @@ public class HoodieTableFileSystemViewTest {
// Make it safe
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2);
commitTimeline.saveAsComplete(instant2, Optional.empty());
commitTimeline.saveAsComplete(instant2, Option.empty());
refreshFsView();
assertEquals("", fileName2,
roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
@@ -1060,9 +1060,9 @@ public class HoodieTableFileSystemViewTest {
HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1);
HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2);
commitTimeline.saveAsComplete(instant1, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant2, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant3, Optional.empty());
commitTimeline.saveAsComplete(instant1, Option.empty());
commitTimeline.saveAsComplete(deltaInstant2, Option.empty());
commitTimeline.saveAsComplete(deltaInstant3, Option.empty());
// Now we list all partitions
FileStatus[] statuses = metaClient.getFs().listStatus(new Path[] {
@@ -1094,7 +1094,7 @@ public class HoodieTableFileSystemViewTest {
String compactionRequestedTime = "2";
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, TEST_WRITE_TOKEN, fileId);
HoodieCompactionPlan compactionPlan = CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs,
Optional.empty(), Optional.empty());
Option.empty(), Option.empty());
// Create a Data-file for some of the partitions but this should be skipped by view
new File(basePath + "/" + partitionPath1 + "/" + compactDataFileName).createNewFile();
@@ -1125,8 +1125,8 @@ public class HoodieTableFileSystemViewTest {
HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4);
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
commitTimeline.saveAsComplete(deltaInstant4, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant5, Optional.empty());
commitTimeline.saveAsComplete(deltaInstant4, Option.empty());
commitTimeline.saveAsComplete(deltaInstant5, Option.empty());
refreshFsView();
// Test Data Files

View File

@@ -47,6 +47,7 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.CompactionUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.collection.Pair;
import com.uber.hoodie.exception.HoodieException;
import java.io.File;
@@ -57,7 +58,6 @@ import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
@@ -188,7 +188,7 @@ public class IncrementalFSViewSyncTest {
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
metaClient.getActiveTimeline().saveAsComplete(
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs),
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
view.sync();
Assert.assertTrue(view.getLastInstant().isPresent());
@@ -455,7 +455,7 @@ public class IncrementalFSViewSyncTest {
Integer.toString(Integer.parseInt(instant) + 1));
}).collect(Collectors.toList());
HoodieCleanMetadata cleanMetadata = AvroUtils.convertCleanMetadata(cleanInstant, Optional.empty(), cleanStats);
HoodieCleanMetadata cleanMetadata = AvroUtils.convertCleanMetadata(cleanInstant, Option.empty(), cleanStats);
metaClient.getActiveTimeline().saveAsComplete(
new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant),
AvroUtils.serializeCleanMetadata(cleanMetadata));
@@ -480,7 +480,7 @@ public class IncrementalFSViewSyncTest {
rollbacks.add(instant);
HoodieRollbackMetadata rollbackMetadata = AvroUtils
.convertRollbackMetadata(rollbackInstant, Optional.empty(), rollbacks, rollbackStats);
.convertRollbackMetadata(rollbackInstant, Option.empty(), rollbacks, rollbackStats);
if (isRestore) {
HoodieRestoreMetadata metadata = new HoodieRestoreMetadata();
@@ -538,7 +538,7 @@ public class IncrementalFSViewSyncTest {
long initialExpTotalFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).sum();
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(slices, Optional.empty(), Optional.empty());
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(slices, Option.empty(), Option.empty());
HoodieInstant compactionInstant =
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime);
metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant,
@@ -782,7 +782,7 @@ public class IncrementalFSViewSyncTest {
metaClient.getActiveTimeline().saveAsComplete(
new HoodieInstant(true, deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION,
instant),
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
// Delete pending compaction if present
metaClient.getFs().delete(new Path(metaClient.getMetaPath(),
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant).getFileName()));

View File

@@ -41,7 +41,6 @@ import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@@ -129,7 +128,7 @@ public class CompactionTestUtils {
public static void createDeltaCommit(HoodieTableMetaClient metaClient, String instantTime) throws IOException {
metaClient.getActiveTimeline().saveAsComplete(
new HoodieInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime), Optional.empty());
new HoodieInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime), Option.empty());
}
public static void scheduleInflightCompaction(HoodieTableMetaClient metaClient, String instantTime,
@@ -149,9 +148,9 @@ public class CompactionTestUtils {
HoodieTestUtils.createDataFile(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId);
}
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
instantId, fileId, Optional.of(1));
instantId, fileId, Option.of(1));
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
instantId, fileId, Optional.of(2));
instantId, fileId, Option.of(2));
FileSlice slice = new FileSlice(DEFAULT_PARTITION_PATHS[0], instantId, fileId);
if (createDataFile) {
slice.setDataFile(new TestHoodieDataFile(metaClient.getBasePath() + "/" + DEFAULT_PARTITION_PATHS[0]
@@ -159,19 +158,19 @@ public class CompactionTestUtils {
}
String logFilePath1 = HoodieTestUtils
.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId,
Optional.of(1));
Option.of(1));
String logFilePath2 = HoodieTestUtils
.getLogFilePath(metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0], instantId, fileId,
Optional.of(2));
Option.of(2));
slice.addLogFile(new HoodieLogFile(new Path(logFilePath1)));
slice.addLogFile(new HoodieLogFile(new Path(logFilePath2)));
HoodieCompactionOperation op =
CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], slice, Optional.empty());
CompactionUtils.buildFromFileSlice(DEFAULT_PARTITION_PATHS[0], slice, Option.empty());
if (deltaCommitsAfterCompactionRequests) {
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
compactionInstantId, fileId, Optional.of(1));
compactionInstantId, fileId, Option.of(1));
HoodieTestUtils.createNewLogFile(metaClient.getFs(), metaClient.getBasePath(), DEFAULT_PARTITION_PATHS[0],
compactionInstantId, fileId, Optional.of(2));
compactionInstantId, fileId, Option.of(2));
}
return op;
} catch (IOException e) {

View File

@@ -36,7 +36,6 @@ import java.nio.file.Paths;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.UUID;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -119,7 +118,7 @@ public class SchemaTestUtil {
private static HoodieRecord convertToHoodieRecords(IndexedRecord iRecord, String key, String partitionPath) {
return new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord) iRecord)));
new HoodieAvroPayload(Option.of((GenericRecord) iRecord)));
}
public static List<IndexedRecord> updateHoodieTestRecords(List<String> oldRecordKeys, List<IndexedRecord> newRecords,
@@ -143,7 +142,7 @@ public class SchemaTestUtil {
return iRecords
.stream()
.map(r -> new HoodieRecord<>(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
new HoodieAvroPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList());
new HoodieAvroPayload(Option.of((GenericRecord) r)))).collect(Collectors.toList());
}
public static List<HoodieRecord> updateHoodieTestRecordsWithoutHoodieMetadata(List<HoodieRecord> oldRecords,
@@ -157,7 +156,7 @@ public class SchemaTestUtil {
GenericRecord rec = (GenericRecord) r.getData().getInsertValue(schema).get();
rec.put(fieldNameToUpdate, newValue);
return new HoodieRecord<>(r.getKey(),
new HoodieAvroPayload(Optional.of(rec)));
new HoodieAvroPayload(Option.of(rec)));
} catch (IOException io) {
throw new HoodieIOException("unable to get data from hoodie record", io);
}

View File

@@ -26,7 +26,6 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -45,7 +44,7 @@ public class SpillableMapTestUtils {
String partitionPath = ((GenericRecord) r).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
recordKeys.add(key);
HoodieRecord record = new HoodieRecord<>(new HoodieKey(key, partitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord) r)));
new HoodieAvroPayload(Option.of((GenericRecord) r)));
record.setCurrentLocation(new HoodieRecordLocation("DUMMY_COMMIT_TIME", "DUMMY_FILE_ID"));
records.put(key, record);
});

View File

@@ -39,7 +39,6 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@@ -76,14 +75,14 @@ public class TestCompactionUtils {
// Empty File-Slice with no data and log files
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "empty1");
HoodieCompactionOperation op = CompactionUtils.buildFromFileSlice(
DEFAULT_PARTITION_PATHS[0], emptyFileSlice, Optional.of(metricsCaptureFn));
DEFAULT_PARTITION_PATHS[0], emptyFileSlice, Option.of(metricsCaptureFn));
testFileSliceCompactionOpEquality(emptyFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
// File Slice with data-file but no log files
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1");
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
op = CompactionUtils.buildFromFileSlice(
DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Optional.of(metricsCaptureFn));
DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Option.of(metricsCaptureFn));
testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
//File Slice with no data-file but log files present
@@ -93,7 +92,7 @@ public class TestCompactionUtils {
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
op = CompactionUtils.buildFromFileSlice(
DEFAULT_PARTITION_PATHS[0], noDataFileSlice, Optional.of(metricsCaptureFn));
DEFAULT_PARTITION_PATHS[0], noDataFileSlice, Option.of(metricsCaptureFn));
testFileSliceCompactionOpEquality(noDataFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
//File Slice with data-file and log files present
@@ -104,7 +103,7 @@ public class TestCompactionUtils {
fileSlice.addLogFile(new HoodieLogFile(new Path(
FSUtils.makeLogFileName("noData1", ".log", "000", 2, TEST_WRITE_TOKEN))));
op = CompactionUtils.buildFromFileSlice(
DEFAULT_PARTITION_PATHS[0], fileSlice, Optional.of(metricsCaptureFn));
DEFAULT_PARTITION_PATHS[0], fileSlice, Option.of(metricsCaptureFn));
testFileSliceCompactionOpEquality(fileSlice, op, DEFAULT_PARTITION_PATHS[0]);
}
@@ -129,7 +128,7 @@ public class TestCompactionUtils {
List<FileSlice> fileSliceList = Arrays.asList(emptyFileSlice, noDataFileSlice, fileSlice, noLogFileSlice);
List<Pair<String, FileSlice>> input = fileSliceList.stream().map(f -> Pair.of(DEFAULT_PARTITION_PATHS[0], f))
.collect(Collectors.toList());
return Pair.of(input, CompactionUtils.buildFromFileSlices(input, Optional.empty(), Optional.of(metricsCaptureFn)));
return Pair.of(input, CompactionUtils.buildFromFileSlices(input, Option.empty(), Option.of(metricsCaptureFn)));
}
@Test

View File

@@ -30,6 +30,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.HoodieRecordSizeEstimator;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.common.util.SpillableMapTestUtils;
import com.uber.hoodie.common.util.SpillableMapUtils;
@@ -40,7 +41,6 @@ import java.util.ArrayList;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
@@ -171,7 +171,7 @@ public class TestDiskBasedMap {
List<IndexedRecord> indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
hoodieRecords = indexedRecords.stream()
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
new AvroBinaryTestPayload(Optional.of((GenericRecord) r)))).collect(Collectors.toList());
new AvroBinaryTestPayload(Option.of((GenericRecord) r)))).collect(Collectors.toList());
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),
new HoodieRecordSizeEstimator(schema));
assertTrue(payloadSize > 0);
@@ -182,7 +182,7 @@ public class TestDiskBasedMap {
indexedRecords = SchemaTestUtil.generateHoodieTestRecords(0, 1);
hoodieRecords = indexedRecords.stream()
.map(r -> new HoodieRecord(new HoodieKey(UUID.randomUUID().toString(), "0000/00/00"),
new AvroBinaryTestPayload(Optional
new AvroBinaryTestPayload(Option
.of(HoodieAvroUtils.rewriteRecord((GenericRecord) r, simpleSchemaWithMetadata)))))
.collect(Collectors.toList());
payloadSize = SpillableMapUtils.computePayloadSize(hoodieRecords.remove(0),

View File

@@ -30,6 +30,7 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.util.DefaultSizeEstimator;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.HoodieRecordSizeEstimator;
import com.uber.hoodie.common.util.Option;
import com.uber.hoodie.common.util.SchemaTestUtil;
import com.uber.hoodie.common.util.SpillableMapTestUtils;
import java.io.File;
@@ -39,7 +40,6 @@ import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
@@ -137,13 +137,13 @@ public class TestExternalSpillableMap {
String ikey = ((GenericRecord) inMemoryRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String iPartitionPath = ((GenericRecord) inMemoryRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
HoodieRecord inMemoryHoodieRecord = new HoodieRecord<>(new HoodieKey(ikey, iPartitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord) inMemoryRecord)));
new HoodieAvroPayload(Option.of((GenericRecord) inMemoryRecord)));
IndexedRecord onDiskRecord = iRecords.get(99);
String dkey = ((GenericRecord) onDiskRecord).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String dPartitionPath = ((GenericRecord) onDiskRecord).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
HoodieRecord onDiskHoodieRecord = new HoodieRecord<>(new HoodieKey(dkey, dPartitionPath),
new HoodieAvroPayload(Optional.of((GenericRecord) onDiskRecord)));
new HoodieAvroPayload(Option.of((GenericRecord) onDiskRecord)));
// assert size
assert records.size() == 100;
// get should return the same HoodieKey, same location and same value