-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-54346][SS] Introduce state repartition API and repartition runner #53056
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
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,53 @@ | ||
| /* | ||
| * 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.streaming | ||
|
|
||
| /** | ||
| * A class to manage operations on streaming query checkpoints. | ||
| */ | ||
| private[spark] abstract class StreamingCheckpointManager { | ||
|
|
||
| /** | ||
| * Repartition the stateful streaming operators state in the streaming checkpoint to have | ||
| * `numPartitions` partitions. The streaming query MUST not be running. If `numPartitions` is | ||
| * the same as the current number of partitions, this is a no-op, and an exception will be | ||
| * thrown. | ||
| * | ||
| * This produces a new microbatch in the checkpoint that contains the repartitioned state i.e. | ||
| * if the last streaming batch was batch `N`, this will create batch `N+1` with the | ||
| * repartitioned state. Note that this new batch doesn't read input data from sources, it only | ||
| * represents the repartition operation. The next time the streaming query is started, it will | ||
| * pick up from this new batch. | ||
| * | ||
| * This will return only when the repartitioning is complete or fails. | ||
| * | ||
| * @note | ||
| * This operation should only be performed after the streaming query has been stopped. | ||
| * @param checkpointLocation | ||
| * The checkpoint location of the streaming query, should be the `checkpointLocation` option | ||
| * on the DataStreamWriter. | ||
| * @param numPartitions | ||
| * the target number of state partitions. | ||
| * @param enforceExactlyOnceSink | ||
| * if we shouldn't allow skipping failed batches, to avoid duplicates in exactly once sinks. | ||
| */ | ||
| private[spark] def repartition( | ||
| checkpointLocation: String, | ||
| numPartitions: Int, | ||
| enforceExactlyOnceSink: Boolean = true): Unit | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -145,6 +145,12 @@ class SQLContext private[sql] (override val sparkSession: SparkSession) | |
| */ | ||
| def streams: StreamingQueryManager = sparkSession.streams | ||
|
|
||
| /** | ||
| * Returns a `StreamingCheckpointManager` that allows managing any streaming checkpoint. | ||
| */ | ||
| private[spark] def streamingCheckpointManager: StreamingCheckpointManager = | ||
|
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. We need to add to Spark connect also ?
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. Spark connect and pyspark will be added in subsequent PRs. |
||
| sparkSession.streamingCheckpointManager | ||
|
|
||
| /** @inheritdoc */ | ||
| override def sparkContext: SparkContext = super.sparkContext | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,55 @@ | ||
| /* | ||
| * 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.classic | ||
|
|
||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.sql.execution.streaming.state.{OfflineStateRepartitionErrors, OfflineStateRepartitionRunner} | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.streaming | ||
|
|
||
| /** @inheritdoc */ | ||
| private[spark] class StreamingCheckpointManager( | ||
| sparkSession: SparkSession, | ||
| sqlConf: SQLConf) extends streaming.StreamingCheckpointManager with Logging { | ||
|
|
||
| /** @inheritdoc */ | ||
| override private[spark] def repartition( | ||
| checkpointLocation: String, | ||
| numPartitions: Int, | ||
|
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. I guess the underlying recorded value is Int, but should we consider bumping this to
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. lets keep it as int since that is what we record in checkpoint. Also using Long is very unrealistic. |
||
| enforceExactlyOnceSink: Boolean = true): Unit = { | ||
| checkpointLocation match { | ||
| case null => | ||
| throw OfflineStateRepartitionErrors.parameterIsNullError("checkpointLocation") | ||
| case "" => | ||
| throw OfflineStateRepartitionErrors.parameterIsEmptyError("checkpointLocation") | ||
| case _ => // Valid case, no action needed | ||
| } | ||
|
|
||
| if (numPartitions <= 0) { | ||
| throw OfflineStateRepartitionErrors.parameterIsNotGreaterThanZeroError("numPartitions") | ||
| } | ||
|
|
||
| val runner = new OfflineStateRepartitionRunner( | ||
|
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. Should we encapsulate this whole block in a
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. see |
||
| sparkSession, | ||
| checkpointLocation, | ||
| numPartitions, | ||
| enforceExactlyOnceSink | ||
| ) | ||
| runner.run() | ||
|
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. Can we also add some logging to indicate the repartition started/ended and the time it took to complete the operation along with other identifying information about the query ?
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. see the runner |
||
| } | ||
| } | ||
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.
nit:
INVALID_OPTIONS?Uh oh!
There was an error while loading. Please reload this page.
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.
I want to avoid confusion, since options in spark means
.option(). So using parameter instead.