[jira] [Commented] (HADOOP-15915) Report problems w/ local S3A buffer directory meaningfully

2022-10-20 Thread Zbigniew Kostrzewa (Jira)


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

Zbigniew Kostrzewa commented on HADOOP-15915:
-

I've recently stumbled upon this with {{{}3.2.2{}}}. For me the problem was 
that I did not change {{hadoop.tmp.dir}} and so the {{s3ablock-0001-}} were 
created in {{/tmp/hadoop-/s3a}} directory. At the same time, 
on CentOS 7 in my case, there is a systemd service 
{{systemd-tmpfiles-clean.service}} run once a day which cleans up {{/tmp}} of 
files and directories older than 10 days. However, Node Manager after it caches 
that {{/tmp/hadoop-/s3a}} exists it does not re-check it and 
does not re-create that directory if it no longer exists, I believe the code 
responsible for this is:
{code:java}
  /** This method gets called everytime before any read/write to make sure
     * that any change to localDirs is reflected immediately.
     */
    private Context confChanged(Configuration conf)
        throws IOException {
      ...
      if (!newLocalDirs.equals(ctx.savedLocalDirs)) { {code}
and when the directory is missing log aggregation fails with this 
{{DiskChecker}} error.

> Report problems w/ local S3A buffer directory meaningfully
> --
>
> Key: HADOOP-15915
> URL: https://issues.apache.org/jira/browse/HADOOP-15915
> Project: Hadoop Common
>  Issue Type: Sub-task
>  Components: fs/s3
>Affects Versions: 3.1.1
>Reporter: Steve Loughran
>Priority: Major
>
> When there's a problem working with the temp directory used for block output 
> and the staging committers the actual path (and indeed config option) aren't 
> printed. 
> Improvements: tell the user which directory isn't writeable



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[jira] [Updated] (HADOOP-17568) Mapred/YARN job fails due to kms-dt can't be found in cache with LoadBalancingKMSClientProvider + Kerberos

2021-04-25 Thread Zbigniew Kostrzewa (Jira)


 [ 
https://issues.apache.org/jira/browse/HADOOP-17568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Zbigniew Kostrzewa updated HADOOP-17568:

Component/s: documentation

> Mapred/YARN job fails due to kms-dt can't be found in cache with 
> LoadBalancingKMSClientProvider + Kerberos
> --
>
> Key: HADOOP-17568
> URL: https://issues.apache.org/jira/browse/HADOOP-17568
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: documentation, kms, security
>Affects Versions: 3.2.2
>Reporter: Zbigniew Kostrzewa
>Priority: Major
>
> I deployed Hadoop 3.2.2 cluster with KMS in HA using 
> LoadBalancingKMSClientProvider with Kerberos authentication. KMS instances 
> are configured with ZooKeeper for storing the shared secret.
> I have created an encryption key and an encryption zone in `/test` directory 
> and executed `randomtextwriter` from mapreduce examples passing it a 
> sub-directory in the encryption zone:
> {code:java}
> hadoop jar hadoop-mapreduce-examples-3.2.2.jar randomtextwriter 
> /test/randomtextwriter
> {code}
> Unfortunately the job keeps failing with errors like:
> {code:java}
> java.io.IOException: 
> org.apache.hadoop.security.authentication.client.AuthenticationException: 
> org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
> owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
> maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in 
> cache
>   at 
> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:363)
>   at 
> org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:532)
>   at 
> org.apache.hadoop.hdfs.HdfsKMSUtil.decryptEncryptedDataEncryptionKey(HdfsKMSUtil.java:212)
>   at 
> org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:972)
>   at 
> org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:952)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:536)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:530)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:544)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:471)
>   at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1125)
>   at 
> org.apache.hadoop.io.SequenceFile$Writer.(SequenceFile.java:1168)
>   at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:285)
>   at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:542)
>   at 
> org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getSequenceWriter(SequenceFileOutputFormat.java:64)
>   at 
> org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getRecordWriter(SequenceFileOutputFormat.java:75)
>   at 
> org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.(MapTask.java:659)
>   at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:779)
>   at org.apache.hadoop.mapred.MapTask.run(MapTask.java:347)
>   at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:174)
>   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:1762)
>   at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:168)
> Caused by: 
> org.apache.hadoop.security.authentication.client.AuthenticationException: 
> org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
> owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
> maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in 
> cache
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>   at 
> org.apache.hadoop.util.HttpExceptionUtils.validateResponse(HttpExceptionUtils.java:154)
>   at 
> org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:592)
>   at 
> org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:540)
>   at 
> org.apache.hadoop.crypto.key.kms.KMSClientProvider.dec

