Refactoring HoodieTableFileSystemView using FileGroups/FileSlices
- Merged all filter* and get* methods - new constructor takes filestatus[] - All existing tests pass - FileGroup is all files that belong to a fileID within a partition - FileSlice is a generation of data and log files, starting at a base commit
This commit is contained in:
@@ -0,0 +1,97 @@
|
||||
/*
|
||||
* Copyright (c) 2017 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 java.io.Serializable;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Within a file group, a slice is a combination of data file written at a commit time
|
||||
* and list of log files, containing changes to the data file from that commit time
|
||||
*/
|
||||
public class FileSlice implements Serializable {
|
||||
|
||||
/**
|
||||
* id of the slice
|
||||
*/
|
||||
private String fileId;
|
||||
|
||||
/**
|
||||
* Point in the timeline, at which the slice was created
|
||||
*/
|
||||
private String baseCommitTime;
|
||||
|
||||
/**
|
||||
* data file, with the compacted data, for this slice
|
||||
*
|
||||
*/
|
||||
private HoodieDataFile dataFile;
|
||||
|
||||
/**
|
||||
* List of appendable log files with real time data
|
||||
* - Sorted with greater log version first
|
||||
* - Always empty for copy_on_write storage.
|
||||
*/
|
||||
private final TreeSet<HoodieLogFile> logFiles;
|
||||
|
||||
public FileSlice(String baseCommitTime, String fileId) {
|
||||
this.fileId = fileId;
|
||||
this.baseCommitTime = baseCommitTime;
|
||||
this.dataFile = null;
|
||||
this.logFiles = new TreeSet<>(HoodieLogFile.getLogVersionComparator());
|
||||
}
|
||||
|
||||
public void setDataFile(HoodieDataFile dataFile) {
|
||||
this.dataFile = dataFile;
|
||||
}
|
||||
|
||||
public void addLogFile(HoodieLogFile logFile) {
|
||||
this.logFiles.add(logFile);
|
||||
}
|
||||
|
||||
public Stream<HoodieLogFile> getLogFiles() {
|
||||
return logFiles.stream();
|
||||
}
|
||||
|
||||
public String getBaseCommitTime() {
|
||||
return baseCommitTime;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public Optional<HoodieDataFile> getDataFile() {
|
||||
return Optional.of(dataFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("FileSlice {");
|
||||
sb.append("baseCommitTime=").append(baseCommitTime);
|
||||
sb.append(", dataFile='").append(dataFile).append('\'');
|
||||
sb.append(", logFiles='").append(logFiles).append('\'');
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -63,7 +63,7 @@ public class HoodieDataFile {
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieDataFile{");
|
||||
final StringBuilder sb = new StringBuilder("HoodieDataFile {");
|
||||
sb.append("fileStatus=").append(fileStatus);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
|
||||
@@ -0,0 +1,246 @@
|
||||
/*
|
||||
* Copyright (c) 2017 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.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.TreeMap;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* A set of data/base files + set of log files, that make up an unit for all operations
|
||||
*/
|
||||
public class HoodieFileGroup implements Serializable {
|
||||
|
||||
public static Comparator<String> getReverseCommitTimeComparator() {
|
||||
return (o1, o2) -> {
|
||||
// reverse the order
|
||||
return o2.compareTo(o1);
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Partition containing the file group.
|
||||
*/
|
||||
private final String partitionPath;
|
||||
|
||||
/**
|
||||
* uniquely identifies the file group
|
||||
*/
|
||||
private final String id;
|
||||
|
||||
/**
|
||||
* Slices of files in this group, sorted with greater commit first.
|
||||
*/
|
||||
private final TreeMap<String, FileSlice> fileSlices;
|
||||
|
||||
/**
|
||||
* Timeline, based on which all getter work
|
||||
*/
|
||||
private final HoodieTimeline timeline;
|
||||
|
||||
/**
|
||||
* The last completed instant, that acts as a high watermark for all
|
||||
* getters
|
||||
*/
|
||||
private final Optional<HoodieInstant> lastInstant;
|
||||
|
||||
public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) {
|
||||
this.partitionPath = partitionPath;
|
||||
this.id = id;
|
||||
this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator());
|
||||
this.timeline = timeline;
|
||||
this.lastInstant = timeline.lastInstant();
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new datafile into the file group
|
||||
*
|
||||
* @param dataFile
|
||||
*/
|
||||
public void addDataFile(HoodieDataFile dataFile) {
|
||||
if (!fileSlices.containsKey(dataFile.getCommitTime())) {
|
||||
fileSlices.put(dataFile.getCommitTime(), new FileSlice(dataFile.getCommitTime(), id));
|
||||
}
|
||||
fileSlices.get(dataFile.getCommitTime()).setDataFile(dataFile);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new log file into the group
|
||||
*
|
||||
* @param logFile
|
||||
*/
|
||||
public void addLogFile(HoodieLogFile logFile) {
|
||||
if (!fileSlices.containsKey(logFile.getBaseCommitTime())) {
|
||||
fileSlices.put(logFile.getBaseCommitTime(), new FileSlice(logFile.getBaseCommitTime(), id));
|
||||
}
|
||||
fileSlices.get(logFile.getBaseCommitTime()).addLogFile(logFile);
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public String getPartitionPath() {
|
||||
return partitionPath;
|
||||
}
|
||||
|
||||
/**
|
||||
* A FileSlice is considered committed, if one of the following is true
|
||||
* - There is a committed data file
|
||||
* - There are some log files, that are based off a commit or delta commit
|
||||
*
|
||||
* @param slice
|
||||
* @return
|
||||
*/
|
||||
private boolean isFileSliceCommitted(FileSlice slice) {
|
||||
String maxCommitTime = lastInstant.get().getTimestamp();
|
||||
return timeline.containsOrBeforeTimelineStarts(slice.getBaseCommitTime()) &&
|
||||
HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(),
|
||||
maxCommitTime,
|
||||
HoodieTimeline.LESSER_OR_EQUAL);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Provides a stream of committed file slices, sorted reverse base commit time.
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Stream<FileSlice> getAllFileSlices() {
|
||||
if (!timeline.empty()) {
|
||||
return fileSlices.entrySet().stream()
|
||||
.map(sliceEntry -> sliceEntry.getValue())
|
||||
.filter(slice -> isFileSliceCommitted(slice));
|
||||
}
|
||||
return Stream.empty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the latest slice - this can contain either
|
||||
*
|
||||
* - just the log files without data file
|
||||
* - (or) data file with 0 or more log files
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Optional<FileSlice> getLatestFileSlice() {
|
||||
// there should always be one
|
||||
return getAllFileSlices().findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain the latest file slice, upto a commitTime i.e <= maxCommitTime
|
||||
*
|
||||
* @param maxCommitTime
|
||||
* @return
|
||||
*/
|
||||
public Optional<FileSlice> getLatestFileSliceBeforeOrOn(String maxCommitTime) {
|
||||
return getAllFileSlices()
|
||||
.filter(slice ->
|
||||
HoodieTimeline.compareTimestamps(slice.getBaseCommitTime(),
|
||||
maxCommitTime,
|
||||
HoodieTimeline.LESSER_OR_EQUAL))
|
||||
.findFirst();
|
||||
}
|
||||
|
||||
public Optional<FileSlice> getLatestFileSliceInRange(List<String> commitRange) {
|
||||
return getAllFileSlices()
|
||||
.filter(slice -> commitRange.contains(slice.getBaseCommitTime()))
|
||||
.findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
* Stream of committed data files, sorted reverse commit time
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Stream<HoodieDataFile> getAllDataFiles() {
|
||||
return getAllFileSlices()
|
||||
.filter(slice -> slice.getDataFile().isPresent())
|
||||
.map(slice -> slice.getDataFile().get());
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the latest committed data file
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
public Optional<HoodieDataFile> getLatestDataFile() {
|
||||
return getAllDataFiles().findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the latest data file, that is <= max commit time
|
||||
*
|
||||
* @param maxCommitTime
|
||||
* @return
|
||||
*/
|
||||
public Optional<HoodieDataFile> getLatestDataFileBeforeOrOn(String maxCommitTime) {
|
||||
return getAllDataFiles()
|
||||
.filter(dataFile ->
|
||||
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
|
||||
maxCommitTime,
|
||||
HoodieTimeline.LESSER_OR_EQUAL))
|
||||
.findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the latest data file, that is contained within the provided commit range.
|
||||
*
|
||||
* @param commitRange
|
||||
* @return
|
||||
*/
|
||||
public Optional<HoodieDataFile> getLatestDataFileInRange(List<String> commitRange) {
|
||||
return getAllDataFiles()
|
||||
.filter(dataFile -> commitRange.contains(dataFile.getCommitTime()))
|
||||
.findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain the latest log file (based on latest committed data file),
|
||||
* currently being appended to
|
||||
*
|
||||
* @return logfile if present, empty if no log file has been opened already.
|
||||
*/
|
||||
public Optional<HoodieLogFile> getLatestLogFile() {
|
||||
Optional<FileSlice> latestSlice = getLatestFileSlice();
|
||||
if (latestSlice.isPresent() && latestSlice.get().getLogFiles().count() > 0) {
|
||||
return latestSlice.get().getLogFiles().findFirst();
|
||||
}
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieFileGroup {");
|
||||
sb.append("id=").append(id);
|
||||
sb.append(", fileSlices='").append(fileSlices).append('\'');
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
@@ -1,5 +1,5 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
* Copyright (c) 2017 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.
|
||||
@@ -7,20 +7,21 @@
|
||||
*
|
||||
* 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.
|
||||
* 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.log;
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
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.Comparator;
|
||||
import java.util.Optional;
|
||||
@@ -96,9 +97,8 @@ public class HoodieLogFile {
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "HoodieLogFile{" + path + '}';
|
||||
return "HoodieLogFile {" + path + '}';
|
||||
}
|
||||
}
|
||||
@@ -16,15 +16,16 @@
|
||||
|
||||
package com.uber.hoodie.common.table;
|
||||
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
@@ -37,27 +38,40 @@ import java.util.stream.Stream;
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public interface TableFileSystemView {
|
||||
|
||||
/**
|
||||
* Stream all the data files for a specific FileId.
|
||||
* This usually has a single RO file and multiple WO files if present.
|
||||
* Stream all the latest data files in the given partition
|
||||
*
|
||||
* @param partitionPath
|
||||
* @param fileId
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestDataFilesForFileId(final String partitionPath,
|
||||
final String fileId);
|
||||
Stream<HoodieDataFile> getLatestDataFiles(String partitionPath);
|
||||
|
||||
/**
|
||||
* Stream all the latest data files, in the file system view
|
||||
*
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestDataFiles();
|
||||
|
||||
/**
|
||||
* Stream all the latest version data files in the given partition
|
||||
* with precondition that commitTime(file) before maxCommitTime
|
||||
*
|
||||
* @param partitionPathStr
|
||||
* @param partitionPath
|
||||
* @param maxCommitTime
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestVersionInPartition(String partitionPathStr,
|
||||
String maxCommitTime);
|
||||
Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
|
||||
String maxCommitTime);
|
||||
|
||||
/**
|
||||
* Stream all the latest data files pass
|
||||
*
|
||||
* @param commitsToReturn
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn);
|
||||
|
||||
/**
|
||||
* Stream all the data file versions grouped by FileId for a given partition
|
||||
@@ -65,45 +79,50 @@ public interface TableFileSystemView {
|
||||
* @param partitionPath
|
||||
* @return
|
||||
*/
|
||||
Stream<List<HoodieDataFile>> getEveryVersionInPartition(String partitionPath);
|
||||
Stream<HoodieDataFile> getAllDataFiles(String partitionPath);
|
||||
|
||||
/**
|
||||
* Stream all the versions from the passed in fileStatus[] with commit times containing in commitsToReturn.
|
||||
*
|
||||
* @param fileStatuses
|
||||
* @param commitsToReturn
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestVersionInRange(FileStatus[] fileStatuses,
|
||||
List<String> commitsToReturn);
|
||||
|
||||
/**
|
||||
* Stream the latest version from the passed in FileStatus[] with commit times less than maxCommitToReturn
|
||||
*
|
||||
* @param fileStatuses
|
||||
* @param maxCommitToReturn
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
|
||||
String maxCommitToReturn);
|
||||
|
||||
/**
|
||||
* Stream latest versions from the passed in FileStatus[].
|
||||
* Similar to calling getLatestVersionsBeforeOrOn(fileStatuses, currentTimeAsCommitTime)
|
||||
*
|
||||
* @param fileStatuses
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestVersions(FileStatus[] fileStatuses);
|
||||
|
||||
/**
|
||||
* Group data files with corresponding delta files
|
||||
* Stream all the latest file slices in the given partition
|
||||
*
|
||||
* @param partitionPath
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
Map<HoodieDataFile, List<HoodieLogFile>> groupLatestDataFileWithLogFiles(String partitionPath) throws IOException;
|
||||
Stream<FileSlice> getLatestFileSlices(String partitionPath);
|
||||
|
||||
/**
|
||||
* Stream all the latest file slices in the given partition
|
||||
* with precondition that commitTime(file) before maxCommitTime
|
||||
*
|
||||
* @param partitionPath
|
||||
* @param maxCommitTime
|
||||
* @return
|
||||
*/
|
||||
Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
|
||||
String maxCommitTime);
|
||||
|
||||
/**
|
||||
* Stream all the latest file slices, in the given range
|
||||
*
|
||||
* @param commitsToReturn
|
||||
* @return
|
||||
*/
|
||||
Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn);
|
||||
|
||||
/**
|
||||
* Stream all the file slices for a given partition, latest or not.
|
||||
*
|
||||
* @param partitionPath
|
||||
* @return
|
||||
*/
|
||||
Stream<FileSlice> getAllFileSlices(String partitionPath);
|
||||
|
||||
/**
|
||||
* Stream all the file groups for a given partition
|
||||
*
|
||||
* @param partitionPath
|
||||
* @return
|
||||
*/
|
||||
Stream<HoodieFileGroup> getAllFileGroups(String partitionPath);
|
||||
|
||||
/**
|
||||
* Get the file Status for the path specified
|
||||
|
||||
@@ -19,6 +19,7 @@ package com.uber.hoodie.common.table.log;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.common.model.HoodieAvroPayload;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
|
||||
|
||||
@@ -16,6 +16,7 @@
|
||||
|
||||
package com.uber.hoodie.common.table.log;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import java.io.Closeable;
|
||||
|
||||
@@ -17,6 +17,8 @@
|
||||
package com.uber.hoodie.common.table.log;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieCorruptBlock;
|
||||
|
||||
@@ -17,6 +17,8 @@
|
||||
package com.uber.hoodie.common.table.log;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFormat.WriterBuilder;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
@@ -56,7 +58,7 @@ public class HoodieLogFormatWriter implements HoodieLogFormat.Writer {
|
||||
* @param sizeThreshold
|
||||
*/
|
||||
HoodieLogFormatWriter(FileSystem fs, HoodieLogFile logFile, Integer bufferSize,
|
||||
Short replication, Long sizeThreshold)
|
||||
Short replication, Long sizeThreshold)
|
||||
throws IOException, InterruptedException {
|
||||
this.fs = fs;
|
||||
this.logFile = logFile;
|
||||
|
||||
@@ -18,19 +18,16 @@ package com.uber.hoodie.common.table.view;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.common.model.HoodieCompactionMetadata;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
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.log.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.util.function.BinaryOperator;
|
||||
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -38,185 +35,243 @@ import org.apache.hadoop.fs.Path;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.Set;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.stream.Collector;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Common abstract implementation for multiple TableFileSystemView Implementations.
|
||||
* 2 possible implementations are ReadOptimizedView and RealtimeView
|
||||
* <p>
|
||||
* Concrete implementations extending this abstract class, should only implement
|
||||
* listDataFilesInPartition which includes files to be included in the view
|
||||
* Common abstract implementation for multiple TableFileSystemView Implementations. 2 possible
|
||||
* implementations are ReadOptimizedView and RealtimeView <p> Concrete implementations extending
|
||||
* this abstract class, should only implement getDataFilesInPartition which includes files to be
|
||||
* included in the view
|
||||
*
|
||||
* @see TableFileSystemView
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public class HoodieTableFileSystemView 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;
|
||||
protected HoodieTimeline visibleActiveTimeline;
|
||||
|
||||
// mapping from partition paths to file groups contained within them
|
||||
protected HashMap<String, List<HoodieFileGroup>> partitionToFileGroupsMap;
|
||||
// mapping from file id to the file group.
|
||||
protected HashMap<String, HoodieFileGroup> fileGroupMap;
|
||||
|
||||
/**
|
||||
* Create a file system view, as of the given timeline
|
||||
*
|
||||
* @param metaClient
|
||||
* @param visibleActiveTimeline
|
||||
*/
|
||||
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
|
||||
HoodieTimeline visibleActiveCommitTimeline) {
|
||||
HoodieTimeline visibleActiveTimeline) {
|
||||
this.metaClient = metaClient;
|
||||
this.fs = metaClient.getFs();
|
||||
this.visibleActiveCommitTimeline = visibleActiveCommitTimeline;
|
||||
this.visibleActiveTimeline = visibleActiveTimeline;
|
||||
this.fileGroupMap = new HashMap<>();
|
||||
this.partitionToFileGroupsMap = new HashMap<>();
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create a file system view, as of the given timeline, with the provided file statuses.
|
||||
*
|
||||
* @param metaClient
|
||||
* @param visibleActiveTimeline
|
||||
* @param fileStatuses
|
||||
*/
|
||||
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
|
||||
HoodieTimeline visibleActiveTimeline,
|
||||
FileStatus[] fileStatuses) {
|
||||
this(metaClient, visibleActiveTimeline);
|
||||
addFilesToView(fileStatuses);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* 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 {
|
||||
throws IOException, ClassNotFoundException {
|
||||
in.defaultReadObject();
|
||||
this.fs = FSUtils.getFs();
|
||||
}
|
||||
|
||||
private void writeObject(java.io.ObjectOutputStream out)
|
||||
throws IOException {
|
||||
throws IOException {
|
||||
out.defaultWriteObject();
|
||||
}
|
||||
|
||||
public Stream<HoodieDataFile> getLatestDataFilesForFileId(final String partitionPath,
|
||||
String fileId) {
|
||||
Optional<HoodieInstant> lastInstant = visibleActiveCommitTimeline.lastInstant();
|
||||
if (lastInstant.isPresent()) {
|
||||
return getLatestVersionInPartition(partitionPath, lastInstant.get().getTimestamp())
|
||||
.filter(hoodieDataFile -> hoodieDataFile.getFileId().equals(fileId));
|
||||
}
|
||||
return Stream.empty();
|
||||
}
|
||||
/**
|
||||
* Adds the provided statuses into the file system view, and also caches it inside this object.
|
||||
*
|
||||
* @param statuses
|
||||
* @return
|
||||
*/
|
||||
private List<HoodieFileGroup> addFilesToView(FileStatus[] statuses) {
|
||||
Map<Pair<String, String>, List<HoodieDataFile>> dataFiles = convertFileStatusesToDataFiles(statuses)
|
||||
.collect(Collectors.groupingBy((dataFile) -> {
|
||||
String partitionPathStr = FSUtils.getRelativePartitionPath(
|
||||
new Path(metaClient.getBasePath()),
|
||||
dataFile.getFileStatus().getPath().getParent());
|
||||
return Pair.of(partitionPathStr , dataFile.getFileId());
|
||||
}));
|
||||
Map<Pair<String, String>, List<HoodieLogFile>> logFiles = convertFileStatusesToLogFiles(statuses)
|
||||
.collect(Collectors.groupingBy((logFile) -> {
|
||||
String partitionPathStr = FSUtils.getRelativePartitionPath(
|
||||
new Path(metaClient.getBasePath()),
|
||||
logFile.getPath().getParent());
|
||||
return Pair.of(partitionPathStr , logFile.getFileId());
|
||||
}));
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestVersionInPartition(String partitionPathStr,
|
||||
String maxCommitTime) {
|
||||
return getLatestVersionsBeforeOrOn(listDataFilesInPartition(partitionPathStr),
|
||||
maxCommitTime);
|
||||
}
|
||||
Set<Pair<String, String>> fileIdSet = new HashSet<>(dataFiles.keySet());
|
||||
fileIdSet.addAll(logFiles.keySet());
|
||||
|
||||
|
||||
@Override
|
||||
public Stream<List<HoodieDataFile>> getEveryVersionInPartition(String partitionPath) {
|
||||
try {
|
||||
if (visibleActiveCommitTimeline.lastInstant().isPresent()) {
|
||||
return getFilesByFileId(listDataFilesInPartition(partitionPath),
|
||||
visibleActiveCommitTimeline.lastInstant().get().getTimestamp());
|
||||
List<HoodieFileGroup> fileGroups = new ArrayList<>();
|
||||
fileIdSet.forEach(pair -> {
|
||||
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), pair.getValue(), visibleActiveTimeline);
|
||||
if (dataFiles.containsKey(pair)) {
|
||||
dataFiles.get(pair).forEach(dataFile -> group.addDataFile(dataFile));
|
||||
}
|
||||
return Stream.empty();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not load all file versions in partition " + partitionPath, e);
|
||||
}
|
||||
if (logFiles.containsKey(pair)) {
|
||||
logFiles.get(pair).forEach(logFile -> group.addLogFile(logFile));
|
||||
}
|
||||
fileGroups.add(group);
|
||||
});
|
||||
|
||||
// add to the cache.
|
||||
fileGroups.forEach(group -> {
|
||||
fileGroupMap.put(group.getId(), group);
|
||||
if (!partitionToFileGroupsMap.containsKey(group.getPartitionPath())) {
|
||||
partitionToFileGroupsMap.put(group.getPartitionPath(), new ArrayList<>());
|
||||
}
|
||||
partitionToFileGroupsMap.get(group.getPartitionPath()).add(group);
|
||||
});
|
||||
|
||||
return fileGroups;
|
||||
}
|
||||
|
||||
protected FileStatus[] listDataFilesInPartition(String partitionPathStr) {
|
||||
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
|
||||
private Stream<HoodieDataFile> convertFileStatusesToDataFiles(FileStatus[] statuses) {
|
||||
Predicate<FileStatus> roFilePredicate = fileStatus ->
|
||||
fileStatus.getPath().getName().contains(metaClient.getTableConfig().getROFileFormat().getFileExtension());
|
||||
return Arrays.stream(statuses).filter(roFilePredicate).map(HoodieDataFile::new);
|
||||
}
|
||||
|
||||
private Stream<HoodieLogFile> convertFileStatusesToLogFiles(FileStatus[] statuses) {
|
||||
Predicate<FileStatus> rtFilePredicate = fileStatus ->
|
||||
fileStatus.getPath().getName().contains(metaClient.getTableConfig().getRTFileFormat().getFileExtension());
|
||||
return Arrays.stream(statuses).filter(rtFilePredicate).map(HoodieLogFile::new);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFiles(final String partitionPath) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> fileGroup.getLatestDataFile())
|
||||
.filter(dataFileOpt -> dataFileOpt.isPresent())
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFiles() {
|
||||
return fileGroupMap.values().stream()
|
||||
.map(fileGroup -> fileGroup.getLatestDataFile())
|
||||
.filter(dataFileOpt -> dataFileOpt.isPresent())
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
|
||||
String maxCommitTime) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> fileGroup.getLatestDataFileBeforeOrOn(maxCommitTime))
|
||||
.filter(dataFileOpt -> dataFileOpt.isPresent())
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
|
||||
return fileGroupMap.values().stream()
|
||||
.map(fileGroup -> fileGroup.getLatestDataFileInRange(commitsToReturn))
|
||||
.filter(dataFileOpt -> dataFileOpt.isPresent())
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> fileGroup.getAllDataFiles())
|
||||
.flatMap(dataFileList -> dataFileList);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> fileGroup.getLatestFileSlice())
|
||||
.filter(dataFileOpt -> dataFileOpt.isPresent())
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
|
||||
.filter(dataFileOpt -> dataFileOpt.isPresent())
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
|
||||
return fileGroupMap.values().stream()
|
||||
.map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
|
||||
.filter(dataFileOpt -> dataFileOpt.isPresent())
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(group -> group.getAllFileSlices())
|
||||
.flatMap(sliceList -> sliceList);
|
||||
}
|
||||
|
||||
/**
|
||||
* Given a partition path, obtain all filegroups within that. All methods, that work at the partition level
|
||||
* go through this.
|
||||
*/
|
||||
@Override
|
||||
public Stream<HoodieFileGroup> getAllFileGroups(String partitionPathStr) {
|
||||
// return any previously fetched groups.
|
||||
if (partitionToFileGroupsMap.containsKey(partitionPathStr)) {
|
||||
return partitionToFileGroupsMap.get(partitionPathStr).stream();
|
||||
}
|
||||
|
||||
try {
|
||||
// Create the path if it does not exist already
|
||||
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
|
||||
FSUtils.createPathIfNotExists(fs, partitionPath);
|
||||
return fs.listStatus(partitionPath, path -> path.getName()
|
||||
.contains(metaClient.getTableConfig().getROFileFormat().getFileExtension()));
|
||||
FileStatus[] statuses = fs.listStatus(partitionPath);
|
||||
List<HoodieFileGroup> fileGroups = addFilesToView(statuses);
|
||||
return fileGroups.stream();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Failed to list data files in partition " + partitionPathStr, e);
|
||||
"Failed to list data files in partition " + partitionPathStr, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestVersionInRange(FileStatus[] fileStatuses,
|
||||
List<String> commitsToReturn) {
|
||||
if (visibleActiveCommitTimeline.empty() || commitsToReturn.isEmpty()) {
|
||||
return Stream.empty();
|
||||
}
|
||||
try {
|
||||
return getFilesByFileId(fileStatuses,
|
||||
visibleActiveCommitTimeline.lastInstant().get().getTimestamp())
|
||||
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
|
||||
for (HoodieDataFile fs : fss) {
|
||||
if (commitsToReturn.contains(fs.getCommitTime())) {
|
||||
return Optional.of(fs);
|
||||
}
|
||||
}
|
||||
return Optional.empty();
|
||||
}).filter(Optional::isPresent).map(Optional::get);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files from commits " + commitsToReturn,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
|
||||
String maxCommitToReturn) {
|
||||
try {
|
||||
if (visibleActiveCommitTimeline.empty()) {
|
||||
return Stream.empty();
|
||||
}
|
||||
return getFilesByFileId(fileStatuses,
|
||||
visibleActiveCommitTimeline.lastInstant().get().getTimestamp())
|
||||
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
|
||||
for (HoodieDataFile fs1 : fss) {
|
||||
if (HoodieTimeline
|
||||
.compareTimestamps(fs1.getCommitTime(), maxCommitToReturn,
|
||||
HoodieTimeline.LESSER_OR_EQUAL)) {
|
||||
return Optional.of(fs1);
|
||||
}
|
||||
}
|
||||
return Optional.empty();
|
||||
}).filter(Optional::isPresent).map(Optional::get);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files for latest version ", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestVersions(FileStatus[] fileStatuses) {
|
||||
try {
|
||||
if (visibleActiveCommitTimeline.empty()) {
|
||||
return Stream.empty();
|
||||
}
|
||||
return getFilesByFileId(fileStatuses,
|
||||
visibleActiveCommitTimeline.lastInstant().get().getTimestamp())
|
||||
.map(statuses -> statuses.get(0));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not filter files for latest version ", e);
|
||||
}
|
||||
}
|
||||
|
||||
public Map<HoodieDataFile, List<HoodieLogFile>> groupLatestDataFileWithLogFiles(
|
||||
String partitionPath) throws IOException {
|
||||
if (metaClient.getTableType() != HoodieTableType.MERGE_ON_READ) {
|
||||
throw new HoodieException("Unsupported table type :" + metaClient.getTableType());
|
||||
}
|
||||
|
||||
// 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(toList(),
|
||||
l -> l.stream().sorted(HoodieLogFile.getLogVersionComparator())
|
||||
.collect(toList())));
|
||||
|
||||
// Filter the delta files by the commit time of the latest base file and collect as a list
|
||||
Optional<HoodieInstant> lastTimestamp = metaClient.getActiveTimeline().lastInstant();
|
||||
return lastTimestamp.map(hoodieInstant -> getLatestVersionInPartition(partitionPath,
|
||||
hoodieInstant.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(Pair::getKey, Pair::getRight))).orElseGet(Maps::newHashMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus getFileStatus(String path) {
|
||||
try {
|
||||
@@ -225,44 +280,4 @@ public class HoodieTableFileSystemView implements TableFileSystemView, Serializa
|
||||
throw new HoodieIOException("Could not get FileStatus on path " + path);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
protected Stream<List<HoodieDataFile>> getFilesByFileId(FileStatus[] files,
|
||||
String maxCommitTime) throws IOException {
|
||||
return groupFilesByFileId(files, maxCommitTime).values().stream();
|
||||
}
|
||||
|
||||
/**
|
||||
* Filters the list of FileStatus to exclude non-committed data files and group by FileID
|
||||
* and sort the actial files by commit time (newer commit first)
|
||||
*
|
||||
* @param files Files to filter and group from
|
||||
* @param maxCommitTime maximum permissible commit time
|
||||
* @return Grouped map by fileId
|
||||
*/
|
||||
private Map<String, List<HoodieDataFile>> groupFilesByFileId(FileStatus[] files,
|
||||
String maxCommitTime) throws IOException {
|
||||
return Arrays.stream(files)
|
||||
// filter out files starting with "."
|
||||
.filter(file -> !file.getPath().getName().startsWith("."))
|
||||
.flatMap(fileStatus -> {
|
||||
HoodieDataFile dataFile = new HoodieDataFile(fileStatus);
|
||||
if (visibleActiveCommitTimeline.containsOrBeforeTimelineStarts(dataFile.getCommitTime())
|
||||
&& HoodieTimeline
|
||||
.compareTimestamps(dataFile.getCommitTime(), maxCommitTime,
|
||||
HoodieTimeline.LESSER_OR_EQUAL)) {
|
||||
return Stream.of(Pair.of(dataFile.getFileId(), dataFile));
|
||||
}
|
||||
return Stream.empty();
|
||||
}).collect(Collectors
|
||||
.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, toSortedFileStatus())));
|
||||
}
|
||||
|
||||
private Collector<HoodieDataFile, ?, List<HoodieDataFile>> toSortedFileStatus() {
|
||||
return Collectors.collectingAndThen(toList(),
|
||||
l -> l.stream().sorted(HoodieDataFile.getCommitTimeComparator())
|
||||
.collect(toList()));
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
||||
@@ -20,7 +20,7 @@ import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.InvalidHoodiePathException;
|
||||
|
||||
Reference in New Issue
Block a user