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
[SPARK-33739] [SQL] Jobs committed through the S3A Magic committer don't track bytes #30714
Changes from all commits
c680138
54f2807
00c19a5
4b08f49
81c0a52
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 |
---|---|---|
|
@@ -18,11 +18,12 @@ | |
package org.apache.spark.sql.execution.datasources | ||
|
||
import java.io.FileNotFoundException | ||
import java.nio.charset.StandardCharsets | ||
|
||
import scala.collection.mutable | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.Path | ||
import org.apache.hadoop.fs.{FileSystem, Path} | ||
|
||
import org.apache.spark.{SparkContext, TaskContext} | ||
import org.apache.spark.internal.Logging | ||
|
@@ -66,14 +67,66 @@ class BasicWriteTaskStatsTracker(hadoopConf: Configuration) | |
private def getFileSize(filePath: String): Option[Long] = { | ||
val path = new Path(filePath) | ||
val fs = path.getFileSystem(hadoopConf) | ||
getFileSize(fs, path) | ||
} | ||
|
||
/** | ||
* Get the size of the file expected to have been written by a worker. | ||
* This supports the XAttr in HADOOP-17414 when the "magic committer" adds | ||
* a custom HTTP header to the a zero byte marker. | ||
* If the output file as returned by getFileStatus > 0 then the length if | ||
* returned. For zero-byte files, the (optional) Hadoop FS API getXAttr() is | ||
* invoked. If a parseable, non-negative length can be retrieved, this | ||
* is returned instead of the length. | ||
* @return the file size or None if the file was not found. | ||
*/ | ||
private [datasources] def getFileSize(fs: FileSystem, path: Path): Option[Long] = { | ||
// the normal file status probe. | ||
try { | ||
Some(fs.getFileStatus(path).getLen()) | ||
val len = fs.getFileStatus(path).getLen | ||
if (len > 0) { | ||
return Some(len) | ||
} | ||
} catch { | ||
case e: FileNotFoundException => | ||
// may arise against eventually consistent object stores | ||
// may arise against eventually consistent object stores. | ||
logDebug(s"File $path is not yet visible", e) | ||
None | ||
return None | ||
} | ||
|
||
// Output File Size is 0. Look to see if it has an attribute | ||
// declaring a future-file-length. | ||
// Failure of API call, parsing, invalid value all return the | ||
// 0 byte length. | ||
|
||
var len = 0L | ||
try { | ||
val attr = fs.getXAttr(path, BasicWriteJobStatsTracker.FILE_LENGTH_XATTR) | ||
if (attr != null && attr.nonEmpty) { | ||
val str = new String(attr, StandardCharsets.UTF_8) | ||
logDebug(s"File Length statistics for $path retrieved from XAttr: $str") | ||
// a non-empty header was found. parse to a long via the java class | ||
val l = java.lang.Long.parseLong(str) | ||
if (l > 0) { | ||
len = l | ||
} else { | ||
logDebug("Ignoring negative value in XAttr file length") | ||
} | ||
} | ||
} catch { | ||
case e: NumberFormatException => | ||
// warn but don't dump the whole stack | ||
logInfo(s"Failed to parse" + | ||
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. do we want this warn instead or perhaps just extend saying file stats may be wrong? 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. OK. I'd say at info as warn seems overkill for progress report issues. |
||
s" ${BasicWriteJobStatsTracker.FILE_LENGTH_XATTR}:$e;" + | ||
s" bytes written may be under-reported"); | ||
case e: UnsupportedOperationException => | ||
// this is not unusual; ignore | ||
logDebug(s"XAttr not supported on path $path", e); | ||
case e: Exception => | ||
// Something else. Log at debug and continue. | ||
logDebug(s"XAttr processing failure on $path", e); | ||
} | ||
Some(len) | ||
} | ||
|
||
|
||
|
@@ -170,6 +223,8 @@ object BasicWriteJobStatsTracker { | |
private val NUM_OUTPUT_BYTES_KEY = "numOutputBytes" | ||
private val NUM_OUTPUT_ROWS_KEY = "numOutputRows" | ||
private val NUM_PARTS_KEY = "numParts" | ||
/** XAttr key of the data length header added in HADOOP-17414. */ | ||
HyukjinKwon marked this conversation as resolved.
Show resolved
Hide resolved
|
||
val FILE_LENGTH_XATTR = "header.x-hadoop-s3a-magic-data-length" | ||
|
||
def metrics: Map[String, SQLMetric] = { | ||
val sparkContext = SparkContext.getActive.get | ||
|
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.
Shall we remove line 60 together?
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 worry about openstack swift. The original version was woefully inconsistent. Not been near it long enough to know what's change. with swift I could consistently replicate an inconsistency in a few lines
No S3 implementation that I know of (i.e. the open source or commercial ones) are inconsistent, nor have they ever had the 404 caching issue. But people should still be aware of the risk.
oh, and I haven' t played with any of the chinese cloud stores for which connectors now exist. So I can't make statements there. All I can say is the "big three outside china" are consistent.
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.
The line deletion is based on the fact S3 is now strong consistency, but that also means we only consider these three vendors. (Probably you're considering more like S3 compatible implementations, but you've also mentioned you don't consider the chinese cloud stores so it can't be exhaustive.) Why not explicitly saying it and update the description tied to these vendors? We would never be able to consider all possible implementations and for some minority the description may be wrong. Let's just make it clear.
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.
Or, what about leaving this line as it is (so that the description enumerates all "possible" issues on object store considering beyond the big three), and elaborate which key differences are no longer in effect with strong consistency in consistency section?
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 don't test them; I don't know their consistency
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.
reviewed the docs, huawei cos and tencent obs both seem consistent. swift is now the outlier. Moving the issue into a paragraph in the "consistency" section
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.
@steveloughran Thanks for the update!
@dongjoon-hyun Could you please check whether the comments and new change make sense to you? Thanks in advance!