[jira] [Commented] (HADOOP-17568) Mapred/YARN job fails due to kms-dt can't be found in cache with LoadBalancingKMSClientProvider + Kerberos

2021-03-11 Thread Zbigniew Kostrzewa (Jira)


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

Zbigniew Kostrzewa commented on HADOOP-17568:
-

While inspecting KMS related source code I found out an undocumented property 
{{hadoop.kms.authentication.zk-dt-secret-manager.enable}} , after setting it to 
{{true}} in {{kms-site.xml}} everything started working. However, since this is 
undocumented should I be needing to enable it to get KMS HA with 
LoadBalancingKMSClientProvider working?

> Mapred/YARN job fails due to kms-dt can't be found in cache with 
> LoadBalancingKMSClientProvider + Kerberos
> --
>
> Key: HADOOP-17568
> URL: https://issues.apache.org/jira/browse/HADOOP-17568
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: kms, security
>Affects Versions: 3.2.2
>Reporter: Zbigniew Kostrzewa
>Priority: Major
>
> I deployed Hadoop 3.2.2 cluster with KMS in HA using 
> LoadBalancingKMSClientProvider with Kerberos authentication. KMS instances 
> are configured with ZooKeeper for storing the shared secret.
> I have created an encryption key and an encryption zone in `/test` directory 
> and executed `randomtextwriter` from mapreduce examples passing it a 
> sub-directory in the encryption zone:
> {code:java}
> hadoop jar hadoop-mapreduce-examples-3.2.2.jar randomtextwriter 
> /test/randomtextwriter
> {code}
> Unfortunately the job keeps failing with errors like:
> {code:java}
> java.io.IOException: 
> org.apache.hadoop.security.authentication.client.AuthenticationException: 
> org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
> owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
> maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in 
> cache
>   at 
> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:363)
>   at 
> org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:532)
>   at 
> org.apache.hadoop.hdfs.HdfsKMSUtil.decryptEncryptedDataEncryptionKey(HdfsKMSUtil.java:212)
>   at 
> org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:972)
>   at 
> org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:952)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:536)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:530)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:544)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:471)
>   at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1125)
>   at 
> org.apache.hadoop.io.SequenceFile$Writer.(SequenceFile.java:1168)
>   at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:285)
>   at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:542)
>   at 
> org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getSequenceWriter(SequenceFileOutputFormat.java:64)
>   at 
> org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getRecordWriter(SequenceFileOutputFormat.java:75)
>   at 
> org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.(MapTask.java:659)
>   at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:779)
>   at org.apache.hadoop.mapred.MapTask.run(MapTask.java:347)
>   at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:174)
>   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:1762)
>   at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:168)
> Caused by: 
> org.apache.hadoop.security.authentication.client.AuthenticationException: 
> org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
> owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
> maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in 
> cache
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>   at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
>  

[jira] [Updated] (HADOOP-17568) Mapred/YARN job fails due to kms-dt can't be found in cache with LoadBalancingKMSClientProvider + Kerberos

2021-03-08 Thread Zbigniew Kostrzewa (Jira)


 [ 
https://issues.apache.org/jira/browse/HADOOP-17568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Zbigniew Kostrzewa updated HADOOP-17568:

Description: 
I deployed Hadoop 3.2.2 cluster with KMS in HA using 
LoadBalancingKMSClientProvider with Kerberos authentication. KMS instances are 
configured with ZooKeeper for storing the shared secret.

I have created an encryption key and an encryption zone in `/test` directory 
and executed `randomtextwriter` from mapreduce examples passing it a 
sub-directory in the encryption zone:
{code:java}
hadoop jar hadoop-mapreduce-examples-3.2.2.jar randomtextwriter 
/test/randomtextwriter
{code}
Unfortunately the job keeps failing with errors like:
{code:java}
java.io.IOException: 
org.apache.hadoop.security.authentication.client.AuthenticationException: 
org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in cache
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:363)
at 
org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:532)
at 
org.apache.hadoop.hdfs.HdfsKMSUtil.decryptEncryptedDataEncryptionKey(HdfsKMSUtil.java:212)
at 
org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:972)
at 
org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:952)
at 
org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:536)
at 
org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:530)
at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:544)
at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:471)
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1125)
at 
org.apache.hadoop.io.SequenceFile$Writer.(SequenceFile.java:1168)
at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:285)
at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:542)
at 
org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getSequenceWriter(SequenceFileOutputFormat.java:64)
at 
org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getRecordWriter(SequenceFileOutputFormat.java:75)
at 
org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.(MapTask.java:659)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:779)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:347)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:174)
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:1762)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:168)
Caused by: 
org.apache.hadoop.security.authentication.client.AuthenticationException: 
org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in cache
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at 
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at 
org.apache.hadoop.util.HttpExceptionUtils.validateResponse(HttpExceptionUtils.java:154)
at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:592)
at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:540)
at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.decryptEncryptedKey(KMSClientProvider.java:833)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:356)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:352)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.doOp(LoadBalancingKMSClientProvider.java:174)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:352)
{code}
 

