Introduce RealtimeTableView and Implement HoodieRealtimeTableCompactor (#73)
This commit is contained in:
committed by
Prasanna Rajaperumal
parent
48fbb0f425
commit
11d2fd3428
@@ -0,0 +1,52 @@
|
||||
/*
|
||||
* 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.common.model;
|
||||
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* This is a payload to wrap a existing Hoodie Avro Record.
|
||||
* Useful to create a HoodieRecord over existing GenericRecords in a hoodie datasets (useful in compactions)
|
||||
*/
|
||||
public class HoodieAvroPayload implements HoodieRecordPayload<HoodieAvroPayload> {
|
||||
private final GenericRecord record;
|
||||
|
||||
public HoodieAvroPayload(GenericRecord record) {
|
||||
this.record = record;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieAvroPayload preCombine(HoodieAvroPayload another) {
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedRecord combineAndGetUpdateValue(IndexedRecord currentValue, Schema schema)
|
||||
throws IOException {
|
||||
return getInsertValue(schema);
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexedRecord getInsertValue(Schema schema) throws IOException {
|
||||
return HoodieAvroUtils.rewriteRecord(record, schema);
|
||||
}
|
||||
}
|
||||
@@ -17,7 +17,7 @@
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
public enum HoodieFileFormat {
|
||||
PARQUET(".parquet");
|
||||
PARQUET(".parquet"), AVRO(".avro");
|
||||
|
||||
private final String extension;
|
||||
|
||||
|
||||
@@ -31,5 +31,5 @@ package com.uber.hoodie.common.model;
|
||||
* SIMPLE_LSM - A simple 2 level LSM tree.
|
||||
*/
|
||||
public enum HoodieTableType {
|
||||
COPY_ON_WRITE
|
||||
COPY_ON_WRITE, MERGE_ON_READ
|
||||
}
|
||||
|
||||
@@ -47,8 +47,12 @@ public class HoodieTableConfig implements Serializable {
|
||||
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
|
||||
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME =
|
||||
"hoodie.table.ro.file.format";
|
||||
public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME =
|
||||
"hoodie.table.rt.file.format";
|
||||
|
||||
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
|
||||
public static final HoodieFileFormat DEFAULT_RO_FILE_FORMAT = HoodieFileFormat.PARQUET;
|
||||
public static final HoodieFileFormat DEFAULT_RT_FILE_FORMAT = HoodieFileFormat.AVRO;
|
||||
private Properties props;
|
||||
|
||||
public HoodieTableConfig(FileSystem fs, String metaPath) {
|
||||
@@ -108,7 +112,7 @@ public class HoodieTableConfig implements Serializable {
|
||||
* @return
|
||||
*/
|
||||
public HoodieTableType getTableType() {
|
||||
if (props.contains(HOODIE_TABLE_TYPE_PROP_NAME)) {
|
||||
if (props.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
|
||||
return HoodieTableType.valueOf(props.getProperty(HOODIE_TABLE_TYPE_PROP_NAME));
|
||||
}
|
||||
return DEFAULT_TABLE_TYPE;
|
||||
@@ -129,9 +133,22 @@ public class HoodieTableConfig implements Serializable {
|
||||
* @return HoodieFileFormat for the Read Optimized Storage format
|
||||
*/
|
||||
public HoodieFileFormat getROFileFormat() {
|
||||
if (props.contains(HOODIE_RO_FILE_FORMAT_PROP_NAME)) {
|
||||
if (props.containsKey(HOODIE_RO_FILE_FORMAT_PROP_NAME)) {
|
||||
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RO_FILE_FORMAT_PROP_NAME));
|
||||
}
|
||||
return DEFAULT_RO_FILE_FORMAT;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the Read Optimized Storage Format
|
||||
*
|
||||
* @return HoodieFileFormat for the Read Optimized Storage format
|
||||
*/
|
||||
public HoodieFileFormat getRTFileFormat() {
|
||||
if (props.containsKey(HOODIE_RT_FILE_FORMAT_PROP_NAME)) {
|
||||
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RT_FILE_FORMAT_PROP_NAME));
|
||||
}
|
||||
return DEFAULT_RT_FILE_FORMAT;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -137,6 +137,14 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
return tableConfig;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the FS implementation for this table
|
||||
* @return
|
||||
*/
|
||||
public FileSystem getFs() {
|
||||
return fs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the active instants as a timeline
|
||||
*
|
||||
|
||||
@@ -41,15 +41,17 @@ public interface HoodieTimeline extends Serializable {
|
||||
String COMMIT_ACTION = "commit";
|
||||
String CLEAN_ACTION = "clean";
|
||||
String SAVEPOINT_ACTION = "savepoint";
|
||||
String COMPACTION_ACTION = "compaction";
|
||||
String INFLIGHT_EXTENSION = ".inflight";
|
||||
String COMMIT_EXTENSION = "." + COMMIT_ACTION;
|
||||
String CLEAN_EXTENSION = "." + CLEAN_ACTION;
|
||||
String SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION;
|
||||
String COMPACTION_EXTENSION = "." + COMPACTION_ACTION;
|
||||
//this is to preserve backwards compatibility on commit in-flight filenames
|
||||
String INFLIGHT_COMMIT_EXTENSION = INFLIGHT_EXTENSION;
|
||||
String INFLIGHT_CLEAN_EXTENSION = "." + CLEAN_ACTION + INFLIGHT_EXTENSION;
|
||||
String INFLIGHT_SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION + INFLIGHT_EXTENSION;
|
||||
|
||||
String INFLIGHT_COMPACTION_EXTENSION = "." + COMPACTION_ACTION + INFLIGHT_EXTENSION;
|
||||
|
||||
/**
|
||||
* Filter this timeline to just include the in-flights
|
||||
@@ -193,6 +195,14 @@ public interface HoodieTimeline extends Serializable {
|
||||
return commitTime + HoodieTimeline.SAVEPOINT_EXTENSION;
|
||||
}
|
||||
|
||||
static String makeInflightCompactionFileName(String commitTime) {
|
||||
return commitTime + HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION;
|
||||
}
|
||||
|
||||
static String makeCompactionFileName(String commitTime) {
|
||||
return commitTime + HoodieTimeline.COMPACTION_EXTENSION;
|
||||
}
|
||||
|
||||
static String getCommitFromCommitFile(String commitFileName) {
|
||||
return commitFileName.split("\\.")[0];
|
||||
}
|
||||
|
||||
@@ -117,6 +117,8 @@ public class HoodieLogAppendConfig {
|
||||
private Integer fileVersion;
|
||||
// Partition path for the log file
|
||||
private Path partitionPath;
|
||||
// The base commit time for which the log files are accumulated
|
||||
private String baseCommitTime;
|
||||
|
||||
public Builder withBufferSize(int bufferSize) {
|
||||
this.bufferSize = bufferSize;
|
||||
@@ -173,6 +175,11 @@ public class HoodieLogAppendConfig {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBaseCommitTime(String commitTime) {
|
||||
this.baseCommitTime = commitTime;
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLogAppendConfig build() throws IOException {
|
||||
log.info("Building HoodieLogAppendConfig");
|
||||
if (schema == null) {
|
||||
@@ -185,6 +192,9 @@ public class HoodieLogAppendConfig {
|
||||
if (fileId == null) {
|
||||
throw new IllegalArgumentException("FileID is not specified");
|
||||
}
|
||||
if (baseCommitTime == null) {
|
||||
throw new IllegalArgumentException("BaseCommitTime is not specified");
|
||||
}
|
||||
if (logFileExtension == null) {
|
||||
throw new IllegalArgumentException("File extension is not specified");
|
||||
}
|
||||
@@ -194,14 +204,14 @@ public class HoodieLogAppendConfig {
|
||||
if (fileVersion == null) {
|
||||
log.info("Computing the next log version for " + fileId + " in " + partitionPath);
|
||||
fileVersion =
|
||||
FSUtils.getCurrentLogVersion(fs, partitionPath, fileId, logFileExtension);
|
||||
FSUtils.getCurrentLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
|
||||
log.info(
|
||||
"Computed the next log version for " + fileId + " in " + partitionPath + " as "
|
||||
+ fileVersion);
|
||||
}
|
||||
|
||||
Path logPath = new Path(partitionPath,
|
||||
FSUtils.makeLogFileName(fileId, logFileExtension, fileVersion));
|
||||
FSUtils.makeLogFileName(fileId, logFileExtension, baseCommitTime, fileVersion));
|
||||
log.info("LogConfig created on path " + logPath);
|
||||
HoodieLogFile logFile = new HoodieLogFile(logPath);
|
||||
|
||||
|
||||
@@ -22,7 +22,7 @@ import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Comparator;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
@@ -51,6 +51,10 @@ public class HoodieLogFile {
|
||||
return FSUtils.getFileIdFromLogPath(path);
|
||||
}
|
||||
|
||||
public String getBaseCommitTime() {
|
||||
return FSUtils.getBaseCommitTimeFromLogPath(path);
|
||||
}
|
||||
|
||||
public int getLogVersion() {
|
||||
return FSUtils.getFileVersionFromLog(path);
|
||||
}
|
||||
@@ -74,10 +78,12 @@ public class HoodieLogFile {
|
||||
|
||||
public HoodieLogFile rollOver(FileSystem fs) throws IOException {
|
||||
String fileId = getFileId();
|
||||
int newVersion =
|
||||
FSUtils.computeNextLogVersion(fs, path.getParent(), fileId, DELTA_EXTENSION);
|
||||
String baseCommitTime = getBaseCommitTime();
|
||||
int newVersion = FSUtils
|
||||
.computeNextLogVersion(fs, path.getParent(), fileId,
|
||||
DELTA_EXTENSION, baseCommitTime);
|
||||
return new HoodieLogFile(new Path(path.getParent(),
|
||||
FSUtils.makeLogFileName(fileId, DELTA_EXTENSION, newVersion)));
|
||||
FSUtils.makeLogFileName(fileId, DELTA_EXTENSION, baseCommitTime, newVersion)));
|
||||
}
|
||||
|
||||
public boolean shouldRollOver(HoodieLogAppender currentWriter, HoodieLogAppendConfig config)
|
||||
@@ -85,6 +91,14 @@ public class HoodieLogFile {
|
||||
return currentWriter.getCurrentSize() > config.getSizeThreshold();
|
||||
}
|
||||
|
||||
public static Comparator<HoodieLogFile> getLogVersionComparator() {
|
||||
return (o1, o2) -> {
|
||||
// reverse the order
|
||||
return new Integer(o2.getLogVersion()).compareTo(o1.getLogVersion());
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "HoodieLogFile{" + path + '}';
|
||||
|
||||
@@ -43,10 +43,10 @@ import java.util.stream.Stream;
|
||||
public class CompositeAvroLogReader {
|
||||
private final Map<Integer, AvroLogReader> readers;
|
||||
|
||||
public CompositeAvroLogReader(Path partitionPath, String fileId, FileSystem fs,
|
||||
public CompositeAvroLogReader(Path partitionPath, String fileId, String baseCommitTime, FileSystem fs,
|
||||
Schema readerSchema, String logFileExtension) throws IOException {
|
||||
Stream<HoodieLogFile> allLogFiles =
|
||||
FSUtils.getAllLogFiles(fs, partitionPath, fileId, logFileExtension);
|
||||
FSUtils.getAllLogFiles(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
|
||||
this.readers = allLogFiles.map(hoodieLogFile -> {
|
||||
try {
|
||||
return new AvroLogReader(hoodieLogFile, fs, readerSchema);
|
||||
|
||||
@@ -31,9 +31,11 @@ import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.Arrays;
|
||||
import java.util.Comparator;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.stream.Collectors;
|
||||
@@ -50,6 +52,8 @@ import java.util.stream.Stream;
|
||||
* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized.
|
||||
*/
|
||||
public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
public static final SimpleDateFormat COMMIT_FORMATTER = new SimpleDateFormat("yyyyMMddHHmmss");
|
||||
|
||||
private final transient static Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
|
||||
private String metaPath;
|
||||
private transient FileSystem fs;
|
||||
@@ -81,8 +85,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
|
||||
public HoodieActiveTimeline(FileSystem fs, String metaPath) {
|
||||
this(fs, metaPath,
|
||||
new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, SAVEPOINT_EXTENSION,
|
||||
INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION});
|
||||
new String[] {COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, COMPACTION_EXTENSION,
|
||||
INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, COMPACTION_EXTENSION});
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -113,6 +117,27 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the commits (inflight and completed) in the compaction timeline
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getCompactionTimeline() {
|
||||
return new HoodieDefaultTimeline(filterInstantsByAction(COMPACTION_ACTION),
|
||||
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a timeline of a specific set of actions. useful to create a merged timeline of multiple actions
|
||||
*
|
||||
* @param actions actions allowed in the timeline
|
||||
* @return
|
||||
*/
|
||||
public HoodieTimeline getTimelineOfActions(Set<String> actions) {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(s -> actions.contains(s.getAction())),
|
||||
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get only the cleaner action (inflight and completed) in the active timeline
|
||||
*
|
||||
|
||||
@@ -93,6 +93,10 @@ public class HoodieInstant implements Serializable {
|
||||
return isInflight ?
|
||||
HoodieTimeline.makeInflightSavePointFileName(timestamp) :
|
||||
HoodieTimeline.makeSavePointFileName(timestamp);
|
||||
} else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) {
|
||||
return isInflight ?
|
||||
HoodieTimeline.makeInflightCompactionFileName(timestamp) :
|
||||
HoodieTimeline.makeCompactionFileName(timestamp);
|
||||
}
|
||||
throw new IllegalArgumentException("Cannot get file name for unknown action " + action);
|
||||
}
|
||||
|
||||
@@ -21,12 +21,15 @@ import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
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 java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
@@ -47,22 +50,38 @@ import java.util.stream.Stream;
|
||||
* @see ReadOptimizedTableView
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public abstract class AbstractTableFileSystemView implements TableFileSystemView {
|
||||
protected final HoodieTableMetaClient metaClient;
|
||||
protected final transient FileSystem fs;
|
||||
protected final HoodieTimeline activeCommitTimeline;
|
||||
public abstract class AbstractTableFileSystemView implements TableFileSystemView, Serializable {
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
protected transient FileSystem fs;
|
||||
// This is the commits that will be visible for all views extending this view
|
||||
protected HoodieTimeline visibleActiveCommitTimeline;
|
||||
|
||||
public AbstractTableFileSystemView(FileSystem fs, HoodieTableMetaClient metaClient) {
|
||||
public AbstractTableFileSystemView(FileSystem fs, HoodieTableMetaClient metaClient,
|
||||
HoodieTimeline visibleActiveCommitTimeline) {
|
||||
this.metaClient = metaClient;
|
||||
this.fs = fs;
|
||||
// Get the active timeline and filter only completed commits
|
||||
this.activeCommitTimeline =
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
this.visibleActiveCommitTimeline = visibleActiveCommitTimeline;
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is only used when this object is deserialized in a spark executor.
|
||||
*
|
||||
* @deprecated
|
||||
*/
|
||||
private void readObject(java.io.ObjectInputStream in)
|
||||
throws IOException, ClassNotFoundException {
|
||||
in.defaultReadObject();
|
||||
this.fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
private void writeObject(java.io.ObjectOutputStream out)
|
||||
throws IOException {
|
||||
out.defaultWriteObject();
|
||||
}
|
||||
|
||||
public Stream<HoodieDataFile> getLatestDataFilesForFileId(final String partitionPath,
|
||||
String fileId) {
|
||||
Optional<HoodieInstant> lastInstant = activeCommitTimeline.lastInstant();
|
||||
Optional<HoodieInstant> lastInstant = visibleActiveCommitTimeline.lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
return getLatestVersionInPartition(partitionPath, lastInstant.get().getTimestamp())
|
||||
.filter(hoodieDataFile -> hoodieDataFile.getFileId().equals(fileId));
|
||||
@@ -73,22 +92,17 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestVersionInPartition(String partitionPathStr,
|
||||
String maxCommitTime) {
|
||||
try {
|
||||
return getLatestVersionsBeforeOrOn(listDataFilesInPartition(partitionPathStr),
|
||||
maxCommitTime);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not get latest versions in Partition " + partitionPathStr, e);
|
||||
}
|
||||
return getLatestVersionsBeforeOrOn(listDataFilesInPartition(partitionPathStr),
|
||||
maxCommitTime);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Stream<List<HoodieDataFile>> getEveryVersionInPartition(String partitionPath) {
|
||||
try {
|
||||
if (activeCommitTimeline.lastInstant().isPresent()) {
|
||||
if (visibleActiveCommitTimeline.lastInstant().isPresent()) {
|
||||
return getFilesByFileId(listDataFilesInPartition(partitionPath),
|
||||
activeCommitTimeline.lastInstant().get().getTimestamp());
|
||||
visibleActiveCommitTimeline.lastInstant().get().getTimestamp());
|
||||
}
|
||||
return Stream.empty();
|
||||
} catch (IOException e) {
|
||||
@@ -97,18 +111,26 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
|
||||
}
|
||||
}
|
||||
|
||||
protected abstract FileStatus[] listDataFilesInPartition(String partitionPathStr)
|
||||
throws IOException;
|
||||
protected FileStatus[] listDataFilesInPartition(String partitionPathStr) {
|
||||
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
|
||||
try {
|
||||
return fs.listStatus(partitionPath, path -> path.getName()
|
||||
.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to list data files in partition " + partitionPathStr, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestVersionInRange(FileStatus[] fileStatuses,
|
||||
List<String> commitsToReturn) {
|
||||
if (activeCommitTimeline.empty() || commitsToReturn.isEmpty()) {
|
||||
if (visibleActiveCommitTimeline.empty() || commitsToReturn.isEmpty()) {
|
||||
return Stream.empty();
|
||||
}
|
||||
try {
|
||||
return getFilesByFileId(fileStatuses,
|
||||
activeCommitTimeline.lastInstant().get().getTimestamp())
|
||||
visibleActiveCommitTimeline.lastInstant().get().getTimestamp())
|
||||
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
|
||||
for (HoodieDataFile fs : fss) {
|
||||
if (commitsToReturn.contains(fs.getCommitTime())) {
|
||||
@@ -127,14 +149,14 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
|
||||
public Stream<HoodieDataFile> getLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
|
||||
String maxCommitToReturn) {
|
||||
try {
|
||||
if (activeCommitTimeline.empty()) {
|
||||
if (visibleActiveCommitTimeline.empty()) {
|
||||
return Stream.empty();
|
||||
}
|
||||
return getFilesByFileId(fileStatuses,
|
||||
activeCommitTimeline.lastInstant().get().getTimestamp())
|
||||
visibleActiveCommitTimeline.lastInstant().get().getTimestamp())
|
||||
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
|
||||
for (HoodieDataFile fs1 : fss) {
|
||||
if (activeCommitTimeline
|
||||
if (visibleActiveCommitTimeline
|
||||
.compareTimestamps(fs1.getCommitTime(), maxCommitToReturn,
|
||||
HoodieTimeline.LESSER_OR_EQUAL)) {
|
||||
return Optional.of(fs1);
|
||||
@@ -150,11 +172,11 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestVersions(FileStatus[] fileStatuses) {
|
||||
try {
|
||||
if (activeCommitTimeline.empty()) {
|
||||
if (visibleActiveCommitTimeline.empty()) {
|
||||
return Stream.empty();
|
||||
}
|
||||
return getFilesByFileId(fileStatuses,
|
||||
activeCommitTimeline.lastInstant().get().getTimestamp())
|
||||
visibleActiveCommitTimeline.lastInstant().get().getTimestamp())
|
||||
.map(statuses -> statuses.get(0));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files for latest version ", e);
|
||||
@@ -178,8 +200,9 @@ public abstract class AbstractTableFileSystemView implements TableFileSystemView
|
||||
String maxCommitTime) throws IOException {
|
||||
return Arrays.stream(files).flatMap(fileStatus -> {
|
||||
HoodieDataFile dataFile = new HoodieDataFile(fileStatus);
|
||||
if (activeCommitTimeline.containsOrBeforeTimelineStarts(dataFile.getCommitTime())
|
||||
&& activeCommitTimeline.compareTimestamps(dataFile.getCommitTime(), maxCommitTime,
|
||||
if (visibleActiveCommitTimeline.containsOrBeforeTimelineStarts(dataFile.getCommitTime())
|
||||
&& visibleActiveCommitTimeline
|
||||
.compareTimestamps(dataFile.getCommitTime(), maxCommitTime,
|
||||
HoodieTimeline.LESSER_OR_EQUAL)) {
|
||||
return Stream.of(Pair.of(dataFile.getFileId(), dataFile));
|
||||
}
|
||||
|
||||
@@ -29,19 +29,9 @@ import java.io.IOException;
|
||||
*/
|
||||
public class ReadOptimizedTableView extends AbstractTableFileSystemView {
|
||||
public ReadOptimizedTableView(FileSystem fs, HoodieTableMetaClient metaClient) {
|
||||
super(fs, metaClient);
|
||||
// Get the active timeline and filter only completed commits
|
||||
super(fs, metaClient,
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
|
||||
}
|
||||
|
||||
protected FileStatus[] listDataFilesInPartition(String partitionPathStr) {
|
||||
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
|
||||
try {
|
||||
return fs.listStatus(partitionPath, path -> path.getName()
|
||||
.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to list data files in partition " + partitionPathStr, e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
@@ -0,0 +1,80 @@
|
||||
/*
|
||||
* 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.common.table.view;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
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 java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Realtime Table View which includes both ROStorageformat files and RTStorageFormat files
|
||||
*/
|
||||
public class RealtimeTableView extends AbstractTableFileSystemView {
|
||||
public RealtimeTableView(FileSystem fs, HoodieTableMetaClient metaClient) {
|
||||
// For realtime table view, visibleActiveCommitTimeline is a merged timeline of all commits and compactions
|
||||
super(fs, metaClient, metaClient.getActiveTimeline().getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.COMPACTION_ACTION)).filterCompletedInstants());
|
||||
Preconditions.checkArgument(metaClient.getTableType() == HoodieTableType.MERGE_ON_READ,
|
||||
"Realtime view can only be constructed on Hoodie tables with MERGE_ON_READ storage type");
|
||||
}
|
||||
|
||||
public Map<HoodieDataFile, List<HoodieLogFile>> groupLatestDataFileWithLogFiles(FileSystem fs,
|
||||
String partitionPath) throws IOException {
|
||||
// All the files in the partition
|
||||
FileStatus[] files = fs.listStatus(new Path(metaClient.getBasePath(), partitionPath));
|
||||
// All the log files filtered from the above list, sorted by version numbers
|
||||
List<HoodieLogFile> allLogFiles = Arrays.stream(files).filter(s -> s.getPath().getName()
|
||||
.contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension()))
|
||||
.map(HoodieLogFile::new).collect(Collectors.collectingAndThen(Collectors.toList(),
|
||||
l -> l.stream().sorted(HoodieLogFile.getLogVersionComparator())
|
||||
.collect(Collectors.toList())));
|
||||
|
||||
// Filter the delta files by the commit time of the latest base fine and collect as a list
|
||||
Optional<HoodieInstant> lastTimestamp = metaClient.getActiveTimeline().lastInstant();
|
||||
if(!lastTimestamp.isPresent()) {
|
||||
return Maps.newHashMap();
|
||||
}
|
||||
|
||||
return getLatestVersionInPartition(partitionPath, lastTimestamp.get().getTimestamp()).map(
|
||||
hoodieDataFile -> Pair.of(hoodieDataFile, allLogFiles.stream().filter(
|
||||
s -> s.getFileId().equals(hoodieDataFile.getFileId()) && s.getBaseCommitTime()
|
||||
.equals(hoodieDataFile.getCommitTime())).collect(Collectors.toList()))).collect(
|
||||
Collectors.toMap(
|
||||
(Function<Pair<HoodieDataFile, List<HoodieLogFile>>, HoodieDataFile>) Pair::getKey,
|
||||
(Function<Pair<HoodieDataFile, List<HoodieLogFile>>, List<HoodieLogFile>>) Pair::getRight));
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,70 @@
|
||||
/*
|
||||
* 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.common.util;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.file.DataFileReader;
|
||||
import org.apache.avro.file.FileReader;
|
||||
import org.apache.avro.file.SeekableInput;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.io.DatumReader;
|
||||
import org.apache.hadoop.fs.AvroFSInput;
|
||||
import org.apache.hadoop.fs.FileContext;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
public class AvroUtils {
|
||||
|
||||
public static List<HoodieRecord<HoodieAvroPayload>> loadFromFiles(FileSystem fs,
|
||||
List<String> deltaFilePaths, Schema expectedSchema) {
|
||||
|
||||
List<HoodieRecord<HoodieAvroPayload>> loadedRecords = Lists.newArrayList();
|
||||
deltaFilePaths.forEach(s -> {
|
||||
Path path = new Path(s);
|
||||
try {
|
||||
SeekableInput input =
|
||||
new AvroFSInput(FileContext.getFileContext(fs.getConf()), path);
|
||||
GenericDatumReader<GenericRecord> reader = new GenericDatumReader<>();
|
||||
// Set the expected schema to be the current schema to account for schema evolution
|
||||
reader.setExpected(expectedSchema);
|
||||
|
||||
FileReader<GenericRecord> fileReader = DataFileReader.openReader(input, reader);
|
||||
for (GenericRecord deltaRecord : fileReader) {
|
||||
String key = deltaRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
String partitionPath =
|
||||
deltaRecord.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
|
||||
loadedRecords.add(new HoodieRecord<>(new HoodieKey(key, partitionPath),
|
||||
new HoodieAvroPayload(deltaRecord)));
|
||||
}
|
||||
fileReader.close(); // also closes underlying FsInput
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not read avro records from path " + s, e);
|
||||
}
|
||||
});
|
||||
return loadedRecords;
|
||||
}
|
||||
}
|
||||
@@ -17,6 +17,7 @@
|
||||
package com.uber.hoodie.common.util;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFile;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.InvalidHoodiePathException;
|
||||
@@ -45,8 +46,8 @@ import java.util.stream.Stream;
|
||||
public class FSUtils {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(FSUtils.class);
|
||||
// Log files are of this pattern - b5068208-e1a4-11e6-bf01-fe55135034f3.avro.delta.1
|
||||
private static final Pattern LOG_FILE_PATTERN = Pattern.compile("(.*)\\.(.*)\\.(.*)\\.([0-9]*)");
|
||||
// Log files are of this pattern - b5068208-e1a4-11e6-bf01-fe55135034f3_20170101134598.avro.delta.1
|
||||
private static final Pattern LOG_FILE_PATTERN = Pattern.compile("(.*)_(.*)\\.(.*)\\.(.*)\\.([0-9]*)");
|
||||
private static final int MAX_ATTEMPTS_RECOVER_LEASE = 10;
|
||||
|
||||
public static FileSystem getFs() {
|
||||
@@ -140,7 +141,7 @@ public class FSUtils {
|
||||
if(!matcher.find()) {
|
||||
throw new InvalidHoodiePathException(logPath, "LogFile");
|
||||
}
|
||||
return matcher.group(2) + "." + matcher.group(3);
|
||||
return matcher.group(3) + "." + matcher.group(4);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -158,6 +159,21 @@ public class FSUtils {
|
||||
return matcher.group(1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the first part of the file name in the log file. That will be the fileId.
|
||||
* Log file do not have commitTime in the file name.
|
||||
*
|
||||
* @param path
|
||||
* @return
|
||||
*/
|
||||
public static String getBaseCommitTimeFromLogPath(Path path) {
|
||||
Matcher matcher = LOG_FILE_PATTERN.matcher(path.getName());
|
||||
if(!matcher.find()) {
|
||||
throw new InvalidHoodiePathException(path, "LogFile");
|
||||
}
|
||||
return matcher.group(2);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the last part of the file name in the log file and convert to int.
|
||||
*
|
||||
@@ -169,11 +185,12 @@ public class FSUtils {
|
||||
if(!matcher.find()) {
|
||||
throw new InvalidHoodiePathException(logPath, "LogFile");
|
||||
}
|
||||
return Integer.parseInt(matcher.group(4));
|
||||
return Integer.parseInt(matcher.group(5));
|
||||
}
|
||||
|
||||
public static String makeLogFileName(String fileId, String logFileExtension, int version) {
|
||||
return String.format("%s%s.%d", fileId, logFileExtension, version);
|
||||
public static String makeLogFileName(String fileId, String logFileExtension,
|
||||
String baseCommitTime, int version) {
|
||||
return String.format("%s_%s%s.%d", fileId, baseCommitTime, logFileExtension, version);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -198,10 +215,10 @@ public class FSUtils {
|
||||
* @return
|
||||
*/
|
||||
public static Stream<HoodieLogFile> getAllLogFiles(FileSystem fs, Path partitionPath,
|
||||
final String fileId, final String logFileExtension) throws IOException {
|
||||
final String fileId, final String logFileExtension, final String baseCommitTime) throws IOException {
|
||||
return Arrays.stream(fs.listStatus(partitionPath,
|
||||
path -> path.getName().startsWith(fileId) && path.getName()
|
||||
.contains(logFileExtension))).map(HoodieLogFile::new);
|
||||
path -> path.getName().startsWith(fileId) && path.getName().contains(logFileExtension)))
|
||||
.map(HoodieLogFile::new).filter(s -> s.getBaseCommitTime().equals(baseCommitTime));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -215,9 +232,9 @@ public class FSUtils {
|
||||
* @throws IOException
|
||||
*/
|
||||
public static Optional<Integer> getLatestLogVersion(FileSystem fs, Path partitionPath,
|
||||
final String fileId, final String logFileExtension) throws IOException {
|
||||
final String fileId, final String logFileExtension, final String baseCommitTime) throws IOException {
|
||||
Optional<HoodieLogFile> latestLogFile =
|
||||
getLatestLogFile(getAllLogFiles(fs, partitionPath, fileId, logFileExtension));
|
||||
getLatestLogFile(getAllLogFiles(fs, partitionPath, fileId, logFileExtension, baseCommitTime));
|
||||
if (latestLogFile.isPresent()) {
|
||||
return Optional.of(latestLogFile.get().getLogVersion());
|
||||
}
|
||||
@@ -225,9 +242,9 @@ public class FSUtils {
|
||||
}
|
||||
|
||||
public static int getCurrentLogVersion(FileSystem fs, Path partitionPath,
|
||||
final String fileId, final String logFileExtension) throws IOException {
|
||||
final String fileId, final String logFileExtension, final String baseCommitTime) throws IOException {
|
||||
Optional<Integer> currentVersion =
|
||||
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension);
|
||||
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
|
||||
// handle potential overflow
|
||||
return (currentVersion.isPresent()) ? currentVersion.get() : 1;
|
||||
}
|
||||
@@ -242,9 +259,9 @@ public class FSUtils {
|
||||
* @throws IOException
|
||||
*/
|
||||
public static int computeNextLogVersion(FileSystem fs, Path partitionPath, final String fileId,
|
||||
final String logFileExtension) throws IOException {
|
||||
final String logFileExtension, final String baseCommitTime) throws IOException {
|
||||
Optional<Integer> currentVersion =
|
||||
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension);
|
||||
getLatestLogVersion(fs, partitionPath, fileId, logFileExtension, baseCommitTime);
|
||||
// handle potential overflow
|
||||
return (currentVersion.isPresent()) ? currentVersion.get() + 1 : 1;
|
||||
}
|
||||
@@ -287,4 +304,5 @@ public class FSUtils {
|
||||
return recovered;
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@ package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
|
||||
import com.uber.hoodie.exception.SchemaCompatabilityException;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.*;
|
||||
import org.apache.avro.io.BinaryEncoder;
|
||||
@@ -124,14 +125,13 @@ public class HoodieAvroUtils {
|
||||
/**
|
||||
* Given a avro record with a given schema, rewrites it into the new schema
|
||||
*/
|
||||
public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema)
|
||||
throws Exception {
|
||||
public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema) {
|
||||
GenericRecord newRecord = new GenericData.Record(newSchema);
|
||||
for (Schema.Field f : record.getSchema().getFields()) {
|
||||
newRecord.put(f.name(), record.get(f.name()));
|
||||
}
|
||||
if (!new GenericData().validate(newSchema, newRecord)) {
|
||||
throw new Exception(
|
||||
throw new SchemaCompatabilityException(
|
||||
"Unable to validate the rewritten record " + record + " against schema "
|
||||
+ newSchema);
|
||||
}
|
||||
|
||||
@@ -16,6 +16,8 @@
|
||||
|
||||
package com.uber.hoodie.exception;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Exception thrown for Hoodie failures. The root of
|
||||
@@ -27,7 +29,7 @@ package com.uber.hoodie.exception;
|
||||
* </p>
|
||||
*
|
||||
*/
|
||||
public class HoodieException extends RuntimeException {
|
||||
public class HoodieException extends RuntimeException implements Serializable {
|
||||
public HoodieException() {
|
||||
super();
|
||||
}
|
||||
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* 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.exception;
|
||||
|
||||
public class SchemaCompatabilityException extends HoodieException {
|
||||
public SchemaCompatabilityException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public SchemaCompatabilityException(String message, Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
|
||||
public SchemaCompatabilityException(Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user