Skip to content

Conversation

@steveloughran
Copy link
Contributor

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 verify
full setting of passed down parameters to caller and audit contexts.
This required extracting the functional code of CallerContext.setCurrentContext
into a @VisibleForTesting private[util] method setCurrentContext(Boolean)

Without this, the test suite only ran if the process had been launched
with the configuration option "hadoop.caller.context.enabled being set
to 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

@steveloughran steveloughran marked this pull request as draft February 3, 2025 19:19
@github-actions github-actions bot added the CORE label Feb 3, 2025
Copy link
Member

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?

Copy link
Contributor Author

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

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a 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

Copy link
Contributor

@cnauroth cnauroth left a 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 !

@cnauroth
Copy link
Contributor

cnauroth commented Feb 3, 2025

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

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.

@dongjoon-hyun
Copy link
Member

dongjoon-hyun commented Feb 3, 2025

Ya, IIUC, without this PR, S3A audit has been working already as designed, hasn't it?

@steveloughran
Copy link
Contributor Author

@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

@steveloughran
Copy link
Contributor Author

@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.

@dongjoon-hyun
Copy link
Member

Got it. Please let me know when the PR is ready, @steveloughran .

@steveloughran
Copy link
Contributor Author

I can't think of any changes, unless we want to set that audit stuff even if caller context is not being set.

…ntext

Change-Id: I6bd66ff817b09c7431e8c6de4577fdda1ed67d6d
@steveloughran steveloughran force-pushed the SPARK-51072-caller-context-auditing branch from 5ebeb70 to 7671790 Compare February 17, 2025 15:49
@steveloughran steveloughran marked this pull request as ready for review February 17, 2025 17:03
Change-Id: Icc3c4c0b599761de92e08946bca2000f3bcb7d6c
Copy link
Contributor

@cnauroth cnauroth left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Once again putting a +1 (non-binding) on this, now that it has been updated to be consistent with the testing strategy from #49893 and #49898. Thanks again, @steveloughran .

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1, LGTM. Thank you, @steveloughran and @cnauroth .

Merged to master for Apache Spark 4.1.0.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants