Skip to content

[SPARK-52283][SQL] Declarative Pipelines DataflowGraph creation and resolution #51003

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

Closed
wants to merge 33 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
33 commits
Select commit Hold shift + click to select a range
5e83f0d
1
aakash-db May 23, 2025
573c0bf
2
aakash-db May 23, 2025
1aa2253
3
aakash-db May 23, 2025
249fc3a
4
aakash-db May 23, 2025
a2c3b39
5
aakash-db May 23, 2025
2c22e19
6
aakash-db May 23, 2025
61e1000
comments p1
aakash-db May 28, 2025
d2effa2
1
aakash-db May 28, 2025
676ee9d
queryContext addition
aakash-db May 28, 2025
af7842d
1
aakash-db May 28, 2025
e0e4d13
error formatting
aakash-db May 28, 2025
22cbef2
comments
aakash-db May 29, 2025
71c69af
1
aakash-db May 30, 2025
41a0600
fix
aakash-db May 30, 2025
d350dbf
fix 2
aakash-db May 30, 2025
777189f
remove shim exclusion in spark core test dep
sryza Jun 1, 2025
4ac428c
1
aakash-db Jun 2, 2025
d887aea
remove param
aakash-db Jun 2, 2025
b4cbf08
Remove validateAppendOnceFlows
sryza May 22, 2025
6b20fa0
Clean up PipelinesErrors
sryza May 28, 2025
73a1df3
Remove weird Returns comments for lazy vals in DataflowGraph
sryza Jun 2, 2025
c09a644
Fix LoadTableException again
sryza Jun 2, 2025
a714d35
Add SchemaInferenceUtilsSuite
sryza Jun 3, 2025
7e40fa8
wenchen feedback and try to fix build errors
sryza Jun 3, 2025
ab007fe
another docs fix
sryza Jun 3, 2025
5515167
remove direct spark-core test jar dependency
sryza Jun 3, 2025
54690c7
fix SparkThrowableSuite
gengliangwang Jun 3, 2025
f4e3ecd
try more to fix the build
sryza Jun 3, 2025
5927325
more docs fix
sryza Jun 3, 2025
f310b4f
fix structured logging checks
sryza Jun 4, 2025
55ad0ab
more
sryza Jun 4, 2025
e8185b9
simplify build files for pipelines module
sryza Jun 4, 2025
5d53fd7
fix maven build
sryza Jun 4, 2025
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
71 changes: 71 additions & 0 deletions common/utils/src/main/resources/error/error-conditions.json
Original file line number Diff line number Diff line change
Expand Up @@ -82,6 +82,14 @@
],
"sqlState" : "XX000"
},
"APPEND_ONCE_FROM_BATCH_QUERY" : {
"message" : [
"Creating a streaming table from a batch query prevents incremental loading of new data from source. Offending table: '<table>'.",
"Please use the stream() operator. Example usage:",
"CREATE STREAMING TABLE <target table name> ... AS SELECT ... FROM stream(<source table name>) ..."
],
"sqlState" : "42000"
},
"ARITHMETIC_OVERFLOW" : {
"message" : [
"<message>.<alternative> If necessary set <config> to \"false\" to bypass this error."
Expand Down Expand Up @@ -1372,6 +1380,12 @@
},
"sqlState" : "42734"
},
"DUPLICATE_FLOW_SQL_CONF" : {
"message" : [
"Found duplicate sql conf for dataset '<datasetName>': '<key>' is defined by both '<flowName1>' and '<flowName2>'"
],
"sqlState" : "42710"
},
"DUPLICATE_KEY" : {
"message" : [
"Found duplicate keys <keyColumn>."
Expand Down Expand Up @@ -1943,6 +1957,12 @@
],
"sqlState" : "42818"
},
"INCOMPATIBLE_BATCH_VIEW_READ" : {
"message" : [
"View <datasetIdentifier> is a batch view and must be referenced using SparkSession#read. This check can be disabled by setting Spark conf pipelines.incompatibleViewCheck.enabled = false."
],
"sqlState" : "42000"
},
"INCOMPATIBLE_COLUMN_TYPE" : {
"message" : [
"<operator> can only be performed on tables with compatible column types. The <columnOrdinalNumber> column of the <tableOrdinalNumber> table is <dataType1> type which is not compatible with <dataType2> at the same column of the first table.<hint>."
Expand Down Expand Up @@ -2019,6 +2039,12 @@
],
"sqlState" : "42613"
},
"INCOMPATIBLE_STREAMING_VIEW_READ" : {
"message" : [
"View <datasetIdentifier> is a streaming view and must be referenced using SparkSession#readStream. This check can be disabled by setting Spark conf pipelines.incompatibleViewCheck.enabled = false."
],
"sqlState" : "42000"
},
"INCOMPATIBLE_VIEW_SCHEMA_CHANGE" : {
"message" : [
"The SQL query of view <viewName> has an incompatible schema change and column <colName> cannot be resolved. Expected <expectedNum> columns named <colName> but got <actualCols>.",
Expand Down Expand Up @@ -3119,6 +3145,12 @@
},
"sqlState" : "KD002"
},
"INVALID_NAME_IN_USE_COMMAND" : {
"message" : [
"Invalid name '<name>' in <command> command. Reason: <reason>"
],
"sqlState" : "42000"
},
"INVALID_NON_DETERMINISTIC_EXPRESSIONS" : {
"message" : [
"The operator expects a deterministic expression, but the actual expression is <sqlExprs>."
Expand Down Expand Up @@ -3384,6 +3416,12 @@
],
"sqlState" : "22023"
},
"INVALID_RESETTABLE_DEPENDENCY" : {
"message" : [
"Tables <upstreamResettableTables> are resettable but have a non-resettable downstream dependency '<downstreamTable>'. `reset` will fail as Spark Streaming does not support deleted source data. You can either remove the <resetAllowedKey>=false property from '<downstreamTable>' or add it to its upstream dependencies."
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
"Tables <upstreamResettableTables> are resettable but have a non-resettable downstream dependency '<downstreamTable>'. `reset` will fail as Spark Streaming does not support deleted source data. You can either remove the <resetAllowedKey>=false property from '<downstreamTable>' or add it to its upstream dependencies."
"Tables <upstreamResettableTables> are resettable but have a non-resettable downstream dependency '<downstreamTable>'. `reset` will fail as Spark Streaming does not support deleting source data. You can either remove the <resetAllowedKey>=false property from '<downstreamTable>' or add it to its upstream dependencies."

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think deleted is actually correct in this case. It's not that Spark Streaming would be trying to delete the source data. The problem is that upstream source data would be deleted (as part of the "reset"), but we wouldn't know how to handle those upstream deletes in downstream tables that are derived from the upstream tables.

],
"sqlState" : "42000"
},
"INVALID_RESET_COMMAND_FORMAT" : {
"message" : [
"Expected format is 'RESET' or 'RESET key'. If you want to include special characters in key, please use quotes, e.g., RESET `key`."
Expand Down Expand Up @@ -5419,6 +5457,19 @@
],
"sqlState" : "58030"
},
"UNABLE_TO_INFER_PIPELINE_TABLE_SCHEMA" : {
"message" : [
"Failed to infer the schema for table <tableName> from its upstream flows.",
"Please modify the flows that write to this table to make their schemas compatible.",
"",
"Inferred schema so far:",
"<inferredDataSchema>",
"",
"Incompatible schema:",
"<incompatibleDataSchema>"
],
"sqlState" : "42KD9"
},
"UNABLE_TO_INFER_SCHEMA" : {
"message" : [
"Unable to infer schema for <format>. It must be specified manually."
Expand Down Expand Up @@ -5590,6 +5641,12 @@
],
"sqlState" : "42883"
},
"UNRESOLVED_TABLE_PATH" : {
"message" : [
"Storage path for table <identifier> cannot be resolved."
],
"sqlState" : "22KD1"
},
"UNRESOLVED_USING_COLUMN_FOR_JOIN" : {
"message" : [
"USING column <colName> cannot be resolved on the <side> side of the join. The <side>-side columns: [<suggestion>]."
Expand Down Expand Up @@ -6571,6 +6628,20 @@
],
"sqlState" : "P0001"
},
"USER_SPECIFIED_AND_INFERRED_SCHEMA_NOT_COMPATIBLE" : {
"message" : [
"Table '<tableName>' has a user-specified schema that is incompatible with the schema",
"inferred from its query.",
"<streamingTableHint>",
"",
"Declared schema:",
"<specifiedSchema>",
"",
"Inferred schema:",
"<inferredDataSchema>"
],
"sqlState" : "42000"
},
"VARIABLE_ALREADY_EXISTS" : {
"message" : [
"Cannot create the variable <variableName> because it already exists.",
Expand Down
76 changes: 74 additions & 2 deletions sql/pipelines/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,8 @@
~ limitations under the License.
-->

<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What's the difference from before? Can we not modify this line?

xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<modelVersion>4.0.0</modelVersion>
<parent>
<groupId>org.apache.spark</groupId>
Expand All @@ -40,7 +41,6 @@
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
Expand All @@ -49,6 +49,78 @@
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
Copy link
Contributor

@LuciferYang LuciferYang Jun 4, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I changed

<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-core_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
to

        <dependency>
            <groupId>org.apache.spark</groupId>
            <artifactId>spark-core_${scala.binary.version}</artifactId>
            <version>${project.version}</version>
        </dependency>
        <dependency>
            <groupId>org.apache.spark</groupId>
            <artifactId>spark-core_${scala.binary.version}</artifactId>
            <version>${project.version}</version>
            <type>test-jar</type>
            <scope>test</scope>
        </dependency>

and the Maven compilation passed. However, when I executed command build/mvn test -pl sql/pipelines to run the Maven test, there were test failures:

ConnectValidPipelineSuite:
org.apache.spark.sql.pipelines.graph.ConnectValidPipelineSuite *** ABORTED ***
  java.lang.IllegalStateException: Cannot call methods on a stopped SparkContext.
This stopped SparkContext was created at:

org.apache.spark.sql.pipelines.graph.ConnectInvalidPipelineSuite.<init>(ConnectInvalidPipelineSuite.scala:30)
java.base/jdk.internal.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
java.base/jdk.internal.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:77)
java.base/jdk.internal.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
java.base/java.lang.reflect.Constructor.newInstanceWithCaller(Constructor.java:500)
java.base/java.lang.reflect.ReflectAccess.newInstance(ReflectAccess.java:128)
java.base/jdk.internal.reflect.ReflectionFactory.newInstance(ReflectionFactory.java:347)
java.base/java.lang.Class.newInstance(Class.java:645)
org.scalatest.tools.DiscoverySuite$.getSuiteInstance(DiscoverySuite.scala:66)
org.scalatest.tools.DiscoverySuite.$anonfun$nestedSuites$1(DiscoverySuite.scala:38)
scala.collection.immutable.Vector1.map(Vector.scala:2155)
scala.collection.immutable.Vector1.map(Vector.scala:386)
org.scalatest.tools.DiscoverySuite.<init>(DiscoverySuite.scala:37)
org.scalatest.tools.Runner$.genDiscoSuites$1(Runner.scala:1131)
org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1225)
org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:992)
org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:970)
org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1481)
org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:970)
org.scalatest.tools.Runner$.main(Runner.scala:775)

