Skip to content

[SPARK-20883][SPARK-20376][SS] Refactored StateStore APIs and added conf to choose implementation#18107

Closed
tdas wants to merge 6 commits intoapache:masterfrom
tdas:SPARK-20376
Closed

[SPARK-20883][SPARK-20376][SS] Refactored StateStore APIs and added conf to choose implementation#18107
tdas wants to merge 6 commits intoapache:masterfrom
tdas:SPARK-20376

Conversation

@tdas
Copy link
Copy Markdown
Contributor

@tdas tdas commented May 25, 2017

What changes were proposed in this pull request?

A bunch of changes to the StateStore APIs and implementation.
Current state store API has a bunch of problems that causes too many transient objects causing memory pressure.

  • StateStore.get(): Option forces creation of Some/None objects for every get. Changed this to return the row or null.
  • StateStore.iterator(): (UnsafeRow, UnsafeRow) forces creation of new tuple for each record returned. Changed this to return a UnsafeRowTuple which can be reused across records.
  • StateStore.updates() requires the implementation to keep track of updates, while this is used minimally (only by Append mode in streaming aggregations). Removed updates() and updated StateStoreSaveExec accordingly.
  • StateStore.filter(condition) and StateStore.remove(condition) has been merge into a single API getRange(start, end) which allows a state store to do optimized range queries (i.e. avoid full scans). Stateful operators have been updated accordingly.
  • Removed a lot of unnecessary row copies Each operator copied rows before calling StateStore.put() even if the implementation does not require it to be copied. It is left up to the implementation on whether to copy the row or not.

Additionally,

  • Added a name to the StateStoreId so that each operator+partition can use multiple state stores (different names)
  • Added a configuration that allows the user to specify which implementation to use.
  • Added new metrics to understand the time taken to update keys, remove keys and commit all changes to the state store. These metrics will be visible on the plan diagram in the SQL tab of the UI.
  • Refactored unit tests such that they can be reused to test any implementation of StateStore.

How was this patch tested?

Old and new unit tests

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

update this description

@SparkQA
Copy link
Copy Markdown

SparkQA commented May 25, 2017

Test build #77362 has finished for PR 18107 at commit 03f5bf3.

  • This patch fails build dependency tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • case class StateStoreId(
  • case class StateStoreStats()
  • case class UnsafeRowTuple(var key: UnsafeRow = null, var value: UnsafeRow = null)
  • trait StateStoreWriter extends StatefulOperator

partitionId: Int,
name: String = "")

case class StateStoreStats()
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

remove this

*/
def remove(key: UnsafeRow): Unit

def getRange(start: Option[UnsafeRow], end: Option[UnsafeRow]): Iterator[UnsafeRowTuple]
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

add docs

def key: UnsafeRow
def value: UnsafeRow
object StateStoreProvider {
def instantiate(
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

add docs.

val key = getKey(row)
store.put(key.copy(), row.copy())
numUpdatedStateRows += 1
allUpdatesTimeMs += timeTakenMs {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

this update is to accommodate for removal of StateStore.updates()

@SparkQA
Copy link
Copy Markdown

SparkQA commented May 25, 2017

Test build #3755 has finished for PR 18107 at commit d645b41.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • case class StateStoreId(
  • case class StateStoreStats()
  • case class UnsafeRowTuple(var key: UnsafeRow = null, var value: UnsafeRow = null)
  • trait StateStoreWriter extends StatefulOperator

@SparkQA
Copy link
Copy Markdown

SparkQA commented May 25, 2017

Test build #77363 has finished for PR 18107 at commit d645b41.

  • This patch fails Spark unit tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • case class StateStoreId(
  • case class StateStoreStats()
  • case class UnsafeRowTuple(var key: UnsafeRow = null, var value: UnsafeRow = null)
  • trait StateStoreWriter extends StatefulOperator

expectedState = Some(5), // state should change
expectedTimeoutTimestamp = 5000) // timestamp should change

test("StateStoreUpdater - rows are cloned before writing to StateStore") {
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

This is not needed any more as the operator is not responsible for cloning the rows when writing to the store.

@SparkQA
Copy link
Copy Markdown

SparkQA commented May 25, 2017

Test build #77386 has finished for PR 18107 at commit 324fc24.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link
Copy Markdown

SparkQA commented May 26, 2017

Test build #77402 has finished for PR 18107 at commit 3e49621.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • case class StateStoreId(
  • case class UnsafeRowPair(var key: UnsafeRow = null, var value: UnsafeRow = null)

Copy link
Copy Markdown
Member

@zsxwing zsxwing left a comment

Choose a reason for hiding this comment

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

Looks pretty good. My major comment is Prefer nanoTime over currentTimeMillis


def optimizerInSetConversionThreshold: Int = getConf(OPTIMIZER_INSET_CONVERSION_THRESHOLD)

def stateStoreProviderClass: Option[String] = getConf(STATE_STORE_PROVIDER_CLASS)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Also add this to StateStoreConf for consistency?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Good idea.


/** Records the duration of running `body` for the next query progress update. */
protected def timeTakenMs(body: => Unit): Long = {
val startTime = System.currentTimeMillis
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: Use nanoTime instead

.internal()
.doc(
"The class used to manage state data in stateful streaming queries. This class must" +
"be a subclass of StateStoreProvider, and must have a zero-arg constructor.")
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: missing space before be.

storeConfs: StateStoreConf,
hadoopConf: Configuration): Unit = {
throw new Exception("Successfully instantiated")

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: extra empty line.

indexOrdinal: Option[Int], // for sorting the data
storeConf: StateStoreConf,
hadoopConf: Configuration): StateStoreProvider = {
val provider = Utils.getContextOrSparkClassLoader
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: Use Utils.classForName(providerClass).

// Update and output modified rows from the StateStore.
case Some(Update) =>

val updatesStartTimeMs = System.currentTimeMillis
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: please use nanoTime

override def output: Seq[Attribute] = child.output

override def outputPartitioning: Partitioning = child.outputPartitioning

Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: extra empty lines

case None => iter
}

val updatesStartTimeMs = System.currentTimeMillis
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: please use nanoTime

CompletionIterator[InternalRow, Iterator[InternalRow]](result, {
watermarkPredicateForKeys.foreach(f => store.remove(f.eval _))
store.commit()
allUpdatesTimeMs += System.currentTimeMillis - updatesStartTimeMs
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

nit: please use nanoTime

override protected def getNext(): InternalRow = {
var removedValueRow: InternalRow = null
while(rangeIter.hasNext && removedValueRow == null) {
val UnsafeRowPair(keyRow, valueRow) = rangeIter.next()
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Case class's unapply will create a Tuple. You should not use this Scala syntactic sugar :)

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

That is true! I had assumed unapply will get desugared into something simple, but its probably best to not to rely on the Scala compiler so much.

@zsxwing
Copy link
Copy Markdown
Member

zsxwing commented May 30, 2017

LGTM pending tests.

@SparkQA
Copy link
Copy Markdown

SparkQA commented May 30, 2017

Test build #77546 has finished for PR 18107 at commit baba63d.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds the following public classes (experimental):
  • class UnsafeRowPair(var key: UnsafeRow = null, var value: UnsafeRow = null)

@SparkQA
Copy link
Copy Markdown

SparkQA commented May 30, 2017

Test build #77547 has finished for PR 18107 at commit 5c0961c.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link
Copy Markdown

SparkQA commented May 30, 2017

Test build #77549 has finished for PR 18107 at commit fdfdcab.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@zsxwing
Copy link
Copy Markdown
Member

zsxwing commented May 30, 2017

Thanks! Merging to master.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants