1
0

Implement Merge on Read Storage (#76)

1. Create HoodieTable abstraction for commits and fileSystemView
2. HoodieMergeOnReadTable created
3. View is now always obtained from the table and the correct view based on the table type is returned
This commit is contained in:
prazanna
2017-02-21 15:24:00 -08:00
committed by Prasanna Rajaperumal
parent 11d2fd3428
commit eb46e7c72b
47 changed files with 1113 additions and 421 deletions

View File

@@ -0,0 +1,163 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed 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 com.uber.hoodie.io;
import com.clearspring.analytics.util.Lists;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieDeltaWriteStat;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.log.HoodieLogAppendConfig;
import com.uber.hoodie.common.table.log.HoodieLogFile;
import com.uber.hoodie.common.table.log.avro.RollingAvroLogAppender;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieAppendException;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
private static Logger logger = LogManager.getLogger(HoodieUpdateHandle.class);
private static AtomicLong recordIndex = new AtomicLong(1);
private final WriteStatus writeStatus;
private final String fileId;
private String partitionPath;
private RollingAvroLogAppender logAppender;
private List<HoodieRecord<T>> records;
private long recordsWritten = 0;
private HoodieLogFile currentLogFile;
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime,
HoodieTable<T> hoodieTable, String fileId, Iterator<HoodieRecord<T>> recordItr) {
super(config, commitTime, hoodieTable);
WriteStatus writeStatus = new WriteStatus();
writeStatus.setStat(new HoodieDeltaWriteStat());
this.writeStatus = writeStatus;
this.fileId = fileId;
init(recordItr);
}
private void init(Iterator<HoodieRecord<T>> recordItr) {
List<HoodieRecord<T>> records = Lists.newArrayList();
recordItr.forEachRemaining(record -> {
records.add(record);
// extract some information from the first record
if (partitionPath == null) {
partitionPath = record.getPartitionPath();
String latestValidFilePath =
fileSystemView.getLatestDataFilesForFileId(record.getPartitionPath(), fileId)
.findFirst().get().getFileName();
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
writeStatus.getStat().setPrevCommit(baseCommitTime);
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(record.getPartitionPath());
writeStatus.getStat().setFileId(fileId);
try {
HoodieLogAppendConfig logConfig = HoodieLogAppendConfig.newBuilder()
.onPartitionPath(
new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
.withFileId(fileId).withBaseCommitTime(baseCommitTime).withSchema(schema)
.withFs(fs).withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
this.logAppender = new RollingAvroLogAppender(logConfig);
this.currentLogFile = logAppender.getConfig().getLogFile();
((HoodieDeltaWriteStat) writeStatus.getStat())
.setLogVersion(currentLogFile.getLogVersion());
((HoodieDeltaWriteStat) writeStatus.getStat())
.setLogOffset(logAppender.getCurrentSize());
} catch (Exception e) {
logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId
+ " on commit " + commitTime + " on HDFS path " + hoodieTable
.getMetaClient().getBasePath());
}
writeStatus.getStat().setFullPath(currentLogFile.getPath().toString());
}
// update the new location of the record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
});
this.records = records;
}
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
try {
IndexedRecord avroRecord = hoodieRecord.getData().getInsertValue(schema);
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
recordIndex.getAndIncrement());
HoodieAvroUtils
.addHoodieKeyToRecord((GenericRecord) avroRecord, hoodieRecord.getRecordKey(),
hoodieRecord.getPartitionPath(), fileId);
HoodieAvroUtils
.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
hoodieRecord.deflate();
writeStatus.markSuccess(hoodieRecord);
recordsWritten++;
return Optional.of(avroRecord);
} catch (Exception e) {
logger.error("Error writing record " + hoodieRecord, e);
writeStatus.markFailure(hoodieRecord, e);
}
return Optional.empty();
}
public void doAppend() {
Iterator<IndexedRecord> recordItr =
records.stream().map(this::getIndexedRecord).filter(Optional::isPresent)
.map(Optional::get).iterator();
try {
logAppender.append(recordItr);
} catch (Exception e) {
throw new HoodieAppendException(
"Failed while appeding records to " + currentLogFile.getPath(), e);
}
}
public void close() {
try {
if (logAppender != null) {
logAppender.close();
}
writeStatus.getStat().setNumWrites(recordsWritten);
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
}
public WriteStatus getWriteStatus() {
return writeStatus;
}
}

View File

@@ -17,14 +17,12 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -58,18 +56,16 @@ public class HoodieCleaner {
private final TableFileSystemView fileSystemView;
private final HoodieTimeline commitTimeline;
private HoodieTableMetaClient metaClient;
private HoodieTable hoodieTable;
private HoodieWriteConfig config;
private FileSystem fs;
public HoodieCleaner(HoodieTableMetaClient metaClient, HoodieWriteConfig config,
FileSystem fs) {
this.metaClient = metaClient;
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
this.commitTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
public HoodieCleaner(HoodieTable hoodieTable, HoodieWriteConfig config) {
this.hoodieTable = hoodieTable;
this.fileSystemView = hoodieTable.getCompactedFileSystemView();
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
this.config = config;
this.fs = fs;
this.fs = hoodieTable.getFs();
}

View File

@@ -16,15 +16,14 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.view.ReadOptimizedTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -39,20 +38,19 @@ public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
protected final String commitTime;
protected final HoodieWriteConfig config;
protected final FileSystem fs;
protected final HoodieTableMetaClient metaClient;
protected final HoodieTimeline hoodieTimeline;
protected final TableFileSystemView fileSystemView;
protected final HoodieTable<T> hoodieTable;
protected HoodieTimeline hoodieTimeline;
protected TableFileSystemView fileSystemView;
protected final Schema schema;
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
HoodieTableMetaClient metaClient) {
HoodieTable<T> hoodieTable) {
this.commitTime = commitTime;
this.config = config;
this.fs = FSUtils.getFs();
this.metaClient = metaClient;
this.hoodieTimeline =
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
this.fileSystemView = new ReadOptimizedTableView(fs, metaClient);
this.hoodieTable = hoodieTable;
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
this.fileSystemView = hoodieTable.getFileSystemView();
this.schema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
}

View File

@@ -16,17 +16,17 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
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.util.FSUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieInsertException;
import com.uber.hoodie.io.storage.HoodieStorageWriter;
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
@@ -45,8 +45,8 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
private int recordsWritten = 0;
public HoodieInsertHandle(HoodieWriteConfig config, String commitTime,
HoodieTableMetaClient metadata, String partitionPath) {
super(config, commitTime, metadata);
HoodieTable<T> hoodieTable, String partitionPath) {
super(config, commitTime, hoodieTable);
this.status = new WriteStatus();
status.setFileId(UUID.randomUUID().toString());
status.setPartitionPath(partitionPath);
@@ -54,7 +54,7 @@ public class HoodieInsertHandle<T extends HoodieRecordPayload> extends HoodieIOH
this.path = makeNewPath(partitionPath, TaskContext.getPartitionId(), status.getFileId());
try {
this.storageWriter =
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, metadata, config, schema);
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, hoodieTable, config, schema);
} catch (IOException e) {
throw new HoodieInsertException(
"Failed to initialize HoodieStorageWriter for path " + path, e);

View File

@@ -16,7 +16,6 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodieRecord;
@@ -27,6 +26,7 @@ import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieUpsertException;
import com.uber.hoodie.io.storage.HoodieStorageWriter;
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
@@ -38,11 +38,12 @@ import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
@SuppressWarnings("Duplicates") public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
@SuppressWarnings("Duplicates")
public class HoodieUpdateHandle <T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
private static Logger logger = LogManager.getLogger(HoodieUpdateHandle.class);
private final WriteStatus writeStatus;
private final HashMap<String, HoodieRecord<T>> keyToNewRecords;
private WriteStatus writeStatus;
private HashMap<String, HoodieRecord<T>> keyToNewRecords;
private HoodieStorageWriter<IndexedRecord> storageWriter;
private Path newFilePath;
private Path oldFilePath;
@@ -52,22 +53,23 @@ import java.util.Iterator;
public HoodieUpdateHandle(HoodieWriteConfig config,
String commitTime,
HoodieTableMetaClient metaClient,
HoodieTable<T> hoodieTable,
Iterator<HoodieRecord<T>> recordItr,
String fileId) {
super(config, commitTime, metaClient);
WriteStatus writeStatus = new WriteStatus();
writeStatus.setStat(new HoodieWriteStat());
this.writeStatus = writeStatus;
this.fileId = fileId;
this.keyToNewRecords = new HashMap<>();
init(recordItr);
super(config, commitTime, hoodieTable);
init(fileId, recordItr);
}
/**
* Load the new incoming records in a map, and extract the old file path.
*/
private void init(Iterator<HoodieRecord<T>> newRecordsItr) {
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
WriteStatus writeStatus = new WriteStatus();
writeStatus.setStat(new HoodieWriteStat());
this.writeStatus = writeStatus;
this.fileId = fileId;
this.keyToNewRecords = new HashMap<>();
try {
// Load the new records in a map
while (newRecordsItr.hasNext()) {
@@ -104,14 +106,14 @@ import java.util.Iterator;
}
// Create the writer for writing the new version file
storageWriter = HoodieStorageWriterFactory
.getStorageWriter(commitTime, newFilePath, metaClient, config, schema);
.getStorageWriter(commitTime, newFilePath, hoodieTable, config, schema);
} catch (Exception e) {
logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
+ commitTime + " on HDFS path " + metaClient.getBasePath());
+ commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), e);
}
}

