Skip to content
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

[MINOR] Fix inconsistency log level among delegation token providers #23418

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
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ private[security] class HBaseDelegationTokenProvider
creds.addToken(token.getService, token)
} catch {
case NonFatal(e) =>
logDebug(s"Failed to get token from service $serviceName", e)
logWarning(s"Failed to get token from service $serviceName", e)
}

None
Expand All @@ -71,7 +71,7 @@ private[security] class HBaseDelegationTokenProvider
confCreate.invoke(null, conf).asInstanceOf[Configuration]
} catch {
case NonFatal(e) =>
logDebug("Fail to invoke HBaseConfiguration", e)
logWarning("Fail to invoke HBaseConfiguration", e)
conf
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.deploy.security

import scala.collection.JavaConverters._
import scala.util.Try
import scala.util.control.NonFatal

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.FileSystem
Expand All @@ -44,28 +45,34 @@ private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: () => Set[Fil
hadoopConf: Configuration,
sparkConf: SparkConf,
creds: Credentials): Option[Long] = {
val fsToGetTokens = fileSystems()
val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fsToGetTokens, creds)
try {
val fsToGetTokens = fileSystems()
val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fsToGetTokens, creds)

// Get the token renewal interval if it is not set. It will only be called once.
if (tokenRenewalInterval == null) {
tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf, fsToGetTokens)
}
// Get the token renewal interval if it is not set. It will only be called once.
if (tokenRenewalInterval == null) {
tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf, fsToGetTokens)
}

// Get the time of next renewal.
val nextRenewalDate = tokenRenewalInterval.flatMap { interval =>
val nextRenewalDates = fetchCreds.getAllTokens.asScala
.filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier])
.map { token =>
val identifier = token
.decodeIdentifier()
.asInstanceOf[AbstractDelegationTokenIdentifier]
identifier.getIssueDate + interval
}
if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min)
}
// Get the time of next renewal.
val nextRenewalDate = tokenRenewalInterval.flatMap { interval =>
val nextRenewalDates = fetchCreds.getAllTokens.asScala
.filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier])
.map { token =>
val identifier = token
.decodeIdentifier()
.asInstanceOf[AbstractDelegationTokenIdentifier]
identifier.getIssueDate + interval
}
if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min)
}

nextRenewalDate
nextRenewalDate
} catch {
case NonFatal(e) =>
Copy link
Member

Choose a reason for hiding this comment

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

This seems to change behavior - is it pretty clearly OK, or is it necessary?

Copy link
Contributor Author

@HeartSaVioR HeartSaVioR Dec 31, 2018

Choose a reason for hiding this comment

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

This is another one of inconsistency between HDFS delegation token provider and others. While others log the error message and swallow the exception, this doesn't swallow the exception.

Personally I prefer not to swallow the error, but majority are swallowing so I just changed this. Please let me know if we think it would be better to re-raise the exception.

If we are unclear about this, I'm OK to make changes to only log level.

logWarning(s"Failed to get token from service $serviceName", e)
None
}
}

override def delegationTokensRequired(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ private[spark] class HiveDelegationTokenProvider
new HiveConf(hadoopConf, classOf[HiveConf])
} catch {
case NonFatal(e) =>
logDebug("Fail to create Hive Configuration", e)
logWarning("Fail to create Hive Configuration", e)
hadoopConf
case e: NoClassDefFoundError =>
logWarning(classNotFoundErrorStr)
Expand Down Expand Up @@ -104,7 +104,7 @@ private[spark] class HiveDelegationTokenProvider
None
} catch {
case NonFatal(e) =>
logDebug(s"Failed to get token from service $serviceName", e)
logWarning(s"Failed to get token from service $serviceName", e)
None
case e: NoClassDefFoundError =>
logWarning(classNotFoundErrorStr)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ private[security] class KafkaDelegationTokenProvider
return Some(nextRenewalDate)
} catch {
case NonFatal(e) =>
logInfo(s"Failed to get token from service $serviceName", e)
logWarning(s"Failed to get token from service $serviceName", e)
}
None
}
Expand Down