diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java index 3129974ff..ebd097911 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/HoodiePrintHelper.java @@ -19,9 +19,9 @@ package com.uber.hoodie.cli; import com.jakewharton.fliptables.FlipTable; +import com.uber.hoodie.common.util.Option; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.function.Function; /** @@ -62,9 +62,9 @@ public class HoodiePrintHelper { } Table table = new Table(rowHeader, fieldNameToConverterMap, - Optional.ofNullable(sortByField.isEmpty() ? null : sortByField), - Optional.ofNullable(isDescending), - Optional.ofNullable(limit <= 0 ? null : limit)).addAllRows(rows).flip(); + Option.ofNullable(sortByField.isEmpty() ? null : sortByField), + Option.ofNullable(isDescending), + Option.ofNullable(limit <= 0 ? null : limit)).addAllRows(rows).flip(); return HoodiePrintHelper.print(table); } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java index 203f96d2e..c4cd43a97 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/Table.java @@ -18,13 +18,13 @@ package com.uber.hoodie.cli; +import com.uber.hoodie.common.util.Option; import java.util.ArrayList; import java.util.Arrays; import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; @@ -40,12 +40,12 @@ public class Table implements Iterable> { private final TableHeader rowHeader; // User-specified conversions before rendering private final Map> fieldNameToConverterMap; - // Optional attribute to track sorting field - private final Optional orderingFieldNameOptional; + // Option attribute to track sorting field + private final Option orderingFieldNameOptional; // Whether sorting has to be in descending order (by default : optional) - private final Optional isDescendingOptional; + private final Option isDescendingOptional; // Limit the number of entries rendered - private final Optional limitOptional; + private final Option limitOptional; // Raw list of rows private final List> rawRows; // Flag to determine if all the rows have been added @@ -55,9 +55,9 @@ public class Table implements Iterable> { public Table(TableHeader rowHeader, Map> fieldNameToConverterMap, - Optional orderingFieldNameOptional, - Optional isDescendingOptional, - Optional limitOptional) { + Option orderingFieldNameOptional, + Option isDescendingOptional, + Option limitOptional) { this.rowHeader = rowHeader; this.fieldNameToConverterMap = fieldNameToConverterMap; this.orderingFieldNameOptional = orderingFieldNameOptional; diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java index 7ec46378b..89cec5ff2 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/CompactionCommand.java @@ -36,6 +36,7 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import com.uber.hoodie.common.util.AvroUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; import java.io.ObjectInputStream; @@ -44,7 +45,6 @@ import java.util.Collections; 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.function.Function; @@ -475,7 +475,8 @@ public class CompactionCommand implements CommandMarker { private String getRenamesToBePrinted(List res, Integer limit, String sortByField, boolean descending, boolean headerOnly, String operation) { - Optional result = res.stream().map(r -> r.isExecuted() && r.isSuccess()).reduce(Boolean::logicalAnd); + Option result = Option.fromJavaOptional( + res.stream().map(r -> r.isExecuted() && r.isSuccess()).reduce(Boolean::logicalAnd)); if (result.isPresent()) { System.out.println("There were some file renames that needed to be done to " + operation); diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java index 0c4698e4a..2cd5d63fa 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/FileSystemViewCommand.java @@ -28,13 +28,13 @@ import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.NumericUtils; +import com.uber.hoodie.common.util.Option; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.function.BiPredicate; import java.util.function.Function; import java.util.stream.Collectors; @@ -266,7 +266,7 @@ public class FileSystemViewCommand implements CommandMarker { } HoodieTimeline filteredTimeline = new HoodieDefaultTimeline(instantsStream, - (Function> & Serializable) metaClient.getActiveTimeline()::getInstantDetails); + (Function> & Serializable) metaClient.getActiveTimeline()::getInstantDetails); return new HoodieTableFileSystemView(metaClient, filteredTimeline, statuses); } } diff --git a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java index 29ce9120e..a7ad9a3cf 100644 --- a/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java +++ b/hoodie-cli/src/main/java/com/uber/hoodie/cli/commands/HoodieLogFileCommand.java @@ -33,6 +33,7 @@ import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock; 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.Option; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieMemoryConfig; import com.uber.hoodie.hive.util.SchemaUtil; @@ -42,7 +43,6 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -200,7 +200,7 @@ public class HoodieLogFileCommand implements CommandMarker { Integer.valueOf(HoodieMemoryConfig.DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE), HoodieMemoryConfig.DEFAULT_SPILLABLE_MAP_BASE_PATH); for (HoodieRecord hoodieRecord : scanner) { - Optional record = hoodieRecord.getData().getInsertValue(readerSchema); + Option record = hoodieRecord.getData().getInsertValue(readerSchema); if (allRecords.size() >= limit) { break; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/AbstractHoodieClient.java b/hoodie-client/src/main/java/com/uber/hoodie/AbstractHoodieClient.java index c03bd218c..aae8e8030 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/AbstractHoodieClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/AbstractHoodieClient.java @@ -22,10 +22,10 @@ import com.uber.hoodie.client.embedded.EmbeddedTimelineService; import com.uber.hoodie.client.utils.ClientUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieWriteConfig; import java.io.IOException; import java.io.Serializable; -import java.util.Optional; import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -50,15 +50,15 @@ public abstract class AbstractHoodieClient implements Serializable { * of the cached file-system view. New completed actions will be synced automatically * in an incremental fashion. */ - private transient Optional timelineServer; + private transient Option timelineServer; private final boolean shouldStopTimelineServer; protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) { - this(jsc, clientConfig, Optional.empty()); + this(jsc, clientConfig, Option.empty()); } protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, - Optional timelineServer) { + Option timelineServer) { this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration()); this.jsc = jsc; this.basePath = clientConfig.getBasePath(); @@ -82,7 +82,7 @@ public abstract class AbstractHoodieClient implements Serializable { timelineServer.get().stop(); } - timelineServer = Optional.empty(); + timelineServer = Option.empty(); // Reset Storage Config to Client specified config if (resetViewStorageConfig) { config.resetViewStorageConfig(); @@ -94,7 +94,7 @@ public abstract class AbstractHoodieClient implements Serializable { if (!timelineServer.isPresent()) { // Run Embedded Timeline Server logger.info("Starting Timeline service !!"); - timelineServer = Optional.of(new EmbeddedTimelineService(jsc.hadoopConfiguration(), jsc.getConf(), + timelineServer = Option.of(new EmbeddedTimelineService(jsc.hadoopConfiguration(), jsc.getConf(), config.getClientSpecifiedViewStorageConfig())); try { timelineServer.get().startServer(); @@ -116,7 +116,7 @@ public abstract class AbstractHoodieClient implements Serializable { return config; } - public Optional getTimelineServer() { + public Option getTimelineServer() { return timelineServer; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java b/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java index 03bec5905..dc8da0d9c 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/CompactionAdminClient.java @@ -20,7 +20,6 @@ package com.uber.hoodie; import static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION; -import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.uber.hoodie.avro.model.HoodieCompactionOperation; import com.uber.hoodie.avro.model.HoodieCompactionPlan; @@ -39,6 +38,7 @@ import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; 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.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; @@ -70,7 +70,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { } public CompactionAdminClient(JavaSparkContext jsc, String basePath, - java.util.Optional timelineServer) { + Option timelineServer) { super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build(), timelineServer); } @@ -92,7 +92,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { .map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList()); return jsc.parallelize(ops, parallelism).map(op -> { try { - return validateCompactionOperation(metaClient, compactionInstant, op, Optional.of(fsView)); + return validateCompactionOperation(metaClient, compactionInstant, op, Option.of(fsView)); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } @@ -117,14 +117,14 @@ public class CompactionAdminClient extends AbstractHoodieClient { HoodieTableMetaClient metaClient = createMetaClient(false); List> renameActions = getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, parallelism, - Optional.absent(), skipValidation); + Option.empty(), skipValidation); List res = runRenamingOps(metaClient, renameActions, parallelism, dryRun); - java.util.Optional success = - res.stream().map(r -> (r.isExecuted() && r.isSuccess())).reduce(Boolean::logicalAnd); - Optional allSuccess = success.isPresent() ? Optional.of(success.get()) : Optional.absent(); + Option success = + Option.fromJavaOptional(res.stream().map(r -> (r.isExecuted() && r.isSuccess())).reduce(Boolean::logicalAnd)); + Option allSuccess = success.isPresent() ? Option.of(success.get()) : Option.empty(); // Only if all operations are successfully executed if (!dryRun && allSuccess.isPresent() && allSuccess.get()) { @@ -159,7 +159,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { HoodieTableMetaClient metaClient = createMetaClient(false); List> renameActions = getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fgId, - Optional.absent(), skipValidation); + Option.empty(), skipValidation); List res = runRenamingOps(metaClient, renameActions, 1, dryRun); @@ -211,7 +211,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { metaClient.getCommitsAndCompactionTimeline()); List> renameActions = failed.stream().flatMap(v -> getRenamingActionsToAlignWithCompactionOperation(metaClient, compactionInstant, - v.getOperation(), Optional.of(fsView)).stream()).collect(Collectors.toList()); + v.getOperation(), Option.of(fsView)).stream()).collect(Collectors.toList()); return runRenamingOps(metaClient, renameActions, parallelism, dryRun); } @@ -237,7 +237,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { */ protected static List> getRenamingActionsToAlignWithCompactionOperation( HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation op, - Optional fsViewOpt) { + Option fsViewOpt) { HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() : new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); HoodieInstant lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant().get(); @@ -286,19 +286,19 @@ public class CompactionAdminClient extends AbstractHoodieClient { * @param fsViewOpt File System View */ private ValidationOpResult validateCompactionOperation(HoodieTableMetaClient metaClient, - String compactionInstant, CompactionOperation operation, Optional fsViewOpt) + String compactionInstant, CompactionOperation operation, Option fsViewOpt) throws IOException { HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() : new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); - java.util.Optional lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant(); + Option lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant(); try { if (lastInstant.isPresent()) { - java.util.Optional fileSliceOptional = - fileSystemView.getLatestUnCompactedFileSlices(operation.getPartitionPath()) - .filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst(); + Option fileSliceOptional = + Option.fromJavaOptional(fileSystemView.getLatestUnCompactedFileSlices(operation.getPartitionPath()) + .filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst()); if (fileSliceOptional.isPresent()) { FileSlice fs = fileSliceOptional.get(); - java.util.Optional df = fs.getDataFile(); + Option df = fs.getDataFile(); if (operation.getDataFilePath().isPresent()) { String expPath = metaClient.getFs().getFileStatus(new Path(operation.getDataFilePath().get())).getPath() .toString(); @@ -341,9 +341,9 @@ public class CompactionAdminClient extends AbstractHoodieClient { + "be pointing to stale file-slices"); } } catch (CompactionValidationException | IllegalArgumentException e) { - return new ValidationOpResult(operation, false, Optional.of(e)); + return new ValidationOpResult(operation, false, Option.of(e)); } - return new ValidationOpResult(operation, true, Optional.absent()); + return new ValidationOpResult(operation, true, Option.empty()); } /** @@ -364,18 +364,18 @@ public class CompactionAdminClient extends AbstractHoodieClient { try { log.info("RENAME " + lfPair.getLeft().getPath() + " => " + lfPair.getRight().getPath()); renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight()); - return new RenameOpResult(lfPair, true, Optional.absent()); + return new RenameOpResult(lfPair, true, Option.empty()); } catch (IOException e) { log.error("Error renaming log file", e); log.error("\n\n\n***NOTE Compaction is in inconsistent state. Try running \"compaction repair " + lfPair.getLeft().getBaseCommitTime() + "\" to recover from failure ***\n\n\n"); - return new RenameOpResult(lfPair, false, Optional.of(e)); + return new RenameOpResult(lfPair, false, Option.of(e)); } }).collect(); } else { log.info("Dry-Run Mode activated for rename operations"); return renameActions.parallelStream() - .map(lfPair -> new RenameOpResult(lfPair, false, false, Optional.absent())) + .map(lfPair -> new RenameOpResult(lfPair, false, false, Option.empty())) .collect(Collectors.toList()); } } @@ -394,7 +394,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { */ protected List> getRenamingActionsForUnschedulingCompactionPlan( HoodieTableMetaClient metaClient, String compactionInstant, int parallelism, - Optional fsViewOpt, boolean skipValidation) throws IOException { + Option fsViewOpt, boolean skipValidation) throws IOException { HoodieTableFileSystemView fsView = fsViewOpt.isPresent() ? fsViewOpt.get() : new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); HoodieCompactionPlan plan = getCompactionPlan(metaClient, compactionInstant); @@ -406,7 +406,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { return jsc.parallelize(ops, parallelism).flatMap(op -> { try { return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, - op, Optional.of(fsView), skipValidation).iterator(); + op, Option.of(fsView), skipValidation).iterator(); } catch (IOException ioe) { throw new HoodieIOException(ioe.getMessage(), ioe); } catch (CompactionValidationException ve) { @@ -432,12 +432,12 @@ public class CompactionAdminClient extends AbstractHoodieClient { */ public List> getRenamingActionsForUnschedulingCompactionOperation( HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation operation, - Optional fsViewOpt, boolean skipValidation) throws IOException { + Option fsViewOpt, boolean skipValidation) throws IOException { List> result = new ArrayList<>(); HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() : new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline()); if (!skipValidation) { - validateCompactionOperation(metaClient, compactionInstant, operation, Optional.of(fileSystemView)); + validateCompactionOperation(metaClient, compactionInstant, operation, Option.of(fileSystemView)); } HoodieInstant lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant().get(); FileSlice merged = @@ -480,7 +480,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { */ public List> getRenamingActionsForUnschedulingCompactionForFileId( HoodieTableMetaClient metaClient, HoodieFileGroupId fgId, - Optional fsViewOpt, boolean skipValidation) throws IOException { + Option fsViewOpt, boolean skipValidation) throws IOException { Map> allPendingCompactions = CompactionUtils.getAllPendingCompactionOperations(metaClient); if (allPendingCompactions.containsKey(fgId)) { @@ -500,14 +500,14 @@ public class CompactionAdminClient extends AbstractHoodieClient { } public RenameOpResult(Pair op, boolean success, - Optional exception) { + Option exception) { super(new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(), op.getRight().getPath().toString()), success, exception); } public RenameOpResult( Pair op, boolean executed, boolean success, - Optional exception) { + Option exception) { super(new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(), op.getRight().getPath().toString()), executed, success, exception); } @@ -522,7 +522,7 @@ public class CompactionAdminClient extends AbstractHoodieClient { } public ValidationOpResult( - CompactionOperation operation, boolean success, Optional exception) { + CompactionOperation operation, boolean success, Option exception) { super(operation, success, exception); } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java index 86c2fa560..f660864aa 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieReadClient.java @@ -18,7 +18,6 @@ package com.uber.hoodie; -import com.google.common.base.Optional; import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.client.embedded.EmbeddedTimelineService; import com.uber.hoodie.common.model.HoodieDataFile; @@ -28,6 +27,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.util.CompactionUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; @@ -65,13 +65,13 @@ public class HoodieReadClient extends AbstractHoo private final transient HoodieIndex index; private final HoodieTimeline commitTimeline; private HoodieTable hoodieTable; - private transient Optional sqlContextOpt; + private transient Option sqlContextOpt; /** * @param basePath path to Hoodie dataset */ public HoodieReadClient(JavaSparkContext jsc, String basePath, - java.util.Optional timelineService) { + Option timelineService) { this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath) // by default we use HoodieBloomIndex .withIndexConfig( @@ -83,7 +83,7 @@ public class HoodieReadClient extends AbstractHoo * @param basePath path to Hoodie dataset */ public HoodieReadClient(JavaSparkContext jsc, String basePath) { - this(jsc, basePath, java.util.Optional.empty()); + this(jsc, basePath, Option.empty()); } /** @@ -93,21 +93,21 @@ public class HoodieReadClient extends AbstractHoo */ public HoodieReadClient(JavaSparkContext jsc, String basePath, SQLContext sqlContext) { this(jsc, basePath); - this.sqlContextOpt = Optional.of(sqlContext); + this.sqlContextOpt = Option.of(sqlContext); } /** * @param clientConfig instance of HoodieWriteConfig */ public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) { - this(jsc, clientConfig, java.util.Optional.empty()); + this(jsc, clientConfig, Option.empty()); } /** * @param clientConfig instance of HoodieWriteConfig */ public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, - java.util.Optional timelineService) { + Option timelineService) { super(jsc, clientConfig, timelineService); final String basePath = clientConfig.getBasePath(); // Create a Hoodie table which encapsulated the commits and files visible @@ -115,7 +115,7 @@ public class HoodieReadClient extends AbstractHoo this.hoodieTable = HoodieTable.getHoodieTable(metaClient, clientConfig, jsc); this.commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants(); this.index = HoodieIndex.createIndex(clientConfig, jsc); - this.sqlContextOpt = Optional.absent(); + this.sqlContextOpt = Option.empty(); } /** @@ -135,13 +135,13 @@ public class HoodieReadClient extends AbstractHoo } } - private Optional convertToDataFilePath(Optional> partitionPathFileIDPair) { + private Option convertToDataFilePath(Option> partitionPathFileIDPair) { if (partitionPathFileIDPair.isPresent()) { HoodieDataFile dataFile = hoodieTable.getROFileSystemView() .getLatestDataFile(partitionPathFileIDPair.get().getLeft(), partitionPathFileIDPair.get().getRight()).get(); - return Optional.of(dataFile.getPath()); + return Option.of(dataFile.getPath()); } else { - return Optional.absent(); + return Option.empty(); } } @@ -152,9 +152,9 @@ public class HoodieReadClient extends AbstractHoo */ public Dataset readROView(JavaRDD hoodieKeys, int parallelism) { assertSqlContext(); - JavaPairRDD>> lookupResultRDD = index + JavaPairRDD>> lookupResultRDD = index .fetchRecordLocation(hoodieKeys, jsc, hoodieTable); - JavaPairRDD> keyToFileRDD = lookupResultRDD + JavaPairRDD> keyToFileRDD = lookupResultRDD .mapToPair(r -> new Tuple2<>(r._1, convertToDataFilePath(r._2))); List paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent()) .map(keyFileTuple -> keyFileTuple._2().get()).collect(); @@ -176,12 +176,12 @@ public class HoodieReadClient extends AbstractHoo } /** - * Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[FullFilePath]] + * Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[FullFilePath]] * If the optional FullFilePath value is not present, then the key is not found. If the * FullFilePath value is present, it is the path component (without scheme) of the URI underlying * file */ - public JavaPairRDD> checkExists(JavaRDD hoodieKeys) { + public JavaPairRDD> checkExists(JavaRDD hoodieKeys) { return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java index 0d4b415af..10fc61249 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/HoodieWriteClient.java @@ -47,6 +47,7 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCommitException; @@ -73,7 +74,6 @@ import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.log4j.LogManager; @@ -84,7 +84,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.PairFunction; import org.apache.spark.storage.StorageLevel; -import scala.Option; import scala.Tuple2; /** @@ -125,11 +124,11 @@ public class HoodieWriteClient extends AbstractHo @VisibleForTesting HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, boolean rollbackInFlight, HoodieIndex index) { - this(jsc, clientConfig, rollbackInFlight, index, Optional.empty()); + this(jsc, clientConfig, rollbackInFlight, index, Option.empty()); } public HoodieWriteClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig, - boolean rollbackInFlight, HoodieIndex index, Optional timelineService) { + boolean rollbackInFlight, HoodieIndex index, Option timelineService) { super(jsc, clientConfig, timelineService); this.index = index; this.metrics = new HoodieMetrics(config, config.getTableName()); @@ -343,7 +342,7 @@ public class HoodieWriteClient extends AbstractHo Option bulkInsertPartitioner) { final JavaRDD> repartitionedRecords; final int parallelism = config.getBulkInsertShuffleParallelism(); - if (bulkInsertPartitioner.isDefined()) { + if (bulkInsertPartitioner.isPresent()) { repartitionedRecords = bulkInsertPartitioner.get() .repartitionRecords(dedupedRecords, parallelism); } else { @@ -372,7 +371,7 @@ public class HoodieWriteClient extends AbstractHo String actionType) { if (config.shouldAutoCommit()) { logger.info("Auto commit enabled: Committing " + commitTime); - boolean commitResult = commit(commitTime, resultRDD, Optional.empty(), actionType); + boolean commitResult = commit(commitTime, resultRDD, Option.empty(), actionType); if (!commitResult) { throw new HoodieCommitException("Failed to commit " + commitTime); } @@ -413,10 +412,10 @@ public class HoodieWriteClient extends AbstractHo }); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - Optional instant = + Option instant = activeTimeline.getCommitsTimeline().filterInflightsExcludingCompaction().lastInstant(); activeTimeline.saveToInflight(instant.get(), - Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); } catch (IOException io) { throw new HoodieCommitException( "Failed to commit " + commitTime + " unable to save inflight metadata ", io); @@ -479,7 +478,7 @@ public class HoodieWriteClient extends AbstractHo private JavaRDD> partition(JavaRDD> dedupedRecords, Partitioner partitioner) { return dedupedRecords.mapToPair(record -> new Tuple2<>( - new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record)) + new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record)) .partitionBy(partitioner).map(Tuple2::_2); } @@ -487,20 +486,20 @@ public class HoodieWriteClient extends AbstractHo * Commit changes performed at the given commitTime marker */ public boolean commit(String commitTime, JavaRDD writeStatuses) { - return commit(commitTime, writeStatuses, Optional.empty()); + return commit(commitTime, writeStatuses, Option.empty()); } /** * Commit changes performed at the given commitTime marker */ public boolean commit(String commitTime, JavaRDD writeStatuses, - Optional> extraMetadata) { + Option> extraMetadata) { HoodieTableMetaClient metaClient = createMetaClient(false); return commit(commitTime, writeStatuses, extraMetadata, metaClient.getCommitActionType()); } private boolean commit(String commitTime, JavaRDD writeStatuses, - Optional> extraMetadata, String actionType) { + Option> extraMetadata, String actionType) { logger.info("Commiting " + commitTime); // Create a Hoodie table which encapsulated the commits and files visible @@ -524,7 +523,7 @@ public class HoodieWriteClient extends AbstractHo try { activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, commitTime), - Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); // Save was a success & Do a inline compaction if enabled if (config.isInlineCompaction()) { metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true"); @@ -613,7 +612,7 @@ public class HoodieWriteClient extends AbstractHo if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) { throw new UnsupportedOperationException("Savepointing is not supported or MergeOnRead table types"); } - Optional cleanInstant = table.getCompletedCleanTimeline().lastInstant(); + Option cleanInstant = table.getCompletedCleanTimeline().lastInstant(); HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime); @@ -757,7 +756,7 @@ public class HoodieWriteClient extends AbstractHo restoreToInstant(savepointTime); // Make sure the rollback was successful - Optional lastInstant = activeTimeline.reload().getCommitsAndCompactionTimeline() + Option lastInstant = activeTimeline.reload().getCommitsAndCompactionTimeline() .filterCompletedAndCompactionInstants().lastInstant(); Preconditions.checkArgument(lastInstant.isPresent()); Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime), @@ -899,10 +898,10 @@ public class HoodieWriteClient extends AbstractHo List commitsToRollback, final String startRollbackTime) throws IOException { HoodieTable table = HoodieTable.getHoodieTable( createMetaClient(true), config, jsc); - Optional durationInMs = Optional.empty(); + Option durationInMs = Option.empty(); Long numFilesDeleted = rollbackStats.stream().mapToLong(stat -> stat.getSuccessDeleteFiles().size()).sum(); if (context != null) { - durationInMs = Optional.of(metrics.getDurationInMs(context.stop())); + durationInMs = Option.of(metrics.getDurationInMs(context.stop())); metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted); } HoodieRollbackMetadata rollbackMetadata = AvroUtils @@ -925,7 +924,7 @@ public class HoodieWriteClient extends AbstractHo List commitsToRollback, final String startRestoreTime, final String restoreToInstant) throws IOException { HoodieTable table = HoodieTable.getHoodieTable( createMetaClient(true), config, jsc); - Optional durationInMs = Optional.empty(); + Option durationInMs = Option.empty(); Long numFilesDeleted = 0L; for (Map.Entry> commitToStat : commitToStats.entrySet()) { List stats = commitToStat.getValue(); @@ -933,7 +932,7 @@ public class HoodieWriteClient extends AbstractHo .sum(); } if (context != null) { - durationInMs = Optional.of(metrics.getDurationInMs(context.stop())); + durationInMs = Option.of(metrics.getDurationInMs(context.stop())); metrics.updateRollbackMetrics(durationInMs.get(), numFilesDeleted); } HoodieRestoreMetadata restoreMetadata = AvroUtils @@ -1011,9 +1010,9 @@ public class HoodieWriteClient extends AbstractHo } // Emit metrics (duration, numFilesDeleted) if needed - Optional durationInMs = Optional.empty(); + Option durationInMs = Option.empty(); if (context != null) { - durationInMs = Optional.of(metrics.getDurationInMs(context.stop())); + durationInMs = Option.of(metrics.getDurationInMs(context.stop())); logger.info("cleanerElaspsedTime (Minutes): " + durationInMs.get() / (1000 * 60)); } @@ -1073,11 +1072,11 @@ public class HoodieWriteClient extends AbstractHo /** * Schedules a new compaction instant */ - public Optional scheduleCompaction(Optional> extraMetadata) throws IOException { + public Option scheduleCompaction(Option> extraMetadata) throws IOException { String instantTime = HoodieActiveTimeline.createNewCommitTime(); logger.info("Generate a new instant time " + instantTime); boolean notEmpty = scheduleCompactionAtInstant(instantTime, extraMetadata); - return notEmpty ? Optional.of(instantTime) : Optional.empty(); + return notEmpty ? Option.of(instantTime) : Option.empty(); } /** @@ -1086,7 +1085,7 @@ public class HoodieWriteClient extends AbstractHo * @param instantTime Compaction Instant Time * @param extraMetadata Extra Metadata to be stored */ - public boolean scheduleCompactionAtInstant(String instantTime, Optional> extraMetadata) + public boolean scheduleCompactionAtInstant(String instantTime, Option> extraMetadata) throws IOException { HoodieTableMetaClient metaClient = createMetaClient(true); // if there are inflight writes, their instantTime must not be less than that of compaction instant time @@ -1130,14 +1129,14 @@ public class HoodieWriteClient extends AbstractHo * Commit a compaction operation. Allow passing additional meta-data to be stored in commit instant file. */ public void commitCompaction(String compactionInstantTime, JavaRDD writeStatuses, - Optional> extraMetadata) throws IOException { + Option> extraMetadata) throws IOException { HoodieTableMetaClient metaClient = createMetaClient(true); HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan( timeline.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get()); // Merge extra meta-data passed by user with the one already in inflight compaction - Optional> mergedMetaData = extraMetadata.map(m -> { + Option> mergedMetaData = extraMetadata.map(m -> { Map merged = new HashMap<>(); Map extraMetaDataFromInstantFile = compactionPlan.getExtraMetadata(); if (extraMetaDataFromInstantFile != null) { @@ -1145,8 +1144,8 @@ public class HoodieWriteClient extends AbstractHo } // Overwrite/Merge with the user-passed meta-data merged.putAll(m); - return Optional.of(merged); - }).orElseGet(() -> Optional.ofNullable(compactionPlan.getExtraMetadata())); + return Option.of(merged); + }).orElseGet(() -> Option.ofNullable(compactionPlan.getExtraMetadata())); commitCompaction(writeStatuses, table, compactionInstantTime, true, mergedMetaData); } @@ -1255,7 +1254,7 @@ public class HoodieWriteClient extends AbstractHo statuses.persist(config.getWriteStatusStorageLevel()); // pass extra-metada so that it gets stored in commit file automatically commitCompaction(statuses, table, compactionInstant.getTimestamp(), autoCommit, - Optional.ofNullable(compactionPlan.getExtraMetadata())); + Option.ofNullable(compactionPlan.getExtraMetadata())); return statuses; } @@ -1269,7 +1268,7 @@ public class HoodieWriteClient extends AbstractHo * @param extraMetadata Extra Metadata to store */ protected void commitCompaction(JavaRDD compactedStatuses, HoodieTable table, - String compactionCommitTime, boolean autoCommit, Optional> extraMetadata) { + String compactionCommitTime, boolean autoCommit, Option> extraMetadata) { if (autoCommit) { HoodieCommitMetadata metadata = doCompactionCommit(table, compactedStatuses, compactionCommitTime, extraMetadata); @@ -1295,7 +1294,7 @@ public class HoodieWriteClient extends AbstractHo final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); table.finalizeWrite(jsc, instantTime, stats); if (finalizeCtx != null) { - Optional durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop())); + Option durationInMs = Option.of(metrics.getDurationInMs(finalizeCtx.stop())); durationInMs.ifPresent(duration -> { logger.info("Finalize write elapsed time (milliseconds): " + duration); metrics.updateFinalizeWriteMetrics(duration, stats.size()); @@ -1321,7 +1320,7 @@ public class HoodieWriteClient extends AbstractHo } private HoodieCommitMetadata doCompactionCommit(HoodieTable table, JavaRDD writeStatuses, - String compactionCommitTime, Optional> extraMetadata) { + String compactionCommitTime, Option> extraMetadata) { HoodieTableMetaClient metaClient = table.getMetaClient(); List updateStatusMap = writeStatuses.map(WriteStatus::getStat) .collect(); @@ -1347,7 +1346,7 @@ public class HoodieWriteClient extends AbstractHo try { activeTimeline.transitionCompactionInflightToComplete( new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime), - Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); } catch (IOException e) { throw new HoodieCompactionException( "Failed to commit " + metaClient.getBasePath() + " at time " + compactionCommitTime, e); @@ -1358,8 +1357,8 @@ public class HoodieWriteClient extends AbstractHo /** * Performs a compaction operation on a dataset, serially before or after an insert/upsert action. */ - private Optional forceCompact(Optional> extraMetadata) throws IOException { - Optional compactionInstantTimeOpt = scheduleCompaction(extraMetadata); + private Option forceCompact(Option> extraMetadata) throws IOException { + Option compactionInstantTimeOpt = scheduleCompaction(extraMetadata); compactionInstantTimeOpt.ifPresent(compactionInstantTime -> { try { // inline compaction should auto commit as the user is never given control @@ -1396,13 +1395,13 @@ public class HoodieWriteClient extends AbstractHo rollingStatMetadata.addRollingStat(partitionPath, hoodieRollingStat); } // The last rolling stat should be present in the completed timeline - Optional lastInstant = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() + Option lastInstant = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() .lastInstant(); if (lastInstant.isPresent()) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata .fromBytes(table.getActiveTimeline().getInstantDetails(lastInstant .get()).get(), HoodieCommitMetadata.class); - Optional lastRollingStat = Optional.ofNullable(commitMetadata.getExtraMetadata() + Option lastRollingStat = Option.ofNullable(commitMetadata.getExtraMetadata() .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY)); if (lastRollingStat.isPresent()) { rollingStatMetadata = rollingStatMetadata diff --git a/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java b/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java index a86d3641c..55a476c78 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/WriteStatus.java @@ -21,12 +21,12 @@ package com.uber.hoodie; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieWriteStat; +import com.uber.hoodie.common.util.Option; import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Random; /** @@ -73,7 +73,7 @@ public class WriteStatus implements Serializable { * @param optionalRecordMetadata optional metadata related to data contained in {@link * HoodieRecord} before deflation. */ - public void markSuccess(HoodieRecord record, Optional> optionalRecordMetadata) { + public void markSuccess(HoodieRecord record, Option> optionalRecordMetadata) { if (trackSuccessRecords) { writtenRecords.add(record); } @@ -90,7 +90,7 @@ public class WriteStatus implements Serializable { * @param optionalRecordMetadata optional metadata related to data contained in {@link * HoodieRecord} before deflation. */ - public void markFailure(HoodieRecord record, Throwable t, Optional> optionalRecordMetadata) { + public void markFailure(HoodieRecord record, Throwable t, Option> optionalRecordMetadata) { if (failedRecords.isEmpty() || (random.nextDouble() <= failureFraction)) { // Guaranteed to have at-least one error failedRecords.add(record); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java b/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java index f885ce286..ac4ddd7d8 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/CopyOnWriteLazyInsertIterable.java @@ -21,6 +21,7 @@ package com.uber.hoodie.func; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.config.HoodieWriteConfig; @@ -31,7 +32,6 @@ import com.uber.hoodie.table.HoodieTable; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.Optional; import java.util.function.Function; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -62,16 +62,16 @@ public class CopyOnWriteLazyInsertIterable extend // Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread. static class HoodieInsertValueGenResult { public T record; - public Optional insertValue; + public Option insertValue; // It caches the exception seen while fetching insert value. - public Optional exception = Optional.empty(); + public Option exception = Option.empty(); public HoodieInsertValueGenResult(T record, Schema schema) { this.record = record; try { this.insertValue = record.getData().getInsertValue(schema); } catch (Exception e) { - this.exception = Optional.of(e); + this.exception = Option.of(e); } } } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/OperationResult.java b/hoodie-client/src/main/java/com/uber/hoodie/func/OperationResult.java index 30a9afa63..dbb893f00 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/OperationResult.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/OperationResult.java @@ -18,7 +18,7 @@ package com.uber.hoodie.func; -import com.google.common.base.Optional; +import com.uber.hoodie.common.util.Option; import java.io.Serializable; /** @@ -30,19 +30,19 @@ public class OperationResult implements Serializable { private T operation; private boolean executed; private boolean success; - private Optional exception; + private Option exception; public OperationResult() { } - public OperationResult(T operation, boolean success, Optional exception) { + public OperationResult(T operation, boolean success, Option exception) { this.operation = operation; this.success = success; this.exception = exception; this.executed = true; } - public OperationResult(T operation, boolean executed, boolean success, Optional exception) { + public OperationResult(T operation, boolean executed, boolean success, Option exception) { this.operation = operation; this.success = success; this.exception = exception; @@ -61,7 +61,7 @@ public class OperationResult implements Serializable { return executed; } - public Optional getException() { + public Option getException() { return exception; } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java b/hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java index d35e732b2..47774bff5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/func/SparkBoundedInMemoryExecutor.java @@ -18,13 +18,13 @@ package com.uber.hoodie.func; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer; import com.uber.hoodie.common.util.queue.IteratorBasedQueueProducer; import com.uber.hoodie.config.HoodieWriteConfig; import java.util.Iterator; -import java.util.Optional; import java.util.function.Function; import org.apache.spark.TaskContext; import org.apache.spark.TaskContext$; @@ -46,7 +46,7 @@ public class SparkBoundedInMemoryExecutor extends BoundedInMemoryExecut BoundedInMemoryQueueConsumer consumer, Function bufferedIteratorTransform) { super(hoodieConfig.getWriteBufferLimitBytes(), producer, - Optional.of(consumer), bufferedIteratorTransform); + Option.of(consumer), bufferedIteratorTransform); this.sparkThreadTaskContext = TaskContext.get(); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java index e15d07bdb..881de3657 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/HoodieIndex.java @@ -18,11 +18,11 @@ package com.uber.hoodie.index; -import com.google.common.base.Optional; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIndexException; @@ -64,10 +64,10 @@ public abstract class HoodieIndex implements Seri } /** - * Checks if the given [Keys] exists in the hoodie table and returns [Key, Optional[partitionPath, fileID]] + * Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[partitionPath, fileID]] * If the optional is empty, then the key is not found. */ - public abstract JavaPairRDD>> fetchRecordLocation( + public abstract JavaPairRDD>> fetchRecordLocation( JavaRDD hoodieKeys, final JavaSparkContext jsc, HoodieTable hoodieTable); /** diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java index 61da5ab89..4fef0add0 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/InMemoryHashIndex.java @@ -18,12 +18,12 @@ package com.uber.hoodie.index; -import com.google.common.base.Optional; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieTable; @@ -56,7 +56,7 @@ public class InMemoryHashIndex extends HoodieInde } @Override - public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, + public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, JavaSparkContext jsc, HoodieTable hoodieTable) { throw new UnsupportedOperationException("InMemory index does not implement check exist yet"); } @@ -76,7 +76,7 @@ public class InMemoryHashIndex extends HoodieInde for (HoodieRecord record : writeStatus.getWrittenRecords()) { if (!writeStatus.isErrored(record.getKey())) { HoodieKey key = record.getKey(); - java.util.Optional newLocation = record.getNewLocation(); + Option newLocation = record.getNewLocation(); if (newLocation.isPresent()) { recordLocationMap.put(key, newLocation.get()); } else { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java index 195738ad8..7e9d76dc5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieBloomIndex.java @@ -23,13 +23,13 @@ import static java.util.stream.Collectors.mapping; import static java.util.stream.Collectors.toList; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.MetadataNotFoundException; @@ -107,7 +107,7 @@ public class HoodieBloomIndex extends HoodieIndex } /** - * Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Optional.Empty if the key is + * Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option.Empty if the key is * not found. * * @param hoodieKeys keys to lookup @@ -115,7 +115,7 @@ public class HoodieBloomIndex extends HoodieIndex * @param hoodieTable hoodie table object */ @Override - public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, + public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, JavaSparkContext jsc, HoodieTable hoodieTable) { JavaPairRDD partitionRecordKeyPairRDD = hoodieKeys .mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey())); @@ -126,11 +126,11 @@ public class HoodieBloomIndex extends HoodieIndex JavaPairRDD keyHoodieKeyPairRDD = hoodieKeys.mapToPair(key -> new Tuple2<>(key, null)); return keyHoodieKeyPairRDD.leftOuterJoin(recordKeyLocationRDD).mapToPair(keyLoc -> { - Optional> partitionPathFileidPair; + Option> partitionPathFileidPair; if (keyLoc._2._2.isPresent()) { - partitionPathFileidPair = Optional.of(Pair.of(keyLoc._1().getPartitionPath(), keyLoc._2._2.get().getFileId())); + partitionPathFileidPair = Option.of(Pair.of(keyLoc._1().getPartitionPath(), keyLoc._2._2.get().getFileId())); } else { - partitionPathFileidPair = Optional.absent(); + partitionPathFileidPair = Option.empty(); } return new Tuple2<>(keyLoc._1, partitionPathFileidPair); }); @@ -233,7 +233,7 @@ public class HoodieBloomIndex extends HoodieIndex List> partitionPathFileIDList = jsc .parallelize(partitions, Math.max(partitions.size(), 1)) .flatMap(partitionPath -> { - java.util.Optional latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline() + Option latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline() .filterCompletedInstants().lastInstant(); List> filteredFiles = new ArrayList<>(); if (latestCommitTime.isPresent()) { @@ -361,7 +361,7 @@ public class HoodieBloomIndex extends HoodieIndex } HoodieRecord getTaggedRecord(HoodieRecord inputRecord, - org.apache.spark.api.java.Optional location) { + Option location) { HoodieRecord record = inputRecord; if (location.isPresent()) { // When you have a record in multiple files in the same partition, then rowKeyRecordPairRDD @@ -384,7 +384,8 @@ public class HoodieBloomIndex extends HoodieIndex .mapToPair(record -> new Tuple2<>(record.getKey(), record)); // Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), // so we do left outer join. - return keyRecordPairRDD.leftOuterJoin(keyFilenamePairRDD).values().map(v1 -> getTaggedRecord(v1._1, v1._2)); + return keyRecordPairRDD.leftOuterJoin(keyFilenamePairRDD).values().map( + v1 -> getTaggedRecord(v1._1, Option.ofNullable(v1._2.orNull()))); } @Override diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java index 4ec51d90a..188502a1b 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/bloom/HoodieGlobalBloomIndex.java @@ -25,6 +25,7 @@ import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.table.HoodieTable; @@ -114,7 +115,7 @@ public class HoodieGlobalBloomIndex extends Hoodi // Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null), // so we do left outer join. return rowKeyRecordPairRDD.leftOuterJoin(keyFilenamePairRDD.mapToPair(p -> new Tuple2<>(p._1.getRecordKey(), p._2))) - .values().map(value -> getTaggedRecord(value._1, value._2)); + .values().map(value -> getTaggedRecord(value._1, Option.ofNullable(value._2.orNull()))); } @Override diff --git a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java index 276472142..5012c49d4 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/index/hbase/HBaseIndex.java @@ -19,7 +19,6 @@ package com.uber.hoodie.index.hbase; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Optional; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; @@ -28,6 +27,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieIndexConfig; @@ -125,7 +125,7 @@ public class HBaseIndex extends HoodieIndex { } @Override - public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, + public JavaPairRDD>> fetchRecordLocation(JavaRDD hoodieKeys, JavaSparkContext jsc, HoodieTable hoodieTable) { throw new UnsupportedOperationException("HBase index does not implement check exist"); } @@ -301,7 +301,7 @@ public class HBaseIndex extends HoodieIndex { try { for (HoodieRecord rec : writeStatus.getWrittenRecords()) { if (!writeStatus.isErrored(rec.getKey())) { - java.util.Optional loc = rec.getNewLocation(); + Option loc = rec.getNewLocation(); if (loc.isPresent()) { if (rec.getCurrentLocation() != null) { // This is an update, no need to update index diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java index 64452d785..c9906cc62 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieAppendHandle.java @@ -47,7 +47,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -148,13 +147,13 @@ public class HoodieAppendHandle extends HoodieWri } } - private Optional getIndexedRecord(HoodieRecord hoodieRecord) { - Optional recordMetadata = hoodieRecord.getData().getMetadata(); + private Option getIndexedRecord(HoodieRecord hoodieRecord) { + Option recordMetadata = hoodieRecord.getData().getMetadata(); try { - Optional avroRecord = hoodieRecord.getData().getInsertValue(originalSchema); + Option avroRecord = hoodieRecord.getData().getInsertValue(originalSchema); if (avroRecord.isPresent()) { // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema - avroRecord = Optional.of(rewriteRecord((GenericRecord) avroRecord.get())); + avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get())); String seqId = HoodieRecord.generateSequenceId(instantTime, TaskContext.getPartitionId(), recordIndex.getAndIncrement()); HoodieAvroUtils @@ -183,7 +182,7 @@ public class HoodieAppendHandle extends HoodieWri logger.error("Error writing record " + hoodieRecord, e); writeStatus.markFailure(hoodieRecord, e, recordMetadata); } - return Optional.empty(); + return Option.empty(); } // TODO (NA) - Perform a writerSchema check of current input record with the last writerSchema on log file @@ -225,8 +224,8 @@ public class HoodieAppendHandle extends HoodieWri } @Override - public void write(HoodieRecord record, Optional insertValue) { - Optional recordMetadata = record.getData().getMetadata(); + public void write(HoodieRecord record, Option insertValue) { + Option recordMetadata = record.getData().getMetadata(); try { init(record); flushToDiskIfRequired(record); @@ -278,7 +277,7 @@ public class HoodieAppendHandle extends HoodieWri private Writer createLogWriter(Option fileSlice, String baseCommitTime) throws IOException, InterruptedException { - Optional latestLogFile = fileSlice.get().getLatestLogFile(); + Option latestLogFile = fileSlice.get().getLatestLogFile(); return HoodieLogFormat.newWriterBuilder() .onParentPath(FSUtils.getPartitionPath(hoodieTable.getMetaClient().getBasePath(), partitionPath)) @@ -294,7 +293,7 @@ public class HoodieAppendHandle extends HoodieWri private void writeToBuffer(HoodieRecord record) { // update the new location of the record, so we know where to find it next record.setNewLocation(new HoodieRecordLocation(instantTime, fileId)); - Optional indexedRecord = getIndexedRecord(record); + Option indexedRecord = getIndexedRecord(record); if (indexedRecord.isPresent()) { recordList.add(indexedRecord.get()); } else { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java index afe228d09..74a88b361 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCleanHelper.java @@ -29,6 +29,7 @@ import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.SyncableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.table.HoodieTable; @@ -37,7 +38,6 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -99,7 +99,7 @@ public class HoodieCleanHelper> { while (fileSliceIterator.hasNext() && keepVersions > 0) { // Skip this most recent version FileSlice nextSlice = fileSliceIterator.next(); - Optional dataFile = nextSlice.getDataFile(); + Option dataFile = nextSlice.getDataFile(); if (dataFile.isPresent() && savepointedFiles.contains(dataFile.get().getFileName())) { // do not clean up a savepoint data file continue; @@ -165,7 +165,7 @@ public class HoodieCleanHelper> { // Ensure there are more than 1 version of the file (we only clean old files from updates) // i.e always spare the last commit. for (FileSlice aSlice : fileSliceList) { - Optional aFile = aSlice.getDataFile(); + Option aFile = aSlice.getDataFile(); String fileCommitTime = aSlice.getBaseInstantTime(); if (aFile.isPresent() && savepointedFiles.contains(aFile.get().getFileName())) { // do not clean up a savepoint data file @@ -240,8 +240,8 @@ public class HoodieCleanHelper> { /** * Returns earliest commit to retain based on cleaning policy. */ - public Optional getEarliestCommitToRetain() { - Optional earliestCommitToRetain = Optional.empty(); + public Option getEarliestCommitToRetain() { + Option earliestCommitToRetain = Option.empty(); int commitsRetained = config.getCleanerCommitsRetained(); if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS && commitTimeline.countInstants() > commitsRetained) { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java index 9bb272946..cba520168 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCommitArchiveLog.java @@ -43,6 +43,7 @@ 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.AvroUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCommitException; import com.uber.hoodie.exception.HoodieException; @@ -53,7 +54,6 @@ import java.util.ArrayList; import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.avro.Schema; @@ -153,12 +153,12 @@ public class HoodieCommitArchiveLog { //TODO (na) : Add a way to return actions associated with a timeline and then merge/unify // with logic above to avoid Stream.concats HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); - Optional oldestPendingCompactionInstant = + Option oldestPendingCompactionInstant = table.getActiveTimeline().filterPendingCompactionTimeline().firstInstant(); // We cannot have any holes in the commit timeline. We cannot archive any commits which are // made after the first savepoint present. - Optional firstSavepoint = table.getCompletedSavepointTimeline().firstInstant(); + Option firstSavepoint = table.getCompletedSavepointTimeline().firstInstant(); if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) { // Actually do the commits instants = Stream.concat(instants, commitTimeline.getInstants() @@ -196,12 +196,12 @@ public class HoodieCommitArchiveLog { } // Remove older meta-data from auxiliary path too - Optional latestCommitted = - archivedInstants.stream() + Option latestCommitted = + Option.fromJavaOptional(archivedInstants.stream() .filter(i -> { return i.isCompleted() && (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION))); - }).max(Comparator.comparing(HoodieInstant::getTimestamp)); + }).max(Comparator.comparing(HoodieInstant::getTimestamp))); if (latestCommitted.isPresent()) { success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get()); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java index 38303130d..317a8faea 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieCreateHandle.java @@ -26,6 +26,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.model.HoodieWriteStat; import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieInsertException; import com.uber.hoodie.io.storage.HoodieStorageWriter; @@ -33,7 +34,6 @@ import com.uber.hoodie.io.storage.HoodieStorageWriterFactory; import com.uber.hoodie.table.HoodieTable; import java.io.IOException; import java.util.Iterator; -import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; @@ -93,8 +93,8 @@ public class HoodieCreateHandle extends HoodieWri /** * Perform the actual writing of the given record into the backing file. */ - public void write(HoodieRecord record, Optional avroRecord) { - Optional recordMetadata = record.getData().getMetadata(); + public void write(HoodieRecord record, Option avroRecord) { + Option recordMetadata = record.getData().getMetadata(); try { if (avroRecord.isPresent()) { // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java index 8ebf31ef2..22c573713 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieMergeHandle.java @@ -30,6 +30,7 @@ import com.uber.hoodie.common.util.DefaultSizeEstimator; import com.uber.hoodie.common.util.FSUtils; 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.collection.ExternalSpillableMap; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieIOException; @@ -41,7 +42,6 @@ import java.io.IOException; import java.util.HashSet; import java.util.Iterator; import java.util.Map; -import java.util.Optional; import java.util.Set; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -120,15 +120,15 @@ public class HoodieMergeHandle extends HoodieWrit /** * Perform the actual writing of the given record into the backing file. */ - public void write(HoodieRecord record, Optional insertValue) { + public void write(HoodieRecord record, Option insertValue) { // NO_OP } /** * Perform the actual writing of the given record into the backing file. */ - public void write(HoodieRecord record, Optional avroRecord, Optional exception) { - Optional recordMetadata = record.getData().getMetadata(); + public void write(HoodieRecord record, Option avroRecord, Option exception) { + Option recordMetadata = record.getData().getMetadata(); if (exception.isPresent() && exception.get() instanceof Throwable) { // Not throwing exception from here, since we don't want to fail the entire job for a single record writeStatus.markFailure(record, exception.get(), recordMetadata); @@ -222,15 +222,15 @@ public class HoodieMergeHandle extends HoodieWrit return partitionPath; } - private boolean writeUpdateRecord(HoodieRecord hoodieRecord, Optional indexedRecord) { + private boolean writeUpdateRecord(HoodieRecord hoodieRecord, Option indexedRecord) { if (indexedRecord.isPresent()) { updatedRecordsWritten++; } return writeRecord(hoodieRecord, indexedRecord); } - private boolean writeRecord(HoodieRecord hoodieRecord, Optional indexedRecord) { - Optional recordMetadata = hoodieRecord.getData().getMetadata(); + private boolean writeRecord(HoodieRecord hoodieRecord, Option indexedRecord) { + Option recordMetadata = hoodieRecord.getData().getMetadata(); try { if (indexedRecord.isPresent()) { // Convert GenericRecord to GenericRecord with hoodie commit metadata in schema @@ -266,7 +266,7 @@ public class HoodieMergeHandle extends HoodieWrit // writing the first record. So make a copy of the record to be merged HoodieRecord hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key)); try { - Optional combinedAvroRecord = hoodieRecord.getData() + Option combinedAvroRecord = hoodieRecord.getData() .combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchema : originalSchema); if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) { /* ONLY WHEN diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieWriteHandle.java b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieWriteHandle.java index aa3eec2cc..9ec09598d 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieWriteHandle.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/HoodieWriteHandle.java @@ -24,13 +24,13 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; import com.uber.hoodie.common.util.HoodieTimer; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.table.HoodieTable; 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; @@ -134,15 +134,15 @@ public abstract class HoodieWriteHandle extends H /** * Perform the actual writing of the given record into the backing file. */ - public void write(HoodieRecord record, Optional insertValue) { + public void write(HoodieRecord record, Option insertValue) { // NO_OP } /** * Perform the actual writing of the given record into the backing file. */ - public void write(HoodieRecord record, Optional avroRecord, Optional exception) { - Optional recordMetadata = record.getData().getMetadata(); + public void write(HoodieRecord record, Option avroRecord, Option exception) { + Option recordMetadata = record.getData().getMetadata(); if (exception.isPresent() && exception.get() instanceof Throwable) { // Not throwing exception from here, since we don't want to fail the entire job for a single record writeStatus.markFailure(record, exception.get(), recordMetadata); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java index 4ee570d8f..923f436d6 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/HoodieRealtimeTableCompactor.java @@ -49,7 +49,6 @@ import java.io.IOException; import java.util.Collection; import java.util.Iterator; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -208,7 +207,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor { // Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO // for spark Map operations and collecting them finally in Avro generated classes for storing // into meta files. - Optional dataFile = s.getDataFile(); + Option dataFile = s.getDataFile(); return new CompactionOperation(dataFile, partitionPath, logFiles, config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles)); }) diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java index eaf4cffeb..9db21022a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/CompactionStrategy.java @@ -24,11 +24,11 @@ import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieWriteConfig; import java.io.Serializable; import java.util.List; import java.util.Map; -import java.util.Optional; /** * Strategy for compaction. Pluggable implementation to define how compaction should be done. The @@ -56,7 +56,7 @@ public abstract class CompactionStrategy implements Serializable { * @param logFiles - List of log files to compact with the base file * @return Map[String, Object] - metrics captured */ - public Map captureMetrics(HoodieWriteConfig writeConfig, Optional dataFile, + public Map captureMetrics(HoodieWriteConfig writeConfig, Option dataFile, String partitionPath, List logFiles) { Map metrics = Maps.newHashMap(); Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize(); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java index b7b422a3f..e5a4061a0 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/io/compact/strategy/LogFileSizeBasedCompactionStrategy.java @@ -22,11 +22,11 @@ import com.uber.hoodie.avro.model.HoodieCompactionOperation; import com.uber.hoodie.avro.model.HoodieCompactionPlan; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieWriteConfig; import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; /** @@ -42,7 +42,7 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE"; @Override - public Map captureMetrics(HoodieWriteConfig config, Optional dataFile, + public Map captureMetrics(HoodieWriteConfig config, Option dataFile, String partitionPath, List logFiles) { Map metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles); diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java index e1c4b99df..da04a311a 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieCopyOnWriteTable.java @@ -34,6 +34,7 @@ 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.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; @@ -76,7 +77,6 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFlatMapFunction; -import scala.Option; import scala.Tuple2; /** @@ -776,7 +776,7 @@ public class HoodieCopyOnWriteTable extends Hoodi public int getPartition(Object key) { Tuple2> keyLocation = (Tuple2>) key; - if (keyLocation._2().isDefined()) { + if (keyLocation._2().isPresent()) { HoodieRecordLocation location = keyLocation._2().get(); return updateLocationToBucket.get(location.getFileId()); } else { diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java index 8542999fd..97e8c7551 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/HoodieMergeOnReadTable.java @@ -42,6 +42,7 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.exception.HoodieCompactionException; import com.uber.hoodie.exception.HoodieException; @@ -61,7 +62,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import org.apache.hadoop.fs.FileStatus; @@ -136,7 +136,7 @@ public class HoodieMergeOnReadTable extends @Override public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime) { logger.info("Checking if compaction needs to be run on " + config.getBasePath()); - Optional lastCompaction = getActiveTimeline().getCommitTimeline() + Option lastCompaction = getActiveTimeline().getCommitTimeline() .filterCompletedInstants().lastInstant(); String deltaCommitsSinceTs = "0"; if (lastCompaction.isPresent()) { @@ -186,11 +186,11 @@ public class HoodieMergeOnReadTable extends // NOTE {@link HoodieCompactionConfig#withCompactionLazyBlockReadEnabled} needs to be set to TRUE. This is // required to avoid OOM when merging multiple LogBlocks performed during nested rollbacks. // Atomically un-publish all non-inflight commits - Optional commitOrCompactionOption = this.getActiveTimeline() + Option commitOrCompactionOption = Option.fromJavaOptional(this.getActiveTimeline() .getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION)).getInstants() .filter(i -> commit.equals(i.getTimestamp())) - .findFirst(); + .findFirst()); HoodieInstant instantToRollback = commitOrCompactionOption.get(); // Atomically un-publish all non-inflight commits if (!instantToRollback.isInflight()) { @@ -336,12 +336,12 @@ public class HoodieMergeOnReadTable extends @Override protected HoodieRollingStatMetadata getRollingStats() { try { - Optional lastInstant = this.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants() + Option lastInstant = this.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants() .lastInstant(); if (lastInstant.isPresent()) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( this.getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); - Optional lastRollingStat = Optional.ofNullable(commitMetadata.getExtraMetadata() + Option lastRollingStat = Option.ofNullable(commitMetadata.getExtraMetadata() .get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY)); if (lastRollingStat.isPresent()) { return HoodieCommitMetadata @@ -383,13 +383,13 @@ public class HoodieMergeOnReadTable extends if (!index.canIndexLogFiles()) { // TODO : choose last N small files since there can be multiple small files written to a single partition // by different spark partitions in a single batch - Optional smallFileSlice = getRTFileSystemView() + Option smallFileSlice = Option.fromJavaOptional(getRTFileSystemView() .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false).filter( fileSlice -> fileSlice.getLogFiles().count() < 1 && fileSlice.getDataFile().get().getFileSize() < config .getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) -> left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize() - ? -1 : 1).findFirst(); + ? -1 : 1).findFirst()); if (smallFileSlice.isPresent()) { allSmallFileSlices.add(smallFileSlice.get()); } diff --git a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java index 735ca372c..0975048f5 100644 --- a/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java +++ b/hoodie-client/src/main/java/com/uber/hoodie/table/WorkloadProfile.java @@ -21,12 +21,12 @@ package com.uber.hoodie.table; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordLocation; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.Option; import java.io.Serializable; import java.util.HashMap; import java.util.Map; import java.util.Set; import org.apache.spark.api.java.JavaRDD; -import scala.Option; import scala.Tuple2; /** @@ -62,7 +62,7 @@ public class WorkloadProfile implements Serializa Map>, Long> partitionLocationCounts = taggedRecords .mapToPair(record -> new Tuple2<>( - new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())), + new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record)).countByKey(); for (Map.Entry>, Long> e : partitionLocationCounts @@ -75,7 +75,7 @@ public class WorkloadProfile implements Serializa partitionPathStatMap.put(partitionPath, new WorkloadStat()); } - if (locOption.isDefined()) { + if (locOption.isPresent()) { // update partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count); globalStat.addUpdates(locOption.get(), count); diff --git a/hoodie-client/src/test/java/HoodieClientExample.java b/hoodie-client/src/test/java/HoodieClientExample.java index e32e250ad..44787ed35 100644 --- a/hoodie-client/src/test/java/HoodieClientExample.java +++ b/hoodie-client/src/test/java/HoodieClientExample.java @@ -26,12 +26,12 @@ import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.index.HoodieIndex.IndexType; import java.util.List; -import java.util.Optional; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -119,9 +119,9 @@ public class HoodieClientExample { * Schedule a compaction and also perform compaction on a MOR dataset */ if (HoodieTableType.valueOf(tableType) == HoodieTableType.MERGE_ON_READ) { - Optional instant = client.scheduleCompaction(Optional.empty()); + Option instant = client.scheduleCompaction(Option.empty()); JavaRDD writeStatues = client.compact(instant.get()); - client.commitCompaction(instant.get(), writeStatues, Optional.empty()); + client.commitCompaction(instant.get(), writeStatues, Option.empty()); } } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java index 4fcc32ae6..d7da020e9 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestAsyncCompaction.java @@ -42,6 +42,7 @@ import com.uber.hoodie.common.table.view.FileSystemViewStorageType; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; 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.collection.Pair; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; @@ -54,7 +55,6 @@ import java.util.ArrayList; import java.util.Arrays; 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; @@ -485,7 +485,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase { private void scheduleCompaction(String compactionInstantTime, HoodieWriteClient client, HoodieWriteConfig cfg) throws IOException { - client.scheduleCompactionAtInstant(compactionInstantTime, Optional.empty()); + client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieInstant instant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().get(); assertEquals("Last compaction instant must be the one set", @@ -545,7 +545,7 @@ public class TestAsyncCompaction extends TestHoodieClientBase { client.commit(instantTime, statuses); } - Optional deltaCommit = metaClient.getActiveTimeline().reload().getDeltaCommitTimeline() + Option deltaCommit = metaClient.getActiveTimeline().reload().getDeltaCommitTimeline() .filterCompletedInstants().lastInstant(); if (skipCommit && !cfg.shouldAutoCommit()) { assertTrue("Delta commit should not be latest instant", diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java index ffb6daf23..ff78494a4 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCleaner.java @@ -50,6 +50,7 @@ import com.uber.hoodie.common.util.AvroUtils; import com.uber.hoodie.common.util.CompactionUtils; import com.uber.hoodie.common.util.ConsistencyGuardConfig; 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.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; @@ -62,7 +63,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.TreeSet; import java.util.function.Predicate; @@ -78,7 +78,6 @@ import org.apache.spark.scheduler.SparkListenerTaskEnd; import org.apache.spark.util.AccumulatorV2; import org.junit.Assert; import org.junit.Test; -import scala.Option; import scala.collection.Iterator; /** @@ -214,11 +213,11 @@ public class TestCleaner extends TestHoodieClientBase { HoodieTable table = HoodieTable.getHoodieTable(metadata, getConfig(), jsc); for (String partitionPath : dataGen.getPartitionPaths()) { TableFileSystemView fsView = table.getFileSystemView(); - Optional added = fsView.getAllFileGroups(partitionPath).findFirst() + Option added = Option.fromJavaOptional(fsView.getAllFileGroups(partitionPath).findFirst() .map(fg -> { fg.getLatestFileSlice().map(fs -> compactionFileIdToLatestFileSlice.put(fg.getFileGroupId(), fs)); return true; - }); + })); if (added.isPresent()) { // Select only one file-group for compaction break; @@ -229,7 +228,7 @@ public class TestCleaner extends TestHoodieClientBase { List> partitionFileSlicePairs = compactionFileIdToLatestFileSlice.entrySet().stream() .map(e -> Pair.of(e.getKey().getPartitionPath(), e.getValue())).collect(Collectors.toList()); HoodieCompactionPlan compactionPlan = - CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Optional.empty(), Optional.empty()); + CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Option.empty(), Option.empty()); List instantTimes = HoodieTestUtils.monotonicIncreasingCommitTimestamps(9, 1); String compactionTime = instantTimes.get(0); table.getActiveTimeline().saveToCompactionRequested( @@ -275,11 +274,11 @@ public class TestCleaner extends TestHoodieClientBase { for (HoodieFileGroup fileGroup : fileGroups) { if (compactionFileIdToLatestFileSlice.containsKey(fileGroup.getFileGroupId())) { // Ensure latest file-slice selected for compaction is retained - Optional dataFileForCompactionPresent = - fileGroup.getAllDataFiles().filter(df -> { + Option dataFileForCompactionPresent = + Option.fromJavaOptional(fileGroup.getAllDataFiles().filter(df -> { return compactionFileIdToLatestFileSlice.get(fileGroup.getFileGroupId()) .getBaseInstantTime().equals(df.getCommitTime()); - }).findAny(); + }).findAny()); Assert.assertTrue("Data File selected for compaction is retained", dataFileForCompactionPresent.isPresent()); } else { @@ -386,7 +385,7 @@ public class TestCleaner extends TestHoodieClientBase { HoodieTableMetaClient metadata = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath); HoodieTable table1 = HoodieTable.getHoodieTable(metadata, cfg, jsc); HoodieTimeline activeTimeline = table1.getCompletedCommitsTimeline(); - Optional earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); + Option earliestRetainedCommit = activeTimeline.nthFromLastInstant(maxCommits - 1); Set acceptableCommits = activeTimeline.getInstants().collect(Collectors.toSet()); if (earliestRetainedCommit.isPresent()) { acceptableCommits.removeAll( @@ -503,20 +502,20 @@ public class TestCleaner extends TestHoodieClientBase { // Make 3 files, one base file and 2 log files associated with base file String file1P0 = HoodieTestUtils.createNewDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "000"); String file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Optional.empty()); + .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Option.empty()); String file2P0L1 = HoodieTestUtils - .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Optional.of(2)); + .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0, Option.of(2)); // make 1 compaction commit HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "000"); // Make 4 files, one base file and 3 log files associated with base file HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0); file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Optional.empty()); + .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Option.empty()); file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Optional.of(2)); + .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Option.of(2)); file2P0L0 = HoodieTestUtils - .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Optional.of(3)); + .createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, "001", file1P0, Option.of(3)); // make 1 compaction commit HoodieTestUtils.createCompactionCommitFiles(fs, basePath, "001"); @@ -526,9 +525,9 @@ public class TestCleaner extends TestHoodieClientBase { getCleanStat(hoodieCleanStats, DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().size()); assertFalse(HoodieTestUtils.doesDataFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file1P0)); assertFalse( - HoodieTestUtils.doesLogFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file2P0L0, Optional.empty())); + HoodieTestUtils.doesLogFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file2P0L0, Option.empty())); assertFalse( - HoodieTestUtils.doesLogFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file2P0L0, Optional.of(2))); + HoodieTestUtils.doesLogFileExist(basePath, DEFAULT_FIRST_PARTITION_PATH, "000", file2P0L0, Option.of(2))); } /** @@ -810,9 +809,9 @@ public class TestCleaner extends TestHoodieClientBase { final String fileId = HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0], fileIds[i]); HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0], - fileId, Optional.empty()); + fileId, Option.empty()); HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[0], - fileId, Optional.of(2)); + fileId, Option.of(2)); fileIdToLatestInstantBeforeCompaction.put(fileId, instants[0]); for (int j = 1; j <= i; j++) { if (j == i && j <= maxNumFileIdsForCompaction) { @@ -830,15 +829,15 @@ public class TestCleaner extends TestHoodieClientBase { compactionInstantsToFileSlices.put(compactionInstants[j], slices); // Add log-files to simulate delta-commits after pending compaction HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, compactionInstants[j], - fileId, Optional.empty()); + fileId, Option.empty()); HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, compactionInstants[j], - fileId, Optional.of(2)); + fileId, Option.of(2)); } else { HoodieTestUtils.createDataFile(basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId); HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId, - Optional.empty()); + Option.empty()); HoodieTestUtils.createNewLogFile(fs, basePath, DEFAULT_FIRST_PARTITION_PATH, instants[j], fileId, - Optional.of(2)); + Option.of(2)); fileIdToLatestInstantBeforeCompaction.put(fileId, instants[j]); } } @@ -867,9 +866,10 @@ public class TestCleaner extends TestHoodieClientBase { expFileIdToPendingCompaction.entrySet().stream().forEach(entry -> { String fileId = entry.getKey(); String baseInstantForCompaction = fileIdToLatestInstantBeforeCompaction.get(fileId); - Optional fileSliceForCompaction = - hoodieTable.getRTFileSystemView().getLatestFileSlicesBeforeOrOn(DEFAULT_FIRST_PARTITION_PATH, - baseInstantForCompaction, true).filter(fs -> fs.getFileId().equals(fileId)).findFirst(); + Option fileSliceForCompaction = + Option.fromJavaOptional( + hoodieTable.getRTFileSystemView().getLatestFileSlicesBeforeOrOn(DEFAULT_FIRST_PARTITION_PATH, + baseInstantForCompaction, true).filter(fs -> fs.getFileId().equals(fileId)).findFirst()); Assert.assertTrue("Base Instant for Compaction must be preserved", fileSliceForCompaction.isPresent()); Assert.assertTrue("FileSlice has data-file", fileSliceForCompaction.get().getDataFile().isPresent()); Assert.assertEquals("FileSlice has log-files", 2, diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java b/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java index 4509db6b5..825d1d14e 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestCompactionAdminClient.java @@ -20,7 +20,6 @@ package com.uber.hoodie; import static com.uber.hoodie.common.model.HoodieTestUtils.getDefaultHadoopConf; -import com.google.common.base.Optional; import com.uber.hoodie.CompactionAdminClient.ValidationOpResult; import com.uber.hoodie.common.model.CompactionOperation; import com.uber.hoodie.common.model.HoodieLogFile; @@ -30,6 +29,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.CompactionTestUtils; import com.uber.hoodie.common.util.CompactionUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.exception.HoodieIOException; @@ -139,7 +139,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { // Now repair List> undoFiles = result.stream().flatMap(r -> client.getRenamingActionsToAlignWithCompactionOperation(metaClient, - compactionInstant, r.getOperation(), Optional.absent()).stream()) + compactionInstant, r.getOperation(), Option.empty()).stream()) .map(rn -> { try { client.renameLogFile(metaClient, rn.getKey(), rn.getValue()); @@ -238,7 +238,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { // Check suggested rename operations List> renameFiles = client.getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, 1, - Optional.absent(), false); + Option.empty(), false); metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); // Log files belonging to file-slices created because of compaction request must be renamed @@ -311,7 +311,7 @@ public class TestCompactionAdminClient extends TestHoodieClientBase { // Check suggested rename operations List> renameFiles = client.getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op, - Optional.absent(), false); + Option.empty(), false); metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true); // Log files belonging to file-slices created because of compaction request must be renamed diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java index 7f0f0d656..5d20d9e53 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientBase.java @@ -39,6 +39,7 @@ import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig; import com.uber.hoodie.common.table.view.FileSystemViewStorageType; import com.uber.hoodie.common.util.ConsistencyGuardConfig; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieStorageConfig; @@ -54,7 +55,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocalFileSystem; @@ -341,7 +341,7 @@ public class TestHoodieClientBase implements Serializable { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); - return writeBatch(client, newCommitTime, initCommitTime, Optional.empty(), initCommitTime, + return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expRecordsInThisCommit, 1); } @@ -370,7 +370,7 @@ public class TestHoodieClientBase implements Serializable { HoodieWriteClient client, String newCommitTime, String prevCommitTime, - Optional> commitTimesBetweenPrevAndNew, + Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function3, HoodieWriteClient, JavaRDD, String> writeFn, @@ -409,7 +409,7 @@ public class TestHoodieClientBase implements Serializable { HoodieWriteClient client, String newCommitTime, String prevCommitTime, - Optional> commitTimesBetweenPrevAndNew, + Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function2, String, Integer> recordGenFunction, diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java index ad5fae899..caf96289a 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieClientOnCopyOnWriteStorage.java @@ -41,6 +41,7 @@ import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.ConsistencyGuardConfig; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.ParquetUtils; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieCompactionConfig; @@ -58,7 +59,6 @@ import java.util.Collection; 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; @@ -68,7 +68,6 @@ import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; import org.junit.Assert; import org.junit.Test; -import scala.Option; @SuppressWarnings("unchecked") public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { @@ -283,7 +282,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { numRecords = 100; String commitTimeBetweenPrevAndNew = "002"; updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime, - Optional.of(Arrays.asList(commitTimeBetweenPrevAndNew)), + Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), initCommitTime, numRecords, writeFn, isPrepped, true, numRecords, 200, 2); } @@ -311,7 +310,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { recordsInFirstBatch.addAll(fewRecordsForDelete); return recordsInFirstBatch; }; - writeBatch(client, newCommitTime, initCommitTime, Optional.empty(), initCommitTime, + writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, //unused as genFn uses hard-coded number of inserts/updates/deletes -1, recordGenFunction, HoodieWriteClient::upsert, true, @@ -332,7 +331,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends TestHoodieClientBase { recordsInSecondBatch.addAll(fewRecordsForUpdate); return recordsInSecondBatch; }; - writeBatch(client, newCommitTime, prevCommitTime, Optional.empty(), initCommitTime, + writeBatch(client, newCommitTime, prevCommitTime, Option.empty(), initCommitTime, 100, recordGenFunction, HoodieWriteClient::upsert, true, 50, 150, 2); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java index 2101577ae..4bd85db42 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/TestHoodieReadClient.java @@ -21,17 +21,16 @@ package com.uber.hoodie; import static org.junit.Assert.assertTrue; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieWriteConfig; import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; import org.apache.spark.api.java.JavaRDD; import org.junit.Assert; import org.junit.Test; -import scala.Option; @SuppressWarnings("unchecked") /** @@ -192,7 +191,7 @@ public class TestHoodieReadClient extends TestHoodieClientBase { numRecords = 100; String commitTimeBetweenPrevAndNew = "002"; result = updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime, - Optional.of(Arrays.asList(commitTimeBetweenPrevAndNew)), + Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), initCommitTime, numRecords, updateFn, isPrepped, true, numRecords, 200, 2); recordRDD = diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java index 1f79bb8fe..5c7a028f0 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/HoodieTestDataGenerator.java @@ -31,6 +31,7 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.AvroUtils; 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.exception.HoodieIOException; import java.io.IOException; import java.io.Serializable; @@ -41,7 +42,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.UUID; @@ -126,7 +126,7 @@ public class HoodieTestDataGenerator { */ public static HoodieAvroPayload generateAvroPayload(HoodieKey key, String commitTime) throws IOException { GenericRecord rec = generateGenericRecord(key.getRecordKey(), "rider-" + commitTime, "driver-" + commitTime, 0.0); - return new HoodieAvroPayload(Optional.of(rec)); + return new HoodieAvroPayload(Option.of(rec)); } public static GenericRecord generateGenericRecord(String rowKey, String riderName, String driverName, @@ -290,7 +290,7 @@ public class HoodieTestDataGenerator { } public HoodieRecord generateDeleteRecord(HoodieKey key) throws IOException { - TestRawTripPayload payload = new TestRawTripPayload(Optional.empty(), key.getRecordKey(), key.getPartitionPath(), + TestRawTripPayload payload = new TestRawTripPayload(Option.empty(), key.getRecordKey(), key.getPartitionPath(), null, true); return new HoodieRecord(key, payload); } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java b/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java index 813361919..6fcb5a877 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/common/TestRawTripPayload.java @@ -23,6 +23,7 @@ import com.uber.hoodie.WriteStatus; import com.uber.hoodie.avro.MercifulJsonConverter; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.Option; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -31,7 +32,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Optional; import java.util.zip.Deflater; import java.util.zip.DeflaterOutputStream; import java.util.zip.InflaterInputStream; @@ -52,7 +52,7 @@ public class TestRawTripPayload implements HoodieRecordPayload jsonData, String rowKey, String partitionPath, String schemaStr, + public TestRawTripPayload(Option jsonData, String rowKey, String partitionPath, String schemaStr, Boolean isDeleted) throws IOException { if (jsonData.isPresent()) { this.jsonDataCompressed = compressData(jsonData.get()); @@ -64,7 +64,7 @@ public class TestRawTripPayload implements HoodieRecordPayload combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException { + public Option combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException { return this.getInsertValue(schema); } @Override - public Optional getInsertValue(Schema schema) throws IOException { + public Option getInsertValue(Schema schema) throws IOException { if (isDeleted) { - return Optional.empty(); + return Option.empty(); } else { MercifulJsonConverter jsonConverter = new MercifulJsonConverter(schema); - return Optional.of(jsonConverter.convert(getJsonData())); + return Option.of(jsonConverter.convert(getJsonData())); } } @Override - public Optional> getMetadata() { + public Option> getMetadata() { // Let's assume we want to count the number of input row change events // that are processed. Let the time-bucket for this row change event be 1506582000. Map metadataMap = new HashMap<>(); metadataMap.put("InputRecordCount_1506582000", "2"); - return Optional.of(metadataMap); + return Option.of(metadataMap); } public String getRowKey() { @@ -174,7 +174,7 @@ public class TestRawTripPayload implements HoodieRecordPayload> recordMetadata) { + public void markSuccess(HoodieRecord record, Option> recordMetadata) { super.markSuccess(record, recordMetadata); if (recordMetadata.isPresent()) { mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap); @@ -182,7 +182,7 @@ public class TestRawTripPayload implements HoodieRecordPayload> recordMetadata) { + public void markFailure(HoodieRecord record, Throwable t, Option> recordMetadata) { super.markFailure(record, t, recordMetadata); if (recordMetadata.isPresent()) { mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java index 7f76bdb19..1e10fc0d8 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryExecutor.java @@ -25,11 +25,11 @@ import static org.mockito.Mockito.when; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueConsumer; import com.uber.hoodie.config.HoodieWriteConfig; import com.uber.hoodie.func.CopyOnWriteLazyInsertIterable.HoodieInsertValueGenResult; import java.util.List; -import java.util.Optional; import org.apache.avro.generic.IndexedRecord; import org.junit.After; import org.junit.Assert; @@ -41,7 +41,7 @@ public class TestBoundedInMemoryExecutor { private final HoodieTestDataGenerator hoodieTestDataGenerator = new HoodieTestDataGenerator(); private final String commitTime = HoodieActiveTimeline.createNewCommitTime(); private SparkBoundedInMemoryExecutor>, Integer> executor = null; + Tuple2>, Integer> executor = null; @After public void afterTest() { diff --git a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java index 132a69318..6b760a8e7 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/func/TestBoundedInMemoryQueue.java @@ -26,6 +26,7 @@ import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; 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.common.util.queue.BoundedInMemoryQueue; import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer; @@ -38,7 +39,6 @@ import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -94,7 +94,7 @@ public class TestBoundedInMemoryQueue { int recordsRead = 0; while (queue.iterator().hasNext()) { final HoodieRecord originalRecord = originalRecordIterator.next(); - final Optional originalInsertValue = originalRecord.getData() + final Option originalInsertValue = originalRecord.getData() .getInsertValue(HoodieTestDataGenerator.avroSchema); final HoodieInsertValueGenResult payload = queue.iterator().next(); // Ensure that record ordering is guaranteed. @@ -263,7 +263,7 @@ public class TestBoundedInMemoryQueue { public void testException() throws Exception { final int numRecords = 256; final List hoodieRecords = hoodieTestDataGenerator.generateInserts(commitTime, numRecords); - final SizeEstimator>> sizeEstimator = + final SizeEstimator>> sizeEstimator = new DefaultSizeEstimator<>(); // queue memory limit HoodieInsertValueGenResult payload = getTransformFunction(HoodieTestDataGenerator.avroSchema) @@ -274,7 +274,7 @@ public class TestBoundedInMemoryQueue { // first let us throw exception from queueIterator reader and test that queueing thread // stops and throws // correct exception back. - BoundedInMemoryQueue>> queue1 = + BoundedInMemoryQueue>> queue1 = new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema)); // Produce @@ -305,7 +305,7 @@ public class TestBoundedInMemoryQueue { final Iterator mockHoodieRecordsIterator = mock(Iterator.class); when(mockHoodieRecordsIterator.hasNext()).thenReturn(true); when(mockHoodieRecordsIterator.next()).thenThrow(expectedException); - BoundedInMemoryQueue>> queue2 = + BoundedInMemoryQueue>> queue2 = new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.avroSchema)); // Produce diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHBaseQPSResourceAllocator.java b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHBaseQPSResourceAllocator.java index 4f83c25d3..9eca394aa 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/TestHBaseQPSResourceAllocator.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/TestHBaseQPSResourceAllocator.java @@ -21,6 +21,7 @@ package com.uber.hoodie.index; import com.uber.hoodie.common.HoodieClientTestUtils; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieTestUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieHBaseIndexConfig; import com.uber.hoodie.config.HoodieIndexConfig; @@ -30,7 +31,6 @@ import com.uber.hoodie.index.hbase.DefaultHBaseQPSResourceAllocator; import com.uber.hoodie.index.hbase.HBaseIndex; import com.uber.hoodie.index.hbase.HBaseIndexQPSResourceAllocator; import java.io.File; -import java.util.Optional; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.HBaseTestingUtility; import org.apache.spark.api.java.JavaSparkContext; @@ -84,7 +84,7 @@ public class TestHBaseQPSResourceAllocator { @Test public void testsDefaultQPSResourceAllocator() { - HoodieWriteConfig config = getConfig(Optional.empty()); + HoodieWriteConfig config = getConfig(Option.empty()); HBaseIndex index = new HBaseIndex(config); HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config); Assert.assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(), @@ -95,7 +95,7 @@ public class TestHBaseQPSResourceAllocator { @Test public void testsExplicitDefaultQPSResourceAllocator() { - HoodieWriteConfig config = getConfig(Optional.of(HoodieHBaseIndexConfig.DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS)); + HoodieWriteConfig config = getConfig(Option.of(HoodieHBaseIndexConfig.DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS)); HBaseIndex index = new HBaseIndex(config); HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config); Assert.assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(), @@ -106,7 +106,7 @@ public class TestHBaseQPSResourceAllocator { @Test public void testsInvalidQPSResourceAllocator() { - HoodieWriteConfig config = getConfig(Optional.of("InvalidResourceAllocatorClassName")); + HoodieWriteConfig config = getConfig(Option.of("InvalidResourceAllocatorClassName")); HBaseIndex index = new HBaseIndex(config); HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config); Assert.assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(), @@ -115,7 +115,7 @@ public class TestHBaseQPSResourceAllocator { hBaseIndexQPSResourceAllocator.acquireQPSResources(config.getHbaseIndexQPSFraction(), 100), 0.0f); } - private HoodieWriteConfig getConfig(Optional resourceAllocatorClass) { + private HoodieWriteConfig getConfig(Option resourceAllocatorClass) { HoodieHBaseIndexConfig hoodieHBaseIndexConfig = getConfigWithResourceAllocator(resourceAllocatorClass); return getConfigBuilder(hoodieHBaseIndexConfig).build(); } @@ -132,7 +132,7 @@ public class TestHBaseQPSResourceAllocator { .build()); } - private HoodieHBaseIndexConfig getConfigWithResourceAllocator(Optional resourceAllocatorClass) { + private HoodieHBaseIndexConfig getConfigWithResourceAllocator(Option resourceAllocatorClass) { HoodieHBaseIndexConfig.Builder builder = new HoodieHBaseIndexConfig.Builder() .hbaseZkPort(Integer.valueOf(hbaseConfig.get("hbase.zookeeper.property.clientPort"))) diff --git a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java index f2e078488..15512a804 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/index/bloom/TestHoodieBloomIndex.java @@ -25,7 +25,6 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import com.google.common.base.Optional; import com.google.common.collect.Lists; import com.uber.hoodie.common.BloomFilter; import com.uber.hoodie.common.HoodieClientTestUtils; @@ -36,6 +35,7 @@ import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; 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 com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieWriteConfig; @@ -419,11 +419,11 @@ public class TestHoodieBloomIndex { // Let's tag HoodieBloomIndex bloomIndex = new HoodieBloomIndex(config); - JavaPairRDD>> taggedRecordRDD = bloomIndex + JavaPairRDD>> taggedRecordRDD = bloomIndex .fetchRecordLocation(keysRDD, jsc, table); // Should not find any files - for (Tuple2>> record : taggedRecordRDD.collect()) { + for (Tuple2>> record : taggedRecordRDD.collect()) { assertTrue(!record._2.isPresent()); } @@ -441,7 +441,7 @@ public class TestHoodieBloomIndex { taggedRecordRDD = bloomIndex.fetchRecordLocation(keysRDD, jsc, table); // Check results - for (Tuple2>> record : taggedRecordRDD.collect()) { + for (Tuple2>> record : taggedRecordRDD.collect()) { if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) { assertTrue(record._2.isPresent()); assertEquals(FSUtils.getFileId(filename1), record._2.get().getRight()); diff --git a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java index 0962bd0ab..064eabe24 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/io/strategy/TestHoodieCompactionStrategy.java @@ -27,6 +27,7 @@ import com.google.common.collect.Maps; import com.uber.hoodie.avro.model.HoodieCompactionOperation; import com.uber.hoodie.common.model.HoodieDataFile; import com.uber.hoodie.common.model.HoodieLogFile; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieWriteConfig; @@ -41,7 +42,6 @@ import java.util.ArrayList; import java.util.Date; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Random; import java.util.stream.Collectors; import org.apache.commons.lang3.time.DateUtils; @@ -259,7 +259,7 @@ public class TestHoodieCompactionStrategy { df.getPath(), df.getFileId(), partitionPath, - config.getCompactionStrategy().captureMetrics(config, Optional.of(df), partitionPath, logFiles))); + config.getCompactionStrategy().captureMetrics(config, Option.of(df), partitionPath, logFiles))); }); return operations; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java index ca9ad487b..306f9683f 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestCopyOnWriteTable.java @@ -36,6 +36,7 @@ import com.uber.hoodie.common.model.HoodieTestUtils; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.ParquetUtils; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieCompactionConfig; @@ -63,7 +64,6 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import scala.Option; import scala.Tuple2; public class TestCopyOnWriteTable { @@ -418,7 +418,7 @@ public class TestCopyOnWriteTable { HoodieCopyOnWriteTable.UpsertPartitioner partitioner = (HoodieCopyOnWriteTable.UpsertPartitioner) table.getUpsertPartitioner(profile); assertEquals("Update record should have gone to the 1 update partiton", 0, partitioner.getPartition( - new Tuple2<>(updateRecords.get(0).getKey(), Option.apply(updateRecords.get(0).getCurrentLocation())))); + new Tuple2<>(updateRecords.get(0).getKey(), Option.ofNullable(updateRecords.get(0).getCurrentLocation())))); return partitioner; } diff --git a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java index 2a663c3db..2857f12e6 100644 --- a/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java +++ b/hoodie-client/src/test/java/com/uber/hoodie/table/TestMergeOnReadTable.java @@ -49,6 +49,7 @@ import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant.State; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.config.HoodieIndexConfig; import com.uber.hoodie.config.HoodieStorageConfig; @@ -62,7 +63,6 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.avro.generic.GenericRecord; @@ -182,11 +182,11 @@ public class TestMergeOnReadTable { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); - Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Option commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); @@ -225,7 +225,7 @@ public class TestMergeOnReadTable { commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); - String compactionCommitTime = client.scheduleCompaction(Optional.empty()).get().toString(); + String compactionCommitTime = client.scheduleCompaction(Option.empty()).get().toString(); client.compact(compactionCommitTime); allFiles = HoodieTestUtils.listAllDataFilesInPath(dfs, cfg.getBasePath()); @@ -289,11 +289,11 @@ public class TestMergeOnReadTable { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); - Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Option commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); @@ -373,7 +373,7 @@ public class TestMergeOnReadTable { assertNoWriteErrors(statuses); HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Option commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertTrue(commit.isPresent()); assertEquals("commit should be 001", "001", commit.get().getTimestamp()); @@ -436,11 +436,11 @@ public class TestMergeOnReadTable { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); - Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Option commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); @@ -539,9 +539,9 @@ public class TestMergeOnReadTable { metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); - String compactionInstantTime = client.scheduleCompaction(Optional.empty()).get().toString(); + String compactionInstantTime = client.scheduleCompaction(Option.empty()).get().toString(); JavaRDD ws = client.compact(compactionInstantTime); - client.commitCompaction(compactionInstantTime, ws, Optional.empty()); + client.commitCompaction(compactionInstantTime, ws, Option.empty()); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); @@ -601,11 +601,11 @@ public class TestMergeOnReadTable { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); - Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Option commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); @@ -666,7 +666,7 @@ public class TestMergeOnReadTable { String compactionInstantTime = "004"; allCommits.add(compactionInstantTime); - client.scheduleCompactionAtInstant(compactionInstantTime, Optional.empty()); + client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); // Compaction commit /** @@ -689,9 +689,9 @@ public class TestMergeOnReadTable { compactionInstantTime = "006"; allCommits.add(compactionInstantTime); - client.scheduleCompactionAtInstant(compactionInstantTime, Optional.empty()); + client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); JavaRDD ws = client.compact(compactionInstantTime); - client.commitCompaction(compactionInstantTime, ws, Optional.empty()); + client.commitCompaction(compactionInstantTime, ws, Option.empty()); allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); @@ -773,11 +773,11 @@ public class TestMergeOnReadTable { HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.getBasePath()); HoodieTable hoodieTable = HoodieTable.getHoodieTable(metaClient, cfg, jsc); - Optional deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); + Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); assertEquals("Delta commit should be 001", "001", deltaCommit.get().getTimestamp()); - Optional commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); + Option commit = metaClient.getActiveTimeline().getCommitTimeline().firstInstant(); assertFalse(commit.isPresent()); FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(metaClient.getFs(), cfg.getBasePath()); @@ -879,10 +879,10 @@ public class TestMergeOnReadTable { // Mark 2nd delta-instant as completed metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Optional.empty()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty()); // Do a compaction - String compactionInstantTime = writeClient.scheduleCompaction(Optional.empty()).get().toString(); + String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString(); JavaRDD result = writeClient.compact(compactionInstantTime); // Verify that recently written compacted data file has no log file @@ -941,9 +941,9 @@ public class TestMergeOnReadTable { Assert.assertTrue(totalUpsertTime > 0); // Do a compaction - String compactionInstantTime = writeClient.scheduleCompaction(Optional.empty()).get().toString(); + String compactionInstantTime = writeClient.scheduleCompaction(Option.empty()).get().toString(); statuses = writeClient.compact(compactionInstantTime); - writeClient.commitCompaction(compactionInstantTime, statuses, Optional.empty()); + writeClient.commitCompaction(compactionInstantTime, statuses, Option.empty()); // total time taken for scanning log files should be greater than 0 long timeTakenForScanner = statuses.map(writeStatus -> writeStatus.getStat().getRuntimeStats().getTotalScanTime()) .reduce((a, b) -> a + b).longValue(); @@ -982,11 +982,11 @@ public class TestMergeOnReadTable { Assert.assertTrue(numLogFiles > 0); // Do a compaction - String commitTime = writeClient.scheduleCompaction(Optional.empty()).get().toString(); + String commitTime = writeClient.scheduleCompaction(Option.empty()).get().toString(); statuses = writeClient.compact(commitTime); Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles); Assert.assertEquals(statuses.count(), numLogFiles); - writeClient.commitCompaction(commitTime, statuses, Optional.empty()); + writeClient.commitCompaction(commitTime, statuses, Option.empty()); } @Test @@ -1077,12 +1077,12 @@ public class TestMergeOnReadTable { Assert.assertTrue(numLogFiles > 0); // Do a compaction - newCommitTime = writeClient.scheduleCompaction(Optional.empty()).get().toString(); + newCommitTime = writeClient.scheduleCompaction(Option.empty()).get().toString(); statuses = writeClient.compact(newCommitTime); // Ensure all log files have been compacted into parquet files Assert.assertTrue(statuses.map(status -> status.getStat().getPath().contains("parquet")).count() == numLogFiles); Assert.assertEquals(statuses.count(), numLogFiles); - writeClient.commitCompaction(newCommitTime, statuses, Optional.empty()); + writeClient.commitCompaction(newCommitTime, statuses, Option.empty()); // Trigger a rollback of compaction writeClient.rollback(newCommitTime); table = HoodieTable.getHoodieTable(new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath), config, jsc); @@ -1112,7 +1112,7 @@ public class TestMergeOnReadTable { String commitActionType = table.getMetaClient().getCommitActionType(); HoodieInstant instant = new HoodieInstant(true, commitActionType, "000"); activeTimeline.createInflight(instant); - activeTimeline.saveAsComplete(instant, Optional.empty()); + activeTimeline.saveAsComplete(instant, Option.empty()); String commitTime = "001"; client.startCommitWithTime(commitTime); @@ -1259,9 +1259,9 @@ public class TestMergeOnReadTable { // Test small file handling after compaction commitTime = "002"; - client.scheduleCompactionAtInstant(commitTime, Optional.of(metadata.getExtraMetadata())); + client.scheduleCompactionAtInstant(commitTime, Option.of(metadata.getExtraMetadata())); statuses = client.compact(commitTime); - client.commitCompaction(commitTime, statuses, Optional.empty()); + client.commitCompaction(commitTime, statuses, Option.empty()); // Read from commit file table = HoodieTable.getHoodieTable(metaClient, cfg, jsc); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java index 7df42f1bd..3e3aebb53 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieCleanStat.java @@ -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 earliestCommitToRetain) { + public Builder withEarliestCommitRetained(Option earliestCommitToRetain) { this.earliestCommitToRetain = (earliestCommitToRetain.isPresent()) ? earliestCommitToRetain.get().getTimestamp() : "-1"; return this; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieJsonPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieJsonPayload.java index 145fba313..ddbf5d6fe 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieJsonPayload.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/HoodieJsonPayload.java @@ -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 } @Override - public Optional combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) + public Option combineAndGetUpdateValue(IndexedRecord oldRec, Schema schema) throws IOException { return getInsertValue(schema); } @Override - public Optional getInsertValue(Schema schema) throws IOException { + public Option 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 { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java index ee842f6ef..b69159896 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/CompactionOperation.java @@ -60,7 +60,7 @@ public class CompactionOperation implements Serializable { this.metrics = metrics; } - public CompactionOperation(java.util.Optional dataFile, String partitionPath, + public CompactionOperation(Option dataFile, String partitionPath, List logFiles, Map metrics) { if (dataFile.isPresent()) { this.baseInstantTime = dataFile.get().getCommitTime(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java index 95c98cf17..305f33704 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/FileSlice.java @@ -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 getDataFile() { - return Optional.ofNullable(dataFile); + public Option getDataFile() { + return Option.ofNullable(dataFile); } - public Optional getLatestLogFile() { - return logFiles.stream().findFirst(); + public Option getLatestLogFile() { + return Option.fromJavaOptional(logFiles.stream().findFirst()); } /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java index 2d1584e83..a606912dd 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieAvroPayload.java @@ -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 // java serializable private final byte [] recordBytes; - public HoodieAvroPayload(Optional record) { + public HoodieAvroPayload(Option record) { try { if (record.isPresent()) { this.recordBytes = HoodieAvroUtils.avroToBytes(record.get()); @@ -54,16 +54,16 @@ public class HoodieAvroPayload implements HoodieRecordPayload } @Override - public Optional combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { return getInsertValue(schema); } @Override - public Optional getInsertValue(Schema schema) throws IOException { + public Option 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)); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java index 18e89eeec..d06e112dc 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieFileGroup.java @@ -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 getLatestFileSlicesIncludingInflight() { - return getAllFileSlicesIncludingInflight().findFirst(); + public Option getLatestFileSlicesIncludingInflight() { + return Option.fromJavaOptional(getAllFileSlicesIncludingInflight().findFirst()); } /** @@ -147,28 +147,28 @@ public class HoodieFileGroup implements Serializable { *

