-
Notifications
You must be signed in to change notification settings - Fork 29.2k
[SPARK-21669] Internal API for collecting metrics/stats during FileFormatWriter jobs #18884
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
adrian-ionescu
wants to merge
10
commits into
apache:master
from
adrian-ionescu:write-stats-tracker-api
Closed
Changes from 5 commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
67e333e
initial
adrian-ionescu 176726e
tests pass; missing docs
adrian-ionescu 6f40246
newPartition() takes InternalRow instead of String
adrian-ionescu e6ab459
bug fix + docs
adrian-ionescu 3665f2f
minor
adrian-ionescu 0eae6f1
minor
adrian-ionescu f599236
address easy review remarks
adrian-ionescu dbf58b7
Merge branch 'master' into write-stats-tracker-api
adrian-ionescu 7ec545b
address review feedback from cloud-fan
adrian-ionescu e8884b7
minor
adrian-ionescu 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
133 changes: 133 additions & 0 deletions
133
...re/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.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,133 @@ | ||
| /* | ||
| * 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.datasources | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
|
|
||
| import org.apache.spark.SparkContext | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.execution.SQLExecution | ||
| import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} | ||
| import org.apache.spark.util.SerializableConfiguration | ||
|
|
||
|
|
||
| /** | ||
| * Simple metrics collected during an instance of [[FileFormatWriter.ExecuteWriteTask]]. | ||
| * These were first introduced in https://github.com/apache/spark/pull/18159 (SPARK-20703). | ||
| */ | ||
| case class BasicWriteTaskStats( | ||
| numPartitions: Int, | ||
| numFiles: Int, | ||
| numBytes: Long, | ||
| numRows: Long) | ||
| extends WriteTaskStats | ||
|
|
||
|
|
||
| /** | ||
| * Simple [[WriteTaskStatsTracker]] implementation that produces [[BasicWriteTaskStats]]. | ||
| * @param hadoopConf | ||
| */ | ||
| class BasicWriteTaskStatsTracker(hadoopConf: Configuration) | ||
| extends WriteTaskStatsTracker { | ||
|
|
||
| var numPartitions: Int = 0 | ||
| var numFiles: Int = 0 | ||
| var numBytes: Long = 0L | ||
| var numRows: Long = 0L | ||
|
|
||
| var curFile: String = null | ||
|
|
||
|
|
||
| private def getFileSize(filePath: String): Long = { | ||
| val path = new Path(filePath) | ||
| val fs = path.getFileSystem(hadoopConf) | ||
| fs.getFileStatus(path).getLen() | ||
| } | ||
|
|
||
|
|
||
| override def newPartition(partitionValues: InternalRow): Unit = { | ||
| numPartitions += 1 | ||
| } | ||
|
|
||
| override def newBucket(bucketId: Int): Unit = { | ||
| // currently unhandled | ||
| } | ||
|
|
||
| override def newFile(filePath: String): Unit = { | ||
| if (numFiles > 0) { | ||
| // we assume here that we've finished writing to disk the previous file by now | ||
| numBytes += getFileSize(curFile) | ||
| } | ||
| curFile = filePath | ||
| numFiles += 1 | ||
| } | ||
|
|
||
| override def newRow(row: InternalRow): Unit = { | ||
| numRows += 1 | ||
| } | ||
|
|
||
| override def getFinalStats(): WriteTaskStats = { | ||
| if (numFiles > 0) { | ||
| numBytes += getFileSize(curFile) | ||
| } | ||
| BasicWriteTaskStats(numPartitions, numFiles, numBytes, numRows) | ||
| } | ||
| } | ||
|
|
||
|
|
||
| /** | ||
| * Simple [[WriteJobStatsTracker]] implementation that's serializable, capable of | ||
| * instantiating [[BasicWriteTaskStatsTracker]] on executors and processing the | ||
| * [[BasicWriteTaskStats]] they produce by aggregating the metrics and posting them | ||
| * as DriverMetricUpdates. | ||
| */ | ||
| class BasicWriteJobStatsTracker( | ||
| serializableHadoopConf: SerializableConfiguration, | ||
| @transient val metrics: Map[String, SQLMetric]) | ||
| extends WriteJobStatsTracker { | ||
|
|
||
| override def newTaskInstance(): WriteTaskStatsTracker = { | ||
| new BasicWriteTaskStatsTracker(serializableHadoopConf.value) | ||
| } | ||
|
|
||
| override def processStats(stats: Seq[WriteTaskStats]): Unit = { | ||
| val sparkContext = SparkContext.getActive.get | ||
| var numPartitions: Long = 0L | ||
| var numFiles: Long = 0L | ||
| var totalNumBytes: Long = 0L | ||
| var totalNumOutput: Long = 0L | ||
|
|
||
| val basicStats = stats.map(_.asInstanceOf[BasicWriteTaskStats]) | ||
|
|
||
| basicStats.foreach { summary => | ||
| numPartitions += summary.numPartitions | ||
| numFiles += summary.numFiles | ||
| totalNumBytes += summary.numBytes | ||
| totalNumOutput += summary.numRows | ||
| } | ||
|
|
||
| metrics("numFiles").add(numFiles) | ||
| metrics("numOutputBytes").add(totalNumBytes) | ||
| metrics("numOutputRows").add(totalNumOutput) | ||
| metrics("numParts").add(numPartitions) | ||
|
|
||
| val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) | ||
| SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toList) | ||
| } | ||
| } | ||
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.
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.
private[this] ?