Skip to content

[SPARK-34482][SS] Correct the active SparkSession for StreamExecution.logicalPlan #31600

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 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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
Original file line number Diff line number Diff line change
Expand Up @@ -323,26 +323,28 @@ abstract class StreamExecution(
startLatch.countDown()

// While active, repeatedly attempt to run batches.
SparkSession.setActiveSession(sparkSession)

updateStatusMessage("Initializing sources")
// force initialization of the logical plan so that the sources can be created
logicalPlan

// Adaptive execution can change num shuffle partitions, disallow
sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false")
// Disable cost-based join optimization as we do not want stateful operations to be rearranged
sparkSessionForStream.conf.set(SQLConf.CBO_ENABLED.key, "false")
offsetSeqMetadata = OffsetSeqMetadata(
batchWatermarkMs = 0, batchTimestampMs = 0, sparkSessionForStream.conf)

if (state.compareAndSet(INITIALIZING, ACTIVE)) {
// Unblock `awaitInitialization`
initializationLatch.countDown()
runActivatedStream(sparkSessionForStream)
updateStatusMessage("Stopped")
} else {
// `stop()` is already called. Let `finally` finish the cleanup.
sparkSessionForStream.withActive {
Copy link
Member Author

Choose a reason for hiding this comment

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

Using withActive to keep safe for the StreamExecution to avoid using the wrong active session.

// Adaptive execution can change num shuffle partitions, disallow
sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false")
// Disable cost-based join optimization as we do not want stateful operations
// to be rearranged
sparkSessionForStream.conf.set(SQLConf.CBO_ENABLED.key, "false")

updateStatusMessage("Initializing sources")
// force initialization of the logical plan so that the sources can be created
logicalPlan

offsetSeqMetadata = OffsetSeqMetadata(
batchWatermarkMs = 0, batchTimestampMs = 0, sparkSessionForStream.conf)

if (state.compareAndSet(INITIALIZING, ACTIVE)) {
// Unblock `awaitInitialization`
initializationLatch.countDown()
runActivatedStream(sparkSessionForStream)
updateStatusMessage("Stopped")
} else {
// `stop()` is already called. Let `finally` finish the cleanup.
}
}
} catch {
case e if isInterruptedByStop(e, sparkSession.sparkContext) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ import org.scalatest.time.SpanSugar._
import org.apache.spark.{SparkConf, SparkContext, TaskContext, TestUtils}
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.plans.logical.Range
import org.apache.spark.sql.catalyst.plans.logical.{Range, RepartitionByExpression}
import org.apache.spark.sql.catalyst.streaming.{InternalOutputModes, StreamingRelationV2}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.{LocalLimitExec, SimpleMode, SparkPlan}
Expand Down Expand Up @@ -1266,6 +1266,37 @@ class StreamSuite extends StreamTest {
}
}
}

test("SPARK-34482: correct active SparkSession for logicalPlan") {
withSQLConf(
SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true",
SQLConf.COALESCE_PARTITIONS_INITIAL_PARTITION_NUM.key -> "10") {
val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load()
var query: StreamExecution = null
try {
query =
df.repartition($"a")
.writeStream
.format("memory")
.queryName("memory")
.start()
.asInstanceOf[StreamingQueryWrapper]
.streamingQuery
query.awaitInitialization(streamingTimeout.toMillis)
val plan = query.logicalPlan
val numPartition = plan
.find { _.isInstanceOf[RepartitionByExpression] }
.map(_.asInstanceOf[RepartitionByExpression].numPartitions)
// Before the fix of SPARK-34482, the numPartition is the value of
// `COALESCE_PARTITIONS_INITIAL_PARTITION_NUM`.
assert(numPartition.get === spark.sqlContext.conf.getConf(SQLConf.SHUFFLE_PARTITIONS))
} finally {
if (query != null) {
query.stop()
}
}
}
}
}

abstract class FakeSource extends StreamSourceProvider {
Expand Down