Timeline Service with Incremental View Syncing support
This commit is contained in:
committed by
vinoth chandar
parent
446f99aa0f
commit
64fec64097
@@ -92,6 +92,11 @@
|
||||
"name":"totalRollbackBlocks",
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
},
|
||||
{
|
||||
"name":"fileSizeInBytes",
|
||||
"type":["null","long"],
|
||||
"default" : null
|
||||
}
|
||||
]
|
||||
}
|
||||
|
||||
@@ -16,15 +16,17 @@
|
||||
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionOperation;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/**
|
||||
* Encapsulates all the needed information about a compaction and make a decision whether this
|
||||
@@ -34,10 +36,9 @@ import java.util.stream.Collectors;
|
||||
public class CompactionOperation implements Serializable {
|
||||
|
||||
private String baseInstantTime;
|
||||
// Using Guava Optional as it is serializable
|
||||
private Optional<String> dataFileCommitTime;
|
||||
private Option<String> dataFileCommitTime;
|
||||
private List<String> deltaFilePaths;
|
||||
private Optional<String> dataFilePath;
|
||||
private Option<String> dataFilePath;
|
||||
private HoodieFileGroupId id;
|
||||
private Map<String, Double> metrics;
|
||||
|
||||
@@ -46,19 +47,30 @@ public class CompactionOperation implements Serializable {
|
||||
public CompactionOperation() {
|
||||
}
|
||||
|
||||
public CompactionOperation(String fileId, String partitionPath, String baseInstantTime,
|
||||
Option<String> dataFileCommitTime, List<String> deltaFilePaths, Option<String> dataFilePath,
|
||||
Map<String, Double> metrics) {
|
||||
this.baseInstantTime = baseInstantTime;
|
||||
this.dataFileCommitTime = dataFileCommitTime;
|
||||
this.deltaFilePaths = deltaFilePaths;
|
||||
this.dataFilePath = dataFilePath;
|
||||
this.id = new HoodieFileGroupId(partitionPath, fileId);
|
||||
this.metrics = metrics;
|
||||
}
|
||||
|
||||
public CompactionOperation(java.util.Optional<HoodieDataFile> dataFile, String partitionPath,
|
||||
List<HoodieLogFile> logFiles, Map<String, Double> metrics) {
|
||||
if (dataFile.isPresent()) {
|
||||
this.baseInstantTime = dataFile.get().getCommitTime();
|
||||
this.dataFilePath = Optional.of(dataFile.get().getPath());
|
||||
this.dataFilePath = Option.of(dataFile.get().getPath());
|
||||
this.id = new HoodieFileGroupId(partitionPath, dataFile.get().getFileId());
|
||||
this.dataFileCommitTime = Optional.of(dataFile.get().getCommitTime());
|
||||
this.dataFileCommitTime = Option.of(dataFile.get().getCommitTime());
|
||||
} else {
|
||||
assert logFiles.size() > 0;
|
||||
this.dataFilePath = Optional.absent();
|
||||
this.dataFilePath = Option.empty();
|
||||
this.baseInstantTime = FSUtils.getBaseCommitTimeFromLogPath(logFiles.get(0).getPath());
|
||||
this.id = new HoodieFileGroupId(partitionPath, FSUtils.getFileIdFromLogPath(logFiles.get(0).getPath()));
|
||||
this.dataFileCommitTime = Optional.absent();
|
||||
this.dataFileCommitTime = Option.empty();
|
||||
}
|
||||
|
||||
this.deltaFilePaths = logFiles.stream().map(s -> s.getPath().toString())
|
||||
@@ -70,7 +82,7 @@ public class CompactionOperation implements Serializable {
|
||||
return baseInstantTime;
|
||||
}
|
||||
|
||||
public Optional<String> getDataFileCommitTime() {
|
||||
public Option<String> getDataFileCommitTime() {
|
||||
return dataFileCommitTime;
|
||||
}
|
||||
|
||||
@@ -78,7 +90,7 @@ public class CompactionOperation implements Serializable {
|
||||
return deltaFilePaths;
|
||||
}
|
||||
|
||||
public Optional<String> getDataFilePath() {
|
||||
public Option<String> getDataFilePath() {
|
||||
return dataFilePath;
|
||||
}
|
||||
|
||||
@@ -106,10 +118,45 @@ public class CompactionOperation implements Serializable {
|
||||
public static CompactionOperation convertFromAvroRecordInstance(HoodieCompactionOperation operation) {
|
||||
CompactionOperation op = new CompactionOperation();
|
||||
op.baseInstantTime = operation.getBaseInstantTime();
|
||||
op.dataFilePath = Optional.fromNullable(operation.getDataFilePath());
|
||||
op.dataFilePath = Option.ofNullable(operation.getDataFilePath());
|
||||
op.dataFileCommitTime =
|
||||
op.dataFilePath.map(p -> FSUtils.getCommitTime(new Path(p).getName()));
|
||||
op.deltaFilePaths = new ArrayList<>(operation.getDeltaFilePaths());
|
||||
op.id = new HoodieFileGroupId(operation.getPartitionPath(), operation.getFileId());
|
||||
op.metrics = operation.getMetrics() == null ? new HashMap<>() : new HashMap<>(operation.getMetrics());
|
||||
return op;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "CompactionOperation{"
|
||||
+ "baseInstantTime='" + baseInstantTime + '\''
|
||||
+ ", dataFileCommitTime=" + dataFileCommitTime
|
||||
+ ", deltaFilePaths=" + deltaFilePaths
|
||||
+ ", dataFilePath=" + dataFilePath
|
||||
+ ", id='" + id + '\''
|
||||
+ ", metrics=" + metrics
|
||||
+ '}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
CompactionOperation operation = (CompactionOperation) o;
|
||||
return Objects.equals(baseInstantTime, operation.baseInstantTime)
|
||||
&& Objects.equals(dataFileCommitTime, operation.dataFileCommitTime)
|
||||
&& Objects.equals(deltaFilePaths, operation.deltaFilePaths)
|
||||
&& Objects.equals(dataFilePath, operation.dataFilePath)
|
||||
&& Objects.equals(id, operation.id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(baseInstantTime, id);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package com.uber.hoodie.common.model;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Stream;
|
||||
@@ -93,14 +94,42 @@ public class FileSlice implements Serializable {
|
||||
return Optional.ofNullable(dataFile);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if there is no data file and no log files. Happens as part of pending compaction
|
||||
* @return
|
||||
*/
|
||||
public boolean isEmpty() {
|
||||
return (dataFile == null) && (logFiles.isEmpty());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("FileSlice {");
|
||||
sb.append("fileGroupId=").append(fileGroupId);
|
||||
sb.append("baseCommitTime=").append(baseInstantTime);
|
||||
sb.append(", baseCommitTime=").append(baseInstantTime);
|
||||
sb.append(", dataFile='").append(dataFile).append('\'');
|
||||
sb.append(", logFiles='").append(logFiles).append('\'');
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
FileSlice slice = (FileSlice) o;
|
||||
return Objects.equals(fileGroupId, slice.fileGroupId)
|
||||
&& Objects.equals(baseInstantTime, slice.baseInstantTime)
|
||||
&& Objects.equals(dataFile, slice.dataFile)
|
||||
&& Objects.equals(logFiles, slice.logFiles);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(fileGroupId, baseInstantTime);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,53 +18,82 @@ package com.uber.hoodie.common.model;
|
||||
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import java.io.Serializable;
|
||||
import java.util.Comparator;
|
||||
import java.util.Objects;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
public class HoodieDataFile implements Serializable {
|
||||
|
||||
private FileStatus fileStatus;
|
||||
private transient FileStatus fileStatus;
|
||||
private final String fullPath;
|
||||
private long fileLen;
|
||||
|
||||
public HoodieDataFile(FileStatus fileStatus) {
|
||||
this.fileStatus = fileStatus;
|
||||
this.fullPath = fileStatus.getPath().toString();
|
||||
this.fileLen = fileStatus.getLen();
|
||||
}
|
||||
|
||||
public HoodieDataFile(String filePath) {
|
||||
this.fileStatus = null;
|
||||
this.fullPath = filePath;
|
||||
this.fileLen = -1;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return FSUtils.getFileId(fileStatus.getPath().getName());
|
||||
return FSUtils.getFileId(getFileName());
|
||||
}
|
||||
|
||||
public String getCommitTime() {
|
||||
return FSUtils.getCommitTime(fileStatus.getPath().getName());
|
||||
return FSUtils.getCommitTime(getFileName());
|
||||
}
|
||||
|
||||
public String getPath() {
|
||||
return fileStatus.getPath().toString();
|
||||
return fullPath;
|
||||
}
|
||||
|
||||
public String getFileName() {
|
||||
return fileStatus.getPath().getName();
|
||||
return new Path(fullPath).getName();
|
||||
}
|
||||
|
||||
public FileStatus getFileStatus() {
|
||||
return fileStatus;
|
||||
}
|
||||
|
||||
public static Comparator<HoodieDataFile> getCommitTimeComparator() {
|
||||
return (o1, o2) -> {
|
||||
// reverse the order
|
||||
return o2.getCommitTime().compareTo(o1.getCommitTime());
|
||||
};
|
||||
public long getFileSize() {
|
||||
return fileLen;
|
||||
}
|
||||
|
||||
public long getFileSize() {
|
||||
return fileStatus.getLen();
|
||||
public void setFileLen(long fileLen) {
|
||||
this.fileLen = fileLen;
|
||||
}
|
||||
|
||||
public long getFileLen() {
|
||||
return fileLen;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
HoodieDataFile dataFile = (HoodieDataFile) o;
|
||||
return Objects.equals(fullPath, dataFile.fullPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(fullPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder sb = new StringBuilder("HoodieDataFile {");
|
||||
sb.append("fileStatus=").append(fileStatus);
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
return "HoodieDataFile{"
|
||||
+ "fullPath=" + fullPath
|
||||
+ ", fileLen=" + fileLen
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ package com.uber.hoodie.common.model;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import java.io.Serializable;
|
||||
import java.util.*;
|
||||
import java.util.stream.Stream;
|
||||
@@ -51,7 +52,7 @@ public class HoodieFileGroup implements Serializable {
|
||||
/**
|
||||
* The last completed instant, that acts as a high watermark for all getters
|
||||
*/
|
||||
private final Optional<HoodieInstant> lastInstant;
|
||||
private final Option<HoodieInstant> lastInstant;
|
||||
|
||||
public HoodieFileGroup(String partitionPath, String id, HoodieTimeline timeline) {
|
||||
this(new HoodieFileGroupId(partitionPath, id), timeline);
|
||||
@@ -61,7 +62,7 @@ public class HoodieFileGroup implements Serializable {
|
||||
this.fileGroupId = fileGroupId;
|
||||
this.fileSlices = new TreeMap<>(HoodieFileGroup.getReverseCommitTimeComparator());
|
||||
this.timeline = timeline;
|
||||
this.lastInstant = timeline.lastInstant();
|
||||
this.lastInstant = Option.fromJavaOptional(timeline.lastInstant());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -151,6 +152,13 @@ public class HoodieFileGroup implements Serializable {
|
||||
return getAllFileSlices().findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the latest data file
|
||||
*/
|
||||
public Optional<HoodieDataFile> getLatestDataFile() {
|
||||
return getAllDataFiles().findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain the latest file slice, upto a commitTime i.e <= maxCommitTime
|
||||
*/
|
||||
@@ -197,7 +205,20 @@ public class HoodieFileGroup implements Serializable {
|
||||
final StringBuilder sb = new StringBuilder("HoodieFileGroup {");
|
||||
sb.append("id=").append(fileGroupId);
|
||||
sb.append(", fileSlices='").append(fileSlices).append('\'');
|
||||
sb.append(", lastInstant='").append(lastInstant).append('\'');
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public void addFileSlice(FileSlice slice) {
|
||||
fileSlices.put(slice.getBaseInstantTime(), slice);
|
||||
}
|
||||
|
||||
public Stream<FileSlice> getAllRawFileSlices() {
|
||||
return fileSlices.values().stream();
|
||||
}
|
||||
|
||||
public HoodieTimeline getTimeline() {
|
||||
return timeline;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,14 +22,14 @@ import com.uber.hoodie.common.util.FSUtils;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Comparator;
|
||||
import java.util.Optional;
|
||||
import java.util.Objects;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/**
|
||||
* Abstracts a single log file. Contains methods to extract metadata like the fileId, version and
|
||||
* extension from the log file path.
|
||||
* Abstracts a single log file. Contains methods to extract metadata like the fileId, version and extension from the log
|
||||
* file path.
|
||||
* <p>
|
||||
* Also contains logic to roll-over the log file
|
||||
*/
|
||||
@@ -38,54 +38,68 @@ public class HoodieLogFile implements Serializable {
|
||||
public static final String DELTA_EXTENSION = ".log";
|
||||
public static final Integer LOGFILE_BASE_VERSION = 1;
|
||||
|
||||
private final Path path;
|
||||
private Optional<FileStatus> fileStatus;
|
||||
private transient FileStatus fileStatus;
|
||||
private final String pathStr;
|
||||
private long fileLen;
|
||||
|
||||
public HoodieLogFile(FileStatus fileStatus) {
|
||||
this(fileStatus.getPath());
|
||||
this.fileStatus = Optional.of(fileStatus);
|
||||
this.fileStatus = fileStatus;
|
||||
this.pathStr = fileStatus.getPath().toString();
|
||||
this.fileLen = fileStatus.getLen();
|
||||
}
|
||||
|
||||
public HoodieLogFile(Path logPath) {
|
||||
this.path = logPath;
|
||||
this.fileStatus = Optional.empty();
|
||||
this.fileStatus = null;
|
||||
this.pathStr = logPath.toString();
|
||||
this.fileLen = 0;
|
||||
}
|
||||
|
||||
public HoodieLogFile(String logPathStr) {
|
||||
this.fileStatus = null;
|
||||
this.pathStr = logPathStr;
|
||||
this.fileLen = -1;
|
||||
}
|
||||
|
||||
public String getFileId() {
|
||||
return FSUtils.getFileIdFromLogPath(path);
|
||||
return FSUtils.getFileIdFromLogPath(getPath());
|
||||
}
|
||||
|
||||
public String getBaseCommitTime() {
|
||||
return FSUtils.getBaseCommitTimeFromLogPath(path);
|
||||
return FSUtils.getBaseCommitTimeFromLogPath(getPath());
|
||||
}
|
||||
|
||||
public int getLogVersion() {
|
||||
return FSUtils.getFileVersionFromLog(path);
|
||||
return FSUtils.getFileVersionFromLog(getPath());
|
||||
}
|
||||
|
||||
public String getFileExtension() {
|
||||
return FSUtils.getFileExtensionFromLog(path);
|
||||
return FSUtils.getFileExtensionFromLog(getPath());
|
||||
}
|
||||
|
||||
public Path getPath() {
|
||||
return path;
|
||||
return new Path(pathStr);
|
||||
}
|
||||
|
||||
public String getFileName() {
|
||||
return path.getName();
|
||||
return getPath().getName();
|
||||
}
|
||||
|
||||
public Optional<FileStatus> getFileStatus() {
|
||||
public void setFileLen(long fileLen) {
|
||||
this.fileLen = fileLen;
|
||||
}
|
||||
|
||||
public long getFileSize() {
|
||||
return fileLen;
|
||||
}
|
||||
|
||||
public FileStatus getFileStatus() {
|
||||
return fileStatus;
|
||||
}
|
||||
|
||||
public Optional<Long> getFileSize() {
|
||||
return fileStatus.map(FileStatus::getLen);
|
||||
}
|
||||
|
||||
public HoodieLogFile rollOver(FileSystem fs) throws IOException {
|
||||
String fileId = getFileId();
|
||||
String baseCommitTime = getBaseCommitTime();
|
||||
Path path = getPath();
|
||||
String extension = "." + FSUtils.getFileExtensionFromLog(path);
|
||||
int newVersion = FSUtils
|
||||
.computeNextLogVersion(fs, path.getParent(), fileId,
|
||||
@@ -95,7 +109,16 @@ public class HoodieLogFile implements Serializable {
|
||||
}
|
||||
|
||||
public static Comparator<HoodieLogFile> getBaseInstantAndLogVersionComparator() {
|
||||
return (o1, o2) -> {
|
||||
return new BaseInstantAndLogVersionComparator();
|
||||
}
|
||||
|
||||
/**
|
||||
* Comparator to order log-files
|
||||
*/
|
||||
private static class BaseInstantAndLogVersionComparator implements Comparator<HoodieLogFile>, Serializable {
|
||||
|
||||
@Override
|
||||
public int compare(HoodieLogFile o1, HoodieLogFile o2) {
|
||||
String baseInstantTime1 = o1.getBaseCommitTime();
|
||||
String baseInstantTime2 = o2.getBaseCommitTime();
|
||||
if (baseInstantTime1.equals(baseInstantTime2)) {
|
||||
@@ -104,7 +127,7 @@ public class HoodieLogFile implements Serializable {
|
||||
}
|
||||
// reverse the order by base-commits
|
||||
return baseInstantTime2.compareTo(baseInstantTime1);
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -116,16 +139,19 @@ public class HoodieLogFile implements Serializable {
|
||||
return false;
|
||||
}
|
||||
HoodieLogFile that = (HoodieLogFile) o;
|
||||
return path != null ? path.equals(that.path) : that.path == null;
|
||||
return Objects.equals(pathStr, that.pathStr);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return path != null ? path.hashCode() : 0;
|
||||
return Objects.hash(pathStr);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "HoodieLogFile {" + path + '}';
|
||||
return "HoodieLogFile{"
|
||||
+ "pathStr='" + pathStr + '\''
|
||||
+ ", fileLen=" + fileLen
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
|
||||
@@ -133,6 +133,11 @@ public class HoodieWriteStat implements Serializable {
|
||||
*/
|
||||
private long totalRollbackBlocks;
|
||||
|
||||
/**
|
||||
* File Size as of close
|
||||
*/
|
||||
private long fileSizeInBytes;
|
||||
|
||||
@Nullable
|
||||
@JsonIgnore
|
||||
private RuntimeStats runtimeStats;
|
||||
@@ -285,6 +290,14 @@ public class HoodieWriteStat implements Serializable {
|
||||
this.totalRollbackBlocks = totalRollbackBlocks;
|
||||
}
|
||||
|
||||
public long getFileSizeInBytes() {
|
||||
return fileSizeInBytes;
|
||||
}
|
||||
|
||||
public void setFileSizeInBytes(long fileSizeInBytes) {
|
||||
this.fileSizeInBytes = fileSizeInBytes;
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public RuntimeStats getRuntimeStats() {
|
||||
return runtimeStats;
|
||||
|
||||
@@ -77,6 +77,10 @@ public class HoodieTableConfig implements Serializable {
|
||||
this.props = props;
|
||||
}
|
||||
|
||||
public HoodieTableConfig(Properties props) {
|
||||
this.props = props;
|
||||
}
|
||||
|
||||
/**
|
||||
* For serailizing and de-serializing
|
||||
*
|
||||
|
||||
@@ -194,6 +194,16 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
return activeTimeline;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reload ActiveTimeline and cache
|
||||
*
|
||||
* @return Active instants timeline
|
||||
*/
|
||||
public synchronized HoodieActiveTimeline reloadActiveTimeline() {
|
||||
activeTimeline = new HoodieActiveTimeline(this);
|
||||
return activeTimeline;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the archived commits as a timeline. This is costly operation, as all data from the archived
|
||||
* files are read. This should not be used, unless for historical debugging purposes
|
||||
@@ -406,4 +416,20 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
sb.append('}');
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
public void setBasePath(String basePath) {
|
||||
this.basePath = basePath;
|
||||
}
|
||||
|
||||
public void setMetaPath(String metaPath) {
|
||||
this.metaPath = metaPath;
|
||||
}
|
||||
|
||||
public void setActiveTimeline(HoodieActiveTimeline activeTimeline) {
|
||||
this.activeTimeline = activeTimeline;
|
||||
}
|
||||
|
||||
public void setTableConfig(HoodieTableConfig tableConfig) {
|
||||
this.tableConfig = tableConfig;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,6 +23,7 @@ import com.uber.hoodie.common.util.StringUtils;
|
||||
import java.io.Serializable;
|
||||
import java.util.Optional;
|
||||
import java.util.function.BiPredicate;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
@@ -70,6 +71,8 @@ public interface HoodieTimeline extends Serializable {
|
||||
String INFLIGHT_RESTORE_EXTENSION = "." + RESTORE_ACTION + INFLIGHT_EXTENSION;
|
||||
String RESTORE_EXTENSION = "." + RESTORE_ACTION;
|
||||
|
||||
String INVALID_INSTANT_TS = "0";
|
||||
|
||||
/**
|
||||
* Filter this timeline to just include the in-flights
|
||||
*
|
||||
@@ -116,6 +119,11 @@ public interface HoodieTimeline extends Serializable {
|
||||
*/
|
||||
HoodieTimeline findInstantsAfter(String commitTime, int numCommits);
|
||||
|
||||
/**
|
||||
* Custom Filter of Instants
|
||||
*/
|
||||
HoodieTimeline filter(Predicate<HoodieInstant> filter);
|
||||
|
||||
/**
|
||||
* If the timeline has any instants
|
||||
*
|
||||
@@ -143,6 +151,13 @@ public interface HoodieTimeline extends Serializable {
|
||||
*/
|
||||
Optional<HoodieInstant> lastInstant();
|
||||
|
||||
|
||||
/**
|
||||
* Get hash of timeline
|
||||
* @return
|
||||
*/
|
||||
String getTimelineHash();
|
||||
|
||||
/**
|
||||
* @return nth completed instant going back from the last completed instant
|
||||
*/
|
||||
|
||||
@@ -0,0 +1,44 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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;
|
||||
|
||||
/*
|
||||
* A consolidated file-system view interface exposing both realtime and read-optimized views along with
|
||||
* update operations.
|
||||
*/
|
||||
public interface SyncableFileSystemView extends TableFileSystemView, TableFileSystemView.ReadOptimizedView,
|
||||
TableFileSystemView.RealtimeView {
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Allow View to release resources and close
|
||||
*/
|
||||
void close();
|
||||
|
||||
/**
|
||||
* Reset View so that they can be refreshed
|
||||
*/
|
||||
void reset();
|
||||
|
||||
/**
|
||||
* Read the latest timeline and refresh the file-system view to match the current state of the file-system.
|
||||
* The refresh can either be done incrementally (from reading file-slices in metadata files) or from scratch by
|
||||
* reseting view storage
|
||||
*/
|
||||
void sync();
|
||||
}
|
||||
@@ -16,9 +16,13 @@
|
||||
|
||||
package com.uber.hoodie.common.table;
|
||||
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@@ -30,55 +34,69 @@ import java.util.stream.Stream;
|
||||
public interface TableFileSystemView {
|
||||
|
||||
/**
|
||||
* ReadOptimizedView - methods to provide a view of columnar data files only.
|
||||
* ReadOptimizedView with methods to only access latest version of file for the instant(s) passed.
|
||||
*/
|
||||
interface ReadOptimizedView {
|
||||
interface ReadOptimizedViewWithLatestSlice {
|
||||
|
||||
/**
|
||||
* Stream all the latest data files in the given partition
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestDataFiles(String partitionPath);
|
||||
|
||||
/**
|
||||
* Get Latest data file for a partition and file-Id
|
||||
*/
|
||||
Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId);
|
||||
|
||||
/**
|
||||
* Stream all the latest data files, in the file system view
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestDataFiles();
|
||||
|
||||
/**
|
||||
* Stream all the latest version data files in the given partition with precondition that
|
||||
* commitTime(file) before maxCommitTime
|
||||
* Stream all the latest version data files in the given partition with precondition that commitTime(file) before
|
||||
* maxCommitTime
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
|
||||
String maxCommitTime);
|
||||
|
||||
/**
|
||||
* Stream all the latest version data files in the given partition with precondition that
|
||||
* instant time of file matches passed in instant time.
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestDataFilesOn(String partitionPath, String instantTime);
|
||||
|
||||
/**
|
||||
* Stream all the latest data files pass
|
||||
*/
|
||||
Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn);
|
||||
}
|
||||
|
||||
/**
|
||||
* ReadOptimizedView - methods to provide a view of columnar data files only.
|
||||
*/
|
||||
interface ReadOptimizedView extends ReadOptimizedViewWithLatestSlice {
|
||||
/**
|
||||
* Stream all the data file versions grouped by FileId for a given partition
|
||||
*/
|
||||
Stream<HoodieDataFile> getAllDataFiles(String partitionPath);
|
||||
|
||||
/**
|
||||
* Get the version of data file matching the instant time in the given partition
|
||||
*/
|
||||
Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* RealtimeView - methods to access a combination of columnar data files + log files with real
|
||||
* time data.
|
||||
* RealtimeView with methods to only access latest version of file-slice for the instant(s) passed.
|
||||
*/
|
||||
interface RealtimeView {
|
||||
interface RealtimeViewWithLatestSlice {
|
||||
|
||||
/**
|
||||
* Stream all the latest file slices in the given partition
|
||||
*/
|
||||
Stream<FileSlice> getLatestFileSlices(String partitionPath);
|
||||
|
||||
/**
|
||||
* Get Latest File Slice for a given fileId in a given partition
|
||||
*/
|
||||
Option<FileSlice> getLatestFileSlice(String partitionPath, String fileId);
|
||||
|
||||
/**
|
||||
* Stream all the latest uncompacted file slices in the given partition
|
||||
*/
|
||||
@@ -106,15 +124,39 @@ public interface TableFileSystemView {
|
||||
* Stream all the latest file slices, in the given range
|
||||
*/
|
||||
Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn);
|
||||
}
|
||||
|
||||
/**
|
||||
* RealtimeView - methods to access a combination of columnar data files + log files with real time data.
|
||||
*/
|
||||
interface RealtimeView extends RealtimeViewWithLatestSlice {
|
||||
|
||||
/**
|
||||
* Stream all the file slices for a given partition, latest or not.
|
||||
*/
|
||||
Stream<FileSlice> getAllFileSlices(String partitionPath);
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Stream all the file groups for a given partition
|
||||
*/
|
||||
Stream<HoodieFileGroup> getAllFileGroups(String partitionPath);
|
||||
|
||||
/**
|
||||
* Return Pending Compaction Operations
|
||||
*
|
||||
* @return Pair<Pair<InstantTime,CompactionOperation>>
|
||||
*/
|
||||
Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations();
|
||||
|
||||
/**
|
||||
* Last Known Instant on which the view is built
|
||||
*/
|
||||
Option<HoodieInstant> getLastInstant();
|
||||
|
||||
/**
|
||||
* Timeline corresponding to the view
|
||||
*/
|
||||
HoodieTimeline getTimeline();
|
||||
}
|
||||
|
||||
@@ -17,6 +17,7 @@
|
||||
package com.uber.hoodie.common.table.timeline;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
@@ -58,7 +59,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION}));
|
||||
|
||||
private static final transient Logger log = LogManager.getLogger(HoodieActiveTimeline.class);
|
||||
private HoodieTableMetaClient metaClient;
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
|
||||
/**
|
||||
* Returns next commit time in the {@link #COMMIT_FORMATTER} format.
|
||||
@@ -71,9 +72,9 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
// Filter all the filter in the metapath and include only the extensions passed and
|
||||
// convert them into HoodieInstant
|
||||
try {
|
||||
this.instants = HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
|
||||
new Path(metaClient.getMetaPath()), includedExtensions);
|
||||
log.info("Loaded instants " + instants);
|
||||
this.setInstants(HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
|
||||
new Path(metaClient.getMetaPath()), includedExtensions));
|
||||
log.info("Loaded instants " + getInstants());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to scan metadata", e);
|
||||
}
|
||||
@@ -85,7 +86,9 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
}
|
||||
|
||||
public HoodieActiveTimeline(HoodieTableMetaClient metaClient) {
|
||||
this(metaClient, VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
|
||||
this(metaClient,
|
||||
new ImmutableSet.Builder<String>()
|
||||
.addAll(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE).build());
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -158,7 +161,7 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
* @param actions actions allowed in the timeline
|
||||
*/
|
||||
public HoodieTimeline getTimelineOfActions(Set<String> actions) {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(s -> actions.contains(s.getAction())),
|
||||
return new HoodieDefaultTimeline(getInstants().filter(s -> actions.contains(s.getAction())),
|
||||
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
|
||||
}
|
||||
|
||||
@@ -195,9 +198,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline {
|
||||
(Function<HoodieInstant, Optional<byte[]>> & Serializable) this::getInstantDetails);
|
||||
}
|
||||
|
||||
|
||||
protected Stream<HoodieInstant> filterInstantsByAction(String action) {
|
||||
return instants.stream().filter(s -> s.getAction().equals(action));
|
||||
return getInstants().filter(s -> s.getAction().equals(action));
|
||||
}
|
||||
|
||||
public void createInflight(HoodieInstant instant) {
|
||||
|
||||
@@ -61,9 +61,9 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
|
||||
// This is okay because only tooling will load the archived commit timeline today
|
||||
readCommits.put(key.toString(), Arrays.copyOf(val.getBytes(), val.getLength()));
|
||||
}
|
||||
this.instants = readCommits.keySet().stream().map(
|
||||
this.setInstants(readCommits.keySet().stream().map(
|
||||
s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)).collect(
|
||||
Collectors.toList());
|
||||
Collectors.toList()));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(
|
||||
"Could not load archived commit timeline from path " + archiveLogPath, e);
|
||||
|
||||
@@ -17,11 +17,17 @@
|
||||
package com.uber.hoodie.common.table.timeline;
|
||||
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.util.StringUtils;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.security.MessageDigest;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.commons.codec.binary.Hex;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -36,13 +42,29 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
|
||||
private static final transient Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
|
||||
|
||||
protected Function<HoodieInstant, Optional<byte[]>> details;
|
||||
protected List<HoodieInstant> instants;
|
||||
private static final String HASHING_ALGORITHM = "SHA-256";
|
||||
|
||||
protected transient Function<HoodieInstant, Optional<byte[]>> details;
|
||||
private List<HoodieInstant> instants;
|
||||
private String timelineHash;
|
||||
|
||||
public HoodieDefaultTimeline(Stream<HoodieInstant> instants,
|
||||
Function<HoodieInstant, Optional<byte[]>> details) {
|
||||
this.instants = instants.collect(Collectors.toList());
|
||||
this.details = details;
|
||||
setInstants(instants.collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
public void setInstants(List<HoodieInstant> instants) {
|
||||
this.instants = instants;
|
||||
final MessageDigest md;
|
||||
try {
|
||||
md = MessageDigest.getInstance(HASHING_ALGORITHM);
|
||||
this.instants.stream().forEach(i -> md.update(
|
||||
StringUtils.joinUsingDelim("_", i.getTimestamp(), i.getAction(), i.getState().name()).getBytes()));
|
||||
} catch (NoSuchAlgorithmException nse) {
|
||||
throw new HoodieException(nse);
|
||||
}
|
||||
this.timelineHash = new String(Hex.encodeHex(md.digest()));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -101,6 +123,11 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
.limit(numCommits), details);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline filter(Predicate<HoodieInstant> filter) {
|
||||
return new HoodieDefaultTimeline(instants.stream().filter(filter), details);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean empty() {
|
||||
return !instants.stream().findFirst().isPresent();
|
||||
@@ -148,6 +175,11 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
|| isBeforeTimelineStarts(instant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getTimelineHash() {
|
||||
return timelineHash;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieInstant> getInstants() {
|
||||
return instants.stream();
|
||||
@@ -160,7 +192,6 @@ public class HoodieDefaultTimeline implements HoodieTimeline {
|
||||
&& HoodieTimeline.compareTimestamps(instant, firstCommit.get().getTimestamp(), LESSER);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Optional<byte[]> getInstantDetails(HoodieInstant instant) {
|
||||
return details.apply(instant);
|
||||
|
||||
@@ -39,7 +39,9 @@ public class HoodieInstant implements Serializable {
|
||||
// Inflight instant
|
||||
INFLIGHT,
|
||||
// Committed instant
|
||||
COMPLETED
|
||||
COMPLETED,
|
||||
// Invalid instant
|
||||
INVALID
|
||||
}
|
||||
|
||||
private State state = State.COMPLETED;
|
||||
|
||||
@@ -0,0 +1,75 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.timeline.dto;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class CompactionOpDTO {
|
||||
|
||||
@JsonProperty("baseInstant")
|
||||
String baseInstantTime;
|
||||
|
||||
@JsonProperty("compactionInstant")
|
||||
String compactionInstantTime;
|
||||
|
||||
@JsonProperty("dataFileInstant")
|
||||
private String dataFileCommitTime;
|
||||
|
||||
@JsonProperty("deltaFiles")
|
||||
private List<String> deltaFilePaths;
|
||||
|
||||
@JsonProperty("baseFile")
|
||||
private String dataFilePath;
|
||||
|
||||
@JsonProperty("id")
|
||||
private String fileId;
|
||||
|
||||
@JsonProperty("partition")
|
||||
private String partitionPath;
|
||||
|
||||
@JsonProperty("metrics")
|
||||
private Map<String, Double> metrics;
|
||||
|
||||
public static CompactionOpDTO fromCompactionOperation(String compactionInstantTime,
|
||||
CompactionOperation op) {
|
||||
CompactionOpDTO dto = new CompactionOpDTO();
|
||||
dto.fileId = op.getFileId();
|
||||
dto.compactionInstantTime = compactionInstantTime;
|
||||
dto.baseInstantTime = op.getBaseInstantTime();
|
||||
dto.dataFileCommitTime = op.getDataFileCommitTime().orElse(null);
|
||||
dto.dataFilePath = op.getDataFilePath().orElse(null);
|
||||
dto.deltaFilePaths = new ArrayList<>(op.getDeltaFilePaths());
|
||||
dto.partitionPath = op.getPartitionPath();
|
||||
dto.metrics = op.getMetrics() == null ? new HashMap<>() : new HashMap<>(op.getMetrics());
|
||||
return dto;
|
||||
}
|
||||
|
||||
public static Pair<String, CompactionOperation> toCompactionOperation(CompactionOpDTO dto) {
|
||||
return Pair.of(dto.compactionInstantTime, new CompactionOperation(dto.fileId, dto.partitionPath,
|
||||
dto.baseInstantTime, Option.ofNullable(dto.dataFileCommitTime), dto.deltaFilePaths,
|
||||
Option.ofNullable(dto.dataFilePath), dto.metrics));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,60 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.timeline.dto;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class DataFileDTO {
|
||||
|
||||
@JsonProperty("fileStatus")
|
||||
private FileStatusDTO fileStatus;
|
||||
@JsonProperty("fullPath")
|
||||
private String fullPath;
|
||||
@JsonProperty("fileLen")
|
||||
private long fileLen;
|
||||
|
||||
public static HoodieDataFile toHoodieDataFile(DataFileDTO dto) {
|
||||
if (null == dto) {
|
||||
return null;
|
||||
}
|
||||
|
||||
HoodieDataFile dataFile = null;
|
||||
if (null != dto.fileStatus) {
|
||||
dataFile = new HoodieDataFile(FileStatusDTO.toFileStatus(dto.fileStatus));
|
||||
} else {
|
||||
dataFile = new HoodieDataFile(dto.fullPath);
|
||||
dataFile.setFileLen(dto.fileLen);
|
||||
}
|
||||
return dataFile;
|
||||
}
|
||||
|
||||
public static DataFileDTO fromHoodieDataFile(HoodieDataFile dataFile) {
|
||||
if (null == dataFile) {
|
||||
return null;
|
||||
}
|
||||
|
||||
DataFileDTO dto = new DataFileDTO();
|
||||
dto.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus());
|
||||
dto.fullPath = dataFile.getPath();
|
||||
dto.fileLen = dataFile.getFileLen();
|
||||
return dto;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,61 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.timeline.dto;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import java.io.Serializable;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
|
||||
/**
|
||||
* A serializable FS Permission
|
||||
*/
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class FSPermissionDTO implements Serializable {
|
||||
|
||||
@JsonProperty("useraction")
|
||||
FsAction useraction;
|
||||
|
||||
@JsonProperty("groupaction")
|
||||
FsAction groupaction;
|
||||
|
||||
@JsonProperty("otheraction")
|
||||
FsAction otheraction;
|
||||
|
||||
@JsonProperty("stickyBit")
|
||||
boolean stickyBit;
|
||||
|
||||
public static FSPermissionDTO fromFsPermission(FsPermission permission) {
|
||||
if (null == permission) {
|
||||
return null;
|
||||
}
|
||||
FSPermissionDTO dto = new FSPermissionDTO();
|
||||
dto.useraction = permission.getUserAction();
|
||||
dto.groupaction = permission.getGroupAction();
|
||||
dto.otheraction = permission.getOtherAction();
|
||||
dto.stickyBit = permission.getStickyBit();
|
||||
return dto;
|
||||
}
|
||||
|
||||
public static FsPermission fromFsPermissionDTO(FSPermissionDTO dto) {
|
||||
if (null == dto) {
|
||||
return null;
|
||||
}
|
||||
return new FsPermission(dto.useraction, dto.groupaction, dto.otheraction, dto.stickyBit);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.timeline.dto;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class FileGroupDTO {
|
||||
|
||||
@JsonProperty("partition")
|
||||
String partition;
|
||||
|
||||
@JsonProperty("fileId")
|
||||
String id;
|
||||
|
||||
@JsonProperty("slices")
|
||||
List<FileSliceDTO> slices;
|
||||
|
||||
@JsonProperty("timeline")
|
||||
TimelineDTO timeline;
|
||||
|
||||
public static FileGroupDTO fromFileGroup(HoodieFileGroup fileGroup) {
|
||||
FileGroupDTO dto = new FileGroupDTO();
|
||||
dto.partition = fileGroup.getPartitionPath();
|
||||
dto.id = fileGroup.getFileGroupId().getFileId();
|
||||
dto.slices = fileGroup.getAllRawFileSlices().map(FileSliceDTO::fromFileSlice).collect(Collectors.toList());
|
||||
dto.timeline = TimelineDTO.fromTimeline(fileGroup.getTimeline());
|
||||
return dto;
|
||||
}
|
||||
|
||||
public static HoodieFileGroup toFileGroup(FileGroupDTO dto, HoodieTableMetaClient metaClient) {
|
||||
HoodieFileGroup fileGroup = new HoodieFileGroup(dto.partition, dto.id,
|
||||
TimelineDTO.toTimeline(dto.timeline, metaClient));
|
||||
dto.slices.stream().map(FileSliceDTO::toFileSlice).forEach(fileSlice -> fileGroup.addFileSlice(fileSlice));
|
||||
return fileGroup;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.timeline.dto;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class FilePathDTO {
|
||||
|
||||
@JsonProperty("uri")
|
||||
private String uri;
|
||||
|
||||
public static FilePathDTO fromPath(Path path) {
|
||||
if (null == path) {
|
||||
return null;
|
||||
}
|
||||
FilePathDTO dto = new FilePathDTO();
|
||||
dto.uri = path.toUri().toString();
|
||||
return dto;
|
||||
}
|
||||
|
||||
public static Path toPath(FilePathDTO dto) {
|
||||
if (null == dto) {
|
||||
return null;
|
||||
}
|
||||
|
||||
try {
|
||||
return new Path(new URI(dto.uri));
|
||||
} catch (URISyntaxException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.timeline.dto;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class FileSliceDTO {
|
||||
|
||||
@JsonProperty("dataFile")
|
||||
DataFileDTO dataFile;
|
||||
@JsonProperty("logFiles")
|
||||
List<LogFileDTO> logFiles;
|
||||
@JsonProperty("partition")
|
||||
private String partitionPath;
|
||||
@JsonProperty("fileId")
|
||||
private String fileId;
|
||||
@JsonProperty("baseInstant")
|
||||
private String baseInstantTime;
|
||||
|
||||
public static FileSliceDTO fromFileSlice(FileSlice slice) {
|
||||
FileSliceDTO dto = new FileSliceDTO();
|
||||
dto.partitionPath = slice.getPartitionPath();
|
||||
dto.baseInstantTime = slice.getBaseInstantTime();
|
||||
dto.fileId = slice.getFileId();
|
||||
dto.dataFile = slice.getDataFile().map(DataFileDTO::fromHoodieDataFile).orElse(null);
|
||||
dto.logFiles = slice.getLogFiles().map(LogFileDTO::fromHoodieLogFile).collect(Collectors.toList());
|
||||
return dto;
|
||||
}
|
||||
|
||||
public static FileSlice toFileSlice(FileSliceDTO dto) {
|
||||
FileSlice slice = new FileSlice(dto.partitionPath, dto.baseInstantTime, dto.fileId);
|
||||
slice.setDataFile(DataFileDTO.toHoodieDataFile(dto.dataFile));
|
||||
dto.logFiles.stream().forEach(lf -> slice.addLogFile(LogFileDTO.toHoodieLogFile(lf)));
|
||||
return slice;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,97 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.timeline.dto;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class FileStatusDTO {
|
||||
|
||||
@JsonProperty("path")
|
||||
FilePathDTO path;
|
||||
@JsonProperty("length")
|
||||
long length;
|
||||
@JsonProperty("isdir")
|
||||
boolean isdir;
|
||||
@JsonProperty("blockReplication")
|
||||
short blockReplication;
|
||||
@JsonProperty("blocksize")
|
||||
long blocksize;
|
||||
@JsonProperty("modificationTime")
|
||||
long modificationTime;
|
||||
@JsonProperty("accessTime")
|
||||
long accessTime;
|
||||
@JsonProperty("permission")
|
||||
FSPermissionDTO permission;
|
||||
@JsonProperty("owner")
|
||||
String owner;
|
||||
@JsonProperty("group")
|
||||
String group;
|
||||
@JsonProperty("symlink")
|
||||
FilePathDTO symlink;
|
||||
|
||||
public static FileStatusDTO fromFileStatus(FileStatus fileStatus) {
|
||||
if (null == fileStatus) {
|
||||
return null;
|
||||
}
|
||||
|
||||
FileStatusDTO dto = new FileStatusDTO();
|
||||
try {
|
||||
dto.path = FilePathDTO.fromPath(fileStatus.getPath());
|
||||
dto.length = fileStatus.getLen();
|
||||
dto.isdir = fileStatus.isDirectory();
|
||||
dto.blockReplication = fileStatus.getReplication();
|
||||
dto.blocksize = fileStatus.getBlockSize();
|
||||
dto.modificationTime = fileStatus.getModificationTime();
|
||||
dto.accessTime = fileStatus.getModificationTime();
|
||||
dto.symlink = fileStatus.isSymlink() ? FilePathDTO.fromPath(fileStatus.getSymlink()) : null;
|
||||
safeReadAndSetMetadata(dto, fileStatus);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieException(ioe);
|
||||
}
|
||||
return dto;
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to safely handle FileStatus calls which might fail on some FileSystem implementation.
|
||||
* (DeprecatedLocalFileSystem)
|
||||
*/
|
||||
private static void safeReadAndSetMetadata(FileStatusDTO dto, FileStatus fileStatus) {
|
||||
try {
|
||||
dto.owner = fileStatus.getOwner();
|
||||
dto.group = fileStatus.getGroup();
|
||||
dto.permission = FSPermissionDTO.fromFsPermission(fileStatus.getPermission());
|
||||
} catch (IllegalArgumentException ie) {
|
||||
// Deprecated File System (testing) does not work well with this call
|
||||
// skipping
|
||||
}
|
||||
}
|
||||
|
||||
public static FileStatus toFileStatus(FileStatusDTO dto) {
|
||||
if (null == dto) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return new FileStatus(dto.length, dto.isdir, dto.blockReplication, dto.blocksize, dto.modificationTime,
|
||||
dto.accessTime, FSPermissionDTO.fromFsPermissionDTO(dto.permission), dto.owner, dto.group,
|
||||
FilePathDTO.toPath(dto.symlink), FilePathDTO.toPath(dto.path));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,53 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.timeline.dto;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class InstantDTO {
|
||||
|
||||
@JsonProperty("action")
|
||||
String action;
|
||||
@JsonProperty("ts")
|
||||
String timestamp;
|
||||
@JsonProperty("state")
|
||||
String state;
|
||||
|
||||
public static InstantDTO fromInstant(HoodieInstant instant) {
|
||||
if (null == instant) {
|
||||
return null;
|
||||
}
|
||||
|
||||
InstantDTO dto = new InstantDTO();
|
||||
dto.action = instant.getAction();
|
||||
dto.timestamp = instant.getTimestamp();
|
||||
dto.state = instant.getState().toString();
|
||||
return dto;
|
||||
}
|
||||
|
||||
public static HoodieInstant toInstant(InstantDTO dto) {
|
||||
if (null == dto) {
|
||||
return null;
|
||||
}
|
||||
|
||||
return new HoodieInstant(HoodieInstant.State.valueOf(dto.state), dto.action, dto.timestamp);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,48 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.timeline.dto;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class LogFileDTO {
|
||||
|
||||
@JsonProperty("fileStatus")
|
||||
private FileStatusDTO fileStatus;
|
||||
@JsonProperty("path")
|
||||
private String pathStr;
|
||||
@JsonProperty("len")
|
||||
private long fileLen;
|
||||
|
||||
public static HoodieLogFile toHoodieLogFile(LogFileDTO dto) {
|
||||
FileStatus status = FileStatusDTO.toFileStatus(dto.fileStatus);
|
||||
HoodieLogFile logFile = (status == null) ? new HoodieLogFile(dto.pathStr) : new HoodieLogFile(status);
|
||||
logFile.setFileLen(dto.fileLen);
|
||||
return logFile;
|
||||
}
|
||||
|
||||
public static LogFileDTO fromHoodieLogFile(HoodieLogFile dataFile) {
|
||||
LogFileDTO logFile = new LogFileDTO();
|
||||
logFile.fileLen = dataFile.getFileSize();
|
||||
logFile.pathStr = dataFile.getPath().toString();
|
||||
logFile.fileStatus = FileStatusDTO.fromFileStatus(dataFile.getFileStatus());
|
||||
return logFile;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,44 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.timeline.dto;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@JsonIgnoreProperties(ignoreUnknown = true)
|
||||
public class TimelineDTO {
|
||||
|
||||
@JsonProperty("instants")
|
||||
List<InstantDTO> instants;
|
||||
|
||||
public static TimelineDTO fromTimeline(HoodieTimeline timeline) {
|
||||
TimelineDTO dto = new TimelineDTO();
|
||||
dto.instants = timeline.getInstants().map(InstantDTO::fromInstant).collect(Collectors.toList());
|
||||
return dto;
|
||||
}
|
||||
|
||||
public static HoodieTimeline toTimeline(TimelineDTO dto, HoodieTableMetaClient metaClient) {
|
||||
//TODO: For Now, we will assume, only active-timeline will be transferred.
|
||||
return new HoodieDefaultTimeline(dto.instants.stream().map(InstantDTO::toInstant),
|
||||
metaClient.getActiveTimeline()::getInstantDetails);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,823 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.CompactionUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieTimer;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Common thread-safe implementation for multiple TableFileSystemView Implementations.
|
||||
* Provides uniform handling of
|
||||
* (a) Loading file-system views from underlying file-system
|
||||
* (b) Pending compaction operations and changing file-system views based on that
|
||||
* (c) Thread-safety in loading and managing file system views for this dataset.
|
||||
* (d) resetting file-system views
|
||||
* The actual mechanism of fetching file slices from different view storages is delegated to sub-classes.
|
||||
*/
|
||||
public abstract class AbstractTableFileSystemView implements SyncableFileSystemView, Serializable {
|
||||
|
||||
private static Logger log = LogManager.getLogger(AbstractTableFileSystemView.class);
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
|
||||
// This is the commits that will be visible for all views extending this view
|
||||
protected HoodieTimeline visibleActiveTimeline;
|
||||
|
||||
// Used to concurrently load and populate partition views
|
||||
private ConcurrentHashMap<String, Boolean> addedPartitions = new ConcurrentHashMap<>(4096);
|
||||
|
||||
// Locks to control concurrency. Sync operations use write-lock blocking all fetch operations.
|
||||
// For the common-case, we allow concurrent read of single or multiple partitions
|
||||
private final ReentrantReadWriteLock globalLock = new ReentrantReadWriteLock();
|
||||
private final ReadLock readLock = globalLock.readLock();
|
||||
private final WriteLock writeLock = globalLock.writeLock();
|
||||
|
||||
private String getPartitionPathFromFilePath(String fullPath) {
|
||||
return FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), new Path(fullPath).getParent());
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the view.
|
||||
*/
|
||||
protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
|
||||
this.metaClient = metaClient;
|
||||
this.visibleActiveTimeline = visibleActiveTimeline;
|
||||
// Load Pending Compaction Operations
|
||||
resetPendingCompactionOperations(
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient).values()
|
||||
.stream().map(e -> Pair.of(e.getKey(),
|
||||
CompactionOperation.convertFromAvroRecordInstance(e.getValue()))));
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the provided statuses into the file system view, and also caches it inside this object.
|
||||
*/
|
||||
protected List<HoodieFileGroup> addFilesToView(FileStatus[] statuses) {
|
||||
HoodieTimer timer = new HoodieTimer().startTimer();
|
||||
List<HoodieFileGroup> fileGroups = buildFileGroups(statuses, visibleActiveTimeline, true);
|
||||
long fgBuildTimeTakenMs = timer.endTimer();
|
||||
timer.startTimer();
|
||||
// Group by partition for efficient updates for both InMemory and DiskBased stuctures.
|
||||
fileGroups.stream().collect(Collectors.groupingBy(HoodieFileGroup::getPartitionPath)).entrySet()
|
||||
.forEach(entry -> {
|
||||
String partition = entry.getKey();
|
||||
if (!isPartitionAvailableInStore(partition)) {
|
||||
storePartitionView(partition, entry.getValue());
|
||||
}
|
||||
});
|
||||
long storePartitionsTs = timer.endTimer();
|
||||
log.info("addFilesToView: NumFiles=" + statuses.length + ", FileGroupsCreationTime=" + fgBuildTimeTakenMs
|
||||
+ ", StoreTimeTaken=" + storePartitionsTs);
|
||||
return fileGroups;
|
||||
}
|
||||
|
||||
/**
|
||||
* Build FileGroups from passed in file-status
|
||||
*/
|
||||
protected List<HoodieFileGroup> buildFileGroups(FileStatus[] statuses, HoodieTimeline timeline,
|
||||
boolean addPendingCompactionFileSlice) {
|
||||
return buildFileGroups(convertFileStatusesToDataFiles(statuses), convertFileStatusesToLogFiles(statuses), timeline,
|
||||
addPendingCompactionFileSlice);
|
||||
}
|
||||
|
||||
protected List<HoodieFileGroup> buildFileGroups(Stream<HoodieDataFile> dataFileStream,
|
||||
Stream<HoodieLogFile> logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) {
|
||||
|
||||
Map<Pair<String, String>, List<HoodieDataFile>> dataFiles = dataFileStream
|
||||
.collect(Collectors.groupingBy((dataFile) -> {
|
||||
String partitionPathStr = getPartitionPathFromFilePath(dataFile.getPath());
|
||||
return Pair.of(partitionPathStr, dataFile.getFileId());
|
||||
}));
|
||||
|
||||
Map<Pair<String, String>, List<HoodieLogFile>> logFiles = logFileStream
|
||||
.collect(Collectors.groupingBy((logFile) -> {
|
||||
String partitionPathStr = FSUtils.getRelativePartitionPath(
|
||||
new Path(metaClient.getBasePath()),
|
||||
logFile.getPath().getParent());
|
||||
return Pair.of(partitionPathStr, logFile.getFileId());
|
||||
}));
|
||||
|
||||
Set<Pair<String, String>> fileIdSet = new HashSet<>(dataFiles.keySet());
|
||||
fileIdSet.addAll(logFiles.keySet());
|
||||
|
||||
List<HoodieFileGroup> fileGroups = new ArrayList<>();
|
||||
fileIdSet.forEach(pair -> {
|
||||
String fileId = pair.getValue();
|
||||
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, timeline);
|
||||
if (dataFiles.containsKey(pair)) {
|
||||
dataFiles.get(pair).forEach(group::addDataFile);
|
||||
}
|
||||
if (logFiles.containsKey(pair)) {
|
||||
logFiles.get(pair).forEach(group::addLogFile);
|
||||
}
|
||||
if (addPendingCompactionFileSlice) {
|
||||
Option<Pair<String, CompactionOperation>> pendingCompaction =
|
||||
getPendingCompactionOperationWithInstant(group.getFileGroupId());
|
||||
if (pendingCompaction.isPresent()) {
|
||||
// If there is no delta-commit after compaction request, this step would ensure a new file-slice appears
|
||||
// so that any new ingestion uses the correct base-instant
|
||||
group.addNewFileSliceAtInstant(pendingCompaction.get().getKey());
|
||||
}
|
||||
}
|
||||
fileGroups.add(group);
|
||||
});
|
||||
|
||||
return fileGroups;
|
||||
}
|
||||
|
||||
/**
|
||||
* Clears the partition Map and reset view states
|
||||
*/
|
||||
public final void reset() {
|
||||
try {
|
||||
writeLock.lock();
|
||||
|
||||
addedPartitions.clear();
|
||||
resetViewState();
|
||||
|
||||
// Initialize with new Hoodie timeline.
|
||||
init(metaClient, visibleActiveTimeline);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Allows all view metadata in file system view storage to be reset by subclasses
|
||||
*/
|
||||
protected abstract void resetViewState();
|
||||
|
||||
/**
|
||||
* Allows lazily loading the partitions if needed
|
||||
*
|
||||
* @param partition partition to be loaded if not present
|
||||
*/
|
||||
private void ensurePartitionLoadedCorrectly(String partition) {
|
||||
|
||||
Preconditions.checkArgument(!isClosed(), "View is already closed");
|
||||
|
||||
// ensure we list files only once even in the face of concurrency
|
||||
addedPartitions.computeIfAbsent(partition, (partitionPathStr) -> {
|
||||
long beginTs = System.currentTimeMillis();
|
||||
if (!isPartitionAvailableInStore(partitionPathStr)) {
|
||||
// Not loaded yet
|
||||
try {
|
||||
log.info("Building file system view for partition (" + partitionPathStr + ")");
|
||||
|
||||
// Create the path if it does not exist already
|
||||
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
|
||||
FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath);
|
||||
long beginLsTs = System.currentTimeMillis();
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(partitionPath);
|
||||
long endLsTs = System.currentTimeMillis();
|
||||
log.info("#files found in partition (" + partitionPathStr + ") =" + statuses.length
|
||||
+ ", Time taken =" + (endLsTs - beginLsTs));
|
||||
List<HoodieFileGroup> groups = addFilesToView(statuses);
|
||||
|
||||
if (groups.isEmpty()) {
|
||||
storePartitionView(partitionPathStr, new ArrayList<>());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to list data files in partition " + partitionPathStr, e);
|
||||
}
|
||||
} else {
|
||||
log.debug("View already built for Partition :" + partitionPathStr + ", FOUND is ");
|
||||
}
|
||||
long endTs = System.currentTimeMillis();
|
||||
log.info("Time to load partition (" + partitionPathStr + ") =" + (endTs - beginTs));
|
||||
return true;
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to convert file-status to data-files
|
||||
*
|
||||
* @param statuses List of Fole-Status
|
||||
*/
|
||||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to convert file-status to log-files
|
||||
*
|
||||
* @param statuses List of FIle-Status
|
||||
*/
|
||||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
* With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore those
|
||||
* data-files
|
||||
*
|
||||
* @param dataFile Data File
|
||||
*/
|
||||
protected boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) {
|
||||
final String partitionPath = getPartitionPathFromFilePath(dataFile.getPath());
|
||||
|
||||
Option<Pair<String, CompactionOperation>> compactionWithInstantTime =
|
||||
getPendingCompactionOperationWithInstant(new HoodieFileGroupId(partitionPath, dataFile.getFileId()));
|
||||
return (compactionWithInstantTime.isPresent()) && (null != compactionWithInstantTime.get().getKey())
|
||||
&& dataFile.getCommitTime().equals(compactionWithInstantTime.get().getKey());
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the file-group is under pending-compaction and the file-slice' baseInstant matches compaction
|
||||
* Instant
|
||||
*
|
||||
* @param fileSlice File Slice
|
||||
*/
|
||||
protected boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) {
|
||||
Option<Pair<String, CompactionOperation>> compactionWithInstantTime =
|
||||
getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
|
||||
return (compactionWithInstantTime.isPresent())
|
||||
&& fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.get().getKey());
|
||||
}
|
||||
|
||||
/**
|
||||
* With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore those
|
||||
* data-files
|
||||
*
|
||||
* @param fileSlice File Slice
|
||||
*/
|
||||
protected FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) {
|
||||
if (isFileSliceAfterPendingCompaction(fileSlice)) {
|
||||
// Data file is filtered out of the file-slice as the corresponding compaction
|
||||
// instant not completed yet.
|
||||
FileSlice transformed = new FileSlice(fileSlice.getPartitionPath(),
|
||||
fileSlice.getBaseInstantTime(), fileSlice.getFileId());
|
||||
fileSlice.getLogFiles().forEach(transformed::addLogFile);
|
||||
return transformed;
|
||||
}
|
||||
return fileSlice;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
|
||||
try {
|
||||
readLock.lock();
|
||||
return fetchPendingCompactionOperations();
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<HoodieDataFile> getLatestDataFiles(String partitionStr) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partitionPath = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partitionPath);
|
||||
return fetchLatestDataFiles(partitionPath);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<HoodieDataFile> getLatestDataFiles() {
|
||||
try {
|
||||
readLock.lock();
|
||||
return fetchLatestDataFiles();
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionStr, String maxCommitTime) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partitionPath = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partitionPath);
|
||||
return fetchAllStoredFileGroups(partitionPath)
|
||||
.map(fileGroup -> fileGroup.getAllDataFiles()
|
||||
.filter(dataFile ->
|
||||
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
|
||||
maxCommitTime,
|
||||
HoodieTimeline.LESSER_OR_EQUAL))
|
||||
.filter(df -> !isDataFileDueToPendingCompaction(df))
|
||||
.findFirst())
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Option<HoodieDataFile> getDataFileOn(String partitionStr, String instantTime, String fileId) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partitionPath = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partitionPath);
|
||||
return fetchHoodieFileGroup(partitionPath, fileId)
|
||||
.map(fileGroup -> fileGroup.getAllDataFiles()
|
||||
.filter(dataFile ->
|
||||
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
|
||||
instantTime, HoodieTimeline.EQUAL))
|
||||
.filter(df -> !isDataFileDueToPendingCompaction(df))
|
||||
.findFirst().orElse(null));
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Latest data file for a partition and file-Id
|
||||
*/
|
||||
public final Option<HoodieDataFile> getLatestDataFile(String partitionStr, String fileId) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partitionPath = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partitionPath);
|
||||
return fetchLatestDataFile(partitionPath, fileId);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
|
||||
try {
|
||||
readLock.lock();
|
||||
return fetchAllStoredFileGroups().map(fileGroup -> {
|
||||
return fileGroup.getAllDataFiles()
|
||||
.filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime())
|
||||
&& !isDataFileDueToPendingCompaction(dataFile))
|
||||
.findFirst();
|
||||
}).filter(Optional::isPresent).map(Optional::get);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<HoodieDataFile> getAllDataFiles(String partitionStr) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partitionPath = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partitionPath);
|
||||
return fetchAllDataFiles(partitionPath)
|
||||
.filter(df -> visibleActiveTimeline.containsOrBeforeTimelineStarts(df.getCommitTime()))
|
||||
.filter(df -> !isDataFileDueToPendingCompaction(df));
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<FileSlice> getLatestFileSlices(String partitionStr) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partitionPath = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partitionPath);
|
||||
return fetchLatestFileSlices(partitionPath).map(fs -> filterDataFileAfterPendingCompaction(fs));
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Latest File Slice for a given fileId in a given partition
|
||||
*/
|
||||
public final Option<FileSlice> getLatestFileSlice(String partitionStr, String fileId) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partitionPath = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partitionPath);
|
||||
Option<FileSlice> fs = fetchLatestFileSlice(partitionPath, fileId);
|
||||
return fs.map(f -> filterDataFileAfterPendingCompaction(f));
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionStr) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partitionPath = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partitionPath);
|
||||
return fetchAllStoredFileGroups(partitionPath)
|
||||
.map(fileGroup -> {
|
||||
FileSlice fileSlice = fileGroup.getLatestFileSlice().get();
|
||||
// if the file-group is under compaction, pick the latest before compaction instant time.
|
||||
Option<Pair<String, CompactionOperation>> compactionWithInstantPair =
|
||||
getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
|
||||
if (compactionWithInstantPair.isPresent()) {
|
||||
String compactionInstantTime = compactionWithInstantPair.get().getLeft();
|
||||
return fileGroup.getLatestFileSliceBefore(compactionInstantTime);
|
||||
}
|
||||
return Optional.of(fileSlice);
|
||||
})
|
||||
.map(Optional::get);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionStr, String maxCommitTime) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partitionPath = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partitionPath);
|
||||
return fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime)
|
||||
.map(fs -> filterDataFileAfterPendingCompaction(fs));
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionStr, String maxInstantTime) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partition = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partition);
|
||||
return fetchAllStoredFileGroups(partition)
|
||||
.map(fileGroup -> {
|
||||
Optional<FileSlice> fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime);
|
||||
// if the file-group is under construction, pick the latest before compaction instant time.
|
||||
if (fileSlice.isPresent()) {
|
||||
fileSlice = Optional.of(fetchMergedFileSlice(fileGroup, fileSlice.get()));
|
||||
}
|
||||
return fileSlice;
|
||||
})
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
|
||||
try {
|
||||
readLock.lock();
|
||||
return fetchLatestFileSliceInRange(commitsToReturn);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<FileSlice> getAllFileSlices(String partitionStr) {
|
||||
try {
|
||||
readLock.lock();
|
||||
String partition = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partition);
|
||||
return fetchAllFileSlices(partition);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure there is consistency in handling trailing slash in partition-path. Always trim it which is what is done in
|
||||
* other places.
|
||||
*/
|
||||
private String formatPartitionKey(String partitionStr) {
|
||||
return partitionStr.endsWith("/") ? partitionStr.substring(0, partitionStr.length() - 1) : partitionStr;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final Stream<HoodieFileGroup> getAllFileGroups(String partitionStr) {
|
||||
try {
|
||||
readLock.lock();
|
||||
// Ensure there is consistency in handling trailing slash in partition-path. Always trim it which is what is done
|
||||
// in other places.
|
||||
String partition = formatPartitionKey(partitionStr);
|
||||
ensurePartitionLoadedCorrectly(partition);
|
||||
return fetchAllStoredFileGroups(partition);
|
||||
} finally {
|
||||
readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
// Fetch APIs to be implemented by concrete sub-classes
|
||||
|
||||
/**
|
||||
* Check if there is an outstanding compaction scheduled for this file
|
||||
*
|
||||
* @param fgId File-Group Id
|
||||
* @return true if there is a pending compaction, false otherwise
|
||||
*/
|
||||
protected abstract boolean isPendingCompactionScheduledForFileId(HoodieFileGroupId fgId);
|
||||
|
||||
/**
|
||||
* resets the pending compaction operation and overwrite with the new list
|
||||
*
|
||||
* @param operations Pending Compaction Operations
|
||||
*/
|
||||
abstract void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
|
||||
|
||||
/**
|
||||
* Add pending compaction operations to store
|
||||
*
|
||||
* @param operations Pending compaction operations to be added
|
||||
*/
|
||||
abstract void addPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
|
||||
|
||||
/**
|
||||
* Remove pending compaction operations from store
|
||||
*
|
||||
* @param operations Pending compaction operations to be removed
|
||||
*/
|
||||
abstract void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations);
|
||||
|
||||
/**
|
||||
* Return pending compaction operation for a file-group
|
||||
*
|
||||
* @param fileGroupId File-Group Id
|
||||
*/
|
||||
protected abstract Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(
|
||||
HoodieFileGroupId fileGroupId);
|
||||
|
||||
/**
|
||||
* Fetch all pending compaction operations
|
||||
*/
|
||||
abstract Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations();
|
||||
|
||||
/**
|
||||
* Checks if partition is pre-loaded and available in store
|
||||
*
|
||||
* @param partitionPath Partition Path
|
||||
*/
|
||||
abstract boolean isPartitionAvailableInStore(String partitionPath);
|
||||
|
||||
/**
|
||||
* Add a complete partition view to store
|
||||
*
|
||||
* @param partitionPath Partition Path
|
||||
* @param fileGroups File Groups for the partition path
|
||||
*/
|
||||
abstract void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups);
|
||||
|
||||
/**
|
||||
* Fetch all file-groups stored for a partition-path
|
||||
*
|
||||
* @param partitionPath Partition path for which the file-groups needs to be retrieved.
|
||||
* @return file-group stream
|
||||
*/
|
||||
abstract Stream<HoodieFileGroup> fetchAllStoredFileGroups(String partitionPath);
|
||||
|
||||
/**
|
||||
* Fetch all Stored file-groups across all partitions loaded
|
||||
*
|
||||
* @return file-group stream
|
||||
*/
|
||||
abstract Stream<HoodieFileGroup> fetchAllStoredFileGroups();
|
||||
|
||||
/**
|
||||
* Check if the view is already closed
|
||||
*/
|
||||
abstract boolean isClosed();
|
||||
|
||||
/**
|
||||
* Default implementation for fetching latest file-slice in commit range
|
||||
*
|
||||
* @param commitsToReturn Commits
|
||||
*/
|
||||
Stream<FileSlice> fetchLatestFileSliceInRange(List<String> commitsToReturn) {
|
||||
return fetchAllStoredFileGroups().map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
/**
|
||||
* Default implementation for fetching all file-slices for a partition-path
|
||||
*
|
||||
* @param partitionPath Partition path
|
||||
* @return file-slice stream
|
||||
*/
|
||||
Stream<FileSlice> fetchAllFileSlices(String partitionPath) {
|
||||
return fetchAllStoredFileGroups(partitionPath)
|
||||
.map(HoodieFileGroup::getAllFileSlices)
|
||||
.flatMap(sliceList -> sliceList);
|
||||
}
|
||||
|
||||
/**
|
||||
* Default implementation for fetching latest data-files for the partition-path
|
||||
*/
|
||||
Stream<HoodieDataFile> fetchLatestDataFiles(final String partitionPath) {
|
||||
return fetchAllStoredFileGroups(partitionPath)
|
||||
.map(this::getLatestDataFile)
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
|
||||
protected Optional<HoodieDataFile> getLatestDataFile(HoodieFileGroup fileGroup) {
|
||||
return fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst();
|
||||
}
|
||||
|
||||
/**
|
||||
* Default implementation for fetching latest data-files across all partitions
|
||||
*/
|
||||
Stream<HoodieDataFile> fetchLatestDataFiles() {
|
||||
return fetchAllStoredFileGroups()
|
||||
.map(this::getLatestDataFile)
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
/**
|
||||
* Default implementation for fetching all data-files for a partition
|
||||
*
|
||||
* @param partitionPath partition-path
|
||||
*/
|
||||
Stream<HoodieDataFile> fetchAllDataFiles(String partitionPath) {
|
||||
return fetchAllStoredFileGroups(partitionPath)
|
||||
.map(HoodieFileGroup::getAllDataFiles)
|
||||
.flatMap(dataFileList -> dataFileList);
|
||||
}
|
||||
|
||||
/**
|
||||
* Default implementation for fetching file-group
|
||||
*/
|
||||
Option<HoodieFileGroup> fetchHoodieFileGroup(String partitionPath, String fileId) {
|
||||
return Option.fromJavaOptional(fetchAllStoredFileGroups(partitionPath)
|
||||
.filter(fileGroup -> fileGroup.getFileGroupId().getFileId().equals(fileId)).findFirst());
|
||||
}
|
||||
|
||||
/**
|
||||
* Default implementation for fetching latest file-slices for a partition path
|
||||
*/
|
||||
Stream<FileSlice> fetchLatestFileSlices(String partitionPath) {
|
||||
return fetchAllStoredFileGroups(partitionPath)
|
||||
.map(HoodieFileGroup::getLatestFileSlice)
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
/**
|
||||
* Default implementation for fetching latest file-slices for a partition path as of instant
|
||||
*
|
||||
* @param partitionPath Partition Path
|
||||
* @param maxCommitTime Instant Time
|
||||
*/
|
||||
Stream<FileSlice> fetchLatestFileSlicesBeforeOrOn(String partitionPath,
|
||||
String maxCommitTime) {
|
||||
return fetchAllStoredFileGroups(partitionPath)
|
||||
.map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to merge last 2 file-slices. These 2 file-slices do not have compaction done yet.
|
||||
*
|
||||
* @param lastSlice Latest File slice for a file-group
|
||||
* @param penultimateSlice Penultimate file slice for a file-group in commit timeline order
|
||||
*/
|
||||
private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, FileSlice penultimateSlice) {
|
||||
FileSlice merged = new FileSlice(penultimateSlice.getPartitionPath(),
|
||||
penultimateSlice.getBaseInstantTime(), penultimateSlice.getFileId());
|
||||
if (penultimateSlice.getDataFile().isPresent()) {
|
||||
merged.setDataFile(penultimateSlice.getDataFile().get());
|
||||
}
|
||||
// Add Log files from penultimate and last slices
|
||||
penultimateSlice.getLogFiles().forEach(merged::addLogFile);
|
||||
lastSlice.getLogFiles().forEach(merged::addLogFile);
|
||||
return merged;
|
||||
}
|
||||
|
||||
/**
|
||||
* If the file-slice is because of pending compaction instant, this method merges the file-slice with the one before
|
||||
* the compaction instant time
|
||||
*
|
||||
* @param fileGroup File Group for which the file slice belongs to
|
||||
* @param fileSlice File Slice which needs to be merged
|
||||
*/
|
||||
private FileSlice fetchMergedFileSlice(HoodieFileGroup fileGroup, FileSlice fileSlice) {
|
||||
// if the file-group is under construction, pick the latest before compaction instant time.
|
||||
Option<Pair<String, CompactionOperation>> compactionOpWithInstant =
|
||||
getPendingCompactionOperationWithInstant(fileGroup.getFileGroupId());
|
||||
if (compactionOpWithInstant.isPresent()) {
|
||||
String compactionInstantTime = compactionOpWithInstant.get().getKey();
|
||||
if (fileSlice.getBaseInstantTime().equals(compactionInstantTime)) {
|
||||
Optional<FileSlice> prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime);
|
||||
if (prevFileSlice.isPresent()) {
|
||||
return mergeCompactionPendingFileSlices(fileSlice, prevFileSlice.get());
|
||||
}
|
||||
}
|
||||
}
|
||||
return fileSlice;
|
||||
}
|
||||
|
||||
/**
|
||||
* Default implementation for fetching latest data-file
|
||||
* @param partitionPath Partition path
|
||||
* @param fileId File Id
|
||||
* @return Data File if present
|
||||
*/
|
||||
protected Option<HoodieDataFile> fetchLatestDataFile(String partitionPath, String fileId) {
|
||||
return Option.fromJavaOptional(fetchLatestDataFiles(partitionPath)
|
||||
.filter(fs -> fs.getFileId().equals(fileId)).findFirst());
|
||||
}
|
||||
|
||||
/**
|
||||
* Default implementation for fetching file-slice
|
||||
* @param partitionPath Partition path
|
||||
* @param fileId File Id
|
||||
* @return File Slice if present
|
||||
*/
|
||||
protected Option<FileSlice> fetchLatestFileSlice(String partitionPath, String fileId) {
|
||||
return Option.fromJavaOptional(fetchLatestFileSlices(partitionPath)
|
||||
.filter(fs -> fs.getFileId().equals(fileId)).findFirst());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieInstant> getLastInstant() {
|
||||
return Option.fromJavaOptional(visibleActiveTimeline.lastInstant());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline getTimeline() {
|
||||
return visibleActiveTimeline;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sync() {
|
||||
HoodieTimeline oldTimeline = getTimeline();
|
||||
HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants();
|
||||
try {
|
||||
writeLock.lock();
|
||||
runSync(oldTimeline, newTimeline);
|
||||
} finally {
|
||||
writeLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs complete reset of file-system view. Subsequent partition view calls will load file slices against latest
|
||||
* timeline
|
||||
*
|
||||
* @param oldTimeline Old Hoodie Timeline
|
||||
* @param newTimeline New Hoodie Timeline
|
||||
*/
|
||||
protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) {
|
||||
visibleActiveTimeline = newTimeline;
|
||||
addedPartitions.clear();
|
||||
resetViewState();
|
||||
// Initialize with new Hoodie timeline.
|
||||
init(metaClient, newTimeline);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,216 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.SerializableConfiguration;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.util.Functions.Function2;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* A container that can potentially hold one or more dataset's
|
||||
* file-system views. There is one view for each dataset. This is a view built against a timeline containing completed
|
||||
* actions. In an embedded timeline-server mode, this typically holds only one dataset's view.
|
||||
* In a stand-alone server mode, this can hold more than one dataset's views.
|
||||
*
|
||||
* FileSystemView can be stored "locally" using the following storage mechanisms:
|
||||
* a. In Memory
|
||||
* b. Spillable Map
|
||||
* c. RocksDB
|
||||
*
|
||||
* But there can be cases where the file-system view is managed remoted. For example : Embedded Timeline Server). In
|
||||
* this case, the clients will configure a remote filesystem view client (RemoteHoodieTableFileSystemView) for the
|
||||
* dataset which can connect to the remote file system view and fetch views. THere are 2 modes here : REMOTE_FIRST and
|
||||
* REMOTE_ONLY
|
||||
* REMOTE_FIRST : The file-system view implementation on client side will act as a remote proxy. In case, if there
|
||||
* is problem (or exceptions) querying remote file-system view, a backup local file-system view(using
|
||||
* either one of in-memory, spillable, rocksDB) is used to server file-system view queries
|
||||
* REMOTE_ONLY : In this case, there is no backup local file-system view. If there is problem (or exceptions)
|
||||
* querying remote file-system view, then the exceptions are percolated back to client.
|
||||
*
|
||||
* FileSystemViewManager is designed to encapsulate the file-system view storage from clients using the file-system
|
||||
* view. FileSystemViewManager uses a factory to construct specific implementation of file-system view and passes it to
|
||||
* clients for querying.
|
||||
*/
|
||||
public class FileSystemViewManager {
|
||||
private static Logger logger = LogManager.getLogger(FileSystemViewManager.class);
|
||||
|
||||
private final SerializableConfiguration conf;
|
||||
// The View Storage config used to store file-system views
|
||||
private final FileSystemViewStorageConfig viewStorageConfig;
|
||||
// Map from Base-Path to View
|
||||
private final ConcurrentHashMap<String, SyncableFileSystemView> globalViewMap;
|
||||
// Factory Map to create file-system views
|
||||
private final Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator;
|
||||
|
||||
public FileSystemViewManager(SerializableConfiguration conf, FileSystemViewStorageConfig viewStorageConfig,
|
||||
Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator) {
|
||||
this.conf = conf;
|
||||
this.viewStorageConfig = viewStorageConfig;
|
||||
this.globalViewMap = new ConcurrentHashMap<>();
|
||||
this.viewCreator = viewCreator;
|
||||
}
|
||||
|
||||
/**
|
||||
* Drops reference to File-System Views. Future calls to view results in creating a new view
|
||||
* @param basePath
|
||||
*/
|
||||
public void clearFileSystemView(String basePath) {
|
||||
SyncableFileSystemView view = globalViewMap.remove(basePath);
|
||||
if (view != null) {
|
||||
view.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Main API to get the file-system view for the base-path
|
||||
* @param basePath
|
||||
* @return
|
||||
*/
|
||||
public SyncableFileSystemView getFileSystemView(String basePath) {
|
||||
return globalViewMap.computeIfAbsent(basePath,
|
||||
(path) -> viewCreator.apply(path, viewStorageConfig));
|
||||
}
|
||||
|
||||
/**
|
||||
* Closes all views opened
|
||||
*/
|
||||
public void close() {
|
||||
this.globalViewMap.values().stream().forEach(v -> v.close());
|
||||
this.globalViewMap.clear();
|
||||
}
|
||||
|
||||
// FACTORY METHODS FOR CREATING FILE-SYSTEM VIEWS
|
||||
|
||||
/**
|
||||
* Create RocksDB based file System view for a dataset
|
||||
* @param conf Hadoop Configuration
|
||||
* @param viewConf View Storage Configuration
|
||||
* @param basePath Base Path of dataset
|
||||
* @return
|
||||
*/
|
||||
private static RocksDbBasedFileSystemView createRocksDBBasedFileSystemView(SerializableConfiguration conf,
|
||||
FileSystemViewStorageConfig viewConf, String basePath) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.get(), basePath, true);
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
|
||||
return new RocksDbBasedFileSystemView(metaClient, timeline, viewConf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a spillable Map based file System view for a dataset
|
||||
* @param conf Hadoop Configuration
|
||||
* @param viewConf View Storage Configuration
|
||||
* @param basePath Base Path of dataset
|
||||
* @return
|
||||
*/
|
||||
private static SpillableMapBasedFileSystemView createSpillableMapBasedFileSystemView(SerializableConfiguration conf,
|
||||
FileSystemViewStorageConfig viewConf, String basePath) {
|
||||
logger.info("Creating SpillableMap based view for basePath " + basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.get(), basePath, true);
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline, viewConf);
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create an in-memory file System view for a dataset
|
||||
* @param conf Hadoop Configuration
|
||||
* @param viewConf View Storage Configuration
|
||||
* @param basePath Base Path of dataset
|
||||
* @return
|
||||
*/
|
||||
private static HoodieTableFileSystemView createInMemoryFileSystemView(SerializableConfiguration conf,
|
||||
FileSystemViewStorageConfig viewConf, String basePath) {
|
||||
logger.info("Creating InMemory based view for basePath " + basePath);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.get(), basePath, true);
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
|
||||
return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled());
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a remote file System view for a dataset
|
||||
* @param conf Hadoop Configuration
|
||||
* @param viewConf View Storage Configuration
|
||||
* @param metaClient Hoodie Table MetaClient for the dataset.
|
||||
* @return
|
||||
*/
|
||||
private static RemoteHoodieTableFileSystemView createRemoteFileSystemView(SerializableConfiguration conf,
|
||||
FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
|
||||
logger.info("Creating remote view for basePath " + metaClient.getBasePath() + ". Server="
|
||||
+ viewConf.getRemoteViewServerHost() + ":" + viewConf.getRemoteViewServerPort());
|
||||
return new RemoteHoodieTableFileSystemView(viewConf.getRemoteViewServerHost(),
|
||||
viewConf.getRemoteViewServerPort(), metaClient);
|
||||
}
|
||||
|
||||
/**
|
||||
* Main Factory method for building file-system views
|
||||
* @param conf Hadoop Configuration
|
||||
* @param config View Storage Configuration
|
||||
* @return
|
||||
*/
|
||||
public static FileSystemViewManager createViewManager(
|
||||
final SerializableConfiguration conf, final FileSystemViewStorageConfig config) {
|
||||
logger.info("Creating View Manager with storage type :" + config.getStorageType());
|
||||
switch (config.getStorageType()) {
|
||||
case EMBEDDED_KV_STORE:
|
||||
logger.info("Creating embedded rocks-db based Table View");
|
||||
return new FileSystemViewManager(conf, config,
|
||||
(basePath, viewConf) -> createRocksDBBasedFileSystemView(conf, viewConf, basePath));
|
||||
case SPILLABLE_DISK:
|
||||
logger.info("Creating Spillable Disk based Table View");
|
||||
return new FileSystemViewManager(conf, config,
|
||||
(basePath, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, basePath));
|
||||
case MEMORY:
|
||||
logger.info("Creating in-memory based Table View");
|
||||
return new FileSystemViewManager(conf, config,
|
||||
(basePath, viewConfig) -> createInMemoryFileSystemView(conf, viewConfig, basePath));
|
||||
case REMOTE_ONLY:
|
||||
logger.info("Creating remote only table view");
|
||||
return new FileSystemViewManager(conf, config,
|
||||
(basePath, viewConfig) -> createRemoteFileSystemView(conf, viewConfig,
|
||||
new HoodieTableMetaClient(conf.get(), basePath)));
|
||||
case REMOTE_FIRST:
|
||||
logger.info("Creating remote first table view");
|
||||
return new FileSystemViewManager(conf, config, (basePath, viewConfig) -> {
|
||||
RemoteHoodieTableFileSystemView remoteFileSystemView =
|
||||
createRemoteFileSystemView(conf, viewConfig, new HoodieTableMetaClient(conf.get(), basePath));
|
||||
SyncableFileSystemView secondaryView = null;
|
||||
switch (viewConfig.getSecondaryStorageType()) {
|
||||
case MEMORY:
|
||||
secondaryView = createInMemoryFileSystemView(conf, viewConfig, basePath);
|
||||
break;
|
||||
case EMBEDDED_KV_STORE:
|
||||
secondaryView = createRocksDBBasedFileSystemView(conf, viewConfig, basePath);
|
||||
break;
|
||||
case SPILLABLE_DISK:
|
||||
secondaryView = createSpillableMapBasedFileSystemView(conf, viewConfig, basePath);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Secondary Storage type can only be in-memory or spillable. Was :"
|
||||
+ viewConfig.getSecondaryStorageType());
|
||||
}
|
||||
return new PriorityBasedFileSystemView(remoteFileSystemView, secondaryView);
|
||||
});
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown file system view type :" + config.getStorageType());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,197 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.config.DefaultHoodieConfig;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* File System View Storage Configurations
|
||||
*/
|
||||
public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
|
||||
|
||||
//Property Names
|
||||
public static final String FILESYSTEM_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.type";
|
||||
public static final String FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "hoodie.filesystem.view.incr.timeline.sync.enable";
|
||||
public static final String FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.secondary.type";
|
||||
public static final String FILESYSTEM_VIEW_REMOTE_HOST = "hoodie.filesystem.view.remote.host";
|
||||
public static final String FILESYSTEM_VIEW_REMOTE_PORT = "hoodie.filesystem.view.remote.port";
|
||||
public static final String FILESYSTEM_VIEW_SPILLABLE_DIR = "hoodie.filesystem.view.spillable.dir";
|
||||
public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = "hoodie.filesystem.view.spillable.mem";
|
||||
public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION =
|
||||
"hoodie.filesystem.view.spillable.compaction.mem.fraction";
|
||||
private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path";
|
||||
|
||||
public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
|
||||
public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
|
||||
public static final String DEFAULT_ROCKSDB_BASE_PATH = "/tmp/hoodie_timeline_rocksdb";
|
||||
|
||||
public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "false";
|
||||
public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = "localhost";
|
||||
public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = 26754;
|
||||
|
||||
public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/";
|
||||
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01;
|
||||
private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB
|
||||
|
||||
public static FileSystemViewStorageConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
private FileSystemViewStorageConfig(Properties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public FileSystemViewStorageType getStorageType() {
|
||||
return FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_VIEW_STORAGE_TYPE));
|
||||
}
|
||||
|
||||
public boolean isIncrementalTimelineSyncEnabled() {
|
||||
return Boolean.valueOf(props.getProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE));
|
||||
}
|
||||
|
||||
public String getRemoteViewServerHost() {
|
||||
return props.getProperty(FILESYSTEM_VIEW_REMOTE_HOST);
|
||||
}
|
||||
|
||||
public Integer getRemoteViewServerPort() {
|
||||
return Integer.parseInt(props.getProperty(FILESYSTEM_VIEW_REMOTE_PORT));
|
||||
}
|
||||
|
||||
public long getMaxMemoryForFileGroupMap() {
|
||||
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
|
||||
return totalMemory - getMaxMemoryForPendingCompaction();
|
||||
}
|
||||
|
||||
public long getMaxMemoryForPendingCompaction() {
|
||||
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
|
||||
long reservedForPendingComaction = new Double(totalMemory * Double.parseDouble(
|
||||
props.getProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION))).longValue();
|
||||
return reservedForPendingComaction;
|
||||
}
|
||||
|
||||
public String getBaseStoreDir() {
|
||||
return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR);
|
||||
}
|
||||
|
||||
public FileSystemViewStorageType getSecondaryStorageType() {
|
||||
return FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
|
||||
}
|
||||
|
||||
public String getRocksdbBasePath() {
|
||||
return props.getProperty(ROCKSDB_BASE_PATH_PROP);
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
FileReader reader = new FileReader(propertiesFile);
|
||||
try {
|
||||
props.load(reader);
|
||||
return this;
|
||||
} finally {
|
||||
reader.close();
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withStorageType(FileSystemViewStorageType storageType) {
|
||||
props.setProperty(FILESYSTEM_VIEW_STORAGE_TYPE, storageType.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSecondaryStorageType(FileSystemViewStorageType storageType) {
|
||||
props.setProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, storageType.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIncrementalTimelineSync(boolean enableIncrTimelineSync) {
|
||||
props.setProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, Boolean.toString(enableIncrTimelineSync));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRemoteServerHost(String remoteServerHost) {
|
||||
props.setProperty(FILESYSTEM_VIEW_REMOTE_HOST, remoteServerHost);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRemoteServerPort(Integer remoteServerPort) {
|
||||
props.setProperty(FILESYSTEM_VIEW_REMOTE_PORT, remoteServerPort.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemoryForView(Long maxMemoryForView) {
|
||||
props.setProperty(FILESYSTEM_VIEW_SPILLABLE_MEM, maxMemoryForView.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMemFractionForPendingCompaction(Double memFractionForPendingCompaction) {
|
||||
props.setProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBaseStoreDir(String baseStorePath) {
|
||||
props.setProperty(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRocksDBPath(String basePath) {
|
||||
props.setProperty(ROCKSDB_BASE_PATH_PROP, basePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public FileSystemViewStorageConfig build() {
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_STORAGE_TYPE),
|
||||
FILESYSTEM_VIEW_STORAGE_TYPE, DEFAULT_VIEW_STORAGE_TYPE.name());
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE),
|
||||
FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE);
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE),
|
||||
FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, DEFAULT_SECONDARY_VIEW_STORAGE_TYPE.name());
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REMOTE_HOST),
|
||||
FILESYSTEM_VIEW_REMOTE_HOST, DEFUALT_REMOTE_VIEW_SERVER_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REMOTE_PORT),
|
||||
FILESYSTEM_VIEW_REMOTE_PORT, DEFAULT_REMOTE_VIEW_SERVER_PORT.toString());
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_SPILLABLE_DIR),
|
||||
FILESYSTEM_VIEW_SPILLABLE_DIR, DEFAULT_VIEW_SPILLABLE_DIR);
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_SPILLABLE_MEM),
|
||||
FILESYSTEM_VIEW_SPILLABLE_MEM, DEFAULT_MAX_MEMORY_FOR_VIEW.toString());
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION),
|
||||
FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION.toString());
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(ROCKSDB_BASE_PATH_PROP),
|
||||
ROCKSDB_BASE_PATH_PROP, DEFAULT_ROCKSDB_BASE_PATH);
|
||||
|
||||
// Validations
|
||||
FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_VIEW_STORAGE_TYPE));
|
||||
FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
|
||||
Preconditions.checkArgument(Integer.parseInt(props.getProperty(FILESYSTEM_VIEW_REMOTE_PORT)) > 0);
|
||||
return new FileSystemViewStorageConfig(props);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* Copyright (c) 2018 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;
|
||||
|
||||
/**
|
||||
* Storage Type used to store/retrieve File system view of a table
|
||||
*/
|
||||
public enum FileSystemViewStorageType {
|
||||
// In-memory storage of file-system view
|
||||
MEMORY,
|
||||
// Constrained Memory storage for file-system view with overflow data spilled to disk
|
||||
SPILLABLE_DISK,
|
||||
// EMBEDDED Key Value Storage for file-system view
|
||||
EMBEDDED_KV_STORE,
|
||||
// Delegate file-system view to remote server
|
||||
REMOTE_ONLY,
|
||||
// A composite storage where file-system view calls are first delegated to Remote server ( REMOTE_ONLY )
|
||||
// In case of failures, switches subsequent calls to secondary local storage type
|
||||
REMOTE_FIRST
|
||||
}
|
||||
@@ -16,99 +16,98 @@
|
||||
|
||||
package com.uber.hoodie.common.table.view;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
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.util.CompactionUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
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.Set;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* 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
|
||||
*
|
||||
* TableFileSystemView Implementations based on in-memory storage.
|
||||
* @see TableFileSystemView
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public class HoodieTableFileSystemView implements TableFileSystemView,
|
||||
TableFileSystemView.ReadOptimizedView,
|
||||
TableFileSystemView.RealtimeView, Serializable {
|
||||
public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystemView {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HoodieTableFileSystemView.class);
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
// This is the commits that will be visible for all views extending this view
|
||||
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<HoodieFileGroupId, HoodieFileGroup> fileGroupMap;
|
||||
protected Map<String, List<HoodieFileGroup>> partitionToFileGroupsMap;
|
||||
|
||||
/**
|
||||
* PartitionPath + File-Id to pending compaction instant time
|
||||
*/
|
||||
private final Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction;
|
||||
protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction;
|
||||
|
||||
/**
|
||||
* Flag to determine if closed
|
||||
*/
|
||||
private boolean closed = false;
|
||||
|
||||
HoodieTableFileSystemView(boolean enableIncrementalTimelineSync) {
|
||||
super(enableIncrementalTimelineSync);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a file system view, as of the given timeline
|
||||
*/
|
||||
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
|
||||
HoodieTimeline visibleActiveTimeline) {
|
||||
this.metaClient = metaClient;
|
||||
this.visibleActiveTimeline = visibleActiveTimeline;
|
||||
this.fileGroupMap = new HashMap<>();
|
||||
this.partitionToFileGroupsMap = new HashMap<>();
|
||||
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
|
||||
this(metaClient, visibleActiveTimeline, false);
|
||||
}
|
||||
|
||||
// Build fileId to Pending Compaction Instants
|
||||
List<HoodieInstant> pendingCompactionInstants =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().getInstants().collect(Collectors.toList());
|
||||
this.fgIdToPendingCompaction = ImmutableMap.copyOf(
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient).entrySet().stream()
|
||||
.map(entry -> Pair.of(entry.getKey(), Pair.of(entry.getValue().getKey(),
|
||||
CompactionOperation.convertFromAvroRecordInstance(entry.getValue().getValue()))))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
|
||||
/**
|
||||
* Create a file system view, as of the given timeline
|
||||
*/
|
||||
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline,
|
||||
boolean enableIncrementalTimelineSync) {
|
||||
super(enableIncrementalTimelineSync);
|
||||
init(metaClient, visibleActiveTimeline);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
|
||||
this.partitionToFileGroupsMap = createPartitionToFileGroups();
|
||||
super.init(metaClient, visibleActiveTimeline);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void resetViewState() {
|
||||
this.fgIdToPendingCompaction = null;
|
||||
this.partitionToFileGroupsMap = null;
|
||||
}
|
||||
|
||||
protected Map<String, List<HoodieFileGroup>> createPartitionToFileGroups() {
|
||||
return new ConcurrentHashMap<>();
|
||||
}
|
||||
|
||||
protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> createFileIdToPendingCompactionMap(
|
||||
Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fileIdToPendingCompaction) {
|
||||
return fileIdToPendingCompaction;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a file system view, as of the given timeline, with the provided file statuses.
|
||||
*/
|
||||
public HoodieTableFileSystemView(HoodieTableMetaClient metaClient,
|
||||
HoodieTimeline visibleActiveTimeline,
|
||||
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.
|
||||
*
|
||||
@@ -124,293 +123,39 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
||||
out.defaultWriteObject();
|
||||
}
|
||||
|
||||
private String getPartitionPathFromFileStatus(FileStatus fileStatus) {
|
||||
return FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), fileStatus.getPath().getParent());
|
||||
@Override
|
||||
protected boolean isPendingCompactionScheduledForFileId(HoodieFileGroupId fgId) {
|
||||
return fgIdToPendingCompaction.containsKey(fgId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the provided statuses into the file system view, and also caches it inside this object.
|
||||
*/
|
||||
private List<HoodieFileGroup> addFilesToView(FileStatus[] statuses) {
|
||||
Map<Pair<String, String>, List<HoodieDataFile>> dataFiles = convertFileStatusesToDataFiles(
|
||||
statuses)
|
||||
.collect(Collectors.groupingBy((dataFile) -> {
|
||||
String partitionPathStr = getPartitionPathFromFileStatus(dataFile.getFileStatus());
|
||||
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
|
||||
protected void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
|
||||
// Build fileId to Pending Compaction Instants
|
||||
this.fgIdToPendingCompaction = createFileIdToPendingCompactionMap(
|
||||
operations.map(entry -> {
|
||||
return Pair.of(entry.getValue().getFileGroupId(), Pair.of(entry.getKey(),entry.getValue()));
|
||||
}).collect(Collectors.toMap(Pair::getKey, Pair::getValue)));
|
||||
}
|
||||
|
||||
Set<Pair<String, String>> fileIdSet = new HashSet<>(dataFiles.keySet());
|
||||
fileIdSet.addAll(logFiles.keySet());
|
||||
|
||||
List<HoodieFileGroup> fileGroups = new ArrayList<>();
|
||||
fileIdSet.forEach(pair -> {
|
||||
String fileId = pair.getValue();
|
||||
HoodieFileGroup group = new HoodieFileGroup(pair.getKey(), fileId, visibleActiveTimeline);
|
||||
if (dataFiles.containsKey(pair)) {
|
||||
dataFiles.get(pair).forEach(group::addDataFile);
|
||||
}
|
||||
if (logFiles.containsKey(pair)) {
|
||||
logFiles.get(pair).forEach(group::addLogFile);
|
||||
}
|
||||
HoodieFileGroupId fgId = group.getFileGroupId();
|
||||
if (fgIdToPendingCompaction.containsKey(fgId)) {
|
||||
// If there is no delta-commit after compaction request, this step would ensure a new file-slice appears
|
||||
// so that any new ingestion uses the correct base-instant
|
||||
group.addNewFileSliceAtInstant(fgIdToPendingCompaction.get(fgId).getKey());
|
||||
}
|
||||
fileGroups.add(group);
|
||||
@Override
|
||||
protected void addPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
|
||||
operations.forEach(opInstantPair -> {
|
||||
Preconditions.checkArgument(!fgIdToPendingCompaction.containsKey(opInstantPair.getValue().getFileGroupId()),
|
||||
"Duplicate FileGroupId found in pending compaction operations. FgId :"
|
||||
+ opInstantPair.getValue().getFileGroupId());
|
||||
fgIdToPendingCompaction.put(opInstantPair.getValue().getFileGroupId(),
|
||||
Pair.of(opInstantPair.getKey(), opInstantPair.getValue()));
|
||||
});
|
||||
}
|
||||
|
||||
// add to the cache.
|
||||
fileGroups.forEach(group -> {
|
||||
fileGroupMap.put(group.getFileGroupId(), group);
|
||||
if (!partitionToFileGroupsMap.containsKey(group.getPartitionPath())) {
|
||||
partitionToFileGroupsMap.put(group.getPartitionPath(), new ArrayList<>());
|
||||
}
|
||||
partitionToFileGroupsMap.get(group.getPartitionPath()).add(group);
|
||||
@Override
|
||||
protected void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
|
||||
operations.forEach(opInstantPair -> {
|
||||
Preconditions.checkArgument(fgIdToPendingCompaction.containsKey(opInstantPair.getValue().getFileGroupId()),
|
||||
"Trying to remove a FileGroupId which is not found in pending compaction operations. FgId :"
|
||||
+ opInstantPair.getValue().getFileGroupId());
|
||||
fgIdToPendingCompaction.remove(opInstantPair.getValue().getFileGroupId());
|
||||
});
|
||||
|
||||
return fileGroups;
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
/**
|
||||
* With async compaction, it is possible to see partial/complete data-files due to inflight-compactions, Ignore
|
||||
* those data-files
|
||||
*
|
||||
* @param dataFile Data File
|
||||
*/
|
||||
private boolean isDataFileDueToPendingCompaction(HoodieDataFile dataFile) {
|
||||
final String partitionPath = getPartitionPathFromFileStatus(dataFile.getFileStatus());
|
||||
HoodieFileGroupId fgId = new HoodieFileGroupId(partitionPath, dataFile.getFileId());
|
||||
Pair<String, CompactionOperation> compactionWithInstantTime = fgIdToPendingCompaction.get(fgId);
|
||||
if ((null != compactionWithInstantTime) && (null != compactionWithInstantTime.getLeft())
|
||||
&& dataFile.getCommitTime().equals(compactionWithInstantTime.getKey())) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFiles(final String partitionPath) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> {
|
||||
return fileGroup.getAllDataFiles()
|
||||
.filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst();
|
||||
})
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFiles() {
|
||||
return fileGroupMap.values().stream()
|
||||
.map(fileGroup -> fileGroup.getAllDataFiles().filter(df -> !isDataFileDueToPendingCompaction(df)).findFirst())
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath,
|
||||
String maxCommitTime) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> fileGroup.getAllDataFiles()
|
||||
.filter(dataFile ->
|
||||
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
|
||||
maxCommitTime,
|
||||
HoodieTimeline.LESSER_OR_EQUAL))
|
||||
.filter(df -> !isDataFileDueToPendingCompaction(df))
|
||||
.findFirst())
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
|
||||
return fileGroupMap.values().stream()
|
||||
.map(fileGroup -> fileGroup.getAllDataFiles()
|
||||
.filter(dataFile -> commitsToReturn.contains(dataFile.getCommitTime())
|
||||
&& !isDataFileDueToPendingCompaction(dataFile))
|
||||
.findFirst())
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFilesOn(String partitionPath, String instantTime) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> fileGroup.getAllDataFiles()
|
||||
.filter(dataFile ->
|
||||
HoodieTimeline.compareTimestamps(dataFile.getCommitTime(),
|
||||
instantTime,
|
||||
HoodieTimeline.EQUAL))
|
||||
.filter(df -> !isDataFileDueToPendingCompaction(df))
|
||||
.findFirst())
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(HoodieFileGroup::getAllDataFiles)
|
||||
.flatMap(dataFileList -> dataFileList)
|
||||
.filter(df -> !isDataFileDueToPendingCompaction(df));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(HoodieFileGroup::getLatestFileSlice)
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get)
|
||||
.map(fs -> filterDataFileAfterPendingCompaction(fs));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionPath) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> {
|
||||
FileSlice fileSlice = fileGroup.getLatestFileSlice().get();
|
||||
// if the file-group is under compaction, pick the latest before compaction instant time.
|
||||
if (isFileSliceAfterPendingCompaction(fileSlice)) {
|
||||
String compactionInstantTime = fgIdToPendingCompaction.get(fileSlice.getFileGroupId()).getLeft();
|
||||
return fileGroup.getLatestFileSliceBefore(compactionInstantTime);
|
||||
}
|
||||
return Optional.of(fileSlice);
|
||||
})
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the file-group is under pending-compaction and the file-slice' baseInstant matches
|
||||
* compaction Instant
|
||||
* @param fileSlice File Slice
|
||||
* @return
|
||||
*/
|
||||
private boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) {
|
||||
Pair<String, CompactionOperation> compactionWithInstantTime =
|
||||
fgIdToPendingCompaction.get(fileSlice.getFileGroupId());
|
||||
return (null != compactionWithInstantTime)
|
||||
&& fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.getKey());
|
||||
}
|
||||
|
||||
/**
|
||||
* With async compaction, it is possible to see partial/complete data-files due to inflight-compactions,
|
||||
* Ignore those data-files
|
||||
* @param fileSlice File Slice
|
||||
* @return
|
||||
*/
|
||||
private FileSlice filterDataFileAfterPendingCompaction(FileSlice fileSlice) {
|
||||
if (isFileSliceAfterPendingCompaction(fileSlice)) {
|
||||
// Data file is filtered out of the file-slice as the corresponding compaction
|
||||
// instant not completed yet.
|
||||
FileSlice transformed = new FileSlice(fileSlice.getPartitionPath(),
|
||||
fileSlice.getBaseInstantTime(), fileSlice.getFileId());
|
||||
fileSlice.getLogFiles().forEach(transformed::addLogFile);
|
||||
return transformed;
|
||||
}
|
||||
return fileSlice;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath,
|
||||
String maxCommitTime) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> fileGroup.getLatestFileSliceBeforeOrOn(maxCommitTime))
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get)
|
||||
.map(fs -> filterDataFileAfterPendingCompaction(fs));
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to merge last 2 file-slices. These 2 file-slices do not have compaction done yet.
|
||||
*
|
||||
* @param lastSlice Latest File slice for a file-group
|
||||
* @param penultimateSlice Penultimate file slice for a file-group in commit timeline order
|
||||
*/
|
||||
private static FileSlice mergeCompactionPendingFileSlices(FileSlice lastSlice, FileSlice penultimateSlice) {
|
||||
FileSlice merged = new FileSlice(penultimateSlice.getPartitionPath(),
|
||||
penultimateSlice.getBaseInstantTime(), penultimateSlice.getFileId());
|
||||
if (penultimateSlice.getDataFile().isPresent()) {
|
||||
merged.setDataFile(penultimateSlice.getDataFile().get());
|
||||
}
|
||||
// Add Log files from penultimate and last slices
|
||||
penultimateSlice.getLogFiles().forEach(merged::addLogFile);
|
||||
lastSlice.getLogFiles().forEach(merged::addLogFile);
|
||||
return merged;
|
||||
}
|
||||
|
||||
/**
|
||||
* If the file-slice is because of pending compaction instant, this method merges the file-slice with the one before
|
||||
* the compaction instant time
|
||||
* @param fileGroup File Group for which the file slice belongs to
|
||||
* @param fileSlice File Slice which needs to be merged
|
||||
* @return
|
||||
*/
|
||||
private FileSlice getMergedFileSlice(HoodieFileGroup fileGroup, FileSlice fileSlice) {
|
||||
// if the file-group is under construction, pick the latest before compaction instant time.
|
||||
HoodieFileGroupId fgId = fileSlice.getFileGroupId();
|
||||
if (fgIdToPendingCompaction.containsKey(fgId)) {
|
||||
String compactionInstantTime = fgIdToPendingCompaction.get(fgId).getKey();
|
||||
if (fileSlice.getBaseInstantTime().equals(compactionInstantTime)) {
|
||||
Optional<FileSlice> prevFileSlice = fileGroup.getLatestFileSliceBefore(compactionInstantTime);
|
||||
if (prevFileSlice.isPresent()) {
|
||||
return mergeCompactionPendingFileSlices(fileSlice, prevFileSlice.get());
|
||||
}
|
||||
}
|
||||
}
|
||||
return fileSlice;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(fileGroup -> {
|
||||
Optional<FileSlice> fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime);
|
||||
// if the file-group is under construction, pick the latest before compaction instant time.
|
||||
if (fileSlice.isPresent()) {
|
||||
fileSlice = Optional.of(getMergedFileSlice(fileGroup, fileSlice.get()));
|
||||
}
|
||||
return fileSlice;
|
||||
})
|
||||
.filter(Optional::isPresent)
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
|
||||
return fileGroupMap.values().stream()
|
||||
.map(fileGroup -> fileGroup.getLatestFileSliceInRange(commitsToReturn))
|
||||
.map(Optional::get);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
|
||||
return getAllFileGroups(partitionPath)
|
||||
.map(HoodieFileGroup::getAllFileSlices)
|
||||
.flatMap(sliceList -> sliceList);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -418,30 +163,55 @@ public class HoodieTableFileSystemView implements TableFileSystemView,
|
||||
* 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 = FSUtils.getPartitionPath(metaClient.getBasePath(), partitionPathStr);
|
||||
FSUtils.createPathIfNotExists(metaClient.getFs(), partitionPath);
|
||||
FileStatus[] statuses = metaClient.getFs().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);
|
||||
}
|
||||
}
|
||||
|
||||
public Map<HoodieFileGroupId, Pair<String, CompactionOperation>> getFgIdToPendingCompaction() {
|
||||
return fgIdToPendingCompaction;
|
||||
Stream<HoodieFileGroup> fetchAllStoredFileGroups(String partition) {
|
||||
final List<HoodieFileGroup> fileGroups = new ArrayList<>();
|
||||
fileGroups.addAll(partitionToFileGroupsMap.get(partition));
|
||||
return fileGroups.stream();
|
||||
}
|
||||
|
||||
public Stream<HoodieFileGroup> getAllFileGroups() {
|
||||
return fileGroupMap.values().stream();
|
||||
return fetchAllStoredFileGroups();
|
||||
}
|
||||
|
||||
@Override
|
||||
Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
|
||||
return fgIdToPendingCompaction.values().stream();
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(HoodieFileGroupId fgId) {
|
||||
return Option.ofNullable(fgIdToPendingCompaction.get(fgId));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean isPartitionAvailableInStore(String partitionPath) {
|
||||
return partitionToFileGroupsMap.containsKey(partitionPath);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups) {
|
||||
log.info("Adding file-groups for partition :" + partitionPath + ", #FileGroups=" + fileGroups.size());
|
||||
List<HoodieFileGroup> newList = new ArrayList<>(fileGroups);
|
||||
partitionToFileGroupsMap.put(partitionPath, newList);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
|
||||
return partitionToFileGroupsMap.values().stream().flatMap(fg -> {
|
||||
return fg.stream();
|
||||
});
|
||||
}
|
||||
|
||||
public void close() {
|
||||
closed = true;
|
||||
super.reset();
|
||||
partitionToFileGroupsMap = null;
|
||||
fgIdToPendingCompaction = null;
|
||||
}
|
||||
|
||||
public boolean isClosed() {
|
||||
return closed;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,338 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.avro.model.HoodieCleanMetadata;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.avro.model.HoodieRestoreMetadata;
|
||||
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.common.util.CompactionUtils;
|
||||
import com.uber.hoodie.common.util.TimelineDiffHelper;
|
||||
import com.uber.hoodie.common.util.TimelineDiffHelper.TimelineDiffResult;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Adds the capability to incrementally sync the changes to file-system view as and when new instants gets completed.
|
||||
*/
|
||||
public abstract class IncrementalTimelineSyncFileSystemView extends AbstractTableFileSystemView {
|
||||
|
||||
private static Logger log = LogManager.getLogger(IncrementalTimelineSyncFileSystemView.class);
|
||||
|
||||
// Allows incremental Timeline syncing
|
||||
private final boolean incrementalTimelineSyncEnabled;
|
||||
|
||||
protected IncrementalTimelineSyncFileSystemView(boolean enableIncrementalTimelineSync) {
|
||||
this.incrementalTimelineSyncEnabled = enableIncrementalTimelineSync;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void runSync(HoodieTimeline oldTimeline, HoodieTimeline newTimeline) {
|
||||
try {
|
||||
if (incrementalTimelineSyncEnabled) {
|
||||
TimelineDiffResult diffResult = TimelineDiffHelper.getNewInstantsForIncrementalSync(oldTimeline, newTimeline);
|
||||
if (diffResult.canSyncIncrementally()) {
|
||||
log.info("Doing incremental sync");
|
||||
runIncrementalSync(newTimeline, diffResult);
|
||||
log.info("Finished incremental sync");
|
||||
// Reset timeline to latest
|
||||
visibleActiveTimeline = newTimeline;
|
||||
return;
|
||||
}
|
||||
}
|
||||
} catch (Exception ioe) {
|
||||
log.error("Got exception trying to perform incremental sync. Reverting to complete sync", ioe);
|
||||
}
|
||||
|
||||
log.warn("Incremental Sync of timeline is turned off or deemed unsafe. Will revert to full syncing");
|
||||
super.runSync(oldTimeline, newTimeline);
|
||||
}
|
||||
|
||||
/**
|
||||
* Run incremental sync based on the diff result produced.
|
||||
*
|
||||
* @param timeline New Timeline
|
||||
* @param diffResult Timeline Diff Result
|
||||
*/
|
||||
private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diffResult) {
|
||||
|
||||
log.info("Timeline Diff Result is :" + diffResult);
|
||||
|
||||
// First remove pending compaction instants which were completed
|
||||
diffResult.getFinishedCompactionInstants().stream().forEach(instant -> {
|
||||
try {
|
||||
removePendingCompactionInstant(timeline, instant);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
|
||||
// Add new completed instants found in the latest timeline
|
||||
diffResult.getNewlySeenInstants().stream()
|
||||
.filter(instant -> instant.isCompleted() || instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION))
|
||||
.forEach(instant -> {
|
||||
try {
|
||||
if (instant.getAction().equals(HoodieTimeline.COMMIT_ACTION)
|
||||
|| instant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)) {
|
||||
addCommitInstant(timeline, instant);
|
||||
} else if (instant.getAction().equals(HoodieTimeline.RESTORE_ACTION)) {
|
||||
addRestoreInstant(timeline, instant);
|
||||
} else if (instant.getAction().equals(HoodieTimeline.CLEAN_ACTION)) {
|
||||
addCleanInstant(timeline, instant);
|
||||
} else if (instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) {
|
||||
addPendingCompactionInstant(timeline, instant);
|
||||
} else if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) {
|
||||
addRollbackInstant(timeline, instant);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieException(ioe);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Remove Pending compaction instant
|
||||
*
|
||||
* @param timeline New Hoodie Timeline
|
||||
* @param instant Compaction Instant to be removed
|
||||
*/
|
||||
private void removePendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
|
||||
log.info("Removing completed compaction instant (" + instant + ")");
|
||||
HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp());
|
||||
removePendingCompactionOperations(CompactionUtils.getPendingCompactionOperations(instant, plan)
|
||||
.map(instantPair -> Pair.of(instantPair.getValue().getKey(),
|
||||
CompactionOperation.convertFromAvroRecordInstance(instantPair.getValue().getValue()))));
|
||||
}
|
||||
|
||||
/**
|
||||
* Add newly found compaction instant
|
||||
*
|
||||
* @param timeline Hoodie Timeline
|
||||
* @param instant Compaction Instant
|
||||
*/
|
||||
private void addPendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
|
||||
log.info("Syncing pending compaction instant (" + instant + ")");
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp());
|
||||
List<Pair<String, CompactionOperation>> pendingOps =
|
||||
CompactionUtils.getPendingCompactionOperations(instant, compactionPlan).map(p -> Pair.of(p.getValue().getKey(),
|
||||
CompactionOperation.convertFromAvroRecordInstance(p.getValue().getValue()))).collect(Collectors.toList());
|
||||
// First, update Pending compaction instants
|
||||
addPendingCompactionOperations(pendingOps.stream());
|
||||
|
||||
Map<String, List<Pair<String, HoodieFileGroup>>> partitionToFileGroups =
|
||||
pendingOps.stream().map(opPair -> {
|
||||
String compactionInstantTime = opPair.getKey();
|
||||
HoodieFileGroup fileGroup = new HoodieFileGroup(opPair.getValue().getFileGroupId(), timeline);
|
||||
fileGroup.addNewFileSliceAtInstant(compactionInstantTime);
|
||||
return Pair.of(compactionInstantTime, fileGroup);
|
||||
}).collect(Collectors.groupingBy(x -> x.getValue().getPartitionPath()));
|
||||
partitionToFileGroups.entrySet().forEach(entry -> {
|
||||
if (isPartitionAvailableInStore(entry.getKey())) {
|
||||
applyDeltaFileSlicesToPartitionView(entry.getKey(),
|
||||
entry.getValue().stream().map(Pair::getValue).collect(Collectors.toList()), DeltaApplyMode.ADD);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Add newly found commit/delta-commit instant
|
||||
*
|
||||
* @param timeline Hoodie Timeline
|
||||
* @param instant Instant
|
||||
*/
|
||||
private void addCommitInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
|
||||
log.info("Syncing committed instant (" + instant + ")");
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(),
|
||||
HoodieCommitMetadata.class);
|
||||
commitMetadata.getPartitionToWriteStats().entrySet().stream().forEach(entry -> {
|
||||
String partition = entry.getKey();
|
||||
if (isPartitionAvailableInStore(partition)) {
|
||||
log.info("Syncing partition (" + partition + ") of instant (" + instant + ")");
|
||||
FileStatus[] statuses = entry.getValue().stream().map(p -> {
|
||||
FileStatus status = new FileStatus(p.getFileSizeInBytes(), false, 0, 0, 0, 0, null, null, null,
|
||||
new Path(String.format("%s/%s", metaClient.getBasePath(), p.getPath())));
|
||||
return status;
|
||||
}).toArray(FileStatus[]::new);
|
||||
List<HoodieFileGroup> fileGroups = buildFileGroups(statuses, timeline.filterCompletedAndCompactionInstants(),
|
||||
false);
|
||||
applyDeltaFileSlicesToPartitionView(partition, fileGroups, DeltaApplyMode.ADD);
|
||||
} else {
|
||||
log.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
|
||||
}
|
||||
});
|
||||
log.info("Done Syncing committed instant (" + instant + ")");
|
||||
}
|
||||
|
||||
/**
|
||||
* Add newly found restore instant
|
||||
*
|
||||
* @param timeline Hoodie Timeline
|
||||
* @param instant Restore Instant
|
||||
*/
|
||||
private void addRestoreInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
|
||||
log.info("Syncing restore instant (" + instant + ")");
|
||||
HoodieRestoreMetadata metadata = AvroUtils.deserializeAvroMetadata(
|
||||
timeline.getInstantDetails(instant).get(), HoodieRestoreMetadata.class);
|
||||
|
||||
Map<String, List<Pair<String, String>>> partitionFiles =
|
||||
metadata.getHoodieRestoreMetadata().entrySet().stream().flatMap(entry -> {
|
||||
return entry.getValue().stream().flatMap(e -> e.getPartitionMetadata().entrySet().stream().flatMap(e2 -> {
|
||||
return e2.getValue().getSuccessDeleteFiles().stream().map(x -> Pair.of(e2.getKey(), x));
|
||||
}));
|
||||
}).collect(Collectors.groupingBy(Pair::getKey));
|
||||
partitionFiles.entrySet().stream().forEach(e -> {
|
||||
removeFileSlicesForPartition(timeline, instant, e.getKey(),
|
||||
e.getValue().stream().map(x -> x.getValue()).collect(Collectors.toList()));
|
||||
});
|
||||
log.info("Done Syncing restore instant (" + instant + ")");
|
||||
}
|
||||
|
||||
/**
|
||||
* Add newly found rollback instant
|
||||
*
|
||||
* @param timeline Hoodie Timeline
|
||||
* @param instant Rollback Instant
|
||||
*/
|
||||
private void addRollbackInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
|
||||
log.info("Syncing rollback instant (" + instant + ")");
|
||||
HoodieRollbackMetadata metadata = AvroUtils.deserializeAvroMetadata(
|
||||
timeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class);
|
||||
|
||||
metadata.getPartitionMetadata().entrySet().stream().forEach(e -> {
|
||||
removeFileSlicesForPartition(timeline, instant, e.getKey(), e.getValue().getSuccessDeleteFiles());
|
||||
});
|
||||
log.info("Done Syncing rollback instant (" + instant + ")");
|
||||
}
|
||||
|
||||
/**
|
||||
* Add newly found clean instant
|
||||
*
|
||||
* @param timeline Timeline
|
||||
* @param instant Clean instant
|
||||
*/
|
||||
private void addCleanInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException {
|
||||
log.info("Syncing cleaner instant (" + instant + ")");
|
||||
HoodieCleanMetadata cleanMetadata = AvroUtils
|
||||
.deserializeHoodieCleanMetadata(timeline.getInstantDetails(instant).get());
|
||||
cleanMetadata.getPartitionMetadata().entrySet().stream().forEach(entry -> {
|
||||
removeFileSlicesForPartition(timeline, instant, entry.getKey(), entry.getValue().getSuccessDeleteFiles());
|
||||
});
|
||||
log.info("Done Syncing cleaner instant (" + instant + ")");
|
||||
}
|
||||
|
||||
private void removeFileSlicesForPartition(HoodieTimeline timeline, HoodieInstant instant,
|
||||
String partition, List<String> paths) {
|
||||
if (isPartitionAvailableInStore(partition)) {
|
||||
log.info("Removing file slices for partition (" + partition + ") for instant (" + instant + ")");
|
||||
FileStatus[] statuses = paths.stream().map(p -> {
|
||||
FileStatus status = new FileStatus();
|
||||
status.setPath(new Path(p));
|
||||
return status;
|
||||
}).toArray(FileStatus[]::new);
|
||||
List<HoodieFileGroup> fileGroups = buildFileGroups(statuses,
|
||||
timeline.filterCompletedAndCompactionInstants(), false);
|
||||
applyDeltaFileSlicesToPartitionView(partition, fileGroups, DeltaApplyMode.REMOVE);
|
||||
} else {
|
||||
log.warn("Skipping partition (" + partition + ") when syncing instant (" + instant + ") as it is not loaded");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply mode whether to add or remove the delta view
|
||||
*/
|
||||
enum DeltaApplyMode {
|
||||
ADD,
|
||||
REMOVE
|
||||
}
|
||||
|
||||
/**
|
||||
* Apply changes to partition file-system view. Base Implementation overwrites the entire partitions view assuming
|
||||
* some sort of map (in-mem/disk-based) is used. For View implementation which supports fine-granular updates (e:g
|
||||
* RocksDB), override this method.
|
||||
*
|
||||
* @param partition PartitionPath
|
||||
* @param deltaFileGroups Changed file-slices aggregated as file-groups
|
||||
* @param mode Delta Apply mode
|
||||
*/
|
||||
protected void applyDeltaFileSlicesToPartitionView(String partition, List<HoodieFileGroup> deltaFileGroups,
|
||||
DeltaApplyMode mode) {
|
||||
if (deltaFileGroups.isEmpty()) {
|
||||
log.info("No delta file groups for partition :" + partition);
|
||||
return;
|
||||
}
|
||||
|
||||
List<HoodieFileGroup> fileGroups = fetchAllStoredFileGroups(partition).collect(Collectors.toList());
|
||||
/**
|
||||
* Note that while finding the new data/log files added/removed, the path stored in metadata will be missing
|
||||
* the base-path,scheme and authority. Ensure the matching process takes care of this discrepancy.
|
||||
*/
|
||||
Map<String, HoodieDataFile> viewDataFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
|
||||
.map(FileSlice::getDataFile).filter(Optional::isPresent).map(Optional::get)
|
||||
.map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
//Note: Delta Log Files and Data FIles can be empty when adding/removing pending compactions
|
||||
Map<String, HoodieDataFile> deltaDataFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
|
||||
.map(FileSlice::getDataFile).filter(Optional::isPresent).map(Optional::get)
|
||||
.map(df -> Pair.of(Path.getPathWithoutSchemeAndAuthority(new Path(df.getPath())).toString(), df))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
Map<String, HoodieLogFile> viewLogFiles = fileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
|
||||
.flatMap(FileSlice::getLogFiles)
|
||||
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
Map<String, HoodieLogFile> deltaLogFiles = deltaFileGroups.stream().flatMap(HoodieFileGroup::getAllRawFileSlices)
|
||||
.flatMap(FileSlice::getLogFiles)
|
||||
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
switch (mode) {
|
||||
case ADD:
|
||||
viewDataFiles.putAll(deltaDataFiles);
|
||||
viewLogFiles.putAll(deltaLogFiles);
|
||||
break;
|
||||
case REMOVE:
|
||||
deltaDataFiles.keySet().stream().forEach(p -> viewDataFiles.remove(p));
|
||||
deltaLogFiles.keySet().stream().forEach(p -> viewLogFiles.remove(p));
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Unknown diff apply mode=" + mode);
|
||||
}
|
||||
|
||||
HoodieTimeline timeline = deltaFileGroups.stream().map(df -> df.getTimeline()).findAny().get();
|
||||
List<HoodieFileGroup> fgs =
|
||||
buildFileGroups(viewDataFiles.values().stream(), viewLogFiles.values().stream(), timeline, true);
|
||||
storePartitionView(partition, fgs);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,238 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.Functions.Function0;
|
||||
import com.uber.hoodie.common.util.Functions.Function1;
|
||||
import com.uber.hoodie.common.util.Functions.Function2;
|
||||
import com.uber.hoodie.common.util.Functions.Function3;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* A file system view which proxies request to a preferred File System View implementation. In case of error,
|
||||
* flip all subsequent calls to a backup file-system view implementation.
|
||||
*/
|
||||
public class PriorityBasedFileSystemView implements SyncableFileSystemView, Serializable {
|
||||
|
||||
private static Logger log = LogManager.getLogger(PriorityBasedFileSystemView.class);
|
||||
|
||||
private final SyncableFileSystemView preferredView;
|
||||
private final SyncableFileSystemView secondaryView;
|
||||
private boolean errorOnPreferredView;
|
||||
|
||||
public PriorityBasedFileSystemView(SyncableFileSystemView preferredView, SyncableFileSystemView secondaryView) {
|
||||
this.preferredView = preferredView;
|
||||
this.secondaryView = secondaryView;
|
||||
this.errorOnPreferredView = false;
|
||||
}
|
||||
|
||||
private <R> R execute(Function0<R> preferredFunction, Function0<R> secondaryFunction) {
|
||||
if (errorOnPreferredView) {
|
||||
log.warn("Routing request to secondary file-system view");
|
||||
return secondaryFunction.apply();
|
||||
} else {
|
||||
try {
|
||||
return preferredFunction.apply();
|
||||
} catch (RuntimeException re) {
|
||||
log.error("Got error running preferred function. Trying secondary", re);
|
||||
errorOnPreferredView = true;
|
||||
return secondaryFunction.apply();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private <T1, R> R execute(T1 val, Function1<T1, R> preferredFunction, Function1<T1, R> secondaryFunction) {
|
||||
if (errorOnPreferredView) {
|
||||
log.warn("Routing request to secondary file-system view");
|
||||
return secondaryFunction.apply(val);
|
||||
} else {
|
||||
try {
|
||||
return preferredFunction.apply(val);
|
||||
} catch (RuntimeException re) {
|
||||
log.error("Got error running preferred function. Trying secondary", re);
|
||||
errorOnPreferredView = true;
|
||||
return secondaryFunction.apply(val);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private <T1, T2, R> R execute(T1 val, T2 val2, Function2<T1, T2, R> preferredFunction,
|
||||
Function2<T1, T2, R> secondaryFunction) {
|
||||
if (errorOnPreferredView) {
|
||||
log.warn("Routing request to secondary file-system view");
|
||||
return secondaryFunction.apply(val, val2);
|
||||
} else {
|
||||
try {
|
||||
return preferredFunction.apply(val, val2);
|
||||
} catch (RuntimeException re) {
|
||||
log.error("Got error running preferred function. Trying secondary", re);
|
||||
errorOnPreferredView = true;
|
||||
return secondaryFunction.apply(val, val2);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private <T1, T2, T3, R> R execute(T1 val, T2 val2, T3 val3, Function3<T1, T2, T3, R> preferredFunction,
|
||||
Function3<T1, T2, T3, R> secondaryFunction) {
|
||||
if (errorOnPreferredView) {
|
||||
log.warn("Routing request to secondary file-system view");
|
||||
return secondaryFunction.apply(val, val2, val3);
|
||||
} else {
|
||||
try {
|
||||
return preferredFunction.apply(val, val2, val3);
|
||||
} catch (RuntimeException re) {
|
||||
log.error("Got error running preferred function. Trying secondary", re);
|
||||
errorOnPreferredView = true;
|
||||
return secondaryFunction.apply(val, val2, val3);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFiles(String partitionPath) {
|
||||
return execute(partitionPath, preferredView::getLatestDataFiles, secondaryView::getLatestDataFiles);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFiles() {
|
||||
return execute(preferredView::getLatestDataFiles, secondaryView::getLatestDataFiles);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
|
||||
return execute(partitionPath, maxCommitTime, preferredView::getLatestDataFilesBeforeOrOn,
|
||||
secondaryView::getLatestDataFilesBeforeOrOn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId) {
|
||||
return execute(partitionPath, fileId, preferredView::getLatestDataFile, secondaryView::getLatestDataFile);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId) {
|
||||
return execute(partitionPath, instantTime, fileId, preferredView::getDataFileOn,
|
||||
secondaryView::getDataFileOn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
|
||||
return execute(commitsToReturn, preferredView::getLatestDataFilesInRange, secondaryView::getLatestDataFilesInRange);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
|
||||
return execute(partitionPath, preferredView::getAllDataFiles, secondaryView::getAllDataFiles);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
|
||||
return execute(partitionPath, preferredView::getLatestFileSlices, secondaryView::getLatestFileSlices);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionPath) {
|
||||
return execute(partitionPath, preferredView::getLatestUnCompactedFileSlices,
|
||||
secondaryView::getLatestUnCompactedFileSlices);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime) {
|
||||
return execute(partitionPath, maxCommitTime, preferredView::getLatestFileSlicesBeforeOrOn,
|
||||
secondaryView::getLatestFileSlicesBeforeOrOn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime) {
|
||||
return execute(partitionPath, maxInstantTime, preferredView::getLatestMergedFileSlicesBeforeOrOn,
|
||||
secondaryView::getLatestMergedFileSlicesBeforeOrOn);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
|
||||
return execute(commitsToReturn, preferredView::getLatestFileSliceInRange, secondaryView::getLatestFileSliceInRange);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
|
||||
return execute(partitionPath, preferredView::getAllFileSlices, secondaryView::getAllFileSlices);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieFileGroup> getAllFileGroups(String partitionPath) {
|
||||
return execute(partitionPath, preferredView::getAllFileGroups, secondaryView::getAllFileGroups);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
|
||||
return execute(preferredView::getPendingCompactionOperations, secondaryView::getPendingCompactionOperations);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
preferredView.close();
|
||||
secondaryView.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() {
|
||||
preferredView.reset();
|
||||
secondaryView.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieInstant> getLastInstant() {
|
||||
return execute(preferredView::getLastInstant, secondaryView::getLastInstant);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline getTimeline() {
|
||||
return execute(preferredView::getTimeline, secondaryView::getTimeline);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sync() {
|
||||
preferredView.reset();
|
||||
secondaryView.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<FileSlice> getLatestFileSlice(String partitionPath, String fileId) {
|
||||
return execute(partitionPath, fileId, preferredView::getLatestFileSlice, secondaryView::getLatestFileSlice);
|
||||
}
|
||||
|
||||
public SyncableFileSystemView getPreferredView() {
|
||||
return preferredView;
|
||||
}
|
||||
|
||||
public SyncableFileSystemView getSecondaryView() {
|
||||
return secondaryView;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,469 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.timeline.dto.CompactionOpDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.DataFileDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.FileGroupDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.FileSliceDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.InstantDTO;
|
||||
import com.uber.hoodie.common.table.timeline.dto.TimelineDTO;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.StringUtils;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieRemoteException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.http.client.fluent.Request;
|
||||
import org.apache.http.client.fluent.Response;
|
||||
import org.apache.http.client.utils.URIBuilder;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* A proxy for table file-system view which translates local View API calls to REST calls to remote timeline service
|
||||
*/
|
||||
public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, Serializable {
|
||||
|
||||
private static final String BASE_URL = "/v1/hoodie/view";
|
||||
public static final String LATEST_PARTITION_SLICES_URL = String.format("%s/%s", BASE_URL,
|
||||
"slices/partition/latest/");
|
||||
public static final String LATEST_PARTITION_SLICE_URL = String.format("%s/%s", BASE_URL,
|
||||
"slices/file/latest/");
|
||||
public static final String LATEST_PARTITION_UNCOMPACTED_SLICES_URL = String.format("%s/%s", BASE_URL,
|
||||
"slices/uncompacted/partition/latest/");
|
||||
public static final String ALL_SLICES_URL = String.format("%s/%s", BASE_URL, "slices/all");
|
||||
public static final String LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL =
|
||||
String.format("%s/%s", BASE_URL, "slices/merged/beforeoron/latest/");
|
||||
public static final String LATEST_SLICES_RANGE_INSTANT_URL =
|
||||
String.format("%s/%s", BASE_URL, "slices/range/latest/");
|
||||
public static final String LATEST_SLICES_BEFORE_ON_INSTANT_URL =
|
||||
String.format("%s/%s", BASE_URL, "slices/beforeoron/latest/");
|
||||
|
||||
public static final String PENDING_COMPACTION_OPS =
|
||||
String.format("%s/%s", BASE_URL, "compactions/pending/");
|
||||
|
||||
public static final String LATEST_PARTITION_DATA_FILES_URL = String.format("%s/%s", BASE_URL,
|
||||
"datafiles/latest/partition");
|
||||
public static final String LATEST_PARTITION_DATA_FILE_URL = String.format("%s/%s", BASE_URL,
|
||||
"datafile/latest/partition");
|
||||
public static final String ALL_DATA_FILES = String.format("%s/%s", BASE_URL, "datafiles/all");
|
||||
public static final String LATEST_ALL_DATA_FILES = String.format("%s/%s", BASE_URL, "datafiles/all/latest/");
|
||||
public static final String LATEST_DATA_FILE_ON_INSTANT_URL =
|
||||
String.format("%s/%s", BASE_URL, "datafile/on/latest/");
|
||||
|
||||
public static final String LATEST_DATA_FILES_RANGE_INSTANT_URL =
|
||||
String.format("%s/%s", BASE_URL, "datafiles/range/latest/");
|
||||
public static final String LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL =
|
||||
String.format("%s/%s", BASE_URL, "datafiles/beforeoron/latest/");
|
||||
|
||||
public static final String ALL_FILEGROUPS_FOR_PARTITION_URL =
|
||||
String.format("%s/%s", BASE_URL, "filegroups/all/partition/");
|
||||
|
||||
public static final String LAST_INSTANT = String.format("%s/%s", BASE_URL, "timeline/instant/last");
|
||||
public static final String LAST_INSTANTS = String.format("%s/%s", BASE_URL, "timeline/instants/last");
|
||||
|
||||
public static final String TIMELINE = String.format("%s/%s", BASE_URL, "timeline/instants/all");
|
||||
|
||||
// POST Requests
|
||||
public static final String REFRESH_DATASET = String.format("%s/%s", BASE_URL, "refresh/");
|
||||
|
||||
public static final String PARTITION_PARAM = "partition";
|
||||
public static final String BASEPATH_PARAM = "basepath";
|
||||
public static final String INSTANT_PARAM = "instant";
|
||||
public static final String MAX_INSTANT_PARAM = "maxinstant";
|
||||
public static final String INSTANTS_PARAM = "instants";
|
||||
public static final String FILEID_PARAM = "fileid";
|
||||
public static final String LAST_INSTANT_TS = "lastinstantts";
|
||||
public static final String TIMELINE_HASH = "timelinehash";
|
||||
public static final String REFRESH_OFF = "refreshoff";
|
||||
|
||||
private static Logger log = LogManager.getLogger(RemoteHoodieTableFileSystemView.class);
|
||||
|
||||
private final String serverHost;
|
||||
private final int serverPort;
|
||||
private final String basePath;
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieTimeline timeline;
|
||||
private final ObjectMapper mapper;
|
||||
|
||||
private boolean closed = false;
|
||||
|
||||
private enum RequestMethod {
|
||||
GET,
|
||||
POST
|
||||
}
|
||||
|
||||
public RemoteHoodieTableFileSystemView(String server, int port, HoodieTableMetaClient metaClient) {
|
||||
this.basePath = metaClient.getBasePath();
|
||||
this.serverHost = server;
|
||||
this.serverPort = port;
|
||||
this.mapper = new ObjectMapper();
|
||||
this.metaClient = metaClient;
|
||||
this.timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
|
||||
}
|
||||
|
||||
private <T> T executeRequest(String requestPath, Map<String, String> queryParameters, TypeReference reference,
|
||||
RequestMethod method) throws IOException {
|
||||
Preconditions.checkArgument(!closed, "View already closed");
|
||||
|
||||
URIBuilder builder = new URIBuilder().setHost(serverHost).setPort(serverPort).setPath(requestPath)
|
||||
.setScheme("http");
|
||||
|
||||
queryParameters.entrySet().stream().forEach(entry -> {
|
||||
builder.addParameter(entry.getKey(), entry.getValue());
|
||||
});
|
||||
|
||||
// Adding mandatory parameters - Last instants affecting file-slice
|
||||
timeline.lastInstant().ifPresent(instant -> builder.addParameter(LAST_INSTANT_TS, instant.getTimestamp()));
|
||||
builder.addParameter(TIMELINE_HASH, timeline.getTimelineHash());
|
||||
|
||||
String url = builder.toString();
|
||||
log.info("Sending request : (" + url + ")");
|
||||
Response response = null;
|
||||
int timeout = 1000 * 300; // 5 min timeout
|
||||
switch (method) {
|
||||
case GET:
|
||||
response = Request.Get(url).connectTimeout(timeout).socketTimeout(timeout).execute();
|
||||
break;
|
||||
case POST:
|
||||
default:
|
||||
response = Request.Post(url).connectTimeout(timeout).socketTimeout(timeout).execute();
|
||||
break;
|
||||
}
|
||||
String content = response.returnContent().asString();
|
||||
return mapper.readValue(content, reference);
|
||||
}
|
||||
|
||||
private Map<String, String> getParamsWithPartitionPath(String partitionPath) {
|
||||
Map<String, String> paramsMap = new HashMap<>();
|
||||
paramsMap.put(BASEPATH_PARAM, basePath);
|
||||
paramsMap.put(PARTITION_PARAM, partitionPath);
|
||||
return paramsMap;
|
||||
}
|
||||
|
||||
private Map<String, String> getParams() {
|
||||
Map<String, String> paramsMap = new HashMap<>();
|
||||
paramsMap.put(BASEPATH_PARAM, basePath);
|
||||
return paramsMap;
|
||||
}
|
||||
|
||||
private Map<String, String> getParams(String paramName, String instant) {
|
||||
Map<String, String> paramsMap = new HashMap<>();
|
||||
paramsMap.put(BASEPATH_PARAM, basePath);
|
||||
paramsMap.put(paramName, instant);
|
||||
return paramsMap;
|
||||
}
|
||||
|
||||
private Map<String, String> getParamsWithAdditionalParam(String partitionPath, String paramName, String paramVal) {
|
||||
Map<String, String> paramsMap = new HashMap<>();
|
||||
paramsMap.put(BASEPATH_PARAM, basePath);
|
||||
paramsMap.put(PARTITION_PARAM, partitionPath);
|
||||
paramsMap.put(paramName, paramVal);
|
||||
return paramsMap;
|
||||
}
|
||||
|
||||
private Map<String, String> getParamsWithAdditionalParams(String partitionPath, String[] paramNames,
|
||||
String[] paramVals) {
|
||||
Map<String, String> paramsMap = new HashMap<>();
|
||||
paramsMap.put(BASEPATH_PARAM, basePath);
|
||||
paramsMap.put(PARTITION_PARAM, partitionPath);
|
||||
Preconditions.checkArgument(paramNames.length == paramVals.length);
|
||||
for (int i = 0; i < paramNames.length; i++) {
|
||||
paramsMap.put(paramNames[i], paramVals[i]);
|
||||
}
|
||||
return paramsMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFiles(String partitionPath) {
|
||||
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
|
||||
try {
|
||||
List<DataFileDTO> dataFiles = executeRequest(LATEST_PARTITION_DATA_FILES_URL, paramsMap,
|
||||
new TypeReference<List<DataFileDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFiles() {
|
||||
Map<String, String> paramsMap = getParams();
|
||||
try {
|
||||
List<DataFileDTO> dataFiles = executeRequest(LATEST_ALL_DATA_FILES, paramsMap,
|
||||
new TypeReference<List<DataFileDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFilesBeforeOrOn(String partitionPath, String maxCommitTime) {
|
||||
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime);
|
||||
try {
|
||||
List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILES_BEFORE_ON_INSTANT_URL, paramsMap,
|
||||
new TypeReference<List<DataFileDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieDataFile> getDataFileOn(String partitionPath, String instantTime, String fileId) {
|
||||
Map<String, String> paramsMap = getParamsWithAdditionalParams(partitionPath,
|
||||
new String[]{INSTANT_PARAM, FILEID_PARAM},
|
||||
new String[]{instantTime, fileId});
|
||||
try {
|
||||
List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILE_ON_INSTANT_URL, paramsMap,
|
||||
new TypeReference<List<DataFileDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return Option.fromJavaOptional(dataFiles.stream().map(DataFileDTO::toHoodieDataFile).findFirst());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getLatestDataFilesInRange(List<String> commitsToReturn) {
|
||||
Map<String, String> paramsMap = getParams(INSTANTS_PARAM,
|
||||
StringUtils.join(commitsToReturn.toArray(new String[0]), ","));
|
||||
try {
|
||||
List<DataFileDTO> dataFiles = executeRequest(LATEST_DATA_FILES_RANGE_INSTANT_URL, paramsMap,
|
||||
new TypeReference<List<DataFileDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieDataFile> getAllDataFiles(String partitionPath) {
|
||||
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
|
||||
try {
|
||||
List<DataFileDTO> dataFiles = executeRequest(ALL_DATA_FILES, paramsMap,
|
||||
new TypeReference<List<DataFileDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(DataFileDTO::toHoodieDataFile);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSlices(String partitionPath) {
|
||||
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
|
||||
try {
|
||||
List<FileSliceDTO> dataFiles = executeRequest(LATEST_PARTITION_SLICES_URL, paramsMap,
|
||||
new TypeReference<List<FileSliceDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<FileSlice> getLatestFileSlice(String partitionPath, String fileId) {
|
||||
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId);
|
||||
try {
|
||||
List<FileSliceDTO> dataFiles = executeRequest(LATEST_PARTITION_SLICE_URL, paramsMap,
|
||||
new TypeReference<List<FileSliceDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return Option.fromJavaOptional(dataFiles.stream().map(FileSliceDTO::toFileSlice).findFirst());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestUnCompactedFileSlices(String partitionPath) {
|
||||
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
|
||||
try {
|
||||
List<FileSliceDTO> dataFiles = executeRequest(LATEST_PARTITION_UNCOMPACTED_SLICES_URL, paramsMap,
|
||||
new TypeReference<List<FileSliceDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSlicesBeforeOrOn(String partitionPath, String maxCommitTime) {
|
||||
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime);
|
||||
try {
|
||||
List<FileSliceDTO> dataFiles = executeRequest(LATEST_SLICES_BEFORE_ON_INSTANT_URL, paramsMap,
|
||||
new TypeReference<List<FileSliceDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestMergedFileSlicesBeforeOrOn(String partitionPath, String maxInstantTime) {
|
||||
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxInstantTime);
|
||||
try {
|
||||
List<FileSliceDTO> dataFiles = executeRequest(LATEST_SLICES_MERGED_BEFORE_ON_INSTANT_URL, paramsMap,
|
||||
new TypeReference<List<FileSliceDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getLatestFileSliceInRange(List<String> commitsToReturn) {
|
||||
Map<String, String> paramsMap = getParams(INSTANTS_PARAM,
|
||||
StringUtils.join(commitsToReturn.toArray(new String[0]), ","));
|
||||
try {
|
||||
List<FileSliceDTO> dataFiles = executeRequest(LATEST_SLICES_RANGE_INSTANT_URL, paramsMap,
|
||||
new TypeReference<List<FileSliceDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<FileSlice> getAllFileSlices(String partitionPath) {
|
||||
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
|
||||
try {
|
||||
List<FileSliceDTO> dataFiles = executeRequest(ALL_SLICES_URL, paramsMap,
|
||||
new TypeReference<List<FileSliceDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dataFiles.stream().map(FileSliceDTO::toFileSlice);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieFileGroup> getAllFileGroups(String partitionPath) {
|
||||
Map<String, String> paramsMap = getParamsWithPartitionPath(partitionPath);
|
||||
try {
|
||||
List<FileGroupDTO> fileGroups = executeRequest(ALL_FILEGROUPS_FOR_PARTITION_URL, paramsMap,
|
||||
new TypeReference<List<FileGroupDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public boolean refresh() {
|
||||
Map<String, String> paramsMap = getParams();
|
||||
try {
|
||||
return executeRequest(REFRESH_DATASET, paramsMap, new TypeReference<Boolean>() {
|
||||
}, RequestMethod.POST);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<Pair<String, CompactionOperation>> getPendingCompactionOperations() {
|
||||
Map<String, String> paramsMap = getParams();
|
||||
try {
|
||||
List<CompactionOpDTO> dtos = executeRequest(PENDING_COMPACTION_OPS, paramsMap,
|
||||
new TypeReference<List<CompactionOpDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return dtos.stream().map(CompactionOpDTO::toCompactionOperation);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
closed = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void reset() {
|
||||
refresh();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieInstant> getLastInstant() {
|
||||
Map<String, String> paramsMap = getParams();
|
||||
try {
|
||||
List<InstantDTO> instants = executeRequest(LAST_INSTANT, paramsMap,
|
||||
new TypeReference<List<InstantDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return Option.fromJavaOptional(instants.stream().map(InstantDTO::toInstant).findFirst());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTimeline getTimeline() {
|
||||
Map<String, String> paramsMap = getParams();
|
||||
try {
|
||||
TimelineDTO timeline = executeRequest(TIMELINE, paramsMap,
|
||||
new TypeReference<TimelineDTO>() {
|
||||
}, RequestMethod.GET);
|
||||
return TimelineDTO.toTimeline(timeline, metaClient);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void sync() {
|
||||
//noop
|
||||
}
|
||||
|
||||
@Override
|
||||
public Option<HoodieDataFile> getLatestDataFile(String partitionPath, String fileId) {
|
||||
Map<String, String> paramsMap = getParamsWithAdditionalParam(partitionPath, FILEID_PARAM, fileId);
|
||||
try {
|
||||
List<DataFileDTO> dataFiles = executeRequest(LATEST_PARTITION_DATA_FILE_URL, paramsMap,
|
||||
new TypeReference<List<DataFileDTO>>() {
|
||||
}, RequestMethod.GET);
|
||||
return Option.fromJavaOptional(dataFiles.stream().map(DataFileDTO::toHoodieDataFile).findFirst());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieRemoteException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,341 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.common.util.RocksDBDAO;
|
||||
import com.uber.hoodie.common.util.RocksDBSchemaHelper;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* A file-system view implementation on top of embedded Rocks DB store.
|
||||
* For each DataSet : 3 column Family is added for storing
|
||||
* (1) File-Slices and Data Files for View lookups
|
||||
* (2) Pending compaction operations
|
||||
* (3) Partitions tracked
|
||||
*
|
||||
* Fine-grained retrieval API to fetch latest file-slice and data-file which are common operations
|
||||
* for ingestion/compaction are supported.
|
||||
*
|
||||
* TODO: vb The current implementation works in embedded server mode where each restarts blows away the view stores.
|
||||
* To support view-state preservation across restarts, Hoodie timeline also needs to be stored
|
||||
* inorder to detect changes to timeline across restarts.
|
||||
*/
|
||||
public class RocksDbBasedFileSystemView extends IncrementalTimelineSyncFileSystemView {
|
||||
|
||||
private static Logger log = LogManager.getLogger(RocksDbBasedFileSystemView.class);
|
||||
|
||||
private final FileSystemViewStorageConfig config;
|
||||
|
||||
private final RocksDBSchemaHelper schemaHelper;
|
||||
|
||||
private RocksDBDAO rocksDB;
|
||||
|
||||
private boolean closed = false;
|
||||
|
||||
public RocksDbBasedFileSystemView(HoodieTableMetaClient metaClient,
|
||||
HoodieTimeline visibleActiveTimeline, FileSystemViewStorageConfig config) {
|
||||
super(config.isIncrementalTimelineSyncEnabled());
|
||||
this.config = config;
|
||||
this.schemaHelper = new RocksDBSchemaHelper(metaClient);
|
||||
this.rocksDB = new RocksDBDAO(metaClient.getBasePath(), config);
|
||||
init(metaClient, visibleActiveTimeline);
|
||||
}
|
||||
|
||||
public RocksDbBasedFileSystemView(HoodieTableMetaClient metaClient,
|
||||
HoodieTimeline visibleActiveTimeline, FileStatus[] fileStatuses, FileSystemViewStorageConfig config) {
|
||||
this(metaClient, visibleActiveTimeline, config);
|
||||
addFilesToView(fileStatuses);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
|
||||
schemaHelper.getAllColumnFamilies().stream().forEach(rocksDB::addColumnFamily);
|
||||
super.init(metaClient, visibleActiveTimeline);
|
||||
log.info("Created ROCKSDB based file-system view at " + config.getRocksdbBasePath());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean isPendingCompactionScheduledForFileId(HoodieFileGroupId fgId) {
|
||||
return getPendingCompactionOperationWithInstant(fgId).isPresent();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void resetPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
|
||||
rocksDB.writeBatch(batch -> {
|
||||
operations.forEach(opPair -> {
|
||||
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
|
||||
schemaHelper.getKeyForPendingCompactionLookup(opPair.getValue().getFileGroupId()), opPair);
|
||||
});
|
||||
log.info("Initializing pending compaction operations. Count=" + batch.count());
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void addPendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
|
||||
rocksDB.writeBatch(batch -> {
|
||||
operations.forEach(opInstantPair -> {
|
||||
Preconditions.checkArgument(!isPendingCompactionScheduledForFileId(opInstantPair.getValue().getFileGroupId()),
|
||||
"Duplicate FileGroupId found in pending compaction operations. FgId :"
|
||||
+ opInstantPair.getValue().getFileGroupId());
|
||||
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
|
||||
schemaHelper.getKeyForPendingCompactionLookup(opInstantPair.getValue().getFileGroupId()), opInstantPair);
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
void removePendingCompactionOperations(Stream<Pair<String, CompactionOperation>> operations) {
|
||||
rocksDB.writeBatch(batch -> {
|
||||
operations.forEach(opInstantPair -> {
|
||||
Preconditions.checkArgument(
|
||||
getPendingCompactionOperationWithInstant(opInstantPair.getValue().getFileGroupId()) != null,
|
||||
"Trying to remove a FileGroupId which is not found in pending compaction operations. FgId :"
|
||||
+ opInstantPair.getValue().getFileGroupId());
|
||||
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForPendingCompaction(),
|
||||
schemaHelper.getKeyForPendingCompactionLookup(opInstantPair.getValue().getFileGroupId()));
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void resetViewState() {
|
||||
log.info("Deleting all rocksdb data associated with dataset filesystem view");
|
||||
rocksDB.close();
|
||||
rocksDB = new RocksDBDAO(metaClient.getBasePath(), config);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Option<Pair<String, CompactionOperation>> getPendingCompactionOperationWithInstant(HoodieFileGroupId fgId) {
|
||||
String lookupKey = schemaHelper.getKeyForPendingCompactionLookup(fgId);
|
||||
Pair<String, CompactionOperation> instantOperationPair =
|
||||
rocksDB.get(schemaHelper.getColFamilyForPendingCompaction(), lookupKey);
|
||||
return Option.ofNullable(instantOperationPair);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean isPartitionAvailableInStore(String partitionPath) {
|
||||
String lookupKey = schemaHelper.getKeyForPartitionLookup(partitionPath);
|
||||
Serializable obj = rocksDB.get(schemaHelper.getColFamilyForStoredPartitions(), lookupKey);
|
||||
return obj != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void storePartitionView(String partitionPath, List<HoodieFileGroup> fileGroups) {
|
||||
log.info("Resetting and adding new partition (" + partitionPath + ") to ROCKSDB based file-system view at "
|
||||
+ config.getRocksdbBasePath() + ", Total file-groups=" + fileGroups.size());
|
||||
|
||||
String lookupKey = schemaHelper.getKeyForPartitionLookup(partitionPath);
|
||||
rocksDB.delete(schemaHelper.getColFamilyForStoredPartitions(), lookupKey);
|
||||
|
||||
// First delete partition views
|
||||
rocksDB.prefixDelete(schemaHelper.getColFamilyForView(),
|
||||
schemaHelper.getPrefixForSliceViewByPartition(partitionPath));
|
||||
rocksDB.prefixDelete(schemaHelper.getColFamilyForView(),
|
||||
schemaHelper.getPrefixForDataFileViewByPartition(partitionPath));
|
||||
|
||||
// Now add them
|
||||
fileGroups.stream().forEach(fg -> {
|
||||
rocksDB.writeBatch(batch -> {
|
||||
fg.getAllFileSlicesIncludingInflight().forEach(fs -> {
|
||||
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs);
|
||||
fs.getDataFile().ifPresent(df -> {
|
||||
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs),
|
||||
df);
|
||||
});
|
||||
});
|
||||
});
|
||||
});
|
||||
|
||||
// record that partition is loaded.
|
||||
rocksDB.put(schemaHelper.getColFamilyForStoredPartitions(), lookupKey, Boolean.TRUE);
|
||||
log.info("Finished adding new partition (" + partitionPath + ") to ROCKSDB based file-system view at "
|
||||
+ config.getRocksdbBasePath() + ", Total file-groups=" + fileGroups.size());
|
||||
}
|
||||
|
||||
@Override
|
||||
/**
|
||||
* This is overridden to incrementally apply file-slices to rocks DB
|
||||
*/
|
||||
protected void applyDeltaFileSlicesToPartitionView(String partition, List<HoodieFileGroup> deltaFileGroups,
|
||||
DeltaApplyMode mode) {
|
||||
rocksDB.writeBatch(batch -> {
|
||||
deltaFileGroups.stream().forEach(fg -> {
|
||||
fg.getAllRawFileSlices().map(fs -> {
|
||||
FileSlice oldSlice = getFileSlice(partition, fs.getFileId(), fs.getBaseInstantTime());
|
||||
if (null == oldSlice) {
|
||||
return fs;
|
||||
} else {
|
||||
// First remove the file-slice
|
||||
log.info("Removing old Slice in DB. FS=" + oldSlice);
|
||||
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(),
|
||||
schemaHelper.getKeyForSliceView(fg, oldSlice));
|
||||
rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForView(),
|
||||
schemaHelper.getKeyForDataFileView(fg, oldSlice));
|
||||
|
||||
Map<String, HoodieLogFile> logFiles = oldSlice.getLogFiles()
|
||||
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
Map<String, HoodieLogFile> deltaLogFiles = fs.getLogFiles()
|
||||
.map(lf -> Pair.of(Path.getPathWithoutSchemeAndAuthority(lf.getPath()).toString(), lf))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
switch (mode) {
|
||||
case ADD: {
|
||||
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
|
||||
oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
|
||||
fs.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
|
||||
Map<String, HoodieLogFile> newLogFiles = new HashMap<>(logFiles);
|
||||
deltaLogFiles.entrySet().stream().filter(e -> !logFiles.containsKey(e.getKey()))
|
||||
.forEach(p -> newLogFiles.put(p.getKey(), p.getValue()));
|
||||
newLogFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
|
||||
log.info("Adding back new File Slice after add FS=" + newFileSlice);
|
||||
return newFileSlice;
|
||||
}
|
||||
case REMOVE: {
|
||||
log.info("Removing old File Slice =" + fs);
|
||||
FileSlice newFileSlice = new FileSlice(oldSlice.getFileGroupId(), oldSlice.getBaseInstantTime());
|
||||
fs.getDataFile().orElseGet(() -> {
|
||||
oldSlice.getDataFile().ifPresent(df -> newFileSlice.setDataFile(df));
|
||||
return null;
|
||||
});
|
||||
|
||||
deltaLogFiles.keySet().stream().forEach(p -> logFiles.remove(p));
|
||||
//Add remaining log files back
|
||||
logFiles.values().stream().forEach(lf -> newFileSlice.addLogFile(lf));
|
||||
if (newFileSlice.getDataFile().isPresent() || (newFileSlice.getLogFiles().count() > 0)) {
|
||||
log.info("Adding back new file-slice after remove FS=" + newFileSlice);
|
||||
return newFileSlice;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
default:
|
||||
throw new IllegalStateException("Unknown diff apply mode=" + mode);
|
||||
}
|
||||
}
|
||||
}).filter(Objects::nonNull).forEach(fs -> {
|
||||
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForSliceView(fg, fs), fs);
|
||||
fs.getDataFile().ifPresent(df -> {
|
||||
rocksDB.putInBatch(batch, schemaHelper.getColFamilyForView(), schemaHelper.getKeyForDataFileView(fg, fs),
|
||||
df);
|
||||
});
|
||||
});
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
|
||||
return rocksDB.<Pair<String, CompactionOperation>>prefixSearch(
|
||||
schemaHelper.getColFamilyForPendingCompaction(), "").map(Pair::getValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
Stream<HoodieDataFile> fetchAllDataFiles(String partitionPath) {
|
||||
return rocksDB.<HoodieDataFile>prefixSearch(schemaHelper.getColFamilyForView(),
|
||||
schemaHelper.getPrefixForDataFileViewByPartition(partitionPath))
|
||||
.map(Pair::getValue);
|
||||
}
|
||||
|
||||
@Override
|
||||
Stream<HoodieFileGroup> fetchAllStoredFileGroups(String partitionPath) {
|
||||
return getFileGroups(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
|
||||
schemaHelper.getPrefixForSliceViewByPartition(partitionPath)).map(Pair::getValue));
|
||||
}
|
||||
|
||||
@Override
|
||||
Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
|
||||
return getFileGroups(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
|
||||
schemaHelper.getPrefixForSliceView()).map(Pair::getValue));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Option<FileSlice> fetchLatestFileSlice(String partitionPath, String fileId) {
|
||||
// Retries only file-slices of the file and filters for the latest
|
||||
return Option.ofNullable(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
|
||||
schemaHelper.getPrefixForSliceViewByPartitionFile(partitionPath, fileId))
|
||||
.map(Pair::getValue)
|
||||
.reduce(null, (x, y) ->
|
||||
((x == null) ? y : (y == null) ? null : HoodieTimeline.compareTimestamps(x.getBaseInstantTime(),
|
||||
y.getBaseInstantTime(), HoodieTimeline.GREATER) ? x : y)));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Option<HoodieDataFile> fetchLatestDataFile(String partitionPath, String fileId) {
|
||||
// Retries only file-slices of the file and filters for the latest
|
||||
return Option.ofNullable(rocksDB.<HoodieDataFile>prefixSearch(schemaHelper.getColFamilyForView(),
|
||||
schemaHelper.getPrefixForDataFileViewByPartitionFile(partitionPath, fileId))
|
||||
.map(Pair::getValue)
|
||||
.reduce(null, (x, y) ->
|
||||
((x == null) ? y : (y == null) ? null : HoodieTimeline.compareTimestamps(x.getCommitTime(),
|
||||
y.getCommitTime(), HoodieTimeline.GREATER) ? x : y)));
|
||||
}
|
||||
|
||||
@Override
|
||||
Option<HoodieFileGroup> fetchHoodieFileGroup(String partitionPath, String fileId) {
|
||||
return Option.fromJavaOptional(
|
||||
getFileGroups(rocksDB.<FileSlice>prefixSearch(schemaHelper.getColFamilyForView(),
|
||||
schemaHelper.getPrefixForSliceViewByPartitionFile(partitionPath, fileId))
|
||||
.map(Pair::getValue)).findFirst());
|
||||
}
|
||||
|
||||
private Stream<HoodieFileGroup> getFileGroups(Stream<FileSlice> sliceStream) {
|
||||
return sliceStream.map(s -> Pair.of(Pair.of(s.getPartitionPath(), s.getFileId()), s))
|
||||
.collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream().map(slicePair -> {
|
||||
HoodieFileGroup fg =
|
||||
new HoodieFileGroup(slicePair.getKey().getKey(), slicePair.getKey().getValue(), visibleActiveTimeline);
|
||||
slicePair.getValue().forEach(e -> fg.addFileSlice(e.getValue()));
|
||||
return fg;
|
||||
});
|
||||
}
|
||||
|
||||
private FileSlice getFileSlice(String partitionPath, String fileId, String instantTime) {
|
||||
String key = schemaHelper.getKeyForSliceView(partitionPath, fileId, instantTime);
|
||||
return rocksDB.<FileSlice>get(schemaHelper.getColFamilyForView(), key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
closed = true;
|
||||
rocksDB.close();
|
||||
}
|
||||
|
||||
@Override
|
||||
boolean isClosed() {
|
||||
return closed;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,109 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.util.DefaultSizeEstimator;
|
||||
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Table FileSystemView implementation where view is stored in spillable disk using fixed memory
|
||||
*/
|
||||
public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView {
|
||||
|
||||
private static Logger log = LogManager.getLogger(SpillableMapBasedFileSystemView.class);
|
||||
|
||||
private final long maxMemoryForFileGroupMap;
|
||||
private final long maxMemoryForPendingCompaction;
|
||||
private final String baseStoreDir;
|
||||
|
||||
public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient,
|
||||
HoodieTimeline visibleActiveTimeline, FileSystemViewStorageConfig config) {
|
||||
super(config.isIncrementalTimelineSyncEnabled());
|
||||
this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap();
|
||||
this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction();
|
||||
this.baseStoreDir = config.getBaseStoreDir();
|
||||
init(metaClient, visibleActiveTimeline);
|
||||
}
|
||||
|
||||
public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient,
|
||||
HoodieTimeline visibleActiveTimeline, FileStatus[] fileStatuses, FileSystemViewStorageConfig config) {
|
||||
this(metaClient, visibleActiveTimeline, config);
|
||||
addFilesToView(fileStatuses);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Map<String, List<HoodieFileGroup>> createPartitionToFileGroups() {
|
||||
try {
|
||||
log.info("Creating Partition To File groups map using external spillable Map. Max Mem="
|
||||
+ maxMemoryForFileGroupMap + ", BaseDir=" + baseStoreDir);
|
||||
new File(baseStoreDir).mkdirs();
|
||||
return (Map<String, List<HoodieFileGroup>>)
|
||||
(new ExternalSpillableMap<>(maxMemoryForFileGroupMap, baseStoreDir, new DefaultSizeEstimator(),
|
||||
new DefaultSizeEstimator<>()));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
protected Map<HoodieFileGroupId, Pair<String, CompactionOperation>> createFileIdToPendingCompactionMap(
|
||||
Map<HoodieFileGroupId, Pair<String, CompactionOperation>> fgIdToPendingCompaction) {
|
||||
try {
|
||||
log.info("Creating Pending Compaction map using external spillable Map. Max Mem="
|
||||
+ maxMemoryForPendingCompaction + ", BaseDir=" + baseStoreDir);
|
||||
new File(baseStoreDir).mkdirs();
|
||||
Map<HoodieFileGroupId, Pair<String, CompactionOperation>> pendingMap =
|
||||
new ExternalSpillableMap<>(maxMemoryForPendingCompaction, baseStoreDir, new DefaultSizeEstimator(),
|
||||
new DefaultSizeEstimator<>());
|
||||
pendingMap.putAll(fgIdToPendingCompaction);
|
||||
return pendingMap;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public Stream<HoodieFileGroup> getAllFileGroups() {
|
||||
return ((ExternalSpillableMap)partitionToFileGroupsMap).valueStream()
|
||||
.flatMap(fg -> ((List<HoodieFileGroup>)fg).stream());
|
||||
}
|
||||
|
||||
@Override
|
||||
Stream<Pair<String, CompactionOperation>> fetchPendingCompactionOperations() {
|
||||
return ((ExternalSpillableMap)fgIdToPendingCompaction).valueStream();
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Stream<HoodieFileGroup> fetchAllStoredFileGroups() {
|
||||
return ((ExternalSpillableMap)partitionToFileGroupsMap).valueStream().flatMap(fg -> {
|
||||
return ((List<HoodieFileGroup>)fg).stream();
|
||||
});
|
||||
}
|
||||
}
|
||||
@@ -147,17 +147,7 @@ public class CompactionUtils {
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToPendingCompactionWithInstantMap =
|
||||
new HashMap<>();
|
||||
pendingCompactionPlanWithInstants.stream().flatMap(instantPlanPair -> {
|
||||
HoodieInstant instant = instantPlanPair.getKey();
|
||||
HoodieCompactionPlan compactionPlan = instantPlanPair.getValue();
|
||||
List<HoodieCompactionOperation> ops = compactionPlan.getOperations();
|
||||
if (null != ops) {
|
||||
return ops.stream().map(op -> {
|
||||
return Pair.of(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()),
|
||||
Pair.of(instant.getTimestamp(), op));
|
||||
});
|
||||
} else {
|
||||
return Stream.empty();
|
||||
}
|
||||
return getPendingCompactionOperations(instantPlanPair.getKey(), instantPlanPair.getValue());
|
||||
}).forEach(pair -> {
|
||||
// Defensive check to ensure a single-fileId does not have more than one pending compaction
|
||||
if (fgIdToPendingCompactionWithInstantMap.containsKey(pair.getKey())) {
|
||||
@@ -169,4 +159,17 @@ public class CompactionUtils {
|
||||
});
|
||||
return fgIdToPendingCompactionWithInstantMap;
|
||||
}
|
||||
|
||||
public static Stream<Pair<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>>> getPendingCompactionOperations(
|
||||
HoodieInstant instant, HoodieCompactionPlan compactionPlan) {
|
||||
List<HoodieCompactionOperation> ops = compactionPlan.getOperations();
|
||||
if (null != ops) {
|
||||
return ops.stream().map(op -> {
|
||||
return Pair.of(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()),
|
||||
Pair.of(instant.getTimestamp(), op));
|
||||
});
|
||||
} else {
|
||||
return Stream.empty();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,38 @@
|
||||
/*
|
||||
* Copyright (c) 2018 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 java.io.Serializable;
|
||||
|
||||
public interface Functions {
|
||||
|
||||
public interface Function0<R> extends Serializable {
|
||||
R apply();
|
||||
}
|
||||
|
||||
public interface Function1<T1, R> extends Serializable {
|
||||
R apply(T1 val1);
|
||||
}
|
||||
|
||||
public interface Function2<T1, T2, R> extends Serializable {
|
||||
R apply(T1 val1, T2 val2);
|
||||
}
|
||||
|
||||
public interface Function3<T1, T2, T3, R> extends Serializable {
|
||||
R apply(T1 val1, T2 val2, T3 val3);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,42 @@
|
||||
/*
|
||||
* Copyright (c) 2018 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.uber.hoodie.exception.HoodieException;
|
||||
import java.io.IOException;
|
||||
import java.net.ServerSocket;
|
||||
|
||||
public class NetworkUtils {
|
||||
|
||||
public static synchronized String getHostname() {
|
||||
ServerSocket s = null;
|
||||
try {
|
||||
s = new ServerSocket(0);
|
||||
return s.getInetAddress().getHostAddress();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Unable to find server port", e);
|
||||
} finally {
|
||||
if (null != s) {
|
||||
try {
|
||||
s.close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Unable to close server port", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,312 @@
|
||||
/*
|
||||
* 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 java.io.Serializable;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Objects;
|
||||
import java.util.Optional;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Function;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
/**
|
||||
* Copied from java.util.Optional and made Serializable along with methods to convert to/from standard Option
|
||||
*/
|
||||
public final class Option<T> implements Serializable {
|
||||
|
||||
private static final long serialVersionUID = 0L;
|
||||
|
||||
/**
|
||||
* Common instance for {@code empty()}.
|
||||
*/
|
||||
private static final Option<?> EMPTY = new Option<>();
|
||||
|
||||
/**
|
||||
* If non-null, the value; if null, indicates no value is present
|
||||
*/
|
||||
private final T value;
|
||||
|
||||
/**
|
||||
* Constructs an empty instance.
|
||||
*
|
||||
* @implNote Generally only one empty instance, {@link Option#EMPTY}, should exist per VM.
|
||||
*/
|
||||
private Option() {
|
||||
this.value = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an empty {@code Option} instance. No value is present for this Option.
|
||||
*
|
||||
* @param <T> Type of the non-existent value
|
||||
* @return an empty {@code Option}
|
||||
* @apiNote Though it may be tempting to do so, avoid testing if an object is empty by comparing with {@code ==}
|
||||
* against instances returned by {@code Option.empty()}. There is no guarantee that it is a singleton. Instead, use
|
||||
* {@link #isPresent()}.
|
||||
*/
|
||||
public static <T> Option<T> empty() {
|
||||
@SuppressWarnings("unchecked")
|
||||
Option<T> t = (Option<T>) EMPTY;
|
||||
return t;
|
||||
}
|
||||
|
||||
/**
|
||||
* Constructs an instance with the value present.
|
||||
*
|
||||
* @param value the non-null value to be present
|
||||
* @throws NullPointerException if value is null
|
||||
*/
|
||||
private Option(T value) {
|
||||
this.value = Objects.requireNonNull(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an {@code Option} with the specified present non-null value.
|
||||
*
|
||||
* @param <T> the class of the value
|
||||
* @param value the value to be present, which must be non-null
|
||||
* @return an {@code Option} with the value present
|
||||
* @throws NullPointerException if value is null
|
||||
*/
|
||||
public static <T> Option<T> of(T value) {
|
||||
return new Option<>(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an {@code Option} describing the specified value, if non-null, otherwise returns an empty {@code Option}.
|
||||
*
|
||||
* @param <T> the class of the value
|
||||
* @param value the possibly-null value to describe
|
||||
* @return an {@code Option} with a present value if the specified value is non-null, otherwise an empty {@code
|
||||
* Option}
|
||||
*/
|
||||
public static <T> Option<T> ofNullable(T value) {
|
||||
return value == null ? empty() : of(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* If a value is present in this {@code Option}, returns the value, otherwise throws {@code NoSuchElementException}.
|
||||
*
|
||||
* @return the non-null value held by this {@code Option}
|
||||
* @throws NoSuchElementException if there is no value present
|
||||
* @see Option#isPresent()
|
||||
*/
|
||||
public T get() {
|
||||
if (value == null) {
|
||||
throw new NoSuchElementException("No value present");
|
||||
}
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return {@code true} if there is a value present, otherwise {@code false}.
|
||||
*
|
||||
* @return {@code true} if there is a value present, otherwise {@code false}
|
||||
*/
|
||||
public boolean isPresent() {
|
||||
return value != null;
|
||||
}
|
||||
|
||||
/**
|
||||
* If a value is present, invoke the specified consumer with the value, otherwise do nothing.
|
||||
*
|
||||
* @param consumer block to be executed if a value is present
|
||||
* @throws NullPointerException if value is present and {@code consumer} is null
|
||||
*/
|
||||
public void ifPresent(Consumer<? super T> consumer) {
|
||||
if (value != null) {
|
||||
consumer.accept(value);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If a value is present, and the value matches the given predicate, return an {@code Option} describing the value,
|
||||
* otherwise return an empty {@code Option}.
|
||||
*
|
||||
* @param predicate a predicate to apply to the value, if present
|
||||
* @return an {@code Option} describing the value of this {@code Option} if a value is present and the value matches
|
||||
* the given predicate, otherwise an empty {@code Option}
|
||||
* @throws NullPointerException if the predicate is null
|
||||
*/
|
||||
public Option<T> filter(Predicate<? super T> predicate) {
|
||||
Objects.requireNonNull(predicate);
|
||||
if (!isPresent()) {
|
||||
return this;
|
||||
} else {
|
||||
return predicate.test(value) ? this : empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If a value is present, apply the provided mapping function to it, and if the result is non-null, return an {@code
|
||||
* Option} describing the result. Otherwise return an empty {@code Option}.
|
||||
*
|
||||
* @param <U> The type of the result of the mapping function
|
||||
* @param mapper a mapping function to apply to the value, if present
|
||||
* @return an {@code Option} describing the result of applying a mapping function to the value of this {@code Option},
|
||||
* if a value is present, otherwise an empty {@code Option}
|
||||
* @throws NullPointerException if the mapping function is null
|
||||
* @apiNote This method supports post-processing on optional values, without the need to explicitly check for a return
|
||||
* status. For example, the following code traverses a stream of file names, selects one that has not yet been
|
||||
* processed, and then opens that file, returning an {@code Option<FileInputStream>}:
|
||||
*
|
||||
* <pre>{@code
|
||||
* Option<FileInputStream> fis =
|
||||
* names.stream().filter(name -> !isProcessedYet(name))
|
||||
* .findFirst()
|
||||
* .map(name -> new FileInputStream(name));
|
||||
* }</pre>
|
||||
*
|
||||
* Here, {@code findFirst} returns an {@code Option<String>}, and then {@code map} returns an {@code
|
||||
* Option<FileInputStream>} for the desired file if one exists.
|
||||
*/
|
||||
public <U> Option<U> map(Function<? super T, ? extends U> mapper) {
|
||||
Objects.requireNonNull(mapper);
|
||||
if (!isPresent()) {
|
||||
return empty();
|
||||
} else {
|
||||
return Option.ofNullable(mapper.apply(value));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* If a value is present, apply the provided {@code Option}-bearing mapping function to it, return that result,
|
||||
* otherwise return an empty {@code Option}. This method is similar to {@link #map(Function)}, but the provided
|
||||
* mapper is one whose result is already an {@code Option}, and if invoked, {@code flatMap} does not wrap it with an
|
||||
* additional {@code Option}.
|
||||
*
|
||||
* @param <U> The type parameter to the {@code Option} returned by
|
||||
* @param mapper a mapping function to apply to the value, if present the mapping function
|
||||
* @return the result of applying an {@code Option}-bearing mapping function to the value of this {@code Option}, if a
|
||||
* value is present, otherwise an empty {@code Option}
|
||||
* @throws NullPointerException if the mapping function is null or returns a null result
|
||||
*/
|
||||
public <U> Option<U> flatMap(Function<? super T, Option<U>> mapper) {
|
||||
Objects.requireNonNull(mapper);
|
||||
if (!isPresent()) {
|
||||
return empty();
|
||||
} else {
|
||||
return Objects.requireNonNull(mapper.apply(value));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the value if present, otherwise return {@code other}.
|
||||
*
|
||||
* @param other the value to be returned if there is no value present, may be null
|
||||
* @return the value, if present, otherwise {@code other}
|
||||
*/
|
||||
public T orElse(T other) {
|
||||
return value != null ? value : other;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the value if present, otherwise invoke {@code other} and return the result of that invocation.
|
||||
*
|
||||
* @param other a {@code Supplier} whose result is returned if no value is present
|
||||
* @return the value if present otherwise the result of {@code other.get()}
|
||||
* @throws NullPointerException if value is not present and {@code other} is null
|
||||
*/
|
||||
public T orElseGet(Supplier<? extends T> other) {
|
||||
return value != null ? value : other.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the contained value, if present, otherwise throw an exception to be created by the provided supplier.
|
||||
*
|
||||
* @param <X> Type of the exception to be thrown
|
||||
* @param exceptionSupplier The supplier which will return the exception to be thrown
|
||||
* @return the present value
|
||||
* @throws X if there is no value present
|
||||
* @throws NullPointerException if no value is present and {@code exceptionSupplier} is null
|
||||
* @apiNote A method reference to the exception constructor with an empty argument list can be used as the supplier.
|
||||
* For example, {@code IllegalStateException::new}
|
||||
*/
|
||||
public <X extends Throwable> T orElseThrow(Supplier<? extends X> exceptionSupplier) throws X {
|
||||
if (value != null) {
|
||||
return value;
|
||||
} else {
|
||||
throw exceptionSupplier.get();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicates whether some other object is "equal to" this Option. The other object is considered equal if:
|
||||
* <ul>
|
||||
* <li>it is also an {@code Option} and;
|
||||
* <li>both instances have no value present or;
|
||||
* <li>the present values are "equal to" each other via {@code equals()}.
|
||||
* </ul>
|
||||
*
|
||||
* @param obj an object to be tested for equality
|
||||
* @return {code true} if the other object is "equal to" this object otherwise {@code false}
|
||||
*/
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
if (this == obj) {
|
||||
return true;
|
||||
}
|
||||
|
||||
if (!(obj instanceof Option)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
Option<?> other = (Option<?>) obj;
|
||||
return Objects.equals(value, other.value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the hash code value of the present value, if any, or 0 (zero) if no value is present.
|
||||
*
|
||||
* @return hash code value of the present value or 0 if no value is present
|
||||
*/
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hashCode(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a non-empty string representation of this Option suitable for debugging. The exact presentation format is
|
||||
* unspecified and may vary between implementations and versions.
|
||||
*
|
||||
* @return the string representation of this instance
|
||||
* @implSpec If a value is present the result must include its string representation in the result. Empty and present
|
||||
* Optionals must be unambiguously differentiable.
|
||||
*/
|
||||
@Override
|
||||
public String toString() {
|
||||
return value != null
|
||||
? String.format("Option[%s]", value)
|
||||
: "Option.empty";
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert to java Optional
|
||||
*/
|
||||
public Optional<T> toJavaOptional() {
|
||||
return Optional.ofNullable(value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert from java.util.Optional
|
||||
*/
|
||||
public static <T> Option<T> fromJavaOptional(Optional<T> v) {
|
||||
return Option.ofNullable(v.orElse(null));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,389 @@
|
||||
/*
|
||||
* 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.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.rocksdb.ColumnFamilyDescriptor;
|
||||
import org.rocksdb.ColumnFamilyHandle;
|
||||
import org.rocksdb.ColumnFamilyOptions;
|
||||
import org.rocksdb.DBOptions;
|
||||
import org.rocksdb.InfoLogLevel;
|
||||
import org.rocksdb.Options;
|
||||
import org.rocksdb.RocksDB;
|
||||
import org.rocksdb.RocksDBException;
|
||||
import org.rocksdb.RocksIterator;
|
||||
import org.rocksdb.Statistics;
|
||||
import org.rocksdb.WriteBatch;
|
||||
import org.rocksdb.WriteOptions;
|
||||
|
||||
/**
|
||||
* Data access objects for storing and retrieving objects in Rocks DB.
|
||||
*/
|
||||
public class RocksDBDAO {
|
||||
|
||||
protected static final transient Logger log = LogManager.getLogger(RocksDBDAO.class);
|
||||
|
||||
private final FileSystemViewStorageConfig config;
|
||||
private transient ConcurrentHashMap<String, ColumnFamilyHandle> managedHandlesMap;
|
||||
private transient ConcurrentHashMap<String, ColumnFamilyDescriptor> managedDescriptorMap;
|
||||
private transient RocksDB rocksDB;
|
||||
private boolean closed = false;
|
||||
private final String basePath;
|
||||
private final String rocksDBBasePath;
|
||||
|
||||
public RocksDBDAO(String basePath, FileSystemViewStorageConfig config) {
|
||||
this.basePath = basePath;
|
||||
this.config = config;
|
||||
this.rocksDBBasePath = String.format("%s/%s/%s", config.getRocksdbBasePath(),
|
||||
this.basePath.replace("/", "_"), UUID.randomUUID().toString());
|
||||
init();
|
||||
}
|
||||
|
||||
/**
|
||||
* Create RocksDB if not initialized.
|
||||
*/
|
||||
private RocksDB getRocksDB() {
|
||||
if (null == rocksDB) {
|
||||
init();
|
||||
}
|
||||
return rocksDB;
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialized Rocks DB instance
|
||||
*/
|
||||
private void init() throws HoodieException {
|
||||
try {
|
||||
log.warn("DELETING RocksDB persisted at " + rocksDBBasePath);
|
||||
FileUtils.deleteDirectory(new File(rocksDBBasePath));
|
||||
|
||||
managedHandlesMap = new ConcurrentHashMap<>();
|
||||
managedDescriptorMap = new ConcurrentHashMap<>();
|
||||
|
||||
// If already present, loads the existing column-family handles
|
||||
final DBOptions dbOptions = new DBOptions().setCreateIfMissing(true).setCreateMissingColumnFamilies(true)
|
||||
.setWalDir(rocksDBBasePath).setStatsDumpPeriodSec(300).setStatistics(new Statistics());
|
||||
dbOptions.setLogger(new org.rocksdb.Logger(dbOptions) {
|
||||
@Override
|
||||
protected void log(InfoLogLevel infoLogLevel, String logMsg) {
|
||||
log.info("From Rocks DB : " + logMsg);
|
||||
}
|
||||
});
|
||||
final List<ColumnFamilyDescriptor> managedColumnFamilies = loadManagedColumnFamilies(dbOptions);
|
||||
final List<ColumnFamilyHandle> managedHandles = new ArrayList<>();
|
||||
FileUtils.forceMkdir(new File(rocksDBBasePath));
|
||||
rocksDB = RocksDB.open(dbOptions, rocksDBBasePath, managedColumnFamilies, managedHandles);
|
||||
|
||||
Preconditions.checkArgument(managedHandles.size() == managedColumnFamilies.size(),
|
||||
"Unexpected number of handles are returned");
|
||||
for (int index = 0; index < managedHandles.size(); index++) {
|
||||
ColumnFamilyHandle handle = managedHandles.get(index);
|
||||
ColumnFamilyDescriptor descriptor = managedColumnFamilies.get(index);
|
||||
String familyNameFromHandle = new String(handle.getName());
|
||||
String familyNameFromDescriptor = new String(descriptor.getName());
|
||||
|
||||
Preconditions.checkArgument(familyNameFromDescriptor.equals(familyNameFromHandle),
|
||||
"Family Handles not in order with descriptors");
|
||||
managedHandlesMap.put(familyNameFromHandle, handle);
|
||||
managedDescriptorMap.put(familyNameFromDescriptor, descriptor);
|
||||
}
|
||||
} catch (RocksDBException | IOException re) {
|
||||
log.error("Got exception opening rocks db instance ", re);
|
||||
throw new HoodieException(re);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to load managed column family descriptors
|
||||
*/
|
||||
private List<ColumnFamilyDescriptor> loadManagedColumnFamilies(DBOptions dbOptions) throws RocksDBException {
|
||||
final List<ColumnFamilyDescriptor> managedColumnFamilies = new ArrayList<>();
|
||||
final Options options = new Options(dbOptions, new ColumnFamilyOptions());
|
||||
List<byte[]> existing = RocksDB.listColumnFamilies(options, rocksDBBasePath);
|
||||
|
||||
if (existing.isEmpty()) {
|
||||
log.info("No column family found. Loading default");
|
||||
managedColumnFamilies.add(getColumnFamilyDescriptor(RocksDB.DEFAULT_COLUMN_FAMILY));
|
||||
} else {
|
||||
log.info("Loading column families :" + existing.stream().map(String::new).collect(Collectors.toList()));
|
||||
managedColumnFamilies.addAll(existing.stream()
|
||||
.map(RocksDBDAO::getColumnFamilyDescriptor).collect(Collectors.toList()));
|
||||
}
|
||||
return managedColumnFamilies;
|
||||
}
|
||||
|
||||
private static ColumnFamilyDescriptor getColumnFamilyDescriptor(byte[] columnFamilyName) {
|
||||
return new ColumnFamilyDescriptor(columnFamilyName, new ColumnFamilyOptions());
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform a batch write operation
|
||||
*/
|
||||
public void writeBatch(BatchHandler handler) {
|
||||
WriteBatch batch = new WriteBatch();
|
||||
try {
|
||||
handler.apply(batch);
|
||||
getRocksDB().write(new WriteOptions(), batch);
|
||||
} catch (RocksDBException re) {
|
||||
throw new HoodieException(re);
|
||||
} finally {
|
||||
batch.close();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to add put operation in batch
|
||||
*
|
||||
* @param batch Batch Handle
|
||||
* @param columnFamilyName Column Family
|
||||
* @param key Key
|
||||
* @param value Payload
|
||||
* @param <T> Type of payload
|
||||
*/
|
||||
public <T extends Serializable> void putInBatch(WriteBatch batch, String columnFamilyName, String key, T value) {
|
||||
try {
|
||||
byte[] payload = SerializationUtils.serialize(value);
|
||||
batch.put(managedHandlesMap.get(columnFamilyName), key.getBytes(), payload);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform single PUT on a column-family
|
||||
*
|
||||
* @param columnFamilyName Column family name
|
||||
* @param key Key
|
||||
* @param value Payload
|
||||
* @param <T> Type of Payload
|
||||
*/
|
||||
public <T extends Serializable> void put(String columnFamilyName, String key, T value) {
|
||||
try {
|
||||
byte[] payload = SerializationUtils.serialize(value);
|
||||
getRocksDB().put(managedHandlesMap.get(columnFamilyName), key.getBytes(), payload);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to add delete operation in batch
|
||||
*
|
||||
* @param batch Batch Handle
|
||||
* @param columnFamilyName Column Family
|
||||
* @param key Key
|
||||
*/
|
||||
public void deleteInBatch(WriteBatch batch, String columnFamilyName, String key) {
|
||||
try {
|
||||
batch.delete(managedHandlesMap.get(columnFamilyName), key.getBytes());
|
||||
} catch (RocksDBException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform a single Delete operation
|
||||
*
|
||||
* @param columnFamilyName Column Family name
|
||||
* @param key Key to be deleted
|
||||
*/
|
||||
public void delete(String columnFamilyName, String key) {
|
||||
try {
|
||||
getRocksDB().delete(managedHandlesMap.get(columnFamilyName), key.getBytes());
|
||||
} catch (RocksDBException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve a value for a given key in a column family
|
||||
*
|
||||
* @param columnFamilyName Column Family Name
|
||||
* @param key Key to be retrieved
|
||||
* @param <T> Type of object stored.
|
||||
*/
|
||||
public <T extends Serializable> T get(String columnFamilyName, String key) {
|
||||
Preconditions.checkArgument(!closed);
|
||||
try {
|
||||
byte[] val = getRocksDB().get(managedHandlesMap.get(columnFamilyName), key.getBytes());
|
||||
return val == null ? null : SerializationUtils.deserialize(val);
|
||||
} catch (RocksDBException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform a prefix search and return stream of key-value pairs retrieved
|
||||
*
|
||||
* @param columnFamilyName Column Family Name
|
||||
* @param prefix Prefix Key
|
||||
* @param <T> Type of value stored
|
||||
*/
|
||||
public <T extends Serializable> Stream<Pair<String, T>> prefixSearch(String columnFamilyName, String prefix) {
|
||||
Preconditions.checkArgument(!closed);
|
||||
final HoodieTimer timer = new HoodieTimer();
|
||||
timer.startTimer();
|
||||
long timeTakenMicro = 0;
|
||||
List<Pair<String, T>> results = new LinkedList<>();
|
||||
try (final RocksIterator it = getRocksDB().newIterator(managedHandlesMap.get(columnFamilyName))) {
|
||||
it.seek(prefix.getBytes());
|
||||
while (it.isValid() && new String(it.key()).startsWith(prefix)) {
|
||||
long beginTs = System.nanoTime();
|
||||
T val = SerializationUtils.deserialize(it.value());
|
||||
timeTakenMicro += ((System.nanoTime() - beginTs) / 1000);
|
||||
results.add(Pair.of(new String(it.key()), val));
|
||||
it.next();
|
||||
}
|
||||
}
|
||||
|
||||
log.info("Prefix Search for (query=" + prefix + ") on " + columnFamilyName
|
||||
+ ". Total Time Taken (msec)=" + timer.endTimer()
|
||||
+ ". Serialization Time taken(micro)=" + timeTakenMicro + ", num entries=" + results.size());
|
||||
return results.stream();
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform a prefix delete and return stream of key-value pairs retrieved
|
||||
*
|
||||
* @param columnFamilyName Column Family Name
|
||||
* @param prefix Prefix Key
|
||||
* @param <T> Type of value stored
|
||||
*/
|
||||
public <T extends Serializable> void prefixDelete(String columnFamilyName, String prefix) {
|
||||
Preconditions.checkArgument(!closed);
|
||||
log.info("Prefix DELETE (query=" + prefix + ") on " + columnFamilyName);
|
||||
final RocksIterator it = getRocksDB().newIterator(managedHandlesMap.get(columnFamilyName));
|
||||
it.seek(prefix.getBytes());
|
||||
//Find first and last keys to be deleted
|
||||
String firstEntry = null;
|
||||
String lastEntry = null;
|
||||
while (it.isValid() && new String(it.key()).startsWith(prefix)) {
|
||||
String result = new String(it.key());
|
||||
it.next();
|
||||
if (firstEntry == null) {
|
||||
firstEntry = result;
|
||||
}
|
||||
lastEntry = result;
|
||||
}
|
||||
it.close();
|
||||
|
||||
if (null != firstEntry) {
|
||||
try {
|
||||
// This will not delete the last entry
|
||||
getRocksDB().deleteRange(managedHandlesMap.get(columnFamilyName), firstEntry.getBytes(),
|
||||
lastEntry.getBytes());
|
||||
//Delete the last entry
|
||||
getRocksDB().delete(lastEntry.getBytes());
|
||||
} catch (RocksDBException e) {
|
||||
log.error("Got exception performing range delete");
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a new column family to store
|
||||
*
|
||||
* @param columnFamilyName Column family name
|
||||
*/
|
||||
public void addColumnFamily(String columnFamilyName) {
|
||||
Preconditions.checkArgument(!closed);
|
||||
|
||||
managedDescriptorMap.computeIfAbsent(columnFamilyName, colFamilyName -> {
|
||||
try {
|
||||
ColumnFamilyDescriptor descriptor = getColumnFamilyDescriptor(colFamilyName.getBytes());
|
||||
ColumnFamilyHandle handle = getRocksDB().createColumnFamily(descriptor);
|
||||
managedHandlesMap.put(colFamilyName, handle);
|
||||
return descriptor;
|
||||
} catch (RocksDBException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Note : Does not delete from underlying DB. Just closes the handle
|
||||
*
|
||||
* @param columnFamilyName Column Family Name
|
||||
*/
|
||||
public void dropColumnFamily(String columnFamilyName) {
|
||||
Preconditions.checkArgument(!closed);
|
||||
|
||||
managedDescriptorMap.computeIfPresent(columnFamilyName, (colFamilyName, descriptor) -> {
|
||||
ColumnFamilyHandle handle = managedHandlesMap.get(colFamilyName);
|
||||
try {
|
||||
getRocksDB().dropColumnFamily(handle);
|
||||
handle.close();
|
||||
} catch (RocksDBException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
managedHandlesMap.remove(columnFamilyName);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Close the DAO object
|
||||
*/
|
||||
public synchronized void close() {
|
||||
if (!closed) {
|
||||
closed = true;
|
||||
managedHandlesMap.values().forEach(columnFamilyHandle -> {
|
||||
columnFamilyHandle.close();
|
||||
});
|
||||
managedHandlesMap.clear();
|
||||
managedDescriptorMap.clear();
|
||||
getRocksDB().close();
|
||||
try {
|
||||
FileUtils.deleteDirectory(new File(rocksDBBasePath));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
String getRocksDBBasePath() {
|
||||
return rocksDBBasePath;
|
||||
}
|
||||
|
||||
/**
|
||||
* Functional interface for stacking operation to Write batch
|
||||
*/
|
||||
public interface BatchHandler {
|
||||
|
||||
void apply(WriteBatch batch);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,124 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Helper class to generate Key and column names for rocksdb based view
|
||||
*
|
||||
* For RocksDB, 3 colFamilies are used for storing file-system view for each dataset.
|
||||
* (a) View
|
||||
* (b) Partitions Cached
|
||||
* (c) Pending Compactions
|
||||
*
|
||||
*
|
||||
* View : Key : Store both slice and Data file stored.
|
||||
* Slice :
|
||||
* Key = "type=slice,part=<PartitionPath>,id=<FileId>,instant=<Timestamp>"
|
||||
* Value = Serialized FileSlice
|
||||
* Data File :
|
||||
* Key = "type=df,part=<PartitionPath>,id=<FileId>,instant=<Timestamp>"
|
||||
* Value = Serialized DataFile
|
||||
*
|
||||
* Partitions :
|
||||
* Key = "part=<PartitionPath>"
|
||||
* Value = Boolean
|
||||
*
|
||||
* Pending Compactions
|
||||
* Key = "part=<PartitionPath>,id=<FileId>"
|
||||
* Value = Pair<CompactionTime, CompactionOperation>
|
||||
*/
|
||||
public class RocksDBSchemaHelper {
|
||||
|
||||
private final String colFamilyForView;
|
||||
private final String colFamilyForPendingCompaction;
|
||||
private final String colFamilyForStoredPartitions;
|
||||
|
||||
public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) {
|
||||
this.colFamilyForPendingCompaction = "hudi_pending_compaction_" + metaClient.getBasePath().replace("/", "_");
|
||||
this.colFamilyForStoredPartitions = "hudi_partitions_" + metaClient.getBasePath().replace("/", "_");
|
||||
this.colFamilyForView = "hudi_view_" + metaClient.getBasePath().replace("/", "_");
|
||||
}
|
||||
|
||||
public List<String> getAllColumnFamilies() {
|
||||
return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForStoredPartitions());
|
||||
}
|
||||
|
||||
public String getKeyForPartitionLookup(String partition) {
|
||||
return String.format("part=%s", partition);
|
||||
}
|
||||
|
||||
public String getKeyForPendingCompactionLookup(HoodieFileGroupId fgId) {
|
||||
return getPartitionFileIdBasedLookup(fgId);
|
||||
}
|
||||
|
||||
public String getKeyForSliceView(HoodieFileGroup fileGroup, FileSlice slice) {
|
||||
return getKeyForSliceView(fileGroup.getPartitionPath(), fileGroup.getFileGroupId().getFileId(),
|
||||
slice.getBaseInstantTime());
|
||||
}
|
||||
|
||||
public String getKeyForSliceView(String partitionPath, String fileId, String instantTime) {
|
||||
return String.format("type=slice,part=%s,id=%s,instant=%s",partitionPath, fileId, instantTime);
|
||||
}
|
||||
|
||||
public String getPrefixForSliceViewByPartitionFile(String partitionPath, String fileId) {
|
||||
return String.format("type=slice,part=%s,id=%s,instant=",partitionPath, fileId);
|
||||
}
|
||||
|
||||
public String getPrefixForDataFileViewByPartitionFile(String partitionPath, String fileId) {
|
||||
return String.format("type=df,part=%s,id=%s,instant=",partitionPath, fileId);
|
||||
}
|
||||
|
||||
public String getKeyForDataFileView(HoodieFileGroup fileGroup, FileSlice slice) {
|
||||
return String.format("type=df,part=%s,id=%s,instant=%s", fileGroup.getPartitionPath(),
|
||||
fileGroup.getFileGroupId().getFileId(), slice.getBaseInstantTime());
|
||||
}
|
||||
|
||||
public String getPrefixForSliceViewByPartition(String partitionPath) {
|
||||
return String.format("type=slice,part=%s,id=", partitionPath);
|
||||
}
|
||||
|
||||
public String getPrefixForSliceView() {
|
||||
return "type=slice,part=";
|
||||
}
|
||||
|
||||
public String getPrefixForDataFileViewByPartition(String partitionPath) {
|
||||
return String.format("type=df,part=%s,id=", partitionPath);
|
||||
}
|
||||
|
||||
private String getPartitionFileIdBasedLookup(HoodieFileGroupId fgId) {
|
||||
return String.format("part=%s,id=%s", fgId.getPartitionPath(), fgId.getFileId());
|
||||
}
|
||||
|
||||
public String getColFamilyForView() {
|
||||
return colFamilyForView;
|
||||
}
|
||||
|
||||
public String getColFamilyForPendingCompaction() {
|
||||
return colFamilyForPendingCompaction;
|
||||
}
|
||||
|
||||
public String getColFamilyForStoredPartitions() {
|
||||
return colFamilyForStoredPartitions;
|
||||
}
|
||||
}
|
||||
@@ -41,6 +41,10 @@ public class StringUtils {
|
||||
return join(elements, "");
|
||||
}
|
||||
|
||||
public static <T> String joinUsingDelim(String delim, final String... elements) {
|
||||
return join(elements, delim);
|
||||
}
|
||||
|
||||
public static String join(final String[] array, final String separator) {
|
||||
if (array == null) {
|
||||
return null;
|
||||
|
||||
@@ -0,0 +1,134 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
|
||||
public class TimelineDiffHelper {
|
||||
|
||||
protected static Logger log = LogManager.getLogger(TimelineDiffHelper.class);
|
||||
|
||||
public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline oldTimeline,
|
||||
HoodieTimeline newTimeline) {
|
||||
|
||||
HoodieTimeline oldT = oldTimeline.filterCompletedAndCompactionInstants();
|
||||
HoodieTimeline newT = newTimeline.filterCompletedAndCompactionInstants();
|
||||
|
||||
Optional<HoodieInstant> lastSeenInstant = oldT.lastInstant();
|
||||
Optional<HoodieInstant> firstInstantInNewTimeline = newT.firstInstant();
|
||||
|
||||
if (lastSeenInstant.isPresent() && firstInstantInNewTimeline.isPresent()) {
|
||||
if (HoodieTimeline.compareTimestamps(lastSeenInstant.get().getTimestamp(),
|
||||
firstInstantInNewTimeline.get().getTimestamp(), HoodieTimeline.LESSER)) {
|
||||
// The last seen instant is no longer in the timeline. Do not incrementally Sync.
|
||||
return TimelineDiffResult.UNSAFE_SYNC_RESULT;
|
||||
}
|
||||
Set<HoodieInstant> oldTimelineInstants = oldT.getInstants().collect(Collectors.toSet());
|
||||
|
||||
List<HoodieInstant> newInstants = new ArrayList<>();
|
||||
|
||||
// Check If any pending compaction is lost. If so, do not allow incremental timeline sync
|
||||
List<Pair<HoodieInstant, HoodieInstant>> compactionInstants = getPendingCompactionTransitions(oldT, newT);
|
||||
List<HoodieInstant> lostPendingCompactions =
|
||||
compactionInstants.stream().filter(instantPair -> instantPair.getValue() == null).map(Pair::getKey)
|
||||
.collect(Collectors.toList());
|
||||
if (!lostPendingCompactions.isEmpty()) {
|
||||
// If a compaction is unscheduled, fall back to complete refresh of fs view since some log files could have been
|
||||
// moved. Its unsafe to incrementally sync in that case.
|
||||
log.warn("Some pending compactions are no longer in new timeline (unscheduled ?)."
|
||||
+ "They are :" + lostPendingCompactions);
|
||||
return TimelineDiffResult.UNSAFE_SYNC_RESULT;
|
||||
}
|
||||
List<HoodieInstant> finishedCompactionInstants = compactionInstants.stream().filter(instantPair ->
|
||||
instantPair.getValue().getAction().equals(HoodieTimeline.COMMIT_ACTION)
|
||||
&& instantPair.getValue().isCompleted()).map(Pair::getKey).collect(Collectors.toList());
|
||||
|
||||
newT.getInstants().filter(instant -> !oldTimelineInstants.contains(instant)).forEach(newInstants::add);
|
||||
return new TimelineDiffResult(newInstants, finishedCompactionInstants, true);
|
||||
} else {
|
||||
// One or more timelines is empty
|
||||
log.warn("One or more timelines is empty");
|
||||
return TimelineDiffResult.UNSAFE_SYNC_RESULT;
|
||||
}
|
||||
}
|
||||
|
||||
private static List<Pair<HoodieInstant, HoodieInstant>> getPendingCompactionTransitions(HoodieTimeline oldTimeline,
|
||||
HoodieTimeline newTimeline) {
|
||||
Set<HoodieInstant> newTimelineInstants = newTimeline.getInstants().collect(Collectors.toSet());
|
||||
|
||||
return oldTimeline.filterPendingCompactionTimeline().getInstants().map(instant -> {
|
||||
if (newTimelineInstants.contains(instant)) {
|
||||
return Pair.of(instant, instant);
|
||||
} else {
|
||||
HoodieInstant compacted =
|
||||
new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, instant.getTimestamp());
|
||||
if (newTimelineInstants.contains(compacted)) {
|
||||
return Pair.of(instant, compacted);
|
||||
}
|
||||
return Pair.<HoodieInstant, HoodieInstant>of(instant, null);
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static class TimelineDiffResult {
|
||||
|
||||
private final List<HoodieInstant> newlySeenInstants;
|
||||
private final List<HoodieInstant> finishedCompactionInstants;
|
||||
private final boolean canSyncIncrementally;
|
||||
|
||||
public static final TimelineDiffResult UNSAFE_SYNC_RESULT = new TimelineDiffResult(null, null, false);
|
||||
|
||||
public TimelineDiffResult(List<HoodieInstant> newlySeenInstants, List<HoodieInstant> finishedCompactionInstants,
|
||||
boolean canSyncIncrementally) {
|
||||
this.newlySeenInstants = newlySeenInstants;
|
||||
this.finishedCompactionInstants = finishedCompactionInstants;
|
||||
this.canSyncIncrementally = canSyncIncrementally;
|
||||
}
|
||||
|
||||
public List<HoodieInstant> getNewlySeenInstants() {
|
||||
return newlySeenInstants;
|
||||
}
|
||||
|
||||
public List<HoodieInstant> getFinishedCompactionInstants() {
|
||||
return finishedCompactionInstants;
|
||||
}
|
||||
|
||||
public boolean canSyncIncrementally() {
|
||||
return canSyncIncrementally;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "TimelineDiffResult{"
|
||||
+ "newlySeenInstants=" + newlySeenInstants
|
||||
+ ", finishedCompactionInstants=" + finishedCompactionInstants
|
||||
+ ", canSyncIncrementally=" + canSyncIncrementally
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -31,13 +31,16 @@ import java.net.InetAddress;
|
||||
import java.util.AbstractMap;
|
||||
import java.util.Collection;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.Map;
|
||||
import java.util.Queue;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentLinkedQueue;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -51,8 +54,6 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
private static final Logger log = LogManager.getLogger(DiskBasedMap.class);
|
||||
// Stores the key and corresponding value's latest metadata spilled to disk
|
||||
private final Map<T, ValueMetadata> valueMetadataMap;
|
||||
// Read only file access to be able to seek to random positions to readFromDisk values
|
||||
private RandomAccessFile readOnlyFileHandle;
|
||||
// Write only OutputStream to be able to ONLY append to the file
|
||||
private SizeAwareDataOutputStream writeOnlyFileHandle;
|
||||
// FileOutputStream for the file handle to be able to force fsync
|
||||
@@ -62,10 +63,12 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
private AtomicLong filePosition;
|
||||
// FilePath to store the spilled data
|
||||
private String filePath;
|
||||
// Thread-safe random access file
|
||||
private ThreadLocal<RandomAccessFile> randomAccessFile = new ThreadLocal<>();
|
||||
private Queue<RandomAccessFile> openedAccessFiles = new ConcurrentLinkedQueue<>();
|
||||
|
||||
|
||||
protected DiskBasedMap(String baseFilePath) throws IOException {
|
||||
this.valueMetadataMap = new HashMap<>();
|
||||
public DiskBasedMap(String baseFilePath) throws IOException {
|
||||
this.valueMetadataMap = new ConcurrentHashMap<>();
|
||||
File writeOnlyFileHandle = new File(baseFilePath, UUID.randomUUID().toString());
|
||||
this.filePath = writeOnlyFileHandle.getPath();
|
||||
initFile(writeOnlyFileHandle);
|
||||
@@ -74,6 +77,25 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
this.filePosition = new AtomicLong(0L);
|
||||
}
|
||||
|
||||
/**
|
||||
* RandomAcessFile is not thread-safe. This API opens a new file handle per thread and returns.
|
||||
* @return
|
||||
*/
|
||||
private RandomAccessFile getRandomAccessFile() {
|
||||
try {
|
||||
RandomAccessFile readHandle = randomAccessFile.get();
|
||||
if (readHandle == null) {
|
||||
readHandle = new RandomAccessFile(filePath, "r");
|
||||
readHandle.seek(0);
|
||||
randomAccessFile.set(readHandle);
|
||||
openedAccessFiles.offer(readHandle);
|
||||
}
|
||||
return readHandle;
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieException(ioe);
|
||||
}
|
||||
}
|
||||
|
||||
private void initFile(File writeOnlyFileHandle) throws IOException {
|
||||
// delete the file if it exists
|
||||
if (writeOnlyFileHandle.exists()) {
|
||||
@@ -86,9 +108,6 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
log.info(
|
||||
"Spilling to file location " + writeOnlyFileHandle.getAbsolutePath() + " in host (" + InetAddress.getLocalHost()
|
||||
.getHostAddress() + ") with hostname (" + InetAddress.getLocalHost().getHostName() + ")");
|
||||
// Open file in readFromDisk-only mode
|
||||
readOnlyFileHandle = new RandomAccessFile(filePath, "r");
|
||||
readOnlyFileHandle.seek(0);
|
||||
// Make sure file is deleted when JVM exits
|
||||
writeOnlyFileHandle.deleteOnExit();
|
||||
addShutDownHook();
|
||||
@@ -107,6 +126,17 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
fileOutputStream.getChannel().force(false);
|
||||
writeOnlyFileHandle.close();
|
||||
}
|
||||
|
||||
while (!openedAccessFiles.isEmpty()) {
|
||||
RandomAccessFile file = openedAccessFiles.poll();
|
||||
if (null != file) {
|
||||
try {
|
||||
file.close();
|
||||
} catch (IOException ioe) {
|
||||
// skip exception
|
||||
}
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// fail silently for any sort of exception
|
||||
}
|
||||
@@ -118,8 +148,7 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
* Custom iterator to iterate over values written to disk
|
||||
*/
|
||||
public Iterator<R> iterator() {
|
||||
return new LazyFileIterable(readOnlyFileHandle,
|
||||
valueMetadataMap).iterator();
|
||||
return new LazyFileIterable(filePath, valueMetadataMap).iterator();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -155,16 +184,24 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
if (entry == null) {
|
||||
return null;
|
||||
}
|
||||
return get(entry);
|
||||
}
|
||||
|
||||
private R get(ValueMetadata entry) {
|
||||
return get(entry, getRandomAccessFile());
|
||||
}
|
||||
|
||||
public static <R> R get(ValueMetadata entry, RandomAccessFile file) {
|
||||
try {
|
||||
return SerializationUtils.<R>deserialize(SpillableMapUtils
|
||||
.readBytesFromDisk(readOnlyFileHandle, entry.getOffsetOfValue(), entry.getSizeOfValue()));
|
||||
return SerializationUtils.deserialize(SpillableMapUtils.readBytesFromDisk(file,
|
||||
entry.getOffsetOfValue(), entry.getSizeOfValue()));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Unable to readFromDisk Hoodie Record from disk", e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public R put(T key, R value) {
|
||||
public synchronized R put(T key, R value) {
|
||||
try {
|
||||
byte[] val = SerializationUtils.serialize(value);
|
||||
Integer valueSize = val.length;
|
||||
@@ -198,14 +235,8 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
@Override
|
||||
public void clear() {
|
||||
valueMetadataMap.clear();
|
||||
// close input/output streams
|
||||
try {
|
||||
writeOnlyFileHandle.flush();
|
||||
writeOnlyFileHandle.close();
|
||||
new File(filePath).delete();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("unable to clear map or delete file on disk", e);
|
||||
}
|
||||
// Do not delete file-handles & file as there is no way to do it without synchronizing get/put(and
|
||||
// reducing concurrency). Instead, just clear the pointer map. The file will be removed on exit.
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -218,6 +249,12 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
throw new HoodieException("Unsupported Operation Exception");
|
||||
}
|
||||
|
||||
public Stream<R> valueStream() {
|
||||
final RandomAccessFile file = getRandomAccessFile();
|
||||
return valueMetadataMap.values().stream().sorted().sequential()
|
||||
.map(valueMetaData -> (R)get(valueMetaData, file));
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Entry<T, R>> entrySet() {
|
||||
Set<Entry<T, R>> entrySet = new HashSet<>();
|
||||
@@ -277,7 +314,7 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
}
|
||||
}
|
||||
|
||||
public final class ValueMetadata {
|
||||
public static final class ValueMetadata implements Comparable<ValueMetadata> {
|
||||
|
||||
// FilePath to store the spilled data
|
||||
private String filePath;
|
||||
@@ -310,5 +347,10 @@ public final class DiskBasedMap<T extends Serializable, R extends Serializable>
|
||||
public long getTimestamp() {
|
||||
return timestamp;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int compareTo(ValueMetadata o) {
|
||||
return Long.compare(this.offsetOfValue, o.offsetOfValue);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,15 +18,18 @@ package com.uber.hoodie.common.util.collection;
|
||||
|
||||
import com.twitter.common.objectsize.ObjectSizeCalculator;
|
||||
import com.uber.hoodie.common.util.SizeEstimator;
|
||||
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Stream;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -49,7 +52,7 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
|
||||
// Map to store key-values in memory until it hits maxInMemorySizeInBytes
|
||||
private final Map<T, R> inMemoryMap;
|
||||
// Map to store key-valuemetadata important to find the values spilled to disk
|
||||
private final DiskBasedMap<T, R> diskBasedMap;
|
||||
private transient volatile DiskBasedMap<T, R> diskBasedMap;
|
||||
// TODO(na) : a dynamic sizing factor to ensure we have space for other objects in memory and
|
||||
// incorrect payload estimation
|
||||
private final Double sizingFactorForInMemoryMap = 0.8;
|
||||
@@ -63,10 +66,13 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
|
||||
private volatile long estimatedPayloadSize = 0;
|
||||
// Flag to determine whether to stop re-estimating payload size
|
||||
private boolean shouldEstimatePayloadSize = true;
|
||||
|
||||
// Base File Path
|
||||
private final String baseFilePath;
|
||||
|
||||
public ExternalSpillableMap(Long maxInMemorySizeInBytes, String baseFilePath,
|
||||
SizeEstimator<T> keySizeEstimator, SizeEstimator<R> valueSizeEstimator) throws IOException {
|
||||
this.inMemoryMap = new HashMap<>();
|
||||
this.baseFilePath = baseFilePath;
|
||||
this.diskBasedMap = new DiskBasedMap<>(baseFilePath);
|
||||
this.maxInMemorySizeInBytes = (long) Math
|
||||
.floor(maxInMemorySizeInBytes * sizingFactorForInMemoryMap);
|
||||
@@ -75,25 +81,40 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
|
||||
this.valueSizeEstimator = valueSizeEstimator;
|
||||
}
|
||||
|
||||
private DiskBasedMap<T, R> getDiskBasedMap() {
|
||||
if (null == diskBasedMap) {
|
||||
synchronized (this) {
|
||||
if (null == diskBasedMap) {
|
||||
try {
|
||||
diskBasedMap = new DiskBasedMap<>(baseFilePath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return diskBasedMap;
|
||||
}
|
||||
|
||||
/**
|
||||
* A custom iterator to wrap over iterating in-memory + disk spilled data
|
||||
*/
|
||||
public Iterator<R> iterator() {
|
||||
return new IteratorWrapper<>(inMemoryMap.values().iterator(), diskBasedMap.iterator());
|
||||
return new IteratorWrapper<>(inMemoryMap.values().iterator(), getDiskBasedMap().iterator());
|
||||
}
|
||||
|
||||
/**
|
||||
* Number of entries in DiskBasedMap
|
||||
*/
|
||||
public int getDiskBasedMapNumEntries() {
|
||||
return diskBasedMap.size();
|
||||
return getDiskBasedMap().size();
|
||||
}
|
||||
|
||||
/**
|
||||
* Number of bytes spilled to disk
|
||||
*/
|
||||
public long getSizeOfFileOnDiskInBytes() {
|
||||
return diskBasedMap.sizeOfFileOnDiskInBytes();
|
||||
return getDiskBasedMap().sizeOfFileOnDiskInBytes();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -112,30 +133,30 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
|
||||
|
||||
@Override
|
||||
public int size() {
|
||||
return inMemoryMap.size() + diskBasedMap.size();
|
||||
return inMemoryMap.size() + getDiskBasedMap().size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isEmpty() {
|
||||
return inMemoryMap.isEmpty() && diskBasedMap.isEmpty();
|
||||
return inMemoryMap.isEmpty() && getDiskBasedMap().isEmpty();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsKey(Object key) {
|
||||
return inMemoryMap.containsKey(key) || diskBasedMap.containsKey(key);
|
||||
return inMemoryMap.containsKey(key) || getDiskBasedMap().containsKey(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean containsValue(Object value) {
|
||||
return inMemoryMap.containsValue(value) || diskBasedMap.containsValue(value);
|
||||
return inMemoryMap.containsValue(value) || getDiskBasedMap().containsValue(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public R get(Object key) {
|
||||
if (inMemoryMap.containsKey(key)) {
|
||||
return inMemoryMap.get(key);
|
||||
} else if (diskBasedMap.containsKey(key)) {
|
||||
return diskBasedMap.get(key);
|
||||
} else if (getDiskBasedMap().containsKey(key)) {
|
||||
return getDiskBasedMap().get(key);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
@@ -166,19 +187,19 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
|
||||
}
|
||||
inMemoryMap.put(key, value);
|
||||
} else {
|
||||
diskBasedMap.put(key, value);
|
||||
getDiskBasedMap().put(key, value);
|
||||
}
|
||||
return value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public R remove(Object key) {
|
||||
// NOTE : diskBasedMap.remove does not delete the data from disk
|
||||
// NOTE : getDiskBasedMap().remove does not delete the data from disk
|
||||
if (inMemoryMap.containsKey(key)) {
|
||||
currentInMemoryMapSize -= estimatedPayloadSize;
|
||||
return inMemoryMap.remove(key);
|
||||
} else if (diskBasedMap.containsKey(key)) {
|
||||
return diskBasedMap.remove(key);
|
||||
} else if (getDiskBasedMap().containsKey(key)) {
|
||||
return getDiskBasedMap().remove(key);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
@@ -193,7 +214,7 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
|
||||
@Override
|
||||
public void clear() {
|
||||
inMemoryMap.clear();
|
||||
diskBasedMap.clear();
|
||||
getDiskBasedMap().clear();
|
||||
currentInMemoryMapSize = 0L;
|
||||
}
|
||||
|
||||
@@ -201,23 +222,29 @@ public class ExternalSpillableMap<T extends Serializable, R extends Serializable
|
||||
public Set<T> keySet() {
|
||||
Set<T> keySet = new HashSet<T>();
|
||||
keySet.addAll(inMemoryMap.keySet());
|
||||
keySet.addAll(diskBasedMap.keySet());
|
||||
keySet.addAll(getDiskBasedMap().keySet());
|
||||
return keySet;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<R> values() {
|
||||
if (diskBasedMap.isEmpty()) {
|
||||
if (getDiskBasedMap().isEmpty()) {
|
||||
return inMemoryMap.values();
|
||||
}
|
||||
throw new HoodieNotSupportedException("Cannot return all values in memory");
|
||||
List<R> result = new ArrayList<>(inMemoryMap.values());
|
||||
result.addAll(getDiskBasedMap().values());
|
||||
return result;
|
||||
}
|
||||
|
||||
public Stream<R> valueStream() {
|
||||
return Stream.concat(inMemoryMap.values().stream(), getDiskBasedMap().valueStream());
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<Entry<T, R>> entrySet() {
|
||||
Set<Entry<T, R>> entrySet = new HashSet<>();
|
||||
entrySet.addAll(inMemoryMap.entrySet());
|
||||
entrySet.addAll(diskBasedMap.entrySet());
|
||||
entrySet.addAll(getDiskBasedMap().entrySet());
|
||||
return entrySet;
|
||||
}
|
||||
|
||||
|
||||
@@ -16,10 +16,7 @@
|
||||
|
||||
package com.uber.hoodie.common.util.collection;
|
||||
|
||||
import com.uber.hoodie.common.util.SerializationUtils;
|
||||
import com.uber.hoodie.common.util.SpillableMapUtils;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.util.Iterator;
|
||||
@@ -34,19 +31,19 @@ import java.util.stream.Collectors;
|
||||
public class LazyFileIterable<T, R> implements Iterable<R> {
|
||||
|
||||
// Used to access the value written at a specific position in the file
|
||||
private final RandomAccessFile readOnlyFileHandle;
|
||||
private final String filePath;
|
||||
// Stores the key and corresponding value's latest metadata spilled to disk
|
||||
private final Map<T, DiskBasedMap.ValueMetadata> inMemoryMetadataOfSpilledData;
|
||||
|
||||
public LazyFileIterable(RandomAccessFile file, Map<T, DiskBasedMap.ValueMetadata> map) {
|
||||
this.readOnlyFileHandle = file;
|
||||
public LazyFileIterable(String filePath, Map<T, DiskBasedMap.ValueMetadata> map) {
|
||||
this.filePath = filePath;
|
||||
this.inMemoryMetadataOfSpilledData = map;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<R> iterator() {
|
||||
try {
|
||||
return new LazyFileIterator<>(readOnlyFileHandle, inMemoryMetadataOfSpilledData);
|
||||
return new LazyFileIterator<>(filePath, inMemoryMetadataOfSpilledData);
|
||||
} catch (IOException io) {
|
||||
throw new HoodieException("Unable to initialize iterator for file on disk", io);
|
||||
}
|
||||
@@ -57,11 +54,15 @@ public class LazyFileIterable<T, R> implements Iterable<R> {
|
||||
*/
|
||||
public class LazyFileIterator<T, R> implements Iterator<R> {
|
||||
|
||||
private final String filePath;
|
||||
private RandomAccessFile readOnlyFileHandle;
|
||||
private Iterator<Map.Entry<T, DiskBasedMap.ValueMetadata>> metadataIterator;
|
||||
private final Iterator<Map.Entry<T, DiskBasedMap.ValueMetadata>> metadataIterator;
|
||||
|
||||
public LazyFileIterator(String filePath, Map<T, DiskBasedMap.ValueMetadata> map) throws IOException {
|
||||
this.filePath = filePath;
|
||||
this.readOnlyFileHandle = new RandomAccessFile(filePath, "r");
|
||||
readOnlyFileHandle.seek(0);
|
||||
|
||||
public LazyFileIterator(RandomAccessFile file, Map<T, DiskBasedMap.ValueMetadata> map) throws IOException {
|
||||
this.readOnlyFileHandle = file;
|
||||
// sort the map in increasing order of offset of value so disk seek is only in one(forward) direction
|
||||
this.metadataIterator = map
|
||||
.entrySet()
|
||||
@@ -70,23 +71,25 @@ public class LazyFileIterable<T, R> implements Iterable<R> {
|
||||
(Map.Entry<T, DiskBasedMap.ValueMetadata> o1, Map.Entry<T, DiskBasedMap.ValueMetadata> o2) ->
|
||||
o1.getValue().getOffsetOfValue().compareTo(o2.getValue().getOffsetOfValue()))
|
||||
.collect(Collectors.toList()).iterator();
|
||||
this.addShutdownHook();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return this.metadataIterator.hasNext();
|
||||
boolean available = this.metadataIterator.hasNext();
|
||||
if (!available) {
|
||||
close();
|
||||
}
|
||||
return available;
|
||||
}
|
||||
|
||||
@Override
|
||||
public R next() {
|
||||
Map.Entry<T, DiskBasedMap.ValueMetadata> entry = this.metadataIterator.next();
|
||||
try {
|
||||
return SerializationUtils.<R>deserialize(SpillableMapUtils
|
||||
.readBytesFromDisk(readOnlyFileHandle, entry.getValue().getOffsetOfValue(),
|
||||
entry.getValue().getSizeOfValue()));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Unable to read hoodie record from value spilled to disk", e);
|
||||
if (!hasNext()) {
|
||||
throw new IllegalStateException("next() called on EOF'ed stream. File :" + filePath);
|
||||
}
|
||||
Map.Entry<T, DiskBasedMap.ValueMetadata> entry = this.metadataIterator.next();
|
||||
return DiskBasedMap.get(entry.getValue(), readOnlyFileHandle);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -98,5 +101,24 @@ public class LazyFileIterable<T, R> implements Iterable<R> {
|
||||
public void forEachRemaining(Consumer<? super R> action) {
|
||||
action.accept(next());
|
||||
}
|
||||
|
||||
private void close() {
|
||||
if (readOnlyFileHandle != null) {
|
||||
try {
|
||||
readOnlyFileHandle.close();
|
||||
readOnlyFileHandle = null;
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void addShutdownHook() {
|
||||
Runtime.getRuntime().addShutdownHook(new Thread() {
|
||||
public void run() {
|
||||
close();
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,51 @@
|
||||
/*
|
||||
* 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.config;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Default Way to load Hoodie config through a java.util.Properties
|
||||
*/
|
||||
public class DefaultHoodieConfig implements Serializable {
|
||||
|
||||
protected final Properties props;
|
||||
|
||||
public DefaultHoodieConfig(Properties props) {
|
||||
this.props = props;
|
||||
}
|
||||
|
||||
public static void setDefaultOnCondition(Properties props, boolean condition, String propName,
|
||||
String defaultValue) {
|
||||
if (condition) {
|
||||
props.setProperty(propName, defaultValue);
|
||||
}
|
||||
}
|
||||
|
||||
public static void setDefaultOnCondition(Properties props, boolean condition,
|
||||
DefaultHoodieConfig config) {
|
||||
if (condition) {
|
||||
props.putAll(config.getProps());
|
||||
}
|
||||
}
|
||||
|
||||
public Properties getProps() {
|
||||
return props;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,27 @@
|
||||
/*
|
||||
* Copyright (c) 2018 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;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class HoodieRemoteException extends RuntimeException {
|
||||
|
||||
public HoodieRemoteException(IOException t) {
|
||||
super(t.getMessage(), t);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -89,9 +89,14 @@ public class HoodieTestUtils {
|
||||
|
||||
public static HoodieTableMetaClient init(String basePath)
|
||||
throws IOException {
|
||||
return initTableType(getDefaultHadoopConf(), basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
return init(basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
|
||||
public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType)
|
||||
throws IOException {
|
||||
return initTableType(getDefaultHadoopConf(), basePath, tableType);
|
||||
}
|
||||
|
||||
public static HoodieTableMetaClient init(Configuration hadoopConf, String basePath)
|
||||
throws IOException {
|
||||
return initTableType(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
|
||||
@@ -29,13 +29,13 @@ public class MockHoodieTimeline extends HoodieActiveTimeline {
|
||||
|
||||
public MockHoodieTimeline(Stream<String> completed, Stream<String> inflights) throws IOException {
|
||||
super();
|
||||
this.instants = Stream.concat(completed.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
|
||||
this.setInstants(Stream.concat(completed.map(s -> new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, s)),
|
||||
inflights.map(s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s)))
|
||||
.sorted(Comparator.comparing(new Function<HoodieInstant, String>() {
|
||||
@Override
|
||||
public String apply(HoodieInstant hoodieInstant) {
|
||||
return hoodieInstant.getFileName();
|
||||
}
|
||||
})).collect(Collectors.toList());
|
||||
})).collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,14 +23,17 @@ import static org.junit.Assert.assertTrue;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
@@ -43,6 +46,7 @@ import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
@@ -61,35 +65,38 @@ import org.junit.rules.TemporaryFolder;
|
||||
@SuppressWarnings("ResultOfMethodCallIgnored")
|
||||
public class HoodieTableFileSystemViewTest {
|
||||
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private String basePath;
|
||||
private HoodieTableFileSystemView fsView;
|
||||
private TableFileSystemView.ReadOptimizedView roView;
|
||||
private TableFileSystemView.RealtimeView rtView;
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
protected String basePath;
|
||||
protected SyncableFileSystemView fsView;
|
||||
protected TableFileSystemView.ReadOptimizedView roView;
|
||||
protected TableFileSystemView.RealtimeView rtView;
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath());;
|
||||
basePath = metaClient.getBasePath();
|
||||
fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants());
|
||||
roView = (TableFileSystemView.ReadOptimizedView) fsView;
|
||||
rtView = (TableFileSystemView.RealtimeView) fsView;
|
||||
initializeMetaClient();
|
||||
refreshFsView();
|
||||
}
|
||||
|
||||
private void refreshFsView(FileStatus[] statuses) {
|
||||
protected void initializeMetaClient() throws IOException {
|
||||
metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
|
||||
basePath = metaClient.getBasePath();
|
||||
}
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) throws IOException {
|
||||
return new HoodieTableFileSystemView(metaClient, timeline);
|
||||
}
|
||||
|
||||
protected void refreshFsView() throws IOException {
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
if (statuses != null) {
|
||||
fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants(),
|
||||
statuses);
|
||||
} else {
|
||||
fsView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants());
|
||||
if (null != fsView) {
|
||||
fsView.close();
|
||||
fsView = null;
|
||||
}
|
||||
fsView = getFileSystemView(
|
||||
metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().filterCompletedAndCompactionInstants());
|
||||
roView = (TableFileSystemView.ReadOptimizedView) fsView;
|
||||
rtView = (TableFileSystemView.RealtimeView) fsView;
|
||||
}
|
||||
@@ -100,6 +107,11 @@ public class HoodieTableFileSystemViewTest {
|
||||
*/
|
||||
@Test
|
||||
public void testViewForFileSlicesWithNoBaseFile() throws Exception {
|
||||
testViewForFileSlicesWithNoBaseFile(1, 0);
|
||||
}
|
||||
|
||||
protected void testViewForFileSlicesWithNoBaseFile(int expNumTotalFileSlices, int expNumTotalDataFiles)
|
||||
throws Exception {
|
||||
String partitionPath = "2016/05/01";
|
||||
new File(basePath + "/" + partitionPath).mkdirs();
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
@@ -120,7 +132,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
commitTimeline.saveAsComplete(deltaInstant2, Optional.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant3, Optional.empty());
|
||||
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
|
||||
List<HoodieDataFile> dataFiles = roView.getLatestDataFiles().collect(Collectors.toList());
|
||||
assertTrue("No data file expected", dataFiles.isEmpty());
|
||||
@@ -159,26 +171,33 @@ public class HoodieTableFileSystemViewTest {
|
||||
assertEquals("Correct number of log-files shows up in file-slice", 2, logFiles.size());
|
||||
assertEquals("Log File Order check", fileName2, logFiles.get(0).getFileName());
|
||||
assertEquals("Log File Order check", fileName1, logFiles.get(1).getFileName());
|
||||
|
||||
assertEquals("Total number of file-slices in view matches expected", expNumTotalFileSlices,
|
||||
rtView.getAllFileSlices(partitionPath).count());
|
||||
assertEquals("Total number of data-files in view matches expected", expNumTotalDataFiles,
|
||||
roView.getAllDataFiles(partitionPath).count());
|
||||
assertEquals("Total number of file-groups in view matches expected", 1,
|
||||
fsView.getAllFileGroups(partitionPath).count());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testViewForFileSlicesWithNoBaseFileAndRequestedCompaction() throws Exception {
|
||||
testViewForFileSlicesWithAsyncCompaction(true, false);
|
||||
testViewForFileSlicesWithAsyncCompaction(true, false, 2, 1, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testViewForFileSlicesWithBaseFileAndRequestedCompaction() throws Exception {
|
||||
testViewForFileSlicesWithAsyncCompaction(false, false);
|
||||
testViewForFileSlicesWithAsyncCompaction(false, false, 2, 2, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testViewForFileSlicesWithNoBaseFileAndInflightCompaction() throws Exception {
|
||||
testViewForFileSlicesWithAsyncCompaction(true, true);
|
||||
testViewForFileSlicesWithAsyncCompaction(true, true, 2, 1, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testViewForFileSlicesWithBaseFileAndInflightCompaction() throws Exception {
|
||||
testViewForFileSlicesWithAsyncCompaction(false, true);
|
||||
testViewForFileSlicesWithAsyncCompaction(false, true, 2, 2, true);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -210,10 +229,14 @@ public class HoodieTableFileSystemViewTest {
|
||||
* simulate inserts going directly to log files
|
||||
* @param isCompactionInFlight if set, compaction was inflight (running) when view was tested first time,
|
||||
* otherwise compaction was in requested state
|
||||
* @param expTotalFileSlices Total number of file-slices across file-groups in the partition path
|
||||
* @param expTotalDataFiles Total number of data-files across file-groups in the partition path
|
||||
* @param includeInvalidAndInflight Whether view includes inflight and invalid file-groups.
|
||||
* @throws Exception
|
||||
*/
|
||||
private void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFile,
|
||||
boolean isCompactionInFlight) throws Exception {
|
||||
protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingDataFile,
|
||||
boolean isCompactionInFlight, int expTotalFileSlices, int expTotalDataFiles,
|
||||
boolean includeInvalidAndInflight) throws Exception {
|
||||
String partitionPath = "2016/05/01";
|
||||
new File(basePath + "/" + partitionPath).mkdirs();
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
@@ -241,7 +264,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
commitTimeline.saveAsComplete(deltaInstant2, Optional.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant3, Optional.empty());
|
||||
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
List<FileSlice> fileSlices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
String compactionRequestedTime = "4";
|
||||
String compactDataFileName = FSUtils.makeDataFileName(compactionRequestedTime, 1, fileId);
|
||||
@@ -262,6 +285,15 @@ public class HoodieTableFileSystemViewTest {
|
||||
commitTimeline.saveToCompactionRequested(compactionInstant, AvroUtils.serializeCompactionPlan(compactionPlan));
|
||||
}
|
||||
|
||||
// View immediately after scheduling compaction
|
||||
refreshFsView();
|
||||
List<FileSlice> slices = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
assertEquals("Expected latest file-slices", 1, slices.size());
|
||||
assertEquals("Base-Instant must be compaction Instant", compactionRequestedTime,
|
||||
slices.get(0).getBaseInstantTime());
|
||||
assertFalse("Latest File Slice must not have data-file", slices.get(0).getDataFile().isPresent());
|
||||
assertTrue("Latest File Slice must not have any log-files", slices.get(0).getLogFiles().count() == 0);
|
||||
|
||||
// Fake delta-ingestion after compaction-requested
|
||||
String deltaInstantTime4 = "5";
|
||||
String deltaInstantTime5 = "6";
|
||||
@@ -275,7 +307,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
|
||||
commitTimeline.saveAsComplete(deltaInstant4, Optional.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant5, Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
|
||||
List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath).collect(Collectors.toList());
|
||||
if (skipCreatingDataFile) {
|
||||
@@ -381,7 +413,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
// Mark instant as inflight
|
||||
commitTimeline.saveToInflight(new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION,
|
||||
inflightDeltaInstantTime), Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
|
||||
List<FileSlice> allRawFileSlices = getAllRawFileSlices(partitionPath).collect(Collectors.toList());
|
||||
dataFiles = allRawFileSlices.stream().flatMap(slice -> {
|
||||
@@ -390,58 +422,60 @@ public class HoodieTableFileSystemViewTest {
|
||||
}
|
||||
return Stream.empty();
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals("Inflight/Orphan data-file is also expected", 2
|
||||
+ (isCompactionInFlight ? 1 : 0) + (skipCreatingDataFile ? 0 : 1), dataFiles.size());
|
||||
Set<String> fileNames = dataFiles.stream().map(HoodieDataFile::getFileName).collect(Collectors.toSet());
|
||||
assertTrue("Expect orphan data-file to be present", fileNames.contains(orphanDataFileName));
|
||||
assertTrue("Expect inflight data-file to be present", fileNames.contains(inflightDataFileName));
|
||||
if (!skipCreatingDataFile) {
|
||||
assertTrue("Expect old committed data-file", fileNames.contains(dataFileName));
|
||||
|
||||
if (includeInvalidAndInflight) {
|
||||
assertEquals("Inflight/Orphan data-file is also expected", 2
|
||||
+ (isCompactionInFlight ? 1 : 0) + (skipCreatingDataFile ? 0 : 1), dataFiles.size());
|
||||
Set<String> fileNames = dataFiles.stream().map(HoodieDataFile::getFileName).collect(Collectors.toSet());
|
||||
assertTrue("Expect orphan data-file to be present", fileNames.contains(orphanDataFileName));
|
||||
assertTrue("Expect inflight data-file to be present", fileNames.contains(inflightDataFileName));
|
||||
if (!skipCreatingDataFile) {
|
||||
assertTrue("Expect old committed data-file", fileNames.contains(dataFileName));
|
||||
}
|
||||
|
||||
if (isCompactionInFlight) {
|
||||
assertTrue("Expect inflight compacted data file to be present", fileNames.contains(compactDataFileName));
|
||||
}
|
||||
|
||||
fileSliceList = getLatestRawFileSlices(partitionPath).collect(Collectors.toList());
|
||||
assertEquals("Expect both inflight and orphan file-slice to be included",
|
||||
includeInvalidAndInflight ? 5 : 1, fileSliceList.size());
|
||||
Map<String, FileSlice> fileSliceMap =
|
||||
fileSliceList.stream().collect(Collectors.toMap(FileSlice::getFileId, r -> r));
|
||||
FileSlice orphanFileSliceWithDataFile = fileSliceMap.get(orphanFileId1);
|
||||
FileSlice orphanFileSliceWithLogFile = fileSliceMap.get(orphanFileId2);
|
||||
FileSlice inflightFileSliceWithDataFile = fileSliceMap.get(inflightFileId1);
|
||||
FileSlice inflightFileSliceWithLogFile = fileSliceMap.get(inflightFileId2);
|
||||
|
||||
assertEquals("Orphan File Slice with data-file check base-commit", invalidInstantId,
|
||||
orphanFileSliceWithDataFile.getBaseInstantTime());
|
||||
assertEquals("Orphan File Slice with data-file check data-file", orphanDataFileName,
|
||||
orphanFileSliceWithDataFile.getDataFile().get().getFileName());
|
||||
assertEquals("Orphan File Slice with data-file check data-file", 0,
|
||||
orphanFileSliceWithDataFile.getLogFiles().count());
|
||||
assertEquals("Inflight File Slice with data-file check base-commit", inflightDeltaInstantTime,
|
||||
inflightFileSliceWithDataFile.getBaseInstantTime());
|
||||
assertEquals("Inflight File Slice with data-file check data-file", inflightDataFileName,
|
||||
inflightFileSliceWithDataFile.getDataFile().get().getFileName());
|
||||
assertEquals("Inflight File Slice with data-file check data-file", 0,
|
||||
inflightFileSliceWithDataFile.getLogFiles().count());
|
||||
assertEquals("Orphan File Slice with log-file check base-commit", invalidInstantId,
|
||||
orphanFileSliceWithLogFile.getBaseInstantTime());
|
||||
assertFalse("Orphan File Slice with log-file check data-file",
|
||||
orphanFileSliceWithLogFile.getDataFile().isPresent());
|
||||
logFiles = orphanFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
|
||||
assertEquals("Orphan File Slice with log-file check data-file", 1, logFiles.size());
|
||||
assertEquals("Orphan File Slice with log-file check data-file", orphanLogFileName,
|
||||
logFiles.get(0).getFileName());
|
||||
assertEquals("Inflight File Slice with log-file check base-commit", inflightDeltaInstantTime,
|
||||
inflightFileSliceWithLogFile.getBaseInstantTime());
|
||||
assertFalse("Inflight File Slice with log-file check data-file",
|
||||
inflightFileSliceWithLogFile.getDataFile().isPresent());
|
||||
logFiles = inflightFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
|
||||
assertEquals("Inflight File Slice with log-file check data-file", 1, logFiles.size());
|
||||
assertEquals("Inflight File Slice with log-file check data-file", inflightLogFileName,
|
||||
logFiles.get(0).getFileName());
|
||||
}
|
||||
|
||||
if (isCompactionInFlight) {
|
||||
assertTrue("Expect inflight compacted data file to be present", fileNames.contains(compactDataFileName));
|
||||
}
|
||||
|
||||
fileSliceList = getLatestRawFileSlices(partitionPath).collect(Collectors.toList());
|
||||
assertEquals("Expect both inflight and orphan file-slice to be included",
|
||||
5, fileSliceList.size());
|
||||
Map<String, FileSlice> fileSliceMap =
|
||||
fileSliceList.stream().collect(Collectors.toMap(FileSlice::getFileId, r -> r));
|
||||
FileSlice orphanFileSliceWithDataFile = fileSliceMap.get(orphanFileId1);
|
||||
FileSlice orphanFileSliceWithLogFile = fileSliceMap.get(orphanFileId2);
|
||||
FileSlice inflightFileSliceWithDataFile = fileSliceMap.get(inflightFileId1);
|
||||
FileSlice inflightFileSliceWithLogFile = fileSliceMap.get(inflightFileId2);
|
||||
|
||||
assertEquals("Orphan File Slice with data-file check base-commit", invalidInstantId,
|
||||
orphanFileSliceWithDataFile.getBaseInstantTime());
|
||||
assertEquals("Orphan File Slice with data-file check data-file", orphanDataFileName,
|
||||
orphanFileSliceWithDataFile.getDataFile().get().getFileName());
|
||||
assertEquals("Orphan File Slice with data-file check data-file", 0,
|
||||
orphanFileSliceWithDataFile.getLogFiles().count());
|
||||
assertEquals("Inflight File Slice with data-file check base-commit", inflightDeltaInstantTime,
|
||||
inflightFileSliceWithDataFile.getBaseInstantTime());
|
||||
assertEquals("Inflight File Slice with data-file check data-file", inflightDataFileName,
|
||||
inflightFileSliceWithDataFile.getDataFile().get().getFileName());
|
||||
assertEquals("Inflight File Slice with data-file check data-file", 0,
|
||||
inflightFileSliceWithDataFile.getLogFiles().count());
|
||||
assertEquals("Orphan File Slice with log-file check base-commit", invalidInstantId,
|
||||
orphanFileSliceWithLogFile.getBaseInstantTime());
|
||||
assertFalse("Orphan File Slice with log-file check data-file",
|
||||
orphanFileSliceWithLogFile.getDataFile().isPresent());
|
||||
logFiles = orphanFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
|
||||
assertEquals("Orphan File Slice with log-file check data-file", 1, logFiles.size());
|
||||
assertEquals("Orphan File Slice with log-file check data-file", orphanLogFileName,
|
||||
logFiles.get(0).getFileName());
|
||||
assertEquals("Inflight File Slice with log-file check base-commit", inflightDeltaInstantTime,
|
||||
inflightFileSliceWithLogFile.getBaseInstantTime());
|
||||
assertFalse("Inflight File Slice with log-file check data-file",
|
||||
inflightFileSliceWithLogFile.getDataFile().isPresent());
|
||||
logFiles = inflightFileSliceWithLogFile.getLogFiles().collect(Collectors.toList());
|
||||
assertEquals("Inflight File Slice with log-file check data-file", 1, logFiles.size());
|
||||
assertEquals("Inflight File Slice with log-file check data-file", inflightLogFileName,
|
||||
logFiles.get(0).getFileName());
|
||||
|
||||
// Now simulate Compaction completing - Check the view
|
||||
if (!isCompactionInFlight) {
|
||||
// For inflight compaction, we already create a data-file to test concurrent inflight case.
|
||||
@@ -455,14 +489,14 @@ public class HoodieTableFileSystemViewTest {
|
||||
}
|
||||
compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime);
|
||||
commitTimeline.saveAsComplete(compactionInstant, Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
// populate the cache
|
||||
roView.getAllDataFiles(partitionPath);
|
||||
|
||||
fileSliceList = rtView.getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
System.out.println("FILESLICE LIST=" + fileSliceList);
|
||||
dataFiles = fileSliceList.stream().map(FileSlice::getDataFile)
|
||||
.filter(Optional::isPresent).map(Optional::get).collect(Collectors.toList());
|
||||
System.out.println("fileSliceList : " + fileSliceList);
|
||||
assertEquals("Expect only one data-files in latest view as there is only one file-group", 1, dataFiles.size());
|
||||
assertEquals("Data Filename must match", compactDataFileName, dataFiles.get(0).getFileName());
|
||||
assertEquals("Only one latest file-slice in the partition", 1, fileSliceList.size());
|
||||
@@ -502,6 +536,14 @@ public class HoodieTableFileSystemViewTest {
|
||||
assertEquals("Expect data-file created by compaction be returned", df.getCommitTime(),
|
||||
compactionRequestedTime);
|
||||
});
|
||||
|
||||
assertEquals("Total number of file-slices in partitions matches expected", expTotalFileSlices,
|
||||
rtView.getAllFileSlices(partitionPath).count());
|
||||
assertEquals("Total number of data-files in partitions matches expected", expTotalDataFiles,
|
||||
roView.getAllDataFiles(partitionPath).count());
|
||||
// file-groups includes inflight/invalid file-ids
|
||||
assertEquals("Total number of file-groups in partitions matches expected",
|
||||
5, fsView.getAllFileGroups(partitionPath).count());
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -518,7 +560,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
String commitTime1 = "1";
|
||||
String fileName1 = FSUtils.makeDataFileName(commitTime1, 1, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile();
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
assertFalse("No commit, should not find any data file",
|
||||
roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst()
|
||||
.isPresent());
|
||||
@@ -527,7 +569,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline();
|
||||
HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1);
|
||||
commitTimeline.saveAsComplete(instant1, Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
assertEquals("", fileName1,
|
||||
roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
|
||||
.getFileName());
|
||||
@@ -536,7 +578,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
String commitTime2 = "2";
|
||||
String fileName2 = FSUtils.makeDataFileName(commitTime2, 1, fileId);
|
||||
new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile();
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
assertEquals("", fileName1,
|
||||
roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
|
||||
.getFileName());
|
||||
@@ -544,7 +586,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
// Make it safe
|
||||
HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2);
|
||||
commitTimeline.saveAsComplete(instant2, Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
assertEquals("", fileName2,
|
||||
roView.getLatestDataFiles(partitionPath).filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get()
|
||||
.getFileName());
|
||||
@@ -552,6 +594,10 @@ public class HoodieTableFileSystemViewTest {
|
||||
|
||||
@Test
|
||||
public void testStreamLatestVersionInPartition() throws IOException {
|
||||
testStreamLatestVersionInPartition(false);
|
||||
}
|
||||
|
||||
public void testStreamLatestVersionInPartition(boolean isLatestFileSliceOnly) throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
@@ -588,16 +634,16 @@ public class HoodieTableFileSystemViewTest {
|
||||
// Now we list the entire partition
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(11, statuses.length);
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
|
||||
// Check files as of lastest commit.
|
||||
List<FileSlice> allSlices = rtView.getAllFileSlices("2016/05/01").collect(Collectors.toList());
|
||||
assertEquals(8, allSlices.size());
|
||||
assertEquals(isLatestFileSliceOnly ? 4 : 8, allSlices.size());
|
||||
Map<String, Long> fileSliceMap = allSlices.stream().collect(
|
||||
Collectors.groupingBy(slice -> slice.getFileId(), Collectors.counting()));
|
||||
assertEquals(2, fileSliceMap.get(fileId1).longValue());
|
||||
assertEquals(3, fileSliceMap.get(fileId2).longValue());
|
||||
assertEquals(2, fileSliceMap.get(fileId3).longValue());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 2, fileSliceMap.get(fileId1).longValue());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 3, fileSliceMap.get(fileId2).longValue());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 2, fileSliceMap.get(fileId3).longValue());
|
||||
assertEquals(1, fileSliceMap.get(fileId4).longValue());
|
||||
|
||||
List<HoodieDataFile> dataFileList = roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime4)
|
||||
@@ -627,24 +673,35 @@ public class HoodieTableFileSystemViewTest {
|
||||
// Reset the max commit time
|
||||
List<HoodieDataFile> dataFiles = roView.getLatestDataFilesBeforeOrOn("2016/05/01", commitTime3)
|
||||
.collect(Collectors.toList());
|
||||
assertEquals(dataFiles.size(), 3);
|
||||
filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : dataFiles) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(3, dataFiles.size());
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
} else {
|
||||
assertEquals(1, dataFiles.size());
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
}
|
||||
|
||||
logFilesList = rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3).map(slice -> slice.getLogFiles())
|
||||
.flatMap(logFileList -> logFileList).collect(Collectors.toList());
|
||||
logFilesList =
|
||||
rtView.getLatestFileSlicesBeforeOrOn("2016/05/01", commitTime3).map(slice -> slice.getLogFiles())
|
||||
.flatMap(logFileList -> logFileList).collect(Collectors.toList());
|
||||
assertEquals(logFilesList.size(), 1);
|
||||
assertTrue(logFilesList.get(0).getFileName()
|
||||
.equals(FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0)));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStreamEveryVersionInPartition() throws IOException {
|
||||
testStreamEveryVersionInPartition(false);
|
||||
}
|
||||
|
||||
protected void testStreamEveryVersionInPartition(boolean isLatestFileSliceOnly) throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
@@ -673,7 +730,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(7, statuses.length);
|
||||
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups("2016/05/01").collect(Collectors.toList());
|
||||
assertEquals(3, fileGroups.size());
|
||||
|
||||
@@ -684,21 +741,36 @@ public class HoodieTableFileSystemViewTest {
|
||||
assertEquals("All same fileId should be grouped", fileId, dataFile.getFileId());
|
||||
filenames.add(dataFile.getFileName());
|
||||
});
|
||||
Set<String> expFileNames = new HashSet<>();
|
||||
if (fileId.equals(fileId1)) {
|
||||
assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId1),
|
||||
FSUtils.makeDataFileName(commitTime4, 1, fileId1)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime1, 1, fileId1));
|
||||
}
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime4, 1, fileId1));
|
||||
assertEquals(expFileNames, filenames);
|
||||
} else if (fileId.equals(fileId2)) {
|
||||
assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime1, 1, fileId2),
|
||||
FSUtils.makeDataFileName(commitTime2, 1, fileId2), FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime1, 1, fileId2));
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime2, 1, fileId2));
|
||||
}
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime3, 1, fileId2));
|
||||
assertEquals(expFileNames, filenames);
|
||||
} else {
|
||||
assertEquals(filenames, Sets.newHashSet(FSUtils.makeDataFileName(commitTime3, 1, fileId3),
|
||||
FSUtils.makeDataFileName(commitTime4, 1, fileId3)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime3, 1, fileId3));
|
||||
}
|
||||
expFileNames.add(FSUtils.makeDataFileName(commitTime4, 1, fileId3));
|
||||
assertEquals(expFileNames, filenames);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void streamLatestVersionInRange() throws IOException {
|
||||
public void testStreamLatestVersionInRange() throws IOException {
|
||||
testStreamLatestVersionInRange(false);
|
||||
}
|
||||
|
||||
protected void testStreamLatestVersionInRange(boolean isLatestFileSliceOnly) throws IOException {
|
||||
// Put some files in the partition
|
||||
String fullPartitionPath = basePath + "/2016/05/01/";
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
@@ -718,7 +790,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime3, 0))
|
||||
.createNewFile();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
@@ -733,16 +805,23 @@ public class HoodieTableFileSystemViewTest {
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(9, statuses.length);
|
||||
|
||||
refreshFsView(statuses);
|
||||
refreshFsView();
|
||||
// Populate view for partition
|
||||
roView.getAllDataFiles("2016/05/01/");
|
||||
|
||||
List<HoodieDataFile> dataFiles = roView.getLatestDataFilesInRange(Lists.newArrayList(commitTime2, commitTime3))
|
||||
.collect(Collectors.toList());
|
||||
assertEquals(3, dataFiles.size());
|
||||
assertEquals(isLatestFileSliceOnly ? 2 : 3, dataFiles.size());
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : dataFiles) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId2)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime3, 1, fileId3)));
|
||||
}
|
||||
|
||||
List<FileSlice> slices = rtView.getLatestFileSliceInRange(Lists.newArrayList(commitTime3, commitTime4))
|
||||
.collect(Collectors.toList());
|
||||
@@ -753,8 +832,8 @@ public class HoodieTableFileSystemViewTest {
|
||||
assertTrue(slice.getDataFile().isPresent());
|
||||
assertEquals(slice.getLogFiles().count(), 0);
|
||||
} else if (slice.getFileId().equals(fileId2)) {
|
||||
assertEquals(slice.getBaseInstantTime(), commitTime4);
|
||||
assertFalse(slice.getDataFile().isPresent());
|
||||
assertEquals(slice.getBaseInstantTime(), commitTime3);
|
||||
assertTrue(slice.getDataFile().isPresent());
|
||||
assertEquals(slice.getLogFiles().count(), 1);
|
||||
} else if (slice.getFileId().equals(fileId3)) {
|
||||
assertEquals(slice.getBaseInstantTime(), commitTime4);
|
||||
@@ -765,7 +844,11 @@ public class HoodieTableFileSystemViewTest {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void streamLatestVersionsBefore() throws IOException {
|
||||
public void testStreamLatestVersionsBefore() throws IOException {
|
||||
testStreamLatestVersionsBefore(false);
|
||||
}
|
||||
|
||||
protected void testStreamLatestVersionsBefore(boolean isLatestFileSliceOnly) throws IOException {
|
||||
// Put some files in the partition
|
||||
String partitionPath = "2016/05/01/";
|
||||
String fullPartitionPath = basePath + "/" + partitionPath;
|
||||
@@ -795,22 +878,30 @@ public class HoodieTableFileSystemViewTest {
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(7, statuses.length);
|
||||
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
List<HoodieDataFile> dataFiles = roView.getLatestDataFilesBeforeOrOn(partitionPath, commitTime2)
|
||||
.collect(Collectors.toList());
|
||||
assertEquals(2, dataFiles.size());
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : dataFiles) {
|
||||
filenames.add(status.getFileName());
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(2, dataFiles.size());
|
||||
Set<String> filenames = Sets.newHashSet();
|
||||
for (HoodieDataFile status : dataFiles) {
|
||||
filenames.add(status.getFileName());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
|
||||
} else {
|
||||
assertEquals(0, dataFiles.size());
|
||||
}
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime1, 1, fileId1)));
|
||||
assertTrue(filenames.contains(FSUtils.makeDataFileName(commitTime2, 1, fileId2)));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void streamLatestVersions() throws IOException {
|
||||
public void testStreamLatestVersions() throws IOException {
|
||||
testStreamLatestVersions(false);
|
||||
}
|
||||
|
||||
protected void testStreamLatestVersions(boolean isLatestFileSliceOnly) throws IOException {
|
||||
// Put some files in the partition
|
||||
String partitionPath = "2016/05/01/";
|
||||
String partitionPath = "2016/05/01";
|
||||
String fullPartitionPath = basePath + "/" + partitionPath;
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
String commitTime1 = "1";
|
||||
@@ -821,21 +912,28 @@ public class HoodieTableFileSystemViewTest {
|
||||
String fileId2 = UUID.randomUUID().toString();
|
||||
String fileId3 = UUID.randomUUID().toString();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime4, 1, fileId1)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime4, 0)).createNewFile();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0))
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime1, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime2, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeLogFileName(fileId2, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime3, 1, fileId2)).createNewFile();
|
||||
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime3, 1, fileId3)).createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeDataFileName(commitTime4, 1, fileId3)).createNewFile();
|
||||
|
||||
new File(basePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(basePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
@@ -846,26 +944,32 @@ public class HoodieTableFileSystemViewTest {
|
||||
FileStatus[] statuses = metaClient.getFs().listStatus(new Path(fullPartitionPath));
|
||||
assertEquals(10, statuses.length);
|
||||
|
||||
refreshFsView(statuses);
|
||||
|
||||
refreshFsView();
|
||||
fsView.getAllDataFiles(partitionPath);
|
||||
List<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath).collect(Collectors.toList());
|
||||
assertEquals(3, fileGroups.size());
|
||||
for (HoodieFileGroup fileGroup : fileGroups) {
|
||||
List<FileSlice> slices = fileGroup.getAllFileSlices().collect(Collectors.toList());
|
||||
String fileId = fileGroup.getFileGroupId().getFileId();
|
||||
if (fileId.equals(fileId1)) {
|
||||
assertEquals(2, slices.size());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 2, slices.size());
|
||||
assertEquals(commitTime4, slices.get(0).getBaseInstantTime());
|
||||
assertEquals(commitTime1, slices.get(1).getBaseInstantTime());
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(commitTime1, slices.get(1).getBaseInstantTime());
|
||||
}
|
||||
} else if (fileId.equals(fileId2)) {
|
||||
assertEquals(3, slices.size());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 3, slices.size());
|
||||
assertEquals(commitTime3, slices.get(0).getBaseInstantTime());
|
||||
assertEquals(commitTime2, slices.get(1).getBaseInstantTime());
|
||||
assertEquals(commitTime1, slices.get(2).getBaseInstantTime());
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(commitTime2, slices.get(1).getBaseInstantTime());
|
||||
assertEquals(commitTime1, slices.get(2).getBaseInstantTime());
|
||||
}
|
||||
} else if (fileId.equals(fileId3)) {
|
||||
assertEquals(2, slices.size());
|
||||
assertEquals(isLatestFileSliceOnly ? 1 : 2, slices.size());
|
||||
assertEquals(commitTime4, slices.get(0).getBaseInstantTime());
|
||||
assertEquals(commitTime3, slices.get(1).getBaseInstantTime());
|
||||
if (!isLatestFileSliceOnly) {
|
||||
assertEquals(commitTime3, slices.get(1).getBaseInstantTime());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -925,9 +1029,11 @@ public class HoodieTableFileSystemViewTest {
|
||||
new Path(fullPartitionPath1), new Path(fullPartitionPath2), new Path(fullPartitionPath3)
|
||||
});
|
||||
assertEquals(6, statuses.length);
|
||||
refreshFsView(statuses);
|
||||
refreshFsView();
|
||||
Arrays.asList(partitionPath1, partitionPath2, partitionPath3).forEach(p -> fsView.getAllFileGroups(p).count());
|
||||
|
||||
List<HoodieFileGroup> groups = fsView.getAllFileGroups().collect(Collectors.toList());
|
||||
List<HoodieFileGroup> groups = Stream.of(partitionPath1, partitionPath2, partitionPath3)
|
||||
.flatMap(p -> fsView.getAllFileGroups(p)).collect(Collectors.toList());
|
||||
Assert.assertEquals("Expected number of file-groups", 3, groups.size());
|
||||
Assert.assertEquals("Partitions must be different for file-groups", 3,
|
||||
groups.stream().map(HoodieFileGroup::getPartitionPath).collect(Collectors.toSet()).size());
|
||||
@@ -979,7 +1085,7 @@ public class HoodieTableFileSystemViewTest {
|
||||
HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5);
|
||||
commitTimeline.saveAsComplete(deltaInstant4, Optional.empty());
|
||||
commitTimeline.saveAsComplete(deltaInstant5, Optional.empty());
|
||||
refreshFsView(null);
|
||||
refreshFsView();
|
||||
|
||||
// Test Data Files
|
||||
List<HoodieDataFile> dataFiles = roView.getAllDataFiles(partitionPath1).collect(Collectors.toList());
|
||||
@@ -1019,18 +1125,16 @@ public class HoodieTableFileSystemViewTest {
|
||||
assertEquals("Log File Order check", fileName3, logFiles.get(1).getFileName());
|
||||
});
|
||||
|
||||
Assert.assertEquals(3, fsView.getFgIdToPendingCompaction().size());
|
||||
Set<String> partitionsInCompaction =
|
||||
fsView.getFgIdToPendingCompaction().keySet().stream().map(HoodieFileGroupId::getPartitionPath)
|
||||
.collect(Collectors.toSet());
|
||||
Assert.assertEquals(3, fsView.getPendingCompactionOperations().count());
|
||||
Set<String> partitionsInCompaction = fsView.getPendingCompactionOperations().map(Pair::getValue)
|
||||
.map(CompactionOperation::getPartitionPath).collect(Collectors.toSet());
|
||||
Assert.assertEquals(3, partitionsInCompaction.size());
|
||||
Assert.assertTrue(partitionsInCompaction.contains(partitionPath1));
|
||||
Assert.assertTrue(partitionsInCompaction.contains(partitionPath2));
|
||||
Assert.assertTrue(partitionsInCompaction.contains(partitionPath3));
|
||||
|
||||
Set<String> fileIdsInCompaction =
|
||||
fsView.getFgIdToPendingCompaction().keySet().stream().map(HoodieFileGroupId::getFileId)
|
||||
.collect(Collectors.toSet());
|
||||
Set<String> fileIdsInCompaction = fsView.getPendingCompactionOperations().map(Pair::getValue)
|
||||
.map(CompactionOperation::getFileId).collect(Collectors.toSet());
|
||||
Assert.assertEquals(1, fileIdsInCompaction.size());
|
||||
Assert.assertTrue(fileIdsInCompaction.contains(fileId));
|
||||
}
|
||||
|
||||
@@ -0,0 +1,783 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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 static com.uber.hoodie.common.table.HoodieTimeline.COMPACTION_ACTION;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterators;
|
||||
import com.uber.hoodie.avro.model.HoodieCleanMetadata;
|
||||
import com.uber.hoodie.avro.model.HoodieCompactionPlan;
|
||||
import com.uber.hoodie.avro.model.HoodieRestoreMetadata;
|
||||
import com.uber.hoodie.avro.model.HoodieRollbackMetadata;
|
||||
import com.uber.hoodie.common.HoodieCleanStat;
|
||||
import com.uber.hoodie.common.HoodieRollbackStat;
|
||||
import com.uber.hoodie.common.model.CompactionOperation;
|
||||
import com.uber.hoodie.common.model.FileSlice;
|
||||
import com.uber.hoodie.common.model.HoodieCleaningPolicy;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroup;
|
||||
import com.uber.hoodie.common.model.HoodieFileGroupId;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieTestUtils;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant.State;
|
||||
import com.uber.hoodie.common.util.AvroUtils;
|
||||
import com.uber.hoodie.common.util.CompactionUtils;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
public class IncrementalFSViewSyncTest {
|
||||
|
||||
protected HoodieTableMetaClient metaClient;
|
||||
protected String basePath;
|
||||
|
||||
private final List<String> partitions = Arrays.asList("2018/01/01", "2018/01/02",
|
||||
"2019/03/01");
|
||||
private final List<String> fileIdsPerPartition =
|
||||
IntStream.range(0, 10).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList());
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder tmpFolder = new TemporaryFolder();
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
initializeMetaClient();
|
||||
refreshFsView();
|
||||
}
|
||||
|
||||
protected void initializeMetaClient() throws IOException {
|
||||
metaClient = HoodieTestUtils.init(tmpFolder.getRoot().getAbsolutePath(), HoodieTableType.MERGE_ON_READ);
|
||||
basePath = metaClient.getBasePath();
|
||||
partitions.forEach(p -> new File(basePath + "/" + p).mkdirs());
|
||||
}
|
||||
|
||||
protected void refreshFsView() throws IOException {
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
}
|
||||
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient) throws IOException {
|
||||
return getNewFileSystemView(metaClient, metaClient.getActiveTimeline().filterCompletedAndCompactionInstants());
|
||||
}
|
||||
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline)
|
||||
throws IOException {
|
||||
return new HoodieTableFileSystemView(metaClient, timeline, true);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyPartitionsAndTimeline() throws IOException {
|
||||
SyncableFileSystemView view = getNewFileSystemView(metaClient);
|
||||
Assert.assertFalse(view.getLastInstant().isPresent());
|
||||
partitions.forEach(p -> Assert.assertEquals(0, view.getLatestFileSlices(p).count()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAsyncCompaction() throws IOException {
|
||||
SyncableFileSystemView view = getNewFileSystemView(metaClient);
|
||||
view.sync();
|
||||
|
||||
// Run 3 ingestion on MOR table (3 delta commits)
|
||||
Map<String, List<String>> instantsToFiles =
|
||||
testMultipleWriteSteps(view, Arrays.asList("11", "12", "13"), true, "11");
|
||||
|
||||
// Schedule Compaction
|
||||
scheduleCompaction(view, "14");
|
||||
|
||||
// Restore pending compaction
|
||||
unscheduleCompaction(view, "14", "13", "11");
|
||||
|
||||
// Add one more delta instant
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("15"), true, "11"));
|
||||
|
||||
// Schedule Compaction again
|
||||
scheduleCompaction(view, "16");
|
||||
|
||||
// Run Compaction - This will be the second file-slice
|
||||
testMultipleWriteSteps(view, Arrays.asList("16"), false, "16", 2);
|
||||
|
||||
// Run 2 more ingest
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("17", "18"), true, "16", 2));
|
||||
|
||||
// Schedule Compaction again
|
||||
scheduleCompaction(view, "19");
|
||||
|
||||
// Run one more ingestion after pending compaction. THis will be 3rd slice
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("20"), true, "19", 3));
|
||||
|
||||
// Clean first slice
|
||||
testCleans(view, Arrays.asList("21"),
|
||||
new ImmutableMap.Builder<String, List<String>>().put("11", Arrays.asList("12", "13", "15")).build(),
|
||||
instantsToFiles,
|
||||
Arrays.asList("11"));
|
||||
|
||||
// Add one more ingestion instant. This should be 2nd slice now
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("22"), true, "19", 2));
|
||||
|
||||
// Restore last ingestion
|
||||
testRestore(view, Arrays.asList("23"), true, new HashMap<>(), Arrays.asList("22"), "24", false);
|
||||
|
||||
// Run one more ingestion. THis is still 2nd slice
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("24"), true, "19", 2));
|
||||
|
||||
// Finish Compaction
|
||||
instantsToFiles.putAll(
|
||||
testMultipleWriteSteps(view, Arrays.asList("19"), false, "19", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24"))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIngestion() throws IOException {
|
||||
SyncableFileSystemView view = getNewFileSystemView(metaClient);
|
||||
|
||||
// Add an empty ingestion
|
||||
String firstEmptyInstantTs = "11";
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs),
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
Assert.assertEquals("11", view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
Assert.assertEquals(HoodieTimeline.COMMIT_ACTION, view.getLastInstant().get().getAction());
|
||||
partitions.forEach(p -> Assert.assertEquals(0, view.getLatestFileSlices(p).count()));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
|
||||
areViewsConsistent(view, newView, 0L);
|
||||
|
||||
// Add 3 non-empty ingestions to COW table
|
||||
Map<String, List<String>> instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("12", "13", "14"));
|
||||
|
||||
// restore instants in reverse order till we rollback all
|
||||
testRestore(view, Arrays.asList("15", "16", "17"), false, instantsToFiles,
|
||||
Arrays.asList("14", "13", "12"), "17", true);
|
||||
|
||||
// Add 5 non-empty ingestions back-to-back
|
||||
instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("18", "19", "20"));
|
||||
|
||||
// Clean instants.
|
||||
testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests FS View incremental syncing behavior when multiple instants gets committed
|
||||
*/
|
||||
@Test
|
||||
public void testMultipleTransitions() throws IOException {
|
||||
|
||||
SyncableFileSystemView view1 = getNewFileSystemView(metaClient);
|
||||
view1.sync();
|
||||
Map<String, List<String>> instantsToFiles = null;
|
||||
|
||||
/**
|
||||
* Case where incremental syncing is catching up on more than one ingestion at a time
|
||||
*/
|
||||
// Run 1 ingestion on MOR table (1 delta commits). View1 is now sync up to this point
|
||||
instantsToFiles =
|
||||
testMultipleWriteSteps(view1, Arrays.asList("11"), true, "11");
|
||||
|
||||
SyncableFileSystemView view2 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
|
||||
// Run 2 more ingestion on MOR table. View1 is not yet synced but View2 is
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("12", "13"), true, "11"));
|
||||
|
||||
// Now Sync view1 and add 1 more ingestion. Check if view1 is able to catchup correctly
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view1, Arrays.asList("14"), true, "11"));
|
||||
|
||||
view2.sync();
|
||||
SyncableFileSystemView view3 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view3.getLatestFileSlices(p).count());
|
||||
view3.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size());
|
||||
|
||||
/**
|
||||
* Case where a compaction is scheduled and then unscheduled
|
||||
*/
|
||||
scheduleCompaction(view2, "15");
|
||||
unscheduleCompaction(view2, "15", "14", "11");
|
||||
view1.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size());
|
||||
SyncableFileSystemView view4 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view4.getLatestFileSlices(p).count());
|
||||
view4.sync();
|
||||
|
||||
/**
|
||||
* Case where a compaction is scheduled, 2 ingestion happens and then a compaction happens
|
||||
*/
|
||||
scheduleCompaction(view2, "16");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("17", "18"), true, "16", 2));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("16"), false, "16", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "18")));
|
||||
view1.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size() * 2);
|
||||
SyncableFileSystemView view5 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view5.getLatestFileSlices(p).count());
|
||||
view5.sync();
|
||||
|
||||
/**
|
||||
* Case where a clean happened and then rounds of ingestion and compaction happened
|
||||
*/
|
||||
testCleans(view2, Arrays.asList("19"),
|
||||
new ImmutableMap.Builder<String, List<String>>().put("11", Arrays.asList("12", "13", "14")).build(),
|
||||
instantsToFiles,
|
||||
Arrays.asList("11"));
|
||||
scheduleCompaction(view2, "20");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("21", "22"), true, "20", 2));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("20"), false, "20", 2,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "22")));
|
||||
//Run one more round of ingestion
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("23", "24"), true, "20", 2));
|
||||
view1.sync();
|
||||
areViewsConsistent(view1, view2, partitions.size() * fileIdsPerPartition.size() * 2);
|
||||
SyncableFileSystemView view6 =
|
||||
getNewFileSystemView(new HoodieTableMetaClient(metaClient.getHadoopConf(), metaClient.getBasePath()));
|
||||
partitions.stream().forEach(p -> view5.getLatestFileSlices(p).count());
|
||||
view6.sync();
|
||||
|
||||
/**
|
||||
* Case where multiple restores and ingestions happened
|
||||
*/
|
||||
testRestore(view2, Arrays.asList("25"), true, new HashMap<>(), Arrays.asList("24"), "29", true);
|
||||
testRestore(view2, Arrays.asList("26"), true, new HashMap<>(), Arrays.asList("23"), "29", false);
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("27"), true, "20", 2));
|
||||
scheduleCompaction(view2, "28");
|
||||
instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("29"), true, "28", 3));
|
||||
// Compaction
|
||||
testMultipleWriteSteps(view2, Arrays.asList("28"), false, "28", 3,
|
||||
Arrays.asList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "29")));
|
||||
|
||||
Arrays.asList(view1, view2, view3, view4, view5, view6).stream().forEach(v -> {
|
||||
v.sync();
|
||||
areViewsConsistent(v, view1, partitions.size() * fileIdsPerPartition.size() * 3);
|
||||
});
|
||||
}
|
||||
|
||||
/*
|
||||
********************************************************************************************************
|
||||
* HELPER METHODS
|
||||
*********************************************************************************************************
|
||||
*/
|
||||
|
||||
|
||||
/**
|
||||
* Helper to run one or more rounds of cleaning, incrementally syncing the view and then validate
|
||||
*/
|
||||
private void testCleans(SyncableFileSystemView view, List<String> newCleanerInstants,
|
||||
Map<String, List<String>> instantsToFiles, List<String> cleanedInstants) {
|
||||
Map<String, List<String>> deltaInstantMap = cleanedInstants.stream().map(e -> Pair.of(e, new ArrayList()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
testCleans(view, newCleanerInstants, deltaInstantMap, instantsToFiles, cleanedInstants);
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulates one of more cleaning, incrementally sync the view and validate the view
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param newCleanerInstants Cleaner Instants
|
||||
* @param deltaInstantMap File-Slice Base Instants to Delta Instants
|
||||
* @param instantsToFiles List of files associated with each instant
|
||||
* @param cleanedInstants List of cleaned instants
|
||||
*/
|
||||
private void testCleans(SyncableFileSystemView view, List<String> newCleanerInstants,
|
||||
Map<String, List<String>> deltaInstantMap,
|
||||
Map<String, List<String>> instantsToFiles, List<String> cleanedInstants) {
|
||||
Assert.assertEquals(newCleanerInstants.size(), cleanedInstants.size());
|
||||
long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong();
|
||||
long exp = initialFileSlices;
|
||||
System.out.println("Initial File Slices :" + exp);
|
||||
for (int idx = 0; idx < newCleanerInstants.size(); idx++) {
|
||||
String instant = cleanedInstants.get(idx);
|
||||
try {
|
||||
List<String> filesToDelete = new ArrayList<>(instantsToFiles.get(instant));
|
||||
deltaInstantMap.get(instant).stream().forEach(n -> filesToDelete.addAll(instantsToFiles.get(n)));
|
||||
|
||||
performClean(view, instant, filesToDelete, newCleanerInstants.get(idx));
|
||||
|
||||
exp -= fileIdsPerPartition.size();
|
||||
final long expTotalFileSlicesPerPartition = exp;
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
Assert.assertEquals(newCleanerInstants.get(idx), view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
Assert.assertEquals(HoodieTimeline.CLEAN_ACTION, view.getLastInstant().get().getAction());
|
||||
partitions.forEach(p -> {
|
||||
System.out.println("PARTTITION : " + p);
|
||||
System.out.println("\tFileSlices :" + view.getAllFileSlices(p).collect(Collectors.toList()));
|
||||
});
|
||||
|
||||
partitions.forEach(p -> Assert.assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
|
||||
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
areViewsConsistent(view, newView, expTotalFileSlicesPerPartition * partitions.size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulates one of more restores, incrementally sync the view and validate the view
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param newRestoreInstants Restore Instants
|
||||
* @param isDeltaCommit is Delta Commit ?
|
||||
* @param instantsToFiles List of files associated with each instant
|
||||
* @param rolledBackInstants List of rolled-back instants
|
||||
* @param emptyRestoreInstant Restore instant at which dataset becomes empty
|
||||
*/
|
||||
private void testRestore(SyncableFileSystemView view, List<String> newRestoreInstants, boolean isDeltaCommit,
|
||||
Map<String, List<String>> instantsToFiles, List<String> rolledBackInstants,
|
||||
String emptyRestoreInstant, boolean isRestore)
|
||||
throws IOException {
|
||||
Assert.assertEquals(newRestoreInstants.size(), rolledBackInstants.size());
|
||||
long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong();
|
||||
IntStream.range(0, newRestoreInstants.size()).forEach(idx -> {
|
||||
String instant = rolledBackInstants.get(idx);
|
||||
try {
|
||||
performRestore(view, instant, instantsToFiles.get(instant), newRestoreInstants.get(idx), isRestore);
|
||||
final long expTotalFileSlicesPerPartition = isDeltaCommit ? initialFileSlices :
|
||||
initialFileSlices - ((idx + 1) * fileIdsPerPartition.size());
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
System.out.println("Last Instant is :" + view.getLastInstant().get());
|
||||
if (isRestore) {
|
||||
Assert.assertEquals(newRestoreInstants.get(idx), view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(isRestore ? HoodieTimeline.RESTORE_ACTION : HoodieTimeline.ROLLBACK_ACTION,
|
||||
view.getLastInstant().get().getAction());
|
||||
}
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
|
||||
if (HoodieTimeline.compareTimestamps(newRestoreInstants.get(idx), emptyRestoreInstant,
|
||||
HoodieTimeline.GREATER_OR_EQUAL)) {
|
||||
partitions.forEach(p -> Assert.assertEquals(0, view.getLatestFileSlices(p).count()));
|
||||
} else {
|
||||
partitions.forEach(p -> Assert.assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
|
||||
}
|
||||
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
areViewsConsistent(view, newView, expTotalFileSlicesPerPartition * partitions.size());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulate a Cleaner operation cleaning up an instant
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instant Instant to be cleaner
|
||||
* @param files List of files to be deleted
|
||||
* @param cleanInstant Cleaner Instant
|
||||
*/
|
||||
private void performClean(SyncableFileSystemView view, String instant, List<String> files, String cleanInstant)
|
||||
throws IOException {
|
||||
Map<String, List<String>> partititonToFiles = deleteFiles(files);
|
||||
List<HoodieCleanStat> cleanStats = partititonToFiles.entrySet().stream().map(e -> {
|
||||
return new HoodieCleanStat(HoodieCleaningPolicy.KEEP_LATEST_COMMITS,
|
||||
e.getKey(), e.getValue(), e.getValue(), new ArrayList<>(),
|
||||
Integer.toString(Integer.parseInt(instant) + 1));
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
HoodieCleanMetadata cleanMetadata = AvroUtils.convertCleanMetadata(cleanInstant, Optional.empty(), cleanStats);
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant),
|
||||
AvroUtils.serializeCleanMetadata(cleanMetadata));
|
||||
}
|
||||
|
||||
/**
|
||||
* Simulate Restore of an instant in timeline and fsview
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instant Instant to be rolled-back
|
||||
* @param files List of files to be deleted as part of rollback
|
||||
* @param rollbackInstant Restore Instant
|
||||
*/
|
||||
private void performRestore(SyncableFileSystemView view, String instant, List<String> files, String rollbackInstant,
|
||||
boolean isRestore) throws IOException {
|
||||
Map<String, List<String>> partititonToFiles = deleteFiles(files);
|
||||
List<HoodieRollbackStat> rollbackStats = partititonToFiles.entrySet().stream().map(e -> {
|
||||
return new HoodieRollbackStat(e.getKey(), e.getValue(), new ArrayList<>(), new HashMap<>());
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
List<String> rollbacks = new ArrayList<>();
|
||||
rollbacks.add(instant);
|
||||
|
||||
HoodieRollbackMetadata rollbackMetadata = AvroUtils
|
||||
.convertRollbackMetadata(rollbackInstant, Optional.empty(), rollbacks, rollbackStats);
|
||||
if (isRestore) {
|
||||
HoodieRestoreMetadata metadata = new HoodieRestoreMetadata();
|
||||
|
||||
List<HoodieRollbackMetadata> rollbackM = new ArrayList<>();
|
||||
rollbackM.add(rollbackMetadata);
|
||||
metadata.setHoodieRestoreMetadata(
|
||||
new ImmutableMap.Builder().put(rollbackInstant, rollbackM).build());
|
||||
List<String> rollbackInstants = new ArrayList<>();
|
||||
rollbackInstants.add(rollbackInstant);
|
||||
metadata.setInstantsToRollback(rollbackInstants);
|
||||
metadata.setStartRestoreTime(rollbackInstant);
|
||||
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant),
|
||||
AvroUtils.serializeRestoreMetadata(metadata));
|
||||
} else {
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant),
|
||||
AvroUtils.serializeRollbackMetadata(rollbackMetadata));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility to delete a list of files and group the deleted files by partitions
|
||||
*
|
||||
* @param files List of files to be deleted
|
||||
*/
|
||||
private Map<String, List<String>> deleteFiles(List<String> files) {
|
||||
|
||||
if (null == files) {
|
||||
return new HashMap<>();
|
||||
}
|
||||
|
||||
Map<String, List<String>> partititonToFiles = new HashMap<>();
|
||||
partitions.forEach(p -> partititonToFiles.put(p, new ArrayList<>()));
|
||||
|
||||
for (String f : files) {
|
||||
String fullPath = String.format("%s/%s", metaClient.getBasePath(), f);
|
||||
new File(fullPath).delete();
|
||||
String partition = partitions.stream().filter(p -> f.startsWith(p)).findAny().get();
|
||||
partititonToFiles.get(partition).add(fullPath);
|
||||
}
|
||||
return partititonToFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Schedule a pending compaction and validate
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instantTime COmpaction Instant Time
|
||||
*/
|
||||
private void scheduleCompaction(SyncableFileSystemView view, String instantTime) throws IOException {
|
||||
List<Pair<String, FileSlice>> slices = partitions.stream().flatMap(p -> view.getLatestFileSlices(p)
|
||||
.map(s -> Pair.of(p, s))).collect(Collectors.toList());
|
||||
|
||||
long initialExpTotalFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).sum();
|
||||
|
||||
HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(slices, Optional.empty(), Optional.empty());
|
||||
HoodieInstant compactionInstant =
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime);
|
||||
metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant,
|
||||
AvroUtils.serializeCompactionPlan(plan));
|
||||
|
||||
view.sync();
|
||||
partitions.stream().forEach(p -> {
|
||||
view.getLatestFileSlices(p).forEach(fs -> {
|
||||
Assert.assertEquals(instantTime, fs.getBaseInstantTime());
|
||||
Assert.assertEquals(p, fs.getPartitionPath());
|
||||
Assert.assertFalse(fs.getDataFile().isPresent());
|
||||
});
|
||||
view.getLatestMergedFileSlicesBeforeOrOn(p, instantTime).forEach(fs -> {
|
||||
Assert.assertTrue(HoodieTimeline.compareTimestamps(instantTime, fs.getBaseInstantTime(),
|
||||
HoodieTimeline.GREATER));
|
||||
Assert.assertEquals(p, fs.getPartitionPath());
|
||||
});
|
||||
});
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
partitions.forEach(p -> newView.getLatestFileSlices(p).count());
|
||||
areViewsConsistent(view, newView, initialExpTotalFileSlices + partitions.size() * fileIdsPerPartition.size());
|
||||
}
|
||||
|
||||
/**
|
||||
* Unschedule a compaction instant and validate incremental fs view
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param compactionInstantTime Compaction Instant to be removed
|
||||
* @param newLastInstant New Last instant
|
||||
* @param newBaseInstant New Base instant of last file-slice
|
||||
*/
|
||||
private void unscheduleCompaction(SyncableFileSystemView view, String compactionInstantTime, String newLastInstant,
|
||||
String newBaseInstant) throws IOException {
|
||||
HoodieInstant instant = new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstantTime);
|
||||
boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false);
|
||||
Preconditions.checkArgument(deleted, "Unable to delete compaction instant.");
|
||||
|
||||
view.sync();
|
||||
Assert.assertEquals(newLastInstant, view.getLastInstant().get().getTimestamp());
|
||||
partitions.stream().forEach(p -> {
|
||||
view.getLatestFileSlices(p).forEach(fs -> {
|
||||
Assert.assertEquals(newBaseInstant, fs.getBaseInstantTime());
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @param deltaCommit Delta COmmit ?
|
||||
* @param baseInstantForDeltaCommit Base Instant to be used in case of delta-commit
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants,
|
||||
boolean deltaCommit, String baseInstantForDeltaCommit) throws IOException {
|
||||
return testMultipleWriteSteps(view, instants, deltaCommit, baseInstantForDeltaCommit, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @param deltaCommit Delta COmmit ?
|
||||
* @param baseInstantForDeltaCommit Base Instant to be used in case of delta-commit
|
||||
* @param begin initial file-slice offset
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants,
|
||||
boolean deltaCommit, String baseInstantForDeltaCommit, int begin)
|
||||
throws IOException {
|
||||
return testMultipleWriteSteps(view, instants, deltaCommit, baseInstantForDeltaCommit, begin,
|
||||
instants.stream().map(i -> new HoodieInstant(State.COMPLETED,
|
||||
deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, i))
|
||||
.collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants)
|
||||
throws IOException {
|
||||
return testMultipleWriteSteps(view, instants, false, null, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Perform one or more rounds of ingestion/compaction and validate incremental timeline syncing
|
||||
*
|
||||
* @param view Hoodie View
|
||||
* @param instants Ingestion/Commit INstants
|
||||
* @param deltaCommit Delta COmmit ?
|
||||
* @param baseInstantForDeltaCommit Base Instant to be used in case of delta-commit
|
||||
* @param begin initial file-slice offset
|
||||
* @param lastInstants List of Last Instants at each time we ingest/compact
|
||||
* @return List of new file created
|
||||
*/
|
||||
private Map<String, List<String>> testMultipleWriteSteps(SyncableFileSystemView view, List<String> instants,
|
||||
boolean deltaCommit, String baseInstantForDeltaCommit, int begin, List<HoodieInstant> lastInstants)
|
||||
throws IOException {
|
||||
Map<String, List<String>> instantToFiles = new HashMap<>();
|
||||
|
||||
int multiple = begin;
|
||||
for (int idx = 0; idx < instants.size(); idx++) {
|
||||
String instant = instants.get(idx);
|
||||
System.out.println("Adding instant=" + instant);
|
||||
HoodieInstant lastInstant = lastInstants.get(idx);
|
||||
// Add a non-empty ingestion to COW table
|
||||
List<String> filePaths = addInstant(metaClient, instant, deltaCommit,
|
||||
deltaCommit ? baseInstantForDeltaCommit : instant);
|
||||
view.sync();
|
||||
Assert.assertTrue(view.getLastInstant().isPresent());
|
||||
Assert.assertEquals(lastInstant.getTimestamp(), view.getLastInstant().get().getTimestamp());
|
||||
Assert.assertEquals(State.COMPLETED, view.getLastInstant().get().getState());
|
||||
Assert.assertEquals("Expected Last=" + lastInstant + ", Found Instants="
|
||||
+ view.getTimeline().getInstants().collect(Collectors.toList()),
|
||||
lastInstant.getAction(), view.getLastInstant().get().getAction());
|
||||
partitions.forEach(p -> Assert.assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count()));
|
||||
final long expTotalFileSlicesPerPartition = fileIdsPerPartition.size() * multiple;
|
||||
partitions.forEach(p -> Assert.assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count()));
|
||||
if (deltaCommit) {
|
||||
partitions.forEach(p -> {
|
||||
view.getLatestFileSlices(p).forEach(f -> {
|
||||
Assert.assertEquals(baseInstantForDeltaCommit, f.getBaseInstantTime());
|
||||
});
|
||||
});
|
||||
} else {
|
||||
partitions.forEach(p -> {
|
||||
view.getLatestDataFiles(p).forEach(f -> {
|
||||
Assert.assertEquals(instant, f.getCommitTime());
|
||||
});
|
||||
});
|
||||
}
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
SyncableFileSystemView newView = getNewFileSystemView(metaClient);
|
||||
for (String partition : partitions) {
|
||||
newView.getAllFileGroups(partition).count();
|
||||
}
|
||||
areViewsConsistent(view, newView, fileIdsPerPartition.size() * partitions.size() * multiple);
|
||||
instantToFiles.put(instant, filePaths);
|
||||
if (!deltaCommit) {
|
||||
multiple++;
|
||||
}
|
||||
}
|
||||
return instantToFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Check for equality of views
|
||||
*
|
||||
* @param view1 View1
|
||||
* @param view2 View2
|
||||
*/
|
||||
private void areViewsConsistent(SyncableFileSystemView view1, SyncableFileSystemView view2,
|
||||
long expectedTotalFileSlices) {
|
||||
//Timeline check
|
||||
HoodieTimeline timeline1 = view1.getTimeline();
|
||||
HoodieTimeline timeline2 = view2.getTimeline();
|
||||
Assert.assertEquals(view1.getLastInstant(), view2.getLastInstant());
|
||||
Iterators.elementsEqual(timeline1.getInstants().iterator(), timeline2.getInstants().iterator());
|
||||
|
||||
//View Checks
|
||||
Map<HoodieFileGroupId, HoodieFileGroup> fileGroupsMap1 =
|
||||
partitions.stream().flatMap(p -> view1.getAllFileGroups(p))
|
||||
.collect(Collectors.toMap(fg -> fg.getFileGroupId(), fg -> fg));
|
||||
Map<HoodieFileGroupId, HoodieFileGroup> fileGroupsMap2 =
|
||||
partitions.stream().flatMap(p -> view2.getAllFileGroups(p))
|
||||
.collect(Collectors.toMap(fg -> fg.getFileGroupId(), fg -> fg));
|
||||
Assert.assertEquals(fileGroupsMap1.keySet(), fileGroupsMap2.keySet());
|
||||
long gotSlicesCount =
|
||||
fileGroupsMap1.keySet().stream().map(k -> Pair.of(fileGroupsMap1.get(k), fileGroupsMap2.get(k)))
|
||||
.mapToLong(e -> {
|
||||
HoodieFileGroup fg1 = e.getKey();
|
||||
HoodieFileGroup fg2 = e.getValue();
|
||||
Assert.assertEquals(fg1.getFileGroupId(), fg2.getFileGroupId());
|
||||
List<FileSlice> slices1 = fg1.getAllRawFileSlices().collect(Collectors.toList());
|
||||
List<FileSlice> slices2 = fg2.getAllRawFileSlices().collect(Collectors.toList());
|
||||
Assert.assertEquals(slices1.size(), slices2.size());
|
||||
IntStream.range(0, slices1.size()).mapToObj(idx -> Pair.of(slices1.get(idx), slices2.get(idx)))
|
||||
.forEach(e2 -> {
|
||||
FileSlice slice1 = e2.getKey();
|
||||
FileSlice slice2 = e2.getValue();
|
||||
Assert.assertEquals(slice1.getBaseInstantTime(), slice2.getBaseInstantTime());
|
||||
Assert.assertEquals(slice1.getFileId(), slice2.getFileId());
|
||||
Assert.assertEquals(slice1.getDataFile().isPresent(), slice2.getDataFile().isPresent());
|
||||
if (slice1.getDataFile().isPresent()) {
|
||||
HoodieDataFile df1 = slice1.getDataFile().get();
|
||||
HoodieDataFile df2 = slice2.getDataFile().get();
|
||||
Assert.assertEquals(df1.getCommitTime(), df2.getCommitTime());
|
||||
Assert.assertEquals(df1.getFileId(), df2.getFileId());
|
||||
Assert.assertEquals(df1.getFileName(), df2.getFileName());
|
||||
Assert.assertEquals(Path.getPathWithoutSchemeAndAuthority(new Path(df1.getPath())),
|
||||
Path.getPathWithoutSchemeAndAuthority(new Path(df2.getPath())));
|
||||
}
|
||||
List<Path> logPaths1 = slice1.getLogFiles()
|
||||
.map(lf -> Path.getPathWithoutSchemeAndAuthority(lf.getPath())).collect(Collectors.toList());
|
||||
List<Path> logPaths2 = slice2.getLogFiles()
|
||||
.map(lf -> Path.getPathWithoutSchemeAndAuthority(lf.getPath())).collect(Collectors.toList());
|
||||
Assert.assertEquals(logPaths1, logPaths2);
|
||||
});
|
||||
return slices1.size();
|
||||
}).sum();
|
||||
Assert.assertEquals(expectedTotalFileSlices, gotSlicesCount);
|
||||
|
||||
// Pending Compaction Operations Check
|
||||
Set<Pair<String, CompactionOperation>> ops1 = view1.getPendingCompactionOperations().collect(Collectors.toSet());
|
||||
Set<Pair<String, CompactionOperation>> ops2 = view2.getPendingCompactionOperations().collect(Collectors.toSet());
|
||||
Assert.assertEquals(ops1, ops2);
|
||||
}
|
||||
|
||||
private List<String> addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit,
|
||||
String baseInstant)
|
||||
throws IOException {
|
||||
List<Pair<String, HoodieWriteStat>> writeStats = partitions.stream().flatMap(p -> {
|
||||
return fileIdsPerPartition.stream().map(f -> {
|
||||
try {
|
||||
File file = new File(basePath + "/" + p + "/"
|
||||
+ (deltaCommit ? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant)) :
|
||||
FSUtils.makeDataFileName(instant, 0, f)));
|
||||
file.createNewFile();
|
||||
HoodieWriteStat w = new HoodieWriteStat();
|
||||
w.setFileId(f);
|
||||
w.setPath(String.format("%s/%s", p, file.getName()));
|
||||
return Pair.of(p, w);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue()));
|
||||
metaClient.getActiveTimeline().saveAsComplete(
|
||||
new HoodieInstant(true, deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION,
|
||||
instant),
|
||||
Optional.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Delete pending compaction if present
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(),
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instant).getFileName()));
|
||||
return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,33 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import java.io.IOException;
|
||||
|
||||
public class RocksDBBasedIncrementalFSViewSyncTest extends IncrementalFSViewSyncTest {
|
||||
|
||||
@Override
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline)
|
||||
throws IOException {
|
||||
return new RocksDbBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withRocksDBPath(tmpFolder.newFolder().getAbsolutePath())
|
||||
.withIncrementalTimelineSync(true).build());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
import java.io.IOException;
|
||||
|
||||
public class RocksDbBasedFileSystemViewTest extends HoodieTableFileSystemViewTest {
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) throws IOException {
|
||||
return new RocksDbBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withRocksDBPath(tmpFolder.newFolder().getAbsolutePath())
|
||||
.build());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,29 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
|
||||
public class SpillableMapBasedFileSystemViewTest extends HoodieTableFileSystemViewTest {
|
||||
|
||||
protected SyncableFileSystemView getFileSystemView(HoodieTimeline timeline) {
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withStorageType(FileSystemViewStorageType.SPILLABLE_DISK)
|
||||
.withMaxMemoryForView(0L).build()); // pure disk base View
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* Copyright (c) 2019 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.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.SyncableFileSystemView;
|
||||
|
||||
public class SpillableMapBasedIncrementalFSViewSyncTest extends IncrementalFSViewSyncTest {
|
||||
|
||||
@Override
|
||||
protected SyncableFileSystemView getNewFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline timeline) {
|
||||
return new SpillableMapBasedFileSystemView(metaClient, timeline,
|
||||
FileSystemViewStorageConfig.newBuilder().withMaxMemoryForView(0L).withIncrementalTimelineSync(true).build());
|
||||
}
|
||||
}
|
||||
@@ -182,7 +182,7 @@ public class CompactionTestUtils {
|
||||
private final String path;
|
||||
|
||||
public TestHoodieDataFile(String path) {
|
||||
super(null);
|
||||
super("/tmp/ce481ee7-9e53-4a2e-9992-f9e295fa79c0_11_20180918020003.parquet");
|
||||
this.path = path;
|
||||
}
|
||||
|
||||
|
||||
@@ -77,7 +77,7 @@ public class TestCompactionUtils {
|
||||
|
||||
// File Slice with data-file but no log files
|
||||
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1");
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
op = CompactionUtils.buildFromFileSlice(
|
||||
DEFAULT_PARTITION_PATHS[0], noLogFileSlice, Optional.of(metricsCaptureFn));
|
||||
testFileSliceCompactionOpEquality(noLogFileSlice, op, DEFAULT_PARTITION_PATHS[0]);
|
||||
@@ -94,7 +94,7 @@ public class TestCompactionUtils {
|
||||
|
||||
//File Slice with data-file and log files present
|
||||
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
@@ -110,13 +110,13 @@ public class TestCompactionUtils {
|
||||
private Pair<List<Pair<String, FileSlice>>, HoodieCompactionPlan> buildCompactionPlan() {
|
||||
FileSlice emptyFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "empty1");
|
||||
FileSlice fileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
|
||||
fileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
|
||||
fileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 2))));
|
||||
FileSlice noLogFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noLog1");
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog.parquet"));
|
||||
noLogFileSlice.setDataFile(new TestHoodieDataFile("/tmp/noLog_1_000.parquet"));
|
||||
FileSlice noDataFileSlice = new FileSlice(DEFAULT_PARTITION_PATHS[0],"000", "noData1");
|
||||
noDataFileSlice.addLogFile(new HoodieLogFile(new Path(
|
||||
FSUtils.makeLogFileName("noData1", ".log", "000", 1))));
|
||||
|
||||
@@ -0,0 +1,183 @@
|
||||
/*
|
||||
* 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.uber.hoodie.common.table.view.FileSystemViewStorageConfig;
|
||||
import com.uber.hoodie.common.util.collection.Pair;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
import org.apache.log4j.Level;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestRocksDBManager {
|
||||
|
||||
static {
|
||||
RocksDBDAO.log.setLevel(Level.INFO);
|
||||
}
|
||||
|
||||
private static RocksDBDAO dbManager;
|
||||
|
||||
@AfterClass
|
||||
public static void drop() throws IOException {
|
||||
if (dbManager != null) {
|
||||
dbManager.close();
|
||||
dbManager = null;
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRocksDBManager() throws Exception {
|
||||
String prefix1 = "prefix1_";
|
||||
String prefix2 = "prefix2_";
|
||||
String prefix3 = "prefix3_";
|
||||
String prefix4 = "prefix4_";
|
||||
List<String> prefixes = Arrays.asList(prefix1, prefix2, prefix3, prefix4);
|
||||
String family1 = "family1";
|
||||
String family2 = "family2";
|
||||
List<String> colFamilies = Arrays.asList(family1, family2);
|
||||
|
||||
List<Payload> payloads = IntStream.range(0, 100).mapToObj(index -> {
|
||||
String prefix = prefixes.get(index % 4);
|
||||
String key = prefix + UUID.randomUUID().toString();
|
||||
String family = colFamilies.get(index % 2);
|
||||
String val = "VALUE_" + UUID.randomUUID().toString();
|
||||
return new Payload(prefix, key, val, family);
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
dbManager = new RocksDBDAO("/dummy/path",
|
||||
FileSystemViewStorageConfig.newBuilder().build().newBuilder().build());
|
||||
colFamilies.stream().forEach(family -> dbManager.dropColumnFamily(family));
|
||||
colFamilies.stream().forEach(family -> dbManager.addColumnFamily(family));
|
||||
|
||||
Map<String, Map<String, Integer>> countsMap = new HashMap<>();
|
||||
payloads.stream().forEach(payload -> {
|
||||
dbManager.put(payload.getFamily(), payload.getKey(), payload);
|
||||
|
||||
if (!countsMap.containsKey(payload.family)) {
|
||||
countsMap.put(payload.family, new HashMap<>());
|
||||
}
|
||||
Map<String, Integer> c = countsMap.get(payload.family);
|
||||
if (!c.containsKey(payload.prefix)) {
|
||||
c.put(payload.prefix, 0);
|
||||
}
|
||||
int currCount = c.get(payload.prefix);
|
||||
c.put(payload.prefix, currCount + 1);
|
||||
});
|
||||
|
||||
colFamilies.stream().forEach(family -> {
|
||||
prefixes.stream().forEach(prefix -> {
|
||||
List<Pair<String, Payload>> gotPayloads =
|
||||
dbManager.<Payload>prefixSearch(family, prefix).collect(Collectors.toList());
|
||||
Integer expCount = countsMap.get(family).get(prefix);
|
||||
Assert.assertEquals("Size check for prefix (" + prefix + ") and family (" + family + ")",
|
||||
expCount == null ? 0L : expCount.longValue(), gotPayloads.size());
|
||||
gotPayloads.stream().forEach(p -> {
|
||||
Assert.assertEquals(p.getRight().getFamily(), family);
|
||||
Assert.assertTrue(p.getRight().getKey().startsWith(prefix));
|
||||
});
|
||||
});
|
||||
});
|
||||
|
||||
payloads.stream().forEach(payload -> {
|
||||
Payload p = dbManager.get(payload.getFamily(), payload.getKey());
|
||||
Assert.assertEquals("Retrieved correct payload for key :" + payload.getKey(), payload, p);
|
||||
|
||||
// Now, delete the key
|
||||
dbManager.delete(payload.getFamily(), payload.getKey());
|
||||
|
||||
// Now retrieve
|
||||
Payload p2 = dbManager.get(payload.getFamily(), payload.getKey());
|
||||
Assert.assertNull("Retrieved correct payload for key :" + p.getKey(), p2);
|
||||
});
|
||||
|
||||
// Now do a prefix search
|
||||
colFamilies.stream().forEach(family -> {
|
||||
prefixes.stream().forEach(prefix -> {
|
||||
List<Pair<String, Payload>> gotPayloads =
|
||||
dbManager.<Payload>prefixSearch(family, prefix).collect(Collectors.toList());
|
||||
Assert.assertEquals("Size check for prefix (" + prefix + ") and family (" + family + ")",
|
||||
0, gotPayloads.size());
|
||||
});
|
||||
});
|
||||
|
||||
String rocksDBBasePath = dbManager.getRocksDBBasePath();
|
||||
dbManager.close();
|
||||
Assert.assertFalse(new File(rocksDBBasePath).exists());
|
||||
}
|
||||
|
||||
public static class Payload implements Serializable {
|
||||
|
||||
private final String prefix;
|
||||
private final String key;
|
||||
private final String val;
|
||||
private final String family;
|
||||
|
||||
public Payload(String prefix, String key, String val, String family) {
|
||||
this.prefix = prefix;
|
||||
this.key = key;
|
||||
this.val = val;
|
||||
this.family = family;
|
||||
}
|
||||
|
||||
public String getPrefix() {
|
||||
return prefix;
|
||||
}
|
||||
|
||||
public String getKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
public String getVal() {
|
||||
return val;
|
||||
}
|
||||
|
||||
public String getFamily() {
|
||||
return family;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
Payload payload = (Payload) o;
|
||||
return Objects.equals(prefix, payload.prefix)
|
||||
&& Objects.equals(key, payload.key)
|
||||
&& Objects.equals(val, payload.val)
|
||||
&& Objects.equals(family, payload.family);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(prefix, key, val, family);
|
||||
}
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user