forked from apache/paimon
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
- Loading branch information
Showing
8 changed files
with
267 additions
and
18 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
219 changes: 219 additions & 0 deletions
219
...park/paimon-spark-common/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,219 @@ | ||
/* | ||
* 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.paimon.spark | ||
|
||
import org.apache.spark.sql.{Dataset, Row} | ||
import org.apache.spark.sql.execution.streaming.MemoryStream | ||
import org.apache.spark.sql.streaming.StreamTest | ||
|
||
class PaimonCDCSourceTest extends PaimonSparkTestBase with StreamTest { | ||
|
||
import testImplicits._ | ||
|
||
test("Paimon CDC Source: batch write and streaming read change-log with default scan mode") { | ||
withTempDir { | ||
checkpointDir => | ||
val tableName = "T" | ||
spark.sql(s"DROP TABLE IF EXISTS $tableName") | ||
spark.sql(s""" | ||
|CREATE TABLE $tableName (a INT, b STRING) | ||
|TBLPROPERTIES ( | ||
| 'primary-key'='a', | ||
| 'write-mode'='change-log', | ||
| 'bucket'='2', | ||
| 'changelog-producer' = 'lookup') | ||
|""".stripMargin) | ||
|
||
spark.sql(s"INSERT INTO $tableName VALUES (1, 'v_1')") | ||
spark.sql(s"INSERT INTO $tableName VALUES (2, 'v_2')") | ||
spark.sql(s"INSERT INTO $tableName VALUES (2, 'v_2_new')") | ||
|
||
val table = loadTable(tableName) | ||
val location = table.location().getPath | ||
|
||
val readStream = spark.readStream | ||
.format("paimon") | ||
.option("read.readChangeLog", "true") | ||
.load(location) | ||
.writeStream | ||
.format("memory") | ||
.option("checkpointLocation", checkpointDir.getCanonicalPath) | ||
.queryName("mem_table") | ||
.outputMode("append") | ||
.start() | ||
|
||
val currentResult = () => spark.sql("SELECT * FROM mem_table") | ||
try { | ||
readStream.processAllAvailable() | ||
val expertResult1 = Row("+I", 1, "v_1") :: Row("+I", 2, "v_2_new") :: Nil | ||
checkAnswer(currentResult(), expertResult1) | ||
|
||
spark.sql(s"INSERT INTO $tableName VALUES (1, 'v_1_new'), (3, 'v_3')") | ||
readStream.processAllAvailable() | ||
val expertResult2 = | ||
Row("+I", 1, "v_1") :: Row("-U", 1, "v_1") :: Row("+U", 1, "v_1_new") :: Row( | ||
"+I", | ||
2, | ||
"v_2_new") :: Row("+I", 3, "v_3") :: Nil | ||
checkAnswer(currentResult(), expertResult2) | ||
} finally { | ||
readStream.stop() | ||
} | ||
} | ||
} | ||
|
||
test("Paimon CDC Source: batch write and streaming read change-log with scan.snapshot-id") { | ||
withTempDir { | ||
checkpointDir => | ||
val tableName = "T" | ||
spark.sql(s"DROP TABLE IF EXISTS $tableName") | ||
spark.sql(s""" | ||
|CREATE TABLE $tableName (a INT, b STRING) | ||
|TBLPROPERTIES ( | ||
| 'primary-key'='a', | ||
| 'write-mode'='change-log', | ||
| 'bucket'='2', | ||
| 'changelog-producer' = 'lookup') | ||
|""".stripMargin) | ||
|
||
spark.sql(s"INSERT INTO $tableName VALUES (1, 'v_1')") | ||
spark.sql(s"INSERT INTO $tableName VALUES (2, 'v_2')") | ||
spark.sql(s"INSERT INTO $tableName VALUES (2, 'v_2_new')") | ||
|
||
val table = loadTable(tableName) | ||
val location = table.location().getPath | ||
|
||
val readStream = spark.readStream | ||
.format("paimon") | ||
.option("read.readChangeLog", "true") | ||
.option("scan.mode", "from-snapshot") | ||
.option("scan.snapshot-id", 1) | ||
.load(location) | ||
.writeStream | ||
.format("memory") | ||
.option("checkpointLocation", checkpointDir.getCanonicalPath) | ||
.queryName("mem_table") | ||
.outputMode("append") | ||
.start() | ||
|
||
val currentResult = () => spark.sql("SELECT * FROM mem_table") | ||
try { | ||
readStream.processAllAvailable() | ||
val expertResult1 = Row("+I", 1, "v_1") :: Row("+I", 2, "v_2") :: Row( | ||
"-U", | ||
2, | ||
"v_2") :: Row("+U", 2, "v_2_new") :: Nil | ||
checkAnswer(currentResult(), expertResult1) | ||
|
||
spark.sql(s"INSERT INTO $tableName VALUES (1, 'v_1_new'), (3, 'v_3')") | ||
readStream.processAllAvailable() | ||
val expertResult2 = | ||
Row("+I", 1, "v_1") :: Row("-U", 1, "v_1") :: Row("+U", 1, "v_1_new") :: Row( | ||
"+I", | ||
2, | ||
"v_2") :: Row("-U", 2, "v_2") :: Row("+U", 2, "v_2_new") :: Row("+I", 3, "v_3") :: Nil | ||
checkAnswer(currentResult(), expertResult2) | ||
} finally { | ||
readStream.stop() | ||
} | ||
} | ||
} | ||
|
||
test("Paimon CDC Source: streaming write and streaming read change-log") { | ||
withTempDirs { | ||
(checkpointDir1, checkpointDir2) => | ||
val tableName = "T" | ||
spark.sql(s"DROP TABLE IF EXISTS $tableName") | ||
spark.sql(s""" | ||
|CREATE TABLE $tableName (a INT, b STRING) | ||
|TBLPROPERTIES ( | ||
| 'primary-key'='a', | ||
| 'write-mode'='change-log', | ||
| 'bucket'='2', | ||
| 'changelog-producer' = 'lookup') | ||
|""".stripMargin) | ||
|
||
val table = loadTable(tableName) | ||
val location = table.location().getPath | ||
|
||
// streaming write | ||
val inputData = MemoryStream[(Int, String)] | ||
val writeStream = inputData | ||
.toDS() | ||
.toDF("a", "b") | ||
.writeStream | ||
.option("checkpointLocation", checkpointDir1.getCanonicalPath) | ||
.foreachBatch { | ||
(batch: Dataset[Row], _: Long) => | ||
batch.write.format("paimon").mode("append").save(location) | ||
} | ||
.start() | ||
|
||
// streaming read | ||
val readStream = spark.readStream | ||
.format("paimon") | ||
.option("read.readChangeLog", "true") | ||
.option("scan.mode", "from-snapshot") | ||
.option("scan.snapshot-id", 1) | ||
.load(location) | ||
.writeStream | ||
.format("memory") | ||
.option("checkpointLocation", checkpointDir2.getCanonicalPath) | ||
.queryName("mem_table") | ||
.outputMode("append") | ||
.start() | ||
|
||
val currentResult = () => spark.sql("SELECT * FROM mem_table") | ||
try { | ||
inputData.addData((1, "v_1")) | ||
writeStream.processAllAvailable() | ||
readStream.processAllAvailable() | ||
val expertResult1 = Row("+I", 1, "v_1") :: Nil | ||
checkAnswer(currentResult(), expertResult1) | ||
|
||
inputData.addData((2, "v_2")) | ||
writeStream.processAllAvailable() | ||
readStream.processAllAvailable() | ||
val expertResult2 = Row("+I", 1, "v_1") :: Row("+I", 2, "v_2") :: Nil | ||
checkAnswer(currentResult(), expertResult2) | ||
|
||
inputData.addData((2, "v_2_new")) | ||
writeStream.processAllAvailable() | ||
readStream.processAllAvailable() | ||
val expertResult3 = Row("+I", 1, "v_1") :: Row("+I", 2, "v_2") :: Row( | ||
"-U", | ||
2, | ||
"v_2") :: Row("+U", 2, "v_2_new") :: Nil | ||
checkAnswer(currentResult(), expertResult3) | ||
|
||
inputData.addData((1, "v_1_new"), (3, "v_3")) | ||
writeStream.processAllAvailable() | ||
readStream.processAllAvailable() | ||
val expertResult4 = | ||
Row("+I", 1, "v_1") :: Row("-U", 1, "v_1") :: Row("+U", 1, "v_1_new") :: Row( | ||
"+I", | ||
2, | ||
"v_2") :: Row("-U", 2, "v_2") :: Row("+U", 2, "v_2_new") :: Row("+I", 3, "v_3") :: Nil | ||
checkAnswer(currentResult(), expertResult4) | ||
} finally { | ||
readStream.stop() | ||
} | ||
} | ||
} | ||
|
||
} |