And it was stopped at:

org.apache.spark.sql.pipelines.utils.PipelineTest.afterAll(PipelineTest.scala:136)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Investigating now

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I looked into this more deeply. What appears to be happening is:

  • The test framework creates an instance of ConnectInvalidPipelineSuite
    • The initializer invokes SparkSession.builder.getOrCreate() to create a SparkSession and accompanying SparkContext
  • The test framework creates an instance of ConnectValidPipelineSuite
    • The initializer invokes SparkSession.builder.getOrCreate() to create a SparkSession. It is returned the same SparkSession and backing SparkContext that ConnectInvalidPipelineSuite references.
  • The test framework runs all tests inside ConnectInvalidPipelineSuite
  • The test framework invokes the teardown code in ConnectInvalidPipelineSuite, which stops its SparkSession and underlying SparkContext
  • The test framework tries to run beforeEach code in ConnectValidPipelineSuite, but the SparkSession it references is the same one that was referenced in ConnectInvalidPipelineSuite, and that has already been stopped

How does this work normally? There are several suites inside the repo that getOrCreate a SparkSession at instantiation time and stop it afterAll. Do they not run into this problem?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

diff --git a/sql/pipelines/pom.xml b/sql/pipelines/pom.xml
index 4ab7db079e4..5cbcebfbc8f 100644
--- a/sql/pipelines/pom.xml
+++ b/sql/pipelines/pom.xml
@@ -41,6 +41,12 @@
             <groupId>org.apache.spark</groupId>
             <artifactId>spark-core_${scala.binary.version}</artifactId>
             <version>${project.version}</version>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.spark</groupId>
