Skip to content
This repository was archived by the owner on Jan 9, 2020. It is now read-only.

Implements a server for refreshing HDFS tokens, a part of secure HDFS support. #453

Open
wants to merge 43 commits into
base: branch-2.2-kubernetes
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 2 commits
Commits
Show all changes
43 commits
Select commit Hold shift + click to select a range
7abb5ee
Add skeleton
kimoonkim Aug 19, 2017
88c0c03
Renew part
kimoonkim Aug 21, 2017
ca0b583
Compile succeeds
kimoonkim Aug 21, 2017
025e2ba
Login to kerberos
kimoonkim Aug 21, 2017
0a7a15d
Clean up constants
kimoonkim Aug 22, 2017
b3534f1
Refresh server works
kimoonkim Aug 22, 2017
cbe2777
Deployment config file
kimoonkim Aug 22, 2017
4f36793
Fix Dockerfile to match names
kimoonkim Aug 22, 2017
874b8e9
Add as independent project with own pom.xml
kimoonkim Sep 11, 2017
5dc49ca
Add working Dockerfile and deployment yaml file
kimoonkim Sep 12, 2017
388063a
Fix a bug by including hadoop conf dir in the classpath
kimoonkim Sep 13, 2017
1426523
Add token-refresh-server as extra build-only module
kimoonkim Sep 20, 2017
50c3a66
Use akka scheduler for renew tasks
kimoonkim Oct 3, 2017
c2ccaa9
Relogin to Kerberos periodically
kimoonkim Oct 3, 2017
a2aec2b
Renew at 90% mark of deadline
kimoonkim Oct 3, 2017
ec70b47
Get renew time from data item key
kimoonkim Oct 3, 2017
0b049fd
Fix compile error
kimoonkim Oct 4, 2017
d42c568
Obtain new tokens
kimoonkim Oct 4, 2017
5d96879
Fix bugs
kimoonkim Oct 4, 2017
c0e28d4
Write back tokens to K8s secret
kimoonkim Oct 4, 2017
57c847e
Handle recently added secrets
kimoonkim Oct 5, 2017
ce1bb7f
Use k8s client editable to update secret data
kimoonkim Oct 5, 2017
5162339
Add a comment
kimoonkim Oct 5, 2017
196cd8a
Keep only secret metadata in memory
kimoonkim Oct 5, 2017
56ef8e6
Fix a regex match bug
kimoonkim Oct 5, 2017
93b2acf
Tested
kimoonkim Oct 5, 2017
ba2e79a
Updated parent version
kimoonkim Oct 5, 2017
1d74579
Address review comments
kimoonkim Oct 6, 2017
95e68d3
Add TODO for token status rest endpoint
kimoonkim Oct 6, 2017
a006233
Address review comments
kimoonkim Oct 6, 2017
9dc8345
Address review comments
kimoonkim Oct 10, 2017
f4d5ee9
Support configuration
kimoonkim Oct 10, 2017
1462d2c
Fix a typo
kimoonkim Oct 11, 2017
193d0f9
Add some unit tests
kimoonkim Oct 12, 2017
eb10c4a
Add more tests
kimoonkim Oct 12, 2017
078ac2a
Clean up
kimoonkim Oct 12, 2017
aeb269a
Add more tests
kimoonkim Oct 12, 2017
87dedbc
Minor clean-up
kimoonkim Oct 13, 2017
4d1cb74
Add unit tests for renew tasks
kimoonkim Oct 13, 2017
8997f04
Verify test results more
kimoonkim Oct 13, 2017
2ed55af
Rename the new profile to kubernetes-hdfs-extra
kimoonkim Oct 18, 2017
01be03e
Fix style issues
kimoonkim Oct 18, 2017
0baaf0b
Fix Hadoop 2.7 dependency issue
kimoonkim Oct 19, 2017
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
3 changes: 2 additions & 1 deletion resource-managers/kubernetes/README.md
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,8 @@ Below is a list of the submodules for this cluster manager and what they do.

* `core`: Implementation of the Kubernetes cluster manager support.
* `token-refresh-server`: Extra Kubernetes service that refreshes Hadoop
tokens for long-running Spark jobs accessing secure data source like HDFS.
tokens for long-running Spark jobs accessing secure data sources like
Copy link
Member

Choose a reason for hiding this comment

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

accessing/that access

Copy link
Member Author

Choose a reason for hiding this comment

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

Fixed.

Kerberized HDFS.
* `integration-tests`: Integration tests for the project.
* `docker-minimal-bundle`: Base Dockerfiles for the driver and the executors. The Dockerfiles are used for integration
tests as well as being provided in packaged distributions of Spark.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
# limitations under the License.
#
---
apiVersion: extensions/v1beta1
apiVersion: apps/v1beta1
kind: Deployment
metadata:
name: hadoop-token-refresh-server
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,9 @@ private class Server {
}
}

/*
* TODO: Support REST endpoint for checking status of tokens.
*/
object TokenRefreshServer {

private class Arguments(args: List[String]) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac
case Relogin =>
launchReloginTask()
case StartRefresh(secret) =>
addStarterTask(secret)
startRefresh(secret)
case StopRefresh(secret) =>
removeRefreshTask(secret)
case UpdateSecretsToTrack(secrets) =>
Expand All @@ -69,7 +69,12 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac
scheduler.scheduleOnce(Duration(0L, TimeUnit.MILLISECONDS), task)
}

