1
0

Adding ability for inserts to be written to log files

This commit is contained in:
Nishith Agarwal
2018-05-13 16:25:11 -07:00
committed by vinoth chandar
parent 34827d50e1
commit 3da063f83b
52 changed files with 1061 additions and 519 deletions

View File

@@ -40,10 +40,11 @@ import com.uber.hoodie.exception.HoodieException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.HoodieNotSupportedException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.LazyInsertIterable;
import com.uber.hoodie.func.CopyOnWriteLazyInsertIterable;
import com.uber.hoodie.func.ParquetReaderIterator;
import com.uber.hoodie.func.SparkBoundedInMemoryExecutor;
import com.uber.hoodie.io.HoodieCleanHelper;
import com.uber.hoodie.io.HoodieCreateHandle;
import com.uber.hoodie.io.HoodieMergeHandle;
import java.io.IOException;
import java.io.Serializable;
@@ -64,6 +65,7 @@ import org.apache.commons.lang3.tuple.Pair;
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.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.parquet.avro.AvroParquetReader;
@@ -90,8 +92,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
private static Logger logger = LogManager.getLogger(HoodieCopyOnWriteTable.class);
public HoodieCopyOnWriteTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
super(config, metaClient);
public HoodieCopyOnWriteTable(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
}
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String,
@@ -225,7 +227,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
public Iterator<List<WriteStatus>> handleInsert(String commitTime,
Iterator<HoodieRecord<T>> recordItr) throws Exception {
return new LazyInsertIterable<>(recordItr, config, commitTime, this);
return new CopyOnWriteLazyInsertIterable<>(recordItr, config, commitTime, this);
}
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")
@@ -289,17 +299,29 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
*/
protected Map<FileStatus, Boolean> deleteCleanedFiles(String partitionPath, List<String> commits)
throws IOException {
Map<FileStatus, Boolean> results = Maps.newHashMap();
// 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 commits.contains(fileCommitTime);
}
return false;
};
deleteCleanedFiles(results, partitionPath, filter);
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 partitionPath,
PathFilter filter)
throws IOException {
logger.info("Cleaning path " + partitionPath);
FileSystem fs = getMetaClient().getFs();
FileStatus[] toBeDeleted = fs
.listStatus(new Path(config.getBasePath(), partitionPath), path -> {
if (!path.toString().contains(".parquet")) {
return false;
}
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commits.contains(fileCommitTime);
});
Map<FileStatus, Boolean> results = Maps.newHashMap();
FileStatus[] toBeDeleted = fs.listStatus(new Path(config.getBasePath(), partitionPath), filter);
for (FileStatus file : toBeDeleted) {
boolean success = fs.delete(file.getPath(), false);
results.put(file, success);
@@ -311,7 +333,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
@Override
public List<HoodieRollbackStat> rollback(JavaSparkContext jsc, List<String> commits)
throws IOException {
String actionType = this.getCommitActionType();
String actionType = metaClient.getCommitActionType();
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
List<String> inflights = this.getInflightCommitTimeline().getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());

View File

@@ -27,11 +27,12 @@ import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock.HoodieCommandBlockTypeEnum;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.util.FSUtils;
@@ -39,6 +40,7 @@ import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCompactionException;
import com.uber.hoodie.exception.HoodieRollbackException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.func.MergeOnReadLazyInsertIterable;
import com.uber.hoodie.index.HoodieIndex;
import com.uber.hoodie.io.HoodieAppendHandle;
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
@@ -55,6 +57,7 @@ import java.util.Optional;
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.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.Partitioner;
@@ -78,8 +81,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
// UpsertPartitioner for MergeOnRead table type
private MergeOnReadUpsertPartitioner mergeOnReadUpsertPartitioner;
public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
super(config, metaClient);
public HoodieMergeOnReadTable(HoodieWriteConfig config, JavaSparkContext jsc) {
super(config, jsc);
}
@Override
@@ -110,6 +113,17 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
}
}
@Override
public Iterator<List<WriteStatus>> handleInsert(String commitTime,
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);
} else {
return super.handleInsert(commitTime, recordItr);
}
}
@Override
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionCommitTime) {
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
@@ -166,6 +180,20 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
return commits.stream().map(commit -> {
HoodieInstant instant = commitsAndCompactions.get(commit);
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 commits.contains(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 commits.contains(fileCommitTime);
}
return false;
};
switch (instant.getAction()) {
case HoodieTimeline.COMMIT_ACTION:
try {
@@ -180,18 +208,18 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
case HoodieTimeline.DELTA_COMMIT_ACTION:
try {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
this.getCommitTimeline().getInstantDetails(
metaClient.getCommitTimeline().getInstantDetails(
new HoodieInstant(true, instant.getAction(), instant.getTimestamp()))
.get());
// read commit file and (either append delete blocks or delete file)
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
final 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));
// In case all data was inserts and the commit failed, there is no partition stats
if (commitMetadata.getPartitionToWriteStats().size() == 0) {
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
}
// append rollback blocks for updates
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
@@ -201,15 +229,31 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseCommitTime)) : null;
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
.filter(wStat -> {
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT
&& wStat.getPrevCommit() != null;
}).forEach(wStat -> {
HoodieLogFormat.Writer writer = null;
if (wStat != null
&& wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT
&& wStat.getPrevCommit() != null) {
return true;
}
// 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)
try {
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
} catch (IOException io) {
throw new UncheckedIOException(io);
}
return false;
})
.forEach(wStat -> {
Writer writer = null;
String baseCommitTime = wStat.getPrevCommit();
if (hoodieIndex.isGlobal()) {
baseCommitTime = fileIdToLatestCommitTimeMap.get(wStat.getFileId());
}
try {
// TODO : wStat.getPrevCommit() might not give the right commit time in the following
// scenario if a compaction was scheduled, the new commitTime will be used to write the
// new log file. In this case, the commit time for the log file is the
// getBaseCommitTime()
writer = HoodieLogFormat.newWriterBuilder().onParentPath(
new Path(this.getMetaClient().getBasePath(), partitionPath))
.withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime)
@@ -217,15 +261,15 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
Long numRollbackBlocks = 0L;
// generate metadata
Map<HoodieLogBlock.HeaderMetadataType, String> header =
Map<HeaderMetadataType, String> header =
Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME,
header.put(HeaderMetadataType.INSTANT_TIME,
metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME,
header.put(HeaderMetadataType.TARGET_INSTANT_TIME,
commit);
header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String
.valueOf(
HoodieCommandBlock.HoodieCommandBlockTypeEnum
HoodieCommandBlockTypeEnum
.ROLLBACK_PREVIOUS_BLOCK
.ordinal()));
// if update belongs to an existing log file
@@ -263,7 +307,6 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
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));
@@ -298,6 +341,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
if (!commitTimeline.empty()) {
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
// find smallest file in partition and append to it
// TODO - check if index.isglobal then small files are log files too
Optional<FileSlice> smallFileSlice = getRTFileSystemView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).filter(
fileSlice -> fileSlice.getLogFiles().count() < 1
@@ -327,4 +372,5 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.collect(Collectors.toList());
}
}
}

