[
https://issues.apache.org/jira/browse/HADOOP-16350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16863558#comment-16863558
]
Greg Senia commented on HADOOP-16350:
-------------------------------------
And I found why I made the change in the 3.x code line is because in
DFSClient.java the serverdefaults are passed in with no way to override them
hence the need for the custom property. By going and fetching serverDefaults
you automatically have a value from the NN.
{code:java}
/hadoop/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@Override
public URI getKeyProviderUri() throws IOException {
return HdfsKMSUtil.getKeyProviderUri(ugi, namenodeUri,
getServerDefaults().getKeyProviderUri(), conf);
}
{code}
And even if you attempt to override it in
/hadoop/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
isHdfsEncryptionEnabled it will call the above method and return a URI
{code:java}
/**
* Probe for encryption enabled on this filesystem.
* @return true if encryption is enabled
*/
boolean isHDFSEncryptionEnabled() throws IOException {
return getKeyProviderUri() != null;
}{code}
Hence the following code changes below in:
{code:java}
hadoop/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
public static URI getKeyProviderUri(UserGroupInformation ugi,
URI namenodeUri, String keyProviderUriStr, Configuration conf)
throws IOException {
URI keyProviderUri = null;
// Lookup the secret in credentials object for namenodeuri.
Credentials credentials = ugi.getCredentials();
Text credsKey = getKeyProviderMapKey(namenodeUri);
byte[] keyProviderUriBytes =
credentials.getSecretKey(credsKey);
if(keyProviderUriBytes != null) {
keyProviderUri =
URI.create(DFSUtilClient.bytes2String(keyProviderUriBytes));
}
if (keyProviderUri == null) {
// NN is old and doesn't report provider, so use conf.
if (keyProviderUriStr == null) {
keyProviderUri = KMSUtil.getKeyProviderUri(conf, keyProviderUriKeyName);
} else if (!keyProviderUriStr.isEmpty()) {
// Check if KMS Traffic to Remote KMS Server is allowed. Default is
allowed
Boolean isRemoteKMSAllowed =
conf.getBoolean(CommonConfigurationKeysPublic.KMS_CLIENT_ALLOW_REMOTE_KMS,
CommonConfigurationKeysPublic.KMS_CLIENT_ALLOW_REMOTE_KMS_DEFAULT);
if (isRemoteKMSAllowed) {
keyProviderUri = URI.create(keyProviderUriStr);
}
}
if (keyProviderUri != null) {
credentials.addSecretKey(
credsKey, DFSUtilClient.string2Bytes(keyProviderUri.toString()));
}
}
return keyProviderUri;
}
{code}
> Ability to tell Hadoop not to request KMS Information from Remote NN
> ---------------------------------------------------------------------
>
> Key: HADOOP-16350
> URL: https://issues.apache.org/jira/browse/HADOOP-16350
> Project: Hadoop Common
> Issue Type: Improvement
> Components: common, kms
> Affects Versions: 2.8.3, 3.0.0, 2.7.6, 3.1.2
> Reporter: Greg Senia
> Assignee: Greg Senia
> Priority: Major
> Fix For: 3.3.0
>
> Attachments: HADOOP-16350.patch
>
>
> Before HADOOP-14104 Remote KMSServer URIs were not requested from the remote
> NameNode and their associated remote KMSServer delegation token. Many
> customers were using this as a security feature to prevent TDE/Encryption
> Zone data from being distcped to remote clusters. But there was still a use
> case to allow distcp of data residing in folders that are not being encrypted
> with a KMSProvider/Encrypted Zone.
> So after upgrading to a version of Hadoop that contained HADOOP-14104 distcp
> now fails as we along with other customers (HDFS-13696) DO NOT allow
> KMSServer endpoints to be exposed out of our cluster network as data residing
> in these TDE/Zones contain very critical data that cannot be distcped between
> clusters.
> I propose adding a new code block with the following custom property
> "hadoop.security.kms.client.allow.remote.kms" it will default to "true" so
> keeping current feature of HADOOP-14104 but if specified to "false" will
> allow this area of code to operate as it did before HADOOP-14104. I can see
> the value in HADOOP-14104 but the way Hadoop worked before this JIRA/Issue
> should of at least had an option specified to allow Hadoop/KMS code to
> operate similar to how it did before by not requesting remote KMSServer URIs
> which would than attempt to get a delegation token even if not operating on
> encrypted zones.
> Error when KMS Server traffic is not allowed between cluster networks per
> enterprise security standard which cannot be changed they denied the request
> for exception so the only solution is to allow a feature to not attempt to
> request tokens.
> {code:java}
> $ hadoop distcp -Ddfs.namenode.kerberos.principal.pattern=*
> -Dmapreduce.job.hdfs-servers.token-renewal.exclude=tech
> hdfs:///processed/public/opendata/samples/distcp_test/distcp_file.txt
> hdfs://tech/processed/public/opendata/samples/distcp_test/distcp_file2.txt
> 19/05/29 14:06:09 INFO tools.DistCp: Input Options: DistCpOptions
> {atomicCommit=false, syncFolder=false, deleteMissing=false,
> ignoreFailures=false, overwrite=false, append=false, useDiff=false,
> fromSnapshot=null, toSnapshot=null, skipCRC=false, blocking=true,
> numListstatusThreads=0, maxMaps=20, mapBandwidth=100,
> sslConfigurationFile='null', copyStrategy='uniformsize', preserveStatus=[],
> preserveRawXattrs=false, atomicWorkPath=null, logPath=null,
> sourceFileListing=null,
> sourcePaths=[hdfs:/processed/public/opendata/samples/distcp_test/distcp_file.txt],
>
> targetPath=hdfs://tech/processed/public/opendata/samples/distcp_test/distcp_file2.txt,
> targetPathExists=true, filtersFile='null', verboseLog=false}
> 19/05/29 14:06:09 INFO client.AHSProxy: Connecting to Application History
> server at ha21d53mn.unit.hdp.example.com/10.70.49.2:10200
> 19/05/29 14:06:10 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token
> 5093920 for gss2002 on ha-hdfs:unit
> 19/05/29 14:06:10 INFO security.TokenCache: Got dt for hdfs://unit; Kind:
> HDFS_DELEGATION_TOKEN, Service: ha-hdfs:unit, Ident: (HDFS_DELEGATION_TOKEN
> token 5093920 for gss2002)
> 19/05/29 14:06:10 INFO security.TokenCache: Got dt for hdfs://unit; Kind:
> kms-dt, Service: ha21d53en.unit.hdp.example.com:9292, Ident: (owner=gss2002,
> renewer=yarn, realUser=, issueDate=1559153170120, maxDate=1559757970120,
> sequenceNumber=237, masterKeyId=2)
> 19/05/29 14:06:10 INFO tools.SimpleCopyListing: Paths (files+dirs) cnt = 1;
> dirCnt = 0
> 19/05/29 14:06:10 INFO tools.SimpleCopyListing: Build file listing completed.
> 19/05/29 14:06:10 INFO tools.DistCp: Number of paths in the copy list: 1
> 19/05/29 14:06:10 INFO tools.DistCp: Number of paths in the copy list: 1
> 19/05/29 14:06:10 INFO client.AHSProxy: Connecting to Application History
> server at ha21d53mn.unit.hdp.example.com/10.70.49.2:10200
> 19/05/29 14:06:10 INFO hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token
> 556079 for gss2002 on ha-hdfs:tech
> 19/05/29 14:06:10 ERROR tools.DistCp: Exception encountered
> java.io.IOException: java.net.NoRouteToHostException: No route to host (Host
> unreachable)
> at
> org.apache.hadoop.crypto.key.kms.KMSClientProvider.addDelegationTokens(KMSClientProvider.java:1029)
> at
> org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension.addDelegationTokens(KeyProviderDelegationTokenExtension.java:110)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem.addDelegationTokens(DistributedFileSystem.java:2407)
> at
> org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:140)
> at
> org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodesInternal(TokenCache.java:100)
> at
> org.apache.hadoop.mapreduce.security.TokenCache.obtainTokensForNamenodes(TokenCache.java:80)
> at
> org.apache.hadoop.tools.mapred.CopyOutputFormat.checkOutputSpecs(CopyOutputFormat.java:124)
> at org.apache.hadoop.mapreduce.JobSubmitter.checkSpecs(JobSubmitter.java:266)
> at
> org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:139)
> at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1290)
> at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1287)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1869)
> at org.apache.hadoop.mapreduce.Job.submit(Job.java:1287)
> at org.apache.hadoop.tools.DistCp.createAndSubmitJob(DistCp.java:193)
> at org.apache.hadoop.tools.DistCp.execute(DistCp.java:155)
> at org.apache.hadoop.tools.DistCp.run(DistCp.java:128)
> at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
> at org.apache.hadoop.tools.DistCp.main(DistCp.java:462)
> Caused by: java.net.NoRouteToHostException: No route to host (Host
> unreachable)
> at java.net.PlainSocketImpl.socketConnect(Native Method)
> at
> java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:350)
> at
> java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:206)
> at java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:188)
> at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392)
> at java.net.Socket.connect(Socket.java:589)
> at sun.net.NetworkClient.doConnect(NetworkClient.java:175)
> at sun.net.www.http.HttpClient.openServer(HttpClient.java:463)
> at sun.net.www.http.HttpClient.openServer(HttpClient.java:558)
> at sun.net.www.http.HttpClient.<init>(HttpClient.java:242)
> at sun.net.www.http.HttpClient.New(HttpClient.java:339)
> at sun.net.www.http.HttpClient.New(HttpClient.java:357)
> at
> sun.net.www.protocol.http.HttpURLConnection.getNewHttpClient(HttpURLConnection.java:1220)
> at
> sun.net.www.protocol.http.HttpURLConnection.plainConnect0(HttpURLConnection.java:1156)
> at
> sun.net.www.protocol.http.HttpURLConnection.plainConnect(HttpURLConnection.java:1050)
> at
> sun.net.www.protocol.http.HttpURLConnection.connect(HttpURLConnection.java:984)
> at
> org.apache.hadoop.security.authentication.client.KerberosAuthenticator.authenticate(KerberosAuthenticator.java:188)
> at
> org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator.authenticate(DelegationTokenAuthenticator.java:133)
> at
> org.apache.hadoop.security.authentication.client.AuthenticatedURL.openConnection(AuthenticatedURL.java:216)
> at
> org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator.doDelegationTokenOperation(DelegationTokenAuthenticator.java:299)
> at
> org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator.getDelegationToken(DelegationTokenAuthenticator.java:171)
> at
> org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL.getDelegationToken(DelegationTokenAuthenticatedURL.java:373)
> at
> org.apache.hadoop.crypto.key.kms.KMSClientProvider$4.run(KMSClientProvider.java:1016)
> at
> org.apache.hadoop.crypto.key.kms.KMSClientProvider$4.run(KMSClientProvider.java:1011)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1869)
> at
> org.apache.hadoop.crypto.key.kms.KMSClientProvider.addDelegationTokens(KMSClientProvider.java:1011)
> ... 19 more
> {code}
>
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]