-
Notifications
You must be signed in to change notification settings - Fork 29k
[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
Closed
Closed
Changes from all commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
1d34ced
runner
micheal-o c7eb1cd
err clss formating
micheal-o fba27ba
nit
micheal-o 60f7a3f
lint
micheal-o 51a7458
move utils
micheal-o 7eefad3
extra log
micheal-o 9641d7e
spark version and multi skip
micheal-o 11e3414
nit
micheal-o 58f2849
lint
micheal-o File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
54 changes: 54 additions & 0 deletions
54
sql/api/src/main/scala/org/apache/spark/sql/streaming/StreamingCheckpointManager.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,54 @@ | ||
| /* | ||
| * 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. If not, | ||
| * can lead to undefined behavior or checkpoint corruption. | ||
| * @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 | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
55 changes: 55 additions & 0 deletions
55
sql/core/src/main/scala/org/apache/spark/sql/classic/StreamingCheckpointManager.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| 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, | ||
anishshri-db marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| 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( | ||
micheal-o marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| sparkSession, | ||
| checkpointLocation, | ||
| numPartitions, | ||
| enforceExactlyOnceSink | ||
| ) | ||
| runner.run() | ||
micheal-o marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.