1
0

Refactor hoodie-common and create right abstractions for Hoodie Storage V2.0

The following is the gist of changes done

- All low-level operation of creating a commit code was in HoodieClient which made it hard to share code if there was a compaction commit.
- HoodieTableMetadata contained a mix of metadata and filtering files. (Also few operations required FileSystem to be passed in because those were called from TaskExecutors and others had FileSystem as a global variable). Since merge-on-read requires a lot of that code, but will have to change slightly on how it operates on the metadata and how it filters the files. The two set of operation are split into HoodieTableMetaClient and TableFileSystemView.
- Everything (active commits, archived commits, cleaner log, save point log and in future delta and compaction commits) in HoodieTableMetaClient is a HoodieTimeline. Timeline is a series of instants, which has an in-built concept of inflight and completed commit markers.
- A timeline can be queries for ranges, contains and also use to create new datapoint (create a new commit etc). Commit (and all the above metadata) creation/deletion is streamlined in a timeline
- Multiple timelines can be merged into a single timeline, giving us an audit timeline to whatever happened in a hoodie dataset. This also helps with #55.
- Move to java 8 and introduce java 8 succinct syntax in refactored code
This commit is contained in:
Prasanna Rajaperumal
2017-01-04 14:40:26 -08:00
parent 283269e57f
commit 8ee777a9bb
76 changed files with 3480 additions and 1932 deletions

View File

@@ -24,6 +24,7 @@ import org.codehaus.jackson.map.ObjectMapper;
import java.io.IOException;
import java.io.Serializable;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -187,4 +188,8 @@ public class HoodieCommitMetadata implements Serializable {
public int hashCode() {
return partitionToWriteStats != null ? partitionToWriteStats.hashCode() : 0;
}
public static HoodieCommitMetadata fromBytes(byte[] bytes) throws IOException {
return fromJsonString(new String(bytes, Charset.forName("utf-8")));
}
}

View File

@@ -1,191 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.model;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
/**
* Manages the commit meta and provides operations on the commit timeline
*/
public class HoodieCommits implements Serializable {
private List<String> commitList;
public HoodieCommits(List<String> commitList) {
this.commitList = new ArrayList<>(commitList);
Collections.sort(this.commitList);
this.commitList = Collections.unmodifiableList(this.commitList);
}
/**
* Returns the commits which are in the range (startsTs, endTs].
*
* @param startTs - exclusive start commit ts
* @param endTs - inclusive end commit ts
*/
public List<String> findCommitsInRange(String startTs, String endTs) {
if (commitList.isEmpty()) {
return Collections.EMPTY_LIST;
}
int startIndex = 0;
if (startTs != null) {
startIndex = Collections.binarySearch(commitList, startTs);
// If startIndex is negative
if (startIndex < 0) {
startIndex = -(startIndex + 1);
}
}
int endIndex = Collections.binarySearch(commitList, endTs);
// If endIndex is negative
if (endIndex < 0) {
endIndex = -(endIndex + 1);
}
if (endIndex < startIndex) {
throw new IllegalArgumentException(
"Start Commit Ts " + startTs + " cannot be less than end commit ts" + endTs);
}
List<String> returns = new ArrayList<>(commitList.subList(startIndex, endIndex));
if(endIndex < commitList.size()) {
// Be inclusive of the endIndex
returns.add(commitList.get(endIndex));
}
return Collections.unmodifiableList(returns);
}
/**
* Finds the list of commits on or before asOfTs
*/
public List<String> findCommitsAfter(String commitTimeStamp, int numCommits) {
if (commitList.isEmpty()) {
return null;
}
int startIndex = Collections.binarySearch(commitList, commitTimeStamp);
if (startIndex < 0) {
startIndex = -(startIndex + 1);
} else {
// we found asOfTs at startIndex. We want to exclude it.
startIndex++;
}
List<String> commits = new ArrayList<>();
while (numCommits > 0 && startIndex < commitList.size()) {
commits.add(commitList.get(startIndex));
startIndex++;
numCommits--;
}
return Collections.unmodifiableList(commits);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieCommits{");
sb.append("commitList=").append(commitList);
sb.append('}');
return sb.toString();
}
public boolean isEmpty() {
return commitList.isEmpty();
}
public int getNumCommits() {
return commitList.size();
}
public String firstCommit() {
return commitList.isEmpty() ? null : commitList.get(0);
}
public String nthCommit(int n) {
return commitList.isEmpty() || n >= commitList.size() ? null : commitList.get(n);
}
public String lastCommit() {
return commitList.isEmpty() ? null : commitList.get(commitList.size() - 1);
}
/**
* Returns the nth commit from the latest commit such that lastCommit(0) gteq lastCommit()
*/
public String lastCommit(int n) {
if (commitList.size() < n + 1) {
return null;
}
return commitList.get(commitList.size() - 1 - n);
}
public boolean contains(String commitTs) {
return commitList.contains(commitTs);
}
public String max(String commit1, String commit2) {
if (commit1 == null && commit2 == null) {
return null;
}
if (commit1 == null) {
return commit2;
}
if (commit2 == null) {
return commit1;
}
return (isCommit1BeforeOrOn(commit1, commit2) ? commit2 : commit1);
}
public static boolean isCommit1BeforeOrOn(String commit1, String commit2) {
return commit1.compareTo(commit2) <= 0;
}
public static boolean isCommit1After(String commit1, String commit2) {
return commit1.compareTo(commit2) > 0;
}
public List<String> getCommitList() {
return commitList;
}
public boolean isCommitBeforeEarliestCommit(String commitTs) {
return isCommit1BeforeOrOn(commitTs, firstCommit());
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieCommits that = (HoodieCommits) o;
return commitList != null ? commitList.equals(that.commitList) : that.commitList == null;
}
@Override
public int hashCode() {
return commitList != null ? commitList.hashCode() : 0;
}
}

View File

@@ -0,0 +1,71 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import java.util.Comparator;
public class HoodieDataFile {
private FileStatus fileStatus;
public HoodieDataFile(FileStatus fileStatus) {
this.fileStatus = fileStatus;
}
public String getFileId() {
return FSUtils.getFileId(fileStatus.getPath().getName());
}
public String getCommitTime() {
return FSUtils.getCommitTime(fileStatus.getPath().getName());
}
public String getPath() {
return fileStatus.getPath().toString();
}
public String getFileName() {
return fileStatus.getPath().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 fileStatus.getLen();
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieDataFile{");
sb.append("fileStatus=").append(fileStatus);
sb.append('}');
return sb.toString();
}
}

View File

@@ -1,57 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.FSUtils;
import org.apache.hadoop.fs.FileStatus;
public class HoodieFile {
private final FileStatus fileStatus;
private String fileNameWithoutCommitTs;
private String commitTs;
public HoodieFile(FileStatus fileStatus) {
this.fileStatus = fileStatus;
String fileName = fileStatus.getPath().getName();
this.fileNameWithoutCommitTs = FSUtils.getFileId(fileName);
this.commitTs = FSUtils.getCommitTime(fileName);
}
public String getFileNameWithoutCommitTs() {
return fileNameWithoutCommitTs;
}
public String getCommitTs() {
return commitTs;
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieFile{");
sb.append("fileStatus=").append(fileStatus);
sb.append(", fileNameWithoutCommitTs='").append(fileNameWithoutCommitTs).append('\'');
sb.append(", commitTs='").append(commitTs).append('\'');
sb.append('}');
return sb.toString();
}
public FileStatus getFileStatus() {
return fileStatus;
}
}

View File

@@ -0,0 +1,31 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.model;
public enum HoodieStorageType {
PARQUET(".parquet");
private final String extension;
HoodieStorageType(String extension) {
this.extension = extension;
}
public String getFileExtension() {
return extension;
}
}

View File

@@ -1,477 +0,0 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.model;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.DatasetNotFoundException;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.exception.InvalidDatasetException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.SortedMap;
import java.util.TreeMap;
/**
* Manages all file system level interactions for the Hoodie tables.
*/
public class HoodieTableMetadata implements Serializable {
public static final String MAX_COMMIT_TS = String.valueOf(Long.MAX_VALUE);
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
private static final String HOODIE_HDRONE_PROFILE_DEFAULT_VALUE = "HOODIE";
private static final java.lang.String HOODIE_HDRONE_PROFILE_PROP_NAME =
"hoodie.hdrone.dataset.profile";
private static Logger log = LogManager.getLogger(HoodieTableMetadata.class);
private transient final FileSystem fs;
private transient final Path metadataFolder;
private final Properties properties;
private HoodieCommits commits;
private List<String> inflightCommits;
private String basePath;
public static final String METAFOLDER_NAME = ".hoodie";
public static final String COMMIT_FILE_SUFFIX = ".commit";
public static final String INFLIGHT_FILE_SUFFIX = ".inflight";
/**
* Constructor which initializes the hoodie table metadata. It will initialize the meta-data if not already present.
*
* @param fs
* @param basePath
* @param tableName
*/
public HoodieTableMetadata(FileSystem fs, String basePath, String tableName) {
this(fs, basePath, tableName, true);
}
/**
* Constructor which loads the hoodie table metadata, It requires the meta-data to be present already
* @param fs
* @param basePath
*/
public HoodieTableMetadata(FileSystem fs, String basePath) {
this(fs, basePath, null, false);
}
private HoodieTableMetadata(FileSystem fs, String basePath, String tableName,
boolean initOnMissing) {
this.fs = fs;
this.basePath = basePath;
try {
Path basePathDir = new Path(this.basePath);
if (!fs.exists(basePathDir)) {
if (initOnMissing) {
fs.mkdirs(basePathDir);
} else {
throw new DatasetNotFoundException(this.basePath);
}
}
if (!fs.isDirectory(new Path(basePath))) {
throw new DatasetNotFoundException(this.basePath);
}
this.metadataFolder = new Path(this.basePath, METAFOLDER_NAME);
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
if (!fs.exists(propertyPath)) {
if (initOnMissing) {
// create .hoodie folder if it does not exist.
createHoodieProperties(metadataFolder, tableName);
} else {
throw new InvalidDatasetException(this.basePath);
}
}
// Load meta data
this.commits = new HoodieCommits(scanCommits(COMMIT_FILE_SUFFIX));
this.inflightCommits = scanCommits(INFLIGHT_FILE_SUFFIX);
this.properties = readHoodieProperties();
log.info("All commits :" + commits);
} catch (IOException e) {
throw new HoodieIOException("Could not load HoodieMetadata from path " + basePath, e);
}
}
/**
* Returns all the commit metadata for this table. Reads all the commit files from HDFS.
* Expensive operation, use with caution.
*
* @return SortedMap of CommitTime,HoodieCommitMetadata
*/
public SortedMap<String, HoodieCommitMetadata> getAllCommitMetadata() {
try {
TreeMap<String, HoodieCommitMetadata> metadataMap = new TreeMap<>();
for (String commitTs : commits.getCommitList()) {
metadataMap.put(commitTs, getCommitMetadata(commitTs));
}
return Collections.unmodifiableSortedMap(metadataMap);
} catch (IOException e) {
throw new HoodieIOException("Could not load all commits for table " + getTableName(),
e);
}
}
public HoodieCommitMetadata getCommitMetadata(String commitTime) throws IOException {
FSDataInputStream is = fs.open(new Path(metadataFolder, FSUtils.makeCommitFileName(commitTime)));
try {
String jsonStr = IOUtils.toString(is);
return HoodieCommitMetadata.fromJsonString(jsonStr);
} finally {
is.close();
}
}
public HoodieTableType getTableType() {
return HoodieTableType.valueOf(properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME));
}
/**
* Lookup the file name for specified <code>HoodieRecord</code>
*
* TODO(vc): This metadata needs to be cached in each executor, statically, and used across, if
* we need to be nicer to the NameNode
*/
public String getFilenameForRecord(FileSystem fs, final HoodieRecord record) {
String fileId = record.getCurrentLocation().getFileId();
return getFilenameForRecord(fs, record, fileId);
}
public String getFilenameForRecord(FileSystem fs, final HoodieRecord record, String fileId) {
try {
FileStatus[] files = fs.listStatus(new Path(basePath, record.getPartitionPath()));
Map<String, List<FileStatus>> fileIdToVersions =
groupFilesByFileId(files, commits.lastCommit());
// If the record is not found
if(!fileIdToVersions.containsKey(fileId)) {
throw new FileNotFoundException("Cannot find valid versions for fileId " + fileId);
}
List<FileStatus> statuses = fileIdToVersions.get(fileId);
return statuses.get(0).getPath().getName();
} catch (IOException e) {
throw new HoodieIOException(
"Could not get Filename for record " + record, e);
}
}
/**
* Get only the latest file in the partition with precondition commitTime(file) lt maxCommitTime
*
* @param fs
* @param partitionPathStr
* @param maxCommitTime
* @return
*/
public FileStatus[] getLatestVersionInPartition(FileSystem fs, String partitionPathStr,
String maxCommitTime) {
try {
Path partitionPath = new Path(basePath, partitionPathStr);
if(!fs.exists(partitionPath)) {
return new FileStatus[0];
}
FileStatus[] files = fs.listStatus(partitionPath);
Map<String, List<FileStatus>> fileIdToVersions =
groupFilesByFileId(files, commits.lastCommit());
HashMap<String, FileStatus> validFiles = new HashMap<>();
for (String fileId : fileIdToVersions.keySet()) {
List<FileStatus> versions = fileIdToVersions.get(fileId);
for (FileStatus file : versions) {
String filename = file.getPath().getName();
String commitTime = FSUtils.getCommitTime(filename);
if (HoodieCommits.isCommit1BeforeOrOn(commitTime, maxCommitTime)) {
validFiles.put(fileId, file);
break;
}
}
}
return validFiles.values().toArray(new FileStatus[validFiles.size()]);
} catch (IOException e) {
throw new HoodieIOException(
"Could not get latest versions in Partition " + partitionPathStr, e);
}
}
/**
* Get ALL the data files in partition grouped by fileId and sorted by the commitTime
* Given a partition path, provide all the files with a list of their commits, sorted by commit time.
*/
public Map<String, List<FileStatus>> getAllVersionsInPartition(FileSystem fs, String partitionPath) {
try {
FileStatus[] files = fs.listStatus(new Path(basePath, partitionPath));
return groupFilesByFileId(files, commits.lastCommit());
} catch (IOException e) {
throw new HoodieIOException(
"Could not load all file versions in partition " + partitionPath, e);
}
}
/**
* Get all the versions of files, within the commit range provided.
*
* @param commitsToReturn - commits to include
*/
public FileStatus[] getLatestVersionInRange(FileStatus[] fileStatuses, List<String> commitsToReturn) {
if (commitsToReturn.isEmpty()) {
return new FileStatus[0];
}
try {
Map<String, List<FileStatus>> fileIdToVersions =
groupFilesByFileId(fileStatuses, commits.lastCommit());
List<FileStatus> statuses = new ArrayList<>();
for (List<FileStatus> entry : fileIdToVersions.values()) {
for (FileStatus status : entry) {
String commitTime = FSUtils.getCommitTime(status.getPath().getName());
if (commitsToReturn.contains(commitTime)) {
statuses.add(status);
break;
}
}
}
return statuses.toArray(new FileStatus[statuses.size()]);
} catch (IOException e) {
throw new HoodieIOException("Could not filter files from commits " + commitsToReturn, e);
}
}
/**
*
* Get the latest versions of all the files.
*
* @param fileStatuses
* @return
*/
public FileStatus[] getLatestVersions(FileStatus[] fileStatuses) {
try {
Map<String, List<FileStatus>> fileIdToVersions =
groupFilesByFileId(fileStatuses, commits.lastCommit());
List<FileStatus> statuses = new ArrayList<>();
for(List<FileStatus> entry:fileIdToVersions.values()) {
// first file is the latest one
statuses.add(entry.get(0));
}
return statuses.toArray(new FileStatus[statuses.size()]);
} catch (IOException e) {
throw new HoodieIOException("Could not filter files for latest version ", e);
}
}
/**
* Get the base path for the Hoodie Table
*
* @return
*/
public String getBasePath() {
return basePath;
}
public boolean isCommitsEmpty() {
return commits.isEmpty();
}
public boolean isCommitTsSafe(String commitTs) {
return !isCommitsEmpty() && (commits.isCommitBeforeEarliestCommit(commitTs) || commits
.contains(commitTs));
}
public List<String> findCommitsSinceTs(String startTs) {
return commits.findCommitsInRange(startTs, MAX_COMMIT_TS);
}
public List<String> findCommitsInRange(String startTs, String endTs) {
return commits.findCommitsInRange(startTs, endTs);
}
public List<String> findCommitsAfter(String startTs, Integer maxCommits) {
return commits.findCommitsAfter(startTs, maxCommits);
}
public HoodieCommits getAllCommits() {
return commits;
}
public List<String> getAllInflightCommits() {
return inflightCommits;
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieTableMetadata{");
sb.append("commits=").append(commits);
sb.append('}');
return sb.toString();
}
public String getTableName() {
return properties.getProperty(HOODIE_TABLE_NAME_PROP_NAME);
}
public String getHDroneDatasetProfile() {
return properties.getProperty(HOODIE_HDRONE_PROFILE_PROP_NAME, HOODIE_HDRONE_PROFILE_DEFAULT_VALUE);
}
/**
* Initialize the hoodie meta directory and any necessary files inside the meta (including the hoodie.properties)
*
* @param metadataFolder
* @param tableName
* @throws IOException
*/
private void createHoodieProperties(Path metadataFolder, String tableName) throws IOException {
if (!fs.exists(metadataFolder)) {
fs.mkdirs(metadataFolder);
}
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
FSDataOutputStream outputStream = fs.create(propertyPath);
try {
Properties props = new Properties();
props.setProperty(HOODIE_TABLE_NAME_PROP_NAME, tableName);
props.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
props
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
} finally {
outputStream.close();
}
}
/**
* Loads the hoodie table properties from the hoodie.properties file under the .hoodie path
*/
private Properties readHoodieProperties() throws IOException {
Properties props = new Properties();
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
FSDataInputStream inputStream = fs.open(propertyPath);
try {
props.load(inputStream);
} finally {
inputStream.close();
}
return props;
}
/**
* Scan the commit times (only choosing commit file with the given suffix)
*/
private List<String> scanCommits(final String commitFileSuffix) throws IOException {
log.info("Attempting to load the commits under " + metadataFolder + " with suffix " + commitFileSuffix);
final List<String> commitFiles = new ArrayList<>();
fs.listStatus(metadataFolder, new PathFilter() {
@Override
public boolean accept(Path path) {
if (path.getName().endsWith(commitFileSuffix)) {
commitFiles.add(path.getName().split("\\.")[0]);
return true;
}
return false;
}
});
return commitFiles;
}
/**
* Takes a bunch of file versions, and returns a map keyed by fileId, with the necessary
* version safety checking. Returns a map of commitTime and Sorted list of FileStats
* ( by reverse commit time )
*
* @param maxCommitTime maximum permissible commit time
*
* @return
*/
private Map<String, List<FileStatus>> groupFilesByFileId(FileStatus[] files,
String maxCommitTime) throws IOException {
HashMap<String, List<FileStatus>> fileIdtoVersions = new HashMap<>();
for (FileStatus file : files) {
String filename = file.getPath().getName();
String fileId = FSUtils.getFileId(filename);
String commitTime = FSUtils.getCommitTime(filename);
if (isCommitTsSafe(commitTime) && HoodieCommits
.isCommit1BeforeOrOn(commitTime, maxCommitTime)) {
if (!fileIdtoVersions.containsKey(fileId)) {
fileIdtoVersions.put(fileId, new ArrayList<FileStatus>());
}
fileIdtoVersions.get(fileId).add(file);
}
}
for (Map.Entry<String, List<FileStatus>> entry : fileIdtoVersions.entrySet()) {
Collections.sort(fileIdtoVersions.get(entry.getKey()), new Comparator<FileStatus>() {
@Override
public int compare(FileStatus o1, FileStatus o2) {
String o1CommitTime = FSUtils.getCommitTime(o1.getPath().getName());
String o2CommitTime = FSUtils.getCommitTime(o2.getPath().getName());
// Reverse the order
return o2CommitTime.compareTo(o1CommitTime);
}
});
}
return fileIdtoVersions;
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieTableMetadata metadata = (HoodieTableMetadata) o;
if (commits != null ? !commits.equals(metadata.commits) : metadata.commits != null)
return false;
return basePath != null ? basePath.equals(metadata.basePath) : metadata.basePath == null;
}
@Override
public int hashCode() {
int result = commits != null ? commits.hashCode() : 0;
result = 31 * result + (basePath != null ? basePath.hashCode() : 0);
return result;
}
}

View File

@@ -0,0 +1,138 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.model.HoodieStorageType;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.io.Serializable;
import java.util.Date;
import java.util.Properties;
/**
* Configurations on the Hoodie Table like type of ingestion, storage formats, hive table name etc
* Configurations are loaded from hoodie.properties, these properties are usually set during initializing a path as hoodie base path
* and never changes during the lifetime of a hoodie dataset.
*
* @see HoodieTableMetaClient
* @since 0.3.0
*/
public class HoodieTableConfig implements Serializable {
private final transient static Logger log = LogManager.getLogger(HoodieTableConfig.class);
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
public static final String HOODIE_RO_STORAGE_FORMAT_PROP_NAME =
"hoodie.table.ro.storage.format";
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
public static final HoodieStorageType DEFAULT_RO_STORAGE_FORMAT = HoodieStorageType.PARQUET;
private Properties props;
public HoodieTableConfig(FileSystem fs, String metaPath) {
Properties props = new Properties();
Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
log.info("Loading dataset properties from " + propertyPath);
try {
try (FSDataInputStream inputStream = fs.open(propertyPath)) {
props.load(inputStream);
}
} catch (IOException e) {
throw new HoodieIOException("Could not load Hoodie properties from " + propertyPath, e);
}
this.props = props;
}
/**
* For serailizing and de-serializing
* @deprecated
*/
public HoodieTableConfig() {
}
/**
* Initialize the hoodie meta directory and any necessary files inside the meta (including the hoodie.properties)
*
* @param metadataFolder
* @param properties
* @throws IOException
*/
public static void createHoodieProperties(FileSystem fs, Path metadataFolder,
Properties properties) throws IOException {
if (!fs.exists(metadataFolder)) {
fs.mkdirs(metadataFolder);
}
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
FSDataOutputStream outputStream = fs.create(propertyPath);
try {
if (!properties.containsKey(HOODIE_TABLE_NAME_PROP_NAME)) {
throw new IllegalArgumentException(
HOODIE_TABLE_NAME_PROP_NAME + " property needs to be specified");
}
if (!properties.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
properties.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
}
properties
.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
} finally {
outputStream.close();
}
}
/**
* Read the table type from the table properties and if not found, return the default
*
* @return
*/
public HoodieTableType getTableType() {
if (props.contains(HOODIE_TABLE_TYPE_PROP_NAME)) {
return HoodieTableType.valueOf(props.getProperty(HOODIE_TABLE_TYPE_PROP_NAME));
}
return DEFAULT_TABLE_TYPE;
}
/**
* Read the table name
*
* @return
*/
public String getTableName() {
return props.getProperty(HOODIE_TABLE_NAME_PROP_NAME);
}
/**
* Get the Read Optimized Storage Format
*
* @return HoodieStorageType for the Read Optimized Storage format
*/
public HoodieStorageType getROStorageFormat() {
if (props.contains(HOODIE_RO_STORAGE_FORMAT_PROP_NAME)) {
return HoodieStorageType.valueOf(props.getProperty(HOODIE_RO_STORAGE_FORMAT_PROP_NAME));
}
return DEFAULT_RO_STORAGE_FORMAT;
}
}

View File

@@ -0,0 +1,275 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.common.table.timeline.HoodieActiveCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieArchivedCommitTimeline;
import com.uber.hoodie.common.table.timeline.HoodieCleanerTimeline;
import com.uber.hoodie.common.table.timeline.HoodieSavePointTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.DatasetNotFoundException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.PathFilter;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.File;
import java.io.IOException;
import java.io.Serializable;
import java.util.Objects;
import java.util.Properties;
/**
* <code>HoodieTableMetaClient</code> allows to access meta-data about a hoodie table
* It returns meta-data about commits, savepoints, compactions, cleanups as a <code>HoodieTimeline</code>
* Create an instance of the <code>HoodieTableMetaClient</code> with FileSystem and basePath to start getting the meta-data.
* <p>
* All the timelines are computed lazily, once computed the timeline is cached and never refreshed.
* Use the <code>HoodieTimeline.reload()</code> to refresh timelines.
*
* @see HoodieTimeline
* @since 0.3.0
*/
public class HoodieTableMetaClient implements Serializable {
private final transient static Logger log = LogManager.getLogger(HoodieTableMetaClient.class);
public static String METAFOLDER_NAME = ".hoodie";
public static String COMMIT_EXTENSION = ".commit";
public static String CLEAN_EXTENSION = ".clean";
public static String SAVEPOINT_EXTENSION = ".savepoint";
public static String INFLIGHT_FILE_SUFFIX = ".inflight";
private String basePath;
private transient FileSystem fs;
private String metaPath;
private HoodieTableType tableType;
private HoodieTableConfig tableConfig;
private HoodieTimeline activeCommitTimeline;
private HoodieTimeline archivedCommitTimeline;
private HoodieTimeline savePointTimeline;
private HoodieTimeline cleanerTimeline;
public HoodieTableMetaClient(FileSystem fs, String basePath) throws DatasetNotFoundException {
// Do not load any timeline by default
this(fs, basePath, false);
}
public HoodieTableMetaClient(FileSystem fs, String basePath,
boolean loadActiveCommitTimelineOnLoad) throws DatasetNotFoundException {
log.info("Loading HoodieTableMetaClient from " + basePath);
this.basePath = basePath;
this.fs = fs;
Path basePathDir = new Path(this.basePath);
this.metaPath = basePath + File.separator + METAFOLDER_NAME;
Path metaPathDir = new Path(this.metaPath);
DatasetNotFoundException.checkValidDataset(fs, basePathDir, metaPathDir);
this.tableConfig = new HoodieTableConfig(fs, metaPath);
this.tableType = tableConfig.getTableType();
log.info("Finished Loading Table of type " + tableType + " from " + basePath);
if (loadActiveCommitTimelineOnLoad) {
log.info("Loading Active commit timeline for " + basePath);
getActiveCommitTimeline();
}
}
/**
* For serailizing and de-serializing
* @deprecated
*/
public HoodieTableMetaClient() {
}
/**
* This method is only used when this object is deserialized in a spark executor.
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
/**
* @return Base path
*/
public String getBasePath() {
return basePath;
}
/**
* @return Hoodie Table Type
*/
public HoodieTableType getTableType() {
return tableType;
}
/**
* @return Meta path
*/
public String getMetaPath() {
return metaPath;
}
/**
* @return Table Config
*/
public HoodieTableConfig getTableConfig() {
return tableConfig;
}
/**
* Get the active commits as a timeline
*
* @return Active commit timeline
* @throws IOException
*/
public synchronized HoodieTimeline getActiveCommitTimeline() {
if (activeCommitTimeline == null) {
activeCommitTimeline = new HoodieActiveCommitTimeline(fs, metaPath);
}
return activeCommitTimeline;
}
/**
* 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
*
* @return Active commit timeline
* @throws IOException
*/
public HoodieTimeline getArchivedCommitTimeline() {
if (archivedCommitTimeline == null) {
archivedCommitTimeline = new HoodieArchivedCommitTimeline(fs, metaPath);
}
return archivedCommitTimeline;
}
/**
* Get the save points as a timeline.
*
* @return Savepoint timeline
* @throws IOException
*/
public HoodieTimeline getSavePointsTimeline() {
if (savePointTimeline == null) {
savePointTimeline = new HoodieSavePointTimeline(fs, metaPath);
}
return savePointTimeline;
}
/**
* Get the cleaner activity as a timeline.
*
* @return Cleaner activity
* @throws IOException
*/
public HoodieTimeline getCleanerTimeline() {
if (cleanerTimeline == null) {
cleanerTimeline = new HoodieCleanerTimeline(fs, metaPath);
}
return cleanerTimeline;
}
/**
* Helper method to initialize a given path as a hoodie dataset with configs passed in as as Properties
*
* @param fs
* @param basePath
* @param props
* @return Instance of HoodieTableMetaClient
* @throws IOException
*/
public static HoodieTableMetaClient initializePathAsHoodieDataset(FileSystem fs,
String basePath, Properties props) throws IOException {
log.info("Initializing " + basePath + " as hoodie dataset " + basePath);
Path basePathDir = new Path(basePath);
if (!fs.exists(basePathDir)) {
fs.mkdirs(basePathDir);
}
Path metaPathDir = new Path(basePath, METAFOLDER_NAME);
if (!fs.exists(metaPathDir)) {
fs.mkdirs(metaPathDir);
}
HoodieTableConfig.createHoodieProperties(fs, metaPathDir, props);
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs, basePath);
log.info("Finished initializing Table of type " + metaClient.getTableConfig().getTableType()
+ " from " + basePath);
return metaClient;
}
// HELPER METHODS TO CREATE META FILE NAMES
public static String makeCommitFileName(String commitTime) {
return commitTime + COMMIT_EXTENSION;
}
public static String makeInflightCommitFileName(String commitTime) {
return commitTime + INFLIGHT_FILE_SUFFIX;
}
public static String makeCleanerFileName(String instant) {
return instant + CLEAN_EXTENSION;
}
public static String makeInflightCleanerFileName(String instant) {
return instant + CLEAN_EXTENSION + INFLIGHT_FILE_SUFFIX;
}
public static String makeInflightSavePointFileName(String commitTime) {
return commitTime + SAVEPOINT_EXTENSION + INFLIGHT_FILE_SUFFIX;
}
public static String makeSavePointFileName(String commitTime) {
return commitTime + SAVEPOINT_EXTENSION;
}
public static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
}
public static FileStatus[] scanFiles(FileSystem fs, Path metaPath, PathFilter nameFilter)
throws IOException {
return fs.listStatus(metaPath, nameFilter);
}
@Override
public boolean equals(Object o) {
if (this == o)
return true;
if (o == null || getClass() != o.getClass())
return false;
HoodieTableMetaClient that = (HoodieTableMetaClient) o;
return Objects.equals(basePath, that.basePath) && tableType == that.tableType;
}
@Override
public int hashCode() {
return Objects.hash(basePath, tableType);
}
@Override
public String toString() {
final StringBuilder sb = new StringBuilder("HoodieTableMetaClient{");
sb.append("basePath='").append(basePath).append('\'');
sb.append(", metaPath='").append(metaPath).append('\'');
sb.append(", tableType=").append(tableType);
sb.append('}');
return sb.toString();
}
}

View File

@@ -0,0 +1,190 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.table.timeline.HoodieDefaultTimeline;
import java.io.IOException;
import java.io.Serializable;
import java.util.Optional;
import java.util.function.BiPredicate;
import java.util.stream.Stream;
/**
* HoodieTimeline allows representation of meta-data events as a timeline.
* Instants are specific points in time represented as strings.
* in this format YYYYMMDDHHmmSS. e.g. 20170101193218
* Any operation on the timeline starts with the inflight instant and then when complete marks
* the completed instant and removes the inflight instant.
* Completed instants are plainly referred to as just instants
* <p>
* Timelines as immutable once created. Any operation to change the timeline (like create/delete instants)
* will not be reflected unless explicitly reloaded using the reload()
*
* @see com.uber.hoodie.common.table.HoodieTableMetaClient
* @see HoodieDefaultTimeline
* @since 0.3.0
*/
public interface HoodieTimeline extends Serializable {
/**
* Find all the completed instants after startTs and before or on endTs
*
* @param startTs
* @param endTs
* @return Stream of instants
*/
Stream<String> findInstantsInRange(String startTs, String endTs);
/**
* Find all the completed instants after startTs
*
* @param commitTime
* @param numCommits
* @return Stream of instants
*/
Stream<String> findInstantsAfter(String commitTime, int numCommits);
/**
* If the timeline has any completed instants
*
* @return true if timeline is not empty
*/
boolean hasInstants();
/**
* If the timeline has any in-complete instants
*
* @return true if timeline has any in-complete instants
*/
boolean hasInflightInstants();
/**
* @return total number of completed instants
*/
int getTotalInstants();
/**
* @return first completed instant if available
*/
Optional<String> firstInstant();
/**
* @param n
* @return nth completed instant from the first completed instant
*/
Optional<String> nthInstant(int n);
/**
* @return last completed instant if available
*/
Optional<String> lastInstant();
/**
* @param n
* @return nth completed instant going back from the last completed instant
*/
Optional<String> nthFromLastInstant(int n);
/**
* @return true if the passed instant is present as a completed instant on the timeline
*/
boolean containsInstant(String instant);
/**
* @return true if the passed instant is present as a completed instant on the timeline or
* if the instant is before the first completed instant in the timeline
*/
boolean containsOrBeforeTimelineStarts(String instant);
/**
* @return Get the stream of completed instants
*/
Stream<String> getInstants();
/**
* @return Get the stream of in-flight instants
*/
Stream<String> getInflightInstants();
/**
* @return true if the passed in instant is before the first completed instant in the timeline
*/
boolean isInstantBeforeTimelineStarts(String instant);
/**
* Register the passed in instant as a in-flight
*
* @param instant
*/
void saveInstantAsInflight(String instant);
/**
* Register the passed in instant as a completed instant.
* It needs to have a corresponding in-flight instant, otherwise it will fail.
* Pass a optional byte[] to save with the instant.
*
* @param instant
* @param data
*/
void saveInstantAsComplete(String instant, Optional<byte[]> data);
/**
* Un-Register a completed instant as in-flight. This is usually atomic way to
* revert the effects of a operation on hoodie datasets
*
* @param instant
*/
void revertInstantToInflight(String instant);
/**
* Remove the in-flight instant from the timeline
*
* @param instant
*/
void removeInflightFromTimeline(String instant);
/**
* Reload the timeline. Timelines are immutable once created.
*
* @return
* @throws IOException
*/
HoodieTimeline reload() throws IOException;
/**
* Read the completed instant details
*
* @param instant
* @return
*/
Optional<byte[]> readInstantDetails(String instant);
/**
* Helper methods to compare instants
**/
BiPredicate<String, String> GREATER_OR_EQUAL =
(commit1, commit2) -> commit1.compareTo(commit2) >= 0;
BiPredicate<String, String> GREATER = (commit1, commit2) -> commit1.compareTo(commit2) > 0;
BiPredicate<String, String> LESSER_OR_EQUAL =
(commit1, commit2) -> commit1.compareTo(commit2) <= 0;
BiPredicate<String, String> LESSER = (commit1, commit2) -> commit1.compareTo(commit2) < 0;
default boolean compareInstants(String commit1, String commit2,
BiPredicate<String, String> predicateToApply) {
return predicateToApply.test(commit1, commit2);
}
}

View File

@@ -0,0 +1,93 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieRecord;
import org.apache.hadoop.fs.FileStatus;
import java.util.List;
import java.util.stream.Stream;
/**
* Interface for viewing the table file system.
* Dependening on the Hoodie Table Type - The view of the filesystem changes.
* <p>
* ReadOptimizedView - Lets queries run only on organized columnar data files at the expense of latency
* WriteOptimizedView - Lets queries run on columnar data as well as delta files (sequential) at the expense of query execution time
* @since 0.3.0
*/
public interface TableFileSystemView {
/**
* Stream all the data files for a specific FileId.
* This usually has a single RO file and multiple WO files if present.
*
* @param partitionPath
* @param fileId
* @return
*/
Stream<HoodieDataFile> getLatestDataFilesForFileId(final String partitionPath,
final String fileId);
/**
* Stream all the latest version data files in the given partition
* with precondition that commitTime(file) before maxCommitTime
*
* @param partitionPathStr
* @param maxCommitTime
* @return
*/
Stream<HoodieDataFile> streamLatestVersionInPartition(String partitionPathStr,
String maxCommitTime);
/**
* Stream all the data file versions grouped by FileId for a given partition
*
* @param partitionPath
* @return
*/
Stream<List<HoodieDataFile>> streamEveryVersionInPartition(String partitionPath);
/**
* Stream all the versions from the passed in fileStatus[] with commit times containing in commitsToReturn.
*
* @param fileStatuses
* @param commitsToReturn
* @return
*/
Stream<HoodieDataFile> streamLatestVersionInRange(FileStatus[] fileStatuses,
List<String> commitsToReturn);
/**
* Stream the latest version from the passed in FileStatus[] with commit times less than maxCommitToReturn
*
* @param fileStatuses
* @param maxCommitToReturn
* @return
*/
Stream<HoodieDataFile> streamLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
String maxCommitToReturn);
/**
* Stream latest versions from the passed in FileStatus[].
* Similar to calling streamLatestVersionsBeforeOrOn(fileStatuses, currentTimeAsCommitTime)
*
* @param fileStatuses
* @return
*/
Stream<HoodieDataFile> streamLatestVersions(FileStatus[] fileStatuses);
}