+            <artifactId>spark-core_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
             <scope>test</scope>
         </dependency>
         <dependency>
diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala
index 01b2a91bb93..0d74146851c 100644
--- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala
+++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectInvalidPipelineSuite.scala
@@ -29,7 +29,6 @@ import org.apache.spark.sql.types.{IntegerType, StructType}
  */
 class ConnectInvalidPipelineSuite extends PipelineTest {
 
-  import originalSpark.implicits._
   test("Missing source") {
     class P extends TestGraphRegistrationContext(spark) {
       registerView("b", query = readFlowFunc("a"))
@@ -144,6 +143,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("Missing attribute in the schema") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1, 2, 3).toDF("z")))
       registerView("b", query = sqlFlowFunc(spark, "SELECT x FROM a"))
@@ -157,6 +158,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("Joining on a column with different names") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1, 2, 3).toDF("x")))
       registerView("b", query = dfFlowFunc(Seq("a", "b", "c").toDF("y")))
@@ -175,6 +178,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("Writing to one table by unioning flows with different schemas") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1, 2, 3).toDF("x")))
       registerView("b", query = dfFlowFunc(Seq(true, false).toDF("x")))
@@ -225,6 +230,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("Cyclic graph") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1, 2, 3).toDF("x")))
       registerView("b", query = sqlFlowFunc(spark, "SELECT * FROM a UNION SELECT * FROM d"))
