HUDI-123 Rename code packages/constants to org.apache.hudi (#830)
- Rename com.uber.hoodie to org.apache.hudi - Flag to pass com.uber.hoodie Input formats for hoodie-sync - Works with HUDI demo. - Also tested for backwards compatibility with datasets built by com.uber.hoodie packages - Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
committed by
vinoth chandar
parent
722b6be04a
commit
a4f9d7575f
@@ -0,0 +1,126 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.client.utils.ClientUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Abstract class taking care of holding common member variables (FileSystem, SparkContext, HoodieConfigs)
|
||||
* Also, manages embedded timeline-server if enabled.
|
||||
*/
|
||||
public abstract class AbstractHoodieClient implements Serializable {
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(AbstractHoodieClient.class);
|
||||
|
||||
protected final transient FileSystem fs;
|
||||
protected final transient JavaSparkContext jsc;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final String basePath;
|
||||
|
||||
/**
|
||||
* Timeline Server has the same lifetime as that of Client.
|
||||
* Any operations done on the same timeline service will be able to take advantage
|
||||
* of the cached file-system view. New completed actions will be synced automatically
|
||||
* in an incremental fashion.
|
||||
*/
|
||||
private transient Option<EmbeddedTimelineService> timelineServer;
|
||||
private final boolean shouldStopTimelineServer;
|
||||
|
||||
protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
|
||||
this(jsc, clientConfig, Option.empty());
|
||||
}
|
||||
|
||||
protected AbstractHoodieClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
|
||||
Option<EmbeddedTimelineService> timelineServer) {
|
||||
this.fs = FSUtils.getFs(clientConfig.getBasePath(), jsc.hadoopConfiguration());
|
||||
this.jsc = jsc;
|
||||
this.basePath = clientConfig.getBasePath();
|
||||
this.config = clientConfig;
|
||||
this.timelineServer = timelineServer;
|
||||
shouldStopTimelineServer = !timelineServer.isPresent();
|
||||
startEmbeddedServerView();
|
||||
}
|
||||
|
||||
/**
|
||||
* Releases any resources used by the client.
|
||||
*/
|
||||
public void close() {
|
||||
stopEmbeddedServerView(true);
|
||||
}
|
||||
|
||||
private synchronized void stopEmbeddedServerView(boolean resetViewStorageConfig) {
|
||||
if (timelineServer.isPresent() && shouldStopTimelineServer) {
|
||||
// Stop only if owner
|
||||
logger.info("Stopping Timeline service !!");
|
||||
timelineServer.get().stop();
|
||||
}
|
||||
|
||||
timelineServer = Option.empty();
|
||||
// Reset Storage Config to Client specified config
|
||||
if (resetViewStorageConfig) {
|
||||
config.resetViewStorageConfig();
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized void startEmbeddedServerView() {
|
||||
if (config.isEmbeddedTimelineServerEnabled()) {
|
||||
if (!timelineServer.isPresent()) {
|
||||
// Run Embedded Timeline Server
|
||||
logger.info("Starting Timeline service !!");
|
||||
timelineServer = Option.of(new EmbeddedTimelineService(jsc.hadoopConfiguration(), jsc.getConf(),
|
||||
config.getClientSpecifiedViewStorageConfig()));
|
||||
try {
|
||||
timelineServer.get().startServer();
|
||||
// Allow executor to find this newly instantiated timeline service
|
||||
config.setViewStorageConfig(timelineServer.get().getRemoteFileSystemViewConfig());
|
||||
} catch (IOException e) {
|
||||
logger.warn("Unable to start timeline service. Proceeding as if embedded server is disabled", e);
|
||||
stopEmbeddedServerView(false);
|
||||
}
|
||||
} else {
|
||||
logger.info("Timeline Server already running. Not restarting the service");
|
||||
}
|
||||
} else {
|
||||
logger.info("Embedded Timeline Server is disabled. Not starting timeline service");
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieWriteConfig getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
public Option<EmbeddedTimelineService> getTimelineServer() {
|
||||
return timelineServer;
|
||||
}
|
||||
|
||||
protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoad) {
|
||||
return ClientUtils.createMetaClient(jsc, config, loadActiveTimelineOnLoad);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,552 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import static org.apache.hudi.common.table.HoodieTimeline.COMPACTION_ACTION;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.func.OperationResult;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Client to perform admin operations related to compaction
|
||||
*/
|
||||
public class CompactionAdminClient extends AbstractHoodieClient {
|
||||
|
||||
private static Logger log = LogManager.getLogger(CompactionAdminClient.class);
|
||||
|
||||
public CompactionAdminClient(JavaSparkContext jsc, String basePath) {
|
||||
super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build());
|
||||
}
|
||||
|
||||
public CompactionAdminClient(JavaSparkContext jsc, String basePath,
|
||||
Option<EmbeddedTimelineService> timelineServer) {
|
||||
super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build(), timelineServer);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate all compaction operations in a compaction plan. Verifies the file-slices are consistent with corresponding
|
||||
* compaction operations.
|
||||
*
|
||||
* @param metaClient Hoodie Table Meta Client
|
||||
* @param compactionInstant Compaction Instant
|
||||
*/
|
||||
public List<ValidationOpResult> validateCompactionPlan(HoodieTableMetaClient metaClient,
|
||||
String compactionInstant, int parallelism) throws IOException {
|
||||
HoodieCompactionPlan plan = getCompactionPlan(metaClient, compactionInstant);
|
||||
HoodieTableFileSystemView fsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
|
||||
if (plan.getOperations() != null) {
|
||||
List<CompactionOperation> ops = plan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList());
|
||||
return jsc.parallelize(ops, parallelism).map(op -> {
|
||||
try {
|
||||
return validateCompactionOperation(metaClient, compactionInstant, op, Option.of(fsView));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}).collect();
|
||||
}
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Un-schedules compaction plan. Remove All compaction operation scheduled and re-arrange delta-files that were
|
||||
* created after the compaction was scheduled.
|
||||
*
|
||||
* This operation MUST be executed with compactions and writer turned OFF.
|
||||
*
|
||||
* @param compactionInstant Compaction Instant
|
||||
* @param skipValidation Skip validation step
|
||||
* @param parallelism Parallelism
|
||||
* @param dryRun Dry Run
|
||||
*/
|
||||
public List<RenameOpResult> unscheduleCompactionPlan(
|
||||
String compactionInstant, boolean skipValidation, int parallelism, boolean dryRun) throws Exception {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(false);
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
|
||||
getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, parallelism,
|
||||
Option.empty(), skipValidation);
|
||||
|
||||
List<RenameOpResult> res =
|
||||
runRenamingOps(metaClient, renameActions, parallelism, dryRun);
|
||||
|
||||
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()) {
|
||||
// Overwrite compaction request with empty compaction operations
|
||||
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, compactionInstant);
|
||||
Path inflightPath = new Path(metaClient.getMetaPath(), inflight.getFileName());
|
||||
if (metaClient.getFs().exists(inflightPath)) {
|
||||
// We need to rollback data-files because of this inflight compaction before unscheduling
|
||||
throw new IllegalStateException("Please rollback the inflight compaction before unscheduling");
|
||||
}
|
||||
// Leave the trace in aux folder but delete from metapath.
|
||||
// TODO: Add a rollback instant but for compaction
|
||||
HoodieInstant instant = new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstant);
|
||||
boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false);
|
||||
Preconditions.checkArgument(deleted, "Unable to delete compaction instant.");
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove a fileId from pending compaction. Removes the associated compaction operation and rename delta-files
|
||||
* that were generated for that file-id after the compaction operation was scheduled.
|
||||
*
|
||||
* This operation MUST be executed with compactions and writer turned OFF.
|
||||
*
|
||||
* @param fgId FileGroupId to be unscheduled
|
||||
* @param skipValidation Skip validation
|
||||
* @param dryRun Dry Run Mode
|
||||
*/
|
||||
public List<RenameOpResult> unscheduleCompactionFileId(HoodieFileGroupId fgId,
|
||||
boolean skipValidation, boolean dryRun) throws Exception {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(false);
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
|
||||
getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fgId,
|
||||
Option.empty(), skipValidation);
|
||||
|
||||
List<RenameOpResult> res = runRenamingOps(metaClient, renameActions, 1, dryRun);
|
||||
|
||||
if (!dryRun && !res.isEmpty() && res.get(0).isExecuted() && res.get(0).isSuccess()) {
|
||||
// Ready to remove this file-Id from compaction request
|
||||
Pair<String, HoodieCompactionOperation> compactionOperationWithInstant =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient).get(fgId);
|
||||
HoodieCompactionPlan plan = CompactionUtils
|
||||
.getCompactionPlan(metaClient, compactionOperationWithInstant.getKey());
|
||||
List<HoodieCompactionOperation> newOps = plan.getOperations().stream()
|
||||
.filter(op -> (!op.getFileId().equals(fgId.getFileId()))
|
||||
&& (!op.getPartitionPath().equals(fgId.getPartitionPath()))).collect(Collectors.toList());
|
||||
HoodieCompactionPlan newPlan =
|
||||
HoodieCompactionPlan.newBuilder().setOperations(newOps).setExtraMetadata(plan.getExtraMetadata()).build();
|
||||
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION,
|
||||
compactionOperationWithInstant.getLeft());
|
||||
Path inflightPath = new Path(metaClient.getMetaPath(), inflight.getFileName());
|
||||
if (metaClient.getFs().exists(inflightPath)) {
|
||||
// revert if in inflight state
|
||||
metaClient.getActiveTimeline().revertCompactionInflightToRequested(inflight);
|
||||
}
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(
|
||||
new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionOperationWithInstant.getLeft()),
|
||||
AvroUtils.serializeCompactionPlan(newPlan));
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
||||
/**
|
||||
* Renames delta files to make file-slices consistent with the timeline as dictated by Hoodie metadata.
|
||||
* Use when compaction unschedule fails partially.
|
||||
*
|
||||
* This operation MUST be executed with compactions and writer turned OFF.
|
||||
* @param compactionInstant Compaction Instant to be repaired
|
||||
* @param dryRun Dry Run Mode
|
||||
*/
|
||||
public List<RenameOpResult> repairCompaction(String compactionInstant,
|
||||
int parallelism, boolean dryRun) throws Exception {
|
||||
HoodieTableMetaClient metaClient = createMetaClient(false);
|
||||
List<ValidationOpResult> validationResults =
|
||||
validateCompactionPlan(metaClient, compactionInstant, parallelism);
|
||||
List<ValidationOpResult> failed = validationResults.stream()
|
||||
.filter(v -> !v.isSuccess()).collect(Collectors.toList());
|
||||
if (failed.isEmpty()) {
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
final HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getCommitsAndCompactionTimeline());
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions = failed.stream().flatMap(v ->
|
||||
getRenamingActionsToAlignWithCompactionOperation(metaClient, compactionInstant,
|
||||
v.getOperation(), Option.of(fsView)).stream()).collect(Collectors.toList());
|
||||
return runRenamingOps(metaClient, renameActions, parallelism, dryRun);
|
||||
}
|
||||
|
||||
/**
|
||||
* Construction Compaction Plan from compaction instant
|
||||
*/
|
||||
private static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient,
|
||||
String compactionInstant) throws IOException {
|
||||
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
|
||||
metaClient.getActiveTimeline().getInstantAuxiliaryDetails(
|
||||
HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
|
||||
return compactionPlan;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Renaming actions to ensure the log-files of merged file-slices is aligned with compaction operation. This
|
||||
* method is used to recover from failures during unschedule compaction operations.
|
||||
*
|
||||
* @param metaClient Hoodie Table Meta Client
|
||||
* @param compactionInstant Compaction Instant
|
||||
* @param op Compaction Operation
|
||||
* @param fsViewOpt File System View
|
||||
*/
|
||||
protected static List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsToAlignWithCompactionOperation(
|
||||
HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation op,
|
||||
Option<HoodieTableFileSystemView> fsViewOpt) {
|
||||
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() :
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
HoodieInstant lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant().get();
|
||||
FileSlice merged =
|
||||
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(op.getPartitionPath(), lastInstant.getTimestamp())
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId())).findFirst().get();
|
||||
final int maxVersion =
|
||||
op.getDeltaFilePaths().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf)))
|
||||
.reduce((x, y) -> x > y ? x : y).orElse(0);
|
||||
List<HoodieLogFile> logFilesToBeMoved =
|
||||
merged.getLogFiles().filter(lf -> lf.getLogVersion() > maxVersion).collect(Collectors.toList());
|
||||
return logFilesToBeMoved.stream().map(lf -> {
|
||||
Preconditions.checkArgument(lf.getLogVersion() - maxVersion > 0,
|
||||
"Expect new log version to be sane");
|
||||
HoodieLogFile newLogFile = new HoodieLogFile(new Path(lf.getPath().getParent(),
|
||||
FSUtils.makeLogFileName(lf.getFileId(), "." + FSUtils.getFileExtensionFromLog(lf.getPath()),
|
||||
compactionInstant, lf.getLogVersion() - maxVersion, HoodieLogFormat.UNKNOWN_WRITE_TOKEN)));
|
||||
return Pair.of(lf, newLogFile);
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Rename log files. This is done for un-scheduling a pending compaction operation NOTE: Can only be used safely when
|
||||
* no writer (ingestion/compaction) is running.
|
||||
*
|
||||
* @param metaClient Hoodie Table Meta-Client
|
||||
* @param oldLogFile Old Log File
|
||||
* @param newLogFile New Log File
|
||||
*/
|
||||
protected static void renameLogFile(HoodieTableMetaClient metaClient, HoodieLogFile oldLogFile,
|
||||
HoodieLogFile newLogFile) throws IOException {
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(oldLogFile.getPath());
|
||||
Preconditions.checkArgument(statuses.length == 1, "Only one status must be present");
|
||||
Preconditions.checkArgument(statuses[0].isFile(), "Source File must exist");
|
||||
Preconditions.checkArgument(oldLogFile.getPath().getParent().equals(newLogFile.getPath().getParent()),
|
||||
"Log file must only be moved within the parent directory");
|
||||
metaClient.getFs().rename(oldLogFile.getPath(), newLogFile.getPath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if a compaction operation is valid
|
||||
*
|
||||
* @param metaClient Hoodie Table Meta client
|
||||
* @param compactionInstant Compaction Instant
|
||||
* @param operation Compaction Operation
|
||||
* @param fsViewOpt File System View
|
||||
*/
|
||||
private ValidationOpResult validateCompactionOperation(HoodieTableMetaClient metaClient,
|
||||
String compactionInstant, CompactionOperation operation, Option<HoodieTableFileSystemView> fsViewOpt)
|
||||
throws IOException {
|
||||
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() :
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
Option<HoodieInstant> lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant();
|
||||
try {
|
||||
if (lastInstant.isPresent()) {
|
||||
Option<FileSlice> fileSliceOptional =
|
||||
Option.fromJavaOptional(fileSystemView.getLatestUnCompactedFileSlices(operation.getPartitionPath())
|
||||
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst());
|
||||
if (fileSliceOptional.isPresent()) {
|
||||
FileSlice fs = fileSliceOptional.get();
|
||||
Option<HoodieDataFile> df = fs.getDataFile();
|
||||
if (operation.getDataFilePath().isPresent()) {
|
||||
String expPath = metaClient.getFs().getFileStatus(new Path(operation.getDataFilePath().get())).getPath()
|
||||
.toString();
|
||||
Preconditions.checkArgument(df.isPresent(), "Data File must be present. File Slice was : "
|
||||
+ fs + ", operation :" + operation);
|
||||
Preconditions.checkArgument(df.get().getPath().equals(expPath),
|
||||
"Base Path in operation is specified as " + expPath + " but got path " + df.get().getPath());
|
||||
}
|
||||
Set<HoodieLogFile> logFilesInFileSlice = fs.getLogFiles().collect(Collectors.toSet());
|
||||
Set<HoodieLogFile> logFilesInCompactionOp = operation.getDeltaFilePaths().stream()
|
||||
.map(dp -> {
|
||||
try {
|
||||
FileStatus[] fileStatuses = metaClient.getFs().listStatus(new Path(dp));
|
||||
Preconditions.checkArgument(fileStatuses.length == 1, "Expect only 1 file-status");
|
||||
return new HoodieLogFile(fileStatuses[0]);
|
||||
} catch (FileNotFoundException fe) {
|
||||
throw new CompactionValidationException(fe.getMessage());
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}).collect(Collectors.toSet());
|
||||
Set<HoodieLogFile> missing =
|
||||
logFilesInCompactionOp.stream().filter(lf -> !logFilesInFileSlice.contains(lf))
|
||||
.collect(Collectors.toSet());
|
||||
Preconditions.checkArgument(missing.isEmpty(),
|
||||
"All log files specified in compaction operation is not present. Missing :" + missing
|
||||
+ ", Exp :" + logFilesInCompactionOp + ", Got :" + logFilesInFileSlice);
|
||||
Set<HoodieLogFile> diff =
|
||||
logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf))
|
||||
.collect(Collectors.toSet());
|
||||
Preconditions.checkArgument(diff.stream().allMatch(lf -> lf.getBaseCommitTime().equals(compactionInstant)),
|
||||
"There are some log-files which are neither specified in compaction plan "
|
||||
+ "nor present after compaction request instant. Some of these :" + diff);
|
||||
} else {
|
||||
throw new CompactionValidationException("Unable to find file-slice for file-id (" + operation.getFileId()
|
||||
+ " Compaction operation is invalid.");
|
||||
}
|
||||
} else {
|
||||
throw new CompactionValidationException("Unable to find any committed instant. Compaction Operation may "
|
||||
+ "be pointing to stale file-slices");
|
||||
}
|
||||
} catch (CompactionValidationException | IllegalArgumentException e) {
|
||||
return new ValidationOpResult(operation, false, Option.of(e));
|
||||
}
|
||||
return new ValidationOpResult(operation, true, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute Renaming operation
|
||||
*
|
||||
* @param metaClient HoodieTable MetaClient
|
||||
* @param renameActions List of rename operations
|
||||
*/
|
||||
private List<RenameOpResult> runRenamingOps(HoodieTableMetaClient metaClient,
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions, int parallelism, boolean dryRun) {
|
||||
if (renameActions.isEmpty()) {
|
||||
log.info("No renaming of log-files needed. Proceeding to removing file-id from compaction-plan");
|
||||
return new ArrayList<>();
|
||||
} else {
|
||||
log.info("The following compaction renaming operations needs to be performed to un-schedule");
|
||||
if (!dryRun) {
|
||||
return jsc.parallelize(renameActions, parallelism).map(lfPair -> {
|
||||
try {
|
||||
log.info("RENAME " + lfPair.getLeft().getPath() + " => " + lfPair.getRight().getPath());
|
||||
renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight());
|
||||
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, Option.of(e));
|
||||
}
|
||||
}).collect();
|
||||
} else {
|
||||
log.info("Dry-Run Mode activated for rename operations");
|
||||
return renameActions.parallelStream()
|
||||
.map(lfPair -> new RenameOpResult(lfPair, false, false, Option.empty()))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate renaming actions for unscheduling a pending compaction plan. NOTE: Can only be used safely when no writer
|
||||
* (ingestion/compaction) is running.
|
||||
*
|
||||
* @param metaClient Hoodie Table MetaClient
|
||||
* @param compactionInstant Compaction Instant to be unscheduled
|
||||
* @param fsViewOpt Cached File System View
|
||||
* @param skipValidation Skip Validation
|
||||
* @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule
|
||||
* compaction.
|
||||
*/
|
||||
protected List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionPlan(
|
||||
HoodieTableMetaClient metaClient, String compactionInstant, int parallelism,
|
||||
Option<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
HoodieTableFileSystemView fsView = fsViewOpt.isPresent() ? fsViewOpt.get() :
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
HoodieCompactionPlan plan = getCompactionPlan(metaClient, compactionInstant);
|
||||
if (plan.getOperations() != null) {
|
||||
log.info("Number of Compaction Operations :" + plan.getOperations().size()
|
||||
+ " for instant :" + compactionInstant);
|
||||
List<CompactionOperation> ops = plan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList());
|
||||
return jsc.parallelize(ops, parallelism).flatMap(op -> {
|
||||
try {
|
||||
return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant,
|
||||
op, Option.of(fsView), skipValidation).iterator();
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
} catch (CompactionValidationException ve) {
|
||||
throw new HoodieException(ve);
|
||||
}
|
||||
}).collect();
|
||||
}
|
||||
log.warn("No operations for compaction instant : " + compactionInstant);
|
||||
return new ArrayList<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate renaming actions for unscheduling a compaction operation NOTE: Can only be used safely when no writer
|
||||
* (ingestion/compaction) is running.
|
||||
*
|
||||
* @param metaClient Hoodie Table MetaClient
|
||||
* @param compactionInstant Compaction Instant
|
||||
* @param operation Compaction Operation
|
||||
* @param fsViewOpt Cached File System View
|
||||
* @param skipValidation Skip Validation
|
||||
* @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule
|
||||
* compaction.
|
||||
*/
|
||||
public List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionOperation(
|
||||
HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation operation,
|
||||
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, Option.of(fileSystemView));
|
||||
}
|
||||
HoodieInstant lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant().get();
|
||||
FileSlice merged =
|
||||
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(operation.getPartitionPath(), lastInstant.getTimestamp())
|
||||
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
|
||||
List<HoodieLogFile> logFilesToRepair =
|
||||
merged.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(compactionInstant))
|
||||
.sorted(HoodieLogFile.getLogFileComparator())
|
||||
.collect(Collectors.toList());
|
||||
FileSlice fileSliceForCompaction =
|
||||
fileSystemView.getLatestFileSlicesBeforeOrOn(operation.getPartitionPath(), operation.getBaseInstantTime(), true)
|
||||
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
|
||||
int maxUsedVersion =
|
||||
fileSliceForCompaction.getLogFiles().findFirst().map(HoodieLogFile::getLogVersion)
|
||||
.orElse(HoodieLogFile.LOGFILE_BASE_VERSION - 1);
|
||||
String logExtn = fileSliceForCompaction.getLogFiles().findFirst().map(lf -> "." + lf.getFileExtension())
|
||||
.orElse(HoodieLogFile.DELTA_EXTENSION);
|
||||
String parentPath = fileSliceForCompaction.getDataFile().map(df -> new Path(df.getPath()).getParent().toString())
|
||||
.orElse(fileSliceForCompaction.getLogFiles().findFirst().map(lf -> lf.getPath().getParent().toString()).get());
|
||||
for (HoodieLogFile toRepair : logFilesToRepair) {
|
||||
int version = maxUsedVersion + 1;
|
||||
HoodieLogFile newLf = new HoodieLogFile(new Path(parentPath, FSUtils.makeLogFileName(operation.getFileId(),
|
||||
logExtn, operation.getBaseInstantTime(), version, HoodieLogFormat.UNKNOWN_WRITE_TOKEN)));
|
||||
result.add(Pair.of(toRepair, newLf));
|
||||
maxUsedVersion = version;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate renaming actions for unscheduling a fileId from pending compaction. NOTE: Can only be used safely when no
|
||||
* writer (ingestion/compaction) is running.
|
||||
*
|
||||
* @param metaClient Hoodie Table MetaClient
|
||||
* @param fgId FileGroupId to remove compaction
|
||||
* @param fsViewOpt Cached File System View
|
||||
* @param skipValidation Skip Validation
|
||||
* @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule
|
||||
* compaction.
|
||||
*/
|
||||
public List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionForFileId(
|
||||
HoodieTableMetaClient metaClient, HoodieFileGroupId fgId,
|
||||
Option<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> allPendingCompactions =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
if (allPendingCompactions.containsKey(fgId)) {
|
||||
Pair<String, HoodieCompactionOperation> opWithInstant = allPendingCompactions.get(fgId);
|
||||
return getRenamingActionsForUnschedulingCompactionOperation(metaClient, opWithInstant.getKey(),
|
||||
CompactionOperation.convertFromAvroRecordInstance(opWithInstant.getValue()), fsViewOpt, skipValidation);
|
||||
}
|
||||
throw new HoodieException("FileGroupId " + fgId + " not in pending compaction");
|
||||
}
|
||||
|
||||
/**
|
||||
* Holds Operation result for Renaming
|
||||
*/
|
||||
public static class RenameOpResult extends OperationResult<RenameInfo> {
|
||||
|
||||
public RenameOpResult() {
|
||||
}
|
||||
|
||||
public RenameOpResult(Pair<HoodieLogFile, HoodieLogFile> op, boolean success,
|
||||
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,
|
||||
Option<Exception> exception) {
|
||||
super(new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(),
|
||||
op.getRight().getPath().toString()), executed, success, exception);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Holds Operation result for Renaming
|
||||
*/
|
||||
public static class ValidationOpResult extends OperationResult<CompactionOperation> {
|
||||
|
||||
public ValidationOpResult() {
|
||||
}
|
||||
|
||||
public ValidationOpResult(
|
||||
CompactionOperation operation, boolean success, Option<Exception> exception) {
|
||||
super(operation, success, exception);
|
||||
}
|
||||
}
|
||||
|
||||
public static class RenameInfo implements Serializable {
|
||||
|
||||
public String fileId;
|
||||
public String srcPath;
|
||||
public String destPath;
|
||||
|
||||
public RenameInfo() {
|
||||
}
|
||||
|
||||
public RenameInfo(String fileId, String srcPath, String destPath) {
|
||||
this.fileId = fileId;
|
||||
this.srcPath = srcPath;
|
||||
this.destPath = destPath;
|
||||
}
|
||||
}
|
||||
|
||||
public static class CompactionValidationException extends RuntimeException {
|
||||
|
||||
public CompactionValidationException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
224
hudi-client/src/main/java/org/apache/hudi/HoodieReadClient.java
Normal file
224
hudi-client/src/main/java/org/apache/hudi/HoodieReadClient.java
Normal file
@@ -0,0 +1,224 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Provides an RDD based API for accessing/filtering Hoodie tables, based on keys.
|
||||
*/
|
||||
public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoodieClient {
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(HoodieReadClient.class);
|
||||
|
||||
/**
|
||||
* TODO: We need to persist the index type into hoodie.properties and be able to access the index
|
||||
* just with a simple basepath pointing to the dataset. Until, then just always assume a
|
||||
* BloomIndex
|
||||
*/
|
||||
private final transient HoodieIndex<T> index;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private HoodieTable hoodieTable;
|
||||
private transient Option<SQLContext> sqlContextOpt;
|
||||
|
||||
/**
|
||||
* @param basePath path to Hoodie dataset
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath,
|
||||
Option<EmbeddedTimelineService> timelineService) {
|
||||
this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
// by default we use HoodieBloomIndex
|
||||
.withIndexConfig(
|
||||
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.build(), timelineService);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param basePath path to Hoodie dataset
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath) {
|
||||
this(jsc, basePath, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param jsc
|
||||
* @param basePath
|
||||
* @param sqlContext
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, String basePath, SQLContext sqlContext) {
|
||||
this(jsc, basePath);
|
||||
this.sqlContextOpt = Option.of(sqlContext);
|
||||
}
|
||||
|
||||
/**
|
||||
* @param clientConfig instance of HoodieWriteConfig
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig) {
|
||||
this(jsc, clientConfig, Option.empty());
|
||||
}
|
||||
|
||||
/**
|
||||
* @param clientConfig instance of HoodieWriteConfig
|
||||
*/
|
||||
public HoodieReadClient(JavaSparkContext jsc, HoodieWriteConfig clientConfig,
|
||||
Option<EmbeddedTimelineService> timelineService) {
|
||||
super(jsc, clientConfig, timelineService);
|
||||
final String basePath = clientConfig.getBasePath();
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), basePath, true);
|
||||
this.hoodieTable = HoodieTable.getHoodieTable(metaClient, clientConfig, jsc);
|
||||
this.commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
this.index = HoodieIndex.createIndex(clientConfig, jsc);
|
||||
this.sqlContextOpt = Option.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds support for accessing Hoodie built tables from SparkSQL, as you normally would.
|
||||
*
|
||||
* @return SparkConf object to be used to construct the SparkContext by caller
|
||||
*/
|
||||
public static SparkConf addHoodieSupport(SparkConf conf) {
|
||||
conf.set("spark.sql.hive.convertMetastoreParquet", "false");
|
||||
return conf;
|
||||
}
|
||||
|
||||
private void assertSqlContext() {
|
||||
if (!sqlContextOpt.isPresent()) {
|
||||
throw new IllegalStateException(
|
||||
"SQLContext must be set, when performing dataframe operations");
|
||||
}
|
||||
}
|
||||
|
||||
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 Option.of(dataFile.getPath());
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a bunch of hoodie keys, fetches all the individual records out as a data frame
|
||||
*
|
||||
* @return a dataframe
|
||||
*/
|
||||
public Dataset<Row> readROView(JavaRDD<HoodieKey> hoodieKeys, int parallelism) {
|
||||
assertSqlContext();
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> lookupResultRDD = index
|
||||
.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
|
||||
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();
|
||||
|
||||
// record locations might be same for multiple keys, so need a unique list
|
||||
Set<String> uniquePaths = new HashSet<>(paths);
|
||||
Dataset<Row> originalDF = sqlContextOpt.get().read()
|
||||
.parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
|
||||
StructType schema = originalDF.schema();
|
||||
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD().mapToPair(row -> {
|
||||
HoodieKey key = new HoodieKey(row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
|
||||
row.getAs(HoodieRecord.PARTITION_PATH_METADATA_FIELD));
|
||||
return new Tuple2<>(key, row);
|
||||
});
|
||||
|
||||
// Now, we need to further filter out, for only rows that match the supplied hoodie keys
|
||||
JavaRDD<Row> rowRDD = keyRowRDD.join(keyToFileRDD, parallelism).map(tuple -> tuple._2()._1());
|
||||
return sqlContextOpt.get().createDataFrame(rowRDD, schema);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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, Option<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
|
||||
return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Filter out HoodieRecords that already exists in the output folder. This is useful in
|
||||
* deduplication.
|
||||
*
|
||||
* @param hoodieRecords Input RDD of Hoodie records.
|
||||
* @return A subset of hoodieRecords RDD, with existing records filtered out.
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> filterExists(JavaRDD<HoodieRecord<T>> hoodieRecords) {
|
||||
JavaRDD<HoodieRecord<T>> recordsWithLocation = tagLocation(hoodieRecords);
|
||||
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
|
||||
}
|
||||
|
||||
/**
|
||||
* Looks up the index and tags each incoming record with a location of a file that contains the
|
||||
* row (if it is actually present). Input RDD should contain no duplicates if needed.
|
||||
*
|
||||
* @param hoodieRecords Input RDD of Hoodie records
|
||||
* @return Tagged RDD of Hoodie records
|
||||
*/
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords)
|
||||
throws HoodieIndexException {
|
||||
return index.tagLocation(hoodieRecords, jsc, hoodieTable);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return all pending compactions with instant time for clients to decide what to compact next.
|
||||
* @return
|
||||
*/
|
||||
public List<Pair<String, HoodieCompactionPlan>> getPendingCompactions() {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
|
||||
hoodieTable.getMetaClient().getBasePath(), true);
|
||||
return CompactionUtils.getAllPendingCompactionPlans(metaClient).stream()
|
||||
.map(instantWorkloadPair ->
|
||||
Pair.of(instantWorkloadPair.getKey().getTimestamp(), instantWorkloadPair.getValue()))
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
1417
hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java
Normal file
1417
hudi-client/src/main/java/org/apache/hudi/HoodieWriteClient.java
Normal file
File diff suppressed because it is too large
Load Diff
186
hudi-client/src/main/java/org/apache/hudi/WriteStatus.java
Normal file
186
hudi-client/src/main/java/org/apache/hudi/WriteStatus.java
Normal file
@@ -0,0 +1,186 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* Status of a write operation.
|
||||
*/
|
||||
public class WriteStatus implements Serializable {
|
||||
|
||||
private static final long RANDOM_SEED = 9038412832L;
|
||||
|
||||
private final HashMap<HoodieKey, Throwable> errors = new HashMap<>();
|
||||
|
||||
private final List<HoodieRecord> writtenRecords = new ArrayList<>();
|
||||
|
||||
private final List<HoodieRecord> failedRecords = new ArrayList<>();
|
||||
|
||||
private Throwable globalError = null;
|
||||
|
||||
private String fileId = null;
|
||||
|
||||
private String partitionPath = null;
|
||||
|
||||
private HoodieWriteStat stat = null;
|
||||
|
||||
private long totalRecords = 0;
|
||||
private long totalErrorRecords = 0;
|
||||
|
||||
private final double failureFraction;
|
||||
private final boolean trackSuccessRecords;
|
||||
private final transient Random random;
|
||||
|
||||
public WriteStatus(Boolean trackSuccessRecords, Double failureFraction) {
|
||||
this.trackSuccessRecords = trackSuccessRecords;
|
||||
this.failureFraction = failureFraction;
|
||||
this.random = new Random(RANDOM_SEED);
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark write as success, optionally using given parameters for the purpose of calculating some
|
||||
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
|
||||
* objects are collected in Spark Driver.
|
||||
*
|
||||
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
|
||||
* it.
|
||||
* @param optionalRecordMetadata optional metadata related to data contained in {@link
|
||||
* HoodieRecord} before deflation.
|
||||
*/
|
||||
public void markSuccess(HoodieRecord record, Option<Map<String, String>> optionalRecordMetadata) {
|
||||
if (trackSuccessRecords) {
|
||||
writtenRecords.add(record);
|
||||
}
|
||||
totalRecords++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mark write as failed, optionally using given parameters for the purpose of calculating some
|
||||
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
|
||||
* objects are collected in Spark Driver.
|
||||
*
|
||||
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
|
||||
* it.
|
||||
* @param optionalRecordMetadata optional metadata related to data contained in {@link
|
||||
* HoodieRecord} before deflation.
|
||||
*/
|
||||
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);
|
||||
errors.put(record.getKey(), t);
|
||||
}
|
||||
totalRecords++;
|
||||
totalErrorRecords++;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public void setFileId(String fileId) {
|
||||
this.fileId = fileId;
|
||||
}
|
||||
|
||||
public boolean hasErrors() {
|
||||
return totalErrorRecords > 0;
|
||||
}
|
||||
|
||||
public boolean isErrored(HoodieKey key) {
|
||||
return errors.containsKey(key);
|
||||
}
|
||||
|
||||
public HashMap<HoodieKey, Throwable> getErrors() {
|
||||
return errors;
|
||||
}
|
||||
|
||||
public boolean hasGlobalError() {
|
||||
return globalError != null;
|
||||
}
|
||||
|
||||
public Throwable getGlobalError() {
|
||||
return this.globalError;
|
||||
}
|
||||
|
||||
public void setGlobalError(Throwable t) {
|
||||
this.globalError = t;
|
||||
}
|
||||
|
||||
public List<HoodieRecord> getWrittenRecords() {
|
||||
return writtenRecords;
|
||||
}
|
||||
|
||||
public List<HoodieRecord> getFailedRecords() {
|
||||
return failedRecords;
|
||||
}
|
||||
|
||||
public HoodieWriteStat getStat() {
|
||||
return stat;
|
||||
}
|
||||
|
||||
public void setStat(HoodieWriteStat stat) {
|
||||
this.stat = stat;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
public void setPartitionPath(String partitionPath) {
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
public long getTotalRecords() {
|
||||
return totalRecords;
|
||||
}
|
||||
|
||||
public void setTotalRecords(long totalRecords) {
|
||||
this.totalRecords = totalRecords;
|
||||
}
|
||||
|
||||
public long getTotalErrorRecords() {
|
||||
return totalErrorRecords;
|
||||
}
|
||||
|
||||
public void setTotalErrorRecords(long totalErrorRecords) {
|
||||
this.totalErrorRecords = totalErrorRecords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("WriteStatus {");
|
||||
sb.append("fileId=").append(fileId);
|
||||
sb.append(", globalError='").append(globalError).append('\'');
|
||||
sb.append(", hasErrors='").append(hasErrors()).append('\'');
|
||||
sb.append(", errorCount='").append(totalErrorRecords).append('\'');
|
||||
sb.append(", errorPct='").append((100.0 * totalErrorRecords) / totalRecords).append('\'');
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,105 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.embedded;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.common.SerializableConfiguration;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.util.NetworkUtils;
|
||||
import org.apache.hudi.timeline.service.TimelineService;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
|
||||
/**
|
||||
* Timeline Service that runs as part of write client
|
||||
*/
|
||||
public class EmbeddedTimelineService {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(EmbeddedTimelineService.class);
|
||||
|
||||
private int serverPort;
|
||||
private String hostAddr;
|
||||
private final SerializableConfiguration hadoopConf;
|
||||
private final FileSystemViewStorageConfig config;
|
||||
private transient FileSystemViewManager viewManager;
|
||||
private transient TimelineService server;
|
||||
|
||||
public EmbeddedTimelineService(Configuration hadoopConf, SparkConf sparkConf, FileSystemViewStorageConfig config) {
|
||||
setHostAddrFromSparkConf(sparkConf);
|
||||
if (hostAddr == null) {
|
||||
this.hostAddr = NetworkUtils.getHostname();
|
||||
}
|
||||
this.config = config;
|
||||
this.hadoopConf = new SerializableConfiguration(hadoopConf);
|
||||
this.viewManager = createViewManager();
|
||||
}
|
||||
|
||||
private FileSystemViewManager createViewManager() {
|
||||
// Using passed-in configs to build view storage configs
|
||||
FileSystemViewStorageConfig.Builder builder =
|
||||
FileSystemViewStorageConfig.newBuilder().fromProperties(config.getProps());
|
||||
FileSystemViewStorageType storageType = builder.build().getStorageType();
|
||||
if (storageType.equals(FileSystemViewStorageType.REMOTE_ONLY)
|
||||
|| storageType.equals(FileSystemViewStorageType.REMOTE_FIRST)) {
|
||||
// Reset to default if set to Remote
|
||||
builder.withStorageType(FileSystemViewStorageType.MEMORY);
|
||||
}
|
||||
return FileSystemViewManager.createViewManager(hadoopConf, builder.build());
|
||||
}
|
||||
|
||||
public void startServer() throws IOException {
|
||||
server = new TimelineService(0, viewManager, hadoopConf.get());
|
||||
serverPort = server.startService();
|
||||
logger.info("Started embedded timeline server at " + hostAddr + ":" + serverPort);
|
||||
}
|
||||
|
||||
private void setHostAddrFromSparkConf(SparkConf sparkConf) {
|
||||
String hostAddr = sparkConf.get("spark.driver.host", null);
|
||||
if (hostAddr != null) {
|
||||
logger.info("Overriding hostIp to (" + hostAddr + ") found in spark-conf. It was " + this.hostAddr);
|
||||
this.hostAddr = hostAddr;
|
||||
} else {
|
||||
logger.warn("Unable to find driver bind address from spark config");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieves proper view storage configs for remote clients to access this service
|
||||
*/
|
||||
public FileSystemViewStorageConfig getRemoteFileSystemViewConfig() {
|
||||
return FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.REMOTE_FIRST)
|
||||
.withRemoteServerHost(hostAddr).withRemoteServerPort(serverPort).build();
|
||||
}
|
||||
|
||||
public FileSystemViewManager getViewManager() {
|
||||
return viewManager;
|
||||
}
|
||||
|
||||
public void stop() {
|
||||
if (null != server) {
|
||||
this.server.close();
|
||||
this.server = null;
|
||||
this.viewManager = null;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,39 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.client.utils;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
public class ClientUtils {
|
||||
|
||||
/**
|
||||
* Create Consistency Aware MetaClient
|
||||
*
|
||||
* @param jsc JavaSparkContext
|
||||
* @param config HoodieWriteConfig
|
||||
* @param loadActiveTimelineOnLoad early loading of timeline
|
||||
*/
|
||||
public static HoodieTableMetaClient createMetaClient(JavaSparkContext jsc, HoodieWriteConfig config,
|
||||
boolean loadActiveTimelineOnLoad) {
|
||||
return new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), loadActiveTimelineOnLoad,
|
||||
config.getConsistencyGuardConfig());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,312 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.io.compact.strategy.CompactionStrategy;
|
||||
import org.apache.hudi.io.compact.strategy.LogFileSizeBasedCompactionStrategy;
|
||||
|
||||
/**
|
||||
* Compaction related config
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
|
||||
public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
|
||||
// Turn on inline compaction - after fw delta commits a inline compaction will be run
|
||||
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
|
||||
// Run a compaction every N delta commits
|
||||
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP =
|
||||
"hoodie.compact.inline.max" + ".delta.commits";
|
||||
public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP =
|
||||
"hoodie.cleaner.fileversions" + ".retained";
|
||||
public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained";
|
||||
public static final String MAX_COMMITS_TO_KEEP_PROP = "hoodie.keep.max.commits";
|
||||
public static final String MIN_COMMITS_TO_KEEP_PROP = "hoodie.keep.min.commits";
|
||||
public static final String COMMITS_ARCHIVAL_BATCH_SIZE_PROP = "hoodie.commits.archival.batch";
|
||||
// Upsert uses this file size to compact new data onto existing files..
|
||||
public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit";
|
||||
// By default, treat any file <= 100MB as a small file.
|
||||
public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(104857600);
|
||||
/**
|
||||
* Configs related to specific table types
|
||||
**/
|
||||
// Number of inserts, that will be put each partition/bucket for writing
|
||||
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE =
|
||||
"hoodie.copyonwrite.insert" + ".split.size";
|
||||
// The rationale to pick the insert parallelism is the following. Writing out 100MB files,
|
||||
// with atleast 1kb records, means 100K records per file. we just overprovision to 500K
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000);
|
||||
// Config to control whether we control insert split sizes automatically based on average
|
||||
// record sizes
|
||||
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS =
|
||||
"hoodie.copyonwrite.insert" + ".auto.split";
|
||||
// its off by default
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(true);
|
||||
// This value is used as a guessimate for the record size, if we can't determine this from
|
||||
// previous commits
|
||||
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE =
|
||||
"hoodie.copyonwrite" + ".record.size.estimate";
|
||||
// Used to determine how much more can be packed into a small file, before it exceeds the size
|
||||
// limit.
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String
|
||||
.valueOf(1024);
|
||||
public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
|
||||
public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
|
||||
public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
|
||||
// 500GB of target IO per compaction (both read and write)
|
||||
public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
|
||||
public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
|
||||
// 200GB of target IO per compaction
|
||||
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class
|
||||
.getName();
|
||||
// used to merge records written to log file
|
||||
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
|
||||
public static final String PAYLOAD_CLASS_PROP = "hoodie.compaction.payload.class";
|
||||
|
||||
// used to choose a trade off between IO vs Memory when performing compaction process
|
||||
// Depending on outputfile_size and memory provided, choose true to avoid OOM for large file
|
||||
// size + small memory
|
||||
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP =
|
||||
"hoodie.compaction.lazy" + ".block.read";
|
||||
public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "false";
|
||||
// used to choose whether to enable reverse log reading (reverse log traversal)
|
||||
public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP =
|
||||
"hoodie.compaction" + ".reverse.log.read";
|
||||
public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false";
|
||||
private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
.name();
|
||||
private static final String DEFAULT_AUTO_CLEAN = "true";
|
||||
private static final String DEFAULT_INLINE_COMPACT = "false";
|
||||
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "1";
|
||||
private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3";
|
||||
private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "10";
|
||||
private static final String DEFAULT_MAX_COMMITS_TO_KEEP = "30";
|
||||
private static final String DEFAULT_MIN_COMMITS_TO_KEEP = "20";
|
||||
private static final String DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE = String.valueOf(10);
|
||||
public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP = "hoodie.compaction.daybased.target"
|
||||
+ ".partitions";
|
||||
// 500GB of target IO per compaction (both read and write)
|
||||
public static final String DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = String.valueOf(10);
|
||||
|
||||
private HoodieCompactionConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieCompactionConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
public Builder withAutoClean(Boolean autoClean) {
|
||||
props.setProperty(AUTO_CLEAN_PROP, String.valueOf(autoClean));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInlineCompaction(Boolean inlineCompaction) {
|
||||
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder inlineCompactionEvery(int deltaCommits) {
|
||||
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(deltaCommits));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCleanerPolicy(HoodieCleaningPolicy policy) {
|
||||
props.setProperty(CLEANER_POLICY_PROP, policy.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder retainFileVersions(int fileVersionsRetained) {
|
||||
props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP, String.valueOf(fileVersionsRetained));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder retainCommits(int commitsRetained) {
|
||||
props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
|
||||
props.setProperty(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
|
||||
props.setProperty(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder compactionSmallFileSize(long smallFileLimitBytes) {
|
||||
props.setProperty(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder insertSplitSize(int insertSplitSize) {
|
||||
props.setProperty(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder autoTuneInsertSplits(boolean autoTuneInsertSplits) {
|
||||
props.setProperty(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS,
|
||||
String.valueOf(autoTuneInsertSplits));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder approxRecordSize(int recordSizeEstimate) {
|
||||
props.setProperty(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
|
||||
String.valueOf(recordSizeEstimate));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCleanerParallelism(int cleanerParallelism) {
|
||||
props.setProperty(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) {
|
||||
props.setProperty(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withPayloadClass(String payloadClassName) {
|
||||
props.setProperty(PAYLOAD_CLASS_PROP, payloadClassName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
|
||||
props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP,
|
||||
String.valueOf(targetIOPerCompactionInMB));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
|
||||
props.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP,
|
||||
String.valueOf(maxNumDeltaCommitsBeforeCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionLazyBlockReadEnabled(Boolean compactionLazyBlockReadEnabled) {
|
||||
props.setProperty(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
|
||||
String.valueOf(compactionLazyBlockReadEnabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionReverseLogReadEnabled(Boolean compactionReverseLogReadEnabled) {
|
||||
props.setProperty(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP,
|
||||
String.valueOf(compactionReverseLogReadEnabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withTargetPartitionsPerDayBasedCompaction(int targetPartitionsPerCompaction) {
|
||||
props.setProperty(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP,
|
||||
String.valueOf(targetPartitionsPerCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCommitsArchivalBatchSize(int batchSize) {
|
||||
props.setProperty(COMMITS_ARCHIVAL_BATCH_SIZE_PROP, String.valueOf(batchSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieCompactionConfig build() {
|
||||
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP,
|
||||
DEFAULT_AUTO_CLEAN);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), INLINE_COMPACT_PROP,
|
||||
DEFAULT_INLINE_COMPACT);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP),
|
||||
INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP), CLEANER_POLICY_PROP,
|
||||
DEFAULT_CLEANER_POLICY);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP),
|
||||
CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP),
|
||||
CLEANER_COMMITS_RETAINED_PROP, DEFAULT_CLEANER_COMMITS_RETAINED);
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP_PROP), MAX_COMMITS_TO_KEEP_PROP,
|
||||
DEFAULT_MAX_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP_PROP), MIN_COMMITS_TO_KEEP_PROP,
|
||||
DEFAULT_MIN_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES),
|
||||
PARQUET_SMALL_FILE_LIMIT_BYTES, DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
|
||||
COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS),
|
||||
COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE),
|
||||
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
|
||||
DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM), CLEANER_PARALLELISM,
|
||||
DEFAULT_CLEANER_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP),
|
||||
COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY);
|
||||
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP),
|
||||
PAYLOAD_CLASS_PROP, DEFAULT_PAYLOAD_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
|
||||
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP),
|
||||
COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP),
|
||||
COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED);
|
||||
setDefaultOnCondition(props, !props.containsKey(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP),
|
||||
TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMMITS_ARCHIVAL_BATCH_SIZE_PROP),
|
||||
COMMITS_ARCHIVAL_BATCH_SIZE_PROP, DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE);
|
||||
|
||||
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
|
||||
|
||||
// Ensure minInstantsToKeep > cleanerCommitsRetained, otherwise we will archive some
|
||||
// commit instant on timeline, that still has not been cleaned. Could miss some data via incr pull
|
||||
int minInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP));
|
||||
int maxInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
|
||||
int cleanerCommitsRetained = Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
|
||||
Preconditions.checkArgument(maxInstantsToKeep > minInstantsToKeep);
|
||||
Preconditions.checkArgument(minInstantsToKeep > cleanerCommitsRetained,
|
||||
String.format("Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull "
|
||||
+ "missing data from few instants.", HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP,
|
||||
minInstantsToKeep, HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, cleanerCommitsRetained));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,269 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import org.apache.hudi.index.hbase.DefaultHBaseQPSResourceAllocator;
|
||||
|
||||
public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
|
||||
public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
|
||||
public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
|
||||
public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
|
||||
public static final String HBASE_ZK_ZNODEPARENT = "hoodie.index.hbase.zknode.path";
|
||||
/**
|
||||
* Note that if HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP is set to true, this batch size will not
|
||||
* be honored for HBase Puts
|
||||
*/
|
||||
public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
|
||||
|
||||
/**
|
||||
* Property to set which implementation of HBase QPS resource allocator to be used
|
||||
*/
|
||||
public static final String HBASE_INDEX_QPS_ALLOCATOR_CLASS = "hoodie.index.hbase.qps.allocator.class";
|
||||
public static final String DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS = DefaultHBaseQPSResourceAllocator.class.getName();
|
||||
/**
|
||||
* Property to set to enable auto computation of put batch size
|
||||
*/
|
||||
public static final String HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP = "hoodie.index.hbase.put.batch.size.autocompute";
|
||||
public static final String DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = "false";
|
||||
/**
|
||||
* Property to set the fraction of the global share of QPS that should be allocated to this job.
|
||||
* Let's say there are 3 jobs which have input size in terms of number of rows required for
|
||||
* HbaseIndexing as x, 2x, 3x respectively. Then this fraction for the jobs would be (0.17) 1/6,
|
||||
* 0.33 (2/6) and 0.5 (3/6) respectively.
|
||||
*/
|
||||
public static final String HBASE_QPS_FRACTION_PROP = "hoodie.index.hbase.qps.fraction";
|
||||
/**
|
||||
* Property to set maximum QPS allowed per Region Server. This should be same across various
|
||||
* jobs. This is intended to limit the aggregate QPS generated across various jobs to an Hbase
|
||||
* Region Server. It is recommended to set this value based on global indexing throughput needs
|
||||
* and most importantly, how much the HBase installation in use is able to tolerate without
|
||||
* Region Servers going down.
|
||||
*/
|
||||
public static String HBASE_MAX_QPS_PER_REGION_SERVER_PROP = "hoodie.index.hbase.max.qps.per.region.server";
|
||||
/**
|
||||
* Default batch size, used only for Get, but computed for Put
|
||||
*/
|
||||
public static final int DEFAULT_HBASE_BATCH_SIZE = 100;
|
||||
/**
|
||||
* A low default value.
|
||||
*/
|
||||
public static final int DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER = 1000;
|
||||
/**
|
||||
* Default is 50%, which means a total of 2 jobs can run using HbaseIndex without overwhelming
|
||||
* Region Servers
|
||||
*/
|
||||
public static final float DEFAULT_HBASE_QPS_FRACTION = 0.5f;
|
||||
|
||||
/**
|
||||
* Property to decide if HBASE_QPS_FRACTION_PROP is dynamically calculated based on volume
|
||||
*/
|
||||
public static final String HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = "hoodie.index.hbase.dynamic_qps";
|
||||
public static final boolean DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = false;
|
||||
/**
|
||||
* Min and Max for HBASE_QPS_FRACTION_PROP to stabilize skewed volume workloads
|
||||
*/
|
||||
public static final String HBASE_MIN_QPS_FRACTION_PROP = "hoodie.index.hbase.min.qps.fraction";
|
||||
public static final String DEFAULT_HBASE_MIN_QPS_FRACTION_PROP = "0.002";
|
||||
|
||||
public static final String HBASE_MAX_QPS_FRACTION_PROP = "hoodie.index.hbase.max.qps.fraction";
|
||||
public static final String DEFAULT_HBASE_MAX_QPS_FRACTION_PROP = "0.06";
|
||||
/**
|
||||
* Hoodie index desired puts operation time in seconds
|
||||
*/
|
||||
public static final String HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = "hoodie.index.hbase.desired_puts_time_in_secs";
|
||||
public static final int DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = 600;
|
||||
public static final String HBASE_SLEEP_MS_PUT_BATCH_PROP = "hoodie.index.hbase.sleep.ms.for.put.batch";
|
||||
public static final String HBASE_SLEEP_MS_GET_BATCH_PROP = "hoodie.index.hbase.sleep.ms.for.get.batch";
|
||||
public static final String HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS = "hoodie.index.hbase.zk.session_timeout_ms";
|
||||
public static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 60 * 1000;
|
||||
public static final String HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS =
|
||||
"hoodie.index.hbase.zk.connection_timeout_ms";
|
||||
public static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 15 * 1000;
|
||||
public static final String HBASE_ZK_PATH_QPS_ROOT = "hoodie.index.hbase.zkpath.qps_root";
|
||||
public static final String DEFAULT_HBASE_ZK_PATH_QPS_ROOT = "/QPS_ROOT";
|
||||
|
||||
public HoodieHBaseIndexConfig(final Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieHBaseIndexConfig.Builder newBuilder() {
|
||||
return new HoodieHBaseIndexConfig.Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder hbaseZkQuorum(String zkString) {
|
||||
props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder hbaseZkPort(int port) {
|
||||
props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder hbaseTableName(String tableName) {
|
||||
props.setProperty(HBASE_TABLENAME_PROP, tableName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkZnodeQPSPath(String zkZnodeQPSPath) {
|
||||
props.setProperty(HBASE_ZK_PATH_QPS_ROOT, zkZnodeQPSPath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexGetBatchSize(int getBatchSize) {
|
||||
props.setProperty(HBASE_GET_BATCH_SIZE_PROP, String.valueOf(getBatchSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexPutBatchSize(int putBatchSize) {
|
||||
props.setProperty(HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(putBatchSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexPutBatchSizeAutoCompute(boolean putBatchSizeAutoCompute) {
|
||||
props.setProperty(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, String.valueOf(putBatchSizeAutoCompute));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexDesiredPutsTime(int desiredPutsTime) {
|
||||
props.setProperty(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(desiredPutsTime));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexShouldComputeQPSDynamically(boolean shouldComputeQPsDynamically) {
|
||||
props.setProperty(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(shouldComputeQPsDynamically));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexQPSFraction(float qpsFraction) {
|
||||
props.setProperty(HBASE_QPS_FRACTION_PROP, String.valueOf(qpsFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexMinQPSFraction(float minQPSFraction) {
|
||||
props.setProperty(HBASE_MIN_QPS_FRACTION_PROP, String.valueOf(minQPSFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexMaxQPSFraction(float maxQPSFraction) {
|
||||
props.setProperty(HBASE_MAX_QPS_FRACTION_PROP, String.valueOf(maxQPSFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexSleepMsBetweenPutBatch(int sleepMsBetweenPutBatch) {
|
||||
props.setProperty(HBASE_SLEEP_MS_PUT_BATCH_PROP, String.valueOf(sleepMsBetweenPutBatch));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withQPSResourceAllocatorType(String qpsResourceAllocatorClass) {
|
||||
props.setProperty(HBASE_INDEX_QPS_ALLOCATOR_CLASS, qpsResourceAllocatorClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexZkSessionTimeout(int zkSessionTimeout) {
|
||||
props.setProperty(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(zkSessionTimeout));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexZkConnectionTimeout(int zkConnectionTimeout) {
|
||||
props.setProperty(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(zkConnectionTimeout));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkZnodeParent(String zkZnodeParent) {
|
||||
props.setProperty(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Method to set maximum QPS allowed per Region Server. This should be same across various
|
||||
* jobs. This is intended to limit the aggregate QPS generated across various jobs to an
|
||||
* Hbase Region Server.
|
||||
* </p>
|
||||
* <p>
|
||||
* It is recommended to set this value based on your global indexing throughput needs and
|
||||
* most importantly, how much your HBase installation is able to tolerate without Region
|
||||
* Servers going down.
|
||||
* </p>
|
||||
*/
|
||||
public HoodieHBaseIndexConfig.Builder hbaseIndexMaxQPSPerRegionServer(
|
||||
int maxQPSPerRegionServer) {
|
||||
// This should be same across various jobs
|
||||
props.setProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP,
|
||||
String.valueOf(maxQPSPerRegionServer));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig build() {
|
||||
HoodieHBaseIndexConfig config = new HoodieHBaseIndexConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_GET_BATCH_SIZE_PROP),
|
||||
HBASE_GET_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_PROP),
|
||||
HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP),
|
||||
HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, String.valueOf(DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_QPS_FRACTION_PROP),
|
||||
HBASE_QPS_FRACTION_PROP, String.valueOf(DEFAULT_HBASE_QPS_FRACTION));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_MAX_QPS_PER_REGION_SERVER_PROP),
|
||||
HBASE_MAX_QPS_PER_REGION_SERVER_PROP, String.valueOf(DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY),
|
||||
HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS),
|
||||
HBASE_INDEX_QPS_ALLOCATOR_CLASS, String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS),
|
||||
HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_ZK_PATH_QPS_ROOT),
|
||||
HBASE_ZK_PATH_QPS_ROOT, String.valueOf(DEFAULT_HBASE_ZK_PATH_QPS_ROOT));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS),
|
||||
HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(DEFAULT_ZK_SESSION_TIMEOUT_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS),
|
||||
HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS),
|
||||
HBASE_INDEX_QPS_ALLOCATOR_CLASS, String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
|
||||
return config;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,203 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
|
||||
/**
|
||||
* Indexing related config
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String INDEX_TYPE_PROP = "hoodie.index.type";
|
||||
public static final String DEFAULT_INDEX_TYPE = HoodieIndex.IndexType.BLOOM.name();
|
||||
|
||||
// ***** Bloom Index configs *****
|
||||
public static final String BLOOM_FILTER_NUM_ENTRIES = "hoodie.index.bloom.num_entries";
|
||||
public static final String DEFAULT_BLOOM_FILTER_NUM_ENTRIES = "60000";
|
||||
public static final String BLOOM_FILTER_FPP = "hoodie.index.bloom.fpp";
|
||||
public static final String DEFAULT_BLOOM_FILTER_FPP = "0.000000001";
|
||||
public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism";
|
||||
// Disable explicit bloom index parallelism setting by default - hoodie auto computes
|
||||
public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0";
|
||||
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP =
|
||||
"hoodie.bloom.index.prune.by" + ".ranges";
|
||||
public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true";
|
||||
public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching";
|
||||
public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true";
|
||||
public static final String BLOOM_INDEX_TREE_BASED_FILTER_PROP = "hoodie.bloom.index.use.treebased.filter";
|
||||
public static final String DEFAULT_BLOOM_INDEX_TREE_BASED_FILTER = "true";
|
||||
// TODO: On by default. Once stable, we will remove the other mode.
|
||||
public static final String BLOOM_INDEX_BUCKETIZED_CHECKING_PROP = "hoodie.bloom.index.bucketized.checking";
|
||||
public static final String DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING = "true";
|
||||
// 1B bloom filter checks happen in 250 seconds. 500ms to read a bloom filter.
|
||||
// 10M checks in 2500ms, thus amortizing the cost of reading bloom filter across partitions.
|
||||
public static final String BLOOM_INDEX_KEYS_PER_BUCKET_PROP = "hoodie.bloom.index.keys.per.bucket";
|
||||
public static final String DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET = "10000000";
|
||||
|
||||
// ***** HBase Index Configs *****
|
||||
public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
|
||||
public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
|
||||
public static final String HBASE_ZK_ZNODEPARENT = "hoodie.index.hbase.zknode.path";
|
||||
public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
|
||||
public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
|
||||
public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
|
||||
public static final String DEFAULT_HBASE_BATCH_SIZE = "100";
|
||||
|
||||
|
||||
public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL =
|
||||
"hoodie.bloom.index.input.storage" + ".level";
|
||||
public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
|
||||
private HoodieIndexConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieIndexConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIndexType(HoodieIndex.IndexType indexType) {
|
||||
props.setProperty(INDEX_TYPE_PROP, indexType.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withHBaseIndexConfig(HoodieHBaseIndexConfig hBaseIndexConfig) {
|
||||
props.putAll(hBaseIndexConfig.getProps());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomFilterNumEntries(int numEntries) {
|
||||
props.setProperty(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomFilterFPP(double fpp) {
|
||||
props.setProperty(BLOOM_FILTER_FPP, String.valueOf(fpp));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkQuorum(String zkString) {
|
||||
props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkPort(int port) {
|
||||
props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkZnodeParent(String zkZnodeParent) {
|
||||
props.setProperty(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseTableName(String tableName) {
|
||||
props.setProperty(HBASE_TABLENAME_PROP, tableName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexParallelism(int parallelism) {
|
||||
props.setProperty(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexPruneByRanges(boolean pruneRanges) {
|
||||
props.setProperty(BLOOM_INDEX_PRUNE_BY_RANGES_PROP, String.valueOf(pruneRanges));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexUseCaching(boolean useCaching) {
|
||||
props.setProperty(BLOOM_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexTreebasedFilter(boolean useTreeFilter) {
|
||||
props.setProperty(BLOOM_INDEX_TREE_BASED_FILTER_PROP, String.valueOf(useTreeFilter));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexBucketizedChecking(boolean bucketizedChecking) {
|
||||
props.setProperty(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, String.valueOf(bucketizedChecking));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexKeysPerBucket(int keysPerBucket) {
|
||||
props.setProperty(BLOOM_INDEX_KEYS_PER_BUCKET_PROP, String.valueOf(keysPerBucket));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBloomIndexInputStorageLevel(String level) {
|
||||
props.setProperty(BLOOM_INDEX_INPUT_STORAGE_LEVEL, level);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieIndexConfig build() {
|
||||
HoodieIndexConfig config = new HoodieIndexConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP,
|
||||
DEFAULT_INDEX_TYPE);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES),
|
||||
BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), BLOOM_FILTER_FPP,
|
||||
DEFAULT_BLOOM_FILTER_FPP);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP),
|
||||
BLOOM_INDEX_PARALLELISM_PROP, DEFAULT_BLOOM_INDEX_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP),
|
||||
BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP),
|
||||
BLOOM_INDEX_USE_CACHING_PROP, DEFAULT_BLOOM_INDEX_USE_CACHING);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_INPUT_STORAGE_LEVEL),
|
||||
BLOOM_INDEX_INPUT_STORAGE_LEVEL, DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_TREE_BASED_FILTER_PROP),
|
||||
BLOOM_INDEX_TREE_BASED_FILTER_PROP, DEFAULT_BLOOM_INDEX_TREE_BASED_FILTER);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP),
|
||||
BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_KEYS_PER_BUCKET_PROP),
|
||||
BLOOM_INDEX_KEYS_PER_BUCKET_PROP, DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET);
|
||||
// Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
|
||||
HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,181 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.spark.SparkEnv;
|
||||
import org.apache.spark.util.Utils;
|
||||
|
||||
/**
|
||||
* Memory related config
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieMemoryConfig extends DefaultHoodieConfig {
|
||||
|
||||
// This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use
|
||||
// during merge. This makes it easier to scale this value as one increases the spark.executor.memory
|
||||
public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = "hoodie.memory.merge.fraction";
|
||||
// Default max memory fraction during hash-merge, excess spills to disk
|
||||
public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE = String.valueOf(0.6);
|
||||
public static final String MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP = "hoodie.memory.compaction.fraction";
|
||||
// Default max memory fraction during compaction, excess spills to disk
|
||||
public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION = String.valueOf(0.6);
|
||||
// Default memory size per compaction (used if SparkEnv is absent), excess spills to disk
|
||||
public static final long DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES =
|
||||
1024 * 1024 * 1024L; // 1GB
|
||||
// Property to set the max memory for merge
|
||||
public static final String MAX_MEMORY_FOR_MERGE_PROP = "hoodie.memory.merge.max.size";
|
||||
// Property to set the max memory for compaction
|
||||
public static final String MAX_MEMORY_FOR_COMPACTION_PROP = "hoodie.memory.compaction.max.size";
|
||||
// Property to set the max memory for dfs inputstream buffer size
|
||||
public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size";
|
||||
public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 16 * 1024 * 1024; // 16MB
|
||||
public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path";
|
||||
// Default file path prefix for spillable file
|
||||
public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/";
|
||||
|
||||
// Property to control how what fraction of the failed record, exceptions we report back to driver.
|
||||
public static final String WRITESTATUS_FAILURE_FRACTION_PROP = "hoodie.memory.writestatus.failure.fraction";
|
||||
// Default is 10%. If set to 100%, with lot of failures, this can cause memory pressure, cause OOMs and
|
||||
// mask actual data errors.
|
||||
public static final double DEFAULT_WRITESTATUS_FAILURE_FRACTION = 0.1;
|
||||
|
||||
private HoodieMemoryConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieMemoryConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemoryFractionPerPartitionMerge(double maxMemoryFractionPerPartitionMerge) {
|
||||
props.setProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP,
|
||||
String.valueOf(maxMemoryFractionPerPartitionMerge));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemoryFractionPerCompaction(double maxMemoryFractionPerCompaction) {
|
||||
props.setProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP,
|
||||
String.valueOf(maxMemoryFractionPerCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxDFSStreamBufferSize(int maxStreamBufferSize) {
|
||||
props.setProperty(MAX_DFS_STREAM_BUFFER_SIZE_PROP,
|
||||
String.valueOf(maxStreamBufferSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteStatusFailureFraction(double failureFraction) {
|
||||
props.setProperty(WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(failureFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Dynamic calculation of max memory to use for for spillable map. user.available.memory = spark.executor.memory *
|
||||
* (1 - spark.memory.fraction) spillable.available.memory = user.available.memory * hoodie.memory.fraction. Anytime
|
||||
* the spark.executor.memory or the spark.memory.fraction is changed, the memory used for spillable map changes
|
||||
* accordingly
|
||||
*/
|
||||
private long getMaxMemoryAllowedForMerge(String maxMemoryFraction) {
|
||||
final String SPARK_EXECUTOR_MEMORY_PROP = "spark.executor.memory";
|
||||
final String SPARK_EXECUTOR_MEMORY_FRACTION_PROP = "spark.memory.fraction";
|
||||
// This is hard-coded in spark code {@link
|
||||
// https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
|
||||
// spark/memory/UnifiedMemoryManager.scala#L231} so have to re-define this here
|
||||
final String DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION = "0.6";
|
||||
// This is hard-coded in spark code {@link
|
||||
// https://github.com/apache/spark/blob/576c43fb4226e4efa12189b41c3bc862019862c6/core/src/main/scala/org/apache/
|
||||
// spark/SparkContext.scala#L471} so have to re-define this here
|
||||
final String DEFAULT_SPARK_EXECUTOR_MEMORY_MB = "1024"; // in MB
|
||||
|
||||
if (SparkEnv.get() != null) {
|
||||
// 1 GB is the default conf used by Spark, look at SparkContext.scala
|
||||
long executorMemoryInBytes = Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP,
|
||||
DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024
|
||||
* 1024L;
|
||||
// 0.6 is the default value used by Spark,
|
||||
// look at {@link
|
||||
// https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
|
||||
double memoryFraction = Double
|
||||
.valueOf(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP,
|
||||
DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION));
|
||||
double maxMemoryFractionForMerge = Double.valueOf(maxMemoryFraction);
|
||||
double userAvailableMemory = executorMemoryInBytes * (1 - memoryFraction);
|
||||
long maxMemoryForMerge = (long) Math
|
||||
.floor(userAvailableMemory * maxMemoryFractionForMerge);
|
||||
return maxMemoryForMerge;
|
||||
} else {
|
||||
return DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieMemoryConfig build() {
|
||||
HoodieMemoryConfig config = new HoodieMemoryConfig(props);
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP),
|
||||
MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP,
|
||||
DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(MAX_MEMORY_FRACTION_FOR_MERGE_PROP),
|
||||
MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(MAX_MEMORY_FOR_MERGE_PROP),
|
||||
MAX_MEMORY_FOR_MERGE_PROP, String.valueOf(
|
||||
getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP))));
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP),
|
||||
MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf(
|
||||
getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP))));
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(MAX_DFS_STREAM_BUFFER_SIZE_PROP),
|
||||
MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE));
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(SPILLABLE_MAP_BASE_PATH_PROP),
|
||||
SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH);
|
||||
setDefaultOnCondition(props,
|
||||
!props.containsKey(WRITESTATUS_FAILURE_FRACTION_PROP),
|
||||
WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(DEFAULT_WRITESTATUS_FAILURE_FRACTION));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,120 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.hudi.metrics.MetricsReporterType;
|
||||
|
||||
/**
|
||||
* Fetch the configurations used by the Metrics system.
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieMetricsConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String METRIC_PREFIX = "hoodie.metrics";
|
||||
public static final String METRICS_ON = METRIC_PREFIX + ".on";
|
||||
public static final boolean DEFAULT_METRICS_ON = false;
|
||||
public static final String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
|
||||
public static final MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = MetricsReporterType
|
||||
.GRAPHITE;
|
||||
|
||||
// Graphite
|
||||
public static final String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
|
||||
public static final String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host";
|
||||
public static final String DEFAULT_GRAPHITE_SERVER_HOST = "localhost";
|
||||
|
||||
public static final String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port";
|
||||
public static final int DEFAULT_GRAPHITE_SERVER_PORT = 4756;
|
||||
|
||||
public static final String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
|
||||
|
||||
private HoodieMetricsConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieMetricsConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
public Builder on(boolean metricsOn) {
|
||||
props.setProperty(METRICS_ON, String.valueOf(metricsOn));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withReporterType(String reporterType) {
|
||||
props.setProperty(METRICS_REPORTER_TYPE, reporterType);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder toGraphiteHost(String host) {
|
||||
props.setProperty(GRAPHITE_SERVER_HOST, host);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder onGraphitePort(int port) {
|
||||
props.setProperty(GRAPHITE_SERVER_PORT, String.valueOf(port));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder usePrefix(String prefix) {
|
||||
props.setProperty(GRAPHITE_METRIC_PREFIX, prefix);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieMetricsConfig build() {
|
||||
HoodieMetricsConfig config = new HoodieMetricsConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON,
|
||||
String.valueOf(DEFAULT_METRICS_ON));
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE), METRICS_REPORTER_TYPE,
|
||||
DEFAULT_METRICS_REPORTER_TYPE.name());
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST), GRAPHITE_SERVER_HOST,
|
||||
DEFAULT_GRAPHITE_SERVER_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT,
|
||||
String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT,
|
||||
String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,145 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
/**
|
||||
* Storage related config
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieStorageConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String PARQUET_FILE_MAX_BYTES = "hoodie.parquet.max.file.size";
|
||||
public static final String DEFAULT_PARQUET_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
|
||||
public static final String PARQUET_BLOCK_SIZE_BYTES = "hoodie.parquet.block.size";
|
||||
public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = DEFAULT_PARQUET_FILE_MAX_BYTES;
|
||||
public static final String PARQUET_PAGE_SIZE_BYTES = "hoodie.parquet.page.size";
|
||||
public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
|
||||
// used to size log files
|
||||
public static final String LOGFILE_SIZE_MAX_BYTES = "hoodie.logfile.max.size";
|
||||
public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String
|
||||
.valueOf(1024 * 1024 * 1024); // 1 GB
|
||||
// used to size data blocks in log file
|
||||
public static final String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = "hoodie.logfile.data.block.max.size";
|
||||
public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256 * 1024 * 1024); // 256 MB
|
||||
public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio";
|
||||
// Default compression ratio for parquet
|
||||
public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1);
|
||||
public static final String PARQUET_COMPRESSION_CODEC = "hoodie.parquet.compression.codec";
|
||||
// Default compression codec for parquet
|
||||
public static final String DEFAULT_PARQUET_COMPRESSION_CODEC = "gzip";
|
||||
public static final String LOGFILE_TO_PARQUET_COMPRESSION_RATIO = "hoodie.logfile.to.parquet.compression.ratio";
|
||||
// Default compression ratio for log file to parquet, general 3x
|
||||
public static final String DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO = String.valueOf(0.35);
|
||||
|
||||
private HoodieStorageConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static HoodieStorageConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder limitFileSize(long maxFileSize) {
|
||||
props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetBlockSize(int blockSize) {
|
||||
props.setProperty(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetPageSize(int pageSize) {
|
||||
props.setProperty(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder logFileDataBlockMaxSize(int dataBlockSize) {
|
||||
props.setProperty(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, String.valueOf(dataBlockSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder logFileMaxSize(int logFileSize) {
|
||||
props.setProperty(LOGFILE_SIZE_MAX_BYTES, String.valueOf(logFileSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetCompressionRatio(double parquetCompressionRatio) {
|
||||
props.setProperty(PARQUET_COMPRESSION_RATIO, String.valueOf(parquetCompressionRatio));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetCompressionCodec(String parquetCompressionCodec) {
|
||||
props.setProperty(PARQUET_COMPRESSION_CODEC, parquetCompressionCodec);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressionRatio) {
|
||||
props.setProperty(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieStorageConfig build() {
|
||||
HoodieStorageConfig config = new HoodieStorageConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES),
|
||||
PARQUET_FILE_MAX_BYTES, DEFAULT_PARQUET_FILE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_BLOCK_SIZE_BYTES),
|
||||
PARQUET_BLOCK_SIZE_BYTES, DEFAULT_PARQUET_BLOCK_SIZE_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES),
|
||||
PARQUET_PAGE_SIZE_BYTES, DEFAULT_PARQUET_PAGE_SIZE_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES),
|
||||
LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOGFILE_SIZE_MAX_BYTES),
|
||||
LOGFILE_SIZE_MAX_BYTES, DEFAULT_LOGFILE_SIZE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO),
|
||||
PARQUET_COMPRESSION_RATIO, DEFAULT_STREAM_COMPRESSION_RATIO);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_CODEC),
|
||||
PARQUET_COMPRESSION_CODEC, DEFAULT_PARQUET_COMPRESSION_CODEC);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOGFILE_TO_PARQUET_COMPRESSION_RATIO),
|
||||
LOGFILE_TO_PARQUET_COMPRESSION_RATIO, DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO);
|
||||
return config;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,727 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
import org.apache.hudi.HoodieWriteClient;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.util.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.io.compact.strategy.CompactionStrategy;
|
||||
import org.apache.hudi.metrics.MetricsReporterType;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
|
||||
/**
|
||||
* Class storing configs for the {@link HoodieWriteClient}
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieWriteConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static final String TABLE_NAME = "hoodie.table.name";
|
||||
private static final String BASE_PATH_PROP = "hoodie.base.path";
|
||||
private static final String AVRO_SCHEMA = "hoodie.avro.schema";
|
||||
private static final String DEFAULT_PARALLELISM = "1500";
|
||||
private static final String INSERT_PARALLELISM = "hoodie.insert.shuffle.parallelism";
|
||||
private static final String BULKINSERT_PARALLELISM = "hoodie.bulkinsert.shuffle.parallelism";
|
||||
private static final String UPSERT_PARALLELISM = "hoodie.upsert.shuffle.parallelism";
|
||||
private static final String WRITE_BUFFER_LIMIT_BYTES = "hoodie.write.buffer.limit.bytes";
|
||||
private static final String DEFAULT_WRITE_BUFFER_LIMIT_BYTES = String.valueOf(4 * 1024 * 1024);
|
||||
private static final String COMBINE_BEFORE_INSERT_PROP = "hoodie.combine.before.insert";
|
||||
private static final String DEFAULT_COMBINE_BEFORE_INSERT = "false";
|
||||
private static final String COMBINE_BEFORE_UPSERT_PROP = "hoodie.combine.before.upsert";
|
||||
private static final String DEFAULT_COMBINE_BEFORE_UPSERT = "true";
|
||||
private static final String WRITE_STATUS_STORAGE_LEVEL = "hoodie.write.status.storage.level";
|
||||
private static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
private static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit";
|
||||
private static final String DEFAULT_HOODIE_AUTO_COMMIT = "true";
|
||||
private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP =
|
||||
"hoodie.assume.date" + ".partitioning";
|
||||
private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
|
||||
private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
|
||||
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
|
||||
private static final String FINALIZE_WRITE_PARALLELISM = "hoodie.finalize.write.parallelism";
|
||||
private static final String DEFAULT_FINALIZE_WRITE_PARALLELISM = DEFAULT_PARALLELISM;
|
||||
|
||||
private static final String EMBEDDED_TIMELINE_SERVER_ENABLED = "hoodie.embed.timeline.server";
|
||||
private static final String DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED = "false";
|
||||
|
||||
private static final String FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP = "hoodie.fail.on.timeline.archiving";
|
||||
private static final String DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED = "true";
|
||||
// time between successive attempts to ensure written data's metadata is consistent on storage
|
||||
private static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP =
|
||||
"hoodie.consistency.check.initial_interval_ms";
|
||||
private static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 2000L;
|
||||
|
||||
// max interval time
|
||||
private static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = "hoodie.consistency.check.max_interval_ms";
|
||||
private static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = 300000L;
|
||||
|
||||
// maximum number of checks, for consistency of written data. Will wait upto 256 Secs
|
||||
private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
|
||||
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 7;
|
||||
|
||||
private ConsistencyGuardConfig consistencyGuardConfig;
|
||||
|
||||
// Hoodie Write Client transparently rewrites File System View config when embedded mode is enabled
|
||||
// We keep track of original config and rewritten config
|
||||
private final FileSystemViewStorageConfig clientSpecifiedViewStorageConfig;
|
||||
private FileSystemViewStorageConfig viewStorageConfig;
|
||||
|
||||
private HoodieWriteConfig(Properties props) {
|
||||
super(props);
|
||||
Properties newProps = new Properties();
|
||||
newProps.putAll(props);
|
||||
this.consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().fromProperties(newProps).build();
|
||||
this.clientSpecifiedViewStorageConfig = FileSystemViewStorageConfig.newBuilder().fromProperties(newProps).build();
|
||||
this.viewStorageConfig = clientSpecifiedViewStorageConfig;
|
||||
}
|
||||
|
||||
public static HoodieWriteConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
/**
|
||||
* base properties
|
||||
**/
|
||||
public String getBasePath() {
|
||||
return props.getProperty(BASE_PATH_PROP);
|
||||
}
|
||||
|
||||
public String getSchema() {
|
||||
return props.getProperty(AVRO_SCHEMA);
|
||||
}
|
||||
|
||||
public String getTableName() {
|
||||
return props.getProperty(TABLE_NAME);
|
||||
}
|
||||
|
||||
public Boolean shouldAutoCommit() {
|
||||
return Boolean.parseBoolean(props.getProperty(HOODIE_AUTO_COMMIT_PROP));
|
||||
}
|
||||
|
||||
public Boolean shouldAssumeDatePartitioning() {
|
||||
return Boolean.parseBoolean(props.getProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP));
|
||||
}
|
||||
|
||||
public int getBulkInsertShuffleParallelism() {
|
||||
return Integer.parseInt(props.getProperty(BULKINSERT_PARALLELISM));
|
||||
}
|
||||
|
||||
public int getInsertShuffleParallelism() {
|
||||
return Integer.parseInt(props.getProperty(INSERT_PARALLELISM));
|
||||
}
|
||||
|
||||
public int getUpsertShuffleParallelism() {
|
||||
return Integer.parseInt(props.getProperty(UPSERT_PARALLELISM));
|
||||
}
|
||||
|
||||
public int getWriteBufferLimitBytes() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES));
|
||||
}
|
||||
|
||||
public boolean shouldCombineBeforeInsert() {
|
||||
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_INSERT_PROP));
|
||||
}
|
||||
|
||||
public boolean shouldCombineBeforeUpsert() {
|
||||
return Boolean.parseBoolean(props.getProperty(COMBINE_BEFORE_UPSERT_PROP));
|
||||
}
|
||||
|
||||
public StorageLevel getWriteStatusStorageLevel() {
|
||||
return StorageLevel.fromString(props.getProperty(WRITE_STATUS_STORAGE_LEVEL));
|
||||
}
|
||||
|
||||
public String getWriteStatusClassName() {
|
||||
return props.getProperty(HOODIE_WRITE_STATUS_CLASS_PROP);
|
||||
}
|
||||
|
||||
public int getFinalizeWriteParallelism() {
|
||||
return Integer.parseInt(props.getProperty(FINALIZE_WRITE_PARALLELISM));
|
||||
}
|
||||
|
||||
public boolean isEmbeddedTimelineServerEnabled() {
|
||||
return Boolean.parseBoolean(props.getProperty(EMBEDDED_TIMELINE_SERVER_ENABLED));
|
||||
}
|
||||
|
||||
public boolean isFailOnTimelineArchivingEnabled() {
|
||||
return Boolean.parseBoolean(props.getProperty(FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP));
|
||||
}
|
||||
|
||||
public int getMaxConsistencyChecks() {
|
||||
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECKS_PROP));
|
||||
}
|
||||
|
||||
public int getInitialConsistencyCheckIntervalMs() {
|
||||
return Integer.parseInt(props.getProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
|
||||
}
|
||||
|
||||
public int getMaxConsistencyCheckIntervalMs() {
|
||||
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
|
||||
}
|
||||
|
||||
/**
|
||||
* compaction properties
|
||||
**/
|
||||
public HoodieCleaningPolicy getCleanerPolicy() {
|
||||
return HoodieCleaningPolicy
|
||||
.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_POLICY_PROP));
|
||||
}
|
||||
|
||||
public int getCleanerFileVersionsRetained() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
|
||||
}
|
||||
|
||||
public int getCleanerCommitsRetained() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
|
||||
}
|
||||
|
||||
public int getMaxCommitsToKeep() {
|
||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
|
||||
}
|
||||
|
||||
public int getMinCommitsToKeep() {
|
||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP));
|
||||
}
|
||||
|
||||
public int getParquetSmallFileLimit() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES));
|
||||
}
|
||||
|
||||
public int getCopyOnWriteInsertSplitSize() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
|
||||
}
|
||||
|
||||
public int getCopyOnWriteRecordSizeEstimate() {
|
||||
return Integer.parseInt(
|
||||
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE));
|
||||
}
|
||||
|
||||
public boolean shouldAutoTuneInsertSplits() {
|
||||
return Boolean.parseBoolean(
|
||||
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS));
|
||||
}
|
||||
|
||||
public int getCleanerParallelism() {
|
||||
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_PARALLELISM));
|
||||
}
|
||||
|
||||
public boolean isAutoClean() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.AUTO_CLEAN_PROP));
|
||||
}
|
||||
|
||||
public boolean isInlineCompaction() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_PROP));
|
||||
}
|
||||
|
||||
public int getInlineCompactDeltaCommitMax() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
|
||||
}
|
||||
|
||||
public CompactionStrategy getCompactionStrategy() {
|
||||
return ReflectionUtils
|
||||
.loadClass(props.getProperty(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
|
||||
}
|
||||
|
||||
public Long getTargetIOPerCompactionInMB() {
|
||||
return Long
|
||||
.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
|
||||
}
|
||||
|
||||
public Boolean getCompactionLazyBlockReadEnabled() {
|
||||
return Boolean
|
||||
.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP));
|
||||
}
|
||||
|
||||
public Boolean getCompactionReverseLogReadEnabled() {
|
||||
return Boolean.valueOf(
|
||||
props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP));
|
||||
}
|
||||
|
||||
public String getPayloadClass() {
|
||||
return props.getProperty(HoodieCompactionConfig.PAYLOAD_CLASS_PROP);
|
||||
}
|
||||
|
||||
public int getTargetPartitionsPerDayBasedCompaction() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP));
|
||||
}
|
||||
|
||||
public int getCommitArchivalBatchSize() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieCompactionConfig.COMMITS_ARCHIVAL_BATCH_SIZE_PROP));
|
||||
}
|
||||
|
||||
/**
|
||||
* index properties
|
||||
**/
|
||||
public HoodieIndex.IndexType getIndexType() {
|
||||
return HoodieIndex.IndexType.valueOf(props.getProperty(HoodieIndexConfig.INDEX_TYPE_PROP));
|
||||
}
|
||||
|
||||
public int getBloomFilterNumEntries() {
|
||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES));
|
||||
}
|
||||
|
||||
public double getBloomFilterFPP() {
|
||||
return Double.parseDouble(props.getProperty(HoodieIndexConfig.BLOOM_FILTER_FPP));
|
||||
}
|
||||
|
||||
public String getHbaseZkQuorum() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HBASE_ZKQUORUM_PROP);
|
||||
}
|
||||
|
||||
public int getHbaseZkPort() {
|
||||
return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_ZKPORT_PROP));
|
||||
}
|
||||
|
||||
public String getHBaseZkZnodeParent() {
|
||||
return props.getProperty(HoodieIndexConfig.HBASE_ZK_ZNODEPARENT);
|
||||
}
|
||||
|
||||
public String getHbaseTableName() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HBASE_TABLENAME_PROP);
|
||||
}
|
||||
|
||||
public int getHbaseIndexGetBatchSize() {
|
||||
return Integer.valueOf(props.getProperty(HoodieHBaseIndexConfig.HBASE_GET_BATCH_SIZE_PROP));
|
||||
}
|
||||
|
||||
public int getHbaseIndexPutBatchSize() {
|
||||
return Integer.valueOf(props.getProperty(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_PROP));
|
||||
}
|
||||
|
||||
public Boolean getHbaseIndexPutBatchSizeAutoCompute() {
|
||||
return Boolean.valueOf(props.getProperty(HoodieHBaseIndexConfig.HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP));
|
||||
}
|
||||
|
||||
public String getHBaseQPSResourceAllocatorClass() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HBASE_INDEX_QPS_ALLOCATOR_CLASS);
|
||||
}
|
||||
|
||||
public String getHBaseQPSZKnodePath() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HBASE_ZK_PATH_QPS_ROOT);
|
||||
}
|
||||
|
||||
public String getHBaseZkZnodeSessionTimeout() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS);
|
||||
}
|
||||
|
||||
public String getHBaseZkZnodeConnectionTimeout() {
|
||||
return props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS);
|
||||
}
|
||||
|
||||
public boolean getHBaseIndexShouldComputeQPSDynamically() {
|
||||
return Boolean.valueOf(props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY));
|
||||
}
|
||||
|
||||
public int getHBaseIndexDesiredPutsTime() {
|
||||
return Integer.valueOf(props.getProperty(HoodieHBaseIndexConfig.HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS));
|
||||
}
|
||||
|
||||
/**
|
||||
* Fraction of the global share of QPS that should be allocated to this job.
|
||||
* Let's say there are 3 jobs which have input size in terms of number of rows
|
||||
* required for HbaseIndexing as x, 2x, 3x respectively. Then this fraction for
|
||||
* the jobs would be (0.17) 1/6, 0.33 (2/6) and 0.5 (3/6) respectively.
|
||||
*/
|
||||
public float getHbaseIndexQPSFraction() {
|
||||
return Float.parseFloat(props.getProperty(HoodieHBaseIndexConfig.HBASE_QPS_FRACTION_PROP));
|
||||
}
|
||||
|
||||
public float getHBaseIndexMinQPSFraction() {
|
||||
return Float.parseFloat(props.getProperty(HoodieHBaseIndexConfig.HBASE_MIN_QPS_FRACTION_PROP));
|
||||
}
|
||||
|
||||
public float getHBaseIndexMaxQPSFraction() {
|
||||
return Float.parseFloat(props.getProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_FRACTION_PROP));
|
||||
}
|
||||
|
||||
/**
|
||||
* This should be same across various jobs. This is intended to limit the aggregate
|
||||
* QPS generated across various Hoodie jobs to an Hbase Region Server
|
||||
*/
|
||||
public int getHbaseIndexMaxQPSPerRegionServer() {
|
||||
return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP));
|
||||
}
|
||||
|
||||
public int getBloomIndexParallelism() {
|
||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PARALLELISM_PROP));
|
||||
}
|
||||
|
||||
public boolean getBloomIndexPruneByRanges() {
|
||||
return Boolean
|
||||
.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PRUNE_BY_RANGES_PROP));
|
||||
}
|
||||
|
||||
public boolean getBloomIndexUseCaching() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_USE_CACHING_PROP));
|
||||
}
|
||||
|
||||
public boolean useBloomIndexTreebasedFilter() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_TREE_BASED_FILTER_PROP));
|
||||
}
|
||||
|
||||
public boolean useBloomIndexBucketizedChecking() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_BUCKETIZED_CHECKING_PROP));
|
||||
}
|
||||
|
||||
public int getBloomIndexKeysPerBucket() {
|
||||
return Integer.parseInt(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_KEYS_PER_BUCKET_PROP));
|
||||
}
|
||||
|
||||
public StorageLevel getBloomIndexInputStorageLevel() {
|
||||
return StorageLevel
|
||||
.fromString(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
|
||||
}
|
||||
|
||||
/**
|
||||
* storage properties
|
||||
**/
|
||||
public long getParquetMaxFileSize() {
|
||||
return Long.parseLong(props.getProperty(HoodieStorageConfig.PARQUET_FILE_MAX_BYTES));
|
||||
}
|
||||
|
||||
public int getParquetBlockSize() {
|
||||
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_BLOCK_SIZE_BYTES));
|
||||
}
|
||||
|
||||
public int getParquetPageSize() {
|
||||
return Integer.parseInt(props.getProperty(HoodieStorageConfig.PARQUET_PAGE_SIZE_BYTES));
|
||||
}
|
||||
|
||||
public int getLogFileDataBlockMaxSize() {
|
||||
return Integer
|
||||
.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES));
|
||||
}
|
||||
|
||||
public int getLogFileMaxSize() {
|
||||
return Integer.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_SIZE_MAX_BYTES));
|
||||
}
|
||||
|
||||
public double getParquetCompressionRatio() {
|
||||
return Double.valueOf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO));
|
||||
}
|
||||
|
||||
public CompressionCodecName getParquetCompressionCodec() {
|
||||
return CompressionCodecName.fromConf(props.getProperty(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC));
|
||||
}
|
||||
|
||||
public double getLogFileToParquetCompressionRatio() {
|
||||
return Double.valueOf(props.getProperty(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO));
|
||||
}
|
||||
|
||||
/**
|
||||
* metrics properties
|
||||
**/
|
||||
public boolean isMetricsOn() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieMetricsConfig.METRICS_ON));
|
||||
}
|
||||
|
||||
public MetricsReporterType getMetricsReporterType() {
|
||||
return MetricsReporterType
|
||||
.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
|
||||
}
|
||||
|
||||
public String getGraphiteServerHost() {
|
||||
return props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_HOST);
|
||||
}
|
||||
|
||||
public int getGraphiteServerPort() {
|
||||
return Integer.parseInt(props.getProperty(HoodieMetricsConfig.GRAPHITE_SERVER_PORT));
|
||||
}
|
||||
|
||||
public String getGraphiteMetricPrefix() {
|
||||
return props.getProperty(HoodieMetricsConfig.GRAPHITE_METRIC_PREFIX);
|
||||
}
|
||||
|
||||
/**
|
||||
* memory configs
|
||||
*/
|
||||
public Double getMaxMemoryFractionPerPartitionMerge() {
|
||||
return Double.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP));
|
||||
}
|
||||
|
||||
public Double getMaxMemoryFractionPerCompaction() {
|
||||
return Double
|
||||
.valueOf(
|
||||
props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP));
|
||||
}
|
||||
|
||||
public Long getMaxMemoryPerPartitionMerge() {
|
||||
return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP));
|
||||
}
|
||||
|
||||
public Long getMaxMemoryPerCompaction() {
|
||||
return Long.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP));
|
||||
}
|
||||
|
||||
public int getMaxDFSStreamBufferSize() {
|
||||
return Integer.valueOf(props.getProperty(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE_PROP));
|
||||
}
|
||||
|
||||
public String getSpillableMapBasePath() {
|
||||
return props.getProperty(HoodieMemoryConfig.SPILLABLE_MAP_BASE_PATH_PROP);
|
||||
}
|
||||
|
||||
public double getWriteStatusFailureFraction() {
|
||||
return Double.valueOf(props.getProperty(HoodieMemoryConfig.WRITESTATUS_FAILURE_FRACTION_PROP));
|
||||
}
|
||||
|
||||
public ConsistencyGuardConfig getConsistencyGuardConfig() {
|
||||
return consistencyGuardConfig;
|
||||
}
|
||||
|
||||
public void setConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) {
|
||||
this.consistencyGuardConfig = consistencyGuardConfig;
|
||||
}
|
||||
|
||||
public FileSystemViewStorageConfig getViewStorageConfig() {
|
||||
return viewStorageConfig;
|
||||
}
|
||||
|
||||
public void setViewStorageConfig(FileSystemViewStorageConfig viewStorageConfig) {
|
||||
this.viewStorageConfig = viewStorageConfig;
|
||||
}
|
||||
|
||||
public void resetViewStorageConfig() {
|
||||
this.setViewStorageConfig(getClientSpecifiedViewStorageConfig());
|
||||
}
|
||||
|
||||
public FileSystemViewStorageConfig getClientSpecifiedViewStorageConfig() {
|
||||
return clientSpecifiedViewStorageConfig;
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private boolean isIndexConfigSet = false;
|
||||
private boolean isStorageConfigSet = false;
|
||||
private boolean isCompactionConfigSet = false;
|
||||
private boolean isMetricsConfigSet = false;
|
||||
private boolean isMemoryConfigSet = false;
|
||||
private boolean isViewConfigSet = false;
|
||||
private boolean isConsistencyGuardSet = false;
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
this.props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromInputStream(InputStream inputStream) throws IOException {
|
||||
try {
|
||||
this.props.load(inputStream);
|
||||
return this;
|
||||
} finally {
|
||||
inputStream.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder withProps(Map kvprops) {
|
||||
props.putAll(kvprops);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withPath(String basePath) {
|
||||
props.setProperty(BASE_PATH_PROP, basePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSchema(String schemaStr) {
|
||||
props.setProperty(AVRO_SCHEMA, schemaStr);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder forTable(String tableName) {
|
||||
props.setProperty(TABLE_NAME, tableName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBulkInsertParallelism(int bulkInsertParallelism) {
|
||||
props.setProperty(BULKINSERT_PARALLELISM, String.valueOf(bulkInsertParallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withParallelism(int insertShuffleParallelism, int upsertShuffleParallelism) {
|
||||
props.setProperty(INSERT_PARALLELISM, String.valueOf(insertShuffleParallelism));
|
||||
props.setProperty(UPSERT_PARALLELISM, String.valueOf(upsertShuffleParallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteBufferLimitBytes(int writeBufferLimit) {
|
||||
props.setProperty(WRITE_BUFFER_LIMIT_BYTES, String.valueOf(writeBufferLimit));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder combineInput(boolean onInsert, boolean onUpsert) {
|
||||
props.setProperty(COMBINE_BEFORE_INSERT_PROP, String.valueOf(onInsert));
|
||||
props.setProperty(COMBINE_BEFORE_UPSERT_PROP, String.valueOf(onUpsert));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteStatusStorageLevel(String level) {
|
||||
props.setProperty(WRITE_STATUS_STORAGE_LEVEL, level);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIndexConfig(HoodieIndexConfig indexConfig) {
|
||||
props.putAll(indexConfig.getProps());
|
||||
isIndexConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withStorageConfig(HoodieStorageConfig storageConfig) {
|
||||
props.putAll(storageConfig.getProps());
|
||||
isStorageConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionConfig(HoodieCompactionConfig compactionConfig) {
|
||||
props.putAll(compactionConfig.getProps());
|
||||
isCompactionConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMetricsConfig(HoodieMetricsConfig metricsConfig) {
|
||||
props.putAll(metricsConfig.getProps());
|
||||
isMetricsConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMemoryConfig(HoodieMemoryConfig memoryConfig) {
|
||||
props.putAll(memoryConfig.getProps());
|
||||
isMemoryConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAutoCommit(boolean autoCommit) {
|
||||
props.setProperty(HOODIE_AUTO_COMMIT_PROP, String.valueOf(autoCommit));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
|
||||
props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP,
|
||||
String.valueOf(assumeDatePartitioning));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteStatusClass(Class<? extends WriteStatus> writeStatusClass) {
|
||||
props.setProperty(HOODIE_WRITE_STATUS_CLASS_PROP, writeStatusClass.getName());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFileSystemViewConfig(FileSystemViewStorageConfig viewStorageConfig) {
|
||||
props.putAll(viewStorageConfig.getProps());
|
||||
isViewConfigSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withConsistencyGuardConfig(ConsistencyGuardConfig consistencyGuardConfig) {
|
||||
props.putAll(consistencyGuardConfig.getProps());
|
||||
isConsistencyGuardSet = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFinalizeWriteParallelism(int parallelism) {
|
||||
props.setProperty(FINALIZE_WRITE_PARALLELISM, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withEmbeddedTimelineServerEnabled(boolean enabled) {
|
||||
props.setProperty(EMBEDDED_TIMELINE_SERVER_ENABLED, String.valueOf(enabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteConfig build() {
|
||||
// Check for mandatory properties
|
||||
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM,
|
||||
DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM),
|
||||
BULKINSERT_PARALLELISM, DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM,
|
||||
DEFAULT_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP),
|
||||
COMBINE_BEFORE_INSERT_PROP, DEFAULT_COMBINE_BEFORE_INSERT);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_UPSERT_PROP),
|
||||
COMBINE_BEFORE_UPSERT_PROP, DEFAULT_COMBINE_BEFORE_UPSERT);
|
||||
setDefaultOnCondition(props, !props.containsKey(WRITE_STATUS_STORAGE_LEVEL),
|
||||
WRITE_STATUS_STORAGE_LEVEL, DEFAULT_WRITE_STATUS_STORAGE_LEVEL);
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP),
|
||||
HOODIE_AUTO_COMMIT_PROP, DEFAULT_HOODIE_AUTO_COMMIT);
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP),
|
||||
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP),
|
||||
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM),
|
||||
FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(EMBEDDED_TIMELINE_SERVER_ENABLED),
|
||||
EMBEDDED_TIMELINE_SERVER_ENABLED, DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED);
|
||||
setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
|
||||
INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
|
||||
MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECKS_PROP),
|
||||
MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
|
||||
setDefaultOnCondition(props, !props.containsKey(FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP),
|
||||
FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP, DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED);
|
||||
|
||||
// Make sure the props is propagated
|
||||
setDefaultOnCondition(props, !isIndexConfigSet,
|
||||
HoodieIndexConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isStorageConfigSet,
|
||||
HoodieStorageConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isCompactionConfigSet,
|
||||
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isMetricsConfigSet,
|
||||
HoodieMetricsConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isMemoryConfigSet,
|
||||
HoodieMemoryConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isViewConfigSet,
|
||||
FileSystemViewStorageConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !isConsistencyGuardSet,
|
||||
ConsistencyGuardConfig.newBuilder().fromProperties(props).build());
|
||||
|
||||
// Build WriteConfig at the end
|
||||
HoodieWriteConfig config = new HoodieWriteConfig(props);
|
||||
Preconditions.checkArgument(config.getBasePath() != null);
|
||||
return config;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.exception;
|
||||
|
||||
/**
|
||||
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta
|
||||
* commit </p>
|
||||
*/
|
||||
public class HoodieAppendException extends HoodieException {
|
||||
|
||||
public HoodieAppendException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieAppendException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.exception;
|
||||
|
||||
/**
|
||||
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a Commit
|
||||
* </p>
|
||||
*/
|
||||
public class HoodieCommitException extends HoodieException {
|
||||
|
||||
public HoodieCommitException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public HoodieCommitException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.exception;
|
||||
|
||||
public class HoodieCompactionException extends HoodieException {
|
||||
|
||||
public HoodieCompactionException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
public HoodieCompactionException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,36 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.exception;
|
||||
|
||||
|
||||
/**
|
||||
* <p> Exception thrown when dependent system is not available </p>
|
||||
*/
|
||||
public class HoodieDependentSystemUnavailableException extends HoodieException {
|
||||
|
||||
public static final String HBASE = "HBASE";
|
||||
|
||||
public HoodieDependentSystemUnavailableException(String system, String connectURL) {
|
||||
super(getLogMessage(system, connectURL));
|
||||
}
|
||||
|
||||
private static String getLogMessage(String system, String connectURL) {
|
||||
return "System " + system + " unavailable. Tried to connect to " + connectURL;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.exception;
|
||||
|
||||
/**
|
||||
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a bulk
|
||||
* insert </p>
|
||||
*/
|
||||
public class HoodieInsertException extends HoodieException {
|
||||
|
||||
public HoodieInsertException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.exception;
|
||||
|
||||
public class HoodieRollbackException extends HoodieException {
|
||||
|
||||
public HoodieRollbackException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieRollbackException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.exception;
|
||||
|
||||
public class HoodieSavepointException extends HoodieException {
|
||||
|
||||
public HoodieSavepointException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieSavepointException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.exception;
|
||||
|
||||
/**
|
||||
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a
|
||||
* incremental upsert </p>
|
||||
*/
|
||||
public class HoodieUpsertException extends HoodieException {
|
||||
|
||||
public HoodieUpsertException(String msg, Throwable e) {
|
||||
super(msg, e);
|
||||
}
|
||||
|
||||
public HoodieUpsertException(String msg) {
|
||||
super(msg);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.func;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
|
||||
/**
|
||||
* Map function that handles a sorted stream of HoodieRecords
|
||||
*/
|
||||
public class BulkInsertMapFunction<T extends HoodieRecordPayload> implements
|
||||
Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
|
||||
|
||||
private String commitTime;
|
||||
private HoodieWriteConfig config;
|
||||
private HoodieTable<T> hoodieTable;
|
||||
private List<String> fileIDPrefixes;
|
||||
|
||||
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config,
|
||||
HoodieTable<T> hoodieTable, List<String> fileIDPrefixes) {
|
||||
this.commitTime = commitTime;
|
||||
this.config = config;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileIDPrefixes = fileIDPrefixes;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> call(Integer partition, Iterator<HoodieRecord<T>> sortedRecordItr) {
|
||||
return new CopyOnWriteLazyInsertIterable<>(sortedRecordItr, config, commitTime, hoodieTable,
|
||||
fileIDPrefixes.get(partition));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,172 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.func;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
/**
|
||||
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
|
||||
* files.
|
||||
*/
|
||||
public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extends
|
||||
LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
|
||||
|
||||
protected final HoodieWriteConfig hoodieConfig;
|
||||
protected final String commitTime;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
protected final String idPrefix;
|
||||
protected int numFilesWritten;
|
||||
|
||||
public CopyOnWriteLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
||||
String commitTime, HoodieTable<T> hoodieTable, String idPrefix) {
|
||||
super(sortedRecordItr);
|
||||
this.hoodieConfig = config;
|
||||
this.commitTime = commitTime;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.idPrefix = idPrefix;
|
||||
this.numFilesWritten = 0;
|
||||
}
|
||||
|
||||
// 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 Option<IndexedRecord> insertValue;
|
||||
// It caches the exception seen while fetching insert value.
|
||||
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 = Option.of(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some
|
||||
* expensive operations of transformation to the reader thread.
|
||||
*/
|
||||
static <T extends HoodieRecordPayload> Function<HoodieRecord<T>,
|
||||
HoodieInsertValueGenResult<HoodieRecord>> getTransformFunction(Schema schema) {
|
||||
return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void start() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<WriteStatus> computeNext() {
|
||||
// Executor service used for launching writer thread.
|
||||
BoundedInMemoryExecutor<HoodieRecord<T>,
|
||||
HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> bufferedIteratorExecutor = null;
|
||||
try {
|
||||
final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
|
||||
bufferedIteratorExecutor =
|
||||
new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr,
|
||||
getInsertHandler(), getTransformFunction(schema));
|
||||
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
|
||||
assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
|
||||
return result;
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
} finally {
|
||||
if (null != bufferedIteratorExecutor) {
|
||||
bufferedIteratorExecutor.shutdownNow();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void end() {
|
||||
}
|
||||
|
||||
protected String getNextFileId(String idPfx) {
|
||||
return String.format("%s-%d", idPfx, numFilesWritten++);
|
||||
}
|
||||
|
||||
protected CopyOnWriteInsertHandler getInsertHandler() {
|
||||
return new CopyOnWriteInsertHandler();
|
||||
}
|
||||
|
||||
/**
|
||||
* Consumes stream of hoodie records from in-memory queue and
|
||||
* writes to one or more create-handles
|
||||
*/
|
||||
protected class CopyOnWriteInsertHandler extends
|
||||
BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> {
|
||||
|
||||
protected final List<WriteStatus> statuses = new ArrayList<>();
|
||||
protected HoodieWriteHandle handle;
|
||||
|
||||
@Override
|
||||
protected void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> payload) {
|
||||
final HoodieRecord insertPayload = payload.record;
|
||||
// lazily initialize the handle, for the first time
|
||||
if (handle == null) {
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(),
|
||||
getNextFileId(idPrefix));
|
||||
}
|
||||
|
||||
if (handle.canWrite(payload.record)) {
|
||||
// write the payload, if the handle has capacity
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception);
|
||||
} else {
|
||||
// handle is full.
|
||||
statuses.add(handle.close());
|
||||
// Need to handle the rejected payload & open new handle
|
||||
handle = new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, insertPayload.getPartitionPath(),
|
||||
getNextFileId(idPrefix));
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload.
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void finish() {
|
||||
if (handle != null) {
|
||||
statuses.add(handle.close());
|
||||
}
|
||||
handle = null;
|
||||
assert statuses.size() > 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<WriteStatus> getResult() {
|
||||
return statuses;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,131 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.func;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
/**
|
||||
* (NOTE: Adapted from Apache SystemML) This class is a generic base class for lazy, single pass
|
||||
* inputItr classes in order to simplify the implementation of lazy iterators for mapPartitions use
|
||||
* cases. Note [SPARK-3369], which gives the reasons for backwards compatibility with regard to the
|
||||
* iterable API despite Spark's single pass nature.
|
||||
* <p>
|
||||
* Provide a way to obtain a inputItr of type O (output), out of an inputItr of type I (input)
|
||||
* <p>
|
||||
* Things to remember: - Assumes Spark calls hasNext() to check for elements, before calling next()
|
||||
* to obtain them - Assumes hasNext() gets called atleast once. - Concrete Implementation is
|
||||
* responsible for calling inputIterator.next() and doing the processing in computeNext()
|
||||
*/
|
||||
public abstract class LazyIterableIterator<I, O> implements Iterable<O>, Iterator<O> {
|
||||
|
||||
protected Iterator<I> inputItr = null;
|
||||
private boolean consumed = false;
|
||||
private boolean startCalled = false;
|
||||
private boolean endCalled = false;
|
||||
|
||||
public LazyIterableIterator(Iterator<I> in) {
|
||||
inputItr = in;
|
||||
}
|
||||
|
||||
/**
|
||||
* Called once, before any elements are processed
|
||||
*/
|
||||
protected abstract void start();
|
||||
|
||||
/**
|
||||
* Block computation to be overwritten by sub classes.
|
||||
*/
|
||||
protected abstract O computeNext();
|
||||
|
||||
|
||||
/**
|
||||
* Called once, after all elements are processed.
|
||||
*/
|
||||
protected abstract void end();
|
||||
|
||||
//////////////////
|
||||
// iterable implementation
|
||||
|
||||
private void invokeStartIfNeeded() {
|
||||
if (!startCalled) {
|
||||
startCalled = true;
|
||||
try {
|
||||
start();
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException("Error in start()");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void invokeEndIfNeeded() {
|
||||
// make the calls out to begin() & end()
|
||||
if (!endCalled) {
|
||||
endCalled = true;
|
||||
// if we are out of elements, and end has not been called yet
|
||||
try {
|
||||
end();
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException("Error in end()");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<O> iterator() {
|
||||
//check for consumed inputItr
|
||||
if (consumed) {
|
||||
throw new RuntimeException("Invalid repeated inputItr consumption.");
|
||||
}
|
||||
|
||||
//hand out self as inputItr exactly once (note: do not hand out the input
|
||||
//inputItr since it is consumed by the self inputItr implementation)
|
||||
consumed = true;
|
||||
return this;
|
||||
}
|
||||
|
||||
//////////////////
|
||||
// inputItr implementation
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
boolean ret = inputItr.hasNext();
|
||||
// make sure, there is exactly one call to start()
|
||||
invokeStartIfNeeded();
|
||||
if (!ret) {
|
||||
// if we are out of elements, and end has not been called yet
|
||||
invokeEndIfNeeded();
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
public O next() {
|
||||
try {
|
||||
return computeNext();
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove() {
|
||||
throw new RuntimeException("Unsupported remove operation.");
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,72 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.func;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieAppendHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
/**
|
||||
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
|
||||
* log files.
|
||||
*/
|
||||
public class MergeOnReadLazyInsertIterable<T extends HoodieRecordPayload> extends
|
||||
CopyOnWriteLazyInsertIterable<T> {
|
||||
|
||||
public MergeOnReadLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
|
||||
String commitTime, HoodieTable<T> hoodieTable, String idPfx) {
|
||||
super(sortedRecordItr, config, commitTime, hoodieTable, idPfx);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected CopyOnWriteInsertHandler getInsertHandler() {
|
||||
return new MergeOnReadInsertHandler();
|
||||
}
|
||||
|
||||
protected class MergeOnReadInsertHandler extends CopyOnWriteInsertHandler {
|
||||
|
||||
@Override
|
||||
protected void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> payload) {
|
||||
final HoodieRecord insertPayload = payload.record;
|
||||
List<WriteStatus> statuses = new ArrayList<>();
|
||||
// lazily initialize the handle, for the first time
|
||||
if (handle == null) {
|
||||
handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable, getNextFileId(idPrefix));
|
||||
}
|
||||
if (handle.canWrite(insertPayload)) {
|
||||
// write the payload, if the handle has capacity
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception);
|
||||
} else {
|
||||
// handle is full.
|
||||
handle.close();
|
||||
statuses.add(handle.getWriteStatus());
|
||||
// Need to handle the rejected payload & open new handle
|
||||
handle = new HoodieAppendHandle(hoodieConfig, commitTime, hoodieTable, getNextFileId(idPrefix));
|
||||
handle.write(insertPayload, payload.insertValue, payload.exception); // we should be able to write 1 payload.
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,77 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.func;
|
||||
|
||||
import java.io.Serializable;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
/**
|
||||
* Holds Operation result. Used as a result container for Compaction Admin Client (running as part of Spark-launcher
|
||||
* process) to communicate results back to Hoodie CLI process.
|
||||
*/
|
||||
public class OperationResult<T> implements Serializable {
|
||||
|
||||
private T operation;
|
||||
private boolean executed;
|
||||
private boolean success;
|
||||
private Option<Exception> exception;
|
||||
|
||||
public OperationResult() {
|
||||
}
|
||||
|
||||
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, Option<Exception> exception) {
|
||||
this.operation = operation;
|
||||
this.success = success;
|
||||
this.exception = exception;
|
||||
this.executed = executed;
|
||||
}
|
||||
|
||||
public T getOperation() {
|
||||
return operation;
|
||||
}
|
||||
|
||||
public boolean isSuccess() {
|
||||
return success;
|
||||
}
|
||||
|
||||
public boolean isExecuted() {
|
||||
return executed;
|
||||
}
|
||||
|
||||
public Option<Exception> getException() {
|
||||
return exception;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "OperationResult{"
|
||||
+ "operation=" + operation
|
||||
+ ", executed=" + executed
|
||||
+ ", success=" + success
|
||||
+ ", exception=" + exception
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,75 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.func;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueue;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
|
||||
/**
|
||||
* This class wraps a parquet reader and provides an iterator based api to
|
||||
* read from a parquet file. This is used in {@link BoundedInMemoryQueue}
|
||||
*/
|
||||
public class ParquetReaderIterator<T> implements Iterator<T> {
|
||||
|
||||
// Parquet reader for an existing parquet file
|
||||
private final ParquetReader<T> parquetReader;
|
||||
// Holds the next entry returned by the parquet reader
|
||||
private T next;
|
||||
|
||||
public ParquetReaderIterator(ParquetReader<T> parquetReader) {
|
||||
this.parquetReader = parquetReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
try {
|
||||
// To handle when hasNext() is called multiple times for idempotency and/or the first time
|
||||
if (this.next == null) {
|
||||
this.next = parquetReader.read();
|
||||
}
|
||||
return this.next != null;
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to read next record from parquet file ", io);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public T next() {
|
||||
try {
|
||||
// To handle case when next() is called before hasNext()
|
||||
if (this.next == null) {
|
||||
if (!hasNext()) {
|
||||
throw new HoodieIOException("No more records left to read from parquet file");
|
||||
}
|
||||
}
|
||||
T retVal = this.next;
|
||||
this.next = parquetReader.read();
|
||||
return retVal;
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("unable to read next record from parquet file ", io);
|
||||
}
|
||||
}
|
||||
|
||||
public void close() throws IOException {
|
||||
parquetReader.close();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,57 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.func;
|
||||
|
||||
import java.util.Iterator;
|
||||
import java.util.function.Function;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.TaskContext$;
|
||||
|
||||
public class SparkBoundedInMemoryExecutor<I, O, E> extends BoundedInMemoryExecutor<I, O, E> {
|
||||
|
||||
// Need to set current spark thread's TaskContext into newly launched thread so that new thread can access
|
||||
// TaskContext properties.
|
||||
final TaskContext sparkThreadTaskContext;
|
||||
|
||||
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, final Iterator<I> inputItr,
|
||||
BoundedInMemoryQueueConsumer<O, E> consumer,
|
||||
Function<I, O> bufferedIteratorTransform) {
|
||||
this(hoodieConfig, new IteratorBasedQueueProducer<>(inputItr), consumer, bufferedIteratorTransform);
|
||||
}
|
||||
|
||||
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig,
|
||||
BoundedInMemoryQueueProducer<I> producer,
|
||||
BoundedInMemoryQueueConsumer<O, E> consumer,
|
||||
Function<I, O> bufferedIteratorTransform) {
|
||||
super(hoodieConfig.getWriteBufferLimitBytes(), producer,
|
||||
Option.of(consumer), bufferedIteratorTransform);
|
||||
this.sparkThreadTaskContext = TaskContext.get();
|
||||
}
|
||||
|
||||
public void preExecute() {
|
||||
// Passing parent thread's TaskContext to newly launched thread for it to access original TaskContext properties.
|
||||
TaskContext$.MODULE$.setTaskContext(sparkThreadTaskContext);
|
||||
}
|
||||
}
|
||||
127
hudi-client/src/main/java/org/apache/hudi/index/HoodieIndex.java
Normal file
127
hudi-client/src/main/java/org/apache/hudi/index/HoodieIndex.java
Normal file
@@ -0,0 +1,127 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index;
|
||||
|
||||
import java.io.Serializable;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.bloom.HoodieBloomIndex;
|
||||
import org.apache.hudi.index.bloom.HoodieGlobalBloomIndex;
|
||||
import org.apache.hudi.index.hbase.HBaseIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Base class for different types of indexes to determine the mapping from uuid
|
||||
*/
|
||||
public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
protected HoodieIndex(HoodieWriteConfig config) {
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieIndex<T> createIndex(HoodieWriteConfig config,
|
||||
JavaSparkContext jsc) throws HoodieIndexException {
|
||||
switch (config.getIndexType()) {
|
||||
case HBASE:
|
||||
return new HBaseIndex<>(config);
|
||||
case INMEMORY:
|
||||
return new InMemoryHashIndex<>(config);
|
||||
case BLOOM:
|
||||
return new HoodieBloomIndex<>(config);
|
||||
case GLOBAL_BLOOM:
|
||||
return new HoodieGlobalBloomIndex<>(config);
|
||||
default:
|
||||
throw new HoodieIndexException("Index type unspecified, set " + config.getIndexType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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, Option<Pair<String, String>>> fetchRecordLocation(
|
||||
JavaRDD<HoodieKey> hoodieKeys, final JavaSparkContext jsc, HoodieTable<T> hoodieTable);
|
||||
|
||||
/**
|
||||
* Looks up the index and tags each incoming record with a location of a file that contains the
|
||||
* row (if it is actually present)
|
||||
*/
|
||||
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
|
||||
JavaSparkContext jsc, HoodieTable<T> hoodieTable) throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Extracts the location of written records, and updates the index.
|
||||
* <p>
|
||||
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable)
|
||||
throws HoodieIndexException;
|
||||
|
||||
/**
|
||||
* Rollback the efffects of the commit made at commitTime.
|
||||
*/
|
||||
public abstract boolean rollbackCommit(String commitTime);
|
||||
|
||||
/**
|
||||
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the
|
||||
* `partitionPath`. Such an implementation is able to obtain the same mapping, for two hoodie keys
|
||||
* with same `recordKey` but different `partitionPath`
|
||||
*
|
||||
* @return whether or not, the index implementation is global in nature
|
||||
*/
|
||||
public abstract boolean isGlobal();
|
||||
|
||||
/**
|
||||
* This is used by storage to determine, if its safe to send inserts, straight to the log, i.e
|
||||
* having a {@link FileSlice}, with no data file.
|
||||
*
|
||||
* @return Returns true/false depending on whether the impl has this capability
|
||||
*/
|
||||
public abstract boolean canIndexLogFiles();
|
||||
|
||||
|
||||
/**
|
||||
* An index is "implicit" with respect to storage, if just writing new data to a file slice,
|
||||
* updates the index as well. This is used by storage, to save memory footprint in certain cases.
|
||||
*/
|
||||
public abstract boolean isImplicitWithStorage();
|
||||
|
||||
/**
|
||||
* Each index type should implement it's own logic to release any resources acquired during the process.
|
||||
*/
|
||||
public void close() {}
|
||||
|
||||
public enum IndexType {
|
||||
HBASE, INMEMORY, BLOOM, GLOBAL_BLOOM
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,142 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
|
||||
|
||||
/**
|
||||
* Hoodie Index implementation backed by an in-memory Hash map. <p> ONLY USE FOR LOCAL TESTING
|
||||
*/
|
||||
public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
private static ConcurrentMap<HoodieKey, HoodieRecordLocation> recordLocationMap;
|
||||
|
||||
public InMemoryHashIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
synchronized (InMemoryHashIndex.class) {
|
||||
if (recordLocationMap == null) {
|
||||
recordLocationMap = new ConcurrentHashMap<>();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
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");
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(this.new LocationTagFunction(), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD.map(new Function<WriteStatus, WriteStatus>() {
|
||||
@Override
|
||||
public WriteStatus call(WriteStatus writeStatus) {
|
||||
for (HoodieRecord record : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(record.getKey())) {
|
||||
HoodieKey key = record.getKey();
|
||||
Option<HoodieRecordLocation> newLocation = record.getNewLocation();
|
||||
if (newLocation.isPresent()) {
|
||||
recordLocationMap.put(key, newLocation.get());
|
||||
} else {
|
||||
//Delete existing index for a deleted record
|
||||
recordLocationMap.remove(key);
|
||||
}
|
||||
}
|
||||
}
|
||||
return writeStatus;
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Only looks up by recordKey
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping is available in HBase already.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Index needs to be explicitly updated after storage write.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
class LocationTagFunction implements
|
||||
Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
|
||||
|
||||
@Override
|
||||
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
|
||||
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord<T> rec = hoodieRecordIterator.next();
|
||||
if (recordLocationMap.containsKey(rec.getKey())) {
|
||||
rec.setCurrentLocation(recordLocationMap.get(rec.getKey()));
|
||||
}
|
||||
taggedRecords.add(rec);
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,98 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import com.google.common.base.Objects;
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Metadata about a given file group, useful for index lookup
|
||||
*/
|
||||
public class BloomIndexFileInfo implements Serializable {
|
||||
|
||||
private final String fileId;
|
||||
|
||||
private final String minRecordKey;
|
||||
|
||||
private final String maxRecordKey;
|
||||
|
||||
public BloomIndexFileInfo(String fileId, String minRecordKey, String maxRecordKey) {
|
||||
this.fileId = fileId;
|
||||
this.minRecordKey = minRecordKey;
|
||||
this.maxRecordKey = maxRecordKey;
|
||||
}
|
||||
|
||||
public BloomIndexFileInfo(String fileId) {
|
||||
this.fileId = fileId;
|
||||
this.minRecordKey = null;
|
||||
this.maxRecordKey = null;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public String getMinRecordKey() {
|
||||
return minRecordKey;
|
||||
}
|
||||
|
||||
public String getMaxRecordKey() {
|
||||
return maxRecordKey;
|
||||
}
|
||||
|
||||
public boolean hasKeyRanges() {
|
||||
return minRecordKey != null && maxRecordKey != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Does the given key fall within the range (inclusive)
|
||||
*/
|
||||
public boolean isKeyInRange(String recordKey) {
|
||||
return minRecordKey.compareTo(recordKey) <= 0 && maxRecordKey.compareTo(recordKey) >= 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
BloomIndexFileInfo that = (BloomIndexFileInfo) o;
|
||||
return Objects.equal(that.fileId, fileId) && Objects.equal(that.minRecordKey, minRecordKey)
|
||||
&& Objects.equal(that.maxRecordKey, maxRecordKey);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(fileId, minRecordKey, maxRecordKey);
|
||||
}
|
||||
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("BloomIndexFileInfo {");
|
||||
sb.append(" fileId=").append(fileId);
|
||||
sb.append(" minRecordKey=").append(minRecordKey);
|
||||
sb.append(" maxRecordKey=").append(maxRecordKey);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,156 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.hash.Hashing;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
|
||||
/**
|
||||
* Partitions bloom filter checks by spreading out comparisons across buckets of work.
|
||||
*
|
||||
* Each bucket incurs the following cost
|
||||
* <pre>
|
||||
* 1) Read bloom filter from file footer
|
||||
* 2) Check keys against bloom filter
|
||||
* 3) [Conditional] If any key had a hit, open file and check
|
||||
* </pre>
|
||||
*
|
||||
* The partitioner performs a two phase bin packing algorithm, to pack enough work into each bucket such that cost of
|
||||
* (1) & (3) is amortized. Also, avoids any skews in the sort based approach, by directly partitioning by the file to be
|
||||
* checked against and ensuring each partition has similar number of buckets. Performance tests show that this approach
|
||||
* could bound the amount of skew to std_dev(numberOfBucketsPerPartition) * cost of (3), lower than sort partitioning.
|
||||
*
|
||||
* Approach has two goals :
|
||||
* <pre>
|
||||
* 1) Pack as many buckets from same file group into same partition, to amortize cost of (1) and (2) further
|
||||
* 2) Spread buckets across partitions evenly to achieve skew reduction
|
||||
* </pre>
|
||||
*/
|
||||
public class BucketizedBloomCheckPartitioner extends Partitioner {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(BucketizedBloomCheckPartitioner.class);
|
||||
|
||||
private int partitions;
|
||||
|
||||
/**
|
||||
* Stores the final mapping of a file group to a list of partitions for its keys.
|
||||
*/
|
||||
private Map<String, List<Integer>> fileGroupToPartitions;
|
||||
|
||||
/**
|
||||
* Create a partitioner that computes a plan based on provided workload characteristics.
|
||||
*
|
||||
* @param targetPartitions maximum number of partitions to target
|
||||
* @param fileGroupToComparisons number of expected comparisons per file group
|
||||
* @param keysPerBucket maximum number of keys to pack in a single bucket
|
||||
*/
|
||||
public BucketizedBloomCheckPartitioner(int targetPartitions, Map<String, Long> fileGroupToComparisons,
|
||||
int keysPerBucket) {
|
||||
this.fileGroupToPartitions = new HashMap<>();
|
||||
|
||||
Map<String, Integer> bucketsPerFileGroup = new HashMap<>();
|
||||
// Compute the buckets needed per file group, using simple uniform distribution
|
||||
fileGroupToComparisons.forEach((f, c) ->
|
||||
bucketsPerFileGroup.put(f, (int) Math.ceil((c * 1.0) / keysPerBucket)));
|
||||
int totalBuckets = bucketsPerFileGroup.values().stream().mapToInt(i -> i).sum();
|
||||
// If totalBuckets > targetPartitions, no need to have extra partitions
|
||||
this.partitions = Math.min(targetPartitions, totalBuckets);
|
||||
|
||||
// PHASE 1 : start filling upto minimum number of buckets into partitions, taking all but one bucket from each file
|
||||
// This tries to first optimize for goal 1 above, with knowledge that each partition needs a certain minimum number
|
||||
// of buckets and assigns buckets in the same order as file groups. If we were to simply round robin, then buckets
|
||||
// for a file group is more or less guaranteed to be placed on different partitions all the time.
|
||||
int minBucketsPerPartition = Math.max((int) Math.floor((1.0 * totalBuckets) / partitions), 1);
|
||||
logger.info(String.format("TotalBuckets %d, min_buckets/partition %d", totalBuckets, minBucketsPerPartition));
|
||||
int[] bucketsFilled = new int[partitions];
|
||||
Map<String, AtomicInteger> bucketsFilledPerFileGroup = new HashMap<>();
|
||||
int partitionIndex = 0;
|
||||
for (Map.Entry<String, Integer> e : bucketsPerFileGroup.entrySet()) {
|
||||
for (int b = 0; b < Math.max(1, e.getValue() - 1); b++) {
|
||||
// keep filled counts upto date
|
||||
bucketsFilled[partitionIndex]++;
|
||||
AtomicInteger cnt = bucketsFilledPerFileGroup.getOrDefault(e.getKey(), new AtomicInteger(0));
|
||||
cnt.incrementAndGet();
|
||||
bucketsFilledPerFileGroup.put(e.getKey(), cnt);
|
||||
|
||||
// mark this partition against the file group
|
||||
List<Integer> partitionList = this.fileGroupToPartitions.getOrDefault(e.getKey(), new ArrayList<>());
|
||||
partitionList.add(partitionIndex);
|
||||
this.fileGroupToPartitions.put(e.getKey(), partitionList);
|
||||
|
||||
// switch to new partition if needed
|
||||
if (bucketsFilled[partitionIndex] >= minBucketsPerPartition) {
|
||||
partitionIndex = (partitionIndex + 1) % partitions;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// PHASE 2 : for remaining unassigned buckets, round robin over partitions once. Since we withheld 1 bucket from
|
||||
// each file group uniformly, this remaining is also an uniform mix across file groups. We just round robin to
|
||||
// optimize for goal 2.
|
||||
for (Map.Entry<String, Integer> e : bucketsPerFileGroup.entrySet()) {
|
||||
int remaining = e.getValue() - bucketsFilledPerFileGroup.get(e.getKey()).intValue();
|
||||
for (int r = 0; r < remaining; r++) {
|
||||
// mark this partition against the file group
|
||||
this.fileGroupToPartitions.get(e.getKey()).add(partitionIndex);
|
||||
bucketsFilled[partitionIndex]++;
|
||||
partitionIndex = (partitionIndex + 1) % partitions;
|
||||
}
|
||||
}
|
||||
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Partitions assigned per file groups :" + fileGroupToPartitions);
|
||||
StringBuilder str = new StringBuilder();
|
||||
for (int i = 0; i < bucketsFilled.length; i++) {
|
||||
str.append("p" + i + " : " + bucketsFilled[i] + ",");
|
||||
}
|
||||
logger.debug("Num buckets assigned per file group :" + str);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numPartitions() {
|
||||
return partitions;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPartition(Object key) {
|
||||
final Pair<String, String> parts = (Pair<String, String>) key;
|
||||
final long hashOfKey = Hashing.md5().hashString(parts.getRight(), StandardCharsets.UTF_8).asLong();
|
||||
final List<Integer> candidatePartitions = fileGroupToPartitions.get(parts.getLeft());
|
||||
final int idx = (int) Math.floorMod(hashOfKey, candidatePartitions.size());
|
||||
assert idx >= 0;
|
||||
return candidatePartitions.get(idx);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
Map<String, List<Integer>> getFileGroupToPartitions() {
|
||||
return fileGroupToPartitions;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,397 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import static java.util.stream.Collectors.groupingBy;
|
||||
import static java.util.stream.Collectors.mapping;
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.MetadataNotFoundException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.io.HoodieRangeInfoHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.storage.StorageLevel;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Indexing mechanism based on bloom filter. Each parquet file includes its row_key bloom filter in its metadata.
|
||||
*/
|
||||
public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
// we need to limit the join such that it stays within 1.5GB per Spark partition. (SPARK-1476)
|
||||
private static final int SPARK_MAXIMUM_BYTES_PER_PARTITION = 1500 * 1024 * 1024;
|
||||
// this is how much a triplet of (partitionPath, fileId, recordKey) costs.
|
||||
private static final int BYTES_PER_PARTITION_FILE_KEY_TRIPLET = 300;
|
||||
private static Logger logger = LogManager.getLogger(HoodieBloomIndex.class);
|
||||
private static int MAX_ITEMS_PER_SHUFFLE_PARTITION =
|
||||
SPARK_MAXIMUM_BYTES_PER_PARTITION / BYTES_PER_PARTITION_FILE_KEY_TRIPLET;
|
||||
|
||||
public HoodieBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
|
||||
// Step 0: cache the input record RDD
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
recordRDD.persist(config.getBloomIndexInputStorageLevel());
|
||||
}
|
||||
|
||||
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
|
||||
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc,
|
||||
hoodieTable);
|
||||
|
||||
// Cache the result, for subsequent stages.
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
keyFilenamePairRDD.persist(StorageLevel.MEMORY_AND_DISK_SER());
|
||||
}
|
||||
if (logger.isDebugEnabled()) {
|
||||
long totalTaggedRecords = keyFilenamePairRDD.count();
|
||||
logger.debug("Number of update records (ones tagged with a fileID): " + totalTaggedRecords);
|
||||
}
|
||||
|
||||
// Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys
|
||||
// Cost: 4 sec.
|
||||
JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(keyFilenamePairRDD,
|
||||
recordRDD);
|
||||
|
||||
if (config.getBloomIndexUseCaching()) {
|
||||
recordRDD.unpersist(); // unpersist the input Record RDD
|
||||
keyFilenamePairRDD.unpersist();
|
||||
}
|
||||
|
||||
return taggedRecordRDD;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
* @param jsc spark context
|
||||
* @param hoodieTable hoodie table object
|
||||
*/
|
||||
@Override
|
||||
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()));
|
||||
|
||||
// Lookup indexes for all the partition/recordkey pair
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> recordKeyLocationRDD = lookupIndex(partitionRecordKeyPairRDD, jsc,
|
||||
hoodieTable);
|
||||
JavaPairRDD<HoodieKey, String> keyHoodieKeyPairRDD = hoodieKeys.mapToPair(key -> new Tuple2<>(key, null));
|
||||
|
||||
return keyHoodieKeyPairRDD.leftOuterJoin(recordKeyLocationRDD).mapToPair(keyLoc -> {
|
||||
Option<Pair<String, String>> partitionPathFileidPair;
|
||||
if (keyLoc._2._2.isPresent()) {
|
||||
partitionPathFileidPair = Option.of(Pair.of(keyLoc._1().getPartitionPath(), keyLoc._2._2.get().getFileId()));
|
||||
} else {
|
||||
partitionPathFileidPair = Option.empty();
|
||||
}
|
||||
return new Tuple2<>(keyLoc._1, partitionPathFileidPair);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Lookup the location for each record key and return the pair<record_key,location> for all record keys already
|
||||
* present and drop the record keys if not present
|
||||
*/
|
||||
private JavaPairRDD<HoodieKey, HoodieRecordLocation> lookupIndex(
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, final JavaSparkContext jsc,
|
||||
final HoodieTable hoodieTable) {
|
||||
// Obtain records per partition, in the incoming records
|
||||
Map<String, Long> recordsPerPartition = partitionRecordKeyPairRDD.countByKey();
|
||||
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
|
||||
|
||||
// Step 2: Load all involved files as <Partition, filename> pairs
|
||||
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, jsc,
|
||||
hoodieTable);
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
|
||||
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
|
||||
|
||||
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id,
|
||||
// that contains it.
|
||||
Map<String, Long> comparisonsPerFileGroup = computeComparisonsPerFileGroup(recordsPerPartition, partitionToFileInfo,
|
||||
partitionRecordKeyPairRDD);
|
||||
int safeParallelism = computeSafeParallelism(recordsPerPartition, comparisonsPerFileGroup);
|
||||
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), safeParallelism);
|
||||
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, joinParallelism,
|
||||
hoodieTable, comparisonsPerFileGroup);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compute the estimated number of bloom filter comparisons to be performed on each file group
|
||||
*/
|
||||
private Map<String, Long> computeComparisonsPerFileGroup(final Map<String, Long> recordsPerPartition,
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
|
||||
Map<String, Long> fileToComparisons;
|
||||
if (config.getBloomIndexPruneByRanges()) {
|
||||
// we will just try exploding the input and then count to determine comparisons
|
||||
// FIX(vc): Only do sampling here and extrapolate?
|
||||
fileToComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo,
|
||||
partitionRecordKeyPairRDD).mapToPair(t -> t).countByKey();
|
||||
} else {
|
||||
fileToComparisons = new HashMap<>();
|
||||
partitionToFileInfo.entrySet().stream().forEach(e -> {
|
||||
for (BloomIndexFileInfo fileInfo : e.getValue()) {
|
||||
//each file needs to be compared against all the records coming into the partition
|
||||
fileToComparisons.put(fileInfo.getFileId(), recordsPerPartition.get(e.getKey()));
|
||||
}
|
||||
});
|
||||
}
|
||||
return fileToComparisons;
|
||||
}
|
||||
|
||||
/**
|
||||
* Compute the minimum parallelism needed to play well with the spark 2GB limitation.. The index lookup can be skewed
|
||||
* in three dimensions : #files, #partitions, #records <p> To be able to smoothly handle skews, we need to compute how
|
||||
* to split each partitions into subpartitions. We do it here, in a way that keeps the amount of each Spark join
|
||||
* partition to < 2GB. <p> If {@link HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is
|
||||
* specified as a NON-zero number, then that is used explicitly.
|
||||
*/
|
||||
int computeSafeParallelism(Map<String, Long> recordsPerPartition, Map<String, Long> comparisonsPerFileGroup) {
|
||||
long totalComparisons = comparisonsPerFileGroup.values().stream().mapToLong(Long::longValue).sum();
|
||||
long totalFiles = comparisonsPerFileGroup.size();
|
||||
long totalRecords = recordsPerPartition.values().stream().mapToLong(Long::longValue).sum();
|
||||
int parallelism = (int) (totalComparisons / MAX_ITEMS_PER_SHUFFLE_PARTITION + 1);
|
||||
logger.info(String.format("TotalRecords %d, TotalFiles %d, TotalAffectedPartitions %d, TotalComparisons %d, "
|
||||
+ "SafeParallelism %d", totalRecords, totalFiles, recordsPerPartition.size(), totalComparisons, parallelism));
|
||||
return parallelism;
|
||||
}
|
||||
|
||||
/**
|
||||
* Its crucial to pick the right parallelism. <p> totalSubPartitions : this is deemed safe limit, to be nice with
|
||||
* Spark. inputParallelism : typically number of input file splits <p> We pick the max such that, we are always safe,
|
||||
* but go higher if say a there are a lot of input files. (otherwise, we will fallback to number of partitions in
|
||||
* input and end up with slow performance)
|
||||
*/
|
||||
private int determineParallelism(int inputParallelism, int totalSubPartitions) {
|
||||
// If bloom index parallelism is set, use it to to check against the input parallelism and
|
||||
// take the max
|
||||
int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
|
||||
int joinParallelism = Math.max(totalSubPartitions, indexParallelism);
|
||||
logger.info("InputParallelism: ${" + inputParallelism + "}, " + "IndexParallelism: ${" + config
|
||||
.getBloomIndexParallelism() + "}, " + "TotalSubParts: ${" + totalSubPartitions + "}, "
|
||||
+ "Join Parallelism set to : " + joinParallelism);
|
||||
return joinParallelism;
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair RDD.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
|
||||
final HoodieTable hoodieTable) {
|
||||
|
||||
// Obtain the latest data files from all the partitions.
|
||||
List<Pair<String, String>> partitionPathFileIDList = jsc
|
||||
.parallelize(partitions, Math.max(partitions.size(), 1))
|
||||
.flatMap(partitionPath -> {
|
||||
Option<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
List<Pair<String, String>> filteredFiles = new ArrayList<>();
|
||||
if (latestCommitTime.isPresent()) {
|
||||
filteredFiles = hoodieTable.getROFileSystemView()
|
||||
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp())
|
||||
.map(f -> Pair.of(partitionPath, f.getFileId())).collect(toList());
|
||||
}
|
||||
return filteredFiles.iterator();
|
||||
}).collect();
|
||||
|
||||
if (config.getBloomIndexPruneByRanges()) {
|
||||
// also obtain file ranges, if range pruning is enabled
|
||||
return jsc.parallelize(partitionPathFileIDList, Math.max(partitionPathFileIDList.size(), 1)).mapToPair(pf -> {
|
||||
try {
|
||||
HoodieRangeInfoHandle<T> rangeInfoHandle = new HoodieRangeInfoHandle<T>(config, hoodieTable, pf);
|
||||
String[] minMaxKeys = rangeInfoHandle.getMinMaxKeys();
|
||||
return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue(), minMaxKeys[0], minMaxKeys[1]));
|
||||
} catch (MetadataNotFoundException me) {
|
||||
logger.warn("Unable to find range metadata in file :" + pf);
|
||||
return new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue()));
|
||||
}
|
||||
}).collect();
|
||||
} else {
|
||||
return partitionPathFileIDList.stream()
|
||||
.map(pf -> new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue())))
|
||||
.collect(toList());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// Nope, don't need to do anything.
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* This is not global, since we depend on the partitionPath to do the lookup
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* No indexes into log files yet.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Bloom filters are stored, into the same data files.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be
|
||||
* checked. For datasets, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files
|
||||
* to be compared gets cut down a lot from range pruning.
|
||||
*
|
||||
* Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on
|
||||
* recordKey ranges in the index info.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
JavaRDD<Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
IndexFileFilter indexFileFilter = config.useBloomIndexTreebasedFilter()
|
||||
? new IntervalTreeBasedIndexFileFilter(partitionToFileIndexInfo)
|
||||
: new ListBasedIndexFileFilter(partitionToFileIndexInfo);
|
||||
|
||||
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
|
||||
String recordKey = partitionRecordKeyPair._2();
|
||||
String partitionPath = partitionRecordKeyPair._1();
|
||||
|
||||
return indexFileFilter.getMatchingFiles(partitionPath, recordKey).stream()
|
||||
.map(matchingFile -> new Tuple2<>(matchingFile, new HoodieKey(recordKey, partitionPath)))
|
||||
.collect(Collectors.toList());
|
||||
}).flatMap(List::iterator);
|
||||
}
|
||||
|
||||
/**
|
||||
* Find out <RowKey, filename> pair. All workload grouped by file-level. <p> Join PairRDD(PartitionPath, RecordKey)
|
||||
* and PairRDD(PartitionPath, File) & then repartition such that each RDD partition is a file, then for each file, we
|
||||
* do (1) load bloom filter, (2) load rowKeys, (3) Tag rowKey <p> Make sure the parallelism is atleast the groupby
|
||||
* parallelism for tagging location
|
||||
*/
|
||||
@VisibleForTesting
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD, int shuffleParallelism, HoodieTable hoodieTable,
|
||||
Map<String, Long> fileGroupToComparisons) {
|
||||
JavaRDD<Tuple2<String, HoodieKey>> fileComparisonsRDD =
|
||||
explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD);
|
||||
|
||||
if (config.useBloomIndexBucketizedChecking()) {
|
||||
Partitioner partitioner = new BucketizedBloomCheckPartitioner(
|
||||
shuffleParallelism,
|
||||
fileGroupToComparisons,
|
||||
config.getBloomIndexKeysPerBucket()
|
||||
);
|
||||
|
||||
fileComparisonsRDD = fileComparisonsRDD
|
||||
.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t))
|
||||
.repartitionAndSortWithinPartitions(partitioner)
|
||||
.map(Tuple2::_2);
|
||||
} else {
|
||||
fileComparisonsRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, shuffleParallelism);
|
||||
}
|
||||
|
||||
return fileComparisonsRDD
|
||||
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true)
|
||||
.flatMap(List::iterator)
|
||||
.filter(lr -> lr.getMatchingRecordKeys().size() > 0)
|
||||
.flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream()
|
||||
.map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()),
|
||||
new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())))
|
||||
.collect(Collectors.toList())
|
||||
.iterator());
|
||||
}
|
||||
|
||||
HoodieRecord<T> getTaggedRecord(HoodieRecord<T> inputRecord,
|
||||
Option<HoodieRecordLocation> location) {
|
||||
HoodieRecord<T> record = inputRecord;
|
||||
if (location.isPresent()) {
|
||||
// When you have a record in multiple files in the same partition, then rowKeyRecordPairRDD
|
||||
// will have 2 entries with the same exact in memory copy of the HoodieRecord and the 2
|
||||
// separate filenames that the record is found in. This will result in setting
|
||||
// currentLocation 2 times and it will fail the second time. So creating a new in memory
|
||||
// copy of the hoodie record.
|
||||
record = new HoodieRecord<>(inputRecord);
|
||||
record.setCurrentLocation(location.get());
|
||||
}
|
||||
return record;
|
||||
}
|
||||
|
||||
/**
|
||||
* Tag the <rowKey, filename> back to the original HoodieRecord RDD.
|
||||
*/
|
||||
protected JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||
JavaPairRDD<HoodieKey, HoodieRecord<T>> keyRecordPairRDD = recordRDD
|
||||
.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, Option.ofNullable(v1._2.orNull())));
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return writeStatusRDD;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,119 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.func.LazyIterableIterator;
|
||||
import org.apache.hudi.io.HoodieKeyLookupHandle;
|
||||
import org.apache.hudi.io.HoodieKeyLookupHandle.KeyLookupResult;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the
|
||||
* actual files
|
||||
*/
|
||||
public class HoodieBloomIndexCheckFunction implements
|
||||
Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, Iterator<List<KeyLookupResult>>> {
|
||||
|
||||
private final HoodieTable hoodieTable;
|
||||
|
||||
private final HoodieWriteConfig config;
|
||||
|
||||
public HoodieBloomIndexCheckFunction(HoodieTable hoodieTable, HoodieWriteConfig config) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<KeyLookupResult>> call(Integer partition,
|
||||
Iterator<Tuple2<String, HoodieKey>> fileParitionRecordKeyTripletItr) {
|
||||
return new LazyKeyCheckIterator(fileParitionRecordKeyTripletItr);
|
||||
}
|
||||
|
||||
class LazyKeyCheckIterator extends LazyIterableIterator<Tuple2<String, HoodieKey>, List<KeyLookupResult>> {
|
||||
|
||||
private HoodieKeyLookupHandle keyLookupHandle;
|
||||
|
||||
LazyKeyCheckIterator(
|
||||
Iterator<Tuple2<String, HoodieKey>> filePartitionRecordKeyTripletItr) {
|
||||
super(filePartitionRecordKeyTripletItr);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void start() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<HoodieKeyLookupHandle.KeyLookupResult> computeNext() {
|
||||
|
||||
List<HoodieKeyLookupHandle.KeyLookupResult> ret = new ArrayList<>();
|
||||
try {
|
||||
// process one file in each go.
|
||||
while (inputItr.hasNext()) {
|
||||
Tuple2<String, HoodieKey> currentTuple = inputItr.next();
|
||||
String fileId = currentTuple._1;
|
||||
String partitionPath = currentTuple._2.getPartitionPath();
|
||||
String recordKey = currentTuple._2.getRecordKey();
|
||||
Pair<String, String> partitionPathFilePair = Pair.of(partitionPath, fileId);
|
||||
|
||||
// lazily init state
|
||||
if (keyLookupHandle == null) {
|
||||
keyLookupHandle = new HoodieKeyLookupHandle(config, hoodieTable, partitionPathFilePair);
|
||||
}
|
||||
|
||||
// if continue on current file
|
||||
if (keyLookupHandle.getPartitionPathFilePair().equals(partitionPathFilePair)) {
|
||||
keyLookupHandle.addKey(recordKey);
|
||||
} else {
|
||||
// do the actual checking of file & break out
|
||||
ret.add(keyLookupHandle.getLookupResult());
|
||||
keyLookupHandle = new HoodieKeyLookupHandle(config, hoodieTable, partitionPathFilePair);
|
||||
keyLookupHandle.addKey(recordKey);
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
// handle case, where we ran out of input, close pending work, update return val
|
||||
if (!inputItr.hasNext()) {
|
||||
ret.add(keyLookupHandle.getLookupResult());
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
if (e instanceof HoodieException) {
|
||||
throw e;
|
||||
}
|
||||
throw new HoodieIndexException("Error checking bloom filter index. ", e);
|
||||
}
|
||||
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void end() {
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,125 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* This filter will only work with hoodie dataset since it will only load partitions with .hoodie_partition_metadata
|
||||
* file in it.
|
||||
*/
|
||||
public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends HoodieBloomIndex<T> {
|
||||
|
||||
public HoodieGlobalBloomIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all involved files as <Partition, filename> pair RDD from all partitions in the table.
|
||||
*/
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
List<Tuple2<String, BloomIndexFileInfo>> loadInvolvedFiles(List<String> partitions, final JavaSparkContext jsc,
|
||||
final HoodieTable hoodieTable) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
try {
|
||||
List<String> allPartitionPaths = FSUtils
|
||||
.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
return super.loadInvolvedFiles(allPartitionPaths, jsc, hoodieTable);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to load all partitions", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* For each incoming record, produce N output records, 1 each for each file against which the record's key needs to be
|
||||
* checked. For datasets, where the keys have a definite insert order (e.g: timestamp as prefix), the number of files
|
||||
* to be compared gets cut down a lot from range pruning.
|
||||
*
|
||||
* Sub-partition to ensure the records can be looked up against files & also prune file<=>record comparisons based on
|
||||
* recordKey ranges in the index info. the partition path of the incoming record (partitionRecordKeyPairRDD._2()) will
|
||||
* be ignored since the search scope should be bigger than that
|
||||
*/
|
||||
|
||||
@Override
|
||||
@VisibleForTesting
|
||||
JavaRDD<Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
|
||||
Map<String, String> indexToPartitionMap = new HashMap<>();
|
||||
for (Entry<String, List<BloomIndexFileInfo>> entry : partitionToFileIndexInfo.entrySet()) {
|
||||
entry.getValue().forEach(indexFile -> indexToPartitionMap.put(indexFile.getFileId(), entry.getKey()));
|
||||
}
|
||||
|
||||
IndexFileFilter indexFileFilter = config.getBloomIndexPruneByRanges()
|
||||
? new IntervalTreeBasedGlobalIndexFileFilter(partitionToFileIndexInfo)
|
||||
: new ListBasedGlobalIndexFileFilter(partitionToFileIndexInfo);
|
||||
|
||||
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
|
||||
String recordKey = partitionRecordKeyPair._2();
|
||||
String partitionPath = partitionRecordKeyPair._1();
|
||||
|
||||
return indexFileFilter.getMatchingFiles(partitionPath, recordKey).stream()
|
||||
.map(file -> new Tuple2<>(file, new HoodieKey(recordKey, indexToPartitionMap.get(file))))
|
||||
.collect(Collectors.toList());
|
||||
}).flatMap(List::iterator);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Tagging for global index should only consider the record key
|
||||
*/
|
||||
@Override
|
||||
protected JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
|
||||
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
|
||||
JavaPairRDD<String, HoodieRecord<T>> rowKeyRecordPairRDD = recordRDD
|
||||
.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
|
||||
|
||||
// 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, Option.ofNullable(value._2.orNull())));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,38 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* IndexFile filter to assist in look up of a record key.
|
||||
*/
|
||||
public interface IndexFileFilter extends Serializable {
|
||||
|
||||
/**
|
||||
* Fetches all matching files for a given record key and partition.
|
||||
*
|
||||
* @param partitionPath the partition path of interest
|
||||
* @param recordKey the record key to be looked up
|
||||
* @return the {@link Set} of matching file names where the record could potentially be present.
|
||||
*/
|
||||
Set<String> getMatchingFiles(String partitionPath, String recordKey);
|
||||
|
||||
}
|
||||
@@ -0,0 +1,67 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Interval Tree based index look up for Global Index. Builds an {@link KeyRangeLookupTree} for all index files (across
|
||||
* all partitions) and uses it to search for matching index files for any given recordKey that needs to be looked up.
|
||||
*/
|
||||
class IntervalTreeBasedGlobalIndexFileFilter implements IndexFileFilter {
|
||||
|
||||
private final KeyRangeLookupTree indexLookUpTree = new KeyRangeLookupTree();
|
||||
private final Set<String> filesWithNoRanges = new HashSet<>();
|
||||
|
||||
/**
|
||||
* Instantiates {@link IntervalTreeBasedGlobalIndexFileFilter}
|
||||
*
|
||||
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}s
|
||||
*/
|
||||
IntervalTreeBasedGlobalIndexFileFilter(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
|
||||
List<BloomIndexFileInfo> allIndexFiles = partitionToFileIndexInfo.values().stream().flatMap(Collection::stream)
|
||||
.collect(Collectors.toList());
|
||||
// Note that the interval tree implementation doesn't have auto-balancing to ensure logN search time.
|
||||
// So, we are shuffling the input here hoping the tree will not have any skewness. If not, the tree could be skewed
|
||||
// which could result in N search time instead of NlogN.
|
||||
Collections.shuffle(allIndexFiles);
|
||||
allIndexFiles.forEach(indexFile -> {
|
||||
if (indexFile.hasKeyRanges()) {
|
||||
indexLookUpTree.insert(new KeyRangeNode(indexFile.getMinRecordKey(),
|
||||
indexFile.getMaxRecordKey(), indexFile.getFileId()));
|
||||
} else {
|
||||
filesWithNoRanges.add(indexFile.getFileId());
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getMatchingFiles(String partitionPath, String recordKey) {
|
||||
Set<String> toReturn = new HashSet<>();
|
||||
toReturn.addAll(indexLookUpTree.getMatchingIndexFiles(recordKey));
|
||||
toReturn.addAll(filesWithNoRanges);
|
||||
return toReturn;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,76 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Interval Tree based index look up. Builds an {@link KeyRangeLookupTree} for every partition and uses it to search for
|
||||
* matching index files for any given recordKey that needs to be looked up.
|
||||
*/
|
||||
class IntervalTreeBasedIndexFileFilter implements IndexFileFilter {
|
||||
|
||||
private final Map<String, KeyRangeLookupTree> partitionToFileIndexLookUpTree = new HashMap<>();
|
||||
private final Map<String, Set<String>> partitionToFilesWithNoRanges = new HashMap<>();
|
||||
|
||||
/**
|
||||
* Instantiates {@link IntervalTreeBasedIndexFileFilter}
|
||||
*
|
||||
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}s
|
||||
*/
|
||||
IntervalTreeBasedIndexFileFilter(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
|
||||
partitionToFileIndexInfo.forEach((partition, bloomIndexFiles) -> {
|
||||
// Note that the interval tree implementation doesn't have auto-balancing to ensure logN search time.
|
||||
// So, we are shuffling the input here hoping the tree will not have any skewness. If not, the tree could be
|
||||
// skewed which could result in N search time instead of logN.
|
||||
Collections.shuffle(bloomIndexFiles);
|
||||
KeyRangeLookupTree lookUpTree = new KeyRangeLookupTree();
|
||||
bloomIndexFiles.forEach(indexFileInfo -> {
|
||||
if (indexFileInfo.hasKeyRanges()) {
|
||||
lookUpTree.insert(new KeyRangeNode(indexFileInfo.getMinRecordKey(),
|
||||
indexFileInfo.getMaxRecordKey(), indexFileInfo.getFileId()));
|
||||
} else {
|
||||
if (!partitionToFilesWithNoRanges.containsKey(partition)) {
|
||||
partitionToFilesWithNoRanges.put(partition, new HashSet<>());
|
||||
}
|
||||
partitionToFilesWithNoRanges.get(partition).add(indexFileInfo.getFileId());
|
||||
}
|
||||
});
|
||||
partitionToFileIndexLookUpTree.put(partition, lookUpTree);
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getMatchingFiles(String partitionPath, String recordKey) {
|
||||
Set<String> toReturn = new HashSet<>();
|
||||
// could be null, if there are no files in a given partition yet or if all index files have no ranges
|
||||
if (partitionToFileIndexLookUpTree.containsKey(partitionPath)) {
|
||||
toReturn.addAll(partitionToFileIndexLookUpTree.get(partitionPath).getMatchingIndexFiles(recordKey));
|
||||
}
|
||||
if (partitionToFilesWithNoRanges.containsKey(partitionPath)) {
|
||||
toReturn.addAll(partitionToFilesWithNoRanges.get(partitionPath));
|
||||
}
|
||||
return toReturn;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Encapsulates the result from a key lookup
|
||||
*/
|
||||
public class KeyLookupResult {
|
||||
|
||||
private final String fileName;
|
||||
private final List<String> matchingRecordKeys;
|
||||
private final String partitionPath;
|
||||
|
||||
public KeyLookupResult(String fileName, String partitionPath, List<String> matchingRecordKeys) {
|
||||
this.fileName = fileName;
|
||||
this.partitionPath = partitionPath;
|
||||
this.matchingRecordKeys = matchingRecordKeys;
|
||||
}
|
||||
|
||||
public String getFileName() {
|
||||
return fileName;
|
||||
}
|
||||
|
||||
public List<String> getMatchingRecordKeys() {
|
||||
return matchingRecordKeys;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,156 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Look up tree implemented as interval trees to search for any given key in (N logN) time complexity.
|
||||
*/
|
||||
class KeyRangeLookupTree implements Serializable {
|
||||
|
||||
private KeyRangeNode root;
|
||||
|
||||
/**
|
||||
* @return the root of the tree. Could be {@code null}
|
||||
*/
|
||||
public KeyRangeNode getRoot() {
|
||||
return root;
|
||||
}
|
||||
|
||||
/**
|
||||
* Inserts a new {@link KeyRangeNode} to this look up tree.
|
||||
*
|
||||
* @param newNode the new {@link KeyRangeNode} to be inserted
|
||||
*/
|
||||
void insert(KeyRangeNode newNode) {
|
||||
root = insert(getRoot(), newNode);
|
||||
}
|
||||
|
||||
/**
|
||||
* Inserts a new {@link KeyRangeNode} to this look up tree.
|
||||
*
|
||||
* If no root exists, make {@code newNode} as the root and return the new root.
|
||||
*
|
||||
* If current root and newNode matches with min record key and max record key,
|
||||
* merge two nodes. In other words, add files from {@code newNode} to current root.
|
||||
* Return current root.
|
||||
*
|
||||
* If current root is < newNode
|
||||
* if current root has no right sub tree
|
||||
* update current root's right sub tree max and min
|
||||
* set newNode as right sub tree
|
||||
* else
|
||||
* update root's right sub tree min and max with newNode's min and max record key as applicable
|
||||
* recursively call insert() with root's right subtree as new root
|
||||
*
|
||||
* else // current root is >= newNode
|
||||
* if current root has no left sub tree
|
||||
* update current root's left sub tree max and min
|
||||
* set newNode as left sub tree
|
||||
* else
|
||||
* update root's left sub tree min and max with newNode's min and max record key as applicable
|
||||
* recursively call insert() with root's left subtree as new root
|
||||
*
|
||||
* @param root refers to the current root of the look up tree
|
||||
* @param newNode newNode the new {@link KeyRangeNode} to be inserted
|
||||
*/
|
||||
private KeyRangeNode insert(KeyRangeNode root, KeyRangeNode newNode) {
|
||||
if (root == null) {
|
||||
root = newNode;
|
||||
return root;
|
||||
}
|
||||
|
||||
if (root.compareTo(newNode) == 0) {
|
||||
root.addFiles(newNode.getFileNameList());
|
||||
return root;
|
||||
}
|
||||
|
||||
if (root.compareTo(newNode) < 0) {
|
||||
if (root.getRight() == null) {
|
||||
root.setRightSubTreeMax(newNode.getMaxRecordKey());
|
||||
root.setRightSubTreeMin(newNode.getMinRecordKey());
|
||||
root.setRight(newNode);
|
||||
} else {
|
||||
if (root.getRightSubTreeMax().compareTo(newNode.getMaxRecordKey()) < 0) {
|
||||
root.setRightSubTreeMax(newNode.getMaxRecordKey());
|
||||
}
|
||||
if (root.getRightSubTreeMin().compareTo(newNode.getMinRecordKey()) > 0) {
|
||||
root.setRightSubTreeMin(newNode.getMinRecordKey());
|
||||
}
|
||||
insert(root.getRight(), newNode);
|
||||
}
|
||||
} else {
|
||||
if (root.getLeft() == null) {
|
||||
root.setLeftSubTreeMax(newNode.getMaxRecordKey());
|
||||
root.setLeftSubTreeMin(newNode.getMinRecordKey());
|
||||
root.setLeft(newNode);
|
||||
} else {
|
||||
if (root.getLeftSubTreeMax().compareTo(newNode.getMaxRecordKey()) < 0) {
|
||||
root.setLeftSubTreeMax(newNode.getMaxRecordKey());
|
||||
}
|
||||
if (root.getLeftSubTreeMin().compareTo(newNode.getMinRecordKey()) > 0) {
|
||||
root.setLeftSubTreeMin(newNode.getMinRecordKey());
|
||||
}
|
||||
insert(root.getLeft(), newNode);
|
||||
}
|
||||
}
|
||||
return root;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches all the matching index files where the key could possibly be present.
|
||||
*
|
||||
* @param lookupKey the key to be searched for
|
||||
* @return the {@link Set} of matching index file names
|
||||
*/
|
||||
Set<String> getMatchingIndexFiles(String lookupKey) {
|
||||
Set<String> matchingFileNameSet = new HashSet<>();
|
||||
getMatchingIndexFiles(getRoot(), lookupKey, matchingFileNameSet);
|
||||
return matchingFileNameSet;
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches all the matching index files where the key could possibly be present.
|
||||
*
|
||||
* @param root refers to the current root of the look up tree
|
||||
* @param lookupKey the key to be searched for
|
||||
*/
|
||||
private void getMatchingIndexFiles(KeyRangeNode root, String lookupKey, Set<String> matchingFileNameSet) {
|
||||
if (root == null) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (root.getMinRecordKey().compareTo(lookupKey) <= 0 && lookupKey.compareTo(root.getMaxRecordKey()) <= 0) {
|
||||
matchingFileNameSet.addAll(root.getFileNameList());
|
||||
}
|
||||
|
||||
if (root.getLeftSubTreeMax() != null && root.getLeftSubTreeMin().compareTo(lookupKey) <= 0
|
||||
&& lookupKey.compareTo(root.getLeftSubTreeMax()) <= 0) {
|
||||
getMatchingIndexFiles(root.getLeft(), lookupKey, matchingFileNameSet);
|
||||
}
|
||||
|
||||
if (root.getRightSubTreeMax() != null && root.getRightSubTreeMin().compareTo(lookupKey) <= 0
|
||||
&& lookupKey.compareTo(root.getRightSubTreeMax()) <= 0) {
|
||||
getMatchingIndexFiles(root.getRight(), lookupKey, matchingFileNameSet);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,153 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Represents a node in the {@link KeyRangeLookupTree}. Holds information pertaining to a single index file, viz file
|
||||
* name, min record key and max record key.
|
||||
*/
|
||||
class KeyRangeNode implements Comparable<KeyRangeNode>, Serializable {
|
||||
|
||||
private final List<String> fileNameList = new ArrayList<>();
|
||||
private final String minRecordKey;
|
||||
private final String maxRecordKey;
|
||||
private String rightSubTreeMax = null;
|
||||
private String leftSubTreeMax = null;
|
||||
private String rightSubTreeMin = null;
|
||||
private String leftSubTreeMin = null;
|
||||
private KeyRangeNode left = null;
|
||||
private KeyRangeNode right = null;
|
||||
|
||||
/**
|
||||
* Instantiates a new {@link KeyRangeNode}
|
||||
*
|
||||
* @param minRecordKey min record key of the index file
|
||||
* @param maxRecordKey max record key of the index file
|
||||
* @param fileName file name of the index file
|
||||
*/
|
||||
KeyRangeNode(String minRecordKey, String maxRecordKey, String fileName) {
|
||||
this.fileNameList.add(fileName);
|
||||
this.minRecordKey = minRecordKey;
|
||||
this.maxRecordKey = maxRecordKey;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a new file name list to existing list of file names.
|
||||
*
|
||||
* @param newFiles {@link List} of file names to be added
|
||||
*/
|
||||
void addFiles(List<String> newFiles) {
|
||||
this.fileNameList.addAll(newFiles);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "KeyRangeNode{"
|
||||
+ "minRecordKey='" + minRecordKey + '\''
|
||||
+ ", maxRecordKey='" + maxRecordKey + '\''
|
||||
+ ", fileNameList=" + fileNameList
|
||||
+ ", rightSubTreeMax='" + rightSubTreeMax + '\''
|
||||
+ ", leftSubTreeMax='" + leftSubTreeMax + '\''
|
||||
+ ", rightSubTreeMin='" + rightSubTreeMin + '\''
|
||||
+ ", leftSubTreeMin='" + leftSubTreeMin + '\''
|
||||
+ '}';
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the min record key of two nodes, followed by max record key.
|
||||
*
|
||||
* @param that the {@link KeyRangeNode} to be compared with
|
||||
* @return the result of comparison. 0 if both min and max are equal in both. 1 if this {@link KeyRangeNode} is
|
||||
* greater than the {@code that} keyRangeNode. -1 if {@code that} keyRangeNode is greater than this {@link
|
||||
* KeyRangeNode}
|
||||
*/
|
||||
@Override
|
||||
public int compareTo(KeyRangeNode that) {
|
||||
int compareValue = minRecordKey.compareTo(that.minRecordKey);
|
||||
if (compareValue == 0) {
|
||||
return maxRecordKey.compareTo(that.maxRecordKey);
|
||||
} else {
|
||||
return compareValue;
|
||||
}
|
||||
}
|
||||
|
||||
public List<String> getFileNameList() {
|
||||
return fileNameList;
|
||||
}
|
||||
|
||||
public String getMinRecordKey() {
|
||||
return minRecordKey;
|
||||
}
|
||||
|
||||
public String getMaxRecordKey() {
|
||||
return maxRecordKey;
|
||||
}
|
||||
|
||||
public String getRightSubTreeMin() {
|
||||
return rightSubTreeMin;
|
||||
}
|
||||
|
||||
public void setRightSubTreeMin(String rightSubTreeMin) {
|
||||
this.rightSubTreeMin = rightSubTreeMin;
|
||||
}
|
||||
|
||||
public String getLeftSubTreeMin() {
|
||||
return leftSubTreeMin;
|
||||
}
|
||||
|
||||
public void setLeftSubTreeMin(String leftSubTreeMin) {
|
||||
this.leftSubTreeMin = leftSubTreeMin;
|
||||
}
|
||||
|
||||
public String getRightSubTreeMax() {
|
||||
return rightSubTreeMax;
|
||||
}
|
||||
|
||||
public void setRightSubTreeMax(String rightSubTreeMax) {
|
||||
this.rightSubTreeMax = rightSubTreeMax;
|
||||
}
|
||||
|
||||
public String getLeftSubTreeMax() {
|
||||
return leftSubTreeMax;
|
||||
}
|
||||
|
||||
public void setLeftSubTreeMax(String leftSubTreeMax) {
|
||||
this.leftSubTreeMax = leftSubTreeMax;
|
||||
}
|
||||
|
||||
public KeyRangeNode getLeft() {
|
||||
return left;
|
||||
}
|
||||
|
||||
public void setLeft(KeyRangeNode left) {
|
||||
this.left = left;
|
||||
}
|
||||
|
||||
public KeyRangeNode getRight() {
|
||||
return right;
|
||||
}
|
||||
|
||||
public void setRight(KeyRangeNode right) {
|
||||
this.right = right;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,53 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
class ListBasedGlobalIndexFileFilter extends ListBasedIndexFileFilter {
|
||||
|
||||
/**
|
||||
* Instantiates {@link ListBasedGlobalIndexFileFilter}
|
||||
*
|
||||
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}
|
||||
*/
|
||||
ListBasedGlobalIndexFileFilter(
|
||||
Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
|
||||
super(partitionToFileIndexInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getMatchingFiles(String partitionPath, String recordKey) {
|
||||
Set<String> toReturn = new HashSet<>();
|
||||
partitionToFileIndexInfo.values().forEach(indexInfos -> {
|
||||
if (indexInfos != null) { // could be null, if there are no files in a given partition yet.
|
||||
// for each candidate file in partition, that needs to be compared.
|
||||
for (BloomIndexFileInfo indexInfo : indexInfos) {
|
||||
if (shouldCompareWithFile(indexInfo, recordKey)) {
|
||||
toReturn.add(indexInfo.getFileId());
|
||||
}
|
||||
}
|
||||
}
|
||||
});
|
||||
return toReturn;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,65 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.bloom;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Simple implementation of {@link IndexFileFilter}. Sequentially goes through every index file in a given partition to
|
||||
* search for potential index files to be searched for a given record key.
|
||||
*/
|
||||
class ListBasedIndexFileFilter implements IndexFileFilter {
|
||||
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo;
|
||||
|
||||
/**
|
||||
* Instantiates {@link ListBasedIndexFileFilter}
|
||||
*
|
||||
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}
|
||||
*/
|
||||
ListBasedIndexFileFilter(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
|
||||
this.partitionToFileIndexInfo = partitionToFileIndexInfo;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getMatchingFiles(String partitionPath, String recordKey) {
|
||||
List<BloomIndexFileInfo> indexInfos = partitionToFileIndexInfo.get(partitionPath);
|
||||
Set<String> toReturn = new HashSet<>();
|
||||
if (indexInfos != null) { // could be null, if there are no files in a given partition yet.
|
||||
// for each candidate file in partition, that needs to be compared.
|
||||
for (BloomIndexFileInfo indexInfo : indexInfos) {
|
||||
if (shouldCompareWithFile(indexInfo, recordKey)) {
|
||||
toReturn.add(indexInfo.getFileId());
|
||||
}
|
||||
}
|
||||
}
|
||||
return toReturn;
|
||||
}
|
||||
|
||||
/**
|
||||
* if we dont have key ranges, then also we need to compare against the file. no other choice if we do, then only
|
||||
* compare the file if the record key falls in range.
|
||||
*/
|
||||
protected boolean shouldCompareWithFile(BloomIndexFileInfo indexInfo, String recordKey) {
|
||||
return !indexInfo.hasKeyRanges() || indexInfo.isKeyInRange(recordKey);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.hbase;
|
||||
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
public class DefaultHBaseQPSResourceAllocator implements HBaseIndexQPSResourceAllocator {
|
||||
private HoodieWriteConfig hoodieWriteConfig;
|
||||
private static Logger logger = LogManager.getLogger(DefaultHBaseQPSResourceAllocator.class);
|
||||
|
||||
public DefaultHBaseQPSResourceAllocator(HoodieWriteConfig hoodieWriteConfig) {
|
||||
this.hoodieWriteConfig = hoodieWriteConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
public float calculateQPSFractionForPutsTime(final long numPuts, final int numRegionServers) {
|
||||
// Just return the configured qps_fraction without calculating it runtime
|
||||
return hoodieWriteConfig.getHbaseIndexQPSFraction();
|
||||
}
|
||||
|
||||
@Override
|
||||
public float acquireQPSResources(final float desiredQPSFraction, final long numPuts) {
|
||||
// Return the requested QPSFraction in this default implementation
|
||||
return desiredQPSFraction;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void releaseQPSResources() {
|
||||
// Do nothing, as there are no resources locked in default implementation
|
||||
logger.info(String.format("Release QPS resources called for %s with default implementation, do nothing",
|
||||
this.hoodieWriteConfig.getHbaseTableName()));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,548 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.hbase;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.ConnectionFactory;
|
||||
import org.apache.hadoop.hbase.client.Delete;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.RegionLocator;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieDependentSystemUnavailableException;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function2;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Hoodie Index implementation backed by HBase
|
||||
*/
|
||||
public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
|
||||
|
||||
public static final String DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME =
|
||||
"spark.executor.instances";
|
||||
public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME =
|
||||
"spark.dynamicAllocation.enabled";
|
||||
public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME =
|
||||
"spark.dynamicAllocation.maxExecutors";
|
||||
|
||||
private static final byte[] SYSTEM_COLUMN_FAMILY = Bytes.toBytes("_s");
|
||||
private static final byte[] COMMIT_TS_COLUMN = Bytes.toBytes("commit_ts");
|
||||
private static final byte[] FILE_NAME_COLUMN = Bytes.toBytes("file_name");
|
||||
private static final byte[] PARTITION_PATH_COLUMN = Bytes.toBytes("partition_path");
|
||||
private static final int SLEEP_TIME_MILLISECONDS = 100;
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HBaseIndex.class);
|
||||
private static Connection hbaseConnection = null;
|
||||
private HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = null;
|
||||
private float qpsFraction;
|
||||
private int maxQpsPerRegionServer;
|
||||
/**
|
||||
* multiPutBatchSize will be computed and re-set in updateLocation if
|
||||
* {@link HoodieIndexConfig.HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP} is set to true
|
||||
*/
|
||||
private Integer multiPutBatchSize;
|
||||
private Integer numRegionServersForTable;
|
||||
private final String tableName;
|
||||
private HbasePutBatchSizeCalculator putBatchSizeCalculator;
|
||||
|
||||
public HBaseIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
this.tableName = config.getHbaseTableName();
|
||||
addShutDownHook();
|
||||
init(config);
|
||||
}
|
||||
|
||||
private void init(HoodieWriteConfig config) {
|
||||
this.multiPutBatchSize = config.getHbaseIndexGetBatchSize();
|
||||
this.qpsFraction = config.getHbaseIndexQPSFraction();
|
||||
this.maxQpsPerRegionServer = config.getHbaseIndexMaxQPSPerRegionServer();
|
||||
this.putBatchSizeCalculator = new HbasePutBatchSizeCalculator();
|
||||
this.hBaseIndexQPSResourceAllocator = createQPSResourceAllocator(this.config);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public HBaseIndexQPSResourceAllocator createQPSResourceAllocator(HoodieWriteConfig config) {
|
||||
try {
|
||||
logger.info("createQPSResourceAllocator :" + config.getHBaseQPSResourceAllocatorClass());
|
||||
final HBaseIndexQPSResourceAllocator resourceAllocator =
|
||||
(HBaseIndexQPSResourceAllocator) ReflectionUtils.loadClass(
|
||||
config.getHBaseQPSResourceAllocatorClass(), config);
|
||||
return resourceAllocator;
|
||||
} catch (Exception e) {
|
||||
logger.warn("error while instantiating HBaseIndexQPSResourceAllocator", e);
|
||||
}
|
||||
return new DefaultHBaseQPSResourceAllocator(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
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");
|
||||
}
|
||||
|
||||
private Connection getHBaseConnection() {
|
||||
Configuration hbaseConfig = HBaseConfiguration.create();
|
||||
String quorum = config.getHbaseZkQuorum();
|
||||
hbaseConfig.set("hbase.zookeeper.quorum", quorum);
|
||||
String zkZnodeParent = config.getHBaseZkZnodeParent();
|
||||
if (zkZnodeParent != null) {
|
||||
hbaseConfig.set("zookeeper.znode.parent", zkZnodeParent);
|
||||
}
|
||||
String port = String.valueOf(config.getHbaseZkPort());
|
||||
hbaseConfig.set("hbase.zookeeper.property.clientPort", port);
|
||||
try {
|
||||
return ConnectionFactory.createConnection(hbaseConfig);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieDependentSystemUnavailableException(
|
||||
HoodieDependentSystemUnavailableException.HBASE, quorum + ":" + port);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is
|
||||
* closed when JVM exits
|
||||
*/
|
||||
private void addShutDownHook() {
|
||||
Runtime.getRuntime().addShutdownHook(new Thread() {
|
||||
public void run() {
|
||||
try {
|
||||
hbaseConnection.close();
|
||||
} catch (Exception e) {
|
||||
// fail silently for any sort of exception
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure that any resources used for indexing are released here.
|
||||
*/
|
||||
public void close() {
|
||||
this.hBaseIndexQPSResourceAllocator.releaseQPSResources();
|
||||
}
|
||||
|
||||
private Get generateStatement(String key) throws IOException {
|
||||
return new Get(Bytes.toBytes(key)).setMaxVersions(1)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)
|
||||
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN);
|
||||
}
|
||||
|
||||
private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) {
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().filterCompletedInstants();
|
||||
// Check if the last commit ts for this row is 1) present in the timeline or
|
||||
// 2) is less than the first commit ts in the timeline
|
||||
return !commitTimeline.empty() && (commitTimeline
|
||||
.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))
|
||||
|| HoodieTimeline
|
||||
.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs,
|
||||
HoodieTimeline.GREATER));
|
||||
}
|
||||
|
||||
/**
|
||||
* Function that tags each HoodieRecord with an existing location, if known.
|
||||
*/
|
||||
private Function2<Integer, Iterator<HoodieRecord<T>>,
|
||||
Iterator<HoodieRecord<T>>> locationTagFunction(HoodieTableMetaClient metaClient) {
|
||||
|
||||
return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>)
|
||||
(partitionNum, hoodieRecordIterator) -> {
|
||||
|
||||
Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize();
|
||||
|
||||
// Grab the global HBase connection
|
||||
synchronized (HBaseIndex.class) {
|
||||
if (hbaseConnection == null || hbaseConnection.isClosed()) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
|
||||
HTable hTable = null;
|
||||
try {
|
||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||
List<Get> statements = new ArrayList<>();
|
||||
List<HoodieRecord> currentBatchOfRecords = new LinkedList<>();
|
||||
// Do the tagging.
|
||||
while (hoodieRecordIterator.hasNext()) {
|
||||
HoodieRecord rec = hoodieRecordIterator.next();
|
||||
statements.add(generateStatement(rec.getRecordKey()));
|
||||
currentBatchOfRecords.add(rec);
|
||||
// iterator till we reach batch size
|
||||
if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) {
|
||||
// get results for batch from Hbase
|
||||
Result[] results = doGet(hTable, statements);
|
||||
// clear statements to be GC'd
|
||||
statements.clear();
|
||||
for (Result result : results) {
|
||||
// first, attempt to grab location from HBase
|
||||
HoodieRecord currentRecord = currentBatchOfRecords.remove(0);
|
||||
if (result.getRow() != null) {
|
||||
String keyFromResult = Bytes.toString(result.getRow());
|
||||
String commitTs = Bytes
|
||||
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
|
||||
String fileId = Bytes
|
||||
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
|
||||
String partitionPath = Bytes
|
||||
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
|
||||
|
||||
if (checkIfValidCommit(metaClient, commitTs)) {
|
||||
currentRecord = new HoodieRecord(
|
||||
new HoodieKey(currentRecord.getRecordKey(), partitionPath),
|
||||
currentRecord.getData());
|
||||
currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
|
||||
taggedRecords.add(currentRecord);
|
||||
// the key from Result and the key being processed should be same
|
||||
assert (currentRecord.getRecordKey().contentEquals(keyFromResult));
|
||||
} else { //if commit is invalid, treat this as a new taggedRecord
|
||||
taggedRecords.add(currentRecord);
|
||||
}
|
||||
} else {
|
||||
taggedRecords.add(currentRecord);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException(
|
||||
"Failed to Tag indexed locations because of exception with HBase Client", e);
|
||||
} finally {
|
||||
if (hTable != null) {
|
||||
try {
|
||||
hTable.close();
|
||||
} catch (IOException e) {
|
||||
// Ignore
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
return taggedRecords.iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private Result[] doGet(HTable hTable, List<Get> keys) throws IOException {
|
||||
sleepForTime(SLEEP_TIME_MILLISECONDS);
|
||||
return hTable.get(keys);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
return recordRDD.mapPartitionsWithIndex(locationTagFunction(hoodieTable.getMetaClient()), true);
|
||||
}
|
||||
|
||||
private Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>> updateLocationFunction() {
|
||||
|
||||
return (Function2<Integer, Iterator<WriteStatus>, Iterator<WriteStatus>>) (partition, statusIterator) -> {
|
||||
|
||||
List<WriteStatus> writeStatusList = new ArrayList<>();
|
||||
// Grab the global HBase connection
|
||||
synchronized (HBaseIndex.class) {
|
||||
if (hbaseConnection == null || hbaseConnection.isClosed()) {
|
||||
hbaseConnection = getHBaseConnection();
|
||||
}
|
||||
}
|
||||
HTable hTable = null;
|
||||
try {
|
||||
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
|
||||
while (statusIterator.hasNext()) {
|
||||
WriteStatus writeStatus = statusIterator.next();
|
||||
List<Put> puts = new ArrayList<>();
|
||||
List<Delete> deletes = new ArrayList<>();
|
||||
try {
|
||||
for (HoodieRecord rec : writeStatus.getWrittenRecords()) {
|
||||
if (!writeStatus.isErrored(rec.getKey())) {
|
||||
Option<HoodieRecordLocation> loc = rec.getNewLocation();
|
||||
if (loc.isPresent()) {
|
||||
if (rec.getCurrentLocation() != null) {
|
||||
// This is an update, no need to update index
|
||||
continue;
|
||||
}
|
||||
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
|
||||
Bytes.toBytes(loc.get().getInstantTime()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
|
||||
Bytes.toBytes(loc.get().getFileId()));
|
||||
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
|
||||
Bytes.toBytes(rec.getPartitionPath()));
|
||||
puts.add(put);
|
||||
} else {
|
||||
//Delete existing index for a deleted record
|
||||
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
|
||||
deletes.add(delete);
|
||||
}
|
||||
}
|
||||
if (puts.size() + deletes.size() < multiPutBatchSize) {
|
||||
continue;
|
||||
}
|
||||
doPutsAndDeletes(hTable, puts, deletes);
|
||||
}
|
||||
//process remaining puts and deletes, if any
|
||||
doPutsAndDeletes(hTable, puts, deletes);
|
||||
} catch (Exception e) {
|
||||
Exception we = new Exception("Error updating index for " + writeStatus, e);
|
||||
logger.error(we);
|
||||
writeStatus.setGlobalError(we);
|
||||
}
|
||||
writeStatusList.add(writeStatus);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException(
|
||||
"Failed to Update Index locations because of exception with HBase Client", e);
|
||||
} finally {
|
||||
if (hTable != null) {
|
||||
try {
|
||||
hTable.close();
|
||||
} catch (IOException e) {
|
||||
// Ignore
|
||||
}
|
||||
}
|
||||
}
|
||||
return writeStatusList.iterator();
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to facilitate performing puts and deletes in Hbase
|
||||
*/
|
||||
private void doPutsAndDeletes(HTable hTable, List<Put> puts, List<Delete> deletes)
|
||||
throws IOException {
|
||||
if (puts.size() > 0) {
|
||||
hTable.put(puts);
|
||||
}
|
||||
if (deletes.size() > 0) {
|
||||
hTable.delete(deletes);
|
||||
}
|
||||
hTable.flushCommits();
|
||||
puts.clear();
|
||||
deletes.clear();
|
||||
sleepForTime(SLEEP_TIME_MILLISECONDS);
|
||||
}
|
||||
|
||||
private static void sleepForTime(int sleepTimeMs) {
|
||||
try {
|
||||
Thread.sleep(sleepTimeMs);
|
||||
} catch (InterruptedException e) {
|
||||
logger.error("Sleep interrupted during throttling", e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
|
||||
HoodieTable<T> hoodieTable) {
|
||||
final HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = createQPSResourceAllocator(this.config);
|
||||
setPutBatchSize(writeStatusRDD, hBaseIndexQPSResourceAllocator, jsc);
|
||||
logger.info("multiPutBatchSize: before hbase puts" + multiPutBatchSize);
|
||||
JavaRDD<WriteStatus> writeStatusJavaRDD = writeStatusRDD.mapPartitionsWithIndex(
|
||||
updateLocationFunction(), true);
|
||||
// caching the index updated status RDD
|
||||
writeStatusJavaRDD = writeStatusJavaRDD.persist(config.getWriteStatusStorageLevel());
|
||||
return writeStatusJavaRDD;
|
||||
}
|
||||
|
||||
private void setPutBatchSize(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator,
|
||||
final JavaSparkContext jsc) {
|
||||
if (config.getHbaseIndexPutBatchSizeAutoCompute()) {
|
||||
SparkConf conf = jsc.getConf();
|
||||
int maxExecutors = conf.getInt(DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME, 1);
|
||||
if (conf.getBoolean(DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME, false)) {
|
||||
maxExecutors = Math.max(maxExecutors, conf.getInt(
|
||||
DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME, 1));
|
||||
}
|
||||
|
||||
/*
|
||||
Each writeStatus represents status information from a write done in one of the IOHandles.
|
||||
If a writeStatus has any insert, it implies that the corresponding task contacts HBase for
|
||||
doing puts, since we only do puts for inserts from HBaseIndex.
|
||||
*/
|
||||
final Tuple2<Long, Integer> numPutsParallelismTuple = getHBasePutAccessParallelism(writeStatusRDD);
|
||||
final long numPuts = numPutsParallelismTuple._1;
|
||||
final int hbasePutsParallelism = numPutsParallelismTuple._2;
|
||||
this.numRegionServersForTable = getNumRegionServersAliveForTable();
|
||||
final float desiredQPSFraction = hBaseIndexQPSResourceAllocator
|
||||
.calculateQPSFractionForPutsTime(numPuts, this.numRegionServersForTable);
|
||||
logger.info("Desired QPSFraction :" + desiredQPSFraction);
|
||||
logger.info("Number HBase puts :" + numPuts);
|
||||
logger.info("Hbase Puts Parallelism :" + hbasePutsParallelism);
|
||||
final float availableQpsFraction = hBaseIndexQPSResourceAllocator
|
||||
.acquireQPSResources(desiredQPSFraction, numPuts);
|
||||
logger.info("Allocated QPS Fraction :" + availableQpsFraction);
|
||||
multiPutBatchSize = putBatchSizeCalculator
|
||||
.getBatchSize(
|
||||
numRegionServersForTable,
|
||||
maxQpsPerRegionServer,
|
||||
hbasePutsParallelism,
|
||||
maxExecutors,
|
||||
SLEEP_TIME_MILLISECONDS,
|
||||
availableQpsFraction);
|
||||
logger.info("multiPutBatchSize :" + multiPutBatchSize);
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Tuple2<Long, Integer> getHBasePutAccessParallelism(final JavaRDD<WriteStatus> writeStatusRDD) {
|
||||
final JavaPairRDD<Long, Integer> insertOnlyWriteStatusRDD =
|
||||
writeStatusRDD.filter(w -> w.getStat().getNumInserts() > 0)
|
||||
.mapToPair(w -> new Tuple2<>(w.getStat().getNumInserts(), 1));
|
||||
return insertOnlyWriteStatusRDD.fold(new Tuple2<>(0L, 0), (w, c) -> new Tuple2<>(w._1 + c._1, w._2 + c._2));
|
||||
}
|
||||
|
||||
public static class HbasePutBatchSizeCalculator implements Serializable {
|
||||
|
||||
private static final int MILLI_SECONDS_IN_A_SECOND = 1000;
|
||||
private static Logger logger = LogManager.getLogger(HbasePutBatchSizeCalculator.class);
|
||||
|
||||
/**
|
||||
* Calculate putBatch size so that sum of requests across multiple jobs in a second does not exceed
|
||||
* maxQpsPerRegionServer for each Region Server. Multiplying qpsFraction to reduce the aggregate load on common RS
|
||||
* across topics. Assumption here is that all tables have regions across all RS, which is not necessarily true for
|
||||
* smaller tables. So, they end up getting a smaller share of QPS than they deserve, but it might be ok.
|
||||
* <p>
|
||||
* Example: int putBatchSize = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.1f)
|
||||
* </p>
|
||||
* <p>
|
||||
* Expected batchSize is 8 because in that case, total request sent to a Region Server in one second is:
|
||||
*
|
||||
* 8 (batchSize) * 200 (parallelism) * 10 (maxReqsInOneSecond) * 10 (numRegionServers) * 0.1 (qpsFraction)) =>
|
||||
* 16000. We assume requests get distributed to Region Servers uniformly, so each RS gets 1600 requests which
|
||||
* happens to be 10% of 16667 (maxQPSPerRegionServer), as expected.
|
||||
* </p>
|
||||
* <p> Assumptions made here <li> In a batch, writes get evenly distributed to each RS for that
|
||||
* table. Since we do writes only in the case of inserts and not updates, for this assumption to fail, inserts would
|
||||
* have to be skewed towards few RS, likelihood of which is less if Hbase table is pre-split and rowKeys are UUIDs
|
||||
* (random strings). If this assumption fails, then it is possible for some RS to receive more than
|
||||
* maxQpsPerRegionServer QPS, but for simplicity, we are going ahead with this model, since this is meant to be a
|
||||
* lightweight distributed throttling mechanism without maintaining a global context. So if this assumption breaks,
|
||||
* we are hoping the HBase Master relocates hot-spot regions to new Region Servers.
|
||||
*
|
||||
* </li> <li> For Region Server stability, throttling at a second level granularity is fine.
|
||||
* Although, within a second, the sum of queries might be within maxQpsPerRegionServer, there could be peaks at some
|
||||
* sub second intervals. So, the assumption is that these peaks are tolerated by the Region Server (which at max can
|
||||
* be maxQpsPerRegionServer). </li> </p>
|
||||
*/
|
||||
public int getBatchSize(int numRegionServersForTable, int maxQpsPerRegionServer,
|
||||
int numTasksDuringPut, int maxExecutors, int sleepTimeMs, float qpsFraction) {
|
||||
int numRSAlive = numRegionServersForTable;
|
||||
int maxReqPerSec = (int) (qpsFraction * numRSAlive * maxQpsPerRegionServer);
|
||||
int numTasks = numTasksDuringPut;
|
||||
int maxParallelPuts = Math.max(1, Math.min(numTasks, maxExecutors));
|
||||
int maxReqsSentPerTaskPerSec = MILLI_SECONDS_IN_A_SECOND / sleepTimeMs;
|
||||
int multiPutBatchSize = Math.max(1, maxReqPerSec / (maxParallelPuts * maxReqsSentPerTaskPerSec));
|
||||
logger.info("HbaseIndexThrottling: qpsFraction :" + qpsFraction);
|
||||
logger.info("HbaseIndexThrottling: numRSAlive :" + numRSAlive);
|
||||
logger.info("HbaseIndexThrottling: maxReqPerSec :" + maxReqPerSec);
|
||||
logger.info("HbaseIndexThrottling: numTasks :" + numTasks);
|
||||
logger.info("HbaseIndexThrottling: maxExecutors :" + maxExecutors);
|
||||
logger.info("HbaseIndexThrottling: maxParallelPuts :" + maxParallelPuts);
|
||||
logger.info("HbaseIndexThrottling: maxReqsSentPerTaskPerSec :" + maxReqsSentPerTaskPerSec);
|
||||
logger.info("HbaseIndexThrottling: numRegionServersForTable :" + numRegionServersForTable);
|
||||
logger.info("HbaseIndexThrottling: multiPutBatchSize :" + multiPutBatchSize);
|
||||
return multiPutBatchSize;
|
||||
}
|
||||
}
|
||||
|
||||
private Integer getNumRegionServersAliveForTable() {
|
||||
// This is being called in the driver, so there is only one connection
|
||||
// from the driver, so ok to use a local connection variable.
|
||||
if (numRegionServersForTable == null) {
|
||||
try (Connection conn = getHBaseConnection()) {
|
||||
RegionLocator regionLocator = conn
|
||||
.getRegionLocator(TableName.valueOf(tableName));
|
||||
numRegionServersForTable = Math.toIntExact(
|
||||
regionLocator.getAllRegionLocations().stream().map(e -> e.getServerName()).distinct()
|
||||
.count());
|
||||
return numRegionServersForTable;
|
||||
} catch (IOException e) {
|
||||
logger.error(e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
return numRegionServersForTable;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String commitTime) {
|
||||
// Rollback in HbaseIndex is managed via method {@link #checkIfValidCommit()}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Only looks up by recordKey
|
||||
*/
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Mapping is available in HBase already.
|
||||
*/
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Index needs to be explicitly updated after storage write.
|
||||
*/
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setHbaseConnection(Connection hbaseConnection) {
|
||||
HBaseIndex.hbaseConnection = hbaseConnection;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.index.hbase;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* <code>HBaseIndexQPSResourceAllocator</code> defines methods to manage resource allocation for HBase index operations
|
||||
*/
|
||||
public interface HBaseIndexQPSResourceAllocator extends Serializable {
|
||||
|
||||
/**
|
||||
* This method returns the QPS Fraction value that needs to be acquired such that the respective
|
||||
* HBase index operation can be completed in desiredPutsTime.
|
||||
*
|
||||
* @param numPuts Number of inserts to be written to HBase index
|
||||
* @param desiredPutsTimeInSecs Total expected time for the HBase inserts operation
|
||||
* @return QPS fraction that needs to be acquired.
|
||||
*/
|
||||
float calculateQPSFractionForPutsTime(final long numPuts, final int desiredPutsTimeInSecs);
|
||||
|
||||
/**
|
||||
* This method acquires the requested QPS Fraction against HBase cluster for index operation.
|
||||
*
|
||||
* @param desiredQPSFraction QPS fraction that needs to be requested and acquired
|
||||
* @param numPuts Number of inserts to be written to HBase index
|
||||
* @return value of the acquired QPS Fraction.
|
||||
*/
|
||||
float acquireQPSResources(final float desiredQPSFraction, final long numPuts);
|
||||
|
||||
/**
|
||||
* This method releases the acquired QPS Fraction
|
||||
*/
|
||||
void releaseQPSResources();
|
||||
}
|
||||
@@ -0,0 +1,322 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import com.beust.jcommander.internal.Maps;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieDeltaWriteStat;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.RealtimeView;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieDeleteBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieAppendException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.util.SizeEstimator;
|
||||
|
||||
/**
|
||||
* IO Operation to append data onto an existing file.
|
||||
*/
|
||||
public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWriteHandle<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class);
|
||||
// This acts as the sequenceID for records written
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
private final String fileId;
|
||||
// Buffer for holding records in memory before they are flushed to disk
|
||||
private List<IndexedRecord> recordList = new ArrayList<>();
|
||||
// Buffer for holding records (to be deleted) in memory before they are flushed to disk
|
||||
private List<HoodieKey> keysToDelete = new ArrayList<>();
|
||||
|
||||
private String partitionPath;
|
||||
private Iterator<HoodieRecord<T>> recordItr;
|
||||
// Total number of records written during an append
|
||||
private long recordsWritten = 0;
|
||||
// Total number of records deleted during an append
|
||||
private long recordsDeleted = 0;
|
||||
// Total number of records updated during an append
|
||||
private long updatedRecordsWritten = 0;
|
||||
// Average record size for a HoodieRecord. This size is updated at the end of every log block flushed to disk
|
||||
private long averageRecordSize = 0;
|
||||
private HoodieLogFile currentLogFile;
|
||||
private Writer writer;
|
||||
// Flag used to initialize some metadata
|
||||
private boolean doInit = true;
|
||||
// Total number of bytes written during this append phase (an estimation)
|
||||
private long estimatedNumberOfBytesWritten;
|
||||
// Number of records that must be written to meet the max block size for a log block
|
||||
private int numberOfRecords = 0;
|
||||
// Max block size to limit to for a log block
|
||||
private int maxBlockSize = config.getLogFileDataBlockMaxSize();
|
||||
// Header metadata for a log block
|
||||
private Map<HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
// Total number of new records inserted into the delta file
|
||||
private long insertRecordsWritten = 0;
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
super(config, commitTime, fileId, hoodieTable);
|
||||
writeStatus.setStat(new HoodieDeltaWriteStat());
|
||||
this.fileId = fileId;
|
||||
this.recordItr = recordItr;
|
||||
}
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable, String fileId) {
|
||||
this(config, commitTime, hoodieTable, fileId, null);
|
||||
}
|
||||
|
||||
private void init(HoodieRecord record) {
|
||||
if (doInit) {
|
||||
this.partitionPath = record.getPartitionPath();
|
||||
// extract some information from the first record
|
||||
RealtimeView rtView = hoodieTable.getRTFileSystemView();
|
||||
Option<FileSlice> fileSlice = rtView.getLatestFileSlice(partitionPath, fileId);
|
||||
// Set the base commit time as the current commitTime for new inserts into log files
|
||||
String baseInstantTime = instantTime;
|
||||
if (fileSlice.isPresent()) {
|
||||
baseInstantTime = fileSlice.get().getBaseInstantTime();
|
||||
} else {
|
||||
// This means there is no base data file, start appending to a new log file
|
||||
fileSlice = Option.of(new FileSlice(partitionPath, baseInstantTime, this.fileId));
|
||||
logger.info("New InsertHandle for partition :" + partitionPath);
|
||||
}
|
||||
writeStatus.getStat().setPrevCommit(baseInstantTime);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
averageRecordSize = SizeEstimator.estimate(record);
|
||||
try {
|
||||
this.writer = createLogWriter(fileSlice, baseInstantTime);
|
||||
this.currentLogFile = writer.getLogFile();
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion());
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize());
|
||||
} catch (Exception e) {
|
||||
logger.error("Error in update task at commit " + instantTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieAppendHandle for FileId: " + fileId + " on commit "
|
||||
+ instantTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath()
|
||||
+ partitionPath, e);
|
||||
}
|
||||
Path path = new Path(partitionPath, writer.getLogFile().getFileName());
|
||||
writeStatus.getStat().setPath(path.toString());
|
||||
doInit = false;
|
||||
}
|
||||
}
|
||||
|
||||
private Option<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
|
||||
Option recordMetadata = hoodieRecord.getData().getMetadata();
|
||||
try {
|
||||
Option<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(originalSchema);
|
||||
if (avroRecord.isPresent()) {
|
||||
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
|
||||
avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get()));
|
||||
String seqId = HoodieRecord.generateSequenceId(instantTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils
|
||||
.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
|
||||
hoodieRecord.getPartitionPath(), fileId);
|
||||
HoodieAvroUtils
|
||||
.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), instantTime, seqId);
|
||||
// If currentLocation is present, then this is an update
|
||||
if (hoodieRecord.getCurrentLocation() != null) {
|
||||
updatedRecordsWritten++;
|
||||
} else {
|
||||
insertRecordsWritten++;
|
||||
}
|
||||
recordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
hoodieRecord.deflate();
|
||||
return avroRecord;
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record " + hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
// TODO (NA) - Perform a writerSchema check of current input record with the last writerSchema on log file
|
||||
// to make sure we don't append records with older (shorter) writerSchema than already appended
|
||||
public void doAppend() {
|
||||
while (recordItr.hasNext()) {
|
||||
HoodieRecord record = recordItr.next();
|
||||
init(record);
|
||||
flushToDiskIfRequired(record);
|
||||
writeToBuffer(record);
|
||||
}
|
||||
doAppend(header);
|
||||
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
|
||||
}
|
||||
|
||||
private void doAppend(Map<HeaderMetadataType, String> header) {
|
||||
try {
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime);
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writerSchema.toString());
|
||||
if (recordList.size() > 0) {
|
||||
writer = writer.appendBlock(new HoodieAvroDataBlock(recordList, header));
|
||||
recordList.clear();
|
||||
}
|
||||
if (keysToDelete.size() > 0) {
|
||||
writer = writer.appendBlock(
|
||||
new HoodieDeleteBlock(keysToDelete.stream().toArray(HoodieKey[]::new), header));
|
||||
keysToDelete.clear();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieAppendException(
|
||||
"Failed while appending records to " + currentLogFile.getPath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return config.getParquetMaxFileSize() >= estimatedNumberOfBytesWritten * config
|
||||
.getLogFileToParquetCompressionRatio();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(HoodieRecord record, Option<IndexedRecord> insertValue) {
|
||||
Option recordMetadata = record.getData().getMetadata();
|
||||
try {
|
||||
init(record);
|
||||
flushToDiskIfRequired(record);
|
||||
writeToBuffer(record);
|
||||
} catch (Throwable t) {
|
||||
// Not throwing exception from here, since we don't want to fail the entire job
|
||||
// for a single record
|
||||
writeStatus.markFailure(record, t, recordMetadata);
|
||||
logger.error("Error writing record " + record, t);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus close() {
|
||||
try {
|
||||
// flush any remaining records to disk
|
||||
doAppend(header);
|
||||
long sizeInBytes = writer.getCurrentSize();
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
|
||||
HoodieWriteStat stat = writeStatus.getStat();
|
||||
stat.setFileId(this.fileId);
|
||||
stat.setNumWrites(recordsWritten);
|
||||
stat.setNumUpdateWrites(updatedRecordsWritten);
|
||||
stat.setNumInserts(insertRecordsWritten);
|
||||
stat.setNumDeletes(recordsDeleted);
|
||||
stat.setTotalWriteBytes(estimatedNumberOfBytesWritten);
|
||||
stat.setFileSizeInBytes(sizeInBytes);
|
||||
stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalUpsertTime(timer.endTimer());
|
||||
stat.setRuntimeStats(runtimeStats);
|
||||
|
||||
logger.info(String.format("AppendHandle for partitionPath %s fileID %s, took %d ms.",
|
||||
stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalUpsertTime()));
|
||||
|
||||
return writeStatus;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
private Writer createLogWriter(Option<FileSlice> fileSlice, String baseCommitTime)
|
||||
throws IOException, InterruptedException {
|
||||
Option<HoodieLogFile> latestLogFile = fileSlice.get().getLatestLogFile();
|
||||
|
||||
return HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(FSUtils.getPartitionPath(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(
|
||||
latestLogFile.map(HoodieLogFile::getLogVersion).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
|
||||
.withSizeThreshold(config.getLogFileMaxSize()).withFs(fs)
|
||||
.withLogWriteToken(
|
||||
latestLogFile.map(x -> FSUtils.getWriteTokenFromLogPath(x.getPath())).orElse(writeToken))
|
||||
.withRolloverLogWriteToken(writeToken)
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
}
|
||||
|
||||
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));
|
||||
Option<IndexedRecord> indexedRecord = getIndexedRecord(record);
|
||||
if (indexedRecord.isPresent()) {
|
||||
recordList.add(indexedRecord.get());
|
||||
} else {
|
||||
keysToDelete.add(record.getKey());
|
||||
}
|
||||
numberOfRecords++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the number of records have reached the set threshold and then flushes the records to disk
|
||||
*/
|
||||
private void flushToDiskIfRequired(HoodieRecord record) {
|
||||
// Append if max number of records reached to achieve block size
|
||||
if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) {
|
||||
// Recompute averageRecordSize before writing a new block and update existing value with
|
||||
// avg of new and old
|
||||
logger.info("AvgRecordSize => " + averageRecordSize);
|
||||
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2;
|
||||
doAppend(header);
|
||||
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
|
||||
numberOfRecords = 0;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,273 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
|
||||
* <p> 1) It provides sufficient time for existing queries running on older versions, to close <p>
|
||||
* 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done
|
||||
* based on {@link HoodieCommitMetadata}
|
||||
*/
|
||||
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCleanHelper.class);
|
||||
|
||||
private final SyncableFileSystemView fileSystemView;
|
||||
private final HoodieTimeline commitTimeline;
|
||||
private final Map<HoodieFileGroupId, CompactionOperation> fgIdToPendingCompactionOperations;
|
||||
private HoodieTable<T> hoodieTable;
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
public HoodieCleanHelper(HoodieTable<T> hoodieTable, HoodieWriteConfig config) {
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fileSystemView = hoodieTable.getHoodieView();
|
||||
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
|
||||
this.config = config;
|
||||
this.fgIdToPendingCompactionOperations =
|
||||
((SyncableFileSystemView)hoodieTable.getRTFileSystemView()).getPendingCompactionOperations()
|
||||
.map(entry -> Pair.of(new HoodieFileGroupId(entry.getValue().getPartitionPath(),
|
||||
entry.getValue().getFileId()), entry.getValue()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
}
|
||||
|
||||
/**
|
||||
* Selects the older versions of files for cleaning, such that it bounds the number of versions of
|
||||
* each file. This policy is useful, if you are simply interested in querying the table, and you
|
||||
* don't want too many versions for a single file (i.e run it with versionsRetained = 1)
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
|
||||
throws IOException {
|
||||
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
|
||||
.getCleanerFileVersionsRetained() + " file versions. ");
|
||||
List<HoodieFileGroup> fileGroups = fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
||||
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
int keepVersions = config.getCleanerFileVersionsRetained();
|
||||
// do not cleanup slice required for pending compaction
|
||||
Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices()
|
||||
.filter(fs -> !isFileSliceNeededForPendingCompaction(fs)).iterator();
|
||||
if (isFileGroupInPendingCompaction(fileGroup)) {
|
||||
// We have already saved the last version of file-groups for pending compaction Id
|
||||
keepVersions--;
|
||||
}
|
||||
|
||||
while (fileSliceIterator.hasNext() && keepVersions > 0) {
|
||||
// Skip this most recent version
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
Option<HoodieDataFile> dataFile = nextSlice.getDataFile();
|
||||
if (dataFile.isPresent() && savepointedFiles.contains(dataFile.get().getFileName())) {
|
||||
// do not clean up a savepoint data file
|
||||
continue;
|
||||
}
|
||||
keepVersions--;
|
||||
}
|
||||
// Delete the remaining files
|
||||
while (fileSliceIterator.hasNext()) {
|
||||
FileSlice nextSlice = fileSliceIterator.next();
|
||||
if (nextSlice.getDataFile().isPresent()) {
|
||||
HoodieDataFile dataFile = nextSlice.getDataFile().get();
|
||||
deletePaths.add(dataFile.getPath());
|
||||
}
|
||||
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
}
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Selects the versions for file for cleaning, such that it <p> - Leaves the latest version of the
|
||||
* file untouched - For older versions, - It leaves all the commits untouched which has occured in
|
||||
* last <code>config.getCleanerCommitsRetained()</code> commits - It leaves ONE commit before this
|
||||
* window. We assume that the max(query execution time) == commit_batch_time *
|
||||
* config.getCleanerCommitsRetained(). This is 12 hours by default. This is essential to leave the
|
||||
* file used by the query thats running for the max time. <p> This provides the effect of having
|
||||
* lookback into all changes that happened in the last X commits. (eg: if you retain 24 commits,
|
||||
* and commit batch time is 30 mins, then you have 12 hrs of lookback) <p> This policy is the
|
||||
* default.
|
||||
*/
|
||||
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath)
|
||||
throws IOException {
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
logger
|
||||
.info("Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
|
||||
List<String> deletePaths = new ArrayList<>();
|
||||
|
||||
// Collect all the datafiles savepointed by all the savepoints
|
||||
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
|
||||
.flatMap(s -> hoodieTable.getSavepointedDataFiles(s)).collect(Collectors.toList());
|
||||
|
||||
// determine if we have enough commits, to start cleaning.
|
||||
if (commitTimeline.countInstants() > commitsRetained) {
|
||||
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
|
||||
List<HoodieFileGroup> fileGroups = fileSystemView.getAllFileGroups(partitionPath)
|
||||
.collect(Collectors.toList());
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
List<FileSlice> fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList());
|
||||
|
||||
if (fileSliceList.isEmpty()) {
|
||||
continue;
|
||||
}
|
||||
|
||||
String lastVersion = fileSliceList.get(0).getBaseInstantTime();
|
||||
String lastVersionBeforeEarliestCommitToRetain = getLatestVersionBeforeCommit(fileSliceList,
|
||||
earliestCommitToRetain);
|
||||
|
||||
// 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) {
|
||||
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
|
||||
continue;
|
||||
}
|
||||
// Dont delete the latest commit and also the last commit before the earliest commit we
|
||||
// are retaining
|
||||
// The window of commit retain == max query run time. So a query could be running which
|
||||
// still
|
||||
// uses this file.
|
||||
if (fileCommitTime.equals(lastVersion) || (fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||
// move on to the next file
|
||||
continue;
|
||||
}
|
||||
|
||||
// Always keep the last commit
|
||||
if (!isFileSliceNeededForPendingCompaction(aSlice)
|
||||
&& HoodieTimeline
|
||||
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this is a commit, that should be cleaned.
|
||||
aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getPath()));
|
||||
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
// If merge on read, then clean the log files for the commits as well
|
||||
deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString())
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the latest version < commitTime. This version file could still be used by queries.
|
||||
*/
|
||||
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList,
|
||||
HoodieInstant commitTime) {
|
||||
for (FileSlice file : fileSliceList) {
|
||||
String fileCommitTime = file.getBaseInstantTime();
|
||||
if (HoodieTimeline
|
||||
.compareTimestamps(commitTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
|
||||
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the
|
||||
// one we want
|
||||
return fileCommitTime;
|
||||
}
|
||||
}
|
||||
// There is no version of this file which is <= commitTime
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns files to be cleaned for the given partitionPath based on cleaning policy.
|
||||
*/
|
||||
public List<String> getDeletePaths(String partitionPath) throws IOException {
|
||||
HoodieCleaningPolicy policy = config.getCleanerPolicy();
|
||||
List<String> deletePaths;
|
||||
if (policy == HoodieCleaningPolicy.KEEP_LATEST_COMMITS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestCommits(partitionPath);
|
||||
} else if (policy == HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS) {
|
||||
deletePaths = getFilesToCleanKeepingLatestVersions(partitionPath);
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unknown cleaning policy : " + policy.name());
|
||||
}
|
||||
logger.info(deletePaths.size() + " patterns used to delete in partition path:" + partitionPath);
|
||||
|
||||
return deletePaths;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns earliest commit to retain based on cleaning policy.
|
||||
*/
|
||||
public Option<HoodieInstant> getEarliestCommitToRetain() {
|
||||
Option<HoodieInstant> earliestCommitToRetain = Option.empty();
|
||||
int commitsRetained = config.getCleanerCommitsRetained();
|
||||
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
|
||||
&& commitTimeline.countInstants() > commitsRetained) {
|
||||
earliestCommitToRetain = commitTimeline
|
||||
.nthInstant(commitTimeline.countInstants() - commitsRetained);
|
||||
}
|
||||
return earliestCommitToRetain;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determine if file slice needed to be preserved for pending compaction
|
||||
* @param fileSlice File Slice
|
||||
* @return true if file slice needs to be preserved, false otherwise.
|
||||
*/
|
||||
private boolean isFileSliceNeededForPendingCompaction(FileSlice fileSlice) {
|
||||
CompactionOperation op = fgIdToPendingCompactionOperations.get(fileSlice.getFileGroupId());
|
||||
if (null != op) {
|
||||
// If file slice's instant time is newer or same as that of operation, do not clean
|
||||
return HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), op.getBaseInstantTime(),
|
||||
HoodieTimeline.GREATER_OR_EQUAL);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
private boolean isFileGroupInPendingCompaction(HoodieFileGroup fg) {
|
||||
return fgIdToPendingCompactionOperations.containsKey(fg.getFileGroupId());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,337 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.common.model.ActionType;
|
||||
import org.apache.hudi.common.model.HoodieArchivedLogFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Archiver to bound the growth of <action>.commit files
|
||||
*/
|
||||
public class HoodieCommitArchiveLog {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HoodieCommitArchiveLog.class);
|
||||
|
||||
private final Path archiveFilePath;
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieWriteConfig config;
|
||||
private Writer writer;
|
||||
|
||||
public HoodieCommitArchiveLog(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
this.config = config;
|
||||
this.metaClient = metaClient;
|
||||
this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath());
|
||||
}
|
||||
|
||||
private Writer openWriter() {
|
||||
try {
|
||||
if (this.writer == null) {
|
||||
return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent())
|
||||
.withFileId(archiveFilePath.getName())
|
||||
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFs(metaClient.getFs())
|
||||
.overBaseCommit("").build();
|
||||
} else {
|
||||
return this.writer;
|
||||
}
|
||||
} catch (InterruptedException | IOException e) {
|
||||
throw new HoodieException("Unable to initialize HoodieLogFormat writer", e);
|
||||
}
|
||||
}
|
||||
|
||||
private void close() {
|
||||
try {
|
||||
if (this.writer != null) {
|
||||
this.writer.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Unable to close HoodieLogFormat writer", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if commits need to be archived. If yes, archive commits.
|
||||
*/
|
||||
public boolean archiveIfRequired(final JavaSparkContext jsc) throws IOException {
|
||||
try {
|
||||
List<HoodieInstant> instantsToArchive = getInstantsToArchive(jsc).collect(Collectors.toList());
|
||||
boolean success = true;
|
||||
if (instantsToArchive.iterator().hasNext()) {
|
||||
this.writer = openWriter();
|
||||
log.info("Archiving instants " + instantsToArchive);
|
||||
archive(instantsToArchive);
|
||||
success = deleteArchivedInstants(instantsToArchive);
|
||||
} else {
|
||||
log.info("No Instants to archive");
|
||||
}
|
||||
return success;
|
||||
} finally {
|
||||
close();
|
||||
}
|
||||
}
|
||||
|
||||
private Stream<HoodieInstant> getInstantsToArchive(JavaSparkContext jsc) {
|
||||
|
||||
// TODO : rename to max/minInstantsToKeep
|
||||
int maxCommitsToKeep = config.getMaxCommitsToKeep();
|
||||
int minCommitsToKeep = config.getMinCommitsToKeep();
|
||||
|
||||
HoodieTable table = HoodieTable.getHoodieTable(metaClient, config, jsc);
|
||||
|
||||
// GroupBy each action and limit each action timeline to maxCommitsToKeep
|
||||
// TODO: Handle ROLLBACK_ACTION in future
|
||||
// ROLLBACK_ACTION is currently not defined in HoodieActiveTimeline
|
||||
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION))
|
||||
.filterCompletedInstants();
|
||||
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
|
||||
.collect(Collectors.groupingBy(s -> s.getAction())).entrySet().stream().map(i -> {
|
||||
if (i.getValue().size() > maxCommitsToKeep) {
|
||||
return i.getValue().subList(0, i.getValue().size() - minCommitsToKeep);
|
||||
} else {
|
||||
return new ArrayList<HoodieInstant>();
|
||||
}
|
||||
}).flatMap(i -> i.stream());
|
||||
|
||||
//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();
|
||||
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.
|
||||
Option<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
|
||||
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
|
||||
// Actually do the commits
|
||||
instants = Stream.concat(instants, commitTimeline.getInstants()
|
||||
.filter(s -> {
|
||||
// if no savepoint present, then dont filter
|
||||
return !(firstSavepoint.isPresent() && HoodieTimeline
|
||||
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
|
||||
HoodieTimeline.LESSER_OR_EQUAL));
|
||||
})
|
||||
.filter(s -> {
|
||||
// Ensure commits >= oldest pending compaction commit is retained
|
||||
return oldestPendingCompactionInstant.map(instant -> {
|
||||
return HoodieTimeline.compareTimestamps(instant.getTimestamp(), s.getTimestamp(), HoodieTimeline.GREATER);
|
||||
}).orElse(true);
|
||||
})
|
||||
.limit(commitTimeline.countInstants() - minCommitsToKeep));
|
||||
}
|
||||
|
||||
return instants;
|
||||
}
|
||||
|
||||
private boolean deleteArchivedInstants(List<HoodieInstant> archivedInstants) throws IOException {
|
||||
log.info("Deleting instants " + archivedInstants);
|
||||
boolean success = true;
|
||||
for (HoodieInstant archivedInstant : archivedInstants) {
|
||||
Path commitFile = new Path(metaClient.getMetaPath(), archivedInstant.getFileName());
|
||||
try {
|
||||
if (metaClient.getFs().exists(commitFile)) {
|
||||
success &= metaClient.getFs().delete(commitFile, false);
|
||||
log.info("Archived and deleted instant file " + commitFile);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to delete archived instant " + archivedInstant, e);
|
||||
}
|
||||
}
|
||||
|
||||
// Remove older meta-data from auxiliary path too
|
||||
Option<HoodieInstant> latestCommitted =
|
||||
Option.fromJavaOptional(archivedInstants.stream()
|
||||
.filter(i -> {
|
||||
return i.isCompleted()
|
||||
&& (i.getAction().equals(HoodieTimeline.COMMIT_ACTION) || (i.getAction().equals(
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION)));
|
||||
}).max(Comparator.comparing(HoodieInstant::getTimestamp)));
|
||||
if (latestCommitted.isPresent()) {
|
||||
success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get());
|
||||
}
|
||||
return success;
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove older instants from auxiliary meta folder
|
||||
*
|
||||
* @param thresholdInstant Hoodie Instant
|
||||
* @return success if all eligible file deleted successfully
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
private boolean deleteAllInstantsOlderorEqualsInAuxMetaFolder(HoodieInstant thresholdInstant)
|
||||
throws IOException {
|
||||
List<HoodieInstant> instants =
|
||||
HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
|
||||
new Path(metaClient.getMetaAuxiliaryPath()),
|
||||
HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
|
||||
|
||||
List<HoodieInstant> instantsToBeDeleted =
|
||||
instants.stream().filter(instant1 -> HoodieTimeline.compareTimestamps(instant1.getTimestamp(),
|
||||
thresholdInstant.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL)).collect(Collectors.toList());
|
||||
|
||||
boolean success = true;
|
||||
for (HoodieInstant deleteInstant : instantsToBeDeleted) {
|
||||
log.info("Deleting instant " + deleteInstant + " in auxiliary meta path " + metaClient.getMetaAuxiliaryPath());
|
||||
Path metaFile = new Path(metaClient.getMetaAuxiliaryPath(), deleteInstant.getFileName());
|
||||
if (metaClient.getFs().exists(metaFile)) {
|
||||
success &= metaClient.getFs().delete(metaFile, false);
|
||||
log.info("Deleted instant file in auxiliary metapath : " + metaFile);
|
||||
}
|
||||
}
|
||||
return success;
|
||||
}
|
||||
|
||||
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
|
||||
try {
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline()
|
||||
.filterCompletedInstants();
|
||||
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
|
||||
log.info("Wrapper schema " + wrapperSchema.toString());
|
||||
List<IndexedRecord> records = new ArrayList<>();
|
||||
for (HoodieInstant hoodieInstant : instants) {
|
||||
try {
|
||||
records.add(convertToAvroRecord(commitTimeline, hoodieInstant));
|
||||
if (records.size() >= this.config.getCommitArchivalBatchSize()) {
|
||||
writeToFile(wrapperSchema, records);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
log.error("Failed to archive commits, .commit file: " + hoodieInstant.getFileName(), e);
|
||||
if (this.config.isFailOnTimelineArchivingEnabled()) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
writeToFile(wrapperSchema, records);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieCommitException("Failed to archive commits", e);
|
||||
}
|
||||
}
|
||||
|
||||
public Path getArchiveFilePath() {
|
||||
return archiveFilePath;
|
||||
}
|
||||
|
||||
private void writeToFile(Schema wrapperSchema, List<IndexedRecord> records) throws Exception {
|
||||
if (records.size() > 0) {
|
||||
Map<HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString());
|
||||
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header);
|
||||
this.writer = writer.appendBlock(block);
|
||||
records.clear();
|
||||
}
|
||||
}
|
||||
|
||||
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline,
|
||||
HoodieInstant hoodieInstant) throws IOException {
|
||||
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
|
||||
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
|
||||
switch (hoodieInstant.getAction()) {
|
||||
case HoodieTimeline.CLEAN_ACTION: {
|
||||
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils
|
||||
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||
HoodieCleanMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.clean.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.COMMIT_ACTION: {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class);
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.ROLLBACK_ACTION: {
|
||||
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils
|
||||
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||
HoodieRollbackMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.rollback.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.SAVEPOINT_ACTION: {
|
||||
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils
|
||||
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
|
||||
HoodieSavepointMetadata.class));
|
||||
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
|
||||
break;
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION: {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class);
|
||||
archivedMetaWrapper.setHoodieCommitMetadata(commitMetadataConverter(commitMetadata));
|
||||
archivedMetaWrapper.setActionType(ActionType.commit.name());
|
||||
break;
|
||||
}
|
||||
default:
|
||||
throw new UnsupportedOperationException("Action not fully supported yet");
|
||||
}
|
||||
return archivedMetaWrapper;
|
||||
}
|
||||
|
||||
private org.apache.hudi.avro.model.HoodieCommitMetadata commitMetadataConverter(
|
||||
HoodieCommitMetadata hoodieCommitMetadata) {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
//Need this to ignore other public get() methods
|
||||
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
|
||||
org.apache.hudi.avro.model.HoodieCommitMetadata avroMetaData = mapper
|
||||
.convertValue(hoodieCommitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class);
|
||||
// Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer
|
||||
avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, "");
|
||||
return avroMetaData;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,183 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.io.storage.HoodieStorageWriter;
|
||||
import org.apache.hudi.io.storage.HoodieStorageWriterFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWriteHandle<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCreateHandle.class);
|
||||
|
||||
private final HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private final Path path;
|
||||
private long recordsWritten = 0;
|
||||
private long insertRecordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private Iterator<HoodieRecord<T>> recordIterator;
|
||||
private boolean useWriterSchema = false;
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath, String fileId) {
|
||||
super(config, commitTime, fileId, hoodieTable);
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
|
||||
this.path = makeNewPath(partitionPath);
|
||||
|
||||
try {
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime,
|
||||
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
createMarkerFile(partitionPath);
|
||||
this.storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(commitTime, path, hoodieTable, config, writerSchema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to initialize HoodieStorageWriter for path " + path, e);
|
||||
}
|
||||
logger.info("New CreateHandle for partition :" + partitionPath + " with fileId " + fileId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the compactor code path
|
||||
*/
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordIterator) {
|
||||
this(config, commitTime, hoodieTable, partitionPath, fileId);
|
||||
this.recordIterator = recordIterator;
|
||||
this.useWriterSchema = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return storageWriter.canWrite() && record.getPartitionPath().equals(writeStatus.getPartitionPath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
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
|
||||
IndexedRecord recordWithMetadataInSchema = rewriteRecord((GenericRecord) avroRecord.get());
|
||||
storageWriter.writeAvroWithMetadata(recordWithMetadataInSchema, record);
|
||||
// update the new location of record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(instantTime, writeStatus.getFileId()));
|
||||
recordsWritten++;
|
||||
insertRecordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
writeStatus.markSuccess(record, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
record.deflate();
|
||||
} catch (Throwable t) {
|
||||
// Not throwing exception from here, since we don't want to fail the entire job
|
||||
// for a single record
|
||||
writeStatus.markFailure(record, t, recordMetadata);
|
||||
logger.error("Error writing record " + record, t);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes all records passed
|
||||
*/
|
||||
public void write() {
|
||||
try {
|
||||
while (recordIterator.hasNext()) {
|
||||
HoodieRecord<T> record = recordIterator.next();
|
||||
if (useWriterSchema) {
|
||||
write(record, record.getData().getInsertValue(writerSchema));
|
||||
} else {
|
||||
write(record, record.getData().getInsertValue(originalSchema));
|
||||
}
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to insert records for path " + path, io);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs actions to durably, persist the current changes and returns a WriteStatus object
|
||||
*/
|
||||
@Override
|
||||
public WriteStatus close() {
|
||||
logger.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records "
|
||||
+ recordsWritten);
|
||||
try {
|
||||
|
||||
storageWriter.close();
|
||||
|
||||
HoodieWriteStat stat = new HoodieWriteStat();
|
||||
stat.setPartitionPath(writeStatus.getPartitionPath());
|
||||
stat.setNumWrites(recordsWritten);
|
||||
stat.setNumDeletes(recordsDeleted);
|
||||
stat.setNumInserts(insertRecordsWritten);
|
||||
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
|
||||
stat.setFileId(writeStatus.getFileId());
|
||||
stat.setPath(new Path(config.getBasePath()), path);
|
||||
long fileSizeInBytes = FSUtils.getFileSize(fs, path);
|
||||
stat.setTotalWriteBytes(fileSizeInBytes);
|
||||
stat.setFileSizeInBytes(fileSizeInBytes);
|
||||
stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalCreateTime(timer.endTimer());
|
||||
stat.setRuntimeStats(runtimeStats);
|
||||
writeStatus.setStat(stat);
|
||||
|
||||
logger.info(String.format("CreateHandle for partitionPath %s fileID %s, took %d ms.",
|
||||
stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalCreateTime()));
|
||||
|
||||
return writeStatus;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,42 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
|
||||
public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
|
||||
|
||||
protected final String instantTime;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final FileSystem fs;
|
||||
protected final HoodieTable<T> hoodieTable;
|
||||
|
||||
HoodieIOHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable) {
|
||||
this.instantTime = instantTime;
|
||||
this.config = config;
|
||||
this.hoodieTable = hoodieTable;
|
||||
this.fs = getFileSystem();
|
||||
}
|
||||
|
||||
protected abstract FileSystem getFileSystem();
|
||||
}
|
||||
@@ -0,0 +1,158 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Takes a bunch of keys and returns ones that are present in the file group.
|
||||
*/
|
||||
public class HoodieKeyLookupHandle<T extends HoodieRecordPayload> extends HoodieReadHandle<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieKeyLookupHandle.class);
|
||||
|
||||
private final HoodieTableType tableType;
|
||||
|
||||
private final BloomFilter bloomFilter;
|
||||
|
||||
private final List<String> candidateRecordKeys;
|
||||
|
||||
private long totalKeysChecked;
|
||||
|
||||
public HoodieKeyLookupHandle(HoodieWriteConfig config, HoodieTable<T> hoodieTable,
|
||||
Pair<String, String> partitionPathFilePair) {
|
||||
super(config, null, hoodieTable, partitionPathFilePair);
|
||||
this.tableType = hoodieTable.getMetaClient().getTableType();
|
||||
this.candidateRecordKeys = new ArrayList<>();
|
||||
this.totalKeysChecked = 0;
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
this.bloomFilter = ParquetUtils.readBloomFilterFromParquetMetadata(hoodieTable.getHadoopConf(),
|
||||
new Path(getLatestDataFile().getPath()));
|
||||
logger.info(String.format("Read bloom filter from %s in %d ms", partitionPathFilePair, timer.endTimer()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a list of row keys and one file, return only row keys existing in that file.
|
||||
*/
|
||||
public static List<String> checkCandidatesAgainstFile(Configuration configuration,
|
||||
List<String> candidateRecordKeys, Path filePath) throws HoodieIndexException {
|
||||
List<String> foundRecordKeys = new ArrayList<>();
|
||||
try {
|
||||
// Load all rowKeys from the file, to double-confirm
|
||||
if (!candidateRecordKeys.isEmpty()) {
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
Set<String> fileRowKeys = ParquetUtils.filterParquetRowKeys(configuration, filePath,
|
||||
new HashSet<>(candidateRecordKeys));
|
||||
foundRecordKeys.addAll(fileRowKeys);
|
||||
logger.info(String.format("Checked keys against file %s, in %d ms. #candidates (%d) #found (%d)", filePath,
|
||||
timer.endTimer(), candidateRecordKeys.size(), foundRecordKeys.size()));
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Keys matching for file " + filePath + " => " + foundRecordKeys);
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieIndexException("Error checking candidate keys against file.", e);
|
||||
}
|
||||
return foundRecordKeys;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the key for look up.
|
||||
*/
|
||||
public void addKey(String recordKey) {
|
||||
// check record key against bloom filter of current file & add to possible keys if needed
|
||||
if (bloomFilter.mightContain(recordKey)) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("Record key " + recordKey + " matches bloom filter in " + partitionPathFilePair);
|
||||
}
|
||||
candidateRecordKeys.add(recordKey);
|
||||
}
|
||||
totalKeysChecked++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Of all the keys, that were added, return a list of keys that were actually found in the file group.
|
||||
*/
|
||||
public KeyLookupResult getLookupResult() {
|
||||
if (logger.isDebugEnabled()) {
|
||||
logger.debug("#The candidate row keys for " + partitionPathFilePair + " => " + candidateRecordKeys);
|
||||
}
|
||||
|
||||
HoodieDataFile dataFile = getLatestDataFile();
|
||||
List<String> matchingKeys = checkCandidatesAgainstFile(hoodieTable.getHadoopConf(), candidateRecordKeys,
|
||||
new Path(dataFile.getPath()));
|
||||
logger.info(String.format("Total records (%d), bloom filter candidates (%d)/fp(%d), actual matches (%d)",
|
||||
totalKeysChecked, candidateRecordKeys.size(), candidateRecordKeys.size() - matchingKeys.size(),
|
||||
matchingKeys.size()));
|
||||
return new KeyLookupResult(partitionPathFilePair.getRight(), partitionPathFilePair.getLeft(),
|
||||
dataFile.getCommitTime(), matchingKeys);
|
||||
}
|
||||
|
||||
/**
|
||||
* Encapsulates the result from a key lookup
|
||||
*/
|
||||
public static class KeyLookupResult {
|
||||
|
||||
private final String fileId;
|
||||
private final String baseInstantTime;
|
||||
private final List<String> matchingRecordKeys;
|
||||
private final String partitionPath;
|
||||
|
||||
public KeyLookupResult(String fileId, String partitionPath, String baseInstantTime,
|
||||
List<String> matchingRecordKeys) {
|
||||
this.fileId = fileId;
|
||||
this.partitionPath = partitionPath;
|
||||
this.baseInstantTime = baseInstantTime;
|
||||
this.matchingRecordKeys = matchingRecordKeys;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public String getBaseInstantTime() {
|
||||
return baseInstantTime;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
public List<String> getMatchingRecordKeys() {
|
||||
return matchingRecordKeys;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,361 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.HoodieRecordSizeEstimator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.io.storage.HoodieStorageWriter;
|
||||
import org.apache.hudi.io.storage.HoodieStorageWriterFactory;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
@SuppressWarnings("Duplicates")
|
||||
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWriteHandle<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
|
||||
|
||||
private Map<String, HoodieRecord<T>> keyToNewRecords;
|
||||
private Set<String> writtenRecordKeys;
|
||||
private HoodieStorageWriter<IndexedRecord> storageWriter;
|
||||
private Path newFilePath;
|
||||
private Path oldFilePath;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private long updatedRecordsWritten = 0;
|
||||
private long insertRecordsWritten = 0;
|
||||
private boolean useWriterSchema;
|
||||
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String fileId) {
|
||||
super(config, commitTime, fileId, hoodieTable);
|
||||
String partitionPath = init(fileId, recordItr);
|
||||
init(fileId, partitionPath,
|
||||
hoodieTable.getROFileSystemView().getLatestDataFile(partitionPath, fileId).get());
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by compactor code path
|
||||
*/
|
||||
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, String fileId, HoodieDataFile dataFileToBeMerged) {
|
||||
super(config, commitTime, fileId, hoodieTable);
|
||||
this.keyToNewRecords = keyToNewRecords;
|
||||
this.useWriterSchema = true;
|
||||
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get())
|
||||
.getPartitionPath(), dataFileToBeMerged);
|
||||
}
|
||||
|
||||
|
||||
public static Schema createHoodieWriteSchema(Schema originalSchema) {
|
||||
return HoodieAvroUtils.addMetadataFields(originalSchema);
|
||||
}
|
||||
|
||||
public Path makeNewPath(String partitionPath) {
|
||||
Path path = FSUtils.getPartitionPath(config.getBasePath(), partitionPath);
|
||||
try {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
|
||||
return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, writeToken, fileId));
|
||||
}
|
||||
|
||||
public Schema getWriterSchema() {
|
||||
return writerSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||
* <p>
|
||||
* - Whether it belongs to the same partitionPath as existing records - Whether the current file written bytes lt max
|
||||
* file size
|
||||
*/
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
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, 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);
|
||||
logger.error("Error writing record " + record, exception.get());
|
||||
} else {
|
||||
write(record, avroRecord);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields
|
||||
*/
|
||||
protected GenericRecord rewriteRecord(GenericRecord record) {
|
||||
return HoodieAvroUtils.rewriteRecord(record, writerSchema);
|
||||
}
|
||||
|
||||
/**
|
||||
* Extract old file path, initialize StorageWriter and WriteStatus
|
||||
*/
|
||||
private void init(String fileId, String partitionPath, HoodieDataFile dataFileToBeMerged) {
|
||||
logger.info("partitionPath:" + partitionPath + ", fileId to be merged:" + fileId);
|
||||
this.writtenRecordKeys = new HashSet<>();
|
||||
writeStatus.setStat(new HoodieWriteStat());
|
||||
try {
|
||||
String latestValidFilePath = dataFileToBeMerged.getFileName();
|
||||
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
|
||||
|
||||
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, instantTime,
|
||||
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
|
||||
partitionMetadata.trySave(TaskContext.getPartitionId());
|
||||
|
||||
oldFilePath = new Path(
|
||||
config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath);
|
||||
String relativePath = new Path((partitionPath.isEmpty() ? "" : partitionPath + "/")
|
||||
+ FSUtils.makeDataFileName(instantTime, writeToken, fileId)).toString();
|
||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||
|
||||
logger.info(String
|
||||
.format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(),
|
||||
newFilePath.toString()));
|
||||
// file name is same for all records, in this bunch
|
||||
writeStatus.setFileId(fileId);
|
||||
writeStatus.setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setPartitionPath(partitionPath);
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
writeStatus.getStat().setPath(new Path(config.getBasePath()), newFilePath);
|
||||
|
||||
// Create Marker file
|
||||
createMarkerFile(partitionPath);
|
||||
|
||||
// Create the writer for writing the new version file
|
||||
storageWriter = HoodieStorageWriterFactory
|
||||
.getStorageWriter(instantTime, newFilePath, hoodieTable, config, writerSchema);
|
||||
} catch (IOException io) {
|
||||
logger.error("Error in update task at commit " + instantTime, io);
|
||||
writeStatus.setGlobalError(io);
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
|
||||
+ instantTime + " on path " + hoodieTable.getMetaClient().getBasePath(), io);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Load the new incoming records in a map and return partitionPath
|
||||
*/
|
||||
private String init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
try {
|
||||
// Load the new records in a map
|
||||
logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge());
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(),
|
||||
config.getSpillableMapBasePath(), new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(originalSchema));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
||||
}
|
||||
String partitionPath = null;
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> record = newRecordsItr.next();
|
||||
partitionPath = record.getPartitionPath();
|
||||
keyToNewRecords.put(record.getRecordKey(), record);
|
||||
// update the new location of the record, so we know where to find it next
|
||||
record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
|
||||
}
|
||||
logger.info("Number of entries in MemoryBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries()
|
||||
+ "Total size in bytes of MemoryBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize()
|
||||
+ "Number of entries in DiskBasedMap => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries()
|
||||
+ "Size of file spilled to disk => "
|
||||
+ ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Option<IndexedRecord> indexedRecord) {
|
||||
if (indexedRecord.isPresent()) {
|
||||
updatedRecordsWritten++;
|
||||
}
|
||||
return writeRecord(hoodieRecord, indexedRecord);
|
||||
}
|
||||
|
||||
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
|
||||
IndexedRecord recordWithMetadataInSchema = rewriteRecord((GenericRecord) indexedRecord.get());
|
||||
storageWriter.writeAvroWithMetadata(recordWithMetadataInSchema, hoodieRecord);
|
||||
recordsWritten++;
|
||||
} else {
|
||||
recordsDeleted++;
|
||||
}
|
||||
|
||||
writeStatus.markSuccess(hoodieRecord, recordMetadata);
|
||||
// deflate record payload after recording success. This will help users access payload as a
|
||||
// part of marking
|
||||
// record successful.
|
||||
hoodieRecord.deflate();
|
||||
return true;
|
||||
} catch (Exception e) {
|
||||
logger.error("Error writing record " + hoodieRecord, e);
|
||||
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Go through an old record. Here if we detect a newer version shows up, we write the new one to
|
||||
* the file.
|
||||
*/
|
||||
public void write(GenericRecord oldRecord) {
|
||||
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
boolean copyOldRecord = true;
|
||||
if (keyToNewRecords.containsKey(key)) {
|
||||
// If we have duplicate records that we are updating, then the hoodie record will be deflated after
|
||||
// writing the first record. So make a copy of the record to be merged
|
||||
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key));
|
||||
try {
|
||||
Option<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
|
||||
.combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchema : originalSchema);
|
||||
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
|
||||
/* ONLY WHEN
|
||||
* 1) we have an update for this key AND
|
||||
* 2) We are able to successfully write the the combined new value
|
||||
*
|
||||
* We no longer need to copy the old record over.
|
||||
*/
|
||||
copyOldRecord = false;
|
||||
}
|
||||
writtenRecordKeys.add(key);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieUpsertException(
|
||||
"Failed to combine/merge new record with old value in storage, for new record {"
|
||||
+ keyToNewRecords.get(key) + "}, old value {" + oldRecord + "}", e);
|
||||
}
|
||||
}
|
||||
|
||||
if (copyOldRecord) {
|
||||
// this should work as it is, since this is an existing record
|
||||
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
|
||||
+ getOldFilePath() + " to new file " + newFilePath;
|
||||
try {
|
||||
storageWriter.writeAvro(key, oldRecord);
|
||||
} catch (ClassCastException e) {
|
||||
logger.error("Schema mismatch when rewriting old record " + oldRecord + " from file "
|
||||
+ getOldFilePath() + " to file " + newFilePath + " with writerSchema " + writerSchema
|
||||
.toString(true));
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
} catch (IOException e) {
|
||||
logger.error("Failed to merge old record into new file for key " + key + " from old file "
|
||||
+ getOldFilePath() + " to new file " + newFilePath, e);
|
||||
throw new HoodieUpsertException(errMsg, e);
|
||||
}
|
||||
recordsWritten++;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus close() {
|
||||
try {
|
||||
// write out any pending records (this can happen when inserts are turned into updates)
|
||||
for (String key : keyToNewRecords.keySet()) {
|
||||
if (!writtenRecordKeys.contains(key)) {
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
if (useWriterSchema) {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(writerSchema));
|
||||
} else {
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(originalSchema));
|
||||
}
|
||||
insertRecordsWritten++;
|
||||
}
|
||||
}
|
||||
keyToNewRecords.clear();
|
||||
writtenRecordKeys.clear();
|
||||
|
||||
if (storageWriter != null) {
|
||||
storageWriter.close();
|
||||
}
|
||||
|
||||
long fileSizeInBytes = FSUtils.getFileSize(fs, newFilePath);
|
||||
HoodieWriteStat stat = writeStatus.getStat();
|
||||
|
||||
stat.setTotalWriteBytes(fileSizeInBytes);
|
||||
stat.setFileSizeInBytes(fileSizeInBytes);
|
||||
stat.setNumWrites(recordsWritten);
|
||||
stat.setNumDeletes(recordsDeleted);
|
||||
stat.setNumUpdateWrites(updatedRecordsWritten);
|
||||
stat.setNumInserts(insertRecordsWritten);
|
||||
stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalUpsertTime(timer.endTimer());
|
||||
stat.setRuntimeStats(runtimeStats);
|
||||
|
||||
logger.info(String.format("MergeHandle for partitionPath %s fileID %s, took %d ms.",
|
||||
stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalUpsertTime()));
|
||||
|
||||
return writeStatus;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
|
||||
}
|
||||
}
|
||||
|
||||
public Path getOldFilePath() {
|
||||
return oldFilePath;
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus getWriteStatus() {
|
||||
return writeStatus;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,43 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
/**
|
||||
* Extract range information for a given file slice
|
||||
*/
|
||||
public class HoodieRangeInfoHandle<T extends HoodieRecordPayload> extends HoodieReadHandle<T> {
|
||||
|
||||
public HoodieRangeInfoHandle(HoodieWriteConfig config, HoodieTable<T> hoodieTable,
|
||||
Pair<String, String> partitionPathFilePair) {
|
||||
super(config, null, hoodieTable, partitionPathFilePair);
|
||||
}
|
||||
|
||||
public String[] getMinMaxKeys() {
|
||||
HoodieDataFile dataFile = getLatestDataFile();
|
||||
return ParquetUtils.readMinMaxRecordKeys(hoodieTable.getHadoopConf(), new Path(dataFile.getPath()));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,58 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
/**
|
||||
* Base class for read operations done logically on the file group.
|
||||
*/
|
||||
public abstract class HoodieReadHandle<T extends HoodieRecordPayload> extends HoodieIOHandle {
|
||||
|
||||
protected final Pair<String, String> partitionPathFilePair;
|
||||
|
||||
public HoodieReadHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T> hoodieTable,
|
||||
Pair<String, String> partitionPathFilePair) {
|
||||
super(config, instantTime, hoodieTable);
|
||||
this.partitionPathFilePair = partitionPathFilePair;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected FileSystem getFileSystem() {
|
||||
return hoodieTable.getMetaClient().getFs();
|
||||
}
|
||||
|
||||
public Pair<String, String> getPartitionPathFilePair() {
|
||||
return partitionPathFilePair;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return partitionPathFilePair.getRight();
|
||||
}
|
||||
|
||||
protected HoodieDataFile getLatestDataFile() {
|
||||
return hoodieTable.getROFileSystemView()
|
||||
.getLatestDataFile(partitionPathFilePair.getLeft(), partitionPathFilePair.getRight()).get();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,170 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
/**
|
||||
* Base class for all write operations logically performed at the file group level.
|
||||
*/
|
||||
public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends HoodieIOHandle {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieWriteHandle.class);
|
||||
protected final Schema originalSchema;
|
||||
protected final Schema writerSchema;
|
||||
protected HoodieTimer timer;
|
||||
protected final WriteStatus writeStatus;
|
||||
protected final String fileId;
|
||||
protected final String writeToken;
|
||||
|
||||
public HoodieWriteHandle(HoodieWriteConfig config, String instantTime, String fileId, HoodieTable<T> hoodieTable) {
|
||||
super(config, instantTime, hoodieTable);
|
||||
this.fileId = fileId;
|
||||
this.writeToken = makeSparkWriteToken();
|
||||
this.originalSchema = new Schema.Parser().parse(config.getSchema());
|
||||
this.writerSchema = createHoodieWriteSchema(originalSchema);
|
||||
this.timer = new HoodieTimer().startTimer();
|
||||
this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(),
|
||||
!hoodieTable.getIndex().isImplicitWithStorage(),
|
||||
config.getWriteStatusFailureFraction());
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate a write token based on the currently running spark task and its place in the spark dag.
|
||||
*/
|
||||
private static String makeSparkWriteToken() {
|
||||
return FSUtils.makeWriteToken(TaskContext.getPartitionId(), TaskContext.get().stageId(),
|
||||
TaskContext.get().taskAttemptId());
|
||||
}
|
||||
|
||||
public static Schema createHoodieWriteSchema(Schema originalSchema) {
|
||||
return HoodieAvroUtils.addMetadataFields(originalSchema);
|
||||
}
|
||||
|
||||
public Path makeNewPath(String partitionPath) {
|
||||
Path path = FSUtils.getPartitionPath(config.getBasePath(), partitionPath);
|
||||
try {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
|
||||
return new Path(path.toString(), FSUtils.makeDataFileName(instantTime, writeToken, fileId));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an empty marker file corresponding to storage writer path
|
||||
*
|
||||
* @param partitionPath Partition path
|
||||
*/
|
||||
protected void createMarkerFile(String partitionPath) {
|
||||
Path markerPath = makeNewMarkerPath(partitionPath);
|
||||
try {
|
||||
logger.info("Creating Marker Path=" + markerPath);
|
||||
fs.create(markerPath, false).close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Failed to create marker file " + markerPath, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* THe marker path will be <base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename
|
||||
*/
|
||||
private Path makeNewMarkerPath(String partitionPath) {
|
||||
Path markerRootPath = new Path(hoodieTable.getMetaClient().getMarkerFolderPath(instantTime));
|
||||
Path path = FSUtils.getPartitionPath(markerRootPath, partitionPath);
|
||||
try {
|
||||
fs.mkdirs(path); // create a new partition as needed.
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to make dir " + path, e);
|
||||
}
|
||||
return new Path(path.toString(), FSUtils.makeMarkerFile(instantTime, writeToken, fileId));
|
||||
}
|
||||
|
||||
public Schema getWriterSchema() {
|
||||
return writerSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||
* <p>
|
||||
* - Whether it belongs to the same partitionPath as existing records - Whether the current file written bytes lt max
|
||||
* file size
|
||||
*/
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return false;
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the actual writing of the given record into the backing file.
|
||||
*/
|
||||
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, 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);
|
||||
logger.error("Error writing record " + record, exception.get());
|
||||
} else {
|
||||
write(record, avroRecord);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields
|
||||
*/
|
||||
protected GenericRecord rewriteRecord(GenericRecord record) {
|
||||
return HoodieAvroUtils.rewriteRecord(record, writerSchema);
|
||||
}
|
||||
|
||||
public abstract WriteStatus close();
|
||||
|
||||
public abstract WriteStatus getWriteStatus();
|
||||
|
||||
@Override
|
||||
protected FileSystem getFileSystem() {
|
||||
return hoodieTable.getMetaClient().getFs();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,59 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.compact;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Set;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* A HoodieCompactor runs compaction on a hoodie table
|
||||
*/
|
||||
public interface HoodieCompactor extends Serializable {
|
||||
|
||||
/**
|
||||
* Generate a new compaction plan for scheduling
|
||||
*
|
||||
* @param jsc Spark Context
|
||||
* @param hoodieTable Hoodie Table
|
||||
* @param config Hoodie Write Configuration
|
||||
* @param compactionCommitTime scheduled compaction commit time
|
||||
* @param fgIdsInPendingCompactions partition-fileId pairs for which compaction is pending
|
||||
* @return Compaction Plan
|
||||
* @throws IOException when encountering errors
|
||||
*/
|
||||
HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc,
|
||||
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime,
|
||||
Set<HoodieFileGroupId> fgIdsInPendingCompactions)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Execute compaction operations and report back status
|
||||
*/
|
||||
JavaRDD<WriteStatus> compact(JavaSparkContext jsc,
|
||||
HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config,
|
||||
String compactionInstantTime) throws IOException;
|
||||
}
|
||||
@@ -0,0 +1,236 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.compact;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.StreamSupport;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.TableFileSystemView.RealtimeView;
|
||||
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.compact.strategy.CompactionStrategy;
|
||||
import org.apache.hudi.table.HoodieCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.FlatMapFunction;
|
||||
import org.apache.spark.util.AccumulatorV2;
|
||||
import org.apache.spark.util.LongAccumulator;
|
||||
|
||||
/**
|
||||
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
|
||||
* possible compactions, passes it through a CompactionFilter and executes all the compactions and
|
||||
* writes a new version of base files and make a normal commit
|
||||
*
|
||||
* @see HoodieCompactor
|
||||
*/
|
||||
public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HoodieRealtimeTableCompactor.class);
|
||||
// Accumulator to keep track of total log files for a dataset
|
||||
private AccumulatorV2<Long, Long> totalLogFiles;
|
||||
// Accumulator to keep track of total log file slices for a dataset
|
||||
private AccumulatorV2<Long, Long> totalFileSlices;
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc,
|
||||
HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config,
|
||||
String compactionInstantTime) throws IOException {
|
||||
if (compactionPlan == null || (compactionPlan.getOperations() == null)
|
||||
|| (compactionPlan.getOperations().isEmpty())) {
|
||||
return jsc.emptyRDD();
|
||||
}
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
|
||||
List<CompactionOperation> operations = compactionPlan.getOperations().stream().map(
|
||||
CompactionOperation::convertFromAvroRecordInstance).collect(toList());
|
||||
log.info("Compactor compacting " + operations + " files");
|
||||
|
||||
return jsc.parallelize(operations, operations.size())
|
||||
.map(s -> compact(table, metaClient, config, s, compactionInstantTime))
|
||||
.flatMap(List::iterator);
|
||||
}
|
||||
|
||||
private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
|
||||
HoodieWriteConfig config,
|
||||
CompactionOperation operation, String commitTime) throws IOException {
|
||||
FileSystem fs = metaClient.getFs();
|
||||
Schema readerSchema = HoodieAvroUtils
|
||||
.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
|
||||
|
||||
log.info("Compacting base " + operation.getDataFilePath() + " with delta files " + operation
|
||||
.getDeltaFilePaths() + " for commit " + commitTime);
|
||||
// TODO - FIX THIS
|
||||
// Reads the entire avro file. Always only specific blocks should be read from the avro file
|
||||
// (failure recover).
|
||||
// Load all the delta commits since the last compaction commit and get all the blocks to be
|
||||
// loaded and load it using CompositeAvroLogReader
|
||||
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
|
||||
String maxInstantTime = metaClient.getActiveTimeline()
|
||||
.getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction());
|
||||
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs,
|
||||
metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime,
|
||||
config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(),
|
||||
config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(),
|
||||
config.getSpillableMapBasePath());
|
||||
if (!scanner.iterator().hasNext()) {
|
||||
return Lists.<WriteStatus>newArrayList();
|
||||
}
|
||||
|
||||
Option<HoodieDataFile> oldDataFileOpt = operation.getBaseFile();
|
||||
|
||||
// Compacting is very similar to applying updates to existing file
|
||||
Iterator<List<WriteStatus>> result;
|
||||
// If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a
|
||||
// new base parquet file.
|
||||
if (oldDataFileOpt.isPresent()) {
|
||||
result = hoodieCopyOnWriteTable
|
||||
.handleUpdate(commitTime, operation.getFileId(), scanner.getRecords(), oldDataFileOpt.get());
|
||||
} else {
|
||||
result = hoodieCopyOnWriteTable
|
||||
.handleInsert(commitTime, operation.getPartitionPath(), operation.getFileId(), scanner.iterator());
|
||||
}
|
||||
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
||||
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
|
||||
.peek(s -> {
|
||||
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
|
||||
s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
|
||||
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
|
||||
s.getStat().setPartitionPath(operation.getPartitionPath());
|
||||
s.getStat().setTotalLogSizeCompacted(operation.getMetrics().get(
|
||||
CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue());
|
||||
s.getStat().setTotalLogBlocks(scanner.getTotalLogBlocks());
|
||||
s.getStat().setTotalCorruptLogBlock(scanner.getTotalCorruptBlocks());
|
||||
s.getStat().setTotalRollbackBlocks(scanner.getTotalRollbacks());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
|
||||
s.getStat().setRuntimeStats(runtimeStats);
|
||||
}).collect(toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc,
|
||||
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime,
|
||||
Set<HoodieFileGroupId> fgIdsInPendingCompactions) throws IOException {
|
||||
|
||||
totalLogFiles = new LongAccumulator();
|
||||
totalFileSlices = new LongAccumulator();
|
||||
jsc.sc().register(totalLogFiles);
|
||||
jsc.sc().register(totalFileSlices);
|
||||
|
||||
Preconditions
|
||||
.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
|
||||
"HoodieRealtimeTableCompactor can only compact table of type "
|
||||
+ HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient()
|
||||
.getTableType().name());
|
||||
|
||||
//TODO : check if maxMemory is not greater than JVM or spark.executor memory
|
||||
// TODO - rollback any compactions in flight
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
|
||||
List<String> partitionPaths = FSUtils
|
||||
.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
|
||||
// filter the partition paths if needed to reduce list status
|
||||
partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths);
|
||||
|
||||
if (partitionPaths.isEmpty()) {
|
||||
// In case no partitions could be picked, return no compaction plan
|
||||
return null;
|
||||
}
|
||||
|
||||
RealtimeView fileSystemView = hoodieTable.getRTFileSystemView();
|
||||
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
|
||||
List<HoodieCompactionOperation> operations =
|
||||
jsc.parallelize(partitionPaths, partitionPaths.size())
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.filter(slice ->
|
||||
!fgIdsInPendingCompactions.contains(slice.getFileGroupId()))
|
||||
.map(
|
||||
s -> {
|
||||
List<HoodieLogFile> logFiles = s.getLogFiles().sorted(HoodieLogFile
|
||||
.getLogFileComparator()).collect(Collectors.toList());
|
||||
totalLogFiles.add((long) logFiles.size());
|
||||
totalFileSlices.add(1L);
|
||||
// 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.
|
||||
Option<HoodieDataFile> dataFile = s.getDataFile();
|
||||
return new CompactionOperation(dataFile, partitionPath, logFiles,
|
||||
config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
|
||||
})
|
||||
.filter(c -> !c.getDeltaFilePaths().isEmpty())
|
||||
.collect(toList()).iterator()).collect().stream().map(CompactionUtils::buildHoodieCompactionOperation)
|
||||
.collect(toList());
|
||||
log.info("Total of " + operations.size() + " compactions are retrieved");
|
||||
log.info("Total number of latest files slices " + totalFileSlices.value());
|
||||
log.info("Total number of log files " + totalLogFiles.value());
|
||||
log.info("Total number of file slices " + totalFileSlices.value());
|
||||
// Filter the compactions with the passed in filter. This lets us choose most effective
|
||||
// compactions only
|
||||
HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations,
|
||||
CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList()));
|
||||
Preconditions.checkArgument(compactionPlan.getOperations().stream().noneMatch(
|
||||
op -> fgIdsInPendingCompactions.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))),
|
||||
"Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. "
|
||||
+ "Please fix your strategy implementation."
|
||||
+ "FileIdsWithPendingCompactions :" + fgIdsInPendingCompactions
|
||||
+ ", Selected workload :" + compactionPlan);
|
||||
if (compactionPlan.getOperations().isEmpty()) {
|
||||
log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());
|
||||
}
|
||||
return compactionPlan;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,53 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.compact.strategy;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import java.util.List;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
/**
|
||||
* CompactionStrategy which looks at total IO to be done for the compaction (read + write) and
|
||||
* limits the list of compactions to be under a configured limit on the IO
|
||||
*
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
public class BoundedIOCompactionStrategy extends CompactionStrategy {
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// Iterate through the operations in order and accept operations as long as we are within the
|
||||
// IO limit
|
||||
// Preserves the original ordering of compactions
|
||||
List<HoodieCompactionOperation> finalOperations = Lists.newArrayList();
|
||||
long targetIORemaining = writeConfig.getTargetIOPerCompactionInMB();
|
||||
for (HoodieCompactionOperation op : operations) {
|
||||
long opIo = op.getMetrics().get(TOTAL_IO_MB).longValue();
|
||||
targetIORemaining -= opIo;
|
||||
finalOperations.add(op);
|
||||
if (targetIORemaining <= 0) {
|
||||
return finalOperations;
|
||||
}
|
||||
}
|
||||
return finalOperations;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,73 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.compact.strategy;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.lang3.time.DateUtils;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
/**
|
||||
* This strategy ensures that the last N partitions are picked up even if there are later partitions created for the
|
||||
* dataset. lastNPartitions is defined as the N partitions before the currentDate.
|
||||
* currentDay = 2018/01/01
|
||||
* The dataset has partitions for 2018/02/02 and 2018/03/03 beyond the currentDay
|
||||
* This strategy will pick up the following partitions for compaction :
|
||||
* (2018/01/01, allPartitionsInRange[(2018/01/01 - lastNPartitions) to 2018/01/01), 2018/02/02, 2018/03/03)
|
||||
*/
|
||||
public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionStrategy {
|
||||
|
||||
SimpleDateFormat dateFormat = new SimpleDateFormat(datePartitionFormat);
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// The earliest partition to compact - current day minus the target partitions limit
|
||||
String earliestPartitionPathToCompact = dateFormat.format(DateUtils.addDays(new Date(), -1 * writeConfig
|
||||
.getTargetPartitionsPerDayBasedCompaction()));
|
||||
// Filter out all partitions greater than earliestPartitionPathToCompact
|
||||
List<HoodieCompactionOperation> eligibleCompactionOperations = operations.stream()
|
||||
.collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet().stream()
|
||||
.sorted(Map.Entry.comparingByKey(comparator))
|
||||
.filter(e -> comparator.compare(earliestPartitionPathToCompact, e.getKey()) >= 0)
|
||||
.flatMap(e -> e.getValue().stream())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
return eligibleCompactionOperations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> partitionPaths) {
|
||||
// The earliest partition to compact - current day minus the target partitions limit
|
||||
String earliestPartitionPathToCompact = dateFormat.format(DateUtils.addDays(new Date(), -1 * writeConfig
|
||||
.getTargetPartitionsPerDayBasedCompaction()));
|
||||
// Get all partitions and sort them
|
||||
List<String> filteredPartitionPaths = partitionPaths.stream().map(partition -> partition.replace("/", "-"))
|
||||
.sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
|
||||
.filter(e -> comparator.compare(earliestPartitionPathToCompact, e) >= 0)
|
||||
.collect(Collectors.toList());
|
||||
return filteredPartitionPaths;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,126 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.compact.strategy;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor;
|
||||
|
||||
/**
|
||||
* Strategy for compaction. Pluggable implementation to define how compaction should be done. The
|
||||
* over-ridden implementations of this abstract class can capture the relevant metrics to order
|
||||
* and filter the final list of compaction operation to run in a single compaction.
|
||||
* Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be
|
||||
* passed in every time
|
||||
*
|
||||
* @see HoodieRealtimeTableCompactor
|
||||
*/
|
||||
public abstract class CompactionStrategy implements Serializable {
|
||||
|
||||
public static final String TOTAL_IO_READ_MB = "TOTAL_IO_READ_MB";
|
||||
public static final String TOTAL_IO_WRITE_MB = "TOTAL_IO_WRITE_MB";
|
||||
public static final String TOTAL_IO_MB = "TOTAL_IO_MB";
|
||||
public static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILES_SIZE";
|
||||
public static final String TOTAL_LOG_FILES = "TOTAL_LOG_FILES";
|
||||
|
||||
/**
|
||||
* Callback hook when a HoodieCompactionOperation is created. Individual strategies can capture the
|
||||
* metrics they need to decide on the priority.
|
||||
*
|
||||
* @param dataFile - Base file to compact
|
||||
* @param partitionPath - Partition path
|
||||
* @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, Option<HoodieDataFile> dataFile,
|
||||
String partitionPath, List<HoodieLogFile> logFiles) {
|
||||
Map<String, Double> metrics = Maps.newHashMap();
|
||||
Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize();
|
||||
// Total size of all the log files
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0)
|
||||
.reduce((size1, size2) -> size1 + size2).orElse(0L);
|
||||
// Total read will be the base file + all the log files
|
||||
Long totalIORead = FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L)
|
||||
+ totalLogFileSize);
|
||||
// Total write will be similar to the size of the base file
|
||||
Long totalIOWrite = FSUtils
|
||||
.getSizeInMB(dataFile.isPresent() ? dataFile.get().getFileSize() : defaultMaxParquetFileSize);
|
||||
// Total IO will the the IO for read + write
|
||||
Long totalIO = totalIORead + totalIOWrite;
|
||||
// Save these metrics and we will use during the filter
|
||||
metrics.put(TOTAL_IO_READ_MB, totalIORead.doubleValue());
|
||||
metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite.doubleValue());
|
||||
metrics.put(TOTAL_IO_MB, totalIO.doubleValue());
|
||||
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
|
||||
metrics.put(TOTAL_LOG_FILES, (double) logFiles.size());
|
||||
return metrics;
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate Compaction plan. Allows clients to order and filter the list of compactions to be set. The default
|
||||
* implementation takes care of setting compactor Id from configuration allowing subclasses to only worry about
|
||||
* ordering and filtering compaction operations
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param operations Compaction Operations to be ordered and filtered
|
||||
* @param pendingCompactionPlans Pending Compaction Plans for strategy to schedule next compaction plan
|
||||
* @return Compaction plan to be scheduled.
|
||||
*/
|
||||
public HoodieCompactionPlan generateCompactionPlan(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// Strategy implementation can overload this method to set specific compactor-id
|
||||
return HoodieCompactionPlan.newBuilder()
|
||||
.setOperations(orderAndFilter(writeConfig, operations, pendingCompactionPlans))
|
||||
.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Order and Filter the list of compactions. Use the metrics captured with the captureMetrics to order and filter out
|
||||
* compactions
|
||||
*
|
||||
* @param writeConfig config for this compaction is passed in
|
||||
* @param operations list of compactions collected
|
||||
* @param pendingCompactionPlans Pending Compaction Plans for strategy to schedule next compaction plan
|
||||
* @return list of compactions to perform in this run
|
||||
*/
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations,
|
||||
List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
return operations;
|
||||
}
|
||||
|
||||
/**
|
||||
* Filter the partition paths based on compaction strategy
|
||||
* @param writeConfig
|
||||
* @param allPartitionPaths
|
||||
* @return
|
||||
*/
|
||||
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> allPartitionPaths) {
|
||||
return allPartitionPaths;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,83 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
|
||||
package org.apache.hudi.io.compact.strategy;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import java.text.ParseException;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Comparator;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
/**
|
||||
* This strategy orders compactions in reverse order of creation of Hive Partitions. It helps to
|
||||
* compact data in latest partitions first and then older capped at the Total_IO allowed.
|
||||
*/
|
||||
public class DayBasedCompactionStrategy extends CompactionStrategy {
|
||||
|
||||
// For now, use SimpleDateFormat as default partition format
|
||||
protected static String datePartitionFormat = "yyyy/MM/dd";
|
||||
// Sorts compaction in LastInFirstCompacted order
|
||||
protected static Comparator<String> comparator = (String leftPartition,
|
||||
String rightPartition) -> {
|
||||
try {
|
||||
Date left = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH)
|
||||
.parse(leftPartition);
|
||||
Date right = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH)
|
||||
.parse(rightPartition);
|
||||
return left.after(right) ? -1 : right.after(left) ? 1 : 0;
|
||||
} catch (ParseException e) {
|
||||
throw new HoodieException("Invalid Partition Date Format", e);
|
||||
}
|
||||
};
|
||||
|
||||
@VisibleForTesting
|
||||
public Comparator<String> getComparator() {
|
||||
return comparator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// Iterate through the operations and accept operations as long as we are within the configured target partitions
|
||||
// limit
|
||||
List<HoodieCompactionOperation> filteredList = operations.stream()
|
||||
.collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet().stream()
|
||||
.sorted(Map.Entry.comparingByKey(comparator)).limit(writeConfig.getTargetPartitionsPerDayBasedCompaction())
|
||||
.flatMap(e -> e.getValue().stream())
|
||||
.collect(Collectors.toList());
|
||||
return filteredList;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> allPartitionPaths) {
|
||||
List<String> filteredPartitionPaths = allPartitionPaths.stream().map(partition -> partition.replace("/", "-"))
|
||||
.sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
|
||||
.collect(Collectors.toList()).subList(0, writeConfig.getTargetPartitionsPerDayBasedCompaction());
|
||||
return filteredPartitionPaths;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,74 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.compact.strategy;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
/**
|
||||
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size and
|
||||
* limits the compactions within a configured IO bound
|
||||
*
|
||||
* @see BoundedIOCompactionStrategy
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrategy implements
|
||||
Comparator<HoodieCompactionOperation> {
|
||||
|
||||
private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE";
|
||||
|
||||
@Override
|
||||
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);
|
||||
|
||||
// Total size of all the log files
|
||||
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize)
|
||||
.filter(size -> size >= 0).reduce((size1, size2) -> size1 + size2)
|
||||
.orElse(0L);
|
||||
// save the metrics needed during the order
|
||||
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
|
||||
return metrics;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
|
||||
// Order the operations based on the reverse size of the logs and limit them by the IO
|
||||
return super
|
||||
.orderAndFilter(writeConfig,
|
||||
operations.stream().sorted(this).collect(Collectors.toList()), pendingCompactionPlans);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compare(HoodieCompactionOperation op1, HoodieCompactionOperation op2) {
|
||||
Long totalLogSize1 = op1.getMetrics().get(TOTAL_LOG_FILE_SIZE).longValue();
|
||||
Long totalLogSize2 = op2.getMetrics().get(TOTAL_LOG_FILE_SIZE).longValue();
|
||||
// Reverse the comparison order - so that larger log file size is compacted first
|
||||
return totalLogSize2.compareTo(totalLogSize1);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,40 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.compact.strategy;
|
||||
|
||||
import java.util.List;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
/**
|
||||
* UnBoundedCompactionStrategy will not change ordering or filter any compaction. It is a
|
||||
* pass-through and will compact all the base files which has a log file. This usually means
|
||||
* no-intelligence on compaction.
|
||||
*
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
public class UnBoundedCompactionStrategy extends CompactionStrategy {
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig config,
|
||||
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionWorkloads) {
|
||||
return operations;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,65 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.compact.strategy;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
/**
|
||||
* UnBoundedPartitionAwareCompactionStrategy is a custom UnBounded Strategy.
|
||||
* This will filter all the partitions that are eligible to be compacted by a
|
||||
* {@link BoundedPartitionAwareCompactionStrategy} and return the result.
|
||||
* This is done so that a long running UnBoundedPartitionAwareCompactionStrategy does not step over partitions
|
||||
* in a shorter running BoundedPartitionAwareCompactionStrategy. Essentially, this is an inverse of the
|
||||
* partitions chosen in BoundedPartitionAwareCompactionStrategy
|
||||
*
|
||||
* @see CompactionStrategy
|
||||
*/
|
||||
public class UnBoundedPartitionAwareCompactionStrategy extends CompactionStrategy {
|
||||
|
||||
@Override
|
||||
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig config,
|
||||
final List<HoodieCompactionOperation> operations, final List<HoodieCompactionPlan> pendingCompactionWorkloads) {
|
||||
BoundedPartitionAwareCompactionStrategy boundedPartitionAwareCompactionStrategy
|
||||
= new BoundedPartitionAwareCompactionStrategy();
|
||||
List<HoodieCompactionOperation> operationsToExclude = boundedPartitionAwareCompactionStrategy
|
||||
.orderAndFilter(config, operations, pendingCompactionWorkloads);
|
||||
List<HoodieCompactionOperation> allOperations = new ArrayList<>(operations);
|
||||
allOperations.removeAll(operationsToExclude);
|
||||
return allOperations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> partitionPaths) {
|
||||
List<String> allPartitionPaths = partitionPaths.stream().map(partition -> partition.replace("/", "-"))
|
||||
.sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
|
||||
.collect(Collectors.toList());
|
||||
BoundedPartitionAwareCompactionStrategy boundedPartitionAwareCompactionStrategy
|
||||
= new BoundedPartitionAwareCompactionStrategy();
|
||||
List<String> partitionsToExclude = boundedPartitionAwareCompactionStrategy.filterPartitionPaths(writeConfig,
|
||||
partitionPaths);
|
||||
allPartitionPaths.removeAll(partitionsToExclude);
|
||||
return allPartitionPaths;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,74 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
public class HoodieParquetConfig {
|
||||
|
||||
private HoodieAvroWriteSupport writeSupport;
|
||||
private CompressionCodecName compressionCodecName;
|
||||
private int blockSize;
|
||||
private int pageSize;
|
||||
private long maxFileSize;
|
||||
private Configuration hadoopConf;
|
||||
private double compressionRatio;
|
||||
|
||||
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport,
|
||||
CompressionCodecName compressionCodecName, int blockSize, int pageSize, long maxFileSize,
|
||||
Configuration hadoopConf, double compressionRatio) {
|
||||
this.writeSupport = writeSupport;
|
||||
this.compressionCodecName = compressionCodecName;
|
||||
this.blockSize = blockSize;
|
||||
this.pageSize = pageSize;
|
||||
this.maxFileSize = maxFileSize;
|
||||
this.hadoopConf = hadoopConf;
|
||||
this.compressionRatio = compressionRatio;
|
||||
}
|
||||
|
||||
public HoodieAvroWriteSupport getWriteSupport() {
|
||||
return writeSupport;
|
||||
}
|
||||
|
||||
public CompressionCodecName getCompressionCodecName() {
|
||||
return compressionCodecName;
|
||||
}
|
||||
|
||||
public int getBlockSize() {
|
||||
return blockSize;
|
||||
}
|
||||
|
||||
public int getPageSize() {
|
||||
return pageSize;
|
||||
}
|
||||
|
||||
public long getMaxFileSize() {
|
||||
return maxFileSize;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return hadoopConf;
|
||||
}
|
||||
|
||||
public double getCompressionRatio() {
|
||||
return compressionRatio;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,106 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.io.storage.HoodieWrapperFileSystem;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.HoodieAvroUtils;
|
||||
import org.apache.parquet.hadoop.ParquetFileWriter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.spark.TaskContext;
|
||||
|
||||
/**
|
||||
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides
|
||||
* a way to check if the current file can take more records with the <code>canWrite()</code>
|
||||
*/
|
||||
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord> extends
|
||||
ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
|
||||
|
||||
private static AtomicLong recordIndex = new AtomicLong(1);
|
||||
|
||||
private final Path file;
|
||||
private final HoodieWrapperFileSystem fs;
|
||||
private final long maxFileSize;
|
||||
private final HoodieAvroWriteSupport writeSupport;
|
||||
private final String commitTime;
|
||||
private final Schema schema;
|
||||
|
||||
|
||||
public HoodieParquetWriter(String commitTime, Path file, HoodieParquetConfig parquetConfig,
|
||||
Schema schema) throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(),
|
||||
parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(),
|
||||
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED,
|
||||
ParquetWriter.DEFAULT_WRITER_VERSION,
|
||||
registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file
|
||||
.getFileSystem(registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
// We cannot accurately measure the snappy compressed output file size. We are choosing a
|
||||
// conservative 10%
|
||||
// TODO - compute this compression ratio dynamically by looking at the bytes written to the
|
||||
// stream and the actual file size reported by HDFS
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
|
||||
.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.commitTime = commitTime;
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
public static Configuration registerFileSystem(Path file, Configuration conf) {
|
||||
Configuration returnConf = new Configuration(conf);
|
||||
String scheme = FSUtils.getFs(file.toString(), conf).getScheme();
|
||||
returnConf.set("fs." + HoodieWrapperFileSystem.getHoodieScheme(scheme) + ".impl",
|
||||
HoodieWrapperFileSystem.class.getName());
|
||||
return returnConf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
|
||||
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
|
||||
recordIndex.getAndIncrement());
|
||||
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(),
|
||||
record.getPartitionPath(), file.getName());
|
||||
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
|
||||
super.write(avroRecord);
|
||||
writeSupport.add(record.getRecordKey());
|
||||
}
|
||||
|
||||
public boolean canWrite() {
|
||||
return fs.getBytesWritten(file) < maxFileSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeAvro(String key, IndexedRecord object) throws IOException {
|
||||
super.write(object);
|
||||
writeSupport.add(key);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
public interface HoodieStorageWriter<R extends IndexedRecord> {
|
||||
|
||||
void writeAvroWithMetadata(R newRecord, HoodieRecord record) throws IOException;
|
||||
|
||||
boolean canWrite();
|
||||
|
||||
void close() throws IOException;
|
||||
|
||||
void writeAvro(String key, R oldRecord) throws IOException;
|
||||
}
|
||||
@@ -0,0 +1,58 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.BloomFilter;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
|
||||
public class HoodieStorageWriterFactory {
|
||||
|
||||
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
|
||||
String commitTime, Path path, HoodieTable<T> hoodieTable,
|
||||
HoodieWriteConfig config, Schema schema) throws IOException {
|
||||
//TODO - based on the metadata choose the implementation of HoodieStorageWriter
|
||||
// Currently only parquet is supported
|
||||
return newParquetStorageWriter(commitTime, path, config, schema, hoodieTable);
|
||||
}
|
||||
|
||||
private static <T extends HoodieRecordPayload,
|
||||
R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(String commitTime, Path path,
|
||||
HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable) throws IOException {
|
||||
BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(),
|
||||
config.getBloomFilterFPP());
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
|
||||
new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
|
||||
HoodieParquetConfig parquetConfig =
|
||||
new HoodieParquetConfig(writeSupport, config.getParquetCompressionCodec(),
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(),
|
||||
config.getParquetMaxFileSize(), hoodieTable.getHadoopConf(),
|
||||
config.getParquetCompressionRatio());
|
||||
|
||||
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,186 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.metrics;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Wrapper for metrics-related operations.
|
||||
*/
|
||||
public class HoodieMetrics {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieMetrics.class);
|
||||
// Some timers
|
||||
public String rollbackTimerName = null;
|
||||
public String cleanTimerName = null;
|
||||
public String commitTimerName = null;
|
||||
public String deltaCommitTimerName = null;
|
||||
public String finalizeTimerName = null;
|
||||
public String compactionTimerName = null;
|
||||
private HoodieWriteConfig config = null;
|
||||
private String tableName = null;
|
||||
private Timer rollbackTimer = null;
|
||||
private Timer cleanTimer = null;
|
||||
private Timer commitTimer = null;
|
||||
private Timer deltaCommitTimer = null;
|
||||
private Timer finalizeTimer = null;
|
||||
private Timer compactionTimer = null;
|
||||
|
||||
public HoodieMetrics(HoodieWriteConfig config, String tableName) {
|
||||
this.config = config;
|
||||
this.tableName = tableName;
|
||||
if (config.isMetricsOn()) {
|
||||
Metrics.init(config);
|
||||
this.rollbackTimerName = getMetricsName("timer", HoodieTimeline.ROLLBACK_ACTION);
|
||||
this.cleanTimerName = getMetricsName("timer", HoodieTimeline.CLEAN_ACTION);
|
||||
this.commitTimerName = getMetricsName("timer", HoodieTimeline.COMMIT_ACTION);
|
||||
this.deltaCommitTimerName = getMetricsName("timer", HoodieTimeline.DELTA_COMMIT_ACTION);
|
||||
this.finalizeTimerName = getMetricsName("timer", "finalize");
|
||||
this.compactionTimerName = getMetricsName("timer", HoodieTimeline.COMPACTION_ACTION);
|
||||
}
|
||||
}
|
||||
|
||||
private Timer createTimer(String name) {
|
||||
return config.isMetricsOn() ? Metrics.getInstance().getRegistry().timer(name) : null;
|
||||
}
|
||||
|
||||
public Timer.Context getRollbackCtx() {
|
||||
if (config.isMetricsOn() && rollbackTimer == null) {
|
||||
rollbackTimer = createTimer(rollbackTimerName);
|
||||
}
|
||||
return rollbackTimer == null ? null : rollbackTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getCompactionCtx() {
|
||||
if (config.isMetricsOn() && compactionTimer == null) {
|
||||
compactionTimer = createTimer(commitTimerName);
|
||||
}
|
||||
return compactionTimer == null ? null : compactionTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getCleanCtx() {
|
||||
if (config.isMetricsOn() && cleanTimer == null) {
|
||||
cleanTimer = createTimer(cleanTimerName);
|
||||
}
|
||||
return cleanTimer == null ? null : cleanTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getCommitCtx() {
|
||||
if (config.isMetricsOn() && commitTimer == null) {
|
||||
commitTimer = createTimer(commitTimerName);
|
||||
}
|
||||
return commitTimer == null ? null : commitTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getFinalizeCtx() {
|
||||
if (config.isMetricsOn() && finalizeTimer == null) {
|
||||
finalizeTimer = createTimer(finalizeTimerName);
|
||||
}
|
||||
return finalizeTimer == null ? null : finalizeTimer.time();
|
||||
}
|
||||
|
||||
public Timer.Context getDeltaCommitCtx() {
|
||||
if (config.isMetricsOn() && deltaCommitTimer == null) {
|
||||
deltaCommitTimer = createTimer(deltaCommitTimerName);
|
||||
}
|
||||
return deltaCommitTimer == null ? null : deltaCommitTimer.time();
|
||||
}
|
||||
|
||||
public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs,
|
||||
HoodieCommitMetadata metadata, String actionType) {
|
||||
if (config.isMetricsOn()) {
|
||||
long totalPartitionsWritten = metadata.fetchTotalPartitionsWritten();
|
||||
long totalFilesInsert = metadata.fetchTotalFilesInsert();
|
||||
long totalFilesUpdate = metadata.fetchTotalFilesUpdated();
|
||||
long totalRecordsWritten = metadata.fetchTotalRecordsWritten();
|
||||
long totalUpdateRecordsWritten = metadata.fetchTotalUpdateRecordsWritten();
|
||||
long totalInsertRecordsWritten = metadata.fetchTotalInsertRecordsWritten();
|
||||
long totalBytesWritten = metadata.fetchTotalBytesWritten();
|
||||
long totalTimeTakenByScanner = metadata.getTotalScanTime();
|
||||
long totalTimeTakenForInsert = metadata.getTotalCreateTime();
|
||||
long totalTimeTakenForUpsert = metadata.getTotalUpsertTime();
|
||||
long totalCompactedRecordsUpdated = metadata.getTotalCompactedRecordsUpdated();
|
||||
long totalLogFilesCompacted = metadata.getTotalLogFilesCompacted();
|
||||
long totalLogFilesSize = metadata.getTotalLogFilesSize();
|
||||
Metrics.registerGauge(getMetricsName(actionType, "duration"), durationInMs);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalPartitionsWritten"), totalPartitionsWritten);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalFilesInsert"), totalFilesInsert);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalFilesUpdate"), totalFilesUpdate);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalRecordsWritten"), totalRecordsWritten);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalUpdateRecordsWritten"), totalUpdateRecordsWritten);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalInsertRecordsWritten"), totalInsertRecordsWritten);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalBytesWritten"), totalBytesWritten);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "commitTime"), commitEpochTimeInMs);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalScanTime"), totalTimeTakenByScanner);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalCreateTime"), totalTimeTakenForInsert);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalUpsertTime"), totalTimeTakenForUpsert);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalCompactedRecordsUpdated"), totalCompactedRecordsUpdated);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalLogFilesCompacted"), totalLogFilesCompacted);
|
||||
Metrics.registerGauge(getMetricsName(actionType, "totalLogFilesSize"), totalLogFilesSize);
|
||||
}
|
||||
}
|
||||
|
||||
public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
|
||||
if (config.isMetricsOn()) {
|
||||
logger.info(String
|
||||
.format("Sending rollback metrics (duration=%d, numFilesDeleted=%d)", durationInMs,
|
||||
numFilesDeleted));
|
||||
Metrics.registerGauge(getMetricsName("rollback", "duration"), durationInMs);
|
||||
Metrics.registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
|
||||
}
|
||||
}
|
||||
|
||||
public void updateCleanMetrics(long durationInMs, int numFilesDeleted) {
|
||||
if (config.isMetricsOn()) {
|
||||
logger.info(String
|
||||
.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)", durationInMs,
|
||||
numFilesDeleted));
|
||||
Metrics.registerGauge(getMetricsName("clean", "duration"), durationInMs);
|
||||
Metrics.registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted);
|
||||
}
|
||||
}
|
||||
|
||||
public void updateFinalizeWriteMetrics(long durationInMs, long numFilesFinalized) {
|
||||
if (config.isMetricsOn()) {
|
||||
logger.info(String
|
||||
.format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)",
|
||||
durationInMs, numFilesFinalized));
|
||||
Metrics.registerGauge(getMetricsName("finalize", "duration"), durationInMs);
|
||||
Metrics.registerGauge(getMetricsName("finalize", "numFilesFinalized"), numFilesFinalized);
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
String getMetricsName(String action, String metric) {
|
||||
return config == null ? null : String.format("%s.%s.%s", tableName, action, metric);
|
||||
}
|
||||
|
||||
/**
|
||||
* By default, the timer context returns duration with nano seconds. Convert it to millisecond.
|
||||
*/
|
||||
public long getDurationInMs(long ctxDuration) {
|
||||
return ctxDuration / 1000000;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,40 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.metrics;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
* Used for testing.
|
||||
*/
|
||||
public class InMemoryMetricsReporter extends MetricsReporter {
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void report() {
|
||||
}
|
||||
|
||||
@Override
|
||||
public Closeable getReporter() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
100
hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java
Normal file
100
hudi-client/src/main/java/org/apache/hudi/metrics/Metrics.java
Normal file
@@ -0,0 +1,100 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.metrics;
|
||||
|
||||
import com.codahale.metrics.Gauge;
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
import com.google.common.io.Closeables;
|
||||
import java.io.Closeable;
|
||||
import org.apache.commons.configuration.ConfigurationException;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* This is the main class of the metrics system.
|
||||
*/
|
||||
public class Metrics {
|
||||
private static Logger logger = LogManager.getLogger(Metrics.class);
|
||||
|
||||
private static volatile boolean initialized = false;
|
||||
private static Metrics metrics = null;
|
||||
private final MetricRegistry registry;
|
||||
private MetricsReporter reporter = null;
|
||||
|
||||
private Metrics(HoodieWriteConfig metricConfig) throws ConfigurationException {
|
||||
registry = new MetricRegistry();
|
||||
|
||||
reporter = MetricsReporterFactory.createReporter(metricConfig, registry);
|
||||
if (reporter == null) {
|
||||
throw new RuntimeException("Cannot initialize Reporter.");
|
||||
}
|
||||
// reporter.start();
|
||||
|
||||
Runtime.getRuntime().addShutdownHook(new Thread() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
reporter.report();
|
||||
Closeables.close(reporter.getReporter(), true);
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public static Metrics getInstance() {
|
||||
assert initialized;
|
||||
return metrics;
|
||||
}
|
||||
|
||||
public static synchronized void init(HoodieWriteConfig metricConfig) {
|
||||
if (initialized) {
|
||||
return;
|
||||
}
|
||||
try {
|
||||
metrics = new Metrics(metricConfig);
|
||||
} catch (ConfigurationException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
public static void registerGauge(String metricName, final long value) {
|
||||
try {
|
||||
MetricRegistry registry = Metrics.getInstance().getRegistry();
|
||||
registry.register(metricName, (Gauge<Long>) () -> value);
|
||||
} catch (Exception e) {
|
||||
// Here we catch all exception, so the major upsert pipeline will not be affected if the
|
||||
// metrics system
|
||||
// has some issues.
|
||||
logger.error("Failed to send metrics: ", e);
|
||||
}
|
||||
}
|
||||
|
||||
public MetricRegistry getRegistry() {
|
||||
return registry;
|
||||
}
|
||||
|
||||
public Closeable getReporter() {
|
||||
return reporter.getReporter();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,91 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.metrics;
|
||||
|
||||
import com.codahale.metrics.MetricFilter;
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
import com.codahale.metrics.graphite.Graphite;
|
||||
import com.codahale.metrics.graphite.GraphiteReporter;
|
||||
import java.io.Closeable;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Implementation of Graphite reporter, which connects to the Graphite server, and send metrics to
|
||||
* that server.
|
||||
*/
|
||||
public class MetricsGraphiteReporter extends MetricsReporter {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(MetricsGraphiteReporter.class);
|
||||
private final MetricRegistry registry;
|
||||
private final GraphiteReporter graphiteReporter;
|
||||
private final HoodieWriteConfig config;
|
||||
private String serverHost;
|
||||
private int serverPort;
|
||||
|
||||
public MetricsGraphiteReporter(HoodieWriteConfig config, MetricRegistry registry) {
|
||||
this.registry = registry;
|
||||
this.config = config;
|
||||
|
||||
// Check the serverHost and serverPort here
|
||||
this.serverHost = config.getGraphiteServerHost();
|
||||
this.serverPort = config.getGraphiteServerPort();
|
||||
if (serverHost == null || serverPort == 0) {
|
||||
throw new RuntimeException(String
|
||||
.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].",
|
||||
serverHost, serverPort));
|
||||
}
|
||||
|
||||
this.graphiteReporter = createGraphiteReport();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
if (graphiteReporter != null) {
|
||||
graphiteReporter.start(30, TimeUnit.SECONDS);
|
||||
} else {
|
||||
logger.error("Cannot start as the graphiteReporter is null.");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void report() {
|
||||
if (graphiteReporter != null) {
|
||||
graphiteReporter.report();
|
||||
} else {
|
||||
logger.error("Cannot report metrics as the graphiteReporter is null.");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Closeable getReporter() {
|
||||
return graphiteReporter;
|
||||
}
|
||||
|
||||
private GraphiteReporter createGraphiteReport() {
|
||||
Graphite graphite = new Graphite(new InetSocketAddress(serverHost, serverPort));
|
||||
String reporterPrefix = config.getGraphiteMetricPrefix();
|
||||
return GraphiteReporter.forRegistry(registry).prefixedWith(reporterPrefix)
|
||||
.convertRatesTo(TimeUnit.SECONDS).convertDurationsTo(TimeUnit.MILLISECONDS)
|
||||
.filter(MetricFilter.ALL).build(graphite);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,39 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.metrics;
|
||||
|
||||
import java.io.Closeable;
|
||||
|
||||
/**
|
||||
* Interface for implementing a Reporter.
|
||||
*/
|
||||
public abstract class MetricsReporter {
|
||||
|
||||
/**
|
||||
* Push out metrics at scheduled intervals
|
||||
*/
|
||||
public abstract void start();
|
||||
|
||||
/**
|
||||
* Deterministically push out metrics
|
||||
*/
|
||||
public abstract void report();
|
||||
|
||||
public abstract Closeable getReporter();
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.metrics;
|
||||
|
||||
import com.codahale.metrics.MetricRegistry;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Factory class for creating MetricsReporter.
|
||||
*/
|
||||
public class MetricsReporterFactory {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(MetricsReporterFactory.class);
|
||||
|
||||
public static MetricsReporter createReporter(HoodieWriteConfig config, MetricRegistry registry) {
|
||||
MetricsReporterType type = config.getMetricsReporterType();
|
||||
MetricsReporter reporter = null;
|
||||
switch (type) {
|
||||
case GRAPHITE:
|
||||
reporter = new MetricsGraphiteReporter(config, registry);
|
||||
break;
|
||||
case INMEMORY:
|
||||
reporter = new InMemoryMetricsReporter();
|
||||
break;
|
||||
default:
|
||||
logger.error("Reporter type[" + type + "] is not supported.");
|
||||
break;
|
||||
}
|
||||
return reporter;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,27 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.metrics;
|
||||
|
||||
/**
|
||||
* Types of the reporter. Right now we only support Graphite. We can include JMX and CSV in the
|
||||
* future.
|
||||
*/
|
||||
public enum MetricsReporterType {
|
||||
GRAPHITE, INMEMORY
|
||||
}
|
||||
@@ -0,0 +1,806 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import com.google.common.hash.Hashing;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieDataFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.func.CopyOnWriteLazyInsertIterable;
|
||||
import org.apache.hudi.func.ParquetReaderIterator;
|
||||
import org.apache.hudi.func.SparkBoundedInMemoryExecutor;
|
||||
import org.apache.hudi.io.HoodieCleanHelper;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroParquetReader;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
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.Tuple2;
|
||||
|
||||
/**
|
||||
* Implementation of a very heavily read-optimized Hoodie Table where
|
||||
* <p>
|
||||
* INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing
|
||||
* file, to expand it
|
||||
* <p>
|
||||
* UPDATES - Produce a new version of the file, just replacing the updated records with new values
|
||||
*/
|
||||
public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends HoodieTable<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class);
|
||||
|
||||
public HoodieCopyOnWriteTable(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
}
|
||||
|
||||
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String,
|
||||
PartitionCleanStat> deleteFilesFunc(
|
||||
HoodieTable table) {
|
||||
return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>)
|
||||
iter -> {
|
||||
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
|
||||
|
||||
FileSystem fs = table.getMetaClient().getFs();
|
||||
while (iter.hasNext()) {
|
||||
Tuple2<String, String> partitionDelFileTuple = iter.next();
|
||||
String partitionPath = partitionDelFileTuple._1();
|
||||
String deletePathStr = partitionDelFileTuple._2();
|
||||
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
|
||||
if (!partitionCleanStatMap.containsKey(partitionPath)) {
|
||||
partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
|
||||
}
|
||||
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
|
||||
partitionCleanStat.addDeleteFilePatterns(deletePathStr);
|
||||
partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult);
|
||||
}
|
||||
|
||||
return partitionCleanStatMap.entrySet().stream()
|
||||
.map(e -> new Tuple2<>(e.getKey(), e.getValue()))
|
||||
.collect(Collectors.toList()).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private static PairFlatMapFunction<String, String, String> getFilesToDeleteFunc(HoodieTable table,
|
||||
HoodieWriteConfig config) {
|
||||
return (PairFlatMapFunction<String, String, String>) partitionPathToClean -> {
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(table, config);
|
||||
return cleaner.getDeletePaths(partitionPathToClean).stream()
|
||||
.map(deleteFile -> new Tuple2<>(partitionPathToClean, deleteFile.toString())).iterator();
|
||||
};
|
||||
}
|
||||
|
||||
private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr)
|
||||
throws IOException {
|
||||
Path deletePath = new Path(deletePathStr);
|
||||
logger.debug("Working on delete path :" + deletePath);
|
||||
boolean deleteResult = fs.delete(deletePath, false);
|
||||
if (deleteResult) {
|
||||
logger.debug("Cleaned file at path :" + deletePath);
|
||||
}
|
||||
return deleteResult;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
|
||||
if (profile == null) {
|
||||
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
|
||||
}
|
||||
return new UpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getInsertPartitioner(WorkloadProfile profile) {
|
||||
return getUpsertPartitioner(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isWorkloadProfileNeeded() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String commitTime) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionInstantTime,
|
||||
HoodieCompactionPlan compactionPlan) {
|
||||
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
|
||||
if (!recordItr.hasNext()) {
|
||||
logger.info("Empty partition with fileId => " + fileId);
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, recordItr);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileId);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieDataFile oldDataFile) throws IOException {
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(commitTime, fileId, keyToNewRecords, oldDataFile);
|
||||
return handleUpdateInternal(upsertHandle, commitTime, fileId);
|
||||
}
|
||||
|
||||
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle,
|
||||
String commitTime, String fileId)
|
||||
throws IOException {
|
||||
if (upsertHandle.getOldFilePath() == null) {
|
||||
throw new HoodieUpsertException(
|
||||
"Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId);
|
||||
} else {
|
||||
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getWriterSchema());
|
||||
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
||||
try (ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(upsertHandle.getOldFilePath())
|
||||
.withConf(getHadoopConf()).build()) {
|
||||
wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader),
|
||||
new UpdateHandler(upsertHandle), x -> x);
|
||||
wrapper.execute();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
} finally {
|
||||
upsertHandle.close();
|
||||
if (null != wrapper) {
|
||||
wrapper.shutdownNow();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//TODO(vc): This needs to be revisited
|
||||
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
|
||||
logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath()
|
||||
+ ", " + upsertHandle.getWriteStatus());
|
||||
}
|
||||
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus()))
|
||||
.iterator();
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileId);
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieDataFile dataFileToBeMerged) {
|
||||
return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileId, dataFileToBeMerged);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String idPfx,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
|
||||
if (!recordItr.hasNext()) {
|
||||
logger.info("Empty partition");
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
return new CopyOnWriteLazyInsertIterable<>(recordItr, config, commitTime, this, idPfx);
|
||||
}
|
||||
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String partitionPath, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) {
|
||||
HoodieCreateHandle createHandle = new HoodieCreateHandle(config, commitTime, this, partitionPath, fileId,
|
||||
recordItr);
|
||||
createHandle.write();
|
||||
return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator();
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime, Integer partition,
|
||||
Iterator recordItr, Partitioner partitioner) {
|
||||
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
|
||||
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
|
||||
BucketType btype = binfo.bucketType;
|
||||
try {
|
||||
if (btype.equals(BucketType.INSERT)) {
|
||||
return handleInsert(commitTime, binfo.fileIdPrefix, recordItr);
|
||||
} else if (btype.equals(BucketType.UPDATE)) {
|
||||
return handleUpdate(commitTime, binfo.fileIdPrefix, recordItr);
|
||||
} else {
|
||||
throw new HoodieUpsertException(
|
||||
"Unknown bucketType " + btype + " for partition :" + partition);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
|
||||
logger.error(msg, t);
|
||||
throw new HoodieUpsertException(msg, t);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsertPartition(String commitTime, Integer partition,
|
||||
Iterator recordItr, Partitioner partitioner) {
|
||||
return handleUpsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs cleaning of partition paths according to cleaning policy and returns the number of
|
||||
* files cleaned. Handles skews in partitions to clean by making files to clean as the unit of
|
||||
* task distribution.
|
||||
*
|
||||
* @throws IllegalArgumentException if unknown cleaning policy is provided
|
||||
*/
|
||||
@Override
|
||||
public List<HoodieCleanStat> clean(JavaSparkContext jsc) {
|
||||
try {
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
List<String> partitionsToClean = FSUtils
|
||||
.getAllPartitionPaths(fs, getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning());
|
||||
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config
|
||||
.getCleanerPolicy());
|
||||
if (partitionsToClean.isEmpty()) {
|
||||
logger.info("Nothing to clean here mom. It is already clean");
|
||||
return Collections.emptyList();
|
||||
}
|
||||
return cleanPartitionPaths(partitionsToClean, jsc);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to clean up after commit", e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Common method used for cleaning out parquet files under a partition path during rollback of a
|
||||
* set of commits
|
||||
*/
|
||||
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, String partitionPath,
|
||||
PathFilter filter)
|
||||
throws IOException {
|
||||
logger.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
|
||||
for (FileStatus file : toBeDeleted) {
|
||||
boolean success = fs.delete(file.getPath(), false);
|
||||
results.put(file, success);
|
||||
logger.info("Delete file " + file.getPath() + "\t" + success);
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
/**
|
||||
* Common method used for cleaning out parquet files under a partition path during rollback of a
|
||||
* set of commits
|
||||
*/
|
||||
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, String commit, String
|
||||
partitionPath)
|
||||
throws IOException {
|
||||
logger.info("Cleaning path " + partitionPath);
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
PathFilter filter = (path) -> {
|
||||
if (path.toString().contains(".parquet")) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||
return commit.equals(fileCommitTime);
|
||||
}
|
||||
return false;
|
||||
};
|
||||
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
|
||||
for (FileStatus file : toBeDeleted) {
|
||||
boolean success = fs.delete(file.getPath(), false);
|
||||
results.put(file, success);
|
||||
logger.info("Delete file " + file.getPath() + "\t" + success);
|
||||
}
|
||||
return results;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||
throws IOException {
|
||||
String actionType = metaClient.getCommitActionType();
|
||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
|
||||
List<String> inflights = this.getInflightCommitTimeline().getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
// Atomically unpublish the commits
|
||||
if (!inflights.contains(commit)) {
|
||||
activeTimeline.revertToInflight(new HoodieInstant(false, actionType, commit));
|
||||
}
|
||||
logger.info("Unpublished " + commit);
|
||||
|
||||
// delete all the data files for this commit
|
||||
logger.info("Clean out all parquet files generated for commit: " + commit);
|
||||
List<HoodieRollbackStat> stats = jsc.parallelize(FSUtils
|
||||
.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// Scan all partitions files with this commit time
|
||||
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
deleteCleanedFiles(filesToDeletedStatus, commit, partitionPath);
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(filesToDeletedStatus).build();
|
||||
}).collect();
|
||||
|
||||
// Delete Inflight instant if enabled
|
||||
deleteInflightInstant(deleteInstants, activeTimeline,
|
||||
new HoodieInstant(true, actionType, commit));
|
||||
return stats;
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete Inflight instant if enabled
|
||||
* @param deleteInstant Enable Deletion of Inflight instant
|
||||
* @param activeTimeline Hoodie active timeline
|
||||
* @param instantToBeDeleted Instant to be deleted
|
||||
*/
|
||||
protected void deleteInflightInstant(boolean deleteInstant, HoodieActiveTimeline activeTimeline,
|
||||
HoodieInstant instantToBeDeleted) {
|
||||
// Remove marker files always on rollback
|
||||
deleteMarkerDir(instantToBeDeleted.getTimestamp());
|
||||
|
||||
// Remove the rolled back inflight commits
|
||||
if (deleteInstant) {
|
||||
activeTimeline.deleteInflight(instantToBeDeleted);
|
||||
logger.info("Deleted inflight commit " + instantToBeDeleted);
|
||||
} else {
|
||||
logger.warn("Rollback finished without deleting inflight instant file. Instant=" + instantToBeDeleted);
|
||||
}
|
||||
}
|
||||
|
||||
private List<HoodieCleanStat> cleanPartitionPaths(List<String> partitionsToClean,
|
||||
JavaSparkContext jsc) {
|
||||
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
|
||||
logger.info("Using cleanerParallelism: " + cleanerParallelism);
|
||||
List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
|
||||
.parallelize(partitionsToClean, cleanerParallelism)
|
||||
.flatMapToPair(getFilesToDeleteFunc(this, config))
|
||||
.repartition(cleanerParallelism) // repartition to remove skews
|
||||
.mapPartitionsToPair(deleteFilesFunc(this)).reduceByKey(
|
||||
// merge partition level clean stats below
|
||||
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1
|
||||
.merge(e2)).collect();
|
||||
|
||||
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
|
||||
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
|
||||
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
|
||||
// Return PartitionCleanStat for each partition passed.
|
||||
return partitionsToClean.stream().map(partitionPath -> {
|
||||
PartitionCleanStat partitionCleanStat =
|
||||
(partitionCleanStatsMap.containsKey(partitionPath)) ? partitionCleanStatsMap
|
||||
.get(partitionPath) : new PartitionCleanStat(partitionPath);
|
||||
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy())
|
||||
.withPartitionPath(partitionPath)
|
||||
.withEarliestCommitRetained(cleaner.getEarliestCommitToRetain())
|
||||
.withDeletePathPattern(partitionCleanStat.deletePathPatterns)
|
||||
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles)
|
||||
.withFailedDeletes(partitionCleanStat.failedDeleteFiles).build();
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
enum BucketType {
|
||||
UPDATE, INSERT
|
||||
}
|
||||
|
||||
/**
|
||||
* Consumer that dequeues records from queue and sends to Merge Handle
|
||||
*/
|
||||
private static class UpdateHandler extends BoundedInMemoryQueueConsumer<GenericRecord, Void> {
|
||||
|
||||
private final HoodieMergeHandle upsertHandle;
|
||||
|
||||
private UpdateHandler(HoodieMergeHandle upsertHandle) {
|
||||
this.upsertHandle = upsertHandle;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void consumeOneRecord(GenericRecord record) {
|
||||
upsertHandle.write(record);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void finish() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Void getResult() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private static class PartitionCleanStat implements Serializable {
|
||||
|
||||
private final String partitionPath;
|
||||
private final List<String> deletePathPatterns = new ArrayList<>();
|
||||
private final List<String> successDeleteFiles = new ArrayList<>();
|
||||
private final List<String> failedDeleteFiles = new ArrayList<>();
|
||||
|
||||
private PartitionCleanStat(String partitionPath) {
|
||||
this.partitionPath = partitionPath;
|
||||
}
|
||||
|
||||
private void addDeletedFileResult(String deletePathStr, Boolean deletedFileResult) {
|
||||
if (deletedFileResult) {
|
||||
successDeleteFiles.add(deletePathStr);
|
||||
} else {
|
||||
failedDeleteFiles.add(deletePathStr);
|
||||
}
|
||||
}
|
||||
|
||||
private void addDeleteFilePatterns(String deletePathStr) {
|
||||
deletePathPatterns.add(deletePathStr);
|
||||
}
|
||||
|
||||
private PartitionCleanStat merge(PartitionCleanStat other) {
|
||||
if (!this.partitionPath.equals(other.partitionPath)) {
|
||||
throw new RuntimeException(String
|
||||
.format("partitionPath is not a match: (%s, %s)", partitionPath, other.partitionPath));
|
||||
}
|
||||
successDeleteFiles.addAll(other.successDeleteFiles);
|
||||
deletePathPatterns.addAll(other.deletePathPatterns);
|
||||
failedDeleteFiles.addAll(other.failedDeleteFiles);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper class for a small file's location and its actual size on disk
|
||||
*/
|
||||
static class SmallFile implements Serializable {
|
||||
|
||||
HoodieRecordLocation location;
|
||||
long sizeBytes;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("SmallFile {");
|
||||
sb.append("location=").append(location).append(", ");
|
||||
sb.append("sizeBytes=").append(sizeBytes);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper class for an insert bucket along with the weight [0.0, 0.1] that defines the amount of
|
||||
* incoming inserts that should be allocated to the bucket
|
||||
*/
|
||||
class InsertBucket implements Serializable {
|
||||
|
||||
int bucketNumber;
|
||||
// fraction of total inserts, that should go into this bucket
|
||||
double weight;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("WorkloadStat {");
|
||||
sb.append("bucketNumber=").append(bucketNumber).append(", ");
|
||||
sb.append("weight=").append(weight);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper class for a bucket's type (INSERT and UPDATE) and its file location
|
||||
*/
|
||||
class BucketInfo implements Serializable {
|
||||
|
||||
BucketType bucketType;
|
||||
String fileIdPrefix;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("BucketInfo {");
|
||||
sb.append("bucketType=").append(bucketType).append(", ");
|
||||
sb.append("fileIdPrefix=").append(fileIdPrefix);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Packs incoming records to be upserted, into buckets (1 bucket = 1 RDD partition)
|
||||
*/
|
||||
class UpsertPartitioner extends Partitioner {
|
||||
|
||||
/**
|
||||
* List of all small files to be corrected
|
||||
*/
|
||||
List<SmallFile> smallFiles = new ArrayList<SmallFile>();
|
||||
/**
|
||||
* Total number of RDD partitions, is determined by total buckets we want to pack the incoming
|
||||
* workload into
|
||||
*/
|
||||
private int totalBuckets = 0;
|
||||
/**
|
||||
* Stat for the current workload. Helps in determining total inserts, upserts etc.
|
||||
*/
|
||||
private WorkloadStat globalStat;
|
||||
/**
|
||||
* Helps decide which bucket an incoming update should go to.
|
||||
*/
|
||||
private HashMap<String, Integer> updateLocationToBucket;
|
||||
/**
|
||||
* Helps us pack inserts into 1 or more buckets depending on number of incoming records.
|
||||
*/
|
||||
private HashMap<String, List<InsertBucket>> partitionPathToInsertBuckets;
|
||||
/**
|
||||
* Remembers what type each bucket is for later.
|
||||
*/
|
||||
private HashMap<Integer, BucketInfo> bucketInfoMap;
|
||||
|
||||
/**
|
||||
* Rolling stats for files
|
||||
*/
|
||||
protected HoodieRollingStatMetadata rollingStatMetadata;
|
||||
protected long averageRecordSize;
|
||||
|
||||
UpsertPartitioner(WorkloadProfile profile) {
|
||||
updateLocationToBucket = new HashMap<>();
|
||||
partitionPathToInsertBuckets = new HashMap<>();
|
||||
bucketInfoMap = new HashMap<>();
|
||||
globalStat = profile.getGlobalStat();
|
||||
rollingStatMetadata = getRollingStats();
|
||||
assignUpdates(profile);
|
||||
assignInserts(profile);
|
||||
|
||||
logger.info(
|
||||
"Total Buckets :" + totalBuckets + ", " + "buckets info => " + bucketInfoMap + ", \n"
|
||||
+ "Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n"
|
||||
+ "UpdateLocations mapped to buckets =>" + updateLocationToBucket);
|
||||
}
|
||||
|
||||
private void assignUpdates(WorkloadProfile profile) {
|
||||
// each update location gets a partition
|
||||
WorkloadStat gStat = profile.getGlobalStat();
|
||||
for (Map.Entry<String, Pair<String, Long>> updateLocEntry : gStat.getUpdateLocationToCount()
|
||||
.entrySet()) {
|
||||
addUpdateBucket(updateLocEntry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
private int addUpdateBucket(String fileIdHint) {
|
||||
int bucket = totalBuckets;
|
||||
updateLocationToBucket.put(fileIdHint, bucket);
|
||||
BucketInfo bucketInfo = new BucketInfo();
|
||||
bucketInfo.bucketType = BucketType.UPDATE;
|
||||
bucketInfo.fileIdPrefix = fileIdHint;
|
||||
bucketInfoMap.put(totalBuckets, bucketInfo);
|
||||
totalBuckets++;
|
||||
return bucket;
|
||||
}
|
||||
|
||||
private void assignInserts(WorkloadProfile profile) {
|
||||
// for new inserts, compute buckets depending on how many records we have for each partition
|
||||
Set<String> partitionPaths = profile.getPartitionPaths();
|
||||
long averageRecordSize = averageBytesPerRecord();
|
||||
logger.info("AvgRecordSize => " + averageRecordSize);
|
||||
for (String partitionPath : partitionPaths) {
|
||||
WorkloadStat pStat = profile.getWorkloadStat(partitionPath);
|
||||
if (pStat.getNumInserts() > 0) {
|
||||
|
||||
List<SmallFile> smallFiles = getSmallFiles(partitionPath);
|
||||
logger.info("For partitionPath : " + partitionPath + " Small Files => " + smallFiles);
|
||||
|
||||
long totalUnassignedInserts = pStat.getNumInserts();
|
||||
List<Integer> bucketNumbers = new ArrayList<>();
|
||||
List<Long> recordsPerBucket = new ArrayList<>();
|
||||
|
||||
// first try packing this into one of the smallFiles
|
||||
for (SmallFile smallFile : smallFiles) {
|
||||
long recordsToAppend = Math
|
||||
.min((config.getParquetMaxFileSize() - smallFile.sizeBytes) / averageRecordSize,
|
||||
totalUnassignedInserts);
|
||||
if (recordsToAppend > 0 && totalUnassignedInserts > 0) {
|
||||
// create a new bucket or re-use an existing bucket
|
||||
int bucket;
|
||||
if (updateLocationToBucket.containsKey(smallFile.location.getFileId())) {
|
||||
bucket = updateLocationToBucket.get(smallFile.location.getFileId());
|
||||
logger.info("Assigning " + recordsToAppend + " inserts to existing update bucket "
|
||||
+ bucket);
|
||||
} else {
|
||||
bucket = addUpdateBucket(smallFile.location.getFileId());
|
||||
logger.info(
|
||||
"Assigning " + recordsToAppend + " inserts to new update bucket " + bucket);
|
||||
}
|
||||
bucketNumbers.add(bucket);
|
||||
recordsPerBucket.add(recordsToAppend);
|
||||
totalUnassignedInserts -= recordsToAppend;
|
||||
}
|
||||
}
|
||||
|
||||
// if we have anything more, create new insert buckets, like normal
|
||||
if (totalUnassignedInserts > 0) {
|
||||
long insertRecordsPerBucket = config.getCopyOnWriteInsertSplitSize();
|
||||
if (config.shouldAutoTuneInsertSplits()) {
|
||||
insertRecordsPerBucket = config.getParquetMaxFileSize() / averageRecordSize;
|
||||
}
|
||||
|
||||
int insertBuckets = (int) Math.ceil((1.0 * totalUnassignedInserts) / insertRecordsPerBucket);
|
||||
logger.info(
|
||||
"After small file assignment: unassignedInserts => " + totalUnassignedInserts
|
||||
+ ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => "
|
||||
+ insertRecordsPerBucket);
|
||||
for (int b = 0; b < insertBuckets; b++) {
|
||||
bucketNumbers.add(totalBuckets);
|
||||
recordsPerBucket.add(totalUnassignedInserts / insertBuckets);
|
||||
BucketInfo bucketInfo = new BucketInfo();
|
||||
bucketInfo.bucketType = BucketType.INSERT;
|
||||
bucketInfo.fileIdPrefix = FSUtils.createNewFileIdPfx();
|
||||
bucketInfoMap.put(totalBuckets, bucketInfo);
|
||||
totalBuckets++;
|
||||
}
|
||||
}
|
||||
|
||||
// Go over all such buckets, and assign weights as per amount of incoming inserts.
|
||||
List<InsertBucket> insertBuckets = new ArrayList<>();
|
||||
for (int i = 0; i < bucketNumbers.size(); i++) {
|
||||
InsertBucket bkt = new InsertBucket();
|
||||
bkt.bucketNumber = bucketNumbers.get(i);
|
||||
bkt.weight = (1.0 * recordsPerBucket.get(i)) / pStat.getNumInserts();
|
||||
insertBuckets.add(bkt);
|
||||
}
|
||||
logger.info(
|
||||
"Total insert buckets for partition path " + partitionPath + " => " + insertBuckets);
|
||||
partitionPathToInsertBuckets.put(partitionPath, insertBuckets);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a list of small files in the given partition path
|
||||
*/
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
HoodieTimeline commitTimeline = getCompletedCommitsTimeline();
|
||||
|
||||
if (!commitTimeline.empty()) { // if we have some commits
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
List<HoodieDataFile> allFiles = getROFileSystemView()
|
||||
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp())
|
||||
.collect(Collectors.toList());
|
||||
|
||||
for (HoodieDataFile file : allFiles) {
|
||||
if (file.getFileSize() < config.getParquetSmallFileLimit()) {
|
||||
String filename = file.getFileName();
|
||||
SmallFile sf = new SmallFile();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename),
|
||||
FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = file.getFileSize();
|
||||
smallFileLocations.add(sf);
|
||||
// Update the global small files list
|
||||
smallFiles.add(sf);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtains the average record size based on records written during last commit. Used for
|
||||
* estimating how many records pack into one file.
|
||||
*/
|
||||
protected long averageBytesPerRecord() {
|
||||
long avgSize = 0L;
|
||||
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants();
|
||||
try {
|
||||
if (!commitTimeline.empty()) {
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(commitTimeline.getInstantDetails(latestCommitTime).get(), HoodieCommitMetadata.class);
|
||||
avgSize = (long) Math.ceil(
|
||||
(1.0 * commitMetadata.fetchTotalBytesWritten()) / commitMetadata
|
||||
.fetchTotalRecordsWritten());
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
// make this fail safe.
|
||||
logger.error("Error trying to compute average bytes/record ", t);
|
||||
}
|
||||
return avgSize <= 0L ? config.getCopyOnWriteRecordSizeEstimate() : avgSize;
|
||||
}
|
||||
|
||||
public BucketInfo getBucketInfo(int bucketNumber) {
|
||||
return bucketInfoMap.get(bucketNumber);
|
||||
}
|
||||
|
||||
public List<InsertBucket> getInsertBuckets(String partitionPath) {
|
||||
return partitionPathToInsertBuckets.get(partitionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int numPartitions() {
|
||||
return totalBuckets;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getPartition(Object key) {
|
||||
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation = (Tuple2<HoodieKey,
|
||||
Option<HoodieRecordLocation>>) key;
|
||||
if (keyLocation._2().isPresent()) {
|
||||
HoodieRecordLocation location = keyLocation._2().get();
|
||||
return updateLocationToBucket.get(location.getFileId());
|
||||
} else {
|
||||
List<InsertBucket> targetBuckets = partitionPathToInsertBuckets
|
||||
.get(keyLocation._1().getPartitionPath());
|
||||
// pick the target bucket to use based on the weights.
|
||||
double totalWeight = 0.0;
|
||||
final long totalInserts = Math.max(1, globalStat.getNumInserts());
|
||||
final long hashOfKey = Hashing.md5()
|
||||
.hashString(keyLocation._1().getRecordKey(), StandardCharsets.UTF_8).asLong();
|
||||
final double r = 1.0 * Math.floorMod(hashOfKey, totalInserts) / totalInserts;
|
||||
for (InsertBucket insertBucket : targetBuckets) {
|
||||
totalWeight += insertBucket.weight;
|
||||
if (r <= totalWeight) {
|
||||
return insertBucket.bucketNumber;
|
||||
}
|
||||
}
|
||||
// return first one, by default
|
||||
return targetBuckets.get(0).bucketNumber;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected HoodieRollingStatMetadata getRollingStats() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,547 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieRollingStat;
|
||||
import org.apache.hudi.common.model.HoodieRollingStatMetadata;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat;
|
||||
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCommandBlock;
|
||||
import org.apache.hudi.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCompactionException;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.func.MergeOnReadLazyInsertIterable;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.io.HoodieAppendHandle;
|
||||
import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
|
||||
/**
|
||||
* Implementation of a more real-time read-optimized Hoodie Table where <p> INSERTS - Same as
|
||||
* HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the
|
||||
* smallest existing file, to expand it </p> <p> UPDATES - Appends the changes to a rolling log file
|
||||
* maintained per file Id. Compaction merges the log file into the base file. </p> <p> WARNING - MOR
|
||||
* table type does not support nested rollbacks, every rollback must be followed by an attempted
|
||||
* commit action </p>
|
||||
*/
|
||||
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
HoodieCopyOnWriteTable<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
||||
|
||||
// UpsertPartitioner for MergeOnRead table type
|
||||
private MergeOnReadUpsertPartitioner mergeOnReadUpsertPartitioner;
|
||||
|
||||
public HoodieMergeOnReadTable(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
super(config, jsc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Partitioner getUpsertPartitioner(WorkloadProfile profile) {
|
||||
if (profile == null) {
|
||||
throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner.");
|
||||
}
|
||||
mergeOnReadUpsertPartitioner = new MergeOnReadUpsertPartitioner(profile);
|
||||
return mergeOnReadUpsertPartitioner;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
|
||||
|
||||
if (!index.canIndexLogFiles() && mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
|
||||
logger.info(
|
||||
"Small file corrections for updates for commit " + commitTime + " for file " + fileId);
|
||||
return super.handleUpdate(commitTime, fileId, recordItr);
|
||||
} else {
|
||||
HoodieAppendHandle<T> appendHandle = new HoodieAppendHandle<>(config, commitTime, this,
|
||||
fileId, recordItr);
|
||||
appendHandle.doAppend();
|
||||
appendHandle.close();
|
||||
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
|
||||
.iterator();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String idPfx,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws Exception {
|
||||
// If canIndexLogFiles, write inserts to log files else write inserts to parquet files
|
||||
if (index.canIndexLogFiles()) {
|
||||
return new MergeOnReadLazyInsertIterable<>(recordItr, config, commitTime, this, idPfx);
|
||||
} else {
|
||||
return super.handleInsert(commitTime, idPfx, recordItr);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime) {
|
||||
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||
Option<HoodieInstant> lastCompaction = getActiveTimeline().getCommitTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
String deltaCommitsSinceTs = "0";
|
||||
if (lastCompaction.isPresent()) {
|
||||
deltaCommitsSinceTs = lastCompaction.get().getTimestamp();
|
||||
}
|
||||
|
||||
int deltaCommitsSinceLastCompaction = getActiveTimeline().getDeltaCommitTimeline()
|
||||
.findInstantsAfter(deltaCommitsSinceTs, Integer.MAX_VALUE).countInstants();
|
||||
if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
|
||||
logger.info("Not running compaction as only " + deltaCommitsSinceLastCompaction
|
||||
+ " delta commits was found since last compaction " + deltaCommitsSinceTs
|
||||
+ ". Waiting for " + config.getInlineCompactDeltaCommitMax());
|
||||
return new HoodieCompactionPlan();
|
||||
}
|
||||
|
||||
logger.info("Compacting merge on read table " + config.getBasePath());
|
||||
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
|
||||
try {
|
||||
return compactor.generateCompactionPlan(jsc, this, config, instantTime,
|
||||
((SyncableFileSystemView)getRTFileSystemView()).getPendingCompactionOperations()
|
||||
.map(instantTimeCompactionopPair -> instantTimeCompactionopPair.getValue().getFileGroupId())
|
||||
.collect(Collectors.toSet()));
|
||||
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not schedule compaction " + config.getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionInstantTime,
|
||||
HoodieCompactionPlan compactionPlan) {
|
||||
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
|
||||
try {
|
||||
return compactor.compact(jsc, compactionPlan, this, config, compactionInstantTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||
throws IOException {
|
||||
|
||||
// At the moment, MOR table type does not support bulk nested rollbacks. Nested rollbacks is an experimental
|
||||
// feature that is expensive. To perform nested rollbacks, initiate multiple requests of client.rollback
|
||||
// (commitToRollback).
|
||||
// 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
|
||||
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());
|
||||
HoodieInstant instantToRollback = commitOrCompactionOption.get();
|
||||
// Atomically un-publish all non-inflight commits
|
||||
if (!instantToRollback.isInflight()) {
|
||||
this.getActiveTimeline().revertToInflight(instantToRollback);
|
||||
}
|
||||
logger.info("Unpublished " + commit);
|
||||
Long startTime = System.currentTimeMillis();
|
||||
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize(FSUtils
|
||||
.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
|
||||
config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
HoodieActiveTimeline activeTimeline = this.getActiveTimeline().reload();
|
||||
HoodieRollbackStat hoodieRollbackStats = null;
|
||||
// Need to put the path filter here since Filter is not serializable
|
||||
// PathFilter to get all parquet files and log files that need to be deleted
|
||||
PathFilter filter = (path) -> {
|
||||
if (path.toString().contains(".parquet")) {
|
||||
String fileCommitTime = FSUtils.getCommitTime(path.getName());
|
||||
return commit.equals(fileCommitTime);
|
||||
} else if (path.toString().contains(".log")) {
|
||||
// Since the baseCommitTime is the only commit for new log files, it's okay here
|
||||
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
|
||||
return commit.equals(fileCommitTime);
|
||||
}
|
||||
return false;
|
||||
};
|
||||
|
||||
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
|
||||
switch (instantToRollback.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
try {
|
||||
// Rollback of a commit should delete the newly created parquet files along with any log
|
||||
// files created with this as baseCommit. This is required to support multi-rollbacks in a MOR table.
|
||||
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
try {
|
||||
// If there is no delta commit present after the current commit (if compaction), no action, else we
|
||||
// need to make sure that a compaction commit rollback also deletes any log files written as part of the
|
||||
// succeeding deltacommit.
|
||||
boolean higherDeltaCommits = !activeTimeline.getDeltaCommitTimeline()
|
||||
.filterCompletedInstants().findInstantsAfter(commit, 1).empty();
|
||||
if (higherDeltaCommits) {
|
||||
// Rollback of a compaction action with no higher deltacommit means that the compaction is scheduled
|
||||
// and has not yet finished. In this scenario we should delete only the newly created parquet files
|
||||
// and not corresponding base commit log files created with this as baseCommit since updates would
|
||||
// have been written to the log files.
|
||||
super.deleteCleanedFiles(filesToDeletedStatus, commit, partitionPath);
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
|
||||
} else {
|
||||
// No deltacommits present after this compaction commit (inflight or requested). In this case, we
|
||||
// can also delete any log files that were created with this compaction commit as base
|
||||
// commit.
|
||||
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
|
||||
}
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
// --------------------------------------------------------------------------------------------------
|
||||
// (A) The following cases are possible if index.canIndexLogFiles and/or index.isGlobal
|
||||
// --------------------------------------------------------------------------------------------------
|
||||
// (A.1) Failed first commit - Inserts were written to log files and HoodieWriteStat has no entries. In
|
||||
// this scenario we would want to delete these log files.
|
||||
// (A.2) Failed recurring commit - Inserts/Updates written to log files. In this scenario,
|
||||
// HoodieWriteStat will have the baseCommitTime for the first log file written, add rollback blocks.
|
||||
// (A.3) Rollback triggered for first commit - Inserts were written to the log files but the commit is
|
||||
// being reverted. In this scenario, HoodieWriteStat will be `null` for the attribute prevCommitTime and
|
||||
// and hence will end up deleting these log files. This is done so there are no orphan log files
|
||||
// lying around.
|
||||
// (A.4) Rollback triggered for recurring commits - Inserts/Updates are being rolled back, the actions
|
||||
// taken in this scenario is a combination of (A.2) and (A.3)
|
||||
// ---------------------------------------------------------------------------------------------------
|
||||
// (B) The following cases are possible if !index.canIndexLogFiles and/or !index.isGlobal
|
||||
// ---------------------------------------------------------------------------------------------------
|
||||
// (B.1) Failed first commit - Inserts were written to parquet files and HoodieWriteStat has no entries.
|
||||
// In this scenario, we delete all the parquet files written for the failed commit.
|
||||
// (B.2) Failed recurring commits - Inserts were written to parquet files and updates to log files. In
|
||||
// this scenario, perform (A.1) and for updates written to log files, write rollback blocks.
|
||||
// (B.3) Rollback triggered for first commit - Same as (B.1)
|
||||
// (B.4) Rollback triggered for recurring commits - Same as (B.2) plus we need to delete the log files
|
||||
// as well if the base parquet file gets deleted.
|
||||
try {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
metaClient.getCommitTimeline().getInstantDetails(
|
||||
new HoodieInstant(true, instantToRollback.getAction(), instantToRollback.getTimestamp()))
|
||||
.get(), HoodieCommitMetadata.class);
|
||||
|
||||
// read commit file and (either append delete blocks or delete file)
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
|
||||
|
||||
// In case all data was inserts and the commit failed, delete the file belonging to that commit
|
||||
// We do not know fileIds for inserts (first inserts are either log files or parquet files),
|
||||
// delete all files for the corresponding failed commit, if present (same as COW)
|
||||
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
|
||||
final Set<String> deletedFiles = filesToDeletedStatus.entrySet().stream()
|
||||
.map(entry -> {
|
||||
Path filePath = entry.getKey().getPath();
|
||||
return FSUtils.getFileIdFromFilePath(filePath);
|
||||
}).collect(Collectors.toSet());
|
||||
|
||||
// append rollback blocks for updates
|
||||
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
|
||||
hoodieRollbackStats = rollback(index, partitionPath, commit, commitMetadata, filesToDeletedStatus,
|
||||
filesToNumBlocksRollback, deletedFiles);
|
||||
}
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return hoodieRollbackStats;
|
||||
}).filter(Objects::nonNull).collect();
|
||||
|
||||
// Delete Inflight instants if enabled
|
||||
deleteInflightInstant(deleteInstants, this.getActiveTimeline(), new HoodieInstant(true, instantToRollback
|
||||
.getAction(), instantToRollback.getTimestamp()));
|
||||
|
||||
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
|
||||
return allRollbackStats;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void finalizeWrite(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats)
|
||||
throws HoodieIOException {
|
||||
// delegate to base class for MOR tables
|
||||
super.finalizeWrite(jsc, instantTs, stats);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieRollingStatMetadata getRollingStats() {
|
||||
try {
|
||||
Option<HoodieInstant> lastInstant = this.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants()
|
||||
.lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
this.getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class);
|
||||
Option<String> lastRollingStat = Option.ofNullable(commitMetadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY));
|
||||
if (lastRollingStat.isPresent()) {
|
||||
return HoodieCommitMetadata
|
||||
.fromBytes(lastRollingStat.get().getBytes(), HoodieRollingStatMetadata.class);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet
|
||||
* files to larger ones without the need for an index in the logFile.
|
||||
*/
|
||||
class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner {
|
||||
|
||||
MergeOnReadUpsertPartitioner(WorkloadProfile profile) {
|
||||
super(profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
// Init here since this class (and member variables) might not have been initialized
|
||||
HoodieTimeline commitTimeline = getCompletedCommitsTimeline();
|
||||
|
||||
// Find out all eligible small file slices
|
||||
if (!commitTimeline.empty()) {
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
// find smallest file in partition and append to it
|
||||
List<FileSlice> allSmallFileSlices = new ArrayList<>();
|
||||
// If we cannot index log files, then we choose the smallest parquet file in the partition and add inserts to
|
||||
// it. Doing this overtime for a partition, we ensure that we handle small file issues
|
||||
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
|
||||
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());
|
||||
if (smallFileSlice.isPresent()) {
|
||||
allSmallFileSlices.add(smallFileSlice.get());
|
||||
}
|
||||
} else {
|
||||
// If we can index log files, we can add more inserts to log files for fileIds including those under
|
||||
// pending compaction.
|
||||
List<FileSlice> allFileSlices = getRTFileSystemView()
|
||||
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), true)
|
||||
.collect(Collectors.toList());
|
||||
for (FileSlice fileSlice : allFileSlices) {
|
||||
if (isSmallFile(partitionPath, fileSlice)) {
|
||||
allSmallFileSlices.add(fileSlice);
|
||||
}
|
||||
}
|
||||
}
|
||||
// Create SmallFiles from the eligible file slices
|
||||
for (FileSlice smallFileSlice : allSmallFileSlices) {
|
||||
SmallFile sf = new SmallFile();
|
||||
if (smallFileSlice.getDataFile().isPresent()) {
|
||||
// TODO : Move logic of file name, file id, base commit time handling inside file slice
|
||||
String filename = smallFileSlice.getDataFile().get().getFileName();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = getTotalFileSize(partitionPath, smallFileSlice);
|
||||
smallFileLocations.add(sf);
|
||||
// Update the global small files list
|
||||
smallFiles.add(sf);
|
||||
} else {
|
||||
HoodieLogFile logFile = smallFileSlice.getLogFiles().findFirst().get();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()),
|
||||
FSUtils.getFileIdFromLogPath(logFile.getPath()));
|
||||
sf.sizeBytes = getTotalFileSize(partitionPath, smallFileSlice);
|
||||
smallFileLocations.add(sf);
|
||||
// Update the global small files list
|
||||
smallFiles.add(sf);
|
||||
}
|
||||
}
|
||||
}
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
public List<String> getSmallFileIds() {
|
||||
return (List<String>) smallFiles.stream()
|
||||
.map(smallFile -> ((SmallFile) smallFile).location.getFileId())
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private long getTotalFileSize(String partitionPath, FileSlice fileSlice) {
|
||||
if (rollingStatMetadata != null) {
|
||||
Map<String, HoodieRollingStat> partitionRollingStats =
|
||||
rollingStatMetadata.getPartitionToRollingStats().get(partitionPath);
|
||||
if (partitionRollingStats != null) {
|
||||
HoodieRollingStat rollingStatForFile = partitionRollingStats.get(fileSlice.getFileId());
|
||||
if (rollingStatForFile != null) {
|
||||
long inserts = rollingStatForFile.getInserts();
|
||||
return averageRecordSize * inserts;
|
||||
}
|
||||
}
|
||||
}
|
||||
// In case Rolling Stats is not present, fall back to sizing log files based on heuristics
|
||||
if (!fileSlice.getDataFile().isPresent()) {
|
||||
return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
|
||||
} else {
|
||||
return fileSlice.getDataFile().get().getFileSize() + convertLogFilesSizeToExpectedParquetSize(fileSlice
|
||||
.getLogFiles().collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isSmallFile(String partitionPath, FileSlice fileSlice) {
|
||||
long totalSize = getTotalFileSize(partitionPath, fileSlice);
|
||||
return totalSize < config.getParquetMaxFileSize();
|
||||
}
|
||||
|
||||
// TODO (NA) : Make this static part of utility
|
||||
@VisibleForTesting
|
||||
public long convertLogFilesSizeToExpectedParquetSize(List<HoodieLogFile> hoodieLogFiles) {
|
||||
long totalSizeOfLogFiles = hoodieLogFiles.stream().map(hoodieLogFile -> hoodieLogFile.getFileSize())
|
||||
.filter(size -> size > 0)
|
||||
.reduce((a, b) -> (a + b)).orElse(0L);
|
||||
// Here we assume that if there is no base parquet file, all log files contain only inserts.
|
||||
// We can then just get the parquet equivalent size of these log files, compare that with
|
||||
// {@link config.getParquetMaxFileSize()} and decide if there is scope to insert more rows
|
||||
long logFilesEquivalentParquetFileSize = (long) (totalSizeOfLogFiles * config
|
||||
.getLogFileToParquetCompressionRatio());
|
||||
return logFilesEquivalentParquetFileSize;
|
||||
}
|
||||
}
|
||||
|
||||
private Map<HeaderMetadataType, String> generateHeader(String commit) {
|
||||
// generate metadata
|
||||
Map<HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, commit);
|
||||
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK
|
||||
.ordinal()));
|
||||
return header;
|
||||
}
|
||||
|
||||
private HoodieRollbackStat rollback(HoodieIndex hoodieIndex, String partitionPath, String commit,
|
||||
HoodieCommitMetadata commitMetadata, final Map<FileStatus, Boolean> filesToDeletedStatus,
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback, Set<String> deletedFiles) {
|
||||
// wStat.getPrevCommit() might not give the right commit time in the following
|
||||
// scenario : If a compaction was scheduled, the new commitTime associated with the requested compaction will be
|
||||
// used to write the new log files. In this case, the commit time for the log file is the compaction requested time.
|
||||
// But the index (global) might store the baseCommit of the parquet and not the requested, hence get the
|
||||
// baseCommit always by listing the file slice
|
||||
Map<String, String> fileIdToBaseCommitTimeForLogMap = this.getRTFileSystemView().getLatestFileSlices(partitionPath)
|
||||
.collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime));
|
||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
||||
.filter(wStat -> {
|
||||
// Filter out stats without prevCommit since they are all inserts
|
||||
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null
|
||||
&& !deletedFiles.contains(wStat.getFileId());
|
||||
}).forEach(wStat -> {
|
||||
Writer writer = null;
|
||||
String baseCommitTime = fileIdToBaseCommitTimeForLogMap.get(wStat.getFileId());
|
||||
boolean success = false;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder().onParentPath(
|
||||
FSUtils.getPartitionPath(this.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime)
|
||||
.withFs(this.metaClient.getFs())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
// generate metadata
|
||||
Map<HeaderMetadataType, String> header = generateHeader(commit);
|
||||
// if update belongs to an existing log file
|
||||
writer = writer.appendBlock(new HoodieCommandBlock(header));
|
||||
success = true;
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException(
|
||||
"Failed to rollback for commit " + commit, io);
|
||||
} finally {
|
||||
try {
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
if (success) {
|
||||
// This step is intentionally done after writer is closed. Guarantees that
|
||||
// getFileStatus would reflect correct stats and FileNotFoundException is not thrown in
|
||||
// cloud-storage : HUDI-168
|
||||
filesToNumBlocksRollback.put(this.getMetaClient().getFs()
|
||||
.getFileStatus(writer.getLogFile().getPath()), 1L);
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
}
|
||||
});
|
||||
return HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(filesToDeletedStatus)
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
|
||||
}
|
||||
|
||||
}
|
||||
430
hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java
Normal file
430
hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java
Normal file
@@ -0,0 +1,430 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.WriteStatus;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.avro.model.HoodieSavepointMetadata;
|
||||
import org.apache.hudi.client.utils.ClientUtils;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.SerializableConfiguration;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.table.TableFileSystemView;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewManager;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.AvroUtils;
|
||||
import org.apache.hudi.common.util.ConsistencyGuard;
|
||||
import org.apache.hudi.common.util.ConsistencyGuard.FileVisibility;
|
||||
import org.apache.hudi.common.util.FSUtils;
|
||||
import org.apache.hudi.common.util.FailSafeConsistencyGuard;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.HoodieSavepointException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.Partitioner;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
|
||||
/**
|
||||
* Abstract implementation of a HoodieTable
|
||||
*/
|
||||
public abstract class HoodieTable<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieTable.class);
|
||||
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final HoodieIndex<T> index;
|
||||
|
||||
private SerializableConfiguration hadoopConfiguration;
|
||||
private transient FileSystemViewManager viewManager;
|
||||
|
||||
protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
this.config = config;
|
||||
this.hadoopConfiguration = new SerializableConfiguration(jsc.hadoopConfiguration());
|
||||
this.viewManager = FileSystemViewManager.createViewManager(
|
||||
new SerializableConfiguration(jsc.hadoopConfiguration()), config.getViewStorageConfig());
|
||||
this.metaClient = ClientUtils.createMetaClient(jsc, config, true);
|
||||
this.index = HoodieIndex.createIndex(config, jsc);
|
||||
}
|
||||
|
||||
private synchronized FileSystemViewManager getViewManager() {
|
||||
if (null == viewManager) {
|
||||
viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration,
|
||||
config.getViewStorageConfig());
|
||||
}
|
||||
return viewManager;
|
||||
}
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
||||
HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return new HoodieCopyOnWriteTable<>(config, jsc);
|
||||
case MERGE_ON_READ:
|
||||
return new HoodieMergeOnReadTable<>(config, jsc);
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a partitioner to perform the upsert operation, based on the workload profile
|
||||
*/
|
||||
public abstract Partitioner getUpsertPartitioner(WorkloadProfile profile);
|
||||
|
||||
/**
|
||||
* Provides a partitioner to perform the insert operation, based on the workload profile
|
||||
*/
|
||||
public abstract Partitioner getInsertPartitioner(WorkloadProfile profile);
|
||||
|
||||
/**
|
||||
* Return whether this HoodieTable implementation can benefit from workload profiling
|
||||
*/
|
||||
public abstract boolean isWorkloadProfileNeeded();
|
||||
|
||||
public HoodieWriteConfig getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient getMetaClient() {
|
||||
return metaClient;
|
||||
}
|
||||
|
||||
public Configuration getHadoopConf() {
|
||||
return metaClient.getHadoopConf();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView getFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitsTimeline());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the read optimized view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
|
||||
return getViewManager().getFileSystemView(metaClient.getBasePath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the real time view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView.RealtimeView getRTFileSystemView() {
|
||||
return getViewManager().getFileSystemView(metaClient.getBasePath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get complete view of the file system for this table with ability to force sync
|
||||
*/
|
||||
public SyncableFileSystemView getHoodieView() {
|
||||
return getViewManager().getFileSystemView(metaClient.getBasePath());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) commit + deltacommit timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedCommitsTimeline() {
|
||||
return metaClient.getCommitsTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) commit timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedCommitTimeline() {
|
||||
return metaClient.getCommitTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the inflights (no-completed) commit timeline
|
||||
*/
|
||||
public HoodieTimeline getInflightCommitTimeline() {
|
||||
return metaClient.getCommitsTimeline().filterInflightsExcludingCompaction();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) clean timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedCleanTimeline() {
|
||||
return getActiveTimeline().getCleanerTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) savepoint timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedSavepointTimeline() {
|
||||
return getActiveTimeline().getSavePointTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of savepoints in this table
|
||||
*/
|
||||
public List<String> getSavepoints() {
|
||||
return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of data file names savepointed
|
||||
*/
|
||||
public Stream<String> getSavepointedDataFiles(String savepointTime) {
|
||||
if (!getSavepoints().contains(savepointTime)) {
|
||||
throw new HoodieSavepointException(
|
||||
"Could not get data files for savepoint " + savepointTime + ". No such savepoint.");
|
||||
}
|
||||
HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION,
|
||||
savepointTime);
|
||||
HoodieSavepointMetadata metadata = null;
|
||||
try {
|
||||
metadata = AvroUtils
|
||||
.deserializeHoodieSavepointMetadata(getActiveTimeline().getInstantDetails(instant).get());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieSavepointException(
|
||||
"Could not get savepointed data files for savepoint " + savepointTime, e);
|
||||
}
|
||||
return metadata.getPartitionMetadata().values().stream()
|
||||
.flatMap(s -> s.getSavepointDataFile().stream());
|
||||
}
|
||||
|
||||
public HoodieActiveTimeline getActiveTimeline() {
|
||||
return metaClient.getActiveTimeline();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the index
|
||||
*/
|
||||
public HoodieIndex<T> getIndex() {
|
||||
return index;
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform the ultimate IO for a given upserted (RDD) partition
|
||||
*/
|
||||
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime,
|
||||
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
/**
|
||||
* Perform the ultimate IO for a given inserted (RDD) partition
|
||||
*/
|
||||
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime,
|
||||
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
/**
|
||||
* Schedule compaction for the instant time
|
||||
* @param jsc Spark Context
|
||||
* @param instantTime Instant Time for scheduling compaction
|
||||
* @return
|
||||
*/
|
||||
public abstract HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime);
|
||||
|
||||
/**
|
||||
* Run Compaction on the table. Compaction arranges the data so that it is optimized for data
|
||||
* access
|
||||
*
|
||||
* @param jsc Spark Context
|
||||
* @param compactionInstantTime Instant Time
|
||||
* @param compactionPlan Compaction Plan
|
||||
*/
|
||||
public abstract JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionInstantTime,
|
||||
HoodieCompactionPlan compactionPlan);
|
||||
|
||||
/**
|
||||
* Clean partition paths according to cleaning policy and returns the number of files cleaned.
|
||||
*/
|
||||
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc);
|
||||
|
||||
/**
|
||||
* Rollback the (inflight/committed) record changes with the given commit time. Four steps: (1)
|
||||
* Atomically unpublish this commit (2) clean indexing data (3) clean new generated parquet files
|
||||
* / log blocks (4) Finally, delete .<action>.commit or .<action>.inflight file if deleteInstants = true
|
||||
*/
|
||||
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
|
||||
throws IOException;
|
||||
|
||||
/**
|
||||
* Finalize the written data onto storage. Perform any final cleanups
|
||||
*
|
||||
* @param jsc Spark Context
|
||||
* @param stats List of HoodieWriteStats
|
||||
* @throws HoodieIOException if some paths can't be finalized on storage
|
||||
*/
|
||||
public void finalizeWrite(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats)
|
||||
throws HoodieIOException {
|
||||
cleanFailedWrites(jsc, instantTs, stats, config.getConsistencyGuardConfig().isConsistencyCheckEnabled());
|
||||
}
|
||||
|
||||
/**
|
||||
* Delete Marker directory corresponding to an instant
|
||||
* @param instantTs Instant Time
|
||||
*/
|
||||
protected void deleteMarkerDir(String instantTs) {
|
||||
try {
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
Path markerDir = new Path(metaClient.getMarkerFolderPath(instantTs));
|
||||
if (fs.exists(markerDir)) {
|
||||
// For append only case, we do not write to marker dir. Hence, the above check
|
||||
logger.info("Removing marker directory=" + markerDir);
|
||||
fs.delete(markerDir, true);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reconciles WriteStats and marker files to detect and safely delete duplicate data files created because of Spark
|
||||
* retries.
|
||||
*
|
||||
* @param jsc Spark Context
|
||||
* @param instantTs Instant Timestamp
|
||||
* @param stats Hoodie Write Stat
|
||||
* @param consistencyCheckEnabled Consistency Check Enabled
|
||||
* @throws HoodieIOException
|
||||
*/
|
||||
protected void cleanFailedWrites(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats,
|
||||
boolean consistencyCheckEnabled) throws HoodieIOException {
|
||||
try {
|
||||
// Reconcile marker and data files with WriteStats so that partially written data-files due to failed
|
||||
// (but succeeded on retry) tasks are removed.
|
||||
String basePath = getMetaClient().getBasePath();
|
||||
FileSystem fs = getMetaClient().getFs();
|
||||
Path markerDir = new Path(metaClient.getMarkerFolderPath(instantTs));
|
||||
|
||||
if (!fs.exists(markerDir)) {
|
||||
// Happens when all writes are appends
|
||||
return;
|
||||
}
|
||||
|
||||
List<String> invalidDataPaths = FSUtils.getAllDataFilesForMarkers(fs, basePath, instantTs, markerDir.toString());
|
||||
List<String> validDataPaths = stats.stream().map(w -> String.format("%s/%s", basePath, w.getPath()))
|
||||
.filter(p -> p.endsWith(".parquet")).collect(Collectors.toList());
|
||||
// Contains list of partially created files. These needs to be cleaned up.
|
||||
invalidDataPaths.removeAll(validDataPaths);
|
||||
if (!invalidDataPaths.isEmpty()) {
|
||||
logger.info("Removing duplicate data files created due to spark retries before committing. Paths="
|
||||
+ invalidDataPaths);
|
||||
}
|
||||
|
||||
Map<String, List<Pair<String, String>>> groupByPartition = invalidDataPaths.stream()
|
||||
.map(dp -> Pair.of(new Path(dp).getParent().toString(), dp))
|
||||
.collect(Collectors.groupingBy(Pair::getKey));
|
||||
|
||||
if (!groupByPartition.isEmpty()) {
|
||||
// Ensure all files in delete list is actually present. This is mandatory for an eventually consistent FS.
|
||||
// Otherwise, we may miss deleting such files. If files are not found even after retries, fail the commit
|
||||
if (consistencyCheckEnabled) {
|
||||
// This will either ensure all files to be deleted are present.
|
||||
waitForAllFiles(jsc, groupByPartition, FileVisibility.APPEAR);
|
||||
}
|
||||
|
||||
// Now delete partially written files
|
||||
jsc.parallelize(new ArrayList<>(groupByPartition.values()), config.getFinalizeWriteParallelism())
|
||||
.map(partitionWithFileList -> {
|
||||
final FileSystem fileSystem = metaClient.getFs();
|
||||
logger.info("Deleting invalid data files=" + partitionWithFileList);
|
||||
if (partitionWithFileList.isEmpty()) {
|
||||
return true;
|
||||
}
|
||||
// Delete
|
||||
partitionWithFileList.stream().map(Pair::getValue).forEach(file -> {
|
||||
try {
|
||||
fileSystem.delete(new Path(file), false);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
});
|
||||
|
||||
return true;
|
||||
}).collect();
|
||||
|
||||
// Now ensure the deleted files disappear
|
||||
if (consistencyCheckEnabled) {
|
||||
// This will either ensure all files to be deleted are absent.
|
||||
waitForAllFiles(jsc, groupByPartition, FileVisibility.DISAPPEAR);
|
||||
}
|
||||
}
|
||||
// Now delete the marker directory
|
||||
deleteMarkerDir(instantTs);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensures all files passed either appear or disappear
|
||||
* @param jsc JavaSparkContext
|
||||
* @param groupByPartition Files grouped by partition
|
||||
* @param visibility Appear/Disappear
|
||||
*/
|
||||
private void waitForAllFiles(JavaSparkContext jsc, Map<String, List<Pair<String, String>>> groupByPartition,
|
||||
FileVisibility visibility) {
|
||||
// This will either ensure all files to be deleted are present.
|
||||
boolean checkPassed =
|
||||
jsc.parallelize(new ArrayList<>(groupByPartition.entrySet()), config.getFinalizeWriteParallelism())
|
||||
.map(partitionWithFileList -> waitForCondition(partitionWithFileList.getKey(),
|
||||
partitionWithFileList.getValue().stream(), visibility))
|
||||
.collect().stream().allMatch(x -> x);
|
||||
if (!checkPassed) {
|
||||
throw new HoodieIOException("Consistency check failed to ensure all files " + visibility);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean waitForCondition(String partitionPath, Stream<Pair<String, String>> partitionFilePaths,
|
||||
FileVisibility visibility) {
|
||||
final FileSystem fileSystem = metaClient.getRawFs();
|
||||
List<String> fileList = partitionFilePaths.map(Pair::getValue).collect(Collectors.toList());
|
||||
try {
|
||||
getFailSafeConsistencyGuard(fileSystem).waitTill(partitionPath, fileList, visibility);
|
||||
} catch (IOException | TimeoutException ioe) {
|
||||
logger.error("Got exception while waiting for files to show up", ioe);
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private ConsistencyGuard getFailSafeConsistencyGuard(FileSystem fileSystem) {
|
||||
return new FailSafeConsistencyGuard(fileSystem, config.getConsistencyGuardConfig());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
/**
|
||||
* Repartition input records into at least expected number of output spark partitions. It should
|
||||
* give below guarantees - Output spark partition will have records from only one hoodie partition.
|
||||
* - Average records per output spark partitions should be almost equal to (#inputRecords /
|
||||
* #outputSparkPartitions) to avoid possible skews.
|
||||
*/
|
||||
public interface UserDefinedBulkInsertPartitioner<T extends HoodieRecordPayload> {
|
||||
|
||||
JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
int outputSparkPartitions);
|
||||
}
|
||||
@@ -0,0 +1,110 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import scala.Tuple2;
|
||||
|
||||
/**
|
||||
* Information about incoming records for upsert/insert obtained either via sampling or
|
||||
* introspecting the data fully
|
||||
* <p>
|
||||
* TODO(vc): Think about obtaining this directly from index.tagLocation
|
||||
*/
|
||||
public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
/**
|
||||
* Input workload
|
||||
*/
|
||||
private final JavaRDD<HoodieRecord<T>> taggedRecords;
|
||||
|
||||
/**
|
||||
* Computed workload profile
|
||||
*/
|
||||
private final HashMap<String, WorkloadStat> partitionPathStatMap;
|
||||
|
||||
|
||||
private final WorkloadStat globalStat;
|
||||
|
||||
|
||||
public WorkloadProfile(JavaRDD<HoodieRecord<T>> taggedRecords) {
|
||||
this.taggedRecords = taggedRecords;
|
||||
this.partitionPathStatMap = new HashMap<>();
|
||||
this.globalStat = new WorkloadStat();
|
||||
buildProfile();
|
||||
}
|
||||
|
||||
private void buildProfile() {
|
||||
|
||||
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = taggedRecords
|
||||
.mapToPair(record -> new Tuple2<>(
|
||||
new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())),
|
||||
record)).countByKey();
|
||||
|
||||
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts
|
||||
.entrySet()) {
|
||||
String partitionPath = e.getKey()._1();
|
||||
Long count = e.getValue();
|
||||
Option<HoodieRecordLocation> locOption = e.getKey()._2();
|
||||
|
||||
if (!partitionPathStatMap.containsKey(partitionPath)) {
|
||||
partitionPathStatMap.put(partitionPath, new WorkloadStat());
|
||||
}
|
||||
|
||||
if (locOption.isPresent()) {
|
||||
// update
|
||||
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
|
||||
globalStat.addUpdates(locOption.get(), count);
|
||||
} else {
|
||||
// insert
|
||||
partitionPathStatMap.get(partitionPath).addInserts(count);
|
||||
globalStat.addInserts(count);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public WorkloadStat getGlobalStat() {
|
||||
return globalStat;
|
||||
}
|
||||
|
||||
public Set<String> getPartitionPaths() {
|
||||
return partitionPathStatMap.keySet();
|
||||
}
|
||||
|
||||
public WorkloadStat getWorkloadStat(String partitionPath) {
|
||||
return partitionPathStatMap.get(partitionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("WorkloadProfile {");
|
||||
sb.append("globalStat=").append(globalStat).append(", ");
|
||||
sb.append("partitionStat=").append(partitionPathStatMap);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
/**
|
||||
* Wraps stats about a single partition path.
|
||||
*/
|
||||
public class WorkloadStat implements Serializable {
|
||||
|
||||
private long numInserts = 0L;
|
||||
|
||||
private long numUpdates = 0L;
|
||||
|
||||
private HashMap<String, Pair<String, Long>> updateLocationToCount;
|
||||
|
||||
public WorkloadStat() {
|
||||
updateLocationToCount = new HashMap<>();
|
||||
}
|
||||
|
||||
long addInserts(long numInserts) {
|
||||
return this.numInserts += numInserts;
|
||||
}
|
||||
|
||||
long addUpdates(HoodieRecordLocation location, long numUpdates) {
|
||||
updateLocationToCount.put(location.getFileId(), Pair.of(location.getInstantTime(), numUpdates));
|
||||
return this.numUpdates += numUpdates;
|
||||
}
|
||||
|
||||
public long getNumUpdates() {
|
||||
return numUpdates;
|
||||
}
|
||||
|
||||
public long getNumInserts() {
|
||||
return numInserts;
|
||||
}
|
||||
|
||||
public HashMap<String, Pair<String, Long>> getUpdateLocationToCount() {
|
||||
return updateLocationToCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("WorkloadStat {");
|
||||
sb.append("numInserts=").append(numInserts).append(", ");
|
||||
sb.append("numUpdates=").append(numUpdates);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user