View File

@@ -0,0 +1,77 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.util.Arrays;
import java.util.stream.Collectors;
/**
* Active commit timeline. Much optimized for reading than the ArchivedTimeline.
*/
public class HoodieActiveCommitTimeline extends HoodieDefaultTimeline {
public HoodieActiveCommitTimeline(FileSystem fs, String metaPath) {
super(fs, metaPath);
String completedInstantExtension = HoodieTableMetaClient.COMMIT_EXTENSION;
String inflightInstantExtension = INFLIGHT_EXTENSION;
FileStatus[] fileStatuses;
try {
fileStatuses = HoodieTableMetaClient.scanFiles(fs, new Path(metaPath),
path -> path.toString().endsWith(completedInstantExtension) || path.toString()
.endsWith(inflightInstantExtension));
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.instants = Arrays.stream(fileStatuses)
.filter(status -> status.getPath().getName().endsWith(completedInstantExtension))
.map(fileStatus -> fileStatus.getPath().getName().replaceAll(completedInstantExtension, ""))
.sorted().collect(Collectors.toList());
this.inflights = Arrays.stream(fileStatuses).filter(
status -> status.getPath().getName().endsWith(inflightInstantExtension)).map(
fileStatus -> fileStatus.getPath().getName()
.replaceAll(inflightInstantExtension, "")).sorted()
.collect(Collectors.toList());
}
@Override
public String getInflightFileName(String instant) {
return HoodieTableMetaClient.makeInflightCommitFileName(instant);
}
@Override
public String getCompletedFileName(String instant) {
return HoodieTableMetaClient.makeCommitFileName(instant);
}
@Override
protected String getTimelineName() {
return "commit";
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieActiveCommitTimeline(fs, metaPath);
}
}

View File

@@ -0,0 +1,121 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.timeline;
import com.google.common.io.Closeables;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.SequenceFile;
import org.apache.hadoop.io.Text;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
/**
* Archived commit timeline. These commits are usually cleaned up and the meta data is archived for
* future triaging
*
* @since 0.3.0
*/
public class HoodieArchivedCommitTimeline extends HoodieDefaultTimeline {
private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE = "commits.archived";
private transient Map<String, byte[]> readCommits = new HashMap<>();
public HoodieArchivedCommitTimeline(FileSystem fs, String metaPath) {
// Read back the commits to make sure
Path archiveLogPath = getArchiveLogPath(metaPath);
try {
SequenceFile.Reader reader =
new SequenceFile.Reader(fs.getConf(), SequenceFile.Reader.file(archiveLogPath));
try {
Text key = new Text();
Text val = new Text();
while (reader.next(key, val)) {
// TODO - limit the number of commits loaded in memory. this could get very large.
// 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 = new ArrayList<>(readCommits.keySet());
this.inflights = new ArrayList<>(0);
} finally {
Closeables.closeQuietly(reader);
}
} catch (IOException e) {
throw new HoodieIOException(
"Could not load archived commit timeline from path " + archiveLogPath, e);
}
}
@Override
public void saveInstantAsInflight(String instant) {
throw new UnsupportedOperationException(
"Could not save inflight instant in ArchivedTimeline " + instant);
}
@Override
public void saveInstantAsComplete(String instant, Optional<byte[]> data) {
throw new UnsupportedOperationException(
"Could not save instant as complete in ArchivedTimeline " + instant);
}
@Override
public void revertInstantToInflight(String instant) {
throw new UnsupportedOperationException(
"Could not revert instant in ArchivedTimeline " + instant);
}
@Override
public void removeInflightFromTimeline(String instant) {
throw new UnsupportedOperationException(
"Could not delete inflight instant from ArchivedTimeline " + instant);
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieArchivedCommitTimeline(fs, metaPath);
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
return Optional.ofNullable(readCommits.get(instant));
}
@Override
protected String getInflightFileName(String instant) {
throw new UnsupportedOperationException("No inflight filename for archived commits");
}
@Override
protected String getCompletedFileName(String instant) {
throw new UnsupportedOperationException("No inflight filename for archived commits");
}
@Override
protected String getTimelineName() {
return "archived-commits";
}
public static Path getArchiveLogPath(String metaPath) {
return new Path(metaPath, HOODIE_COMMIT_ARCHIVE_LOG_FILE);
}
}

View File

@@ -0,0 +1,57 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.util.Optional;
public class HoodieCleanerTimeline extends HoodieDefaultTimeline {
public HoodieCleanerTimeline(FileSystem fs, String path) {
super(fs, path, HoodieTableMetaClient.CLEAN_EXTENSION);
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieCleanerTimeline(fs, metaPath);
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
// TODO - Nothing about the clean written today - this should change
return Optional.empty();
}
@Override
protected String getInflightFileName(String instant) {
return HoodieTableMetaClient.makeInflightCleanerFileName(instant);
}
@Override
protected String getCompletedFileName(String instant) {
return HoodieTableMetaClient.makeCleanerFileName(instant);
}
@Override
protected String getTimelineName() {
return "cleaner";
}
}

View File

@@ -0,0 +1,348 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.timeline;
import com.google.common.io.Closeables;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* HoodieTimeline allows representation of meta-data events as a timeline.
* Instants are specific points in time represented as strings.
* in this format YYYYMMDDHHmmSS. e.g. 20170101193218
* Any operation on the timeline starts with the inflight instant and then when complete marks
* the completed instant and removes the inflight instant.
* Completed instants are plainly referred to as just instants
* <p>
* Timelines as immutable once created. Any operation to change the timeline (like create/delete instants)
* will not be reflected unless explicitly reloaded using the reload()
*
* @see com.uber.hoodie.common.table.HoodieTableMetaClient
* @see HoodieTimeline
* @since 0.3.0
*/
public abstract class HoodieDefaultTimeline implements HoodieTimeline {
private final transient static Logger log = LogManager.getLogger(HoodieDefaultTimeline.class);
public static final String INFLIGHT_EXTENSION = ".inflight";
protected String metaPath;
protected transient FileSystem fs;
protected List<String> inflights;
protected List<String> instants;
public HoodieDefaultTimeline(FileSystem fs, String metaPath, String fileExtension) {
String completedInstantExtension = fileExtension;
String inflightInstantExtension = fileExtension + INFLIGHT_EXTENSION;
FileStatus[] fileStatuses;
try {
fileStatuses = HoodieTableMetaClient.scanFiles(fs, new Path(metaPath),
path -> path.toString().endsWith(completedInstantExtension) || path.toString()
.endsWith(inflightInstantExtension));
} catch (IOException e) {
throw new HoodieIOException("Failed to scan metadata", e);
}
this.instants = Arrays.stream(fileStatuses)
.filter(status -> status.getPath().getName().endsWith(completedInstantExtension))
.map(fileStatus -> fileStatus.getPath().getName().replaceAll(completedInstantExtension, ""))
.sorted().collect(Collectors.toList());
this.inflights = Arrays.stream(fileStatuses).filter(
status -> status.getPath().getName().endsWith(inflightInstantExtension)).map(
fileStatus -> fileStatus.getPath().getName()
.replaceAll(inflightInstantExtension, "")).sorted()
.collect(Collectors.toList());
this.fs = fs;
this.metaPath = metaPath;
}
public HoodieDefaultTimeline(Stream<String> instants, Stream<String> inflights) {
this.instants = instants.collect(Collectors.toList());
this.inflights = inflights.collect(Collectors.toList());
}
/**
* This constructor only supports backwards compatibility in inflight commits in ActiveCommitTimeline.
* This should never be used.
*
* @param fs
* @param metaPath
* @deprecated
*/
public HoodieDefaultTimeline(FileSystem fs, String metaPath) {
this.fs = fs;
this.metaPath = metaPath;
}
/**
* For serailizing and de-serializing
* @deprecated
*/
public HoodieDefaultTimeline() {
}
/**
* This method is only used when this object is deserialized in a spark executor.
* @deprecated
*/
private void readObject(java.io.ObjectInputStream in)
throws IOException, ClassNotFoundException {
in.defaultReadObject();
this.fs = FSUtils.getFs();
}
@Override
public Stream<String> findInstantsInRange(String startTs, String endTs) {
return instants.stream().filter(
s -> compareInstants(s, startTs, GREATER) && compareInstants(s, endTs,
LESSER_OR_EQUAL));
}
@Override
public Stream<String> findInstantsAfter(String commitTime, int numCommits) {
return instants.stream().filter(s -> compareInstants(s, commitTime, GREATER))
.limit(numCommits);
}
@Override
public boolean hasInstants() {
return instants.stream().count() != 0;
}
@Override
public boolean hasInflightInstants() {
return inflights.stream().count() != 0;
}
@Override
public int getTotalInstants() {
return new Long(instants.stream().count()).intValue();
}
@Override
public Optional<String> firstInstant() {
return instants.stream().findFirst();
}
@Override
public Optional<String> nthInstant(int n) {
if(!hasInstants() || n >= getTotalInstants()) {
return Optional.empty();
}
return Optional.of(instants.get(n));
}
@Override
public Optional<String> lastInstant() {
return hasInstants() ? nthInstant(getTotalInstants() - 1) : Optional.empty();
}
@Override
public Optional<String> nthFromLastInstant(int n) {
if(getTotalInstants() < n + 1) {
return Optional.empty();
}
return nthInstant(getTotalInstants() - 1 - n);
}
@Override
public boolean containsInstant(String instant) {
return instants.stream().anyMatch(s -> s.equals(instant));
}
@Override
public boolean containsOrBeforeTimelineStarts(String instant) {
return containsInstant(instant) || isInstantBeforeTimelineStarts(instant);
}
@Override
public Stream<String> getInstants() {
return instants.stream();
}
@Override
public Stream<String> getInflightInstants() {
return inflights.stream();
}
@Override
public boolean isInstantBeforeTimelineStarts(String instant) {
Optional<String> firstCommit = firstInstant();
return firstCommit.isPresent() && compareInstants(instant, firstCommit.get(), LESSER);
}
@Override
public void saveInstantAsInflight(String instant) {
log.info("Creating a new in-flight " + getTimelineName() + " " + instant);
// Create the in-flight file
createFileInMetaPath(getInflightFileName(instant), Optional.empty());
}
@Override
public void saveInstantAsComplete(String instant, Optional<byte[]> data) {
log.info("Marking complete " + getTimelineName() + " " + instant);
moveInflightToComplete(instant, data, getCompletedFileName(instant),
HoodieTableMetaClient.makeInflightCommitFileName(instant));
log.info("Completed " + getTimelineName() + " " + instant);
}
@Override
public void revertInstantToInflight(String instant) {
log.info("Reverting instant to inflight " + getTimelineName() + " " + instant);
moveCompleteToInflight(instant, getCompletedFileName(instant),
getInflightFileName(instant));
log.info("Reverted " + getTimelineName() + " " + instant + " to inflight");
}
@Override
public void removeInflightFromTimeline(String instant) {
log.info("Removing in-flight " + getTimelineName() + " " + instant);
String inFlightCommitFileName = getInflightFileName(instant);
Path inFlightCommitFilePath = new Path(metaPath, inFlightCommitFileName);
try {
fs.delete(inFlightCommitFilePath, false);
log.info("Removed in-flight " + getTimelineName() + " " + instant);
} catch (IOException e) {
throw new HoodieIOException(
"Could not remove inflight commit " + inFlightCommitFilePath, e);
}
}
@Override
public Optional<byte[]> readInstantDetails(String instant) {
Path detailPath = new Path(metaPath, getCompletedFileName(instant));
return readDataFromPath(detailPath);
}
/**
* Get the in-flight instant file name
*
* @param instant
* @return
*/
protected abstract String getInflightFileName(String instant);
/**
* Get the completed instant file name
*
* @param instant
* @return
*/
protected abstract String getCompletedFileName(String instant);
/**
* Get the timeline name
*
* @return
*/
protected abstract String getTimelineName();
protected void moveInflightToComplete(String instant, Optional<byte[]> data,
String commitFileName, String inflightFileName) {
Path commitFilePath = new Path(metaPath, commitFileName);
try {
// open a new file and write the commit metadata in
Path inflightCommitFile = new Path(metaPath, inflightFileName);
createFileInMetaPath(inflightFileName, data);
boolean success = fs.rename(inflightCommitFile, commitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + inflightCommitFile + " to " + commitFilePath);
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete commit " + instant, e);
}
}
protected void moveCompleteToInflight(String instant, String commitFileName,
String inflightFileName) {
Path inFlightCommitFilePath = new Path(metaPath, inflightFileName);
try {
if (!fs.exists(inFlightCommitFilePath)) {
Path commitFilePath = new Path(metaPath, commitFileName);
boolean success = fs.rename(commitFilePath, inFlightCommitFilePath);
if (!success) {
throw new HoodieIOException(
"Could not rename " + commitFilePath + " to " + inFlightCommitFilePath);
}
}
} catch (IOException e) {
throw new HoodieIOException("Could not complete commit revert " + instant, e);
}
}
protected void createFileInMetaPath(String filename, Optional<byte[]> content) {
Path fullPath = new Path(metaPath, filename);
try {
if (!content.isPresent()) {
if (fs.createNewFile(fullPath)) {
log.info("Created a new file in meta path: " + fullPath);
return;
}
} else {
FSDataOutputStream fsout = fs.create(fullPath, true);
fsout.write(content.get());
fsout.close();
return;
}
throw new HoodieIOException("Failed to create file " + fullPath);
} catch (IOException e) {
throw new HoodieIOException("Failed to create file " + fullPath, e);
}
}
protected Optional<byte[]> readDataFromPath(Path detailPath) {
FSDataInputStream is = null;
try {
is = fs.open(detailPath);
return Optional.of(IOUtils.toByteArray(is));
} catch (IOException e) {
throw new HoodieIOException("Could not read commit details from " + detailPath, e);
} finally {
if (is != null) {
Closeables.closeQuietly(is);
}
}
}
@Override
public String toString() {
return this.getClass().getName() + ": " + instants.stream().map(Object::toString)
.collect(Collectors.joining(","));
}
}

View File

@@ -0,0 +1,50 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.timeline;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
public class HoodieSavePointTimeline extends HoodieDefaultTimeline {
public HoodieSavePointTimeline(FileSystem fs, String metaPath) {
super(fs, metaPath, HoodieTableMetaClient.SAVEPOINT_EXTENSION);
}
@Override
public HoodieTimeline reload() throws IOException {
return new HoodieSavePointTimeline(fs, metaPath);
}
@Override
protected String getInflightFileName(String instant) {
return HoodieTableMetaClient.makeInflightSavePointFileName(instant);
}
@Override
protected String getCompletedFileName(String instant) {
return HoodieTableMetaClient.makeSavePointFileName(instant);
}
@Override
protected String getTimelineName() {
return "savepoint";
}
}

View File

@@ -0,0 +1,192 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.view;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.function.Function;
import java.util.stream.Collector;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* Common abstract implementation for multiple TableFileSystemView Implementations.
* 2 possible implementations are ReadOptimizedView and RealtimeView
*
* Concrete implementations extending this abstract class, should only implement
* listDataFilesInPartition which includes files to be included in the view
*
* @see TableFileSystemView
* @see ReadOptimizedTableView
* @since 0.3.0
*/
public abstract class AbstractTableFileSystemView implements TableFileSystemView {
protected final HoodieTableMetaClient metaClient;
protected final transient FileSystem fs;
protected final HoodieTimeline activeCommitTimeline;
public AbstractTableFileSystemView(FileSystem fs, HoodieTableMetaClient metaClient) {
this.metaClient = metaClient;
this.fs = fs;
this.activeCommitTimeline = metaClient.getActiveCommitTimeline();
}
public Stream<HoodieDataFile> getLatestDataFilesForFileId(final String partitionPath,
String fileId) {
Optional<String> lastInstant = activeCommitTimeline.lastInstant();
if (lastInstant.isPresent()) {
return streamLatestVersionInPartition(partitionPath, lastInstant.get())
.filter(hoodieDataFile -> hoodieDataFile.getFileId().equals(fileId));
}
return Stream.empty();
}
@Override
public Stream<HoodieDataFile> streamLatestVersionInPartition(String partitionPathStr,
String maxCommitTime) {
try {
return streamLatestVersionsBeforeOrOn(listDataFilesInPartition(partitionPathStr),
maxCommitTime);
} catch (IOException e) {
throw new HoodieIOException(
"Could not get latest versions in Partition " + partitionPathStr, e);
}
}
@Override
public Stream<List<HoodieDataFile>> streamEveryVersionInPartition(String partitionPath) {
try {
if(activeCommitTimeline.lastInstant().isPresent()) {
return streamFilesByFileId(listDataFilesInPartition(partitionPath),
activeCommitTimeline.lastInstant().get());
}
return Stream.empty();
} catch (IOException e) {
throw new HoodieIOException(
"Could not load all file versions in partition " + partitionPath, e);
}
}
protected abstract FileStatus[] listDataFilesInPartition(String partitionPathStr)
throws IOException;
@Override
public Stream<HoodieDataFile> streamLatestVersionInRange(FileStatus[] fileStatuses,
List<String> commitsToReturn) {
if (!activeCommitTimeline.hasInstants() || commitsToReturn.isEmpty()) {
return Stream.empty();
}
try {
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
for (HoodieDataFile fs : fss) {
if (commitsToReturn.contains(fs.getCommitTime())) {
return Optional.of(fs);
}
}
return Optional.empty();
}).filter(Optional::isPresent).map(Optional::get);
} catch (IOException e) {
throw new HoodieIOException("Could not filter files from commits " + commitsToReturn,
e);
}
}
@Override
public Stream<HoodieDataFile> streamLatestVersionsBeforeOrOn(FileStatus[] fileStatuses,
String maxCommitToReturn) {
try {
if (!activeCommitTimeline.hasInstants()) {
return Stream.empty();
}
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
.map((Function<List<HoodieDataFile>, Optional<HoodieDataFile>>) fss -> {
for (HoodieDataFile fs1 : fss) {
if (activeCommitTimeline
.compareInstants(fs1.getCommitTime(), maxCommitToReturn,
HoodieTimeline.LESSER_OR_EQUAL)) {
return Optional.of(fs1);
}
}
return Optional.empty();
}).filter(Optional::isPresent).map(Optional::get);
} catch (IOException e) {
throw new HoodieIOException("Could not filter files for latest version ", e);
}
}
@Override
public Stream<HoodieDataFile> streamLatestVersions(FileStatus[] fileStatuses) {
try {
if (!activeCommitTimeline.hasInstants()) {
return Stream.empty();
}
return streamFilesByFileId(fileStatuses, activeCommitTimeline.lastInstant().get())
.map(statuses -> statuses.get(0));
} catch (IOException e) {
throw new HoodieIOException("Could not filter files for latest version ", e);
}
}
protected Stream<List<HoodieDataFile>> streamFilesByFileId(FileStatus[] files,
String maxCommitTime) throws IOException {
return groupFilesByFileId(files, maxCommitTime).values().stream();
}
/**
* Filters the list of FileStatus to exclude non-committed data files and group by FileID
* and sort the actial files by commit time (newer commit first)
*
* @param files Files to filter and group from
* @param maxCommitTime maximum permissible commit time
* @return Grouped map by fileId
*/
private Map<String, List<HoodieDataFile>> groupFilesByFileId(FileStatus[] files,
String maxCommitTime) throws IOException {
return Arrays.stream(files).flatMap(fileStatus -> {
HoodieDataFile dataFile = new HoodieDataFile(fileStatus);
if (activeCommitTimeline.containsOrBeforeTimelineStarts(dataFile.getCommitTime())
&& activeCommitTimeline.compareInstants(dataFile.getCommitTime(), maxCommitTime,
HoodieTimeline.LESSER_OR_EQUAL)) {
return Stream.of(Pair.of(dataFile.getFileId(), dataFile));
}
return Stream.empty();
}).collect(Collectors
.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, toSortedFileStatus())));
}
private Collector<HoodieDataFile, ?, List<HoodieDataFile>> toSortedFileStatus() {
return Collectors.collectingAndThen(Collectors.toList(),
l -> l.stream().sorted(HoodieDataFile.getCommitTimeComparator())
.collect(Collectors.toList()));
}
}

