Reformatting code per Google Code Style all over
This commit is contained in:
committed by
vinoth chandar
parent
5a62480a92
commit
e45679f5e2
File diff suppressed because it is too large
Load Diff
@@ -39,13 +39,6 @@ import com.uber.hoodie.exception.HoodieCompactionException;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.io.HoodieAppendHandle;
|
||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.Arrays;
|
||||
@@ -56,179 +49,209 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
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
|
||||
*
|
||||
* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or)
|
||||
* Merge with the smallest existing file, to expand it
|
||||
* Merge with the smallest existing file, to expand it
|
||||
*
|
||||
* UPDATES - Appends the changes to a rolling log file maintained per file Id.
|
||||
* Compaction merges the log file into the base file.
|
||||
* UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the
|
||||
* log file into the base file.
|
||||
*
|
||||
* WARNING - MOR table type does not support nested rollbacks, every rollback
|
||||
* must be followed by an attempted commit action
|
||||
* WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an
|
||||
* attempted commit action
|
||||
*/
|
||||
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieCopyOnWriteTable<T> {
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
||||
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
HoodieCopyOnWriteTable<T> {
|
||||
|
||||
public HoodieMergeOnReadTable(HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
super(config, metaClient);
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
||||
|
||||
public HoodieMergeOnReadTable(HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
super(config, metaClient);
|
||||
}
|
||||
|
||||
@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);
|
||||
HoodieAppendHandle<T> appendHandle =
|
||||
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
|
||||
appendHandle.doAppend();
|
||||
appendHandle.close();
|
||||
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
|
||||
.iterator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc) {
|
||||
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||
Optional<HoodieInstant> lastCompaction = getActiveTimeline().getCompactionTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
String deltaCommitsSinceTs = "0";
|
||||
if (lastCompaction.isPresent()) {
|
||||
deltaCommitsSinceTs = lastCompaction.get().getTimestamp();
|
||||
}
|
||||
|
||||
@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);
|
||||
HoodieAppendHandle<T> appendHandle =
|
||||
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
|
||||
appendHandle.doAppend();
|
||||
appendHandle.close();
|
||||
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
|
||||
.iterator();
|
||||
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 Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc) {
|
||||
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
|
||||
Optional<HoodieInstant> lastCompaction = getActiveTimeline().getCompactionTimeline()
|
||||
.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 Optional.empty();
|
||||
}
|
||||
|
||||
logger.info("Compacting merge on read table " + config.getBasePath());
|
||||
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
|
||||
try {
|
||||
return Optional.of(compactor.compact(jsc, config, this));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
|
||||
}
|
||||
logger.info("Compacting merge on read table " + config.getBasePath());
|
||||
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
|
||||
try {
|
||||
return Optional.of(compactor.compact(jsc, config, this));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieCompactionException("Could not compact " + config.getBasePath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits) throws IOException {
|
||||
@Override
|
||||
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
|
||||
throws IOException {
|
||||
|
||||
//At the moment, MOR table type does not support nested rollbacks
|
||||
if(commits.size() > 1) {
|
||||
throw new UnsupportedOperationException("Nested Rollbacks are not supported");
|
||||
}
|
||||
Map<String, HoodieInstant> commitsAndCompactions =
|
||||
this.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
||||
.getInstants()
|
||||
.filter(i -> commits.contains(i.getTimestamp()))
|
||||
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
||||
|
||||
// Atomically un-publish all non-inflight commits
|
||||
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
|
||||
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
|
||||
|
||||
logger.info("Unpublished " + commits);
|
||||
|
||||
Long startTime = System.currentTimeMillis();
|
||||
|
||||
List<HoodieRollbackStat> allRollbackStats = commits.stream().map(commit -> {
|
||||
HoodieInstant instant = commitsAndCompactions.get(commit);
|
||||
List<HoodieRollbackStat> stats = null;
|
||||
switch (instant.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
try {
|
||||
logger.info("Starting to rollback Commit/Compaction " + instant);
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream().collect(Collectors.toList()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
Map<FileStatus, Boolean> results = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(results).build();
|
||||
}).collect();
|
||||
logger.info("Finished rollback of Commit/Compaction " + instant);
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
try {
|
||||
logger.info("Starting to rollback delta commit " + instant);
|
||||
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream().collect(Collectors.toList()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// read commit file and (either append delete blocks or delete file)
|
||||
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
|
||||
|
||||
// we do not know fileIds for inserts (first inserts are parquet files), delete all parquet files for the corresponding failed commit, if present (same as COW)
|
||||
filesToDeletedStatus = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
|
||||
// append rollback blocks for updates
|
||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream().filter(wStat -> wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT).forEach(wStat -> {
|
||||
HoodieLogFormat.Writer writer = null;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(this.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
|
||||
.withFs(FSUtils.getFs()).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
Long numRollbackBlocks = 0L;
|
||||
// generate metadata
|
||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit);
|
||||
// if update belongs to an existing log file
|
||||
writer.appendBlock(new HoodieCommandBlock(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata));
|
||||
numRollbackBlocks++;
|
||||
if(wStat.getNumDeletes() > 0) {
|
||||
writer.appendBlock(new HoodieCommandBlock(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK, metadata));
|
||||
numRollbackBlocks++;
|
||||
}
|
||||
filesToNumBlocksRollback.put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()), numRollbackBlocks);
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException("Failed to rollback for commit " + commit, io);
|
||||
} finally {
|
||||
try {
|
||||
writer.close();
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
}
|
||||
});
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(filesToDeletedStatus)
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
|
||||
}).collect();
|
||||
logger.info("Fnished rollback of delta commit " + instant);
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
}
|
||||
return stats;
|
||||
}).flatMap(x -> x.stream()).collect(Collectors.toList());
|
||||
|
||||
commitsAndCompactions.entrySet().stream()
|
||||
.map(entry -> new HoodieInstant(true, entry.getValue().getAction(), entry.getValue().getTimestamp()))
|
||||
.forEach(this.getActiveTimeline()::deleteInflight);
|
||||
|
||||
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
|
||||
return allRollbackStats;
|
||||
//At the moment, MOR table type does not support nested rollbacks
|
||||
if (commits.size() > 1) {
|
||||
throw new UnsupportedOperationException("Nested Rollbacks are not supported");
|
||||
}
|
||||
Map<String, HoodieInstant> commitsAndCompactions =
|
||||
this.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.COMPACTION_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
||||
.getInstants()
|
||||
.filter(i -> commits.contains(i.getTimestamp()))
|
||||
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
||||
|
||||
// Atomically un-publish all non-inflight commits
|
||||
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
|
||||
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
|
||||
|
||||
logger.info("Unpublished " + commits);
|
||||
|
||||
Long startTime = System.currentTimeMillis();
|
||||
|
||||
List<HoodieRollbackStat> allRollbackStats = commits.stream().map(commit -> {
|
||||
HoodieInstant instant = commitsAndCompactions.get(commit);
|
||||
List<HoodieRollbackStat> stats = null;
|
||||
switch (instant.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
try {
|
||||
logger.info("Starting to rollback Commit/Compaction " + instant);
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitTimeline().getInstantDetails(
|
||||
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream()
|
||||
.collect(Collectors.toList()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
Map<FileStatus, Boolean> results = super
|
||||
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(results).build();
|
||||
}).collect();
|
||||
logger.info("Finished rollback of Commit/Compaction " + instant);
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
try {
|
||||
logger.info("Starting to rollback delta commit " + instant);
|
||||
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitTimeline().getInstantDetails(
|
||||
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream()
|
||||
.collect(Collectors.toList()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// read commit file and (either append delete blocks or delete file)
|
||||
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
|
||||
|
||||
// we do not know fileIds for inserts (first inserts are parquet files), delete all parquet files for the corresponding failed commit, if present (same as COW)
|
||||
filesToDeletedStatus = super
|
||||
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
|
||||
// append rollback blocks for updates
|
||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
||||
.filter(wStat -> wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT)
|
||||
.forEach(wStat -> {
|
||||
HoodieLogFormat.Writer writer = null;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(
|
||||
new Path(this.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
|
||||
.withFs(FSUtils.getFs())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
Long numRollbackBlocks = 0L;
|
||||
// generate metadata
|
||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME,
|
||||
metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit);
|
||||
// if update belongs to an existing log file
|
||||
writer.appendBlock(new HoodieCommandBlock(
|
||||
HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK,
|
||||
metadata));
|
||||
numRollbackBlocks++;
|
||||
if (wStat.getNumDeletes() > 0) {
|
||||
writer.appendBlock(new HoodieCommandBlock(
|
||||
HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK,
|
||||
metadata));
|
||||
numRollbackBlocks++;
|
||||
}
|
||||
filesToNumBlocksRollback
|
||||
.put(FSUtils.getFs().getFileStatus(writer.getLogFile().getPath()),
|
||||
numRollbackBlocks);
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException(
|
||||
"Failed to rollback for commit " + commit, io);
|
||||
} finally {
|
||||
try {
|
||||
writer.close();
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
}
|
||||
});
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(filesToDeletedStatus)
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
|
||||
}).collect();
|
||||
logger.info("Fnished rollback of delta commit " + instant);
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
}
|
||||
return stats;
|
||||
}).flatMap(x -> x.stream()).collect(Collectors.toList());
|
||||
|
||||
commitsAndCompactions.entrySet().stream()
|
||||
.map(entry -> new HoodieInstant(true, entry.getValue().getAction(),
|
||||
entry.getValue().getTimestamp()))
|
||||
.forEach(this.getActiveTimeline()::deleteInflight);
|
||||
|
||||
logger
|
||||
.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
|
||||
return allRollbackStats;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -34,7 +34,6 @@ import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCommitException;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.exception.HoodieSavepointException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
@@ -43,8 +42,6 @@ import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
@@ -55,291 +52,245 @@ import org.apache.spark.api.java.JavaSparkContext;
|
||||
* Abstract implementation of a HoodieTable
|
||||
*/
|
||||
public abstract class HoodieTable<T extends HoodieRecordPayload> implements Serializable {
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
private static Logger logger = LogManager.getLogger(HoodieTable.class);
|
||||
|
||||
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
this.config = config;
|
||||
this.metaClient = metaClient;
|
||||
protected final HoodieWriteConfig config;
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
private static Logger logger = LogManager.getLogger(HoodieTable.class);
|
||||
|
||||
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
this.config = config;
|
||||
this.metaClient = metaClient;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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 FileSystem getFs() {
|
||||
return metaClient.getFs();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView getFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the read optimized view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the real time view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView.RealtimeView getRTFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the completed (commit + compaction) view of the file system for this table
|
||||
*/
|
||||
public TableFileSystemView getCompletedFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCommitTimeline());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) commit timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedCommitTimeline() {
|
||||
return getCommitTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the inflights (no-completed) commit timeline
|
||||
*/
|
||||
public HoodieTimeline getInflightCommitTimeline() {
|
||||
return getCommitTimeline().filterInflights();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* 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.");
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a partitioner to perform the upsert operation, based on the
|
||||
* workload profile
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public abstract Partitioner getUpsertPartitioner(WorkloadProfile profile);
|
||||
|
||||
|
||||
/**
|
||||
* Provides a partitioner to perform the insert operation, based on the workload profile
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public abstract Partitioner getInsertPartitioner(WorkloadProfile profile);
|
||||
|
||||
|
||||
/**
|
||||
* Return whether this HoodieTable implementation can benefit from workload
|
||||
* profiling
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public abstract boolean isWorkloadProfileNeeded();
|
||||
|
||||
public HoodieWriteConfig getConfig() {
|
||||
return config;
|
||||
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 HoodieTableMetaClient getMetaClient() {
|
||||
return metaClient;
|
||||
public HoodieActiveTimeline getActiveTimeline() {
|
||||
return metaClient.getActiveTimeline();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the commit timeline visible for this table
|
||||
*/
|
||||
public HoodieTimeline getCommitTimeline() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return getActiveTimeline().getCommitTimeline();
|
||||
case MERGE_ON_READ:
|
||||
// We need to include the parquet files written out in delta commits
|
||||
// Include commit action to be able to start doing a MOR over a COW dataset - no migration required
|
||||
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
public FileSystem getFs() {
|
||||
return metaClient.getFs();
|
||||
/**
|
||||
* Get only the completed (no-inflights) compaction commit timeline
|
||||
*/
|
||||
public HoodieTimeline getCompletedCompactionCommitTimeline() {
|
||||
return getCompactionCommitTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the compacted commit timeline visible for this table
|
||||
*/
|
||||
public HoodieTimeline getCompactionCommitTimeline() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||
case MERGE_ON_READ:
|
||||
// We need to include the parquet files written out in delta commits in tagging
|
||||
return getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION));
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the view of the file system for this table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public TableFileSystemView getFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
||||
/**
|
||||
* Gets the commit action type
|
||||
*/
|
||||
public String getCommitActionType() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return HoodieActiveTimeline.COMMIT_ACTION;
|
||||
case MERGE_ON_READ:
|
||||
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
|
||||
}
|
||||
throw new HoodieCommitException(
|
||||
"Could not commit on unknown storage type " + metaClient.getTableType());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the read optimized view of the file system for this table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public TableFileSystemView.ReadOptimizedView getROFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
||||
/**
|
||||
* Gets the action type for a compaction commit
|
||||
*/
|
||||
public String getCompactedCommitActionType() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return HoodieTimeline.COMMIT_ACTION;
|
||||
case MERGE_ON_READ:
|
||||
return HoodieTimeline.COMPACTION_ACTION;
|
||||
}
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the real time view of the file system for this table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public TableFileSystemView.RealtimeView getRTFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCompletedCommitTimeline());
|
||||
|
||||
/**
|
||||
* 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);
|
||||
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
||||
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return new HoodieCopyOnWriteTable<>(config, metaClient);
|
||||
case MERGE_ON_READ:
|
||||
return new HoodieMergeOnReadTable<>(config, metaClient);
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the completed (commit + compaction) view of the file system for this table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public TableFileSystemView getCompletedFileSystemView() {
|
||||
return new HoodieTableFileSystemView(metaClient, getCommitTimeline());
|
||||
}
|
||||
/**
|
||||
* Run Compaction on the table. Compaction arranges the data so that it is optimized for data
|
||||
* access
|
||||
*/
|
||||
public abstract Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc);
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) commit timeline
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCompletedCommitTimeline() {
|
||||
return getCommitTimeline().filterCompletedInstants();
|
||||
}
|
||||
/**
|
||||
* Clean partition paths according to cleaning policy and returns the number of files cleaned.
|
||||
*/
|
||||
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc);
|
||||
|
||||
/**
|
||||
* Get only the inflights (no-completed) commit timeline
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getInflightCommitTimeline() {
|
||||
return getCommitTimeline().filterInflights();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) clean timeline
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCompletedCleanTimeline() {
|
||||
return getActiveTimeline().getCleanerTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) savepoint timeline
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCompletedSavepointTimeline() {
|
||||
return getActiveTimeline().getSavePointTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of savepoints in this table
|
||||
* @return
|
||||
*/
|
||||
public List<String> getSavepoints() {
|
||||
return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of data file names savepointed
|
||||
*
|
||||
* @param savepointTime
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
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();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the commit timeline visible for this table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCommitTimeline() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return getActiveTimeline().getCommitTimeline();
|
||||
case MERGE_ON_READ:
|
||||
// We need to include the parquet files written out in delta commits
|
||||
// Include commit action to be able to start doing a MOR over a COW dataset - no migration required
|
||||
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the completed (no-inflights) compaction commit timeline
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCompletedCompactionCommitTimeline() {
|
||||
return getCompactionCommitTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the compacted commit timeline visible for this table
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCompactionCommitTimeline() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return getActiveTimeline().getCommitsAndCompactionsTimeline();
|
||||
case MERGE_ON_READ:
|
||||
// We need to include the parquet files written out in delta commits in tagging
|
||||
return getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieActiveTimeline.COMPACTION_ACTION));
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the commit action type
|
||||
* @return
|
||||
*/
|
||||
public String getCommitActionType() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return HoodieActiveTimeline.COMMIT_ACTION;
|
||||
case MERGE_ON_READ:
|
||||
return HoodieActiveTimeline.DELTA_COMMIT_ACTION;
|
||||
}
|
||||
throw new HoodieCommitException(
|
||||
"Could not commit on unknown storage type " + metaClient.getTableType());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the action type for a compaction commit
|
||||
* @return
|
||||
*/
|
||||
public String getCompactedCommitActionType() {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return HoodieTimeline.COMMIT_ACTION;
|
||||
case MERGE_ON_READ:
|
||||
return HoodieTimeline.COMPACTION_ACTION;
|
||||
}
|
||||
throw new HoodieException("Unsupported table type :"+ metaClient.getTableType());
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Perform the ultimate IO for a given upserted (RDD) partition
|
||||
*
|
||||
* @param partition
|
||||
* @param recordIterator
|
||||
* @param partitioner
|
||||
*/
|
||||
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
|
||||
*
|
||||
* @param partition
|
||||
* @param recordIterator
|
||||
* @param partitioner
|
||||
*/
|
||||
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime,
|
||||
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
|
||||
|
||||
|
||||
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
|
||||
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
switch (metaClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
return new HoodieCopyOnWriteTable<>(config, metaClient);
|
||||
case MERGE_ON_READ:
|
||||
return new HoodieMergeOnReadTable<>(config, metaClient);
|
||||
default:
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Run Compaction on the table.
|
||||
* Compaction arranges the data so that it is optimized for data access
|
||||
*/
|
||||
public abstract Optional<HoodieCompactionMetadata> compact(JavaSparkContext jsc);
|
||||
|
||||
/**
|
||||
* 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
|
||||
* @param commits
|
||||
* @return
|
||||
* @throws HoodieRollbackException
|
||||
*/
|
||||
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits) throws IOException;
|
||||
/**
|
||||
* 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
|
||||
*/
|
||||
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
|
||||
throws IOException;
|
||||
}
|
||||
|
||||
@@ -20,13 +20,13 @@ import com.uber.hoodie.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.
|
||||
* 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);
|
||||
JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
|
||||
int outputSparkPartitions);
|
||||
}
|
||||
|
||||
@@ -20,15 +20,11 @@ package com.uber.hoodie.table;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.function.PairFunction;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import scala.Option;
|
||||
import scala.Tuple2;
|
||||
|
||||
@@ -40,73 +36,76 @@ import scala.Tuple2;
|
||||
*/
|
||||
public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializable {
|
||||
|
||||
/**
|
||||
* Input workload
|
||||
*/
|
||||
private final JavaRDD<HoodieRecord<T>> taggedRecords;
|
||||
/**
|
||||
* Input workload
|
||||
*/
|
||||
private final JavaRDD<HoodieRecord<T>> taggedRecords;
|
||||
|
||||
/**
|
||||
* Computed workload profile
|
||||
*/
|
||||
private final HashMap<String, WorkloadStat> partitionPathStatMap;
|
||||
/**
|
||||
* Computed workload profile
|
||||
*/
|
||||
private final HashMap<String, WorkloadStat> partitionPathStatMap;
|
||||
|
||||
|
||||
private final WorkloadStat globalStat;
|
||||
private final WorkloadStat globalStat;
|
||||
|
||||
|
||||
public WorkloadProfile(JavaRDD<HoodieRecord<T>> taggedRecords) {
|
||||
this.taggedRecords = taggedRecords;
|
||||
this.partitionPathStatMap = new HashMap<>();
|
||||
this.globalStat = new WorkloadStat();
|
||||
buildProfile();
|
||||
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.apply(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.isDefined()) {
|
||||
// update
|
||||
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
|
||||
globalStat.addUpdates(locOption.get(), count);
|
||||
} else {
|
||||
// insert
|
||||
partitionPathStatMap.get(partitionPath).addInserts(count);
|
||||
globalStat.addInserts(count);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void buildProfile() {
|
||||
public WorkloadStat getGlobalStat() {
|
||||
return globalStat;
|
||||
}
|
||||
|
||||
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = taggedRecords
|
||||
.mapToPair(record ->
|
||||
new Tuple2<>(new Tuple2<>(record.getPartitionPath(), Option.apply(record.getCurrentLocation())), record))
|
||||
.countByKey();
|
||||
public Set<String> getPartitionPaths() {
|
||||
return partitionPathStatMap.keySet();
|
||||
}
|
||||
|
||||
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();
|
||||
public WorkloadStat getWorkloadStat(String partitionPath) {
|
||||
return partitionPathStatMap.get(partitionPath);
|
||||
}
|
||||
|
||||
if (!partitionPathStatMap.containsKey(partitionPath)){
|
||||
partitionPathStatMap.put(partitionPath, new WorkloadStat());
|
||||
}
|
||||
|
||||
if (locOption.isDefined()) {
|
||||
// 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();
|
||||
}
|
||||
@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();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -17,7 +17,6 @@
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
|
||||
@@ -25,43 +24,44 @@ import java.util.HashMap;
|
||||
* Wraps stats about a single partition path.
|
||||
*/
|
||||
public class WorkloadStat implements Serializable {
|
||||
private long numInserts = 0L;
|
||||
|
||||
private long numUpdates = 0L;
|
||||
private long numInserts = 0L;
|
||||
|
||||
private HashMap<String, Long> updateLocationToCount;
|
||||
private long numUpdates = 0L;
|
||||
|
||||
public WorkloadStat() {
|
||||
updateLocationToCount = new HashMap<>();
|
||||
}
|
||||
private HashMap<String, Long> updateLocationToCount;
|
||||
|
||||
long addInserts(long numInserts) {
|
||||
return this.numInserts += numInserts;
|
||||
}
|
||||
public WorkloadStat() {
|
||||
updateLocationToCount = new HashMap<>();
|
||||
}
|
||||
|
||||
long addUpdates(HoodieRecordLocation location, long numUpdates) {
|
||||
updateLocationToCount.put(location.getFileId(), numUpdates);
|
||||
return this.numUpdates += numUpdates;
|
||||
}
|
||||
long addInserts(long numInserts) {
|
||||
return this.numInserts += numInserts;
|
||||
}
|
||||
|
||||
public long getNumUpdates() {
|
||||
return numUpdates;
|
||||
}
|
||||
long addUpdates(HoodieRecordLocation location, long numUpdates) {
|
||||
updateLocationToCount.put(location.getFileId(), numUpdates);
|
||||
return this.numUpdates += numUpdates;
|
||||
}
|
||||
|
||||
public long getNumInserts() {
|
||||
return numInserts;
|
||||
}
|
||||
public long getNumUpdates() {
|
||||
return numUpdates;
|
||||
}
|
||||
|
||||
public HashMap<String, Long> getUpdateLocationToCount() {
|
||||
return updateLocationToCount;
|
||||
}
|
||||
public long getNumInserts() {
|
||||
return numInserts;
|
||||
}
|
||||
|
||||
@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();
|
||||
}
|
||||
public HashMap<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