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

@@ -30,6 +30,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
@@ -126,7 +127,7 @@ public class ParquetUtils {
List<String> footerVals = new ArrayList<>();
ParquetMetadata footer = readMetadata(parquetFilePath);
Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData();
for (String footerName: footerNames) {
for (String footerName : footerNames) {
if (metadata.containsKey(footerName)) {
footerVals.add(metadata.get(footerName));
} else {
@@ -137,6 +138,10 @@ public class ParquetUtils {
return footerVals;
}
public static Schema readAvroSchema(Path parquetFilePath) {
return new AvroSchemaConverter().convert(readSchema(parquetFilePath));
}
/**
* Read out the bloom filter from the parquet file meta data.
*/