feat(SparkDataSource): add structured streaming
This commit is contained in:
committed by
vinoth chandar
parent
7243ce40c9
commit
bf65219b73
@@ -20,14 +20,18 @@ 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
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime}
|
||||
import org.junit.Assert._
|
||||
import org.junit.rules.TemporaryFolder
|
||||
import org.junit.{Before, Test}
|
||||
import org.scalatest.junit.AssertionsForJUnit
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.concurrent.duration.Duration
|
||||
import scala.concurrent.{Await, Future}
|
||||
import scala.concurrent.ExecutionContext.Implicits.global
|
||||
|
||||
/**
|
||||
* Basic tests on the spark datasource
|
||||
@@ -62,7 +66,7 @@ class DataSourceTest extends AssertionsForJUnit {
|
||||
|
||||
@Test def testCopyOnWriteStorage() {
|
||||
// Insert Operation
|
||||
val records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("001", 100)).toList
|
||||
val records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("com.uber.hoodie")
|
||||
.options(commonOpts)
|
||||
@@ -182,4 +186,92 @@ class DataSourceTest extends AssertionsForJUnit {
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt)
|
||||
}
|
||||
|
||||
@Test def testStructuredStreaming(): Unit = {
|
||||
fs.delete(new Path(basePath), true)
|
||||
val sourcePath = basePath + "/source"
|
||||
val destPath = basePath + "/dest"
|
||||
fs.mkdirs(new Path(sourcePath))
|
||||
|
||||
// First chunk of data
|
||||
val records1 = DataSourceTestUtils.convertToStringList(dataGen.generateInserts("000", 100)).toList
|
||||
val inputDF1: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
|
||||
// Second chunk of data
|
||||
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()
|
||||
|
||||
// define the source of streaming
|
||||
val streamingInput =
|
||||
spark.readStream
|
||||
.schema(inputDF1.schema)
|
||||
.json(sourcePath)
|
||||
|
||||
val f1 = Future {
|
||||
println("streaming starting")
|
||||
//'writeStream' can be called only on streaming Dataset/DataFrame
|
||||
streamingInput
|
||||
.writeStream
|
||||
.format("com.uber.hoodie")
|
||||
.options(commonOpts)
|
||||
.trigger(new ProcessingTime(100))
|
||||
.option("checkpointLocation", basePath + "/checkpoint")
|
||||
.outputMode(OutputMode.Append)
|
||||
.start(destPath)
|
||||
.awaitTermination(10000)
|
||||
println("streaming ends")
|
||||
}
|
||||
|
||||
val f2 = Future {
|
||||
inputDF1.write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
Thread.sleep(3000)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, destPath, "000"))
|
||||
val commitInstantTime1: String = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
// Read RO View
|
||||
val hoodieROViewDF1 = spark.read.format("com.uber.hoodie")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assert(hoodieROViewDF1.count() == 100)
|
||||
|
||||
inputDF2.write.mode(SaveMode.Append).json(sourcePath)
|
||||
// wait for spark streaming to process one microbatch
|
||||
Thread.sleep(3000)
|
||||
val commitInstantTime2: String = HoodieDataSourceHelpers.latestCommit(fs, destPath)
|
||||
assertEquals(2, HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").size())
|
||||
// Read RO View
|
||||
val hoodieROViewDF2 = spark.read.format("com.uber.hoodie")
|
||||
.load(destPath + "/*/*/*/*")
|
||||
assertEquals(100, hoodieROViewDF2.count()) // still 100, since we only updated
|
||||
|
||||
|
||||
// Read Incremental View
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").get(0)
|
||||
val hoodieIncViewDF1 = spark.read.format("com.uber.hoodie")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.load(destPath)
|
||||
assertEquals(100, hoodieIncViewDF1.count())
|
||||
// 100 initial inserts must be pulled
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(firstCommit, countsPerCommit(0).get(0))
|
||||
|
||||
// pull the latest commit
|
||||
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(destPath)
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
countsPerCommit = hoodieIncViewDF2.groupBy("_hoodie_commit_time").count().collect()
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(commitInstantTime2, countsPerCommit(0).get(0))
|
||||
}
|
||||
|
||||
Await.result(Future.sequence(Seq(f1, f2)), Duration.Inf)
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user