View File

@@ -0,0 +1,47 @@
/*
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.table.view;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
/**
* ReadOptimized view which includes only the ROStorageformat files
*/
public class ReadOptimizedTableView extends AbstractTableFileSystemView {
public ReadOptimizedTableView(FileSystem fs, HoodieTableMetaClient metaClient) {
super(fs, metaClient);
}
protected FileStatus[] listDataFilesInPartition(String partitionPathStr) {
Path partitionPath = new Path(metaClient.getBasePath(), partitionPathStr);
try {
return fs.listStatus(partitionPath, path -> path.getName()
.contains(metaClient.getTableConfig().getROStorageFormat().getFileExtension()));
} catch (IOException e) {
throw new HoodieIOException(
"Failed to list data files in partition " + partitionPathStr, e);
}
}
}

View File

@@ -16,7 +16,6 @@
package com.uber.hoodie.common.util;
import com.uber.hoodie.common.model.HoodieTableMetadata;
import com.uber.hoodie.exception.HoodieIOException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
@@ -68,14 +67,6 @@ public class FSUtils {
return String.format("*_*_%s.parquet", commitTime);
}
public static String makeInflightCommitFileName(String commitTime) {
return commitTime + HoodieTableMetadata.INFLIGHT_FILE_SUFFIX;
}
public static String makeCommitFileName(String commitTime) {
return commitTime + HoodieTableMetadata.COMMIT_FILE_SUFFIX;
}
public static String getCommitFromCommitFile(String commitFileName) {
return commitFileName.split("\\.")[0];
}

