Sorry for the my previous reply suggesting to use
UserGroupInformation#createRemoteUser(String user, AuthMethod authMethod).
As Chris said, it will not add actual credentials to the UGI.

Hi david,
 now, if I understand you correctly, You want to access the secure cluster
using ugi.doAs(..) right?
     You have done..
         1) kinit for the *hadoop *user.
         2) Trying to create the UGI on client side by calling
UserGroupInformation#createRemoteUser()

   First of all, If you want to access data as same user as kinit, then you
dont need any UGI.
   Filesystem itself will be internally created with the current user's ugi
authenticated using the kerberos ticket cache.

   Still, If you want to get the UGI object on your own,
     then you can get by calling *UserGroupInformation#getCurrentUser() *at
the beginning of your program.
     this will get authenticated if you have the kinit done for the same
user using kerberos ticket cache for the user.

     Remember to call setConfiguration on *UserGroupInformation, *if you
have any custom configurations set programmatically, before
*getCurrentUser().*

   Hi YeQi,
      As per david's update second snippet of the code is working
fine. So  *hadoop.security.authentication
*must be configured in core-site.xml. Then there will not be any difference.

Regards,
Vinay


On Thu, Jun 26, 2014 at 12:54 PM, Yeqi <[email protected]> wrote:

> Hi Vinay/Chris/David
>
> I think this exception comes from client side didn't set "
> hadoop.security.authentication " to "kerberos" of the first snippet.
>
> In the second snippet, since the client user share the same OS user and
> env of the Hadoop superuser, so the access is passed.
>
> My suggestion is to use second snippet even in the different OS user, and
> add below line after conf obj is created.
> conf.set(" hadoop.security.authentication ", "kerberos")
>
> pls correct me if any idea.
> Thanks
> Ye Qi
>
>
> -----邮件原件-----
> 发件人: Chris Nauroth [mailto:[email protected]]
> 发送时间: 2014年6月26日 2:38
> 收件人: [email protected]
> 主题: Re: "SIMPLE authentication is not enabled" error for secured hdfs read
>
> Just to clarify, the new method added in HADOOP-10683 helped fix erroneous
> logs that showed SIMPLE for the authentication method instead of KERBEROS.
>  Switching to that version of the method still won't automatically attach
> credentials to the ugi, so I expect you'll still get an authentication
> failure.
>
> Chris Nauroth
> Hortonworks
> http://hortonworks.com/
>
>
>
> On Wed, Jun 25, 2014 at 3:14 AM, Vinayakumar B <[email protected]>
> wrote:
>
> > Hi,
> >
> > In first snippet of the code following method can be used by providing
> > the corresponding AuthMethod.
> >
> > /**
> >    * Create a user from a login name. It is intended to be used for
> remote
> >    * users in RPC, since it won't have any credentials.
> >    * @param user the full user principal name, must not be empty or null
> >    * @return the UserGroupInformation for the remote user.
> >    */
> >   @InterfaceAudience.Public
> >   @InterfaceStability.Evolving
> >   public static UserGroupInformation createRemoteUser(String user,
> > AuthMethod authMethod) {
> >
> >
> > This has been added recently to trunk and branch-2. Its not yet
> > available in any release.
> >
> > This has been added as fix for HADOOP-10683
> >
> > Regards,
> > Vinay
> >
> >
> > On Wed, Jun 25, 2014 at 3:50 AM, Liu, David <[email protected]>
> wrote:
> >
> > > Hi Nauroth,
> > >
> > > In this case, do you have any example on how to use java api to read
> > > data from secured hdfs?
> > >
> > > Thanks
> > >
> > >
> > >
> > >
> > > On Jun 25, 2014, at 2:24 AM, Chris Nauroth
> > > <[email protected]>
> > > wrote:
> > >
> > > > Hi David,
> > > >
> > > > UserGroupInformation.createRemoteUser does not attach credentials
> > > > to
> > the
> > > > returned ugi.  I expect the server side is rejecting the
> > > > connection due
> > > to
> > > > lack of credentials.  This is actually by design.  The
> > > > UserGroupInformation.createRemoteUser method is primarily intended
> > > > for
> > > use
> > > > on the server side when it wants to run a piece of its code while
> > > > impersonating the client.
> > > >
> > > > I'd say that your second code sample is the correct one.  After
> > > > running kinit to get credentials, you can just run your code.  I
> > > > expect
> > Kerberos
> > > > authentication to work without taking any special measures to call
> > > > UserGroupInformation directly from your code.
> > > >
> > > > Hope this helps.
> > > >
> > > > Chris Nauroth
> > > > Hortonworks
> > > > http://hortonworks.com/
> > > >
> > > >
> > > >
> > > > On Tue, Jun 24, 2014 at 6:29 AM, Liu, David <[email protected]>
> > > wrote:
> > > >
> > > >> Hi experts,
> > > >>
> > > >> After kinit hadoop, When I run this java file on a secured hadoop
> > > cluster,
> > > >> I met the following error:
> > > >> 14/06/24 16:53:41 ERROR security.UserGroupInformation:
> > > >> PriviledgedActionException as:hdfs (auth:SIMPLE)
> > > >> cause:org.apache.hadoop.security.AccessControlException: Client
> > > >> cannot authenticate via:[TOKEN, KERBEROS]
> > > >> 14/06/24 16:53:41 WARN ipc.Client: Exception encountered while
> > > connecting
> > > >> to the server : org.apache.hadoop.security.AccessControlException:
> > > Client
> > > >> cannot authenticate via:[TOKEN, KERBEROS]
> > > >> 14/06/24 16:53:41 ERROR security.UserGroupInformation:
> > > >> PriviledgedActionException as:hdfs (auth:SIMPLE)
> > > cause:java.io.IOException:
> > > >> org.apache.hadoop.security.AccessControlException: Client cannot
> > > >> authenticate via:[TOKEN, KERBEROS]
> > > >> 14/06/24 16:53:41 ERROR security.UserGroupInformation:
> > > >> PriviledgedActionException as:hdfs (auth:SIMPLE)
> > > cause:java.io.IOException:
> > > >> Failed on local exception: java.io.IOException:
> > > >> org.apache.hadoop.security.AccessControlException: Client cannot
> > > >> authenticate via:[TOKEN, KERBEROS]; Host Details : local host is:
> > > >> "hdsh2-a161/10.62.66.161"; destination host is: "
> > hdsh2-a161.lss.emc.com
> > > >> ":8020;
> > > >> Exception in thread "main" java.io.IOException: Failed on local
> > > exception:
> > > >> java.io.IOException:
> > org.apache.hadoop.security.AccessControlException:
> > > >> Client cannot authenticate via:[TOKEN, KERBEROS]; Host Details :
> > > >> local
> > > host
> > > >> is: "hdsh2-a161/10.62.66.161"; destination host is: "
> > > >> hdsh2-a161.lss.emc.com":8020;
> > > >>        at
> > > org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:764)
> > > >>        at org.apache.hadoop.ipc.Client.call(Client.java:1351)
> > > >>        at org.apache.hadoop.ipc.Client.call(Client.java:1300)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngi
> > ne.java:206)
> > > >>        at com.sun.proxy.$Proxy9.getBlockLocations(Unknown Source)
> > > >>        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native
> Method)
> > > >>        at
> > > >>
> > >
> > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.j
> > ava:57)
> > > >>        at
> > > >>
> > >
> > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccess
> > orImpl.java:43)
> > > >>        at java.lang.reflect.Method.invoke(Method.java:606)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryIn
> > vocationHandler.java:186)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocati
> > onHandler.java:102)
> > > >>        at com.sun.proxy.$Proxy9.getBlockLocations(Unknown Source)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.g
> > etBlockLocations(ClientNamenodeProtocolTranslatorPB.java:191)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:
> > 1067)
> > > >>        at
> > > >>
> org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1057)
> > > >>        at
> > > >>
> org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1047)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBloc
> > kLength(DFSInputStream.java:235)
> > > >>        at
> > > >>
> > org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:202
> > )
> > > >>        at
> > > >>
> org.apache.hadoop.hdfs.DFSInputStream.<init>(DFSInputStream.java:195)
> > > >>        at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:1215)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileS
> > ystem.java:290)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileS
> > ystem.java:286)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkReso
> > lver.java:81)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSyste
> > m.java:286)
> > > >>        at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:763)
> > > >>        at Testhdfs$1.run(Testhdfs.java:43)
> > > >>        at Testhdfs$1.run(Testhdfs.java:30)
> > > >>        at java.security.AccessController.doPrivileged(Native Method)
> > > >>        at javax.security.auth.Subject.doAs(Subject.java:415)
> > > >>        at
> > > >>
> > >
> > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformat
> > ion.java:1491)
> > > >>        at Testhdfs.main(Testhdfs.java:30)
> > > >>
> > > >>
> > > >> Here is my code:
> > > >>
> > > >> UserGroupInformation ugi =
> > > UserGroupInformation.createRemoteUser("hadoop");
> > > >>                ugi.doAs(new PrivilegedExceptionAction<Void>() {
> > > >>                        public Void run() throws Exception {
> > > >>                                Configuration conf = new
> > Configuration();
> > > >>                                FileSystem fs =
> > > >> FileSystem.get(URI.create(uri), conf);
> > > >>                                FSDataInputStream in = fs.open(new
> > > >> Path(uri));
> > > >>                                IOUtils.copy(in, System.out, 4096);
> > > >>                                return null;
> > > >>                        }
> > > >>                });
> > > >>
> > > >> But when I run it without UserGroupInformation, like this on the
> > > >> same cluster with the same user, the code works fine.
> > > >> Configuration conf = new Configuration();
> > > >>                                FileSystem fs =
> > > >> FileSystem.get(URI.create(uri), conf);
> > > >>                                FSDataInputStream in = fs.open(new
> > > >> Path(uri));
> > > >>                                IOUtils.copy(in, System.out,
> > > >> 4096);
> > > >>
> > > >> Could anyone help me?
> > > >>
> > > >> Thanks
> > > >
> > > > --
> > > > CONFIDENTIALITY NOTICE
> > > > NOTICE: This message is intended for the use of the individual or
> > entity
> > > to
> > > > which it is addressed and may contain information that is
> > > > confidential, privileged and exempt from disclosure under
> > > > applicable law. If the
> > reader
> > > > of this message is not the intended recipient, you are hereby
> > > > notified
> > > that
> > > > any printing, copying, dissemination, distribution, disclosure or
> > > > forwarding of this communication is strictly prohibited. If you
> > > > have received this communication in error, please contact the
> > > > sender
> > > immediately
> > > > and delete it from your system. Thank You.
> > >
> > >
> >
> >
> > --
> > Regards,
> > Vinay
> >
>
> --
> CONFIDENTIALITY NOTICE
> NOTICE: This message is intended for the use of the individual or entity
> to which it is addressed and may contain information that is confidential,
> privileged and exempt from disclosure under applicable law. If the reader
> of this message is not the intended recipient, you are hereby notified that
> any printing, copying, dissemination, distribution, disclosure or
> forwarding of this communication is strictly prohibited. If you have
> received this communication in error, please contact the sender immediately
> and delete it from your system. Thank You.
>

Reply via email to