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-19168][Structured Streaming] StateStore should be aborted upon error #16547

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -31,6 +31,7 @@ import org.apache.spark.sql.execution.streaming.state._
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType
import org.apache.spark.TaskContext


/** Used to identify the state store for a given operator. */
Expand Down Expand Up @@ -150,6 +151,13 @@ case class StateStoreSaveExec(
val numTotalStateRows = longMetric("numTotalStateRows")
val numUpdatedStateRows = longMetric("numUpdatedStateRows")

// Abort the state store in case of error
TaskContext.get().addTaskCompletionListener(_ => {
if (!store.hasCommitted) {
store.abort()
}
})

outputMode match {
// Update and output all rows in the StateStore.
case Some(Complete) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -203,7 +203,7 @@ private[state] class HDFSBackedStateStoreProvider(
/**
* Whether all updates have been committed
*/
override private[state] def hasCommitted: Boolean = {
override private[streaming] def hasCommitted: Boolean = {
state == COMMITTED
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,7 @@ trait StateStore {
/**
* Whether all updates have been committed
*/
private[state] def hasCommitted: Boolean
private[streaming] def hasCommitted: Boolean
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.streaming.state
import org.apache.spark.sql.internal.SQLConf

/** A class that contains configuration parameters for [[StateStore]]s. */
private[streaming] class StateStoreConf(@transient private val conf: SQLConf) extends Serializable {
private[sql] class StateStoreConf(@transient private val conf: SQLConf) extends Serializable {

def this() = this(new SQLConf)

Expand All @@ -29,7 +29,7 @@ private[streaming] class StateStoreConf(@transient private val conf: SQLConf) ex
val minVersionsToRetain = conf.minBatchesToRetain
}

private[streaming] object StateStoreConf {
private[sql] object StateStoreConf {
val empty = new StateStoreConf()

def apply(conf: SQLConf): StateStoreConf = new StateStoreConf(conf)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,17 +19,26 @@ package org.apache.spark.sql.streaming

import java.util.TimeZone

import scala.collection.mutable
import scala.reflect.runtime.{universe => ru}

import org.apache.hadoop.conf.Configuration
import org.mockito.Mockito
import org.mockito.invocation.InvocationOnMock
import org.mockito.stubbing.Answer
import org.scalatest.BeforeAndAfterAll
import org.scalatest.PrivateMethodTester._

import org.apache.spark.SparkException
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.state.StateStore
import org.apache.spark.sql.execution.streaming.state._
import org.apache.spark.sql.expressions.scalalang.typed
import org.apache.spark.sql.functions._
import org.apache.spark.sql.streaming.OutputMode._
import org.apache.spark.sql.types._

object FailureSinglton {
var firstTime = true
Expand Down Expand Up @@ -335,4 +344,67 @@ class StreamingAggregationSuite extends StreamTest with BeforeAndAfterAll {
CheckLastBatch((90L, 1), (100L, 1), (105L, 1))
)
}

Copy link
Contributor Author

@lw-lin lw-lin Jan 17, 2017

Choose a reason for hiding this comment

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

The idea of the following test is to inject a mock state store, which:

  • throws an error in its commit() method
  • marks the aborted flag in its abort() method

Then at the end of the test, we would check if aborted is true

Copy link
Member

Choose a reason for hiding this comment

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

I prefer to not add this complicated test because:

  • TaskContext.get().addTaskCompletionListener has already been tested in other tests.
  • Scala Reflection is not thread-safe in 2.10 and this test may be flaky since SQL also uses Scala Reflection.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

thanks! I hesitated a little when I added it -- yea it was complicated -- so let me remove it

test("abort StateStore in case of error") {
quietly {
val inputData = MemoryStream[Long]
val aggregated =
inputData.toDS()
.groupBy($"value")
.agg(count("*"))
var aborted = false
testStream(aggregated, Complete)(
// This whole `AssertOnQuery` is used to inject a mock state store
AssertOnQuery(execution => {
// (1) Use reflection to get `StateStore.loadedProviders`
val loadedProviders = {
val field = ru.typeOf[StateStore.type].decl(ru.TermName("loadedProviders")).asTerm
ru.runtimeMirror(StateStore.getClass.getClassLoader)
.reflect(StateStore)
.reflectField(field)
.get
.asInstanceOf[mutable.HashMap[StateStoreId, StateStoreProvider]]
}
// (2) Make a storeId
val storeId = {
val checkpointLocation =
execution invokePrivate PrivateMethod[String]('checkpointFile)("state")
StateStoreId(checkpointLocation, 0L, 0)
}
// (3) Make `mockStore` and `mockProvider`
val (mockStore, mockProvider) = {
val keySchema = StructType(Seq(
StructField("value", LongType, false)))
val valueSchema = StructType(Seq(
StructField("value", LongType, false), StructField("count", LongType, false)))
val storeConf = StateStoreConf.empty
val hadoopConf = new Configuration
(Mockito.spy(
StateStore.get(storeId, keySchema, valueSchema, version = 0, storeConf, hadoopConf)),
Mockito.spy(loadedProviders.get(storeId).get))
}
// (4) Setup `mockStore` and `mockProvider`
Mockito.doAnswer(new Answer[Long] {
override def answer(invocationOnMock: InvocationOnMock): Long = {
sys.error("injected error on commit()")
}
}).when(mockStore).commit()
Mockito.doAnswer(new Answer[Unit] {
override def answer(invocationOnMock: InvocationOnMock): Unit = {
invocationOnMock.callRealMethod()
// Mark the flag for later check
aborted = true
}
}).when(mockStore).abort()
Mockito.doReturn(mockStore).when(mockProvider).getStore(version = 0)
// (5) Inject `mockProvider`, which later on would inject `mockStore`
loadedProviders.put(storeId, mockProvider)
true
}), // End of AssertOnQuery, i.e. end of injecting `mockStore`
AddData(inputData, 1L, 2L, 3L),
ExpectFailure[SparkException](),
AssertOnQuery { _ => aborted } // Check that `mockStore.abort()` is called upon error
)
}
}
}