-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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-51072][CORE] CallerContext to set Hadoop cloud audit context #49779
base: master
Are you sure you want to change the base?
[SPARK-51072][CORE] CallerContext to set Hadoop cloud audit context #49779
Conversation
…ntext Change-Id: I6bd66ff817b09c7431e8c6de4577fdda1ed67d6d
val hdfsContext = new HadoopCallerContextBuilder(context).build() | ||
HadoopCallerContext.setCurrent(hdfsContext) | ||
// audit context as passed down to object stores, use prefix "spark" | ||
currentAuditContext.put("spark", context) |
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.
Thank you for making a PR. This one line seems to be the actual change. Did I understand correctly?
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.
yes! Rest of it is test related
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.
According to the PR title, do you mean S3 Audit Context feature has been broken until now ? Otherwise, could you revise the PR title by narrowing down the scope more specifically?
[SPARK-51072][CORE] CallerContext to set Hadoop cloud audit context
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.
+1 (non-binding). This will be useful. Thanks, @steveloughran !
From the perspective of Spark or other clients of the file system, they are interacting with a general auditing feature defined in the Hadoop Common module. In theory, multiple file systems could implement support for actually recording this audit information. AFAIK, only S3A implements it right now. (We don't have it in the GCS file system.) Other file systems could eventually choose to implement it though. |
Ya, IIUC, without this PR, S3A audit has been working already as designed, hasn't it? |
@dongjoon-hyun audit context has been working properly, but spark info not wired up. Other things get in (process Id, UGI, filesystem id, underlying operation for a sequence), but the actual spark job didn't get in. Once an s3a or manifest committer was started, it'd set the app and job ID values -but they only get involved during the write phase -and they didn't get the full spark context info |
@cnauroth well, you should -if you can get it anywhere into your logs, possibly as a new http header. s3afs attaches as an http referrer as it is the sole entry other than UA which goes into the standard S3 logs -and other things like to set that UA field. |
Got it. Please let me know when the PR is ready, @steveloughran . |
What changes were proposed in this pull request?
When enabled, cloud store client audit context is set to the
same context string as the Hadoop IPC context.
Why are the changes needed?
CallerContext adds information about the spark task to hadoop IPC context and then to HDFS, YARN and HBase server logs.
It is also possible to update the cloud storage "audit context".
Storage clients can attach the audit information to requests to be stored in the service's own logs, where it can be retrieved, parsed and used for analysis.
It is currently supported by the S3A connector, which adds the information to a synthetic referrer header, which is then stored in the S3 Server logs. (Not cloudtrail, sadly).
See S3A Auditing
Does this PR introduce any user-facing change?
If enabled, it adds extra entries in cloud storage server logs through cloud
storage clients which support it.
How was this patch tested?
Expanded existing test
"Set Spark CallerContext"
to verifyfull setting of passed down parameters to caller and audit contexts.
This required extracting the functional code of
CallerContext.setCurrentContext
into a
@VisibleForTesting private[util]
methodsetCurrentContext(Boolean)
Without this, the test suite only ran if the process had been launched
with the configuration option
"hadoop.caller.context.enabled
being setto true -this is not the default, so the existing test suite code
was probably never executed.
Was this patch authored or co-authored using generative AI tooling?
No