1
0

Adding hoodie-spark to support Spark Datasource for Hoodie

- Write with COW/MOR paths work fully
 - Read with RO view works on both storages*
 - Incremental view supported on COW
 - Refactored out HoodieReadClient methods, to just contain key based access
 - HoodieDataSourceHelpers class can be now used to construct inputs to datasource
 - Tests in hoodie-client using new helpers and mechanisms
 - Basic tests around save modes & insert/upserts (more to follow)
 - Bumped up scala to 2.11, since 2.10 is deprecated & complains with scalatest
 - Updated documentation to describe usage
 - New sample app written using the DataSource API
This commit is contained in:
Vinoth Chandar
2017-08-28 01:28:08 -07:00
committed by vinoth chandar
parent c98ee057fc
commit 64e0573aca
44 changed files with 1830 additions and 331 deletions

View File

@@ -59,10 +59,8 @@ import java.util.stream.Collectors;
import scala.Tuple2;
/**
* Provides first class support for accessing Hoodie tables for data processing via Apache Spark.
* Provides an RDD based API for accessing/filtering Hoodie tables, based on keys.
*
*
* TODO: Need to move all read operations here, since Hoodie is a single writer and multiple reader
*/
public class HoodieReadClient implements Serializable {
@@ -81,7 +79,6 @@ public class HoodieReadClient implements Serializable {
private HoodieTable hoodieTable;
private transient Optional<SQLContext> sqlContextOpt;
/**
* @param basePath path to Hoodie dataset
*/
@@ -92,7 +89,6 @@ public class HoodieReadClient implements Serializable {
this.hoodieTable = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
this.commitTimeline = hoodieTable.getCompletedCompactionCommitTimeline();
this.index =
new HoodieBloomIndex(HoodieWriteConfig.newBuilder().withPath(basePath).build(), jsc);
this.sqlContextOpt = Optional.absent();
@@ -161,93 +157,6 @@ public class HoodieReadClient implements Serializable {
return sqlContextOpt.get().createDataFrame(rowRDD, schema);
}
/**
* Reads the paths under the a hoodie dataset out as a DataFrame
*/
public Dataset<Row> read(String... paths) {
assertSqlContext();
List<String> filteredPaths = new ArrayList<>();
try {
for (String path : paths) {
if (!path.contains(hoodieTable.getMetaClient().getBasePath())) {
throw new HoodieException("Path " + path
+ " does not seem to be a part of a Hoodie dataset at base path "
+ hoodieTable.getMetaClient().getBasePath());
}
TableFileSystemView.ReadOptimizedView fileSystemView = new HoodieTableFileSystemView(hoodieTable.getMetaClient(),
hoodieTable.getCompletedCommitTimeline(), fs.globStatus(new Path(path)));
List<HoodieDataFile> latestFiles = fileSystemView.getLatestDataFiles().collect(
Collectors.toList());
for (HoodieDataFile file : latestFiles) {
filteredPaths.add(file.getPath());
}
}
return sqlContextOpt.get().read()
.parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
} catch (Exception e) {
throw new HoodieException("Error reading hoodie dataset as a dataframe", e);
}
}
/**
* Obtain all new data written into the Hoodie dataset since the given timestamp.
*
* If you made a prior call to {@link HoodieReadClient#latestCommit()}, it gives you all data in
* the time window (commitTimestamp, latestCommit)
*/
public Dataset<Row> readSince(String lastCommitTimestamp) {
List<HoodieInstant> commitsToReturn =
commitTimeline.findInstantsAfter(lastCommitTimestamp, Integer.MAX_VALUE)
.getInstants().collect(Collectors.toList());
//TODO: we can potentially trim this down to only affected partitions, using CommitMetadata
try {
// Go over the commit metadata, and obtain the new files that need to be read.
HashMap<String, String> fileIdToFullPath = new HashMap<>();
for (HoodieInstant commit : commitsToReturn) {
HoodieCommitMetadata metadata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
// get files from each commit, and replace any previous versions
String basePath = hoodieTable.getMetaClient().getBasePath();
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
}
return sqlContextOpt.get().read()
.parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTimestamp));
} catch (IOException e) {
throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTimestamp, e);
}
}
/**
* Obtain
*/
public Dataset<Row> readCommit(String commitTime) {
assertSqlContext();
String actionType = hoodieTable.getCompactedCommitActionType();
HoodieInstant commitInstant =
new HoodieInstant(false, actionType, commitTime);
if (!commitTimeline.containsInstant(commitInstant)) {
new HoodieException("No commit exists at " + commitTime);
}
try {
HoodieCommitMetadata commitMetadata =
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commitInstant).get());
String basePath = hoodieTable.getMetaClient().getBasePath();
HashMap<String, String> paths = commitMetadata.getFileIdAndFullPaths(basePath);
return sqlContextOpt.get().read()
.parquet(paths.values().toArray(new String[paths.size()]))
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, commitTime));
} catch (Exception e) {
throw new HoodieException("Error reading commit " + commitTime, e);
}
}
/**
* Checks if the given [Keys] exists in the hoodie table and returns [Key,
* Optional[FullFilePath]] If the optional FullFilePath value is not present, then the key is
@@ -269,29 +178,4 @@ public class HoodieReadClient implements Serializable {
JavaRDD<HoodieRecord> recordsWithLocation = index.tagLocation(hoodieRecords, hoodieTable);
return recordsWithLocation.filter(v1 -> !v1.isCurrentLocationKnown());
}
/**
* Checks if the Hoodie dataset has new data since given timestamp. This can be subsequently
* used to call {@link HoodieReadClient#readSince(String)} to perform incremental processing.
*/
public boolean hasNewCommits(String commitTimestamp) {
return listCommitsSince(commitTimestamp).size() > 0;
}
/**
*
* @param commitTimestamp
* @return
*/
public List<String> listCommitsSince(String commitTimestamp) {
return commitTimeline.findInstantsAfter(commitTimestamp, Integer.MAX_VALUE).getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
}
/**
* Returns the last successful commit (a successful write operation) into a Hoodie table.
*/
public String latestCommit() {
return commitTimeline.lastInstant().get().getTimestamp();
}
}

View File

@@ -368,7 +368,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
JavaRDD<WriteStatus> writeStatuses,
Optional<HashMap<String, String>> extraMetadata) {
logger.info("Comitting " + commitTime);
logger.info("Commiting " + commitTime);
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> table = HoodieTable
.getHoodieTable(new HoodieTableMetaClient(fs, config.getBasePath(), true), config);

View File

@@ -31,6 +31,7 @@ import java.io.File;
import java.io.FileReader;
import java.io.IOException;
import java.io.InputStream;
import java.util.Map;
import java.util.Properties;
/**
@@ -298,6 +299,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
}
public Builder withProps(Map kvprops) {
props.putAll(kvprops);
return this;
}
public Builder withPath(String basePath) {
props.setProperty(BASE_PATH_PROP, basePath);