-
Notifications
You must be signed in to change notification settings - Fork 39
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
Adding HiveServer Credential Provider #210
Open
georgepachitariu
wants to merge
8
commits into
jcrist:master
Choose a base branch
from
georgepachitariu:master
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
6330e04
Adding HiveServer Credential Provider
georgepachitariu 5fe9f54
I fixed a problem
georgepachitariu 7bebfef
I should not have added this file
georgepachitariu 58e419f
I cleaned 2 files
georgepachitariu 397034c
log4j ended up being added twice and we had this problem https://stac…
georgepachitariu 16f669e
I put back the exception message
georgepachitariu 4856530
Implemented the notes from the first review
georgepachitariu 85d3c5c
Added HCat credential provider
georgepachitariu File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
165 changes: 165 additions & 0 deletions
165
java/src/main/java/com/anaconda/skein/DelegationTokenManager.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,165 @@ | ||
package com.anaconda.skein; | ||
|
||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.FileSystem; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.hadoop.hive.conf.HiveConf; | ||
import org.apache.hadoop.io.DataOutputBuffer; | ||
import org.apache.hadoop.io.Text; | ||
import org.apache.hadoop.mapreduce.security.TokenCache; | ||
import org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenIdentifier; | ||
import org.apache.hadoop.security.Credentials; | ||
import org.apache.hadoop.security.SaslRpcServer; | ||
import org.apache.hadoop.security.UserGroupInformation; | ||
import org.apache.hadoop.security.token.Token; | ||
import org.apache.hadoop.security.token.TokenIdentifier; | ||
import org.apache.hadoop.yarn.client.ClientRMProxy; | ||
import org.apache.hadoop.yarn.client.api.YarnClient; | ||
import org.apache.hadoop.yarn.conf.YarnConfiguration; | ||
import org.apache.hadoop.yarn.exceptions.YarnException; | ||
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier; | ||
import org.apache.hadoop.yarn.util.ConverterUtils; | ||
import org.apache.hive.hcatalog.api.HCatClient; | ||
import org.apache.hive.hcatalog.common.HCatException; | ||
import org.apache.hive.jdbc.HiveConnection; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
|
||
import java.io.IOException; | ||
import java.nio.ByteBuffer; | ||
import java.sql.Connection; | ||
import java.sql.DriverManager; | ||
import java.sql.SQLException; | ||
import java.util.LinkedList; | ||
import java.util.List; | ||
import java.util.Map; | ||
|
||
import static org.apache.commons.lang.SystemUtils.USER_NAME; | ||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_RPC_PROTECTION; | ||
|
||
public class DelegationTokenManager { | ||
private static final Logger LOG = LoggerFactory.getLogger(DelegationTokenManager.class); | ||
private Credentials credentials; | ||
private final List<Model.DelegationTokenProvider> delegationTokenProviders; | ||
|
||
public DelegationTokenManager() { this.delegationTokenProviders=new LinkedList<>(); } | ||
|
||
public void addTokenProvider(Model.DelegationTokenProvider p) { this.delegationTokenProviders.add(p); } | ||
|
||
public void initializeCredentials(Credentials credentials) { this.credentials = credentials; } | ||
|
||
public void obtainTokensHDFS(YarnClient yarnClient, FileSystem fs, Model.ApplicationSpec spec, | ||
Configuration conf) throws IOException, YarnException { | ||
// Collect security tokens as needed | ||
LOG.debug("Collecting filesystem delegation tokens"); | ||
|
||
List<Path> l= spec.getFileSystems(); | ||
l.add(0, new Path(fs.getUri())); | ||
|
||
TokenCache.obtainTokensForNamenodes(this.credentials, | ||
l.toArray(new Path[l.size()]), conf); | ||
|
||
boolean hasRMToken = false; | ||
for (Token<?> token: this.credentials.getAllTokens()) { | ||
if (token.getKind().equals(RMDelegationTokenIdentifier.KIND_NAME)) { | ||
LOG.debug("RM delegation token already acquired"); | ||
hasRMToken = true; | ||
break; | ||
} | ||
} | ||
if (!hasRMToken) { | ||
LOG.debug("Adding RM delegation token"); | ||
Text rmDelegationTokenService = ClientRMProxy.getRMDelegationTokenService(conf); | ||
String tokenRenewer = conf.get(YarnConfiguration.RM_PRINCIPAL); | ||
org.apache.hadoop.yarn.api.records.Token rmDelegationToken = | ||
yarnClient.getRMDelegationToken(new Text(tokenRenewer)); | ||
Token<TokenIdentifier> rmToken = ConverterUtils.convertFromYarn( | ||
rmDelegationToken, rmDelegationTokenService | ||
); | ||
this.credentials.addToken(rmDelegationTokenService, rmToken); | ||
} | ||
} | ||
|
||
private Text getUniqueAlias(Token<?> token) { | ||
return new Text(String.format("%s_%s_%d", token.getKind().toString(), | ||
token.getService().toString(), System.currentTimeMillis())); | ||
} | ||
|
||
// Delegation token based connection is explained here: | ||
// https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients#HiveServer2Clients-Multi-UserScenariosandProgrammaticLogintoKerberosKDC | ||
// This method is inspired from org.apache.oozie.action.hadoop.Hive2Credentials which does the same thing for Oozie. | ||
public void obtainTokensHive(Map<String, String> config, String user) { | ||
String jdbcUrl = config.get("hive.jdbc.url"); | ||
String principal = config.get("hive.jdbc.principal"); | ||
String fullUrl = jdbcUrl + ";principal=" + principal + ";hive.server2.proxy.user=" + user; | ||
|
||
try { | ||
// load the driver | ||
Class.forName("org.apache.hive.jdbc.HiveDriver"); | ||
|
||
Connection con = null; | ||
String tokenStr = null; | ||
try { | ||
con = DriverManager.getConnection(fullUrl); | ||
LOG.info("Connected successfully to " + fullUrl); | ||
tokenStr = ((HiveConnection)con).getDelegationToken(user, principal); | ||
} finally { | ||
if (con != null) { con.close(); } | ||
} | ||
LOG.info("Got Hive Server token from " + fullUrl); | ||
|
||
Token<DelegationTokenIdentifier> hiveToken = new Token<DelegationTokenIdentifier>(); | ||
hiveToken.decodeFromUrlString(tokenStr); | ||
credentials.addToken(getUniqueAlias(hiveToken), hiveToken); | ||
} catch (IOException | SQLException | ClassNotFoundException e) { | ||
e.printStackTrace(); | ||
} | ||
} | ||
|
||
// This method is inspired from org.apache.oozie.action.hadoop.HCatCredentials which does the same thing for Oozie. | ||
public void obtainTokensHCat(Map<String, String> config, String user){ | ||
String principal = config.get("hcat.metastore.principal"); | ||
String server = config.get("hcat.metastore.uri"); | ||
|
||
String protection = "authentication"; // default value | ||
if(config.containsKey(HADOOP_RPC_PROTECTION)) | ||
protection = config.get(HADOOP_RPC_PROTECTION); | ||
|
||
try { | ||
HiveConf hiveConf = new HiveConf(); | ||
hiveConf.set("hive.metastore.sasl.enabled", "true"); | ||
hiveConf.set("hive.metastore.kerberos.principal", principal); | ||
hiveConf.set("hive.metastore.local", "false"); | ||
hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, server); | ||
hiveConf.set(HADOOP_RPC_PROTECTION, protection); | ||
|
||
HCatClient hiveClient = HCatClient.create(hiveConf); | ||
String tokenStrForm = hiveClient.getDelegationToken(user, principal); | ||
Token<DelegationTokenIdentifier> hcatToken = new Token<>(); | ||
hcatToken.decodeFromUrlString(tokenStrForm); | ||
credentials.addToken(getUniqueAlias(hcatToken), hcatToken); | ||
} catch (IOException e) { | ||
e.printStackTrace(); | ||
} | ||
} | ||
|
||
// For some systems (like Hive) to obtain the delegation token we need to do it | ||
// while we are authenticated with kerberos, before we impersonate the user. | ||
public void obtainTokensWithoutImpersonation(String userWeAuthenticateFor) throws IllegalArgumentException { | ||
for (Model.DelegationTokenProvider p : this.delegationTokenProviders) { | ||
if(p.getName().equals("hive")) { | ||
this.obtainTokensHive(p.getConfig(), userWeAuthenticateFor); | ||
} else if(p.getName().equals("hcat")) { | ||
this.obtainTokensHCat(p.getConfig(), userWeAuthenticateFor); | ||
} else { | ||
throw new IllegalArgumentException("The Provider for Delegation Token was not found"); | ||
} | ||
} | ||
} | ||
|
||
public ByteBuffer toBytes() throws IOException { | ||
DataOutputBuffer dob = new DataOutputBuffer(); | ||
this.credentials.writeTokenStorageToStream(dob); | ||
return ByteBuffer.wrap(dob.getData(), 0, dob.getLength()); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If a system has hive available, is this jar likely to already be on the classpath (this is common for other hadoop libraries)? If so, we usually set
<scope>provided</scope>
for these types of dependencies to keep the size of our jar down and not worry about dependency mismatches. You should be able to drop all the exclusions as well.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is true. Thanks for explaining why I should use
provided