[HUDI-153] Use com.uber.hoodie.common.util.Option instead of Java and Guava Optional
This commit is contained in:
committed by
Balaji Varadarajan
parent
d288e32833
commit
722b6be04a
@@ -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<EmbeddedTimelineService> timelineServer;
|
||||
private transient Option<EmbeddedTimelineService> 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<EmbeddedTimelineService> timelineServer) {
|
||||
Option<EmbeddedTimelineService> 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<EmbeddedTimelineService> getTimelineServer() {
|
||||
public Option<EmbeddedTimelineService> getTimelineServer() {
|
||||
return timelineServer;
|
||||
}
|
||||
|
||||
|
||||
@@ -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<EmbeddedTimelineService> timelineServer) {
|
||||
Option<EmbeddedTimelineService> 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<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
|
||||
getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, parallelism,
|
||||
Optional.absent(), skipValidation);
|
||||
Option.empty(), skipValidation);
|
||||
|
||||
List<RenameOpResult> res =
|
||||
runRenamingOps(metaClient, renameActions, parallelism, dryRun);
|
||||
|
||||
java.util.Optional<Boolean> success =
|
||||
res.stream().map(r -> (r.isExecuted() && r.isSuccess())).reduce(Boolean::logicalAnd);
|
||||
Optional<Boolean> allSuccess = success.isPresent() ? Optional.of(success.get()) : Optional.absent();
|
||||
Option<Boolean> success =
|
||||
Option.fromJavaOptional(res.stream().map(r -> (r.isExecuted() && r.isSuccess())).reduce(Boolean::logicalAnd));
|
||||
Option<Boolean> 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<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
|
||||
getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fgId,
|
||||
Optional.absent(), skipValidation);
|
||||
Option.empty(), skipValidation);
|
||||
|
||||
List<RenameOpResult> res = runRenamingOps(metaClient, renameActions, 1, dryRun);
|
||||
|
||||
@@ -211,7 +211,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
metaClient.getCommitsAndCompactionTimeline());
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> 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<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsToAlignWithCompactionOperation(
|
||||
HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation op,
|
||||
Optional<HoodieTableFileSystemView> fsViewOpt) {
|
||||
Option<HoodieTableFileSystemView> 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<HoodieTableFileSystemView> fsViewOpt)
|
||||
String compactionInstant, CompactionOperation operation, Option<HoodieTableFileSystemView> fsViewOpt)
|
||||
throws IOException {
|
||||
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() :
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
java.util.Optional<HoodieInstant> lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant();
|
||||
Option<HoodieInstant> lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant();
|
||||
try {
|
||||
if (lastInstant.isPresent()) {
|
||||
java.util.Optional<FileSlice> fileSliceOptional =
|
||||
fileSystemView.getLatestUnCompactedFileSlices(operation.getPartitionPath())
|
||||
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst();
|
||||
Option<FileSlice> 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<HoodieDataFile> df = fs.getDataFile();
|
||||
Option<HoodieDataFile> 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<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionPlan(
|
||||
HoodieTableMetaClient metaClient, String compactionInstant, int parallelism,
|
||||
Optional<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
Option<HoodieTableFileSystemView> 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<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionOperation(
|
||||
HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation operation,
|
||||
Optional<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
Option<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> 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<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionForFileId(
|
||||
HoodieTableMetaClient metaClient, HoodieFileGroupId fgId,
|
||||
Optional<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
Option<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> allPendingCompactions =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
if (allPendingCompactions.containsKey(fgId)) {
|
||||
@@ -500,14 +500,14 @@ public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
}
|
||||
|
||||
public RenameOpResult(Pair<HoodieLogFile, HoodieLogFile> op, boolean success,
|
||||
Optional<Exception> exception) {
|
||||
Option<Exception> exception) {
|
||||
super(new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(),
|
||||
op.getRight().getPath().toString()), success, exception);
|
||||
}
|
||||
|
||||
public RenameOpResult(
|
||||
Pair<HoodieLogFile, HoodieLogFile> op, boolean executed, boolean success,
|
||||
Optional<Exception> exception) {
|
||||
Option<Exception> 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> exception) {
|
||||
CompactionOperation operation, boolean success, Option<Exception> exception) {
|
||||
super(operation, success, exception);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends AbstractHoo
|
||||
private final transient HoodieIndex<T> index;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTable hoodieTable;
|
||||
private transient Optional<SQLContext> sqlContextOpt;
|
||||
private transient Option<SQLContext> sqlContextOpt;
|
||||
|
||||
/**
|
||||
* @param basePath path to Hoodie dataset
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath,
|
||||
java.util.Optional<EmbeddedTimelineService> timelineService) {
|
||||
Option<EmbeddedTimelineService> timelineService) {
|
||||
this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
// by default we use HoodieBloomIndex
|
||||
.withIndexConfig(
|
||||
@@ -83,7 +83,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> 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<EmbeddedTimelineService> timelineService) {
|
||||
Option<EmbeddedTimelineService> 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<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> extends AbstractHoo
|
||||
}
|
||||
}
|
||||
|
||||
private Optional<String> convertToDataFilePath(Optional<Pair<String, String>> partitionPathFileIDPair) {
|
||||
private Option<String> convertToDataFilePath(Option<Pair<String, String>> 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<T extends HoodieRecordPayload> extends AbstractHoo
|
||||
*/
|
||||
public Dataset<Row> readROView(JavaRDD<HoodieKey> hoodieKeys, int parallelism) {
|
||||
assertSqlContext();
|
||||
JavaPairRDD<HoodieKey, Optional<Pair<String, String>>> lookupResultRDD = index
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> lookupResultRDD = index
|
||||
.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
|
||||
JavaPairRDD<HoodieKey, Optional<String>> keyToFileRDD = lookupResultRDD
|
||||
JavaPairRDD<HoodieKey, Option<String>> keyToFileRDD = lookupResultRDD
|
||||
.mapToPair(r -> new Tuple2<>(r._1, convertToDataFilePath(r._2)));
|
||||
List<String> paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent())
|
||||
.map(keyFileTuple -> keyFileTuple._2().get()).collect();
|
||||
@@ -176,12 +176,12 @@ public class HoodieReadClient<T extends HoodieRecordPayload> 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<HoodieKey, Optional<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
|
||||
public JavaPairRDD<HoodieKey, Option<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
|
||||
return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
|
||||
}
|
||||
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> 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<EmbeddedTimelineService> timelineService) {
|
||||
boolean rollbackInFlight, HoodieIndex index, Option<EmbeddedTimelineService> timelineService) {
|
||||
super(jsc, clientConfig, timelineService);
|
||||
this.index = index;
|
||||
this.metrics = new HoodieMetrics(config, config.getTableName());
|
||||
@@ -343,7 +342,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
Option<UserDefinedBulkInsertPartitioner> bulkInsertPartitioner) {
|
||||
final JavaRDD<HoodieRecord<T>> 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<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
});
|
||||
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
Optional<HoodieInstant> instant =
|
||||
Option<HoodieInstant> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
private JavaRDD<HoodieRecord<T>> partition(JavaRDD<HoodieRecord<T>> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* Commit changes performed at the given commitTime marker
|
||||
*/
|
||||
public boolean commit(String commitTime, JavaRDD<WriteStatus> 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<WriteStatus> writeStatuses,
|
||||
Optional<Map<String, String>> extraMetadata) {
|
||||
Option<Map<String, String>> extraMetadata) {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(false);
|
||||
return commit(commitTime, writeStatuses, extraMetadata, metaClient.getCommitActionType());
|
||||
}
|
||||
|
||||
private boolean commit(String commitTime, JavaRDD<WriteStatus> writeStatuses,
|
||||
Optional<Map<String, String>> extraMetadata, String actionType) {
|
||||
Option<Map<String, String>> 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<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
throw new UnsupportedOperationException("Savepointing is not supported or MergeOnRead table types");
|
||||
}
|
||||
Optional<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
|
||||
Option<HoodieInstant> cleanInstant = table.getCompletedCleanTimeline().lastInstant();
|
||||
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION,
|
||||
commitTime);
|
||||
@@ -757,7 +756,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
restoreToInstant(savepointTime);
|
||||
|
||||
// Make sure the rollback was successful
|
||||
Optional<HoodieInstant> lastInstant = activeTimeline.reload().getCommitsAndCompactionTimeline()
|
||||
Option<HoodieInstant> lastInstant = activeTimeline.reload().getCommitsAndCompactionTimeline()
|
||||
.filterCompletedAndCompactionInstants().lastInstant();
|
||||
Preconditions.checkArgument(lastInstant.isPresent());
|
||||
Preconditions.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime),
|
||||
@@ -899,10 +898,10 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
List<String> commitsToRollback, final String startRollbackTime) throws IOException {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
createMetaClient(true), config, jsc);
|
||||
Optional<Long> durationInMs = Optional.empty();
|
||||
Option<Long> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
List<String> commitsToRollback, final String startRestoreTime, final String restoreToInstant) throws IOException {
|
||||
HoodieTable<T> table = HoodieTable.getHoodieTable(
|
||||
createMetaClient(true), config, jsc);
|
||||
Optional<Long> durationInMs = Optional.empty();
|
||||
Option<Long> durationInMs = Option.empty();
|
||||
Long numFilesDeleted = 0L;
|
||||
for (Map.Entry<String, List<HoodieRollbackStat>> commitToStat : commitToStats.entrySet()) {
|
||||
List<HoodieRollbackStat> stats = commitToStat.getValue();
|
||||
@@ -933,7 +932,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
|
||||
// Emit metrics (duration, numFilesDeleted) if needed
|
||||
Optional<Long> durationInMs = Optional.empty();
|
||||
Option<Long> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
/**
|
||||
* Schedules a new compaction instant
|
||||
*/
|
||||
public Optional<String> scheduleCompaction(Optional<Map<String, String>> extraMetadata) throws IOException {
|
||||
public Option<String> scheduleCompaction(Option<Map<String, String>> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* @param instantTime Compaction Instant Time
|
||||
* @param extraMetadata Extra Metadata to be stored
|
||||
*/
|
||||
public boolean scheduleCompactionAtInstant(String instantTime, Optional<Map<String, String>> extraMetadata)
|
||||
public boolean scheduleCompactionAtInstant(String instantTime, Option<Map<String, String>> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* Commit a compaction operation. Allow passing additional meta-data to be stored in commit instant file.
|
||||
*/
|
||||
public void commitCompaction(String compactionInstantTime, JavaRDD<WriteStatus> writeStatuses,
|
||||
Optional<Map<String, String>> extraMetadata) throws IOException {
|
||||
Option<Map<String, String>> extraMetadata) throws IOException {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
HoodieTable<T> 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<Map<String, String>> mergedMetaData = extraMetadata.map(m -> {
|
||||
Option<Map<String, String>> mergedMetaData = extraMetadata.map(m -> {
|
||||
Map<String, String> merged = new HashMap<>();
|
||||
Map<String, String> extraMetaDataFromInstantFile = compactionPlan.getExtraMetadata();
|
||||
if (extraMetaDataFromInstantFile != null) {
|
||||
@@ -1145,8 +1144,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
* @param extraMetadata Extra Metadata to store
|
||||
*/
|
||||
protected void commitCompaction(JavaRDD<WriteStatus> compactedStatuses, HoodieTable<T> table,
|
||||
String compactionCommitTime, boolean autoCommit, Optional<Map<String, String>> extraMetadata) {
|
||||
String compactionCommitTime, boolean autoCommit, Option<Map<String, String>> extraMetadata) {
|
||||
if (autoCommit) {
|
||||
HoodieCommitMetadata metadata =
|
||||
doCompactionCommit(table, compactedStatuses, compactionCommitTime, extraMetadata);
|
||||
@@ -1295,7 +1294,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
|
||||
final Timer.Context finalizeCtx = metrics.getFinalizeCtx();
|
||||
table.finalizeWrite(jsc, instantTime, stats);
|
||||
if (finalizeCtx != null) {
|
||||
Optional<Long> durationInMs = Optional.of(metrics.getDurationInMs(finalizeCtx.stop()));
|
||||
Option<Long> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
}
|
||||
|
||||
private HoodieCommitMetadata doCompactionCommit(HoodieTable<T> table, JavaRDD<WriteStatus> writeStatuses,
|
||||
String compactionCommitTime, Optional<Map<String, String>> extraMetadata) {
|
||||
String compactionCommitTime, Option<Map<String, String>> extraMetadata) {
|
||||
HoodieTableMetaClient metaClient = table.getMetaClient();
|
||||
List<HoodieWriteStat> updateStatusMap = writeStatuses.map(WriteStatus::getStat)
|
||||
.collect();
|
||||
@@ -1347,7 +1346,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
/**
|
||||
* Performs a compaction operation on a dataset, serially before or after an insert/upsert action.
|
||||
*/
|
||||
private Optional<String> forceCompact(Optional<Map<String, String>> extraMetadata) throws IOException {
|
||||
Optional<String> compactionInstantTimeOpt = scheduleCompaction(extraMetadata);
|
||||
private Option<String> forceCompact(Option<Map<String, String>> extraMetadata) throws IOException {
|
||||
Option<String> 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<T extends HoodieRecordPayload> extends AbstractHo
|
||||
rollingStatMetadata.addRollingStat(partitionPath, hoodieRollingStat);
|
||||
}
|
||||
// The last rolling stat should be present in the completed timeline
|
||||
Optional<HoodieInstant> lastInstant = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
|
||||
Option<HoodieInstant> lastInstant = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants()
|
||||
.lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(table.getActiveTimeline().getInstantDetails(lastInstant
|
||||
.get()).get(), HoodieCommitMetadata.class);
|
||||
Optional<String> lastRollingStat = Optional.ofNullable(commitMetadata.getExtraMetadata()
|
||||
Option<String> lastRollingStat = Option.ofNullable(commitMetadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY));
|
||||
if (lastRollingStat.isPresent()) {
|
||||
rollingStatMetadata = rollingStatMetadata
|
||||
|
||||
@@ -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<Map<String, String>> optionalRecordMetadata) {
|
||||
public void markSuccess(HoodieRecord record, Option<Map<String, String>> 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<Map<String, String>> optionalRecordMetadata) {
|
||||
public void markFailure(HoodieRecord record, Throwable t, Option<Map<String, String>> optionalRecordMetadata) {
|
||||
if (failedRecords.isEmpty() || (random.nextDouble() <= failureFraction)) {
|
||||
// Guaranteed to have at-least one error
|
||||
failedRecords.add(record);
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extend
|
||||
// Used for caching HoodieRecord along with insertValue. We need this to offload computation work to buffering thread.
|
||||
static class HoodieInsertValueGenResult<T extends HoodieRecord> {
|
||||
public T record;
|
||||
public Optional<IndexedRecord> insertValue;
|
||||
public Option<IndexedRecord> insertValue;
|
||||
// It caches the exception seen while fetching insert value.
|
||||
public Optional<Exception> exception = Optional.empty();
|
||||
public Option<Exception> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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<T> implements Serializable {
|
||||
private T operation;
|
||||
private boolean executed;
|
||||
private boolean success;
|
||||
private Optional<Exception> exception;
|
||||
private Option<Exception> exception;
|
||||
|
||||
public OperationResult() {
|
||||
}
|
||||
|
||||
public OperationResult(T operation, boolean success, Optional<Exception> exception) {
|
||||
public OperationResult(T operation, boolean success, Option<Exception> exception) {
|
||||
this.operation = operation;
|
||||
this.success = success;
|
||||
this.exception = exception;
|
||||
this.executed = true;
|
||||
}
|
||||
|
||||
public OperationResult(T operation, boolean executed, boolean success, Optional<Exception> exception) {
|
||||
public OperationResult(T operation, boolean executed, boolean success, Option<Exception> exception) {
|
||||
this.operation = operation;
|
||||
this.success = success;
|
||||
this.exception = exception;
|
||||
@@ -61,7 +61,7 @@ public class OperationResult<T> implements Serializable {
|
||||
return executed;
|
||||
}
|
||||
|
||||
public Optional<Exception> getException() {
|
||||
public Option<Exception> getException() {
|
||||
return exception;
|
||||
}
|
||||
|
||||
|
||||
@@ -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<I, O, E> extends BoundedInMemoryExecut
|
||||
BoundedInMemoryQueueConsumer<O, E> consumer,
|
||||
Function<I, O> bufferedIteratorTransform) {
|
||||
super(hoodieConfig.getWriteBufferLimitBytes(), producer,
|
||||
Optional.of(consumer), bufferedIteratorTransform);
|
||||
Option.of(consumer), bufferedIteratorTransform);
|
||||
this.sparkThreadTaskContext = TaskContext.get();
|
||||
}
|
||||
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> 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<HoodieKey, Optional<Pair<String, String>>> fetchRecordLocation(
|
||||
public abstract JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final JavaSparkContext jsc, HoodieTable<T> hoodieTable);
|
||||
|
||||
/**
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends HoodieInde
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
|
||||
throw new UnsupportedOperationException("InMemory index does not implement check exist yet");
|
||||
}
|
||||
@@ -76,7 +76,7 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
|
||||
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(record.getKey())) {
|
||||
HoodieKey key = record.getKey();
|
||||
java.util.Optional<HoodieRecordLocation> newLocation = record.getNewLocation();
|
||||
Option<HoodieRecordLocation> newLocation = record.getNewLocation();
|
||||
if (newLocation.isPresent()) {
|
||||
recordLocationMap.put(key, newLocation.get());
|
||||
} else {
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
* @param hoodieTable hoodie table object
|
||||
*/
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = hoodieKeys
|
||||
.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
|
||||
@@ -126,11 +126,11 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
JavaPairRDD<HoodieKey, String> keyHoodieKeyPairRDD = hoodieKeys.mapToPair(key -> new Tuple2<>(key, null));
|
||||
|
||||
return keyHoodieKeyPairRDD.leftOuterJoin(recordKeyLocationRDD).mapToPair(keyLoc -> {
|
||||
Optional<Pair<String, String>> partitionPathFileidPair;
|
||||
Option<Pair<String, String>> 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<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
List<Pair<String, String>> partitionPathFileIDList = jsc
|
||||
.parallelize(partitions, Math.max(partitions.size(), 1))
|
||||
.flatMap(partitionPath -> {
|
||||
java.util.Optional<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
|
||||
Option<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
List<Pair<String, String>> filteredFiles = new ArrayList<>();
|
||||
if (latestCommitTime.isPresent()) {
|
||||
@@ -361,7 +361,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
|
||||
HoodieRecord<T> getTaggedRecord(HoodieRecord<T> inputRecord,
|
||||
org.apache.spark.api.java.Optional<HoodieRecordLocation> location) {
|
||||
Option<HoodieRecordLocation> location) {
|
||||
HoodieRecord<T> 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<T extends HoodieRecordPayload> 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
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> 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
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaPairRDD<HoodieKey, Optional<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
|
||||
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
|
||||
throw new UnsupportedOperationException("HBase index does not implement check exist");
|
||||
}
|
||||
@@ -301,7 +301,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
try {
|
||||
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(rec.getKey())) {
|
||||
java.util.Optional<HoodieRecordLocation> loc = rec.getNewLocation();
|
||||
Option<HoodieRecordLocation> loc = rec.getNewLocation();
|
||||
if (loc.isPresent()) {
|
||||
if (rec.getCurrentLocation() != null) {
|
||||
// This is an update, no need to update index
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends HoodieWri
|
||||
}
|
||||
}
|
||||
|
||||
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
private Option<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
Option recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
Optional<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(originalSchema);
|
||||
Option<IndexedRecord> 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<T extends HoodieRecordPayload> 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<T extends HoodieRecordPayload> extends HoodieWri
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(HoodieRecord record, Optional<IndexedRecord> insertValue) {
|
||||
Optional recordMetadata = record.getData().getMetadata();
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> insertValue) {
|
||||
Option recordMetadata = record.getData().getMetadata();
|
||||
try {
|
||||
init(record);
|
||||
flushToDiskIfRequired(record);
|
||||
@@ -278,7 +277,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
|
||||
|
||||
private Writer createLogWriter(Option<FileSlice> fileSlice, String baseCommitTime)
|
||||
throws IOException, InterruptedException {
|
||||
Optional<HoodieLogFile> latestLogFile = fileSlice.get().getLatestLogFile();
|
||||
Option<HoodieLogFile> latestLogFile = fileSlice.get().getLatestLogFile();
|
||||
|
||||
return HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(FSUtils.getPartitionPath(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
@@ -294,7 +293,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
|
||||
private void writeToBuffer(HoodieRecord<T> record) {
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
|
||||
Optional<IndexedRecord> indexedRecord = getIndexedRecord(record);
|
||||
Option<IndexedRecord> indexedRecord = getIndexedRecord(record);
|
||||
if (indexedRecord.isPresent()) {
|
||||
recordList.add(indexedRecord.get());
|
||||
} else {
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload<T>> {
|
||||
while (fileSliceIterator.hasNext() && keepVersions > 0) {
|
||||
// Skip this most recent version
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
Optional<HoodieDataFile> dataFile = nextSlice.getDataFile();
|
||||
Option<HoodieDataFile> 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<T extends HoodieRecordPayload<T>> {
|
||||
// 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<HoodieDataFile> aFile = aSlice.getDataFile();
|
||||
Option<HoodieDataFile> 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<T extends HoodieRecordPayload<T>> {
|
||||
/**
|
||||
* Returns earliest commit to retain based on cleaning policy.
|
||||
*/
|
||||
public Optional<HoodieInstant> getEarliestCommitToRetain() {
|
||||
Optional<HoodieInstant> earliestCommitToRetain = Optional.empty();
|
||||
public Option<HoodieInstant> getEarliestCommitToRetain() {
|
||||
Option<HoodieInstant> earliestCommitToRetain = Option.empty();
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
&& commitTimeline.countInstants() > commitsRetained) {
|
||||
|
||||
@@ -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<HoodieInstant> oldestPendingCompactionInstant =
|
||||
Option<HoodieInstant> 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<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
|
||||
Option<HoodieInstant> 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<HoodieInstant> latestCommitted =
|
||||
archivedInstants.stream()
|
||||
Option<HoodieInstant> 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());
|
||||
}
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends HoodieWri
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Optional<IndexedRecord> avroRecord) {
|
||||
Optional recordMetadata = record.getData().getMetadata();
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> avroRecord) {
|
||||
Option recordMetadata = record.getData().getMetadata();
|
||||
try {
|
||||
if (avroRecord.isPresent()) {
|
||||
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends HoodieWrit
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Optional<IndexedRecord> insertValue) {
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> insertValue) {
|
||||
// NO_OP
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Optional<IndexedRecord> avroRecord, Optional<Exception> exception) {
|
||||
Optional recordMetadata = record.getData().getMetadata();
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> avroRecord, Option<Exception> 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<T extends HoodieRecordPayload> extends HoodieWrit
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Optional<IndexedRecord> indexedRecord) {
|
||||
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
|
||||
if (indexedRecord.isPresent()) {
|
||||
updatedRecordsWritten++;
|
||||
}
|
||||
return writeRecord(hoodieRecord, indexedRecord);
|
||||
}
|
||||
|
||||
private boolean writeRecord(HoodieRecord<T> hoodieRecord, Optional<IndexedRecord> indexedRecord) {
|
||||
Optional recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
private boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> 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<T extends HoodieRecordPayload> extends HoodieWrit
|
||||
// writing the first record. So make a copy of the record to be merged
|
||||
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key));
|
||||
try {
|
||||
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
|
||||
Option<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
|
||||
.combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchema : originalSchema);
|
||||
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
||||
/* ONLY WHEN
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends H
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Optional<IndexedRecord> insertValue) {
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> insertValue) {
|
||||
// NO_OP
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
public void write(HoodieRecord record, Optional<IndexedRecord> avroRecord, Optional<Exception> exception) {
|
||||
Optional recordMetadata = record.getData().getMetadata();
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> avroRecord, Option<Exception> 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);
|
||||
|
||||
@@ -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<HoodieDataFile> dataFile = s.getDataFile();
|
||||
Option<HoodieDataFile> dataFile = s.getDataFile();
|
||||
return new CompactionOperation(dataFile, partitionPath, logFiles,
|
||||
config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
|
||||
})
|
||||
|
||||
@@ -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<String, Double> captureMetrics(HoodieWriteConfig writeConfig, Optional<HoodieDataFile> dataFile,
|
||||
public Map<String, Double> captureMetrics(HoodieWriteConfig writeConfig, Option<HoodieDataFile> dataFile,
|
||||
String partitionPath, List<HoodieLogFile> logFiles) {
|
||||
Map<String, Double> metrics = Maps.newHashMap();
|
||||
Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize();
|
||||
|
||||
@@ -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<String, Double> captureMetrics(HoodieWriteConfig config, Optional<HoodieDataFile> dataFile,
|
||||
public Map<String, Double> captureMetrics(HoodieWriteConfig config, Option<HoodieDataFile> dataFile,
|
||||
String partitionPath, List<HoodieLogFile> logFiles) {
|
||||
Map<String, Double> metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles);
|
||||
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends Hoodi
|
||||
public int getPartition(Object key) {
|
||||
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation = (Tuple2<HoodieKey,
|
||||
Option<HoodieRecordLocation>>) key;
|
||||
if (keyLocation._2().isDefined()) {
|
||||
if (keyLocation._2().isPresent()) {
|
||||
HoodieRecordLocation location = keyLocation._2().get();
|
||||
return updateLocationToBucket.get(location.getFileId());
|
||||
} else {
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> extends
|
||||
@Override
|
||||
public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime) {
|
||||
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||
Optional<HoodieInstant> lastCompaction = getActiveTimeline().getCommitTimeline()
|
||||
Option<HoodieInstant> lastCompaction = getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
String deltaCommitsSinceTs = "0";
|
||||
if (lastCompaction.isPresent()) {
|
||||
@@ -186,11 +186,11 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> 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<HoodieInstant> commitOrCompactionOption = this.getActiveTimeline()
|
||||
Option<HoodieInstant> 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<T extends HoodieRecordPayload> extends
|
||||
@Override
|
||||
protected HoodieRollingStatMetadata getRollingStats() {
|
||||
try {
|
||||
Optional<HoodieInstant> lastInstant = this.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants()
|
||||
Option<HoodieInstant> lastInstant = this.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants()
|
||||
.lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
this.getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class);
|
||||
Optional<String> lastRollingStat = Optional.ofNullable(commitMetadata.getExtraMetadata()
|
||||
Option<String> lastRollingStat = Option.ofNullable(commitMetadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY));
|
||||
if (lastRollingStat.isPresent()) {
|
||||
return HoodieCommitMetadata
|
||||
@@ -383,13 +383,13 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> 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<FileSlice> smallFileSlice = getRTFileSystemView()
|
||||
Option<FileSlice> 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());
|
||||
}
|
||||
|
||||
@@ -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<T extends HoodieRecordPayload> implements Serializa
|
||||
|
||||
Map<Tuple2<String, Option<HoodieRecordLocation>>, 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<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts
|
||||
@@ -75,7 +75,7 @@ public class WorkloadProfile<T extends HoodieRecordPayload> 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);
|
||||
|
||||
Reference in New Issue
Block a user