LOKESKUMAR VIJAYAKUMAR created HDFS-13965:
---------------------------------------------

             Summary: hadoop.security.kerberos.ticket.cache.path setting is not 
honored when KMS encryption is enabled.
                 Key: HDFS-13965
                 URL: https://issues.apache.org/jira/browse/HDFS-13965
             Project: Hadoop HDFS
          Issue Type: Bug
          Components: hdfs-client, kms
    Affects Versions: 2.7.7, 2.7.3
            Reporter: LOKESKUMAR VIJAYAKUMAR


_We use the *+hadoop.security.kerberos.ticket.cache.path+* setting to provide a 
custom kerberos cache path for all hadoop operations to be run as specified 
user. But this setting is not honored when KMS encryption is enabled._


_The below program to read a file works when KMS encryption is not enabled, but 
it fails when the KMS encryption is enabled._

_Looks like *hadoop.security.kerberos.ticket.cache.path* setting is not honored 
by *createConnection on KMSClientProvider.java.*_

 

HadoopTest.java (CLASSPATH needs to be set to compile and run)

 

import java.io.InputStream;

import java.net.URI;

import org.apache.hadoop.conf.Configuration;

import org.apache.hadoop.fs.FileSystem;

import org.apache.hadoop.fs.Path;

 

public class HadoopTest {

        public static int runRead(String[] args) throws Exception{

                if (args.length < 3) {

                        System.err.println("HadoopTest hadoop_file_path 
hadoop_user kerberos_cache");

                        return 1;

                }

                Path inputPath = new Path(args[0]);

                Configuration conf = new Configuration();

                URI defaultURI = FileSystem.getDefaultUri(conf);

                conf.set("hadoop.security.kerberos.ticket.cache.path",args[2]);

                FileSystem fs = FileSystem.newInstance(defaultURI,conf,args[1]);

                InputStream is = fs.open(inputPath);

                byte[] buffer = new byte[4096];

                int nr = is.read(buffer);

                while (nr != -1)

                {

                        System.out.write(buffer, 0, nr);

                        nr = is.read(buffer);

                }

                return 0;

        }

        public static void main( String[] args ) throws Exception {

                int returnCode = HadoopTest.runRead(args);

                System.exit(returnCode);

        }

}

 

 

 

[root@lstrost3 testhadoop]# pwd

/testhadoop

 

[root@lstrost3 testhadoop]# ls

HadoopTest.java

 

[root@lstrost3 testhadoop]# export CLASSPATH=`hadoop classpath --glob`:.

 

[root@lstrost3 testhadoop]# javac HadoopTest.java

 

[root@lstrost3 testhadoop]# java HadoopTest

HadoopTest  hadoop_file_path  hadoop_user  kerberos_cache

 

[root@lstrost3 testhadoop]# java HadoopTest /loki/loki.file loki 
/tmp/krb5cc_1006

18/09/27 23:23:20 WARN util.NativeCodeLoader: Unable to load native-hadoop 
library for your platform... using builtin-java classes where applicable

18/09/27 23:23:21 WARN shortcircuit.DomainSocketFactory: The short-circuit 
local reads feature cannot be used because libhadoop cannot be loaded.

Exception in thread "main" java.io.IOException: 
org.apache.hadoop.security.authentication.client.AuthenticationException: 
GSSException: *{color:#FF0000}No valid credentials provided (Mechanism level: 
Failed to find any Kerberos tgt){color}*

        at 
{color:#FF0000}*org.apache.hadoop.crypto.key.kms.KMSClientProvider.createConnection(KMSClientProvider.java:551)*{color}

        at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.decryptEncryptedKey(KMSClientProvider.java:831)

        at 
org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:388)

        at 
org.apache.hadoop.hdfs.DFSClient.decryptEncryptedDataEncryptionKey(DFSClient.java:1393)

        at 
org.apache.hadoop.hdfs.DFSClient.createWrappedInputStream(DFSClient.java:1463)

        at 
org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:333)

        at 
org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:327)

        at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)

        at 
org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:340)

        at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:786)

        at HadoopTest.runRead(HadoopTest.java:18)

        at HadoopTest.main(HadoopTest.java:29)

Caused by: 
org.apache.hadoop.security.authentication.client.AuthenticationException: 
GSSException: No valid credentials provided (Mechanism level: Failed to find 
any Kerberos tgt)

        at 
org.apache.hadoop.security.authentication.client.KerberosAuthenticator.doSpnegoSequence(KerberosAuthenticator.java:333)

        at 
org.apache.hadoop.security.authentication.client.KerberosAuthenticator.authenticate(KerberosAuthenticator.java:203)

        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.DelegationTokenAuthenticatedURL.openConnection(DelegationTokenAuthenticatedURL.java:322)

        at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider$1.run(KMSClientProvider.java:542)

        at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider$1.run(KMSClientProvider.java:537)

        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.createConnection(KMSClientProvider.java:536)

        ... 11 more

Caused by: GSSException: No valid credentials provided (Mechanism level: Failed 
to find any Kerberos tgt)

        at 
sun.security.jgss.krb5.Krb5InitCredential.getInstance(Krb5InitCredential.java:147)

        at 
sun.security.jgss.krb5.Krb5MechFactory.getCredentialElement(Krb5MechFactory.java:122)

        at 
sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:187)

        at 
sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:224)

       at 
sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:212)

        at 
sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179)

        at 
org.apache.hadoop.security.authentication.client.KerberosAuthenticator$1.run(KerberosAuthenticator.java:309)

        at 
org.apache.hadoop.security.authentication.client.KerberosAuthenticator$1.run(KerberosAuthenticator.java:285)

        at java.security.AccessController.doPrivileged(Native Method)

        at javax.security.auth.Subject.doAs(Subject.java:422)

        at 
org.apache.hadoop.security.authentication.client.KerberosAuthenticator.doSpnegoSequence(KerberosAuthenticator.java:285)

        ... 21 more

 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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

Reply via email to