Skip to content

[SPARK-51072][CORE] CallerContext to set Hadoop cloud audit context #49779

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
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 9 additions & 1 deletion core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@ import org.apache.commons.io.IOUtils
import org.apache.commons.lang3.{JavaVersion, SystemUtils}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
import org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext
import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompressionCodec}
import org.apache.hadoop.ipc.{CallerContext => HadoopCallerContext}
import org.apache.hadoop.ipc.CallerContext.{Builder => HadoopCallerContextBuilder}
Expand Down Expand Up @@ -3171,6 +3172,9 @@ private[util] object CallerContext extends Logging {
* specific applications impacting parts of the Hadoop system and potential problems they may be
* creating (e.g. overloading NN). As HDFS mentioned in HDFS-9184, for a given HDFS operation, it's
* very helpful to track which upper level job issues it.
* The context information is also set in the audit context for cloud storage
* connectors. If supported, this gets marshalled as part of the HTTP Referrer header
* or similar field, and so ends up in the store service logs themselves.
*
* @param from who sets up the caller context (TASK, CLIENT, APPMASTER)
*
Expand Down Expand Up @@ -3221,11 +3225,15 @@ private[spark] class CallerContext(

/**
* Set up the caller context [[context]] by invoking Hadoop CallerContext API of
* [[HadoopCallerContext]].
* [[HadoopCallerContext]], which is included in IPC calls,
* and the Hadoop audit context, which may be included in cloud storage
* requests.
*/
def setCurrentContext(): Unit = if (CallerContext.callerContextEnabled) {
val hdfsContext = new HadoopCallerContextBuilder(context).build()
HadoopCallerContext.setCurrent(hdfsContext)
// set the audit context for to object stores, with the prefix "spark"
currentAuditContext.put("spark", context)
}
}

Expand Down
17 changes: 14 additions & 3 deletions core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import org.apache.commons.lang3.SystemUtils
import org.apache.commons.math3.stat.inference.ChiSquareTest
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext
import org.apache.hadoop.ipc.{CallerContext => HadoopCallerContext}
import org.apache.logging.log4j.Level

Expand Down Expand Up @@ -1003,9 +1004,19 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties {
}

test("Set Spark CallerContext") {
val context = "test"
new CallerContext(context).setCurrentContext()
assert(s"SPARK_$context" === HadoopCallerContext.getCurrent.toString)
currentAuditContext.reset
new CallerContext("test",
Some("upstream"),
Some("app"),
Some("attempt"),
Some(1),
Some(2),
Some(3),
Some(4),
Some(5)).setCurrentContext()
val expected = s"SPARK_test_app_attempt_JId_1_SId_2_3_TId_4_5_upstream"
assert(expected === HadoopCallerContext.getCurrent.toString)
assert(expected === currentAuditContext.get("spark"))
}

test("encodeFileNameToURIRawPath") {
Expand Down