View File

@@ -16,18 +16,15 @@
package com.uber.hoodie.io.compact;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.RealtimeTableView;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.table.HoodieTable;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.Serializable;
import java.nio.charset.StandardCharsets;
import java.util.Date;
import java.util.Optional;
/**
* A HoodieCompactor runs compaction on a hoodie table
@@ -38,14 +35,13 @@ public interface HoodieCompactor extends Serializable {
* @throws Exception
*/
HoodieCompactionMetadata compact(JavaSparkContext jsc, final HoodieWriteConfig config,
HoodieTableMetaClient metaClient, RealtimeTableView fsView,
CompactionFilter compactionFilter) throws Exception;
HoodieTable hoodieTable, CompactionFilter compactionFilter) throws Exception;
// Helper methods
default String startCompactionCommit(HoodieTableMetaClient metaClient) {
default String startCompactionCommit(HoodieTable hoodieTable) {
String commitTime = HoodieActiveTimeline.COMMIT_FORMATTER.format(new Date());
HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline();
HoodieActiveTimeline activeTimeline = hoodieTable.getActiveTimeline();
activeTimeline
.createInflight(new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, commitTime));
return commitTime;

View File

@@ -24,13 +24,14 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.common.table.view.RealtimeTableView;
import com.uber.hoodie.common.util.AvroUtils;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieCommitException;
import com.uber.hoodie.table.HoodieCopyOnWriteTable;
import com.uber.hoodie.table.HoodieMergeOnReadTable;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.commons.collections.IteratorUtils;
import org.apache.hadoop.fs.FileSystem;
@@ -61,11 +62,12 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
@Override
public HoodieCompactionMetadata compact(JavaSparkContext jsc, HoodieWriteConfig config,
HoodieTableMetaClient metaClient, RealtimeTableView fsView,
HoodieTable hoodieTable,
CompactionFilter compactionFilter) throws Exception {
// TODO - rollback any compactions in flight
String compactionCommit = startCompactionCommit(metaClient);
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
String compactionCommit = startCompactionCommit(hoodieTable);
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommit);
List<String> partitionPaths =
FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath());
@@ -74,9 +76,9 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
List<CompactionOperation> operations =
jsc.parallelize(partitionPaths, partitionPaths.size())
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> {
FileSystem fileSystem = FSUtils.getFs();
return fsView.groupLatestDataFileWithLogFiles(fileSystem, partitionPath)
.entrySet().stream()
return hoodieTable.getFileSystemView()
.groupLatestDataFileWithLogFiles(partitionPath).entrySet()
.stream()
.map(s -> new CompactionOperation(s.getKey(), partitionPath, s.getValue()))
.collect(Collectors.toList());
}).collect();
@@ -145,9 +147,15 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
List<HoodieRecord<HoodieAvroPayload>> readDeltaFilesInMemory =
AvroUtils.loadFromFiles(fs, operation.getDeltaFilePaths(), schema);
if(readDeltaFilesInMemory.isEmpty()) {
return IteratorUtils.emptyIterator();
}
// Compacting is very similar to applying updates to existing file
HoodieCopyOnWriteTable<HoodieAvroPayload> table =
new HoodieCopyOnWriteTable<>(commitTime, config, metaClient);
return table.handleUpdate(operation.getFileId(), readDeltaFilesInMemory.iterator());
new HoodieCopyOnWriteTable<>(config, metaClient);
return table
.handleUpdate(commitTime, operation.getFileId(), readDeltaFilesInMemory.iterator());
}
public boolean commitCompaction(String commitTime, HoodieTableMetaClient metaClient,

View File

@@ -22,6 +22,7 @@ import com.uber.hoodie.avro.HoodieAvroWriteSupport;
import com.uber.hoodie.common.BloomFilter;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.table.HoodieTable;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
@@ -32,7 +33,7 @@ import java.io.IOException;
public class HoodieStorageWriterFactory {
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
String commitTime, Path path, HoodieTableMetaClient metaClient, HoodieWriteConfig config, Schema schema)
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