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:
committed by
vinoth chandar
parent
c98ee057fc
commit
64e0573aca
@@ -18,10 +18,23 @@ package com.uber.hoodie.common;
|
||||
|
||||
import com.uber.hoodie.HoodieReadClient;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieDataFile;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.table.view.HoodieTableFileSystemView;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
|
||||
import org.apache.spark.SparkConf;
|
||||
|
||||
@@ -29,10 +42,13 @@ import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Utility methods to aid testing inside the HoodieClient module.
|
||||
@@ -90,4 +106,83 @@ public class HoodieClientTestUtils {
|
||||
.setMaster("local[1]");
|
||||
return HoodieReadClient.addHoodieSupport(sparkConf);
|
||||
}
|
||||
|
||||
public static HashMap<String, String> getLatestFileIsToFullPath(String basePath,
|
||||
HoodieTimeline commitTimeline,
|
||||
List<HoodieInstant> commitsToReturn) throws IOException {
|
||||
HashMap<String, String> fileIdToFullPath = new HashMap<>();
|
||||
for (HoodieInstant commit : commitsToReturn) {
|
||||
HoodieCommitMetadata metadata =
|
||||
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get());
|
||||
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
|
||||
}
|
||||
return fileIdToFullPath;
|
||||
}
|
||||
|
||||
public static Dataset<Row> readCommit(String basePath,
|
||||
SQLContext sqlContext,
|
||||
HoodieTimeline commitTimeline,
|
||||
String commitTime) {
|
||||
HoodieInstant commitInstant =
|
||||
new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTime);
|
||||
if (!commitTimeline.containsInstant(commitInstant)) {
|
||||
new HoodieException("No commit exists at " + commitTime);
|
||||
}
|
||||
try {
|
||||
HashMap<String, String> paths = getLatestFileIsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant));
|
||||
return sqlContext.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);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain all new data written into the Hoodie dataset since the given timestamp.
|
||||
*/
|
||||
public static Dataset<Row> readSince(String basePath,
|
||||
SQLContext sqlContext,
|
||||
HoodieTimeline commitTimeline,
|
||||
String lastCommitTime) {
|
||||
List<HoodieInstant> commitsToReturn =
|
||||
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE)
|
||||
.getInstants().collect(Collectors.toList());
|
||||
try {
|
||||
// Go over the commit metadata, and obtain the new files that need to be read.
|
||||
HashMap<String, String> fileIdToFullPath = getLatestFileIsToFullPath(basePath, commitTimeline, commitsToReturn);
|
||||
return sqlContext.read()
|
||||
.parquet(fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]))
|
||||
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the paths under the a hoodie dataset out as a DataFrame
|
||||
*/
|
||||
public static Dataset<Row> read(String basePath,
|
||||
SQLContext sqlContext,
|
||||
FileSystem fs,
|
||||
String... paths) {
|
||||
List<String> filteredPaths = new ArrayList<>();
|
||||
try {
|
||||
HoodieTable hoodieTable = HoodieTable
|
||||
.getHoodieTable(new HoodieTableMetaClient(fs, basePath, true), null);
|
||||
for (String path : paths) {
|
||||
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 sqlContext.read()
|
||||
.parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error reading hoodie dataset as a dataframe", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user