-
Notifications
You must be signed in to change notification settings - Fork 28.6k
[SPARK-26193][SQL] Implement shuffle write metrics in SQL #23207
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
11 commits
Select commit
Hold shift + click to select a range
6b26c62
Commit for fist time success
xuanyuanking a8a1225
Simplify implement by add metrics in ShuffleExchangeExec
xuanyuanking 7c8e516
code clean and comments
xuanyuanking cf35b9f
Address comments and change the ShuffleDependency
xuanyuanking 76d1ca0
better way to deal with duration
xuanyuanking 9966c2a
separate read/write metrics
xuanyuanking a780b70
Create ShuffleWriteProcessor
xuanyuanking 7d104eb
Revert the changes about shuffle read metrics reaname
xuanyuanking d5ee249
self check
xuanyuanking 6378a3d
Delete DefaultShuffleWriteProcessor
xuanyuanking bc2c4f1
Address comments
xuanyuanking 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
74 changes: 74 additions & 0 deletions
74
core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.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,74 @@ | ||
/* | ||
* 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.shuffle | ||
|
||
import org.apache.spark.{Partition, ShuffleDependency, SparkEnv, TaskContext} | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.scheduler.MapStatus | ||
|
||
/** | ||
* The interface for customizing shuffle write process. The driver create a ShuffleWriteProcessor | ||
* and put it into [[ShuffleDependency]], and executors use it in each ShuffleMapTask. | ||
*/ | ||
private[spark] class ShuffleWriteProcessor extends Serializable with Logging { | ||
|
||
/** | ||
* Create a [[ShuffleWriteMetricsReporter]] from the task context. As the reporter is a | ||
* per-row operator, here need a careful consideration on performance. | ||
*/ | ||
protected def createMetricsReporter(context: TaskContext): ShuffleWriteMetricsReporter = { | ||
context.taskMetrics().shuffleWriteMetrics | ||
} | ||
|
||
/** | ||
* The write process for particular partition, it controls the life circle of [[ShuffleWriter]] | ||
* get from [[ShuffleManager]] and triggers rdd compute, finally return the [[MapStatus]] for | ||
* this task. | ||
*/ | ||
def writeProcess( | ||
rdd: RDD[_], | ||
dep: ShuffleDependency[_, _, _], | ||
partitionId: Int, | ||
context: TaskContext, | ||
partition: Partition): MapStatus = { | ||
var writer: ShuffleWriter[Any, Any] = null | ||
try { | ||
val manager = SparkEnv.get.shuffleManager | ||
writer = manager.getWriter[Any, Any]( | ||
dep.shuffleHandle, | ||
partitionId, | ||
context, | ||
createMetricsReporter(context)) | ||
writer.write( | ||
rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) | ||
writer.stop(success = true).get | ||
} catch { | ||
case e: Exception => | ||
try { | ||
if (writer != null) { | ||
writer.stop(success = false) | ||
} | ||
} catch { | ||
case e: Exception => | ||
log.debug("Could not stop writer", e) | ||
} | ||
throw e | ||
} | ||
} | ||
} |
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
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
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.