I've injected a few logs on my own and it seems that the client gets 403 on 
"decrypt" request

[jira] [Commented] (HADOOP-17568) Mapred/YARN job fails due to kms-dt can't be found in cache with LoadBalancingKMSClientProvider + Kerberos

2021-03-08 Thread Zbigniew Kostrzewa (Jira)


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

Zbigniew Kostrzewa commented on HADOOP-17568:
-

[~aajisaka]

Thanks your comment. Actually I had 
`hadoop.kms.authentication.signer.secret.provider.zookeeper.auth.type` set to 
`none`. Nevertheless, I just re-configured my ZooKeeper to use Kerberos and 
updated `kms-site.xml` accordingly. Unfortunately, that did not help.

I added to the description contents of `kms-site.xml` from one of my KMS 
instances (without Kerberos for ZooKeeper).

Do you use *LoadBalancingKMSClientProvider* or *Load-Balancer or VIP*?

> Mapred/YARN job fails due to kms-dt can't be found in cache with 
> LoadBalancingKMSClientProvider + Kerberos
> --
>
> Key: HADOOP-17568
> URL: https://issues.apache.org/jira/browse/HADOOP-17568
> Project: Hadoop Common
>  Issue Type: Bug
>  Components: kms, security
>Affects Versions: 3.2.2
>Reporter: Zbigniew Kostrzewa
>Priority: Major
>
> I deployed Hadoop 3.2.2 cluster with KMS in HA using 
> LoadBalancingKMSClientProvider with Kerberos authentication. KMS instances 
> are configured with ZooKeeper for storing the shared secret.
> I have created an encryption key and an encryption zone in `/test` directory 
> and executed `randomtextwriter` from mapreduce examples passing it a 
> sub-directory in the encryption zone:
> {code:java}
> hadoop jar hadoop-mapreduce-examples-3.2.2.jar randomtextwriter 
> /test/randomtextwriter
> {code}
> Unfortunately the job keeps failing with errors like:
> {code:java}
> java.io.IOException: 
> org.apache.hadoop.security.authentication.client.AuthenticationException: 
> org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
> owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
> maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in 
> cache
>   at 
> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:363)
>   at 
> org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:532)
>   at 
> org.apache.hadoop.hdfs.HdfsKMSUtil.decryptEncryptedDataEncryptionKey(HdfsKMSUtil.java:212)
>   at 
> org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:972)
>   at 
> org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:952)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:536)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:530)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:544)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:471)
>   at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1125)
>   at 
> org.apache.hadoop.io.SequenceFile$Writer.(SequenceFile.java:1168)
>   at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:285)
>   at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:542)
>   at 
> org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getSequenceWriter(SequenceFileOutputFormat.java:64)
>   at 
> org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getRecordWriter(SequenceFileOutputFormat.java:75)
>   at 
> org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.(MapTask.java:659)
>   at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:779)
>   at org.apache.hadoop.mapred.MapTask.run(MapTask.java:347)
>   at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:174)
>   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:1762)
>   at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:168)
> Caused by: 
> org.apache.hadoop.security.authentication.client.AuthenticationException: 
> org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
> owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
> maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in 
> cache
>   at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>   at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>   at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(Deleg

[jira] [Updated] (HADOOP-17568) Mapred/YARN job fails due to kms-dt can't be found in cache with LoadBalancingKMSClientProvider + Kerberos

2021-03-08 Thread Zbigniew Kostrzewa (Jira)


 [ 
https://issues.apache.org/jira/browse/HADOOP-17568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Zbigniew Kostrzewa updated HADOOP-17568:

Description: 
I deployed Hadoop 3.2.2 cluster with KMS in HA using 
LoadBalancingKMSClientProvider with Kerberos authentication. KMS instances are 
configured with ZooKeeper for storing the shared secret.

I have created an encryption key and an encryption zone in `/test` directory 
and executed `randomtextwriter` from mapreduce examples passing it a 
sub-directory in the encryption zone:
{code:java}
hadoop jar hadoop-mapreduce-examples-3.2.2.jar randomtextwriter 
/test/randomtextwriter
{code}
Unfortunately the job keeps failing with errors like:
{code:java}
java.io.IOException: 
org.apache.hadoop.security.authentication.client.AuthenticationException: 
org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in cache
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:363)
at 
org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:532)
at 
org.apache.hadoop.hdfs.HdfsKMSUtil.decryptEncryptedDataEncryptionKey(HdfsKMSUtil.java:212)
at 
org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:972)
at 
org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:952)
at 
org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:536)
at 
org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:530)
at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:544)
at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:471)
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1125)
at 
org.apache.hadoop.io.SequenceFile$Writer.(SequenceFile.java:1168)
at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:285)
at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:542)
at 
org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getSequenceWriter(SequenceFileOutputFormat.java:64)
at 
org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getRecordWriter(SequenceFileOutputFormat.java:75)
at 
org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.(MapTask.java:659)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:779)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:347)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:174)
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:1762)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:168)
Caused by: 
org.apache.hadoop.security.authentication.client.AuthenticationException: 
org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in cache
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at 
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at 
org.apache.hadoop.util.HttpExceptionUtils.validateResponse(HttpExceptionUtils.java:154)
at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:592)
at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:540)
at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.decryptEncryptedKey(KMSClientProvider.java:833)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:356)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:352)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.doOp(LoadBalancingKMSClientProvider.java:174)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:352)
{code}
 

I've injected a few logs on my own and it seems that the client gets 403 on 
"decrypt" request

[jira] [Created] (HADOOP-17568) Mapred/YARN job fails due to kms-dt can't be found in cache with LoadBalancingKMSClientProvider + Kerberos

2021-03-07 Thread Zbigniew Kostrzewa (Jira)
Zbigniew Kostrzewa created HADOOP-17568:
---

 Summary: Mapred/YARN job fails due to kms-dt can't be found in 
cache with LoadBalancingKMSClientProvider + Kerberos
 Key: HADOOP-17568
 URL: https://issues.apache.org/jira/browse/HADOOP-17568
 Project: Hadoop Common
  Issue Type: Bug
  Components: kms, security
Affects Versions: 3.2.2
Reporter: Zbigniew Kostrzewa


I deployed Hadoop 3.2.2 cluster with KMS in HA using 
LoadBalancingKMSClientProvider with Kerberos authentication. KMS instances are 
configured with ZooKeeper for storing the shared secret.

I have created an encryption key and an encryption zone in `/test` directory 
and executed `randomtextwriter` from mapreduce examples passing it a 
sub-directory in the encryption zone:
{code:java}
hadoop jar hadoop-mapreduce-examples-3.2.2.jar randomtextwriter 
/test/randomtextwriter
{code}
Unfortunately the job keeps failing with errors like:
{code:java}
java.io.IOException: 
org.apache.hadoop.security.authentication.client.AuthenticationException: 
org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in cache
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:363)
at 
org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:532)
at 
org.apache.hadoop.hdfs.HdfsKMSUtil.decryptEncryptedDataEncryptionKey(HdfsKMSUtil.java:212)
at 
org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:972)
at 
org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:952)
at 
org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:536)
at 
org.apache.hadoop.hdfs.DistributedFileSystem$8.doCall(DistributedFileSystem.java:530)
at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:544)
at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:471)
at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1125)
at 
org.apache.hadoop.io.SequenceFile$Writer.(SequenceFile.java:1168)
at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:285)
at org.apache.hadoop.io.SequenceFile.createWriter(SequenceFile.java:542)
at 
org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getSequenceWriter(SequenceFileOutputFormat.java:64)
at 
org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat.getRecordWriter(SequenceFileOutputFormat.java:75)
at 
org.apache.hadoop.mapred.MapTask$NewDirectOutputCollector.(MapTask.java:659)
at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:779)
at org.apache.hadoop.mapred.MapTask.run(MapTask.java:347)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:174)
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:1762)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:168)
Caused by: 
org.apache.hadoop.security.authentication.client.AuthenticationException: 
org.apache.hadoop.security.token.SecretManager$InvalidToken: token (kms-dt 
owner=packer, renewer=packer, realUser=, issueDate=1615146155993, 
maxDate=1615750955993, sequenceNumber=1, masterKeyId=2) can't be found in cache
at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
at 
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
at 
org.apache.hadoop.util.HttpExceptionUtils.validateResponse(HttpExceptionUtils.java:154)
at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:592)
at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:540)
at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.decryptEncryptedKey(KMSClientProvider.java:833)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:356)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:352)
at 
org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.doOp(LoadBalancingKMSClie