You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Zheng Hu (JIRA)" <ji...@apache.org> on 2017/07/26 03:23:00 UTC

[jira] [Created] (HBASE-18452) VerifyReplication by Snapshot should cache HDFS token before submit job for kerberos env.

Zheng Hu created HBASE-18452:
--------------------------------

             Summary: VerifyReplication by Snapshot should cache HDFS token before submit job for kerberos env. 
                 Key: HBASE-18452
                 URL: https://issues.apache.org/jira/browse/HBASE-18452
             Project: HBase
          Issue Type: Bug
            Reporter: Zheng Hu
            Assignee: Zheng Hu


I've  ported HBASE-16466 to our internal hbase branch,  and tested the feature under our kerberos cluster.   

The problem we encountered is: 
{code}
17/07/25 21:21:23 INFO mapreduce.Job: Task Id : attempt_1500987232138_0004_m_000003_2, Status : FAILED
Error: 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: "hadoop-yarn-host"; destination host is: "hadoop-namenode-host":15200; 
	at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:775)
	at org.apache.hadoop.ipc.Client.call(Client.java:1481)
	at org.apache.hadoop.ipc.Client.call(Client.java:1408)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
	at com.sun.proxy.$Proxy13.getFileInfo(Unknown Source)
	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:807)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:483)
	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
	at com.sun.proxy.$Proxy14.getFileInfo(Unknown Source)
	at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:2029)
	at org.apache.hadoop.hdfs.DistributedFileSystem$20.doCall(DistributedFileSystem.java:1195)
	at org.apache.hadoop.hdfs.DistributedFileSystem$20.doCall(DistributedFileSystem.java:1191)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1207)
	at org.apache.hadoop.hbase.regionserver.HRegionFileSystem.checkRegionInfoOnFilesystem(HRegionFileSystem.java:778)
	at org.apache.hadoop.hbase.regionserver.HRegion.initializeRegionInternals(HRegion.java:769)
	at org.apache.hadoop.hbase.regionserver.HRegion.initialize(HRegion.java:748)
	at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:5188)
	at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:5153)
	at org.apache.hadoop.hbase.regionserver.HRegion.openHRegion(HRegion.java:5125)
	at org.apache.hadoop.hbase.client.ClientSideRegionScanner.<init>(ClientSideRegionScanner.java:60)
	at org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormatImpl$RecordReader.initialize(TableSnapshotInputFormatImpl.java:191)
	at org.apache.hadoop.hbase.mapreduce.TableSnapshotInputFormat$TableSnapshotRegionRecordReader.initialize(TableSnapshotInputFormat.java:148)
	at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.initialize(MapTask.java:552)
	at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:790)
	at org.apache.hadoop.mapred.MapTask.run(MapTask.java:342)
	at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:163)
	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:1885)
	at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:158)
Caused by: java.io.IOException: org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
	at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:688)
	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:1885)
	at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:651)
	at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:738)
	at org.apache.hadoop.ipc.Client$Connection.access$2900(Client.java:370)
	at org.apache.hadoop.ipc.Client.getConnection(Client.java:1530)
	at org.apache.hadoop.ipc.Client.call(Client.java:1447)
	... 33 more
Caused by: org.apache.hadoop.security.AccessControlException: Client cannot authenticate via:[TOKEN, KERBEROS]
	at org.apache.hadoop.security.SaslRpcClient.selectSaslClient(SaslRpcClient.java:172)
	at org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:396)
	at org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:555)
	at org.apache.hadoop.ipc.Client$Connection.access$1900(Client.java:370)
	at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:730)
	at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:726)
	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:1885)
	at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:726)
	... 36 more
{code}

After check the patch,  Seems like we did not obtain the namenode token before submit the job.  The feature works fine in our kerberos cluster after applied following patch: 

{code}
      TableMapReduceUtil.initTableSnapshotMapperJob(sourceSnapshotName, scan, Verifier.class, null,
        null, job, true, snapshotTempPath);

      // Acquire the delegation Tokens
+      TokenCache.obtainTokensForNamenodes(job.getCredentials(),
+        new Path[] { new Path(sourceSnapshotTmpDir), new Path(peerSnapshotTmpDir) }, conf);
+
    } 
{code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)