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

@@ -16,8 +16,13 @@
package com.uber.hoodie.hadoop;
import com.uber.hoodie.common.model.HoodieDataFile;
import com.uber.hoodie.common.model.HoodieRecord;
import com.uber.hoodie.common.model.HoodieTableMetadata;
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.view.ReadOptimizedTableView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.exception.InvalidDatasetException;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@@ -52,6 +57,7 @@ import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static parquet.filter2.predicate.FilterApi.and;
import static parquet.filter2.predicate.FilterApi.binaryColumn;
@@ -73,11 +79,11 @@ public class HoodieInputFormat extends MapredParquetInputFormat
public FileStatus[] listStatus(JobConf job) throws IOException {
// Get all the file status from FileInputFormat and then do the filter
FileStatus[] fileStatuses = super.listStatus(job);
Map<HoodieTableMetadata, List<FileStatus>> groupedFileStatus = groupFileStatus(fileStatuses);
Map<HoodieTableMetaClient, List<FileStatus>> groupedFileStatus = groupFileStatus(fileStatuses);
LOG.info("Found a total of " + groupedFileStatus.size() + " groups");
List<FileStatus> returns = new ArrayList<FileStatus>();
for(Map.Entry<HoodieTableMetadata, List<FileStatus>> entry:groupedFileStatus.entrySet()) {
HoodieTableMetadata metadata = entry.getKey();
for(Map.Entry<HoodieTableMetaClient, List<FileStatus>> entry:groupedFileStatus.entrySet()) {
HoodieTableMetaClient metadata = entry.getKey();
if(metadata == null) {
// Add all the paths which are not hoodie specific
returns.addAll(entry.getValue());
@@ -86,31 +92,35 @@ public class HoodieInputFormat extends MapredParquetInputFormat
FileStatus[] value = entry.getValue().toArray(new FileStatus[entry.getValue().size()]);
LOG.info("Hoodie Metadata initialized with completed commit Ts as :" + metadata);
String tableName = metadata.getTableName();
String tableName = metadata.getTableConfig().getTableName();
String mode = HoodieHiveUtil.readMode(Job.getInstance(job), tableName);
TableFileSystemView fsView = new ReadOptimizedTableView(FSUtils.getFs(), metadata);
HoodieTimeline timeline = metadata.getActiveCommitTimeline();
if (HoodieHiveUtil.INCREMENTAL_SCAN_MODE.equals(mode)) {
// this is of the form commitTs_partition_sequenceNumber
String lastIncrementalTs = HoodieHiveUtil.readStartCommitTime(Job.getInstance(job), tableName);
// Total number of commits to return in this batch. Set this to -1 to get all the commits.
Integer maxCommits = HoodieHiveUtil.readMaxCommits(Job.getInstance(job), tableName);
LOG.info("Last Incremental timestamp was set as " + lastIncrementalTs);
List<String>
commitsToReturn = metadata.findCommitsAfter(lastIncrementalTs, maxCommits);
FileStatus[] filteredFiles =
metadata.getLatestVersionInRange(value, commitsToReturn);
for (FileStatus filteredFile : filteredFiles) {
List<String> commitsToReturn =
timeline.findInstantsAfter(lastIncrementalTs, maxCommits)
.collect(Collectors.toList());
List<HoodieDataFile> filteredFiles =
fsView.streamLatestVersionInRange(value, commitsToReturn)
.collect(Collectors.toList());
for (HoodieDataFile filteredFile : filteredFiles) {
LOG.info("Processing incremental hoodie file - " + filteredFile.getPath());
returns.add(filteredFile);
returns.add(filteredFile.getFileStatus());
}
LOG.info(
"Total paths to process after hoodie incremental filter " + filteredFiles.length);
"Total paths to process after hoodie incremental filter " + filteredFiles.size());
} else {
// filter files on the latest commit found
FileStatus[] filteredFiles = metadata.getLatestVersions(value);
LOG.info("Total paths to process after hoodie filter " + filteredFiles.length);
for (FileStatus filteredFile : filteredFiles) {
List<HoodieDataFile> filteredFiles = fsView.streamLatestVersions(value).collect(Collectors.toList());
LOG.info("Total paths to process after hoodie filter " + filteredFiles.size());
for (HoodieDataFile filteredFile : filteredFiles) {
LOG.info("Processing latest hoodie file - " + filteredFile.getPath());
returns.add(filteredFile);
returns.add(filteredFile.getFileStatus());
}
}
}
@@ -118,18 +128,18 @@ public class HoodieInputFormat extends MapredParquetInputFormat
}
private Map<HoodieTableMetadata, List<FileStatus>> groupFileStatus(FileStatus[] fileStatuses)
private Map<HoodieTableMetaClient, List<FileStatus>> groupFileStatus(FileStatus[] fileStatuses)
throws IOException {
// This assumes the paths for different tables are grouped together
Map<HoodieTableMetadata, List<FileStatus>> grouped = new HashMap<>();
HoodieTableMetadata metadata = null;
Map<HoodieTableMetaClient, List<FileStatus>> grouped = new HashMap<>();
HoodieTableMetaClient metadata = null;
String nonHoodieBasePath = null;
for(FileStatus status:fileStatuses) {
if ((metadata == null && nonHoodieBasePath == null) || (metadata == null && !status.getPath().toString()
.contains(nonHoodieBasePath)) || (metadata != null && !status.getPath().toString()
.contains(metadata.getBasePath()))) {
try {
metadata = getTableMetadata(status.getPath().getParent());
metadata = getTableMetaClient(status.getPath().getParent());
nonHoodieBasePath = null;
} catch (InvalidDatasetException e) {
LOG.info("Handling a non-hoodie path " + status.getPath());
@@ -138,7 +148,7 @@ public class HoodieInputFormat extends MapredParquetInputFormat
status.getPath().getParent().toString();
}
if(!grouped.containsKey(metadata)) {
grouped.put(metadata, new ArrayList<FileStatus>());
grouped.put(metadata, new ArrayList<>());
}
}
grouped.get(metadata).add(status);
@@ -242,12 +252,12 @@ public class HoodieInputFormat extends MapredParquetInputFormat
* @return
* @throws IOException
*/
private HoodieTableMetadata getTableMetadata(Path dataPath) throws IOException {
private HoodieTableMetaClient getTableMetaClient(Path dataPath) throws IOException {
FileSystem fs = dataPath.getFileSystem(conf);
// TODO - remove this hard-coding. Pass this in job conf, somehow. Or read the Table Location
Path baseDir = dataPath.getParent().getParent().getParent();
LOG.info("Reading hoodie metadata from path " + baseDir.toString());
return new HoodieTableMetadata(fs, baseDir.toString());
return new HoodieTableMetaClient(fs, baseDir.toString());
}
}