Skip to content
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

[SPARK-38723][SS][TESTS] Add test for streaming query resume race condition #43405

Closed
wants to merge 1 commit into from
Closed
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 @@ -25,6 +25,7 @@ import java.util.{Locale, Properties, ServiceConfigurationError}
import org.apache.hadoop.fs.{LocalFileSystem, Path}
import org.apache.hadoop.fs.permission.FsPermission
import org.mockito.Mockito.{mock, spy, when}
import org.scalatest.time.SpanSugar._

import org.apache.spark._
import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, QueryTest, Row, SaveMode}
Expand All @@ -49,6 +50,7 @@ import org.apache.spark.sql.jdbc.{JdbcDialect, JdbcDialects}
import org.apache.spark.sql.streaming.StreamingQueryException
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{DataType, DecimalType, LongType, MetadataBuilder, StructType}
import org.apache.spark.util.ThreadUtils
import org.apache.spark.util.Utils

class QueryExecutionErrorsSuite
Expand Down Expand Up @@ -876,6 +878,52 @@ class QueryExecutionErrorsSuite
assert(e.getCause.isInstanceOf[NullPointerException])
}

test("CONCURRENT_QUERY: streaming query is resumed from many sessions") {
failAfter(90 seconds) {
withSQLConf(SQLConf.STREAMING_STOP_ACTIVE_RUN_ON_RESTART.key -> "true") {
withTempDir { dir =>
val ds = spark.readStream.format("rate").load()

// Queries have the same ID when they are resumed from the same checkpoint.
val chkLocation = new File(dir, "_checkpoint").getCanonicalPath
val dataLocation = new File(dir, "data").getCanonicalPath

// Run an initial query to setup the checkpoint.
val initialQuery = ds.writeStream.format("parquet")
.option("checkpointLocation", chkLocation).start(dataLocation)

// Error is thrown due to a race condition. Ensure it happens with high likelihood in the
// test by spawning many threads.
val exceptions = ThreadUtils.parmap(Seq.range(1, 50), "QueryExecutionErrorsSuite", 50)
{ _ =>
var exception = None : Option[SparkConcurrentModificationException]
try {
val restartedQuery = ds.writeStream.format("parquet")
.option("checkpointLocation", chkLocation).start(dataLocation)
restartedQuery.stop()
restartedQuery.awaitTermination()
} catch {
case e: SparkConcurrentModificationException =>
exception = Some(e)
}
exception
}
assert(exceptions.map(e => e.isDefined).reduceLeft(_ || _))
exceptions.map { e =>
if (e.isDefined) {
checkError(
e.get,
errorClass = "CONCURRENT_QUERY",
sqlState = Some("0A000")
)
}
}
spark.streams.active.foreach(_.stop())
}
}
}
}

test("UNSUPPORTED_EXPR_FOR_WINDOW: to_date is not supported with WINDOW") {
withTable("t") {
sql("CREATE TABLE t(c String) USING parquet")
Expand Down