View File

@@ -16,6 +16,11 @@
package com.uber.hoodie.exception;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
/**
* <p>
* Exception thrown to indicate that a hoodie dataset was not found on the path provided
@@ -29,4 +34,21 @@ public class DatasetNotFoundException extends HoodieException {
private static String getErrorMessage(String basePath) {
return "Hoodie dataset not found in path " + basePath;
}
public static void checkValidDataset(FileSystem fs, Path basePathDir, Path metaPathDir)
throws DatasetNotFoundException {
// Check if the base path is found
try {
if (!fs.exists(basePathDir) || !fs.isDirectory(basePathDir)) {
throw new DatasetNotFoundException(basePathDir.toString());
}
// Check if the meta path is found
if (!fs.exists(metaPathDir) || !fs.isDirectory(metaPathDir)) {
throw new DatasetNotFoundException(metaPathDir.toString());
}
} catch (IOException e) {
throw new HoodieIOException(
"Could not check if dataset " + basePathDir + " is valid dataset", e);
}
}
}

View File

@@ -24,13 +24,17 @@ import java.io.IOException;
* </p>
*/
public class HoodieIOException extends HoodieException {
private final IOException ioException;
private IOException ioException;
public HoodieIOException(String msg, IOException t) {
super(msg, t);
this.ioException = t;
}
public HoodieIOException(String msg) {
super(msg);
}
public IOException getIOException() {
return ioException;
}