Skip to content
Snippets Groups Projects
Commit 4e3685ae authored by Shixiong Zhu's avatar Shixiong Zhu Committed by Michael Armbrust
Browse files

[SPARK-15077][SQL] Use a fair lock to avoid thread starvation in StreamExecution

## What changes were proposed in this pull request?

Right now `StreamExecution.awaitBatchLock` uses an unfair lock. `StreamExecution.awaitOffset` may run too long and fail some test because `StreamExecution.constructNextBatch` keeps getting the lock.

See: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.4/865/testReport/junit/org.apache.spark.sql.streaming/FileStreamSourceStressTestSuite/file_source_stress_test/

This PR uses a fair ReentrantLock to resolve the thread starvation issue.

## How was this patch tested?

Modified `FileStreamSourceStressTestSuite.test("file source stress test")` to run the test codes 100 times locally. It always fails because of timeout without this patch.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #12852 from zsxwing/SPARK-15077.
parent 0fd95be3
No related branches found
No related tags found
No related merge requests found
...@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.streaming ...@@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.streaming
import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.{CountDownLatch, TimeUnit}
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.locks.ReentrantLock
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import scala.util.control.NonFatal import scala.util.control.NonFatal
...@@ -53,8 +54,12 @@ class StreamExecution( ...@@ -53,8 +54,12 @@ class StreamExecution(
val trigger: Trigger) val trigger: Trigger)
extends ContinuousQuery with Logging { extends ContinuousQuery with Logging {
/** An monitor used to wait/notify when batches complete. */ /**
private val awaitBatchLock = new Object * A lock used to wait/notify when batches complete. Use a fair lock to avoid thread starvation.
*/
private val awaitBatchLock = new ReentrantLock(true)
private val awaitBatchLockCondition = awaitBatchLock.newCondition()
private val startLatch = new CountDownLatch(1) private val startLatch = new CountDownLatch(1)
private val terminationLatch = new CountDownLatch(1) private val terminationLatch = new CountDownLatch(1)
...@@ -242,17 +247,22 @@ class StreamExecution( ...@@ -242,17 +247,22 @@ class StreamExecution(
// method. See SPARK-14131. // method. See SPARK-14131.
// //
// Check to see what new data is available. // Check to see what new data is available.
val hasNewData = awaitBatchLock.synchronized { val hasNewData = {
val newData = microBatchThread.runUninterruptibly { awaitBatchLock.lock()
uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) try {
} val newData = microBatchThread.runUninterruptibly {
availableOffsets ++= newData uniqueSources.flatMap(s => s.getOffset.map(o => s -> o))
}
availableOffsets ++= newData
if (dataAvailable) { if (dataAvailable) {
true true
} else { } else {
noNewData = true noNewData = true
false false
}
} finally {
awaitBatchLock.unlock()
} }
} }
if (hasNewData) { if (hasNewData) {
...@@ -269,9 +279,12 @@ class StreamExecution( ...@@ -269,9 +279,12 @@ class StreamExecution(
currentBatchId += 1 currentBatchId += 1
logInfo(s"Committed offsets for batch $currentBatchId.") logInfo(s"Committed offsets for batch $currentBatchId.")
} else { } else {
awaitBatchLock.synchronized { awaitBatchLock.lock()
try {
// Wake up any threads that are waiting for the stream to progress. // Wake up any threads that are waiting for the stream to progress.
awaitBatchLock.notifyAll() awaitBatchLockCondition.signalAll()
} finally {
awaitBatchLock.unlock()
} }
} }
} }
...@@ -332,9 +345,12 @@ class StreamExecution( ...@@ -332,9 +345,12 @@ class StreamExecution(
new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema))
sink.addBatch(currentBatchId - 1, nextBatch) sink.addBatch(currentBatchId - 1, nextBatch)
awaitBatchLock.synchronized { awaitBatchLock.lock()
try {
// Wake up any threads that are waiting for the stream to progress. // Wake up any threads that are waiting for the stream to progress.
awaitBatchLock.notifyAll() awaitBatchLockCondition.signalAll()
} finally {
awaitBatchLock.unlock()
} }
val batchTime = (System.nanoTime() - startTime).toDouble / 1000000 val batchTime = (System.nanoTime() - startTime).toDouble / 1000000
...@@ -374,8 +390,12 @@ class StreamExecution( ...@@ -374,8 +390,12 @@ class StreamExecution(
} }
while (notDone) { while (notDone) {
logInfo(s"Waiting until $newOffset at $source") awaitBatchLock.lock()
awaitBatchLock.synchronized { awaitBatchLock.wait(100) } try {
awaitBatchLockCondition.await(100, TimeUnit.MILLISECONDS)
} finally {
awaitBatchLock.unlock()
}
} }
logDebug(s"Unblocked at $newOffset for $source") logDebug(s"Unblocked at $newOffset for $source")
} }
...@@ -383,16 +403,21 @@ class StreamExecution( ...@@ -383,16 +403,21 @@ class StreamExecution(
/** A flag to indicate that a batch has completed with no new data available. */ /** A flag to indicate that a batch has completed with no new data available. */
@volatile private var noNewData = false @volatile private var noNewData = false
override def processAllAvailable(): Unit = awaitBatchLock.synchronized { override def processAllAvailable(): Unit = {
noNewData = false awaitBatchLock.lock()
while (true) { try {
awaitBatchLock.wait(10000) noNewData = false
if (streamDeathCause != null) { while (true) {
throw streamDeathCause awaitBatchLockCondition.await(10000, TimeUnit.MILLISECONDS)
} if (streamDeathCause != null) {
if (noNewData) { throw streamDeathCause
return }
if (noNewData) {
return
}
} }
} finally {
awaitBatchLock.unlock()
} }
} }
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment