[jira] [Comment Edited] (HADOOP-16350) Ability to tell Hadoop not to request KMS Information from Remote NN

2019-06-20 Thread Xiaoyu Yao (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16868937#comment-16868937
 ] 

Xiaoyu Yao edited comment on HADOOP-16350 at 6/20/19 9:54 PM:
--

Thanks [~gss2002] for reporting the issue and [~ajayydv] [~arp] [~szetszwo] for 
the discussion. I agree with [~ajayydv] that [~gss2002]'s earlier patch has 
missed one case where the local KMS config should still be honored even we 
configured to ignore remote KMS. 

However, with a single boolean, how to we differentiate local vs remote NN and 
their KMS configuration? With the new Hadoop clients where the recommendation 
is to be configureless wrt. local kms and letting NN tell the right KMS uri. 
This may be an issue as we will ignore all KMS returned from NNs, no matter is 
is local or remote. 


was (Author: xyao):
Thanks [~gss2002] for reporting the issue and [~ajayydv] and [~szetszwo] for 
the discussion. I agree with [~ajayydv] that [~gss2002]'s earlier patch has 
missed one case where the local KMS config should still be honored even we 
configured to ignore remote KMS. 

However, with a single boolean, how to we differentiate local vs remote NN and 
their KMS configuration? With the new Hadoop clients where the recommendation 
is to be configureless wrt. local kms and letting NN tell the right KMS uri. 
This may be an issue as we will ignore all KMS returned from NNs, no matter is 
is local or remote. 

> 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
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: HADOOP-16350.00.patch, HADOOP-16350.01.patch, 
> HADOOP-16350.02.patch, HADOOP-16350.03.patch, HADOOP-16350.04.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 

[jira] [Comment Edited] (HADOOP-16350) Ability to tell Hadoop not to request KMS Information from Remote NN

2019-06-13 Thread Greg Senia (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16863543#comment-16863543
 ] 

Greg Senia edited comment on HADOOP-16350 at 6/13/19 11:51 PM:
---

[~szetszwo] the recommended suggestion won't work with the 2.x line of code. 
The custom property is required.. I think that is where the confusion is coming 
in here. In Hadoop 2.x code is much different than 3.x I will review 3.x code 
again

 

2.x code:
{code:java}
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -3670,12 +3670,17 @@

     }

 

     // Query the namenode for the key provider uri.

+    Boolean isRemoteKMSAllowed = 

+            
conf.getBoolean(CommonConfigurationKeysPublic.KMS_CLIENT_ALLOW_REMOTE_KMS, 

+                    
CommonConfigurationKeysPublic.KMS_CLIENT_ALLOW_REMOTE_KMS_DEFAULT);

