-
Notifications
You must be signed in to change notification settings - Fork 29k
[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
Changes from 1 commit
4252f41
941b88d
abec57f
977428c
63dfb5d
e844636
26701a3
60c231e
b4a3807
e0ee04a
8629f59
65801a6
19888ab
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
…backward compatible"
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -871,15 +871,15 @@ object SQLConf { | |
| .intConf | ||
| .createWithDefault(2) | ||
|
|
||
| val ADVANCED_REMOVE_REDUNDANT_IN_STATEFUL_AGGREGATION = | ||
| buildConf("spark.sql.streaming.advanced.removeRedundantInStatefulAggregation") | ||
| val STREAMING_AGGREGATION_STATE_FORMAT_VERSION = | ||
| buildConf("spark.sql.streaming.streamingAggregation.stateFormatVersion") | ||
| .internal() | ||
| .doc("ADVANCED: When true, stateful aggregation tries to remove redundant data " + | ||
| "between key and value in state. Enabling this option helps minimizing state size, " + | ||
| "but no longer be compatible with state with disabling this option." + | ||
| "You can't change this option after starting the query.") | ||
| .booleanConf | ||
| .createWithDefault(false) | ||
| .doc("State format version used by streaming aggregation operations triggered " + | ||
| "explicitly or implicitly via agg() 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) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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") | ||
|
|
@@ -1628,9 +1628,6 @@ class SQLConf extends Serializable with Logging { | |
| def advancedPartitionPredicatePushdownEnabled: Boolean = | ||
| getConf(ADVANCED_PARTITION_PREDICATE_PUSHDOWN) | ||
|
|
||
| def advancedRemoveRedundantInStatefulAggregation: Boolean = | ||
| getConf(ADVANCED_REMOVE_REDUNDANT_IN_STATEFUL_AGGREGATION) | ||
|
|
||
| def fallBackToHdfsForStatsEnabled: Boolean = getConf(ENABLE_FALL_BACK_TO_HDFS_FOR_STATS) | ||
|
|
||
| def preferSortMergeJoin: Boolean = getConf(PREFER_SORTMERGEJOIN) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,10 +22,13 @@ import org.apache.spark.sql.catalyst.InternalRow | |
| import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow} | ||
| import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateUnsafeProjection, GenerateUnsafeRowJoiner} | ||
| import org.apache.spark.sql.execution.streaming.state.{StateStore, UnsafeRowPair} | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.types.StructType | ||
|
|
||
| object StatefulOperatorsHelper { | ||
|
||
|
|
||
| val supportedVersions = Seq(1, 2) | ||
| val legacyVersion = 1 | ||
|
|
||
| sealed trait StreamingAggregationStateManager extends Serializable { | ||
|
||
| def extractKey(row: InternalRow): UnsafeRow | ||
|
||
| def getValueExpressions: Seq[Attribute] | ||
|
||
|
|
@@ -35,16 +38,14 @@ object StatefulOperatorsHelper { | |
| } | ||
|
|
||
| object StreamingAggregationStateManager extends Logging { | ||
| def newImpl( | ||
| def createStateManager( | ||
| keyExpressions: Seq[Attribute], | ||
| childOutput: Seq[Attribute], | ||
|
||
| conf: SQLConf): StreamingAggregationStateManager = { | ||
|
|
||
| if (conf.advancedRemoveRedundantInStatefulAggregation) { | ||
| log.info("Advanced option removeRedundantInStatefulAggregation activated!") | ||
| new StreamingAggregationStateManagerImplV2(keyExpressions, childOutput) | ||
| } else { | ||
| new StreamingAggregationStateManagerImplV1(keyExpressions, childOutput) | ||
| stateFormatVersion: Int): StreamingAggregationStateManager = { | ||
| stateFormatVersion match { | ||
| case 1 => new StreamingAggregationStateManagerImplV1(keyExpressions, childOutput) | ||
| case 2 => new StreamingAggregationStateManagerImplV2(keyExpressions, childOutput) | ||
| case _ => throw new IllegalArgumentException(s"Version $stateFormatVersion is invalid") | ||
| } | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,8 +19,7 @@ package org.apache.spark.sql.streaming | |
|
|
||
| import java.util.{Locale, TimeZone} | ||
|
|
||
| import org.scalatest.Assertions | ||
| import org.scalatest.BeforeAndAfterAll | ||
| import org.scalatest.{Assertions, BeforeAndAfterAll} | ||
|
|
||
| import org.apache.spark.{SparkEnv, SparkException} | ||
| import org.apache.spark.rdd.BlockRDD | ||
|
|
@@ -54,30 +53,35 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
|
|
||
| import testImplicits._ | ||
|
|
||
| val confAndTestNamePostfixMatrix = List( | ||
| (Seq(SQLConf.ADVANCED_REMOVE_REDUNDANT_IN_STATEFUL_AGGREGATION.key -> "false"), ""), | ||
| (Seq(SQLConf.ADVANCED_REMOVE_REDUNDANT_IN_STATEFUL_AGGREGATION.key -> "true"), | ||
| " : enable remove redundant in stateful aggregation") | ||
| ) | ||
| def executeFuncWithStateVersionSQLConf( | ||
| stateVersion: Int, | ||
| confPairs: Seq[(String, String)], | ||
| func: => Any): Unit = { | ||
| withSQLConf(confPairs ++ | ||
| Seq(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION.key -> stateVersion.toString): _*) { | ||
| func | ||
| } | ||
| } | ||
|
|
||
| def testWithAggrOptions(testName: String, pairs: (String, String)*)(testFun: => Any): Unit = { | ||
| confAndTestNamePostfixMatrix.foreach { | ||
| case (conf, testNamePostfix) => withSQLConf(pairs ++ conf: _*) { | ||
| test(testName + testNamePostfix)(testFun) | ||
| def testWithAllStateVersions(name: String, confPairs: (String, String)*) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. super nit: the confPair param is used only in one location, do you think its worth adding it as a param? The only test that needs it can stay unchanged.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Actually it's basically from wondering of how |
||
| (func: => Any): Unit = { | ||
| for (version <- StatefulOperatorsHelper.supportedVersions) { | ||
| test(s"$name - state format version $version") { | ||
| executeFuncWithStateVersionSQLConf(version, confPairs, func) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def testQuietlyWithAggrOptions(testName: String, pairs: (String, String)*) | ||
| (testFun: => Any): Unit = { | ||
| confAndTestNamePostfixMatrix.foreach { | ||
| case (conf, testNamePostfix) => withSQLConf(pairs ++ conf: _*) { | ||
| testQuietly(testName + testNamePostfix)(testFun) | ||
| def testQuietlyWithAllStateVersions(name: String, confPairs: (String, String)*) | ||
| (func: => Any): Unit = { | ||
| for (version <- StatefulOperatorsHelper.supportedVersions) { | ||
| testQuietly(s"$name - state format version $version") { | ||
| executeFuncWithStateVersionSQLConf(version, confPairs, func) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| testWithAggrOptions("simple count, update mode") { | ||
| testWithAllStateVersions("simple count, update mode") { | ||
| val inputData = MemoryStream[Int] | ||
|
|
||
| val aggregated = | ||
|
|
@@ -101,7 +105,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testWithAggrOptions("count distinct") { | ||
| testWithAllStateVersions("count distinct") { | ||
| val inputData = MemoryStream[(Int, Seq[Int])] | ||
|
|
||
| val aggregated = | ||
|
|
@@ -117,7 +121,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testWithAggrOptions("simple count, complete mode") { | ||
| testWithAllStateVersions("simple count, complete mode") { | ||
| val inputData = MemoryStream[Int] | ||
|
|
||
| val aggregated = | ||
|
|
@@ -140,7 +144,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testWithAggrOptions("simple count, append mode") { | ||
| testWithAllStateVersions("simple count, append mode") { | ||
| val inputData = MemoryStream[Int] | ||
|
|
||
| val aggregated = | ||
|
|
@@ -157,7 +161,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| } | ||
| } | ||
|
|
||
| testWithAggrOptions("sort after aggregate in complete mode") { | ||
| testWithAllStateVersions("sort after aggregate in complete mode") { | ||
| val inputData = MemoryStream[Int] | ||
|
|
||
| val aggregated = | ||
|
|
@@ -182,7 +186,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testWithAggrOptions("state metrics") { | ||
| testWithAllStateVersions("state metrics") { | ||
| val inputData = MemoryStream[Int] | ||
|
|
||
| val aggregated = | ||
|
|
@@ -235,7 +239,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testWithAggrOptions("multiple keys") { | ||
| testWithAllStateVersions("multiple keys") { | ||
| val inputData = MemoryStream[Int] | ||
|
|
||
| val aggregated = | ||
|
|
@@ -252,7 +256,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testQuietlyWithAggrOptions("midbatch failure") { | ||
| testQuietlyWithAllStateVersions("midbatch failure") { | ||
| val inputData = MemoryStream[Int] | ||
| FailureSingleton.firstTime = true | ||
| val aggregated = | ||
|
|
@@ -278,7 +282,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testWithAggrOptions("typed aggregators") { | ||
| testWithAllStateVersions("typed aggregators") { | ||
| val inputData = MemoryStream[(String, Int)] | ||
| val aggregated = inputData.toDS().groupByKey(_._1).agg(typed.sumLong(_._2)) | ||
|
|
||
|
|
@@ -288,7 +292,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testWithAggrOptions("prune results by current_time, complete mode") { | ||
| testWithAllStateVersions("prune results by current_time, complete mode") { | ||
| import testImplicits._ | ||
| val clock = new StreamManualClock | ||
| val inputData = MemoryStream[Long] | ||
|
|
@@ -340,7 +344,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testWithAggrOptions("prune results by current_date, complete mode") { | ||
| testWithAllStateVersions("prune results by current_date, complete mode") { | ||
| import testImplicits._ | ||
| val clock = new StreamManualClock | ||
| val tz = TimeZone.getDefault.getID | ||
|
|
@@ -389,7 +393,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testWithAggrOptions("SPARK-19690: do not convert batch aggregation in streaming query " + | ||
| testWithAllStateVersions("SPARK-19690: do not convert batch aggregation in streaming query " + | ||
| "to streaming") { | ||
| val streamInput = MemoryStream[Int] | ||
| val batchDF = Seq(1, 2, 3, 4, 5) | ||
|
|
@@ -454,7 +458,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| true | ||
| } | ||
|
|
||
| testWithAggrOptions("SPARK-21977: coalesce(1) with 0 partition RDD should be " + | ||
| testWithAllStateVersions("SPARK-21977: coalesce(1) with 0 partition RDD should be " + | ||
| "repartitioned to 1") { | ||
| val inputSource = new BlockRDDBackedSource(spark) | ||
| MockSourceProvider.withMockSources(inputSource) { | ||
|
|
@@ -493,8 +497,8 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| } | ||
| } | ||
|
|
||
| testWithAggrOptions("SPARK-21977: coalesce(1) with aggregation should still be repartitioned " + | ||
| "when it has non-empty grouping keys") { | ||
| testWithAllStateVersions("SPARK-21977: coalesce(1) with aggregation should still be " + | ||
| "repartitioned when it has non-empty grouping keys") { | ||
| val inputSource = new BlockRDDBackedSource(spark) | ||
| MockSourceProvider.withMockSources(inputSource) { | ||
| withTempDir { tempDir => | ||
|
|
@@ -546,7 +550,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| } | ||
| } | ||
|
|
||
| testWithAggrOptions("SPARK-22230: last should change with new batches") { | ||
| testWithAllStateVersions("SPARK-22230: last should change with new batches") { | ||
| val input = MemoryStream[Int] | ||
|
|
||
| val aggregated = input.toDF().agg(last('value)) | ||
|
|
@@ -562,7 +566,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest | |
| ) | ||
| } | ||
|
|
||
| testWithAggrOptions("SPARK-23004: Ensure that TypedImperativeAggregate functions " + | ||
| testWithAllStateVersions("SPARK-23004: Ensure that TypedImperativeAggregate functions " + | ||
| "do not throw errors", SQLConf.SHUFFLE_PARTITIONS.key -> "1") { | ||
| // See the JIRA SPARK-23004 for more details. In short, this test reproduces the error | ||
| // by ensuring the following. | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
no need to saying "streaming" in "streamingAggregation" since its already qualified by "spark.sql.streaming."
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah OK. Sounds better. Will fix.