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

@@ -0,0 +1,50 @@
/*
* Copyright (c) 2017 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.
*
*
*/
import com.uber.hoodie.common.TestRawTripPayload;
import com.uber.hoodie.common.model.HoodieRecord;
import org.apache.spark.api.java.JavaRDD;
import java.io.IOException;
import java.util.List;
import java.util.Optional;
import java.util.stream.Collectors;
/**
* Test utils for data source tests.
*/
public class DataSourceTestUtils {
public static Optional<String> convertToString(HoodieRecord record) {
try {
String str = ((TestRawTripPayload) record.getData()).getJsonData();
str = "{" + str.substring(str.indexOf("\"timestamp\":"));
return Optional.of(str.replaceAll("}", ", \"partition\": \"" + record.getPartitionPath() + "\"}"));
} catch (IOException e) {
return Optional.empty();
}
}
public static List<String> convertToStringList(List<HoodieRecord> records) {
return records.stream().map(hr -> convertToString(hr))
.filter(os -> os.isPresent())
.map(os -> os.get())
.collect(Collectors.toList());
}
}

View File

@@ -0,0 +1,148 @@
/*
* Copyright (c) 2017 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.
*
*
*/
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import com.uber.hoodie.DataSourceReadOptions;
import com.uber.hoodie.DataSourceWriteOptions;
import com.uber.hoodie.HoodieDataSourceHelpers;
import com.uber.hoodie.common.HoodieTestDataGenerator;
import com.uber.hoodie.common.model.HoodieTableType;
import com.uber.hoodie.config.HoodieWriteConfig;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import java.util.List;
/**
* Sample program that writes & reads hoodie datasets via the Spark datasource
*/
public class HoodieJavaApp {
@Parameter(names={"--table-path", "-p"}, description = "path for Hoodie sample table")
private String tablePath = "file:///tmp/hoodie/sample-table";
@Parameter(names={"--table-name", "-n"}, description = "table name for Hoodie sample table")
private String tableName = "hoodie_test";
@Parameter(names={"--table-type", "-t"}, description = "One of COPY_ON_WRITE or MERGE_ON_READ")
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
@Parameter(names = {"--help", "-h"}, help = true)
public Boolean help = false;
private static Logger logger = LogManager.getLogger(HoodieJavaApp.class);
public static void main(String[] args) throws Exception {
HoodieJavaApp cli = new HoodieJavaApp();
JCommander cmd = new JCommander(cli, args);
if (cli.help) {
cmd.usage();
System.exit(1);
}
cli.run();
}
public void run() throws Exception {
// Spark session setup..
SparkSession spark = SparkSession.builder()
.appName("Hoodie Spark APP")
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.master("local[1]")
.getOrCreate();
JavaSparkContext jssc = new JavaSparkContext(spark.sparkContext());
FileSystem fs = FileSystem.get(jssc.hadoopConfiguration());
// Generator of some records to be loaded in.
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
/**
* Commit with only inserts
*/
// Generate some input..
List<String> records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("001"/* ignore */, 100));
Dataset<Row> inputDF1 = spark.read().json(jssc.parallelize(records1, 2));
// Save as hoodie dataset (copy on write)
inputDF1.write()
.format("com.uber.hoodie") // specify the hoodie source
.option("hoodie.insert.shuffle.parallelism", "2") // any hoodie client config can be passed like this
.option("hoodie.upsert.shuffle.parallelism", "2") // full list in HoodieWriteConfig & its package
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL()) // insert
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key") // This is the record key
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition") // this is the partition to place it into
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp") // use to combine duplicate records in input/with disk val
.option(HoodieWriteConfig.TABLE_NAME, tableName) // Used by hive sync and queries
.mode(SaveMode.Overwrite) // This will remove any existing data at path below, and create a new dataset if needed
.save(tablePath); // ultimately where the dataset will be placed
String commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
/**
* Commit that updates records
*/
List<String> records2 = DataSourceTestUtils.convertToStringList(dataGen.generateUpdates("002"/* ignore */, 100));
Dataset<Row> inputDF2 = spark.read().json(jssc.parallelize(records2, 2));
inputDF2.write()
.format("com.uber.hoodie")
.option("hoodie.insert.shuffle.parallelism", "2")
.option("hoodie.upsert.shuffle.parallelism", "2")
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
.option(HoodieWriteConfig.TABLE_NAME, tableName)
.mode(SaveMode.Append)
.save(tablePath);
String commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, tablePath);
/**
* Read & do some queries
*/
Dataset<Row> hoodieROViewDF = spark.read()
.format("com.uber.hoodie")
// pass any path glob, can include hoodie & non-hoodie datasets
.load(tablePath + "/*/*/*/*");
hoodieROViewDF.registerTempTable("hoodie_ro");
spark.sql("describe hoodie_ro").show();
// all trips whose fare was greater than 2.
spark.sql("select fare, begin_lon, begin_lat, timestamp from hoodie_ro where fare > 2.0").show();
/**
* Consume incrementally, only changes in commit 2 above.
*/
Dataset<Row> hoodieIncViewDF = spark.read().format("com.uber.hoodie")
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL())
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1) // Only changes in write 2 above
.load(tablePath); // For incremental view, pass in the root/base path of dataset
System.out.println("You will only see records from : " + commitInstantTime2);
hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show();
}
}

View File

@@ -0,0 +1,28 @@
#
# 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.
#
log4j.rootLogger=WARN, A1
log4j.category.com.uber=INFO
log4j.category.com.uber.hoodie.common.utils=WARN
log4j.category.com.uber.hoodie.io=WARN
log4j.category.com.uber.hoodie.common=WARN
log4j.category.com.uber.hoodie.table.log=WARN
log4j.category.org.apache.parquet.hadoop=WARN
# A1 is set to be a ConsoleAppender.
log4j.appender.A1=org.apache.log4j.ConsoleAppender
# A1 uses PatternLayout.
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n

View File

@@ -0,0 +1,137 @@
/*
* Copyright (c) 2017 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.
*
*
*/
import com.uber.hoodie.common.HoodieTestDataGenerator
import com.uber.hoodie.common.util.FSUtils
import com.uber.hoodie.config.HoodieWriteConfig
import com.uber.hoodie.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}
import org.junit.Assert._
import org.junit.{Before, Test}
import org.junit.rules.TemporaryFolder
import org.scalatest.junit.AssertionsForJUnit
import scala.collection.JavaConversions._
/**
* Basic tests on the spark datasource
*/
class DataSourceTest extends AssertionsForJUnit {
var spark: SparkSession = null
var dataGen: HoodieTestDataGenerator = null
val commonOpts = Map(
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4",
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
)
var basePath : String = null
var fs : FileSystem = null
@Before def initialize() {
spark = SparkSession.builder
.appName("Hoodie Datasource test")
.master("local[2]")
.config("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
.getOrCreate
dataGen = new HoodieTestDataGenerator()
val folder = new TemporaryFolder
folder.create
basePath = folder.getRoot.getAbsolutePath
fs = FSUtils.getFs
}
@Test def testCopyOnWriteStorage() {
// Insert Operation
val records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("001", 100)).toList
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("com.uber.hoodie")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.mode(SaveMode.Overwrite)
.save(basePath)
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
// Read RO View
val hoodieROViewDF1 = spark.read.format("com.uber.hoodie")
.load(basePath + "/*/*/*/*");
assertEquals(100, hoodieROViewDF1.count())
val records2 = DataSourceTestUtils.convertToStringList(dataGen.generateUpdates("001", 100)).toList
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
val uniqueKeyCnt = inputDF2.select("_row_key").distinct().count()
// Upsert Operation
inputDF2.write.format("com.uber.hoodie")
.options(commonOpts)
.mode(SaveMode.Append)
.save(basePath)
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, basePath)
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").size())
// Read RO View
val hoodieROViewDF2 = spark.read.format("com.uber.hoodie")
.load(basePath + "/*/*/*/*");
assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated
// Read Incremental View
val hoodieIncViewDF2 = spark.read.format("com.uber.hoodie")
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL)
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
.load(basePath);
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
val countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect();
assertEquals(1, countsPerCommit.length)
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
}
@Test def testMergeOnReadStorage() {
// Bulk Insert Operation
val records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("001", 100)).toList
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("com.uber.hoodie")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_STORAGE_TYPE_OPT_VAL)
.mode(SaveMode.Overwrite)
.save(basePath)
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
// Read RO View
try {
val hoodieROViewDF1 = spark.read.format("com.uber.hoodie")
.load(basePath + "/*/*/*/*")
fail() // we would error out, since no compaction has yet occurred.
} catch {
case e: Exception => {
// do nothing
}
};
}
}