-
Notifications
You must be signed in to change notification settings - Fork 28.8k
[SPARK-52346][SDP] Propagate partition columns from destination for BatchTableWrite #52119
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: master
Are you sure you want to change the base?
Changes from all commits
b4705ed
09bb98d
4ea8b1a
249f280
81a6d27
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -30,7 +30,8 @@ import org.apache.spark.api.python.PythonUtils | |
import org.apache.spark.sql.AnalysisException | ||
import org.apache.spark.sql.catalyst.TableIdentifier | ||
import org.apache.spark.sql.connect.service.SparkConnectService | ||
import org.apache.spark.sql.pipelines.graph.DataflowGraph | ||
import org.apache.spark.sql.connector.catalog.{Identifier, TableCatalog} | ||
import org.apache.spark.sql.pipelines.graph.{DataflowGraph, PipelineUpdateContextImpl} | ||
import org.apache.spark.sql.pipelines.utils.{EventVerificationTestHelpers, TestPipelineUpdateContextMixin} | ||
|
||
/** | ||
|
@@ -434,6 +435,36 @@ class PythonPipelineSuite | |
.map(_.identifier) == Seq(graphIdentifier("a"), graphIdentifier("something"))) | ||
} | ||
|
||
test("MV/ST with partition columns works") { | ||
val graph = buildGraph(""" | ||
|from pyspark.sql.functions import col | ||
| | ||
|@dp.materialized_view(partition_cols = ["id_mod"]) | ||
|def mv(): | ||
| return spark.range(5).withColumn("id_mod", col("id") % 2) | ||
| | ||
|@dp.table(partition_cols = ["id_mod"]) | ||
|def st(): | ||
| return spark.readStream.table("mv") | ||
|""".stripMargin) | ||
|
||
val updateContext = new PipelineUpdateContextImpl(graph, eventCallback = _ => ()) | ||
updateContext.pipelineExecution.runPipeline() | ||
updateContext.pipelineExecution.awaitCompletion() | ||
Comment on lines
+451
to
+453
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: use There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. we actually don't extend |
||
|
||
// check table is created with correct partitioning | ||
val catalog = spark.sessionState.catalogManager.currentCatalog.asInstanceOf[TableCatalog] | ||
|
||
Seq("mv", "st").foreach { tableName => | ||
val table = catalog.loadTable(Identifier.of(Array("default"), tableName)) | ||
assert(table.partitioning().map(_.references().head.fieldNames().head) === Array("id_mod")) | ||
|
||
val rows = spark.table(tableName).collect().map(r => (r.getLong(0), r.getLong(1))).toSet | ||
val expected = (0 until 5).map(id => (id.toLong, (id % 2).toLong)).toSet | ||
assert(rows == expected) | ||
Comment on lines
+462
to
+464
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: use |
||
} | ||
} | ||
|
||
test("create pipeline without table will throw RUN_EMPTY_PIPELINE exception") { | ||
checkError( | ||
exception = intercept[AnalysisException] { | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we also add a test for what happens when a user attempts to change partition columns between pipeline runs?
I'd expect that to either trigger a full refresh or throw an exception. Either are probably acceptable, but it'd be nice for this behavior to be tested.
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we already have a test for this:
specifying partition column different from existing partitioned table
inMaterializeTablesSuite
, it throws aCANNOT_UPDATE_PARTITION_COLUMNS
error