Skip to content

[SPARK-24763][SS] Remove redundant key data from value in streaming aggregation #21733

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 13 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 @@ -871,6 +871,16 @@ object SQLConf {
.intConf
.createWithDefault(2)

val STREAMING_AGGREGATION_STATE_FORMAT_VERSION =
buildConf("spark.sql.streaming.aggregation.stateFormatVersion")
.internal()
.doc("State format version used by streaming aggregation operations in a streaming query. " +
"State between versions are tend to be incompatible, so state format version shouldn't " +
"be modified after running.")
.intConf
.checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2")
.createWithDefault(2)
Copy link
Contributor

Choose a reason for hiding this comment

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

If you intend to change the default to the new version, then you HAVE TO add a test that ensures that existing streaming aggregation checkpoints (generated in Spark 2.3.1 for example) will not fail to recover.

Similar to this test - https://github.com/apache/spark/blob/master/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala#L883

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Nice suggestion. Will add the test.


val UNSUPPORTED_OPERATION_CHECK_ENABLED =
buildConf("spark.sql.streaming.unsupportedOperationCheck")
.internal()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -328,10 +328,13 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
"Streaming aggregation doesn't support group aggregate pandas UDF")
}

val stateVersion = conf.getConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION)

aggregate.AggUtils.planStreamingAggregation(
namedGroupingExpressions,
aggregateExpressions.map(expr => expr.asInstanceOf[AggregateExpression]),
rewrittenResultExpressions,
stateVersion,
planLater(child))

case _ => Nil
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -256,6 +256,7 @@ object AggUtils {
groupingExpressions: Seq[NamedExpression],
functionsWithoutDistinct: Seq[AggregateExpression],
resultExpressions: Seq[NamedExpression],
stateFormatVersion: Int,
child: SparkPlan): Seq[SparkPlan] = {

val groupingAttributes = groupingExpressions.map(_.toAttribute)
Expand Down Expand Up @@ -287,7 +288,8 @@ object AggUtils {
child = partialAggregate)
}

val restored = StateStoreRestoreExec(groupingAttributes, None, partialMerged1)
val restored = StateStoreRestoreExec(groupingAttributes, None, stateFormatVersion,
partialMerged1)

