1
0

[HUDI-1109] Support Spark Structured Streaming read from Hudi table (#2485)

This commit is contained in:
pengzhiwei
2021-02-17 19:36:29 +08:00
committed by GitHub
parent 5d2491d10c
commit 37972071ff
10 changed files with 517 additions and 15 deletions

View File

@@ -0,0 +1,154 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You 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.
*/
package org.apache.hudi.functional
import org.apache.hudi.DataSourceWriteOptions
import org.apache.hudi.DataSourceWriteOptions.{PRECOMBINE_FIELD_OPT_KEY, RECORDKEY_FIELD_OPT_KEY}
import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.config.HoodieWriteConfig.{DELETE_PARALLELISM, INSERT_PARALLELISM, TABLE_NAME, UPSERT_PARALLELISM}
import org.apache.spark.sql.streaming.StreamTest
import org.apache.spark.sql.{Row, SaveMode}
class TestStreamingSource extends StreamTest {
import testImplicits._
private val commonOptions = Map(
RECORDKEY_FIELD_OPT_KEY -> "id",
PRECOMBINE_FIELD_OPT_KEY -> "ts",
INSERT_PARALLELISM -> "4",
UPSERT_PARALLELISM -> "4",
DELETE_PARALLELISM -> "4"
)
private val columns = Seq("id", "name", "price", "ts")
override protected def sparkConf = {
super.sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer")
}
test("test cow stream source") {
withTempDir { inputDir =>
val tablePath = s"${inputDir.getCanonicalPath}/test_cow_stream"
HoodieTableMetaClient.initTableType(spark.sessionState.newHadoopConf(), tablePath,
COPY_ON_WRITE, getTableName(tablePath), DataSourceWriteOptions.DEFAULT_PAYLOAD_OPT_VAL)
addData(tablePath, Seq(("1", "a1", "10", "000")))
val df = spark.readStream
.format("org.apache.hudi")
.load(tablePath)
.select("id", "name", "price", "ts")
testStream(df)(
AssertOnQuery {q => q.processAllAvailable(); true },
CheckAnswerRows(Seq(Row("1", "a1", "10", "000")), lastOnly = true, isSorted = false),
StopStream,
addDataToQuery(tablePath, Seq(("1", "a1", "12", "000"))),
StartStream(),
AssertOnQuery {q => q.processAllAvailable(); true },
CheckAnswerRows(Seq(Row("1", "a1", "12", "000")), lastOnly = true, isSorted = false),
addDataToQuery(tablePath,
Seq(("2", "a2", "12", "000"),
("3", "a3", "12", "000"),
("4", "a4", "12", "000"))),
AssertOnQuery {q => q.processAllAvailable(); true },
CheckAnswerRows(
Seq(Row("2", "a2", "12", "000"),
Row("3", "a3", "12", "000"),
Row("4", "a4", "12", "000")),
lastOnly = true, isSorted = false),
StopStream,
addDataToQuery(tablePath, Seq(("5", "a5", "12", "000"))),
addDataToQuery(tablePath, Seq(("6", "a6", "12", "000"))),
addDataToQuery(tablePath, Seq(("5", "a5", "15", "000"))),
StartStream(),
AssertOnQuery {q => q.processAllAvailable(); true },
CheckAnswerRows(
Seq(Row("6", "a6", "12", "000"),
Row("5", "a5", "15", "000")),
lastOnly = true, isSorted = false)
)
}
}
test("test mor stream source") {
withTempDir { inputDir =>
val tablePath = s"${inputDir.getCanonicalPath}/test_mor_stream"
HoodieTableMetaClient.initTableType(spark.sessionState.newHadoopConf(), tablePath,
MERGE_ON_READ, getTableName(tablePath), DataSourceWriteOptions.DEFAULT_PAYLOAD_OPT_VAL)
addData(tablePath, Seq(("1", "a1", "10", "000")))
val df = spark.readStream
.format("org.apache.hudi")
.load(tablePath)
.select("id", "name", "price", "ts")
testStream(df)(
AssertOnQuery {q => q.processAllAvailable(); true },
CheckAnswerRows(Seq(Row("1", "a1", "10", "000")), lastOnly = true, isSorted = false),
StopStream,
addDataToQuery(tablePath,
Seq(("2", "a2", "12", "000"),
("3", "a3", "12", "000"),
("2", "a2", "10", "001"))),
StartStream(),
AssertOnQuery {q => q.processAllAvailable(); true },
CheckAnswerRows(
Seq(Row("3", "a3", "12", "000"),
Row("2", "a2", "10", "001")),
lastOnly = true, isSorted = false),
StopStream,
addDataToQuery(tablePath, Seq(("5", "a5", "12", "000"))),
addDataToQuery(tablePath, Seq(("6", "a6", "12", "000"))),
StartStream(),
AssertOnQuery {q => q.processAllAvailable(); true },
CheckAnswerRows(
Seq(Row("5", "a5", "12", "000"),
Row("6", "a6", "12", "000")),
lastOnly = true, isSorted = false)
)
}
}
private def addData(inputPath: String, rows: Seq[(String, String, String, String)]): Unit = {
rows.toDF(columns: _*)
.write
.format("org.apache.hudi")
.options(commonOptions)
.option(TABLE_NAME, getTableName(inputPath))
.mode(SaveMode.Append)
.save(inputPath)
}
private def addDataToQuery(inputPath: String,
rows: Seq[(String, String, String, String)]): AssertOnQuery = {
AssertOnQuery { _=>
addData(inputPath, rows)
true
}
}
private def getTableName(inputPath: String): String = {
val start = inputPath.lastIndexOf('/')
inputPath.substring(start + 1)
}
}