@@ -246,6 +253,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("Cyclic graph with materialized nodes") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerTable("a", query = Option(dfFlowFunc(Seq(1, 2, 3).toDF("x"))))
       registerTable(
@@ -270,6 +279,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("Cyclic graph - second query makes it cyclic") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerTable("a", query = Option(dfFlowFunc(Seq(1, 2, 3).toDF("x"))))
       registerTable("b")
@@ -293,6 +304,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("Cyclic graph - all named queries") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerTable("a", query = Option(dfFlowFunc(Seq(1, 2, 3).toDF("x"))))
       registerTable("b")
@@ -317,6 +330,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("view-table conf conflict") {
+    val session = spark
+    import session.implicits._
     val p = new TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1).toDF()), sqlConf = Map("x" -> "a-val"))
       registerTable("b", query = Option(readFlowFunc("a")), sqlConf = Map("x" -> "b-val"))
@@ -338,6 +353,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("view-view conf conflict") {
+    val session = spark
+    import session.implicits._
     val p = new TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1).toDF()), sqlConf = Map("x" -> "a-val"))
       registerView("b", query = dfFlowFunc(Seq(1).toDF()), sqlConf = Map("x" -> "b-val"))
@@ -364,6 +381,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("reading a complete view incrementally") {
+    val session = spark
+    import session.implicits._
     val p = new TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1).toDF()))
       registerTable("b", query = Option(readStreamFlowFunc("a")))
