1
0

Reformatting code per Google Code Style all over

This commit is contained in:
Vinoth Chandar
2017-11-12 22:54:56 -08:00
committed by vinoth chandar
parent 5a62480a92
commit e45679f5e2
254 changed files with 21580 additions and 21108 deletions

View File

@@ -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;
}
}