private def addStarterTask(secret: Secret) = {
private def startRefresh(secret: Secret) = {
Copy link
Member

Choose a reason for hiding this comment

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

startRefreshTask.

recentlyAddedSecretUids.add(getSecretUid(secret.getMetadata))
addRefreshTask(secret)
}

private def addRefreshTask(secret: Secret) = {
secretUidToTaskHandle.getOrElseUpdate(getSecretUid(secret.getMetadata), {
val task = new StarterTask(secret, hadoopConf, self, clock)
val cancellable = scheduler.scheduleOnce(
Expand All @@ -89,13 +94,14 @@ private class TokenRefreshService(kubernetesClient: KubernetesClient) extends Ac
}

private def updateSecretsToTrack(currentSecrets: List[Secret]) : Unit = {
val secretByUid = currentSecrets.map(secret => (getSecretUid(secret.getMetadata), secret)).toMap
val currentUids = secretByUid.keySet
val secretsByUids = currentSecrets.map(secret => (getSecretUid(secret.getMetadata), secret))
.toMap
val currentUids = secretsByUids.keySet
val priorUids = secretUidToTaskHandle.keySet
val uidsToAdd = currentUids -- priorUids
uidsToAdd.foreach(uid => addStarterTask(secretByUid(uid)))
uidsToAdd.foreach(uid => addRefreshTask(secretsByUids(uid)))
val uidsToRemove = priorUids -- currentUids -- recentlyAddedSecretUids
Copy link
Member

Choose a reason for hiding this comment

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

Where uids get added to recentlyAddedSecretUids?

Copy link
Member Author

Choose a reason for hiding this comment

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

Hmm. It seems I forgot to add to this. I'll fix the bug. It will be added whenever we get a StartRefresh command.

Copy link
Member Author

Choose a reason for hiding this comment

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

Done.

uidsToRemove.foreach(uid => removeRefreshTask(secretByUid(uid)))
uidsToRemove.foreach(uid => removeRefreshTask(secretsByUids(uid)))
recentlyAddedSecretUids.clear()
}

Expand Down Expand Up @@ -216,13 +222,16 @@ private class RenewTask(renew: Renew,
private def refresh(token: Token[_ <: TokenIdentifier], expireTime: Long, deadline: Long,
nowMillis: Long) = {
val maybeNewToken = maybeObtainNewToken(token, expireTime, nowMillis)
val maybeNewExpireTime = maybeRenewExpireTime(maybeNewToken, expireTime, deadline, nowMillis)
val maybeNewExpireTime = maybeGetNewExpireTime(maybeNewToken, expireTime, deadline, nowMillis)
(maybeNewToken, maybeNewExpireTime)
}

private def maybeObtainNewToken(token: Token[_ <: TokenIdentifier], expireTime: Long,
nowMills: Long) = {
val maybeNewToken = if (token.getKind.equals(DelegationTokenIdentifier.HDFS_DELEGATION_KIND)) {
Copy link
Member

Choose a reason for hiding this comment

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

Why is this check against DelegationTokenIdentifier.HDFS_DELEGATION_KIND needed?

Copy link
Member Author

Choose a reason for hiding this comment

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

The line below casts the token identifier to AbstractDelegationTokenIdentifier, which is possible only when the token kind matches this, according to the yarn resource manager code. And only then we can get the max date, it seems.

Copy link
Member

Choose a reason for hiding this comment

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

Got it.

// The token can casted to AbstractDelegationTokenIdentifier below only if the token kind
// is HDFS_DELEGATION_KIND, according to the YARN resource manager code. See if this can be
// generalized beyond HDFS tokens.
val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier]
val maxDate = identifier.getMaxDate
if (maxDate - expireTime < RENEW_TASK_REMAINING_TIME_BEFORE_NEW_TOKEN_MILLIS ||
Expand All @@ -241,8 +250,9 @@ private class RenewTask(renew: Renew,
maybeNewToken
}

private def maybeRenewExpireTime(token: Token[_ <: TokenIdentifier], expireTime: Long,
deadline: Long, nowMillis: Long) = {
private def maybeGetNewExpireTime(token: Token[_ <: TokenIdentifier], expireTime: Long,
deadline: Long,
nowMillis: Long) = {
if (expireTime <= deadline || expireTime <= nowMillis) {
try {
logDebug(s"Renewing token $token with current expire time $expireTime," +
Expand Down Expand Up @@ -301,7 +311,9 @@ private class RenewTask(renew: Renew,
editor.addToData(key, value)
val dataItemKeys = editor.getData.keySet().asScala.filter(
_.startsWith(SECRET_DATA_ITEM_KEY_PREFIX_HADOOP_TOKENS)).toSeq.sorted
// Remove data items except the latest two data items.
// Remove data items except the latest two data items. A K8s secret can hold only up to 1 MB
// data. We need to remove old data items. We keep the latest two items to avoid race conditions
// where some newly launching executors may access the previous token.
dataItemKeys.dropRight(2).foreach(editor.removeFromData)
editor.done
logInfo(s"Wrote new tokens $tokenToExpire to a data item $key in ${secretMeta.getSelfLink}")
Expand Down