* - just the log files without data file - (or) data file with 0 or more log files */ - public Optional getLatestFileSlice() { + public Option getLatestFileSlice() { // there should always be one - return getAllFileSlices().findFirst(); + return Option.fromJavaOptional(getAllFileSlices().findFirst()); } /** * Gets the latest data file */ - public Optional getLatestDataFile() { - return getAllDataFiles().findFirst(); + public Option getLatestDataFile() { + return Option.fromJavaOptional(getAllDataFiles().findFirst()); } /** * Obtain the latest file slice, upto a commitTime i.e <= maxCommitTime */ - public Optional getLatestFileSliceBeforeOrOn(String maxCommitTime) { - return getAllFileSlices() + public Option 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 getLatestFileSliceBefore(String maxInstantTime) { - return getAllFileSlices() + public Option getLatestFileSliceBefore(String maxInstantTime) { + return Option.fromJavaOptional(getAllFileSlices() .filter(slice -> HoodieTimeline.compareTimestamps(slice.getBaseInstantTime(), maxInstantTime, HoodieTimeline.LESSER)) - .findFirst(); + .findFirst()); } - public Optional getLatestFileSliceInRange(List commitRange) { - return getAllFileSlices() + public Option getLatestFileSliceInRange(List commitRange) { + return Option.fromJavaOptional(getAllFileSlices() .filter(slice -> commitRange.contains(slice.getBaseInstantTime())) - .findFirst(); + .findFirst()); } /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java index ec24539b8..0deadb300 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecord.java @@ -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 implements Serializable return this; } - public Optional getNewLocation() { - return Optional.ofNullable(this.newLocation); + public Option getNewLocation() { + return Option.ofNullable(this.newLocation); } public boolean isCurrentLocationKnown() { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java index 3d790d648..ca2eb0841 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/model/HoodieRecordPayload.java @@ -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 extends Seri * @return new combined/merged value to be written back to storage. EMPTY to skip writing this * record. */ - Optional combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) + Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException; /** @@ -58,7 +58,7 @@ public interface 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 getInsertValue(Schema schema) throws IOException; + Option 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 extends Seri * compute some aggregate metrics using the metadata in the context of a write success or * failure. */ - default Optional> getMetadata() { - return Optional.empty(); + default Option> getMetadata() { + return Option.empty(); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java index 642c08d64..759343352 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/HoodieTimeline.java @@ -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 firstInstant(); + Option firstInstant(); /** * @return nth completed instant from the first completed instant */ - Optional nthInstant(int n); + Option nthInstant(int n); /** * @return last completed instant if available */ - Optional lastInstant(); + Option lastInstant(); /** @@ -169,7 +169,7 @@ public interface HoodieTimeline extends Serializable { /** * @return nth completed instant going back from the last completed instant */ - Optional nthFromLastInstant(int n); + Option 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 getInstantDetails(HoodieInstant instant); + Option getInstantDetails(HoodieInstant instant); /** * Helper methods to compare instants diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java index 21ffc6040..13b53e83d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFileReader.java @@ -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<>()); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java index 650700a91..8bfbfc74e 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/HoodieLogFormat.java @@ -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> versionAndWriteToken = + Option> versionAndWriteToken = FSUtils.getLatestLogVersion(fs, parentPath, logFileId, fileExtension, commitTime); if (versionAndWriteToken.isPresent()) { logVersion = versionAndWriteToken.get().getKey(); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java index f28694071..edc6f9d61 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieAvroDataBlock.java @@ -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 records, @Nonnull Map header, @Nonnull Map 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 content, @Nonnull FSDataInputStream inputStream, - boolean readBlockLazily, Optional blockContentLocation, + private HoodieAvroDataBlock(Option content, @Nonnull FSDataInputStream inputStream, + boolean readBlockLazily, Option blockContentLocation, Schema readerSchema, @Nonnull Map headers, @Nonnull Map 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 content, + Option content, boolean readBlockLazily, long position, long blockSize, @@ -92,7 +92,7 @@ public class HoodieAvroDataBlock extends HoodieLogBlock { Map 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 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; } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java index cfc7cebf4..acb25ee4e 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCommandBlock.java @@ -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 header) { - this(Optional.empty(), null, false, Optional.empty(), header, new HashMap<>()); + this(Option.empty(), null, false, Option.empty(), header, new HashMap<>()); } - private HoodieCommandBlock(Optional content, FSDataInputStream inputStream, - boolean readBlockLazily, Optional blockContentLocation, + private HoodieCommandBlock(Option content, FSDataInputStream inputStream, + boolean readBlockLazily, Option blockContentLocation, Map header, Map 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 content, + Option content, boolean readBlockLazily, long position, long blockSize, @@ -72,7 +72,7 @@ public class HoodieCommandBlock extends HoodieLogBlock { Map 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); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java index f62957839..7dd4cdbe9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieCorruptBlock.java @@ -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 corruptedBytes, FSDataInputStream inputStream, - boolean readBlockLazily, Optional blockContentLocation, + private HoodieCorruptBlock(Option corruptedBytes, FSDataInputStream inputStream, + boolean readBlockLazily, Option blockContentLocation, Map header, Map 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 corruptedBytes, + Option corruptedBytes, boolean readBlockLazily, long position, long blockSize, @@ -62,7 +62,7 @@ public class HoodieCorruptBlock extends HoodieLogBlock { Map 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); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java index 2dcc0b8b5..a2b695351 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieDeleteBlock.java @@ -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 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 content, FSDataInputStream inputStream, - boolean readBlockLazily, Optional blockContentLocation, + private HoodieDeleteBlock(Option content, FSDataInputStream inputStream, + boolean readBlockLazily, Option blockContentLocation, Map header, Map 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 content, + Option content, boolean readBlockLazily, long position, long blockSize, @@ -112,7 +112,7 @@ public class HoodieDeleteBlock extends HoodieLogBlock { Map 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); } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java index ebd978a7c..d3ad4c258 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/log/block/HoodieLogBlock.java @@ -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 logBlockFooter; // Location of a log block on disk - private final Optional blockContentLocation; + private final Option blockContentLocation; // data for a specific block - private Optional content; + private Option 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 logBlockHeader, @Nonnull Map logBlockFooter, - @Nonnull Optional blockContentLocation, - @Nonnull Optional content, + @Nonnull Option blockContentLocation, + @Nonnull Option 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 getBlockContentLocation() { + public Option getBlockContentLocation() { return this.blockContentLocation; } @@ -104,7 +104,7 @@ public abstract class HoodieLogBlock { return logBlockFooter; } - public Optional getContent() { + public Option 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(); } /** diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java index d725dfbcc..0509e5b67 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieActiveTimeline.java @@ -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> & Serializable) this::getInstantDetails; + (Function> & 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> & Serializable) this::getInstantDetails); + (Function> & Serializable) this::getInstantDetails); } /** @@ -164,7 +164,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { */ public HoodieTimeline getTimelineOfActions(Set actions) { return new HoodieDefaultTimeline(getInstants().filter(s -> actions.contains(s.getAction())), - (Function> & Serializable) this::getInstantDetails); + (Function> & Serializable) this::getInstantDetails); } @@ -173,7 +173,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { */ public HoodieTimeline getCleanerTimeline() { return new HoodieDefaultTimeline(filterInstantsByAction(CLEAN_ACTION), - (Function> & Serializable) this::getInstantDetails); + (Function> & Serializable) this::getInstantDetails); } /** @@ -181,7 +181,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { */ public HoodieTimeline getRollbackTimeline() { return new HoodieDefaultTimeline(filterInstantsByAction(ROLLBACK_ACTION), - (Function> & Serializable) this::getInstantDetails); + (Function> & Serializable) this::getInstantDetails); } /** @@ -189,7 +189,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { */ public HoodieTimeline getSavePointTimeline() { return new HoodieDefaultTimeline(filterInstantsByAction(SAVEPOINT_ACTION), - (Function> & Serializable) this::getInstantDetails); + (Function> & Serializable) this::getInstantDetails); } /** @@ -197,7 +197,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { */ public HoodieTimeline getRestoreTimeline() { return new HoodieDefaultTimeline(filterInstantsByAction(RESTORE_ACTION), - (Function> & Serializable) this::getInstantDetails); + (Function> & Serializable) this::getInstantDetails); } protected Stream 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 data) { + public void saveAsComplete(HoodieInstant instant, Option 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 getInstantDetails(HoodieInstant instant) { + public Option getInstantDetails(HoodieInstant instant) { Path detailPath = new Path(metaClient.getMetaPath(), instant.getFileName()); return readDataFromPath(detailPath); } /** BEGIN - COMPACTION RELATED META-DATA MANAGEMENT **/ - public Optional getInstantAuxiliaryDetails(HoodieInstant instant) { + public Option 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 data) { + public HoodieInstant transitionCompactionInflightToComplete(HoodieInstant inflightInstant, Option 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 data) { + private void createFileInAuxiliaryFolder(HoodieInstant instant, Option 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 data) { + Option 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 content) { + public void saveToInflight(HoodieInstant instant, Option content) { Preconditions.checkArgument(instant.isInflight()); createFileInMetaPath(instant.getFileName(), content); } - public void saveToCompactionRequested(HoodieInstant instant, Optional content) { + public void saveToCompactionRequested(HoodieInstant instant, Option 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 content) { + private void createFileInMetaPath(String filename, Option content) { Path fullPath = new Path(metaClient.getMetaPath(), filename); createFileInPath(fullPath, content); } - private void createFileInPath(Path fullPath, Optional content) { + private void createFileInPath(Path fullPath, Option 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 readDataFromPath(Path detailPath) { + private Option 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); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java index 7a9f17d58..d00e79db4 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieArchivedTimeline.java @@ -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> & Serializable) this::getInstantDetails; + (Function> & Serializable) this::getInstantDetails; this.metaClient = metaClient; } @@ -101,8 +101,8 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline { } @Override - public Optional getInstantDetails(HoodieInstant instant) { - return Optional.ofNullable(readCommits.get(instant.getTimestamp())); + public Option getInstantDetails(HoodieInstant instant) { + return Option.ofNullable(readCommits.get(instant.getTimestamp())); } public HoodieArchivedTimeline reload() { diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java index ed80c3c14..aac13f1b8 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/timeline/HoodieDefaultTimeline.java @@ -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> details; + protected transient Function> details; private List instants; private String timelineHash; public HoodieDefaultTimeline(Stream instants, - Function> details) { + Function> details) { this.details = details; setInstants(instants.collect(Collectors.toList())); } @@ -149,27 +149,27 @@ public class HoodieDefaultTimeline implements HoodieTimeline { } @Override - public Optional firstInstant() { - return instants.stream().findFirst(); + public Option firstInstant() { + return Option.fromJavaOptional(instants.stream().findFirst()); } @Override - public Optional nthInstant(int n) { + public Option 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 lastInstant() { - return empty() ? Optional.empty() : nthInstant(countInstants() - 1); + public Option lastInstant() { + return empty() ? Option.empty() : nthInstant(countInstants() - 1); } @Override - public Optional nthFromLastInstant(int n) { + public Option 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 firstCommit = firstInstant(); + Option firstCommit = firstInstant(); return firstCommit.isPresent() && HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER); } @Override - public Optional getInstantDetails(HoodieInstant instant) { + public Option getInstantDetails(HoodieInstant instant) { return details.apply(instant); } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/AbstractTableFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/AbstractTableFileSystemView.java index 62410acef..0a2863989 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/AbstractTableFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/AbstractTableFileSystemView.java @@ -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 = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime); + Option 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 fetchLatestFileSliceInRange(List 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 fetchLatestDataFiles(final String partitionPath) { return fetchAllStoredFileGroups(partitionPath) .map(this::getLatestDataFile) - .filter(Optional::isPresent) - .map(Optional::get); + .filter(Option::isPresent) + .map(Option::get); } - protected Optional getLatestDataFile(HoodieFileGroup fileGroup) { - return fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst(); + protected Option getLatestDataFile(HoodieFileGroup fileGroup) { + return Option.fromJavaOptional( + fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst()); } /** @@ -692,8 +692,8 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV Stream 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 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 prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime); + Option 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 getLastInstant() { - return Option.fromJavaOptional(getTimeline().lastInstant()); + return getTimeline().lastInstant(); } @Override diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/IncrementalTimelineSyncFileSystemView.java b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/IncrementalTimelineSyncFileSystemView.java index c087de09d..e6ccf59b7 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/IncrementalTimelineSyncFileSystemView.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/table/view/IncrementalTimelineSyncFileSystemView.java @@ -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 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 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)); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java index 57c066979..ade9248e9 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/AvroUtils.java @@ -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 durationInMs, List cleanStats) { + Option durationInMs, List cleanStats) { ImmutableMap.Builder partitionMetadataBuilder = ImmutableMap.builder(); int totalDeleted = 0; @@ -119,7 +118,7 @@ public class AvroUtils { } public static HoodieRestoreMetadata convertRestoreMetadata(String startRestoreTime, - Optional durationInMs, List commits, Map> commitToStats) { + Option durationInMs, List commits, Map> commitToStats) { ImmutableMap.Builder> commitToStatBuilder = ImmutableMap.builder(); for (Map.Entry> 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 durationInMs, List commits, List rollbackStats) { + Option durationInMs, List commits, List rollbackStats) { ImmutableMap.Builder partitionMetadataBuilder = ImmutableMap.builder(); int totalDeleted = 0; @@ -159,32 +158,32 @@ public class AvroUtils { partitionMetadataBuilder.build()); } - public static Optional serializeCompactionPlan(HoodieCompactionPlan compactionWorkload) + public static Option serializeCompactionPlan(HoodieCompactionPlan compactionWorkload) throws IOException { return serializeAvroMetadata(compactionWorkload, HoodieCompactionPlan.class); } - public static Optional serializeCleanMetadata(HoodieCleanMetadata metadata) + public static Option serializeCleanMetadata(HoodieCleanMetadata metadata) throws IOException { return serializeAvroMetadata(metadata, HoodieCleanMetadata.class); } - public static Optional serializeSavepointMetadata(HoodieSavepointMetadata metadata) + public static Option serializeSavepointMetadata(HoodieSavepointMetadata metadata) throws IOException { return serializeAvroMetadata(metadata, HoodieSavepointMetadata.class); } - public static Optional serializeRollbackMetadata( + public static Option serializeRollbackMetadata( HoodieRollbackMetadata rollbackMetadata) throws IOException { return serializeAvroMetadata(rollbackMetadata, HoodieRollbackMetadata.class); } - public static Optional serializeRestoreMetadata( + public static Option serializeRestoreMetadata( HoodieRestoreMetadata restoreMetadata) throws IOException { return serializeAvroMetadata(restoreMetadata, HoodieRestoreMetadata.class); } - public static Optional serializeAvroMetadata(T metadata, + public static Option serializeAvroMetadata(T metadata, Class clazz) throws IOException { DatumWriter datumWriter = new SpecificDatumWriter<>(clazz); DataFileWriter 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) diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java index e0927ce1d..359a0e8ef 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/CompactionUtils.java @@ -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, Map>> metricsCaptureFunction) { + Option, Map>> 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> partitionFileSlicePairs, - Optional> extraMetadata, - Optional, Map>> metricsCaptureFunction) { + Option> extraMetadata, + Option, Map>> metricsCaptureFunction) { HoodieCompactionPlan.Builder builder = HoodieCompactionPlan.newBuilder(); extraMetadata.ifPresent(m -> builder.setExtraMetadata(m)); builder.setOperations(partitionFileSlicePairs.stream().map(pfPair -> diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java index 0d67d034c..2df24a08a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/FSUtils.java @@ -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 getLatestLogFile(Stream logFiles) { - return logFiles.sorted(HoodieLogFile.getReverseLogFileComparator()).findFirst(); + public static Option getLatestLogFile(Stream 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> getLatestLogVersion(FileSystem fs, Path partitionPath, + public static Option> getLatestLogVersion(FileSystem fs, Path partitionPath, final String fileId, final String logFileExtension, final String baseCommitTime) throws IOException { - Optional latestLogFile = + Option 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> currentVersionWithWriteToken = + Option> currentVersionWithWriteToken = getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime); // handle potential overflow return (currentVersionWithWriteToken.isPresent()) ? currentVersionWithWriteToken.get().getKey() + 1 diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java index 020cf3bad..abba2d6c5 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/ParquetUtils.java @@ -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 filterParquetRowKeys(Configuration configuration, Path filePath, Set filter) { - Optional filterFunction = Optional.empty(); + Option 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()); diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java index 65f56346c..13d80585a 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/SpillableMapUtils.java @@ -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 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 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; } } diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/TimelineDiffHelper.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/TimelineDiffHelper.java index 628800b43..7de28746d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/TimelineDiffHelper.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/TimelineDiffHelper.java @@ -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 lastSeenInstant = oldT.lastInstant(); - Optional firstInstantInNewTimeline = newT.firstInstant(); + Option lastSeenInstant = oldT.lastInstant(); + Option firstInstantInNewTimeline = newT.firstInstant(); if (lastSeenInstant.isPresent() && firstInstantInNewTimeline.isPresent()) { if (HoodieTimeline.compareTimestamps(lastSeenInstant.get().getTimestamp(), diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java index f3cb5a939..fe53dca24 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryExecutor.java @@ -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 { // Producers private final List> producers; // Consumer - private final Optional> consumer; + private final Option> consumer; public BoundedInMemoryExecutor(final long bufferLimitInBytes, BoundedInMemoryQueueProducer producer, - Optional> consumer, + Option> consumer, final Function transformFunction) { this(bufferLimitInBytes, Arrays.asList(producer), consumer, transformFunction, new DefaultSizeEstimator<>()); } public BoundedInMemoryExecutor(final long bufferLimitInBytes, List> producers, - Optional> consumer, + Option> consumer, final Function transformFunction, final SizeEstimator sizeEstimator) { this.producers = producers; diff --git a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java index 793d5034a..a77edc63d 100644 --- a/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java +++ b/hoodie-common/src/main/java/com/uber/hoodie/common/util/queue/BoundedInMemoryQueue.java @@ -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 implements Iterable { // used for sampling records with "RECORD_SAMPLING_RATE" frequency. public final AtomicLong samplingRecordCounter = new AtomicLong(-1); // internal queue for records. - private final LinkedBlockingQueue> queue = new + private final LinkedBlockingQueue> queue = new LinkedBlockingQueue<>(); // maximum amount of memory to be used for queueing records. private final long memoryLimit; @@ -176,7 +176,7 @@ public class BoundedInMemoryQueue implements Iterable { // 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 implements Iterable { * 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 readNextRecord() { + private Option readNextRecord() { if (this.isReadDone.get()) { - return Optional.empty(); + return Option.empty(); } rateLimiter.release(); - Optional newRecord = Optional.empty(); + Option newRecord = Option.empty(); while (expectMoreRecords()) { try { throwExceptionIfFailed(); @@ -217,7 +217,7 @@ public class BoundedInMemoryQueue implements Iterable { } 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 implements Iterable { @Override public boolean hasNext() { if (this.nextRecord == null) { - Optional res = readNextRecord(); + Option res = readNextRecord(); this.nextRecord = res.orElse(null); } return this.nextRecord != null; diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java index e716e3adb..1908bf906 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/AvroBinaryTestPayload.java @@ -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 record) { + public AvroBinaryTestPayload(Option record) { try { if (record.isPresent()) { @@ -49,13 +49,13 @@ public class AvroBinaryTestPayload implements HoodieRecordPayload { } @Override - public Optional combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { return getInsertValue(schema); } @Override - public Optional getInsertValue(Schema schema) throws IOException { - return Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema)); + public Option getInsertValue(Schema schema) throws IOException { + return Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema)); } } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java index fd821e737..7a00606d8 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/model/HoodieTestUtils.java @@ -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 version) throws IOException { + String fileID, Option 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> 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 version) { + Option 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 version) { + Option 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()); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java index 6aa6bcbb8..e30bdcc34 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/HoodieTableMetaClientTest.java @@ -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(); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java index 9361a5b11..2f9574abf 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/string/HoodieActiveTimelineTest.java @@ -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"))); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java index 7a21a3dac..0d14d8e5f 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/HoodieTableFileSystemViewTest.java @@ -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 fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList()); @@ -282,7 +282,7 @@ public class HoodieTableFileSystemViewTest { List> 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 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 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 diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/IncrementalFSViewSyncTest.java b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/IncrementalFSViewSyncTest.java index 950d1d208..349c6cca7 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/IncrementalFSViewSyncTest.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/table/view/IncrementalFSViewSyncTest.java @@ -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())); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java index 391aca47e..1cd5d9c9f 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/CompactionTestUtils.java @@ -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) { diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java index 3f8415b0b..ed5491ee8 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SchemaTestUtil.java @@ -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 updateHoodieTestRecords(List oldRecordKeys, List 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 updateHoodieTestRecordsWithoutHoodieMetadata(List 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); } diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java index ffe0ff969..1b7ea41b9 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/SpillableMapTestUtils.java @@ -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); }); diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java index 99795b9de..4f369375a 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/TestCompactionUtils.java @@ -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 fileSliceList = Arrays.asList(emptyFileSlice, noDataFileSlice, fileSlice, noLogFileSlice); List> 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 diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java index fce84bb73..e55eb94b8 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestDiskBasedMap.java @@ -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 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), diff --git a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java index 1cfb25471..05c2a0745 100644 --- a/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java +++ b/hoodie-common/src/test/java/com/uber/hoodie/common/util/collection/TestExternalSpillableMap.java @@ -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 diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java index 9fede5642..47100aa22 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/HoodieRealtimeInputFormat.java @@ -28,6 +28,7 @@ import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; 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 com.uber.hoodie.hadoop.HoodieInputFormat; @@ -38,7 +39,6 @@ import java.util.Arrays; 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; @@ -88,8 +88,8 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf Map partitionsToMetaClient = partitionsToParquetSplits.keySet() .stream().collect(Collectors.toMap(Function.identity(), p -> { // find if we have a metaclient already for this partition. - Optional matchingBasePath = metaClientMap.keySet().stream() - .filter(basePath -> p.toString().startsWith(basePath)).findFirst(); + Option matchingBasePath = Option.fromJavaOptional(metaClientMap.keySet().stream() + .filter(basePath -> p.toString().startsWith(basePath)).findFirst()); if (matchingBasePath.isPresent()) { return metaClientMap.get(matchingBasePath.get()); } @@ -116,7 +116,7 @@ public class HoodieRealtimeInputFormat extends HoodieInputFormat implements Conf try { // Both commit and delta-commits are included - pick the latest completed one - Optional latestCompletedInstant = + Option latestCompletedInstant = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); Stream latestFileSlices = latestCompletedInstant.map(instant -> diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java index 85efe6fa1..7d6413174 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeCompactedRecordReader.java @@ -23,9 +23,9 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner; import com.uber.hoodie.common.util.FSUtils; import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.common.util.Option; import java.io.IOException; import java.util.Map; -import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.io.NullWritable; @@ -81,7 +81,7 @@ class RealtimeCompactedRecordReader extends AbstractRealtimeRecordReader impleme if (deltaRecordMap.containsKey(key)) { // TODO(NA): Invoke preCombine here by converting arrayWritable to Avro. This is required since the // deltaRecord may not be a full record and needs values of columns from the parquet - Optional rec; + Option rec; if (usesCustomPayload) { rec = deltaRecordMap.get(key).getData().getInsertValue(getWriterSchema()); } else { diff --git a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java index da0994507..5e5382d60 100644 --- a/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java +++ b/hoodie-hadoop-mr/src/main/java/com/uber/hoodie/hadoop/realtime/RealtimeUnmergedRecordReader.java @@ -21,6 +21,7 @@ package com.uber.hoodie.hadoop.realtime; import com.uber.hoodie.common.table.log.HoodieUnMergedLogRecordScanner; import com.uber.hoodie.common.util.DefaultSizeEstimator; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.queue.BoundedInMemoryExecutor; import com.uber.hoodie.common.util.queue.BoundedInMemoryQueueProducer; import com.uber.hoodie.common.util.queue.FunctionBasedQueueProducer; @@ -31,7 +32,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; -import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.io.ArrayWritable; import org.apache.hadoop.io.NullWritable; @@ -71,7 +71,7 @@ class RealtimeUnmergedRecordReader extends AbstractRealtimeRecordReader implemen // Iterator for consuming records from parquet file this.parquetRecordsIterator = new RecordReaderValueIterator<>(this.parquetReader); this.executor = new BoundedInMemoryExecutor<>(getMaxCompactionMemoryInBytes(), getParallelProducers(), - Optional.empty(), x -> x, new DefaultSizeEstimator<>()); + Option.empty(), x -> x, new DefaultSizeEstimator<>()); // Consumer of this record reader this.iterator = this.executor.getQueue().iterator(); this.logRecordScanner = new HoodieUnMergedLogRecordScanner( diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java index e2d0e665e..5412dab51 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HiveSyncTool.java @@ -20,6 +20,7 @@ package com.uber.hoodie.hive; import com.beust.jcommander.JCommander; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.exception.InvalidDatasetException; import com.uber.hoodie.hadoop.HoodieInputFormat; import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat; @@ -28,7 +29,6 @@ import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent.PartitionEventType; import com.uber.hoodie.hive.util.SchemaUtil; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -97,7 +97,7 @@ public class HiveSyncTool { LOG.info("Schema sync complete. Syncing partitions for " + cfg.tableName); // Get the last time we successfully synced partitions - Optional lastCommitTimeSynced = Optional.empty(); + Option lastCommitTimeSynced = Option.empty(); if (tableExists) { lastCommitTimeSynced = hoodieHiveClient.getLastCommitTimeSynced(); } diff --git a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java index 746d600b4..16f42ed90 100644 --- a/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java +++ b/hoodie-hive/src/main/java/com/uber/hoodie/hive/HoodieHiveClient.java @@ -29,6 +29,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; 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.HoodieIOException; import com.uber.hoodie.exception.InvalidDatasetException; @@ -44,7 +45,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; import org.apache.commons.dbcp.BasicDataSource; import org.apache.commons.dbcp.ConnectionFactory; @@ -329,13 +329,13 @@ public class HoodieHiveClient { // If this is MOR, depending on whether the latest commit is a delta commit or // compaction commit // Get a datafile written and get the schema from that file - Optional lastCompactionCommit = metaClient.getActiveTimeline() + Option lastCompactionCommit = metaClient.getActiveTimeline() .getCommitTimeline() .filterCompletedInstants() .lastInstant(); LOG.info("Found the last compaction commit as " + lastCompactionCommit); - Optional lastDeltaCommit; + Option lastDeltaCommit; if (lastCompactionCommit.isPresent()) { lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline() .filterCompletedInstants() @@ -394,7 +394,7 @@ public class HoodieHiveClient { * Read schema from a data file from the last compaction commit done. */ @SuppressWarnings("OptionalUsedAsFieldOrParameterType") - private MessageType readSchemaFromLastCompaction(Optional lastCompactionCommitOpt) + private MessageType readSchemaFromLastCompaction(Option lastCompactionCommitOpt) throws IOException { HoodieInstant lastCompactionCommit = lastCompactionCommitOpt.orElseThrow( () -> new HoodieHiveSyncException( @@ -415,7 +415,7 @@ public class HoodieHiveClient { * Read the schema from the log file on path */ @SuppressWarnings("OptionalUsedAsFieldOrParameterType") - private MessageType readSchemaFromLogFile(Optional lastCompactionCommitOpt, + private MessageType readSchemaFromLogFile(Option lastCompactionCommitOpt, Path path) throws IOException { MessageType messageType = SchemaUtil.readSchemaFromLogFile(fs, path); // Fall back to read the schema from last compaction @@ -530,11 +530,11 @@ public class HoodieHiveClient { return fs; } - public Optional getLastCommitTimeSynced() { + public Option getLastCommitTimeSynced() { // Get the last commit time from the TBLproperties try { Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName); - return Optional.ofNullable( + return Option.ofNullable( database.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null)); } catch (Exception e) { throw new HoodieHiveSyncException( @@ -556,7 +556,7 @@ public class HoodieHiveClient { } @SuppressWarnings("OptionalUsedAsFieldOrParameterType") - List getPartitionsWrittenToSince(Optional lastCommitTimeSynced) { + List getPartitionsWrittenToSince(Option lastCommitTimeSynced) { if (!lastCommitTimeSynced.isPresent()) { LOG.info("Last commit time synced is not known, listing all partitions in " + syncConfig.basePath + ",FS :" + fs); try { diff --git a/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java b/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java index de79667be..51dded52d 100644 --- a/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java +++ b/hoodie-hive/src/test/java/com/uber/hoodie/hive/HiveSyncToolTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import com.google.common.collect.Lists; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.SchemaTestUtil; import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent; import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent.PartitionEventType; @@ -30,7 +31,6 @@ import com.uber.hoodie.hive.util.SchemaUtil; import java.io.IOException; import java.net.URISyntaxException; import java.util.List; -import java.util.Optional; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.thrift.TException; import org.joda.time.DateTime; @@ -195,7 +195,7 @@ public class HiveSyncToolTest { hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem); List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince( - Optional.of(commitTime1)); + Option.of(commitTime1)); assertEquals("We should have one partition written after 100 commit", 1, writtenPartitionsSince.size()); List hivePartitions = hiveClient.scanTablePartitions(); diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java index d700ff607..ad754d684 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/DataSourceUtils.java @@ -22,6 +22,7 @@ import com.uber.hoodie.client.embedded.EmbeddedTimelineService; import com.uber.hoodie.common.model.HoodieKey; import com.uber.hoodie.common.model.HoodieRecord; import com.uber.hoodie.common.model.HoodieRecordPayload; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.config.HoodieCompactionConfig; @@ -39,7 +40,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.stream.Collectors; import org.apache.avro.Schema.Field; import org.apache.avro.generic.GenericRecord; @@ -191,7 +191,7 @@ public class DataSourceUtils { @SuppressWarnings("unchecked") public static JavaRDD dropDuplicates(JavaSparkContext jssc, JavaRDD incomingHoodieRecords, - HoodieWriteConfig writeConfig, Optional timelineService) throws Exception { + HoodieWriteConfig writeConfig, Option timelineService) throws Exception { HoodieReadClient client = null; try { client = new HoodieReadClient<>(jssc, writeConfig, timelineService); @@ -212,7 +212,7 @@ public class DataSourceUtils { public static JavaRDD dropDuplicates(JavaSparkContext jssc, JavaRDD incomingHoodieRecords, Map parameters, - Optional timelineService) + Option timelineService) throws Exception { HoodieWriteConfig writeConfig = HoodieWriteConfig .newBuilder() diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/EmptyHoodieRecordPayload.java b/hoodie-spark/src/main/java/com/uber/hoodie/EmptyHoodieRecordPayload.java index 470760ea3..67be81b96 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/EmptyHoodieRecordPayload.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/EmptyHoodieRecordPayload.java @@ -19,7 +19,7 @@ package com.uber.hoodie; import com.uber.hoodie.common.model.HoodieRecordPayload; -import java.util.Optional; +import com.uber.hoodie.common.util.Option; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -37,12 +37,12 @@ public class EmptyHoodieRecordPayload implements HoodieRecordPayload combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) { - return Optional.empty(); + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) { + return Option.empty(); } @Override - public Optional getInsertValue(Schema schema) { - return Optional.empty(); + public Option getInsertValue(Schema schema) { + return Option.empty(); } } diff --git a/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java b/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java index 8630893f0..5a9a5e43f 100644 --- a/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java +++ b/hoodie-spark/src/main/java/com/uber/hoodie/OverwriteWithLatestAvroPayload.java @@ -20,8 +20,8 @@ package com.uber.hoodie; import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.util.HoodieAvroUtils; +import com.uber.hoodie.common.util.Option; 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; @@ -43,7 +43,7 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements super(record, orderingVal); } - public OverwriteWithLatestAvroPayload(Optional record) { + public OverwriteWithLatestAvroPayload(Option record) { this(record.get(), (record1) -> 0); // natural order } @@ -58,14 +58,14 @@ public class OverwriteWithLatestAvroPayload extends BaseAvroPayload implements } @Override - public Optional combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) + public Option combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema) throws IOException { // combining strategy here trivially ignores currentValue on disk and writes this record return getInsertValue(schema); } @Override - public Optional getInsertValue(Schema schema) throws IOException { - return Optional.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema)); + public Option getInsertValue(Schema schema) throws IOException { + return Option.of(HoodieAvroUtils.bytesToAvro(recordBytes, schema)); } } diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala index 414cad4de..32bb72da1 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieSparkSqlWriter.scala @@ -18,7 +18,6 @@ package com.uber.hoodie import java.util -import java.util.Optional import com.uber.hoodie.DataSourceWriteOptions._ import com.uber.hoodie.common.table.HoodieTableMetaClient @@ -26,6 +25,7 @@ import com.uber.hoodie.common.util.{FSUtils, TypedProperties} import com.uber.hoodie.config.HoodieWriteConfig import com.uber.hoodie.exception.HoodieException import com.uber.hoodie.hive.{HiveSyncConfig, HiveSyncTool} +import com.uber.hoodie.common.util.Option import org.apache.avro.generic.GenericRecord import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf @@ -105,7 +105,7 @@ private[hoodie] object HoodieSparkSqlWriter { } if (mode == SaveMode.Ignore && exists) { log.warn(s"hoodie dataset at $basePath already exists. Ignoring & not performing actual writes.") - return (true, None) + return (true, Option.empty()) } if (mode == SaveMode.Overwrite && exists) { log.warn(s"hoodie dataset at $basePath already exists. Deleting existing data & overwriting with new data.") @@ -136,7 +136,7 @@ private[hoodie] object HoodieSparkSqlWriter { if (hoodieRecords.isEmpty()) { log.info("new batch has no new records, skipping...") - return (true, None) + return (true, Option.empty()) } val commitTime = client.startCommit() @@ -153,7 +153,7 @@ private[hoodie] object HoodieSparkSqlWriter { client.commit(commitTime, writeStatuses) } else { client.commit(commitTime, writeStatuses, - Optional.of(new util.HashMap[String, String](mapAsJavaMap(metaMap)))) + Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap)))) } if (commitSuccess) { @@ -189,7 +189,7 @@ private[hoodie] object HoodieSparkSqlWriter { } false } - (writeSuccessful, Some(commitTime)) + (writeSuccessful, Option.ofNullable(commitTime)) } /** diff --git a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieStreamingSink.scala b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieStreamingSink.scala index cbd886b28..2d0776f93 100644 --- a/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieStreamingSink.scala +++ b/hoodie-spark/src/main/scala/com/uber/hoodie/HoodieStreamingSink.scala @@ -56,7 +56,10 @@ class HoodieStreamingSink(sqlContext: SQLContext, ) match { case Success((true, commitOps)) => log.info(s"Micro batch id=$batchId succeeded" - + commitOps.map(commit => s" for commit=$commit").getOrElse(" with no new commits")) + + (commitOps.isPresent match { + case true => s" for commit=${commitOps.get()}" + case _ => s" with no new commits" + })) Success((true, commitOps)) case Failure(e) => // clean up persist rdds in the write process @@ -76,7 +79,10 @@ class HoodieStreamingSink(sqlContext: SQLContext, } case Success((false, commitOps)) => log.error(s"Micro batch id=$batchId ended up with errors" - + commitOps.map(commit => s" for commit=$commit").getOrElse("")) + + (commitOps.isPresent match { + case true => s" for commit=${commitOps.get()}" + case _ => s"" + })) if (ignoreFailedBatch) { log.info(s"Ignore the errors and move on streaming as per " + s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY} configuration") diff --git a/hoodie-spark/src/test/java/DataSourceTestUtils.java b/hoodie-spark/src/test/java/DataSourceTestUtils.java index 3fb413488..12da7acbf 100644 --- a/hoodie-spark/src/test/java/DataSourceTestUtils.java +++ b/hoodie-spark/src/test/java/DataSourceTestUtils.java @@ -18,9 +18,9 @@ import com.uber.hoodie.common.TestRawTripPayload; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.util.Option; import java.io.IOException; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; /** @@ -28,14 +28,14 @@ import java.util.stream.Collectors; */ public class DataSourceTestUtils { - public static Optional convertToString(HoodieRecord record) { + public static Option convertToString(HoodieRecord record) { try { String str = ((TestRawTripPayload) record.getData()).getJsonData(); str = "{" + str.substring(str.indexOf("\"timestamp\":")); - return Optional.of(str.replaceAll("}", + return Option.of(str.replaceAll("}", ", \"partition\": \"" + record.getPartitionPath() + "\"}")); } catch (IOException e) { - return Optional.empty(); + return Option.empty(); } } diff --git a/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala b/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala index b59d389d0..8aa76b67e 100644 --- a/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala +++ b/hoodie-spark/src/test/scala/DataSourceDefaultsTest.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -import java.util.Optional - -import com.uber.hoodie.common.util.{SchemaTestUtil, TypedProperties} +import com.uber.hoodie.common.util.{Option, SchemaTestUtil, TypedProperties} import com.uber.hoodie.exception.HoodieException import com.uber.hoodie.{DataSourceWriteOptions, EmptyHoodieRecordPayload, OverwriteWithLatestAvroPayload, SimpleKeyGenerator, ComplexKeyGenerator} import org.apache.avro.generic.GenericRecord @@ -178,6 +176,6 @@ class DataSourceDefaultsTest extends AssertionsForJUnit { // it will provide an empty record val combinedPayload12 = emptyPayload1.preCombine(emptyPayload2) val combined12 = combinedPayload12.getInsertValue(schema) - assertEquals(Optional.empty(), combined12) + assertEquals(Option.empty(), combined12) } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java index 661f1a114..b5ddb2220 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HDFSParquetImporter.java @@ -32,6 +32,7 @@ import com.uber.hoodie.common.model.HoodieRecordPayload; import com.uber.hoodie.common.table.HoodieTableConfig; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.exception.HoodieIOException; import java.io.IOException; @@ -41,7 +42,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Date; import java.util.List; -import java.util.Optional; import java.util.Properties; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -134,7 +134,7 @@ public class HDFSParquetImporter implements Serializable { .initializePathAsHoodieDataset(jsc.hadoopConfiguration(), cfg.targetPath, properties); HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.targetPath, schemaStr, - cfg.parallelism, Optional.empty(), props); + cfg.parallelism, Option.empty(), props); JavaRDD> hoodieRecords = buildHoodieRecordsForImport(jsc, schemaStr); // Get instant time. diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java index de0550b0a..86788939c 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HiveIncrementalPuller.java @@ -22,6 +22,7 @@ import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.exception.HoodieException; import com.uber.hoodie.utilities.exception.HoodieIncrementalPullException; import com.uber.hoodie.utilities.exception.HoodieIncrementalPullSQLException; @@ -34,7 +35,6 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; import java.util.List; -import java.util.Optional; import java.util.Scanner; import java.util.stream.Collectors; import javax.sql.DataSource; @@ -290,7 +290,7 @@ public class HiveIncrementalPuller { } HoodieTableMetaClient metadata = new HoodieTableMetaClient(fs.getConf(), targetDataPath); - Optional lastCommit = metadata.getActiveTimeline().getCommitsTimeline() + Option lastCommit = metadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants().lastInstant(); if (lastCommit.isPresent()) { return lastCommit.get().getTimestamp(); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java index 564749ea3..0e3657118 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieCompactor.java @@ -23,11 +23,11 @@ import com.beust.jcommander.Parameter; import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import java.io.Serializable; import java.util.ArrayList; import java.util.List; -import java.util.Optional; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; @@ -123,7 +123,7 @@ public class HoodieCompactor { //Get schema. String schemaStr = UtilHelpers.parseSchema(fs, cfg.schemaFile); HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, - Optional.empty(), props); + Option.empty(), props); JavaRDD writeResponse = client.compact(cfg.compactionInstantTime); return UtilHelpers.handleErrors(jsc, cfg.compactionInstantTime, writeResponse); } @@ -131,8 +131,8 @@ public class HoodieCompactor { private int doSchedule(JavaSparkContext jsc) throws Exception { //Get schema. HoodieWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, "", cfg.parallelism, - Optional.of(cfg.strategyClassName), props); - client.scheduleCompactionAtInstant(cfg.compactionInstantTime, Optional.empty()); + Option.of(cfg.strategyClassName), props); + client.scheduleCompactionAtInstant(cfg.compactionInstantTime, Option.empty()); return 0; } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java index 099cdab21..337e1ca2c 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/HoodieSnapshotCopier.java @@ -30,11 +30,11 @@ import com.uber.hoodie.common.table.TableFileSystemView; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.view.HoodieTableFileSystemView; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.List; -import java.util.Optional; import java.util.stream.Stream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -79,7 +79,7 @@ public class HoodieSnapshotCopier implements Serializable { tableMetadata, tableMetadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()); // Get the latest commit - Optional latestCommit = tableMetadata.getActiveTimeline().getCommitsTimeline() + Option latestCommit = tableMetadata.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants().lastInstant(); if (!latestCommit.isPresent()) { logger.warn("No commits present. Nothing to snapshot"); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java index 9e6204ec9..24ff8f599 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/UtilHelpers.java @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.util.DFSPropertiesConfiguration; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.ReflectionUtils; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.config.HoodieCompactionConfig; @@ -40,7 +41,6 @@ import java.nio.ByteBuffer; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -185,7 +185,7 @@ public class UtilHelpers { * @param parallelism Parallelism */ public static HoodieWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, - String schemaStr, int parallelism, Optional compactionStrategyClass, TypedProperties properties) + String schemaStr, int parallelism, Option compactionStrategyClass, TypedProperties properties) throws Exception { HoodieCompactionConfig compactionConfig = compactionStrategyClass.map(strategy -> HoodieCompactionConfig.newBuilder().withInlineCompaction(false) diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/Compactor.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/Compactor.java index d72d7dab2..d07b5e517 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/Compactor.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/Compactor.java @@ -21,10 +21,10 @@ package com.uber.hoodie.utilities.deltastreamer; import com.uber.hoodie.HoodieWriteClient; import com.uber.hoodie.WriteStatus; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.exception.HoodieException; import java.io.IOException; import java.io.Serializable; -import java.util.Optional; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -57,6 +57,6 @@ public class Compactor implements Serializable { + "Errors :" + numWriteErrors); } // Commit compaction - compactionClient.commitCompaction(instant.getTimestamp(), res, Optional.empty()); + compactionClient.commitCompaction(instant.getTimestamp(), res, Option.empty()); } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/DeltaSync.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/DeltaSync.java index ce11d895a..33bb6bfa8 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/DeltaSync.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/DeltaSync.java @@ -34,6 +34,7 @@ import com.uber.hoodie.common.model.HoodieTableType; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.config.HoodieCompactionConfig; @@ -55,7 +56,6 @@ import java.io.Serializable; import java.util.Arrays; import java.util.HashMap; import java.util.List; -import java.util.Optional; import java.util.function.Function; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; @@ -140,7 +140,7 @@ public class DeltaSync implements Serializable { /** * Timeline with completed commits */ - private transient Optional commitTimelineOpt; + private transient Option commitTimelineOpt; /** * Write Client @@ -192,10 +192,10 @@ public class DeltaSync implements Serializable { private void refreshTimeline() throws IOException { if (fs.exists(new Path(cfg.targetBasePath))) { HoodieTableMetaClient meta = new HoodieTableMetaClient(new Configuration(fs.getConf()), cfg.targetBasePath); - this.commitTimelineOpt = Optional.of(meta.getActiveTimeline().getCommitsTimeline() + this.commitTimelineOpt = Option.of(meta.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants()); } else { - this.commitTimelineOpt = Optional.empty(); + this.commitTimelineOpt = Option.empty(); HoodieTableMetaClient.initTableType(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath, cfg.storageType, cfg.targetTableName, "archived"); } @@ -204,8 +204,8 @@ public class DeltaSync implements Serializable { /** * Run one round of delta sync and return new compaction instant if one got scheduled */ - public Optional syncOnce() throws Exception { - Optional scheduledCompaction = Optional.empty(); + public Option syncOnce() throws Exception { + Option scheduledCompaction = Option.empty(); HoodieDeltaStreamerMetrics metrics = new HoodieDeltaStreamerMetrics(getHoodieClientConfig(schemaProvider)); Timer.Context overallTimerContext = metrics.getOverallTimerContext(); @@ -238,18 +238,18 @@ public class DeltaSync implements Serializable { * Read from Upstream Source and apply transformation if needed */ private Pair>> readFromSource( - Optional commitTimelineOpt) throws Exception { + Option commitTimelineOpt) throws Exception { // Retrieve the previous round checkpoints, if any - Optional resumeCheckpointStr = Optional.empty(); + Option resumeCheckpointStr = Option.empty(); if (commitTimelineOpt.isPresent()) { - Optional lastCommit = commitTimelineOpt.get().lastInstant(); + Option lastCommit = commitTimelineOpt.get().lastInstant(); if (lastCommit.isPresent()) { HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( commitTimelineOpt.get().getInstantDetails(lastCommit.get()).get(), HoodieCommitMetadata.class); if (cfg.checkpoint != null && !cfg.checkpoint.equals(commitMetadata.getMetadata(CHECKPOINT_RESET_KEY))) { - resumeCheckpointStr = Optional.of(cfg.checkpoint); + resumeCheckpointStr = Option.of(cfg.checkpoint); } else if (commitMetadata.getMetadata(CHECKPOINT_KEY) != null) { - resumeCheckpointStr = Optional.of(commitMetadata.getMetadata(CHECKPOINT_KEY)); + resumeCheckpointStr = Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY)); } else { throw new HoodieDeltaStreamerException( "Unable to find previous checkpoint. Please double check if this table " @@ -262,11 +262,11 @@ public class DeltaSync implements Serializable { } if (!resumeCheckpointStr.isPresent() && cfg.checkpoint != null) { - resumeCheckpointStr = Optional.of(cfg.checkpoint); + resumeCheckpointStr = Option.of(cfg.checkpoint); } log.info("Checkpoint to resume from : " + resumeCheckpointStr); - final Optional> avroRDDOptional; + final Option> avroRDDOptional; final String checkpointStr; final SchemaProvider schemaProvider; if (transformer != null) { @@ -275,7 +275,7 @@ public class DeltaSync implements Serializable { InputBatch> dataAndCheckpoint = formatAdapter.fetchNewDataInRowFormat( resumeCheckpointStr, cfg.sourceLimit); - Optional> transformed = + Option> transformed = dataAndCheckpoint.getBatch().map(data -> transformer.apply(jssc, sparkSession, data, props)); checkpointStr = dataAndCheckpoint.getCheckpointForNextBatch(); avroRDDOptional = transformed.map(t -> @@ -314,12 +314,12 @@ public class DeltaSync implements Serializable { * @param records Input Records * @param checkpointStr Checkpoint String * @param metrics Metrics - * @return Optional Compaction instant if one is scheduled + * @return Option Compaction instant if one is scheduled */ - private Optional writeToSink(JavaRDD records, String checkpointStr, + private Option writeToSink(JavaRDD records, String checkpointStr, HoodieDeltaStreamerMetrics metrics, Timer.Context overallTimerContext) throws Exception { - Optional scheduledCompactionInstant = Optional.empty(); + Option scheduledCompactionInstant = Option.empty(); // filter dupes if needed if (cfg.filterDupes) { @@ -330,7 +330,7 @@ public class DeltaSync implements Serializable { if (records.isEmpty()) { log.info("No new data, nothing to commit.. "); - return Optional.empty(); + return Option.empty(); } } @@ -365,13 +365,13 @@ public class DeltaSync implements Serializable { } boolean success = writeClient.commit(commitTime, writeStatusRDD, - Optional.of(checkpointCommitMetadata)); + Option.of(checkpointCommitMetadata)); if (success) { log.info("Commit " + commitTime + " successful!"); // Schedule compaction if needed if (cfg.isAsyncCompactionEnabled()) { - scheduledCompactionInstant = writeClient.scheduleCompaction(Optional.of(checkpointCommitMetadata)); + scheduledCompactionInstant = writeClient.scheduleCompaction(Option.of(checkpointCommitMetadata)); } // Sync to hive if enabled diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java index c49f21758..5de5778f6 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/HoodieDeltaStreamer.java @@ -33,6 +33,7 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.table.timeline.HoodieInstant.State; 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.TypedProperties; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.exception.HoodieException; @@ -46,7 +47,6 @@ import java.io.Serializable; import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -385,7 +385,7 @@ public class HoodieDeltaStreamer implements Serializable { try { while (!isShutdownRequested()) { try { - Optional scheduledCompactionInstant = deltaSync.syncOnce(); + Option scheduledCompactionInstant = deltaSync.syncOnce(); if (scheduledCompactionInstant.isPresent()) { log.info("Enqueuing new pending compaction instant (" + scheduledCompactionInstant + ")"); asyncCompactService.enqueuePendingCompaction(new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SchedulerConfGenerator.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SchedulerConfGenerator.java index 592e1de82..7ba6afff1 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SchedulerConfGenerator.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SchedulerConfGenerator.java @@ -18,6 +18,7 @@ package com.uber.hoodie.utilities.deltastreamer; import com.uber.hoodie.common.model.HoodieTableType; +import com.uber.hoodie.common.util.Option; import java.io.BufferedWriter; import java.io.File; import java.io.FileWriter; @@ -29,7 +30,6 @@ import org.apache.commons.lang.text.StrSubstitutor; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.SparkConf; -import scala.Option; /** * Utility Class to generate Spark Scheduling allocation file. This kicks in only when user @@ -94,10 +94,14 @@ public class SchedulerConfGenerator { * @param cfg Config */ public static Map getSparkSchedulingConfigs(HoodieDeltaStreamer.Config cfg) throws Exception { - final Option sparkSchedulerMode = new SparkConf().getOption(SPARK_SCHEDULER_MODE_KEY); + scala.Option scheduleModeKeyOption = new SparkConf().getOption(SPARK_SCHEDULER_MODE_KEY); + final Option sparkSchedulerMode = + scheduleModeKeyOption.isDefined() + ? Option.of(scheduleModeKeyOption.get()) + : Option.empty(); Map additionalSparkConfigs = new HashMap<>(); - if (sparkSchedulerMode.isDefined() && "FAIR".equals(sparkSchedulerMode.get()) + if (sparkSchedulerMode.isPresent() && "FAIR".equals(sparkSchedulerMode.get()) && cfg.continuousMode && cfg.storageType.equals(HoodieTableType.MERGE_ON_READ.name())) { String sparkSchedulingConfFile = generateAndStoreConfig(cfg.deltaSyncSchedulingWeight, cfg.compactSchedulingWeight, cfg.deltaSyncSchedulingMinShare, cfg.compactSchedulingMinShare); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SourceFormatAdapter.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SourceFormatAdapter.java index 8db116ab6..fc6901bdc 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SourceFormatAdapter.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/deltastreamer/SourceFormatAdapter.java @@ -22,13 +22,13 @@ import static com.uber.hoodie.utilities.schema.RowBasedSchemaProvider.HOODIE_REC import static com.uber.hoodie.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME; import com.uber.hoodie.AvroConversionUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.utilities.sources.AvroSource; import com.uber.hoodie.utilities.sources.InputBatch; import com.uber.hoodie.utilities.sources.JsonSource; import com.uber.hoodie.utilities.sources.RowSource; import com.uber.hoodie.utilities.sources.Source; import com.uber.hoodie.utilities.sources.helpers.AvroConvertor; -import java.util.Optional; import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; @@ -55,7 +55,7 @@ public final class SourceFormatAdapter { * @param sourceLimit * @return */ - public InputBatch> fetchNewDataInAvroFormat(Optional lastCkptStr, + public InputBatch> fetchNewDataInAvroFormat(Option lastCkptStr, long sourceLimit) { switch (source.getSourceType()) { case AVRO: @@ -63,13 +63,13 @@ public final class SourceFormatAdapter { case JSON: { InputBatch> r = ((JsonSource)source).fetchNext(lastCkptStr, sourceLimit); AvroConvertor convertor = new AvroConvertor(r.getSchemaProvider().getSourceSchema()); - return new InputBatch<>(Optional.ofNullable( + return new InputBatch<>(Option.ofNullable( r.getBatch().map(rdd -> rdd.map(convertor::fromJson)) .orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider()); } case ROW: { InputBatch> r = ((RowSource)source).fetchNext(lastCkptStr, sourceLimit); - return new InputBatch<>(Optional.ofNullable(r.getBatch().map( + return new InputBatch<>(Option.ofNullable(r.getBatch().map( rdd -> (AvroConversionUtils.createRdd(rdd, HOODIE_RECORD_STRUCT_NAME, HOODIE_RECORD_NAMESPACE).toJavaRDD())) .orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider()); } @@ -85,14 +85,14 @@ public final class SourceFormatAdapter { * @param sourceLimit * @return */ - public InputBatch> fetchNewDataInRowFormat(Optional lastCkptStr, long sourceLimit) { + public InputBatch> fetchNewDataInRowFormat(Option lastCkptStr, long sourceLimit) { switch (source.getSourceType()) { case ROW: return ((RowSource)source).fetchNext(lastCkptStr, sourceLimit); case AVRO: { InputBatch> r = ((AvroSource)source).fetchNext(lastCkptStr, sourceLimit); Schema sourceSchema = r.getSchemaProvider().getSourceSchema(); - return new InputBatch<>(Optional.ofNullable( + return new InputBatch<>(Option.ofNullable( r.getBatch().map(rdd -> AvroConversionUtils.createDataFrame(JavaRDD.toRDD(rdd), sourceSchema.toString(), source.getSparkSession())) .orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider()); @@ -101,7 +101,7 @@ public final class SourceFormatAdapter { InputBatch> r = ((JsonSource)source).fetchNext(lastCkptStr, sourceLimit); Schema sourceSchema = r.getSchemaProvider().getSourceSchema(); StructType dataType = AvroConversionUtils.convertAvroSchemaToStructType(sourceSchema); - return new InputBatch<>(Optional.ofNullable( + return new InputBatch<>(Option.ofNullable( r.getBatch().map(rdd -> source.getSparkSession().read().schema(dataType).json(rdd)) .orElse(null)), r.getCheckpointForNextBatch(), r.getSchemaProvider()); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java index 083f5a98d..7b802a12c 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroDFSSource.java @@ -18,11 +18,11 @@ package com.uber.hoodie.utilities.sources; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.sources.helpers.DFSPathSelector; -import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroKeyInputFormat; @@ -46,14 +46,14 @@ public class AvroDFSSource extends AvroSource { } @Override - protected InputBatch> fetchNewData(Optional lastCkptStr, + protected InputBatch> fetchNewData(Option lastCkptStr, long sourceLimit) { - Pair, String> selectPathsWithMaxModificationTime = + Pair, String> selectPathsWithMaxModificationTime = pathSelector.getNextFilePathsAndMaxModificationTime(lastCkptStr, sourceLimit); return selectPathsWithMaxModificationTime.getLeft().map(pathStr -> new InputBatch<>( - Optional.of(fromFiles(pathStr)), + Option.of(fromFiles(pathStr)), selectPathsWithMaxModificationTime.getRight())) - .orElseGet(() -> new InputBatch<>(Optional.empty(), selectPathsWithMaxModificationTime.getRight())); + .orElseGet(() -> new InputBatch<>(Option.empty(), selectPathsWithMaxModificationTime.getRight())); } private JavaRDD fromFiles(String pathStr) { diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java index baeaaa8d3..915030326 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/AvroKafkaSource.java @@ -18,12 +18,12 @@ package com.uber.hoodie.utilities.sources; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen; import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils; import io.confluent.kafka.serializers.KafkaAvroDecoder; -import java.util.Optional; import kafka.serializer.StringDecoder; import org.apache.avro.generic.GenericRecord; import org.apache.log4j.LogManager; @@ -50,18 +50,18 @@ public class AvroKafkaSource extends AvroSource { } @Override - protected InputBatch> fetchNewData(Optional lastCheckpointStr, + protected InputBatch> fetchNewData(Option lastCheckpointStr, long sourceLimit) { OffsetRange[] offsetRanges = offsetGen.getNextOffsetRanges(lastCheckpointStr, sourceLimit); long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges); if (totalNewMsgs <= 0) { - return new InputBatch<>(Optional.empty(), + return new InputBatch<>(Option.empty(), lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : ""); } else { log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName()); } JavaRDD newDataRDD = toRDD(offsetRanges); - return new InputBatch<>(Optional.of(newDataRDD), + return new InputBatch<>(Option.of(newDataRDD), KafkaOffsetGen.CheckpointUtils.offsetsToStr(offsetRanges)); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java index e1fdafa8e..ca681f5d4 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HiveIncrPullSource.java @@ -20,6 +20,7 @@ package com.uber.hoodie.utilities.sources; import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.exception.HoodieIOException; import com.uber.hoodie.utilities.schema.SchemaProvider; @@ -28,7 +29,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.avro.mapred.AvroKey; @@ -83,7 +83,7 @@ public class HiveIncrPullSource extends AvroSource { /** * Finds the first commit from source, greater than the target's last commit, and reads it out. */ - private Optional findCommitToPull(Optional latestTargetCommit) + private Option findCommitToPull(Option latestTargetCommit) throws IOException { log.info("Looking for commits "); @@ -99,27 +99,27 @@ public class HiveIncrPullSource extends AvroSource { if (!latestTargetCommit.isPresent()) { // start from the beginning - return Optional.of(commitTimes.get(0)); + return Option.of(commitTimes.get(0)); } for (String commitTime : commitTimes) { //TODO(vc): Add an option to delete consumed commits if (commitTime.compareTo(latestTargetCommit.get()) > 0) { - return Optional.of(commitTime); + return Option.of(commitTime); } } - return Optional.empty(); + return Option.empty(); } @Override protected InputBatch> fetchNewData( - Optional lastCheckpointStr, long sourceLimit) { + Option lastCheckpointStr, long sourceLimit) { try { // find the source commit to pull - Optional commitToPull = findCommitToPull(lastCheckpointStr); + Option commitToPull = findCommitToPull(lastCheckpointStr); if (!commitToPull.isPresent()) { - return new InputBatch<>(Optional.empty(), + return new InputBatch<>(Option.empty(), lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : ""); } @@ -131,7 +131,7 @@ public class HiveIncrPullSource extends AvroSource { JavaPairRDD avroRDD = sparkContext.newAPIHadoopFile(pathStr, AvroKeyInputFormat.class, AvroKey.class, NullWritable.class, sparkContext.hadoopConfiguration()); - return new InputBatch<>(Optional.of(avroRDD.keys().map(r -> ((GenericRecord) r.datum()))), + return new InputBatch<>(Option.of(avroRDD.keys().map(r -> ((GenericRecord) r.datum()))), String.valueOf(commitToPull.get())); } catch (IOException ioe) { throw new HoodieIOException( diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HoodieIncrSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HoodieIncrSource.java index 3f53f5550..b0fa59b72 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HoodieIncrSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/HoodieIncrSource.java @@ -21,13 +21,13 @@ package com.uber.hoodie.utilities.sources; import com.uber.hoodie.DataSourceReadOptions; import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.hive.SlashEncodedDayPartitionValueExtractor; import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.sources.helpers.IncrSourceHelper; import java.util.Arrays; -import java.util.Optional; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.DataFrameReader; import org.apache.spark.sql.Dataset; @@ -79,7 +79,7 @@ public class HoodieIncrSource extends RowSource { } @Override - public Pair>, String> fetchNextBatch(Optional lastCkptStr, long sourceLimit) { + public Pair>, String> fetchNextBatch(Option lastCkptStr, long sourceLimit) { DataSourceUtils.checkRequiredProperties(props, Arrays.asList(Config.HOODIE_SRC_BASE_PATH)); @@ -97,15 +97,15 @@ public class HoodieIncrSource extends RowSource { Config.DEFAULT_READ_LATEST_INSTANT_ON_MISSING_CKPT); // Use begin Instant if set and non-empty - Optional beginInstant = - lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Optional.empty() : lastCkptStr : Optional.empty(); + Option beginInstant = + lastCkptStr.isPresent() ? lastCkptStr.get().isEmpty() ? Option.empty() : lastCkptStr : Option.empty(); Pair instantEndpts = IncrSourceHelper.calculateBeginAndEndInstants(sparkContext, srcPath, numInstantsPerFetch, beginInstant, readLatestOnMissingCkpt); if (instantEndpts.getKey().equals(instantEndpts.getValue())) { log.warn("Already caught up. Begin Checkpoint was :" + instantEndpts.getKey()); - return Pair.of(Optional.empty(), instantEndpts.getKey()); + return Pair.of(Option.empty(), instantEndpts.getKey()); } // Do Incr pull. Set end instant if available @@ -153,6 +153,6 @@ public class HoodieIncrSource extends RowSource { final Dataset src = source.drop(HoodieRecord.HOODIE_META_COLUMNS.stream() .filter(x -> !x.equals(HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toArray(String[]::new)); //log.info("Final Schema from Source is :" + src.schema()); - return Pair.of(Optional.of(src), instantEndpts.getRight()); + return Pair.of(Option.of(src), instantEndpts.getRight()); } } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/InputBatch.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/InputBatch.java index c84083442..8625cfac3 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/InputBatch.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/InputBatch.java @@ -18,29 +18,29 @@ package com.uber.hoodie.utilities.sources; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.utilities.schema.SchemaProvider; -import java.util.Optional; public class InputBatch { - private final Optional batch; + private final Option batch; private final String checkpointForNextBatch; private final SchemaProvider schemaProvider; - public InputBatch(Optional batch, String checkpointForNextBatch, + public InputBatch(Option batch, String checkpointForNextBatch, SchemaProvider schemaProvider) { this.batch = batch; this.checkpointForNextBatch = checkpointForNextBatch; this.schemaProvider = schemaProvider; } - public InputBatch(Optional batch, String checkpointForNextBatch) { + public InputBatch(Option batch, String checkpointForNextBatch) { this.batch = batch; this.checkpointForNextBatch = checkpointForNextBatch; this.schemaProvider = null; } - public Optional getBatch() { + public Option getBatch() { return batch; } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java index 9394aef63..5f17acb75 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonDFSSource.java @@ -18,11 +18,11 @@ package com.uber.hoodie.utilities.sources; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.sources.helpers.DFSPathSelector; -import java.util.Optional; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; @@ -41,13 +41,13 @@ public class JsonDFSSource extends JsonSource { } @Override - protected InputBatch> fetchNewData(Optional lastCkptStr, + protected InputBatch> fetchNewData(Option lastCkptStr, long sourceLimit) { - Pair, String> selPathsWithMaxModificationTime = + Pair, String> selPathsWithMaxModificationTime = pathSelector.getNextFilePathsAndMaxModificationTime(lastCkptStr, sourceLimit); return selPathsWithMaxModificationTime.getLeft().map(pathStr -> new InputBatch<>( - Optional.of(fromFiles(pathStr)), selPathsWithMaxModificationTime.getRight())) - .orElse(new InputBatch<>(Optional.empty(), selPathsWithMaxModificationTime.getRight())); + Option.of(fromFiles(pathStr)), selPathsWithMaxModificationTime.getRight())) + .orElse(new InputBatch<>(Option.empty(), selPathsWithMaxModificationTime.getRight())); } private JavaRDD fromFiles(String pathStr) { diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java index fcf114fa1..665a1dccc 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/JsonKafkaSource.java @@ -18,11 +18,11 @@ package com.uber.hoodie.utilities.sources; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen; import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils; -import java.util.Optional; import kafka.serializer.StringDecoder; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -48,17 +48,17 @@ public class JsonKafkaSource extends JsonSource { } @Override - protected InputBatch> fetchNewData(Optional lastCheckpointStr, + protected InputBatch> fetchNewData(Option lastCheckpointStr, long sourceLimit) { OffsetRange[] offsetRanges = offsetGen.getNextOffsetRanges(lastCheckpointStr, sourceLimit); long totalNewMsgs = CheckpointUtils.totalNewMessages(offsetRanges); if (totalNewMsgs <= 0) { - return new InputBatch<>(Optional.empty(), + return new InputBatch<>(Option.empty(), lastCheckpointStr.isPresent() ? lastCheckpointStr.get() : ""); } log.info("About to read " + totalNewMsgs + " from Kafka for topic :" + offsetGen.getTopicName()); JavaRDD newDataRDD = toRDD(offsetRanges); - return new InputBatch<>(Optional.of(newDataRDD), CheckpointUtils.offsetsToStr(offsetRanges)); + return new InputBatch<>(Option.of(newDataRDD), CheckpointUtils.offsetsToStr(offsetRanges)); } private JavaRDD toRDD(OffsetRange[] offsetRanges) { diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/RowSource.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/RowSource.java index f406ff5f7..286943135 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/RowSource.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/RowSource.java @@ -18,11 +18,11 @@ package com.uber.hoodie.utilities.sources; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.collection.Pair; import com.uber.hoodie.utilities.schema.RowBasedSchemaProvider; import com.uber.hoodie.utilities.schema.SchemaProvider; -import java.util.Optional; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -37,12 +37,12 @@ public abstract class RowSource extends Source> { super(props, sparkContext, sparkSession, schemaProvider, SourceType.ROW); } - protected abstract Pair>, String> fetchNextBatch(Optional lastCkptStr, + protected abstract Pair>, String> fetchNextBatch(Option lastCkptStr, long sourceLimit); @Override - protected final InputBatch> fetchNewData(Optional lastCkptStr, long sourceLimit) { - Pair>, String> res = fetchNextBatch(lastCkptStr, sourceLimit); + protected final InputBatch> fetchNewData(Option lastCkptStr, long sourceLimit) { + Pair>, String> res = fetchNextBatch(lastCkptStr, sourceLimit); return res.getKey().map(dsr -> { SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(dsr.schema()); return new InputBatch<>(res.getKey(), res.getValue(), rowSchemaProvider); diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java index d2fdf2e99..1c737d72a 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/Source.java @@ -18,10 +18,10 @@ package com.uber.hoodie.utilities.sources; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.utilities.schema.SchemaProvider; import java.io.Serializable; -import java.util.Optional; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; @@ -60,7 +60,7 @@ public abstract class Source implements Serializable { this.sourceType = sourceType; } - protected abstract InputBatch fetchNewData(Optional lastCkptStr, long sourceLimit); + protected abstract InputBatch fetchNewData(Option lastCkptStr, long sourceLimit); /** * Main API called by Hoodie Delta Streamer to fetch records @@ -68,7 +68,7 @@ public abstract class Source implements Serializable { * @param sourceLimit Source Limit * @return */ - public final InputBatch fetchNext(Optional lastCkptStr, long sourceLimit) { + public final InputBatch fetchNext(Option lastCkptStr, long sourceLimit) { InputBatch batch = fetchNewData(lastCkptStr, sourceLimit); // If overriddenSchemaProvider is passed in CLI, use it return overriddenSchemaProvider == null ? batch : new InputBatch<>(batch.getBatch(), diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/DFSPathSelector.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/DFSPathSelector.java index 63cbc055e..26d2e2da1 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/DFSPathSelector.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/DFSPathSelector.java @@ -20,6 +20,7 @@ package com.uber.hoodie.utilities.sources.helpers; import com.uber.hoodie.DataSourceUtils; import com.uber.hoodie.common.util.FSUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.collection.ImmutablePair; import com.uber.hoodie.common.util.collection.Pair; @@ -55,8 +56,8 @@ public class DFSPathSelector { this.fs = FSUtils.getFs(props.getString(Config.ROOT_INPUT_PATH_PROP), hadoopConf); } - public Pair, String> getNextFilePathsAndMaxModificationTime( - Optional lastCheckpointStr, long sourceLimit) { + public Pair, String> getNextFilePathsAndMaxModificationTime( + Option lastCheckpointStr, long sourceLimit) { try { // obtain all eligible files under root folder. @@ -97,7 +98,7 @@ public class DFSPathSelector { // no data to read if (filteredFiles.size() == 0) { - return new ImmutablePair<>(Optional.empty(), + return new ImmutablePair<>(Option.empty(), lastCheckpointStr.orElseGet(() -> String.valueOf(Long.MIN_VALUE))); } @@ -106,7 +107,7 @@ public class DFSPathSelector { .collect(Collectors.joining(",")); return new ImmutablePair<>( - Optional.ofNullable(pathStr), + Option.ofNullable(pathStr), String.valueOf(maxModificationTime)); } catch (IOException ioe) { throw new HoodieIOException( diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/IncrSourceHelper.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/IncrSourceHelper.java index e34aeb19b..359c772c2 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/IncrSourceHelper.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/IncrSourceHelper.java @@ -22,8 +22,8 @@ import com.google.common.base.Preconditions; import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.collection.Pair; -import java.util.Optional; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Row; @@ -52,7 +52,7 @@ public class IncrSourceHelper { * @return begin and end instants */ public static Pair calculateBeginAndEndInstants( - JavaSparkContext jssc, String srcBasePath, int numInstantsPerFetch, Optional beginInstant, + JavaSparkContext jssc, String srcBasePath, int numInstantsPerFetch, Option beginInstant, boolean readLatestOnMissingBeginInstant) { Preconditions.checkArgument(numInstantsPerFetch > 0, "Make sure the config" + " hoodie.deltastreamer.source.hoodieincr.num_instants is set to a positive value"); @@ -64,7 +64,7 @@ public class IncrSourceHelper { String beginInstantTime = beginInstant.orElseGet(() -> { if (readLatestOnMissingBeginInstant) { - Optional lastInstant = activeCommitTimeline.lastInstant(); + Option lastInstant = activeCommitTimeline.lastInstant(); return lastInstant.map(hoodieInstant -> getStrictlyLowerTimestamp(hoodieInstant.getTimestamp())).orElse("000"); } else { throw new IllegalArgumentException("Missing begin instant for incremental pull. For reading from latest " @@ -72,8 +72,11 @@ public class IncrSourceHelper { } }); - Optional nthInstant = - activeCommitTimeline.findInstantsAfter(beginInstantTime, numInstantsPerFetch).getInstants().reduce((x, y) -> y); + Option nthInstant = Option.fromJavaOptional( + activeCommitTimeline + .findInstantsAfter(beginInstantTime, numInstantsPerFetch) + .getInstants() + .reduce((x, y) -> y)); return Pair.of(beginInstantTime, nthInstant.map(instant -> instant.getTimestamp()).orElse(beginInstantTime)); } diff --git a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/KafkaOffsetGen.java b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/KafkaOffsetGen.java index db98ef201..ae1ef2037 100644 --- a/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/KafkaOffsetGen.java +++ b/hoodie-utilities/src/main/java/com/uber/hoodie/utilities/sources/helpers/KafkaOffsetGen.java @@ -19,6 +19,7 @@ package com.uber.hoodie.utilities.sources.helpers; import com.uber.hoodie.DataSourceUtils; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.exception.HoodieNotSupportedException; import com.uber.hoodie.utilities.exception.HoodieDeltaStreamerException; @@ -27,7 +28,6 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; -import java.util.Optional; import java.util.stream.Collectors; import kafka.common.TopicAndPartition; import org.apache.log4j.LogManager; @@ -189,7 +189,7 @@ public class KafkaOffsetGen { topicName = props.getString(Config.KAFKA_TOPIC_NAME); } - public OffsetRange[] getNextOffsetRanges(Optional lastCheckpointStr, long sourceLimit) { + public OffsetRange[] getNextOffsetRanges(Option lastCheckpointStr, long sourceLimit) { // Obtain current metadata for the topic KafkaCluster cluster = new KafkaCluster(ScalaHelpers.toScalaMap(kafkaParams)); @@ -240,7 +240,7 @@ public class KafkaOffsetGen { // else return earliest offsets private HashMap checkupValidOffsets( KafkaCluster cluster, - Optional lastCheckpointStr, + Option lastCheckpointStr, Set topicPartitions) { HashMap checkpointOffsets = CheckpointUtils.strToOffsets(lastCheckpointStr.get()); diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java index 14b6c6abc..7fe187e01 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/TestHoodieDeltaStreamer.java @@ -30,6 +30,7 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient; import com.uber.hoodie.common.table.HoodieTimeline; import com.uber.hoodie.common.table.timeline.HoodieInstant; import com.uber.hoodie.common.util.DFSPropertiesConfiguration; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.config.HoodieCompactionConfig; import com.uber.hoodie.exception.DatasetNotFoundException; @@ -49,7 +50,6 @@ import com.uber.hoodie.utilities.transform.Transformer; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Optional; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -501,7 +501,7 @@ public class TestHoodieDeltaStreamer extends UtilitiesTestBase { props.setProperty(TestSourceConfig.USE_ROCKSDB_FOR_TEST_DATAGEN_KEYS, "true"); DistributedTestDataSource distributedTestDataSource = new DistributedTestDataSource(props, jsc, sparkSession, null); - InputBatch> batch = distributedTestDataSource.fetchNext(Optional.empty(), 10000000); + InputBatch> batch = distributedTestDataSource.fetchNext(Option.empty(), 10000000); batch.getBatch().get().cache(); long c = batch.getBatch().get().count(); Assert.assertEquals(1000, c); diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/AbstractBaseTestSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/AbstractBaseTestSource.java index b61cef6a0..3016c6730 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/AbstractBaseTestSource.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/AbstractBaseTestSource.java @@ -2,6 +2,7 @@ package com.uber.hoodie.utilities.sources; import com.uber.hoodie.common.HoodieTestDataGenerator; import com.uber.hoodie.common.model.HoodieRecord; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.common.util.collection.RocksDBBasedMap; import com.uber.hoodie.exception.HoodieIOException; @@ -11,7 +12,6 @@ import java.io.File; import java.io.IOException; import java.util.HashMap; import java.util.Map; -import java.util.Optional; import java.util.stream.Stream; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; @@ -96,7 +96,7 @@ public abstract class AbstractBaseTestSource extends AvroSource { private static GenericRecord toGenericRecord(HoodieRecord hoodieRecord, HoodieTestDataGenerator dataGenerator) { try { - Optional recordOpt = hoodieRecord.getData().getInsertValue(dataGenerator.avroSchema); + Option recordOpt = hoodieRecord.getData().getInsertValue(dataGenerator.avroSchema); return (GenericRecord) recordOpt.get(); } catch (IOException e) { return null; diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/DistributedTestDataSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/DistributedTestDataSource.java index 161c6234d..5e42b8a0c 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/DistributedTestDataSource.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/DistributedTestDataSource.java @@ -18,11 +18,11 @@ package com.uber.hoodie.utilities.sources; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.utilities.schema.SchemaProvider; import com.uber.hoodie.utilities.sources.config.TestSourceConfig; import java.util.Iterator; -import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.avro.generic.GenericRecord; @@ -46,14 +46,14 @@ public class DistributedTestDataSource extends AbstractBaseTestSource { } @Override - protected InputBatch> fetchNewData(Optional lastCkptStr, long sourceLimit) { + protected InputBatch> fetchNewData(Option lastCkptStr, long sourceLimit) { int nextCommitNum = lastCkptStr.map(s -> Integer.parseInt(s) + 1).orElse(0); String commitTime = String.format("%05d", nextCommitNum); log.info("Source Limit is set to " + sourceLimit); // No new data. if (sourceLimit <= 0) { - return new InputBatch<>(Optional.empty(), commitTime); + return new InputBatch<>(Option.empty(), commitTime); } TypedProperties newProps = new TypedProperties(); @@ -74,6 +74,6 @@ public class DistributedTestDataSource extends AbstractBaseTestSource { Iterator itr = fetchNextBatch(newProps, perPartitionSourceLimit, commitTime, p).iterator(); return itr; }, true); - return new InputBatch<>(Optional.of(avroRDD), commitTime); + return new InputBatch<>(Option.of(avroRDD), commitTime); } } diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java index 0ec9b7805..0d561290f 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDFSSource.java @@ -22,12 +22,12 @@ import static org.junit.Assert.assertEquals; import com.uber.hoodie.AvroConversionUtils; import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.utilities.UtilitiesTestBase; import com.uber.hoodie.utilities.deltastreamer.SourceFormatAdapter; import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider; import java.io.IOException; -import java.util.Optional; import org.apache.avro.generic.GenericRecord; import org.apache.hadoop.fs.Path; import org.apache.spark.api.java.JavaRDD; @@ -79,17 +79,17 @@ public class TestDFSSource extends UtilitiesTestBase { SourceFormatAdapter jsonSource = new SourceFormatAdapter(jsonDFSSource); // 1. Extract without any checkpoint => get all the data, respecting sourceLimit - assertEquals(Optional.empty(), jsonSource.fetchNewDataInAvroFormat(Optional.empty(), Long.MAX_VALUE).getBatch()); + assertEquals(Option.empty(), jsonSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); UtilitiesTestBase.Helpers.saveStringsToDFS( Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 100)), dfs, dfsBasePath + "/jsonFiles/1.json"); - assertEquals(Optional.empty(), jsonSource.fetchNewDataInAvroFormat(Optional.empty(), 10).getBatch()); + assertEquals(Option.empty(), jsonSource.fetchNewDataInAvroFormat(Option.empty(), 10).getBatch()); InputBatch> fetch1 = - jsonSource.fetchNewDataInAvroFormat(Optional.empty(), 1000000); + jsonSource.fetchNewDataInAvroFormat(Option.empty(), 1000000); assertEquals(100, fetch1.getBatch().get().count()); // Test json -> Row format InputBatch> fetch1AsRows = - jsonSource.fetchNewDataInRowFormat(Optional.empty(), 1000000); + jsonSource.fetchNewDataInRowFormat(Option.empty(), 1000000); assertEquals(100, fetch1AsRows.getBatch().get().count()); // Test Avro -> Row format Dataset fetch1Rows = AvroConversionUtils.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()), @@ -101,12 +101,12 @@ public class TestDFSSource extends UtilitiesTestBase { Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 10000)), dfs, dfsBasePath + "/jsonFiles/2.json"); InputBatch> fetch2 = jsonSource.fetchNewDataInRowFormat( - Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); assertEquals(10000, fetch2.getBatch().get().count()); // 3. Extract with previous checkpoint => gives same data back (idempotent) InputBatch> fetch3 = jsonSource.fetchNewDataInRowFormat( - Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); assertEquals(10000, fetch3.getBatch().get().count()); assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch()); fetch3.getBatch().get().registerTempTable("test_dfs_table"); @@ -115,7 +115,7 @@ public class TestDFSSource extends UtilitiesTestBase { // 4. Extract with latest checkpoint => no new data returned InputBatch> fetch4 = jsonSource.fetchNewDataInAvroFormat( - Optional.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); - assertEquals(Optional.empty(), fetch4.getBatch()); + Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(Option.empty(), fetch4.getBatch()); } } diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java index 99b95d310..3b356ac50 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestDataSource.java @@ -18,10 +18,10 @@ package com.uber.hoodie.utilities.sources; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.utilities.schema.SchemaProvider; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; import org.apache.avro.generic.GenericRecord; import org.apache.log4j.LogManager; @@ -44,7 +44,7 @@ public class TestDataSource extends AbstractBaseTestSource { } @Override - protected InputBatch> fetchNewData(Optional lastCheckpointStr, + protected InputBatch> fetchNewData(Option lastCheckpointStr, long sourceLimit) { int nextCommitNum = lastCheckpointStr.map(s -> Integer.parseInt(s) + 1).orElse(0); @@ -53,12 +53,12 @@ public class TestDataSource extends AbstractBaseTestSource { // No new data. if (sourceLimit <= 0) { - return new InputBatch<>(Optional.empty(), commitTime); + return new InputBatch<>(Option.empty(), commitTime); } List records = fetchNextBatch(props, (int)sourceLimit, commitTime, DEFAULT_PARTITION_NUM) .collect(Collectors.toList()); JavaRDD avroRDD = sparkContext.parallelize(records, 4); - return new InputBatch<>(Optional.of(avroRDD), commitTime); + return new InputBatch<>(Option.of(avroRDD), commitTime); } } diff --git a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java index fb2a3d058..256a58934 100644 --- a/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java +++ b/hoodie-utilities/src/test/java/com/uber/hoodie/utilities/sources/TestKafkaSource.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertEquals; import com.uber.hoodie.AvroConversionUtils; import com.uber.hoodie.common.HoodieTestDataGenerator; +import com.uber.hoodie.common.util.Option; import com.uber.hoodie.common.util.TypedProperties; import com.uber.hoodie.utilities.UtilitiesTestBase; import com.uber.hoodie.utilities.deltastreamer.SourceFormatAdapter; @@ -29,7 +30,6 @@ import com.uber.hoodie.utilities.schema.FilebasedSchemaProvider; import com.uber.hoodie.utilities.sources.helpers.KafkaOffsetGen.CheckpointUtils; import java.io.IOException; import java.util.HashMap; -import java.util.Optional; import kafka.common.TopicAndPartition; import org.apache.avro.generic.GenericRecord; import org.apache.spark.api.java.JavaRDD; @@ -96,9 +96,9 @@ public class TestKafkaSource extends UtilitiesTestBase { SourceFormatAdapter kafkaSource = new SourceFormatAdapter(jsonSource); // 1. Extract without any checkpoint => get all the data, respecting sourceLimit - assertEquals(Optional.empty(), kafkaSource.fetchNewDataInAvroFormat(Optional.empty(), Long.MAX_VALUE).getBatch()); + assertEquals(Option.empty(), kafkaSource.fetchNewDataInAvroFormat(Option.empty(), Long.MAX_VALUE).getBatch()); testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("000", 1000))); - InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Optional.empty(), 900); + InputBatch> fetch1 = kafkaSource.fetchNewDataInAvroFormat(Option.empty(), 900); assertEquals(900, fetch1.getBatch().get().count()); // Test Avro To DataFrame path Dataset fetch1AsRows = AvroConversionUtils.createDataFrame(JavaRDD.toRDD(fetch1.getBatch().get()), @@ -108,28 +108,28 @@ public class TestKafkaSource extends UtilitiesTestBase { // 2. Produce new data, extract new data testUtils.sendMessages(TEST_TOPIC_NAME, Helpers.jsonifyRecords(dataGenerator.generateInserts("001", 1000))); InputBatch> fetch2 = kafkaSource.fetchNewDataInRowFormat( - Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); assertEquals(1100, fetch2.getBatch().get().count()); // 3. Extract with previous checkpoint => gives same data back (idempotent) InputBatch> fetch3 = kafkaSource.fetchNewDataInAvroFormat( - Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); assertEquals(fetch2.getBatch().get().count(), fetch3.getBatch().get().count()); assertEquals(fetch2.getCheckpointForNextBatch(), fetch3.getCheckpointForNextBatch()); // Same using Row API InputBatch> fetch3AsRows = - kafkaSource.fetchNewDataInRowFormat(Optional.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch1.getCheckpointForNextBatch()), Long.MAX_VALUE); assertEquals(fetch2.getBatch().get().count(), fetch3AsRows.getBatch().get().count()); assertEquals(fetch2.getCheckpointForNextBatch(), fetch3AsRows.getCheckpointForNextBatch()); // 4. Extract with latest checkpoint => no new data returned InputBatch> fetch4 = kafkaSource.fetchNewDataInAvroFormat( - Optional.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); - assertEquals(Optional.empty(), fetch4.getBatch()); + Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(Option.empty(), fetch4.getBatch()); // Same using Row API InputBatch> fetch4AsRows = - kafkaSource.fetchNewDataInRowFormat(Optional.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); - assertEquals(Optional.empty(), fetch4AsRows.getBatch()); + kafkaSource.fetchNewDataInRowFormat(Option.of(fetch2.getCheckpointForNextBatch()), Long.MAX_VALUE); + assertEquals(Option.empty(), fetch4AsRows.getBatch()); }