@@ -380,6 +399,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("reading an incremental view completely") {
+    val session = spark
+    import session.implicits._
     val p = new TestGraphRegistrationContext(spark) {
       val mem = MemoryStream[Int]
       mem.addData(1)
@@ -398,6 +419,8 @@ class ConnectInvalidPipelineSuite extends PipelineTest {
   }
 
   test("Inferred schema that isn't a subset of user-specified schema") {
+    val session = spark
+    import session.implicits._
     val graph1 = new TestGraphRegistrationContext(spark) {
       registerTable(
         "a",
diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala
index f8b5133ff16..d64c5f06cdf 100644
--- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala
+++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/graph/ConnectValidPipelineSuite.scala
@@ -32,9 +32,9 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap
  */
 class ConnectValidPipelineSuite extends PipelineTest {
 
-  import originalSpark.implicits._
-
   test("Extra simple") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("b", query = dfFlowFunc(Seq(1, 2, 3).toDF("y")))
     }
@@ -44,6 +44,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Simple") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1, 2, 3).toDF("x")))
       registerView("b", query = sqlFlowFunc(spark, "SELECT x as y FROM a"))
@@ -65,6 +67,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Dependencies") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1, 2, 3).toDF("x")))
       registerView("c", query = sqlFlowFunc(spark, "SELECT y as z FROM b"))
@@ -78,6 +82,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Multi-hop schema merging") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView(
         "b",
@@ -93,6 +99,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Cross product join merges schema") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1, 2, 3).toDF("x")))
       registerView("b", query = dfFlowFunc(Seq(4, 5, 6).toDF("y")))
@@ -111,6 +119,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Real join merges schema") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq((1, "a"), (2, "b"), (3, "c")).toDF("x", "y")))
       registerView("b", query = dfFlowFunc(Seq((2, "m"), (3, "n"), (4, "o")).toDF("x", "z")))
@@ -132,6 +142,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Union of streaming and batch Dataframes") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       val ints = MemoryStream[Int]
       ints.addData(1, 2, 3, 4)
@@ -170,6 +182,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Union of two streaming Dataframes") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       val ints1 = MemoryStream[Int]
       ints1.addData(1, 2, 3, 4)
@@ -214,6 +228,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("MultipleInputs") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1, 2, 3).toDF("x")))
       registerView("b", query = dfFlowFunc(Seq(4, 5, 6).toDF("y")))
@@ -228,6 +244,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Connect retains and fuses confs") {
+    val session = spark
+    import session.implicits._
     // a -> b \
     //          d
     //      c /
@@ -250,6 +268,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Confs aren't fused past materialization points") {
+    val session = spark
+    import session.implicits._
     val p = new TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1).toDF("x")), Map("a" -> "a-val"))
       registerTable("b", query = Option(readFlowFunc("a")), Map("b" -> "b-val"))
@@ -276,6 +296,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Setting the same conf with the same value is totally cool") {
+    val session = spark
+    import session.implicits._
     val p = new TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1, 2, 3).toDF("x")), Map("key" -> "val"))
       registerView("b", query = dfFlowFunc(Seq(1, 2, 3).toDF("x")), Map("key" -> "val"))
@@ -290,6 +312,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Named query only") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       registerView("a", query = dfFlowFunc(Seq(1, 2, 3).toDF("x")))
       registerTable("b")
