Skip to content

Commit c50a32f

Browse files
committed
thanks for reviewing
1 parent 51fac11 commit c50a32f

File tree

4 files changed

+23
-11
lines changed

4 files changed

+23
-11
lines changed

core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -138,8 +138,7 @@ class SparkHadoopUtil extends Logging {
138138

139139
def loginUserFromKeytab(principalName: String, keytabFilename: String): Unit = {
140140
if (!new File(keytabFilename).exists()) {
141-
throw new SparkException(s"Keytab file: ${keytabFilename}" +
142-
" specified in spark.yarn.keytab does not exist")
141+
throw new SparkException(s"Keytab file: ${keytabFilename} does not exist")
143142
} else {
144143
logInfo("Attempting to login to Kerberos" +
145144
s" using principal: ${principalName} and keytab: ${keytabFilename}")

core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@ import javax.net.ssl._
2828

2929
import scala.annotation.tailrec
3030
import scala.collection.mutable.{ArrayBuffer, HashMap, Map}
31-
import scala.util.{Failure, Properties, Success, Try}
31+
import scala.util.Properties
3232

3333
import com.google.common.io.ByteStreams
3434
import org.apache.commons.io.FileUtils

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -230,6 +230,11 @@ private[spark] object HiveUtils extends Logging {
230230
}.toMap
231231
}
232232

233+
/**
234+
* Check current Thread's SessionState type
235+
* @return true when SessionState.get returns an instance of CliSessionState,
236+
* false when it gets non-CliSessionState instance or null
237+
*/
233238
def isCliSessionState(): Boolean = {
234239
val state = SessionState.get
235240
var temp: Class[_] = if (state != null) state.getClass else null

sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala

Lines changed: 16 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -108,8 +108,7 @@ private[hive] class HiveClientImpl(
108108
if (clientLoader.isolationOn) {
109109
// Switch to the initClassLoader.
110110
Thread.currentThread().setContextClassLoader(initClassLoader)
111-
// Set up kerberos credentials for UserGroupInformation.loginUser within
112-
// current class loader
111+
// Set up kerberos credentials for UserGroupInformation.loginUser within current class loader
113112
if (sparkConf.contains("spark.yarn.principal") && sparkConf.contains("spark.yarn.keytab")) {
114113
val principal = sparkConf.get("spark.yarn.principal")
115114
val keytab = sparkConf.get("spark.yarn.keytab")
@@ -121,7 +120,16 @@ private[hive] class HiveClientImpl(
121120
Thread.currentThread().setContextClassLoader(original)
122121
}
123122
} else {
124-
Option(SessionState.get()).getOrElse(newState())
123+
// Isolation off means we detect a CliSessionState instance in current thread. 1: Inside the
124+
// spark project, we have already started a CliSessionState in `SparkSQLCLIDriver`, which
125+
// contains information like configurations from command lines. Later, we call
126+
// `SparkSQLEnv.init()` there, which would new a hive client again into this part again. so we
127+
// should keep those configurations and reuse the existing instance of `CliSessionState`. In
128+
// this case, SessionState.get will always return a CliSessionState.
129+
// In another case, a user app may start a CliSessionState outside spark project with built in
130+
// hive jars, which will turn off isolation, after that, SessionSate.detachSession is called
131+
// to remove the current state, hive client created later will init its state by newState()
132+
Option(SessionState.get).getOrElse(newState())
125133
}
126134
}
127135

@@ -151,11 +159,11 @@ private[hive] class HiveClientImpl(
151159
// 3: we set all entries in config to this hiveConf.
152160
(hadoopConf.iterator().asScala.map(kv => kv.getKey -> kv.getValue)
153161
++ sparkConf.getAll.toMap ++ extraConfig).foreach { case (k, v) =>
154-
if (k.toLowerCase(Locale.ROOT).contains("password")) {
155-
logDebug(s"Applying Spark config to Hive Conf: $k=xxx")
156-
} else {
157-
logDebug(s"Applying Spark config to Hive Conf: $k=$v")
158-
}
162+
logDebug(
163+
s"""
164+
|Applying Hadoop/Hive/Spark and extra properties to Hive Conf:
165+
|$k=${if (k.toLowerCase(Locale.ROOT).contains("password")) "xxx" else v}
166+
""".stripMargin)
159167
hiveConf.set(k, v)
160168
}
161169
val state = new SessionState(hiveConf)

0 commit comments

Comments
 (0)