+    if (isRemoteKMSAllowed) {

     FsServerDefaults serverDefaults = getServerDefaults();

-    if (serverDefaults.getKeyProviderUri() != null) {

-      if (!serverDefaults.getKeyProviderUri().isEmpty()) {

-        keyProviderUri = URI.create(serverDefaults.getKeyProviderUri());

+      if (serverDefaults.getKeyProviderUri() != null) {

+        if (!serverDefaults.getKeyProviderUri().isEmpty()) {

+          keyProviderUri = URI.create(serverDefaults.getKeyProviderUri());

+        }

+        return keyProviderUri;

       }

-      return keyProviderUri;

     }

 

     // Last thing is to trust its own conf to be backwards compatible.
{code}
 Failure:
{code:java}
[gss2002@ha21t51en ~]$ hadoop distcp -Dhadoop.security.key.provider.path="" 
-Ddfs.namenode.kerberos.principal.pattern=* 
-Dmapreduce.job.hdfs-servers.token-renewal.exclude=unit 
hdfs:///processed/public/opendata/samples/distcp_test/distcp_file.txt 
hdfs://unit/processed/public/opendata/samples/distcp_test/distcp_file2.txt 
19/06/13 19:22:58 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://unit/processed/public/opendata/samples/distcp_test/distcp_file2.txt,
 targetPathExists=true, filtersFile='null', verboseLog=false} 19/06/13 19:22:59 
INFO client.AHSProxy: Connecting to Application History server at 
ha21t53mn.tech.hdp.example.com/10.70.33.2:10200 19/06/13 19:22:59 INFO 
hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 561611 for gss2002 on 
ha-hdfs:tech 19/06/13 19:22:59 INFO security.TokenCache: Got dt for 
hdfs://tech; Kind: HDFS_DELEGATION_TOKEN, Service: ha-hdfs:tech, Ident: 
(HDFS_DELEGATION_TOKEN token 561611 for gss2002) 19/06/13 19:22:59 INFO 
security.TokenCache: Got dt for hdfs://tech; Kind: kms-dt, Service: 
ha21t53en.tech.hdp.example.com:9292, Ident: (owner=gss2002, renewer=yarn, 
realUser=, issueDate=1560468179680, maxDate=1561072979680, sequenceNumber=7787, 
masterKeyId=92) 19/06/13 19:23:00 INFO tools.SimpleCopyListing: Paths 
(files+dirs) cnt = 1; dirCnt = 0 19/06/13 19:23:00 INFO 
tools.SimpleCopyListing: Build file listing completed. 19/06/13 19:23:00 INFO 
tools.DistCp: Number of paths in the copy list: 1 19/06/13 19:23:01 INFO 
tools.DistCp: Number of paths in the copy list: 1 19/06/13 19:23:01 INFO 
client.AHSProxy: Connecting to Application History server at 
ha21t53mn.tech.hdp.example.com/10.70.33.2:10200 19/06/13 19:23:01 INFO 
hdfs.DFSClient: Created HDFS_DELEGATION_TOKEN token 5144031 for gss2002 on 
ha-hdfs:unit 19/06/13 19:23:01 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 

[jira] [Comment Edited] (HADOOP-16350) Ability to tell Hadoop not to request KMS Information from Remote NN

2019-06-13 Thread Greg Senia (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16863061#comment-16863061
 ] 

Greg Senia edited comment on HADOOP-16350 at 6/13/19 1:15 PM:
--

[~szetszwo] and [~jojochuang] here is what I didn't send along.. If you look at 
the code you will see that this allows it to operate as it did before by adding 
this param to distcp job it will operate as it did before the patch as that 
code path in the if block is what goes to remote server and in our case the 
remote server is not even exposed to remote networks.

*hadoop.security.kms.client.allow.remote.kms=false* 

 

hadoop distcp -*Dhadoop.security.kms.client.allow.remote.kms=false* 
-Ddfs.namenode.kerberos.principal.pattern=* 
-Dmapreduce.job.hdfs-servers.token-renewal.exclude=unit 
hdfs:///processed/public/opendata/samples/distcp_test/distcp_file.txt 
hdfs://unit/processed/public/opendata/samples/distcp_test/distcp_file2.txt

 

Here is the code that allows it to work by adding the if block the default is 
true so that nothing changes and if you set to false via the prop the remote 
call to the remote cluster does not occur
{code:java}
diff --git 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index 0fa78c2..77222ee 100644
--- 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -837,6 +837,16 @@
* 
* core-default.xml
*/
+  public static final String KMS_CLIENT_ALLOW_REMOTE_KMS =
+  "hadoop.security.kms.client.allow.remote.kms";
+  /** Default value for KMS_CLIENT_ALLOW_REMOTE_KMS (true)*/
+  public static final boolean KMS_CLIENT_ALLOW_REMOTE_KMS_DEFAULT = true;
+
+  /**
+   * @see
+   * 
+   * core-default.xml
+   */
   public static final String HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_KEY 
= 
 "hadoop.security.java.secure.random.algorithm";
   /** Defalt value for HADOOP_SECURITY_JAVA_SECURE_RANDOM_ALGORITHM_KEY */
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
index 30e8aa7..c6daae0 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsKMSUtil.java
@@ -145,7 +145,13 @@
   if (keyProviderUriStr == null) {
 keyProviderUri = KMSUtil.getKeyProviderUri(conf, 
keyProviderUriKeyName);
   } else if (!keyProviderUriStr.isEmpty()) {
-keyProviderUri = URI.create(keyProviderUriStr);
+// 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(
{code}
 


was (Author: gss2002):
[~szetszwo] and [~jojochuang] here is what I didn't send along.. If you look at 
the code you will see that this allows it to operate as it did before by adding 
this param to distcp job it will operate as it did before the patch as that 
code path in the if block is what goes to remote server and in our case the 
remote server is not even exposed to remote networks.

*hadoop.security.kms.client.allow.remote.kms=false* 

 

 hadoop distcp -*Dhadoop.security.kms.client.allow.remote.kms=false* 
-Ddfs.namenode.kerberos.principal.pattern=* 
-Dmapreduce.job.hdfs-servers.token-renewal.exclude=unit 
hdfs:///processed/public/opendata/samples/distcp_test/distcp_file.txt 
hdfs://unit/processed/public/opendata/samples/distcp_test/distcp_file2.txt

 

Here is the code that allows it to work by adding the if block the default is 
true so that nothing changes and if you set to false via the:
{code:java}
diff --git 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
index 0fa78c2..77222ee 100644
--- 
a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
+++ 
b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java
@@ -837,6 +837,16 @@
* 
* core-default.xml
*/
+  public static final String 

[jira] [Comment Edited] (HADOOP-16350) Ability to tell Hadoop not to request KMS Information from Remote NN

2019-06-11 Thread Wei-Chiu Chuang (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16861674#comment-16861674
 ] 

Wei-Chiu Chuang edited comment on HADOOP-16350 at 6/12/19 2:04 AM:
---

Hi [~gss2002] I think I understand the problem statement, but I don't really 
feel this is the proper solution.

In fact, even prior to HADOOP-14104, it was still possible to distcp out of 
encryption zone (in fact, this is [a feature supported by Cloudera 
BDR|https://www.cloudera.com/documentation/enterprise/5-8-x/topics/cm_bdr_replication_and_encryption.html#xd_583c10bfdbd326ba-5676e95c-13ed333c3d9--7ff3])
 by acquiring kms delegation token from the remote cluster manually.

But this is an interesting problem and I've never thought about this use case. 
Don't have a good answer now.


was (Author: jojochuang):
Hi [~gss2002] I think I understand the problem statement, but I don't really 
feel this is the proper solution.

In fact, even prior to HADOOP-14104, it was still possible to distcp out of 
encryption zone (in fact, this is a feature supported by Cloudera BDR) by 
acquiring kms delegation token from the remote cluster manually.

But this is an interesting problem and I've never thought about this use case. 
Don't have a good answer now.

> 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: 

[jira] [Comment Edited] (HADOOP-16350) Ability to tell Hadoop not to request KMS Information from Remote NN

2019-06-11 Thread Greg Senia (JIRA)


[ 
https://issues.apache.org/jira/browse/HADOOP-16350?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16861661#comment-16861661
 ] 

Greg Senia edited comment on HADOOP-16350 at 6/12/19 1:46 AM:
--

[~szetszwo], No problem it solves the interim problem for us.  The real problem 
is we use KMSClientProvider locally in the cluster so what we are trying to 
solve for is to allow for the code to operate is it did before HADOOP-14104. In 
our case we want to just disable call that is made we have verified that the 
patch I provided when set to false reverts the code to operate is it did before 
HADOOP-14104. Please provide more details on how you plan to go about making it 
operate as it did before HADOOP-14104 as I am willing to modify the patch but 
we need the same functionality that existed before HADOOP-14104. Not trying to 
be a pain but our customers and security team basically relied on this feature 
to prevent folders that contained TDE/Encrypted data from being moved from 
cluster to cluster.


was (Author: gss2002):
[~szetszwo] the problem is we use KMSClientProvider locally in the cluster what 
we are trying to solve for is to allow for the code to operate is it did before 
HADOOP-14104. In our case we want to just disable call that is made we have 
verified that the patch I provided when set to false reverts the code to 
operate is it did before HADOOP-14104. Please provide more details on how you 
plan to go about making it operate as it did before HADOOP-14104 as I am 
willing to modify the patch but we need the same functionality that existed 
before HADOOP-14104.

> 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