@@ -308,6 +332,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Default query and named query") {
+    val session = spark
+    import session.implicits._
     class P extends TestGraphRegistrationContext(spark) {
       val mem = MemoryStream[Int]
       registerView("a", query = dfFlowFunc(mem.toDF()))
@@ -348,6 +374,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Correct types of flows after connection") {
+    val session = spark
+    import session.implicits._
     val graph = new TestGraphRegistrationContext(spark) {
       val mem = MemoryStream[Int]
       mem.addData(1, 2)
@@ -402,6 +430,8 @@ class ConnectValidPipelineSuite extends PipelineTest {
   }
 
   test("Pipeline level default spark confs are applied with correct precedence") {
+    val session = spark
+    import session.implicits._
     val P = new TestGraphRegistrationContext(
       spark,
       Map("default.conf" -> "value")
diff --git a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/PipelineTest.scala b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/PipelineTest.scala
index 981fa3cdcae..9039cc0e338 100644
--- a/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/PipelineTest.scala
+++ b/sql/pipelines/src/test/scala/org/apache/spark/sql/pipelines/utils/PipelineTest.scala
@@ -31,7 +31,7 @@ import org.scalatest.matchers.should.Matchers
 import org.apache.spark.{SparkConf, SparkFunSuite}
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.{Column, QueryTest, Row, TypedColumn}
-import org.apache.spark.sql.SparkSession.{clearActiveSession, setActiveSession}
+import org.apache.spark.sql.SparkSession.{clearActiveSession, clearDefaultSession, setActiveSession, setDefaultSession}
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.classic.{DataFrame, Dataset, SparkSession, SQLContext}
 import org.apache.spark.sql.execution._
@@ -48,8 +48,7 @@ abstract class PipelineTest
 
   final protected val storageRoot = createTempDir()
 
-  var spark: SparkSession = createAndInitializeSpark()
-  val originalSpark: SparkSession = spark.cloneSession()
+  var spark: SparkSession = _
 
   implicit def sqlContext: SQLContext = spark.sqlContext
   def sql(text: String): DataFrame = spark.sql(text)
@@ -111,8 +110,23 @@ abstract class PipelineTest
   /** Set up the spark session before each test. */
   protected def initializeSparkBeforeEachTest(): Unit = {
     clearActiveSession()
-    spark = originalSpark.newSession()
+    clearDefaultSession()
+    spark = spark.newSession()
     setActiveSession(spark)
+    setDefaultSession(spark)
+  }
+
+  protected def initializeSession(): Unit = {
+    if (spark == null) {
+      spark = createAndInitializeSpark()
+    }
+  }
+
+  protected override def beforeAll(): Unit = {
+    initializeSession()
+
+    // Ensure we have initialized the context before calling parent code
+    super.beforeAll()
   }
 
   override def beforeEach(): Unit = {
@@ -133,7 +147,23 @@ abstract class PipelineTest
   }
 
   override def afterAll(): Unit = {
-    spark.stop()
+    try {
+      super.afterAll()
+    } finally {
+      try {
+        if (spark != null) {
+          try {
+            spark.sessionState.catalog.reset()
+          } finally {
+            spark.stop()
+            spark = null
+          }
+        }
+      } finally {
+        clearActiveSession()
+        clearDefaultSession()
+      }
+    }
   }
 
   protected def gridTest[A](testNamePrefix: String, testTags: Tag*)(params: Seq[A])(

I made some modifications to PipelineTest by referring to SharedSparkSession as above. Then I executed build/mvn clean install -pl sql/pipelines, and the compilation & testing were successful. The aforementioned modifications are for reference only.

image

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for all the attention you're giving on this. I independently converged on a similar solution actually. I'm still a little bit confused about how this works for other tests that create a SparkSession more globally, like in https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SqlBasedBenchmark.scala.

I pushed this change though. Hopefully adding the spark-core_ back in doesn't bring back the duplicate dependencies issue we were running into before 🤞 .

Copy link
Contributor

@sryza sryza Jun 4, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

(I also pulled in the pom.xml and SparkBuild changes from the downstream PR)

<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-catalyst_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-sql-api_${scala.binary.version}</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
<scope>test</scope>
<exclusions>
<exclusion>
<groupId>org.apache.spark</groupId>
<artifactId>spark-connect-shims_${scala.binary.version}</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.scala-lang.modules</groupId>
<artifactId>scala-parallel-collections_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>org.scalacheck</groupId>
<artifactId>scalacheck_${scala.binary.version}</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.mockito</groupId>
<artifactId>mockito-core</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>net.bytebuddy</groupId>
<artifactId>byte-buddy</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>net.bytebuddy</groupId>
<artifactId>byte-buddy-agent</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-tags_${scala.binary.version}</artifactId>
</dependency>

<!--
This spark-tags test-dep is needed even though it isn't used in this module, otherwise testing-cmds that exclude
them will yield errors.
-->
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-tags_${scala.binary.version}</artifactId>
<type>test-jar</type>
<scope>test</scope>
</dependency>

</dependencies>
<build>
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,33 @@
/*
* 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.spark.sql.pipelines

/** Represents a warning generated as part of graph analysis. */
sealed trait AnalysisWarning

object AnalysisWarning {

/**
* Warning that some streaming reader options are being dropped
*
* @param sourceName Source for which reader options are being dropped.
* @param droppedOptions Set of reader options that are being dropped for a specific source.
*/
case class StreamingReaderOptionsDropped(sourceName: String, droppedOptions: Seq[String])
extends AnalysisWarning
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* 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.spark.sql.pipelines

sealed trait Language {}

object Language {
case class Python() extends Language {}
case class Sql() extends Language {}
}

Loading