val partialMerged2: SparkPlan = {
val aggregateExpressions = functionsWithoutDistinct.map(_.copy(mode = PartialMerge))
Expand All @@ -311,6 +313,7 @@ object AggUtils {
stateInfo = None,
outputMode = None,
eventTimeWatermark = None,
stateFormatVersion = stateFormatVersion,
partialMerged2)

val finalAndCompleteAggregate: SparkPlan = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -100,19 +100,21 @@ class IncrementalExecution(
val state = new Rule[SparkPlan] {

override def apply(plan: SparkPlan): SparkPlan = plan transform {
case StateStoreSaveExec(keys, None, None, None,
case StateStoreSaveExec(keys, None, None, None, stateFormatVersion,
UnaryExecNode(agg,
StateStoreRestoreExec(_, None, child))) =>
StateStoreRestoreExec(_, None, _, child))) =>
val aggStateInfo = nextStatefulOperationStateInfo
StateStoreSaveExec(
keys,
Some(aggStateInfo),
Some(outputMode),
Some(offsetSeqMetadata.batchWatermarkMs),
stateFormatVersion,
agg.withNewChildren(
StateStoreRestoreExec(
keys,
Some(aggStateInfo),
stateFormatVersion,
child) :: Nil))

case StreamingDeduplicateExec(keys, child, None, None) =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.json4s.jackson.Serialization

import org.apache.spark.internal.Logging
import org.apache.spark.sql.RuntimeConfig
import org.apache.spark.sql.execution.streaming.state.FlatMapGroupsWithStateExecHelper
import org.apache.spark.sql.execution.streaming.state.{FlatMapGroupsWithStateExecHelper, StreamingAggregationStateManager}
import org.apache.spark.sql.internal.SQLConf.{FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION, _}

/**
Expand Down Expand Up @@ -89,7 +89,7 @@ object OffsetSeqMetadata extends Logging {
private implicit val format = Serialization.formats(NoTypeHints)
private val relevantSQLConfs = Seq(
SHUFFLE_PARTITIONS, STATE_STORE_PROVIDER_CLASS, STREAMING_MULTIPLE_WATERMARK_POLICY,
FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION)
FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION, STREAMING_AGGREGATION_STATE_FORMAT_VERSION)

/**
* Default values of relevant configurations that are used for backward compatibility.
Expand All @@ -104,7 +104,9 @@ object OffsetSeqMetadata extends Logging {
private val relevantSQLConfDefaultValues = Map[String, String](
STREAMING_MULTIPLE_WATERMARK_POLICY.key -> MultipleWatermarkPolicy.DEFAULT_POLICY_NAME,
FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION.key ->
FlatMapGroupsWithStateExecHelper.legacyVersion.toString
FlatMapGroupsWithStateExecHelper.legacyVersion.toString,
STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key ->
StreamingAggregationStateManager.legacyVersion.toString
)

def apply(json: String): OffsetSeqMetadata = Serialization.read[OffsetSeqMetadata](json)
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,205 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.execution.streaming.state

import org.apache.spark.internal.Logging
import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow}
import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeProjection, GenerateUnsafeRowJoiner}
import org.apache.spark.sql.types.StructType

/**
* Base trait for state manager purposed to be used from streaming aggregations.
*/
sealed trait StreamingAggregationStateManager extends Serializable {

/** Extract columns consisting key from input row, and return the new row for key columns. */
def getKey(row: UnsafeRow): UnsafeRow

/** Calculate schema for the value of state. The schema is mainly passed to the StateStoreRDD. */
def getStateValueSchema: StructType

/** Get the current value of a non-null key from the target state store. */
def get(store: StateStore, key: UnsafeRow): UnsafeRow

/**
* Put a new value for a non-null key to the target state store. Note that key will be
* extracted from the input row, and the key would be same as the result of getKey(inputRow).
*/
def put(store: StateStore, row: UnsafeRow): Unit

/**
* Commit all the updates that have been made to the target state store, and return the
* new version.
*/
def commit(store: StateStore): Long

/** Remove a single non-null key from the target state store. */
def remove(store: StateStore, key: UnsafeRow): Unit

/** Return an iterator containing all the key-value pairs in target state store. */
def iterator(store: StateStore): Iterator[UnsafeRowPair]

/** Return an iterator containing all the keys in target state store. */
def keys(store: StateStore): Iterator[UnsafeRow]

/** Return an iterator containing all the values in target state store. */
def values(store: StateStore): Iterator[UnsafeRow]
}

object StreamingAggregationStateManager extends Logging {
val supportedVersions = Seq(1, 2)
val legacyVersion = 1

def createStateManager(
keyExpressions: Seq[Attribute],
inputRowAttributes: Seq[Attribute],
stateFormatVersion: Int): StreamingAggregationStateManager = {
stateFormatVersion match {
case 1 => new StreamingAggregationStateManagerImplV1(keyExpressions, inputRowAttributes)
case 2 => new StreamingAggregationStateManagerImplV2(keyExpressions, inputRowAttributes)
case _ => throw new IllegalArgumentException(s"Version $stateFormatVersion is invalid")
}
}
}

abstract class StreamingAggregationStateManagerBaseImpl(
protected val keyExpressions: Seq[Attribute],
protected val inputRowAttributes: Seq[Attribute]) extends StreamingAggregationStateManager {

@transient protected lazy val keyProjector =
GenerateUnsafeProjection.generate(keyExpressions, inputRowAttributes)

override def getKey(row: UnsafeRow): UnsafeRow = keyProjector(row)

override def commit(store: StateStore): Long = store.commit()

override def remove(store: StateStore, key: UnsafeRow): Unit = store.remove(key)

override def keys(store: StateStore): Iterator[UnsafeRow] = {
// discard and don't convert values to avoid computation
store.getRange(None, None).map(_.key)
}
}

/**
* The implementation of StreamingAggregationStateManager for state version 1.
* In state version 1, the schema of key and value in state are follow:
*
* - key: Same as key expressions.
* - value: Same as input row attributes. The schema of value contains key expressions as well.
*
* @param keyExpressions The attributes of keys.
* @param inputRowAttributes The attributes of input row.
*/
class StreamingAggregationStateManagerImplV1(
keyExpressions: Seq[Attribute],
inputRowAttributes: Seq[Attribute])
extends StreamingAggregationStateManagerBaseImpl(keyExpressions, inputRowAttributes) {

override def getStateValueSchema: StructType = inputRowAttributes.toStructType

override def get(store: StateStore, key: UnsafeRow): UnsafeRow = {
store.get(key)
}

override def put(store: StateStore, row: UnsafeRow): Unit = {
store.put(getKey(row), row)
}

override def iterator(store: StateStore): Iterator[UnsafeRowPair] = {
store.iterator()
}

override def values(store: StateStore): Iterator[UnsafeRow] = {
store.iterator().map(_.value)
}
}

/**
* The implementation of StreamingAggregationStateManager for state version 2.
* In state version 2, the schema of key and value in state are follow:
*
* - key: Same as key expressions.
* - value: The diff between input row attributes and key expressions.
*
* The schema of value is changed to optimize the memory/space usage in state, via removing
* duplicated columns in key-value pair. Hence key columns are excluded from the schema of value.
*
* @param keyExpressions The attributes of keys.
* @param inputRowAttributes The attributes of input row.
*/
class StreamingAggregationStateManagerImplV2(
keyExpressions: Seq[Attribute],
inputRowAttributes: Seq[Attribute])
extends StreamingAggregationStateManagerBaseImpl(keyExpressions, inputRowAttributes) {

private val valueExpressions: Seq[Attribute] = inputRowAttributes.diff(keyExpressions)
private val keyValueJoinedExpressions: Seq[Attribute] = keyExpressions ++ valueExpressions

// flag to check whether the row needs to be project into input row attributes after join
// e.g. if the fields in the joined row are not in the expected order
private val needToProjectToRestoreValue: Boolean =
keyValueJoinedExpressions != inputRowAttributes

@transient private lazy val valueProjector =
GenerateUnsafeProjection.generate(valueExpressions, inputRowAttributes)

@transient private lazy val joiner =
GenerateUnsafeRowJoiner.create(StructType.fromAttributes(keyExpressions),
StructType.fromAttributes(valueExpressions))
@transient private lazy val restoreValueProjector = GenerateUnsafeProjection.generate(
inputRowAttributes, keyValueJoinedExpressions)

override def getStateValueSchema: StructType = valueExpressions.toStructType

override def get(store: StateStore, key: UnsafeRow): UnsafeRow = {
val savedState = store.get(key)
if (savedState == null) {
return savedState
}

restoreOriginalRow(key, savedState)
}

override def put(store: StateStore, row: UnsafeRow): Unit = {
val key = keyProjector(row)
val value = valueProjector(row)
store.put(key, value)
}

override def iterator(store: StateStore): Iterator[UnsafeRowPair] = {
store.iterator().map(rowPair => new UnsafeRowPair(rowPair.key, restoreOriginalRow(rowPair)))
}

override def values(store: StateStore): Iterator[UnsafeRow] = {
store.iterator().map(rowPair => restoreOriginalRow(rowPair))
}

private def restoreOriginalRow(rowPair: UnsafeRowPair): UnsafeRow = {
restoreOriginalRow(rowPair.key, rowPair.value)
}

private def restoreOriginalRow(key: UnsafeRow, value: UnsafeRow): UnsafeRow = {
val joinedRow = joiner.join(key, value)
if (needToProjectToRestoreValue) {
restoreValueProjector(joinedRow)
} else {
joinedRow
}
}
}
Loading