View File

@@ -31,9 +31,9 @@ import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
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.HoodieSavepointException;
import com.uber.hoodie.index.HoodieIndex;
import java.io.IOException;
import java.io.Serializable;
import java.util.Iterator;
@@ -54,19 +54,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
protected final HoodieWriteConfig config;
protected final HoodieTableMetaClient metaClient;
protected final HoodieIndex<T> index;
protected HoodieTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc) {
this.config = config;
this.metaClient = metaClient;
this.metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true);
this.index = HoodieIndex.createIndex(config, jsc);
}
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return new HoodieCopyOnWriteTable<>(config, metaClient);
return new HoodieCopyOnWriteTable<>(config, jsc);
case MERGE_ON_READ:
return new HoodieMergeOnReadTable<>(config, metaClient);
return new HoodieMergeOnReadTable<>(config, jsc);
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
@@ -124,21 +126,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
* Get the completed (commit + compaction) view of the file system for this table
*/
public TableFileSystemView getCompletedFileSystemView() {
return new HoodieTableFileSystemView(metaClient, getCommitsTimeline());
return new HoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline());
}
/**
* Get only the completed (no-inflights) commit timeline
*/
public HoodieTimeline getCompletedCommitTimeline() {
return getCommitsTimeline().filterCompletedInstants();
return metaClient.getCommitsTimeline().filterCompletedInstants();
}
/**
* Get only the inflights (no-completed) commit timeline
*/
public HoodieTimeline getInflightCommitTimeline() {
return getCommitsTimeline().filterInflights();
return metaClient.getCommitsTimeline().filterInflights();
}
/**
@@ -190,49 +192,10 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
}
/**
* Get the commit timeline visible for this table
* Return the index
*/
public HoodieTimeline getCommitsTimeline() {
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().getCommitsTimeline();
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
/**
* Get the compacted commit timeline visible for this table
*/
public HoodieTimeline getCommitTimeline() {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
case MERGE_ON_READ:
// We need to include the parquet files written out in delta commits in tagging
return getActiveTimeline().getCommitTimeline();
default:
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
}
}
/**
* 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;
default:
throw new HoodieCommitException(
"Could not commit on unknown storage type " + metaClient.getTableType());
}
public HoodieIndex<T> getIndex() {
return index;
}
/**