[
https://issues.apache.org/jira/browse/TAJO-858?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14021514#comment-14021514
]
Prafulla T commented on TAJO-858:
---------------------------------
Hi,
I am done with basic changes required to get Tajo working in kerberized hadoop,
but I would need help from senior Tajo contributors.
My patch can be viewed at following pull request.
https://github.com/apache/tajo/pull/30
My questions.
1. I have developed new api to get FileSystem object. I need to replace all
Path.getFileSystem calls with this new function. New getFileSystem implement
depends on certain paramaters set in TajoConf. How should we reliably make sure
that required parameters are set in TajoConf before .getFileSystem is called?
2. I see that tajo.storage.* class is not accessible in tajo-common. Certain
classes in tajo-common requires new API to get filesystem object. Should I move
my new class FileSystemUtil to tajo-common ?
3. Delegation token is considered as some important and confidential
information in secured hadoop. We would need some way to securely exchange it
in Tajo cluster. I think, This can be done later as separate issue. ( See
http://hortonworks.com/blog/the-role-of-delegation-tokens-in-apache-hadoop-security/
)
Right now, simple query works with kerberized hadoop, but longer query hits
following exception when we try to create staging directory.
java.io.IOException: org.apache.tajo.conf.TajoConfException: Hadoop DFS
delegationToken is null, It should have been set in TajoMaster
at
org.apache.tajo.storage.FileSystemUtil.getFileSystem(FileSystemUtil.java:188)
at
org.apache.tajo.storage.FileSystemUtil.getFileSystem(FileSystemUtil.java:144)
at
org.apache.tajo.master.querymaster.QueryMasterTask.initStagingDir(QueryMasterTask.java:373)
at
org.apache.tajo.master.querymaster.QueryMasterTask.init(QueryMasterTask.java:160)
at
org.apache.tajo.master.querymaster.QueryMaster$QueryStartEventHandler.handle(QueryMaster.java:378)
at
org.apache.tajo.master.querymaster.QueryMaster$QueryStartEventHandler.handle(QueryMaster.java:371)
at
org.apache.tajo.master.TajoAsyncDispatcher.dispatch(TajoAsyncDispatcher.java:137)
at
org.apache.tajo.master.TajoAsyncDispatcher$1.run(TajoAsyncDispatcher.java:79)
at java.lang.Thread.run(Thread.java:724)
Caused by: org.apache.tajo.conf.TajoConfException: Hadoop DFS delegationToken
is null, It should have been set in TajoMaster
at
org.apache.tajo.storage.FileSystemUtil.getDFSUsingDelegationToken(FileSystemUtil.java:56)
at
org.apache.tajo.storage.FileSystemUtil.getFileSystem(FileSystemUtil.java:185)
... 8 more
This happens because required parameter is not set in TajoConf. (Problem (1)
above).
> Support for hadoop kerberos authentication in Tajo
> --------------------------------------------------
>
> Key: TAJO-858
> URL: https://issues.apache.org/jira/browse/TAJO-858
> Project: Tajo
> Issue Type: Bug
> Affects Versions: 0.8.0, 0.9.0
> Reporter: Min Zhou
> Assignee: Prafulla T
> Attachments: TAJO-858.patch
>
>
> The hadoop cluster is configured to use kerberos as authentication mechanism.
> The exception is list below, seems when opening a hdfs file, tajo can't read
> the security related config items from core-site.xml. It still used SIMPLE
> authentication.
> {noformat}
> 2014-05-29 01:00:40,269 WARN security.UserGroupInformation
> (UserGroupInformation.java:doAs(1551)) - PriviledgedActionException as:mzhou
> (auth:SIMPLE) cause:org.apache.hadoop.security.AccessControlException: Client
> cannot authenticate via:[TOKEN, KERBEROS]
> 2014-05-29 01:00:40,270 WARNdomain ipc.Client (Client.java:run(669)) -
> Exception encountered while connecting to the server :
> org.apache.hadoop.security.AccessControlException: Client canhostnot
> authenticate via:[TOKEN, KERBEROS]
> 2014-05-29 01:00:40,270 WARN security.UserGroupInformation
> (UserGroupInformation.java:doAs(1551)) - PriviledgedActionException as:mzhou
> (auth:SIMPLE) cause:java.io.IOException:
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate
> via:[TOKEN, KERBEROS]
> 2014-05-29 01:00:40,278 ERROR worker.Task (Task.java:run(393)) -
> 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:
> "host4.grid.domain.com/172.20.1.34"; destination host is:
> "host1.grid.domain.com":9000;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:764)
> at org.apache.hadoop.ipc.Client.call(Client.java:1410)
> at org.apache.hadoop.ipc.Client.call(Client.java:1359)
> at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
> at com.sun.proxy.$Proxy9.getBlockLocations(Unknown Source)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:606)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:186)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> at com.sun.proxy.$Proxy9.getBlockLocations(Unknown Source)
> at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(ClientNamenodeProtocolTranslatorPB.java:206)
> at
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1131)
> at
> org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1121)
> at
> org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1111)
> at
> org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength(DFSInputStream.java:272)
> at
> org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:239)
> at org.apache.hadoop.hdfs.DFSInputStream.<init>(DFSInputStream.java:232)
> at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:1279)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:296)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:292)
> at
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:292)
> at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:765)
> at org.apache.tajo.storage.CSVFile$CSVScanner.init(CSVFile.java:303)
> at
> org.apache.tajo.engine.planner.physical.SeqScanExec.initScanner(SeqScanExec.java:197)
> at
> org.apache.tajo.engine.planner.physical.SeqScanExec.init(SeqScanExec.java:179)
> at
> org.apache.tajo.engine.planner.physical.UnaryPhysicalExec.init(UnaryPhysicalExec.java:52)
> at
> org.apache.tajo.engine.planner.physical.UnaryPhysicalExec.init(UnaryPhysicalExec.java:52)
> at
> org.apache.tajo.engine.planner.physical.HashShuffleFileWriteExec.init(HashShuffleFileWriteExec.java:81)
> at org.apache.tajo.worker.Task.run(Task.java:383)
> at org.apache.tajo.worker.TaskRunner$1.run(TaskRunner.java:391)
> at java.lang.Thread.run(Thread.java:744)
> 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:674)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1548)
> at
> org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:637)
> at
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:721)
> at org.apache.hadoop.ipc.Client$Connection.access$2700(Client.java:367)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1458)
> at org.apache.hadoop.ipc.Client.call(Client.java:1377)
> ... 32 more
> Caused by: org.apache.hadoop.security.AccessControlException: Client cannot
> authenticate via:[TOKEN, KERBEROS]
> at
> org.apache.hadoop.security.SaslRpcClient.selectSaslClient(SaslRpcClient.java:170)
> at
> org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:387)
> at
> org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:547)
> at org.apache.hadoop.ipc.Client$Connection.access$1800(Client.java:367)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:713)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:709)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1548)
> at
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:708)
> ... 35 more
> 2014-05-29 01:00:40,278 INFO worker.TaskAttemptContext
> (TaskAttemptContext.java:setState(110)) - Query status of
> ta_1401325188560_0001_000001_000000_00 is changed to TA_FAILED
> 2014-05-29 01:00:40,281 INFO worker.Task (Task.java:run(447)) - Task Counter
> - total:1, succeeded: 0, killed: 0, failed: 1
> 2014-05-29 01:00:40,282 INFO worker.TaskRunner (TaskRunner.java:run(332)) -
> Request GetTask:
> eb_1401325188560_0001_000001,container_1401325188560_0001_01_000001
> 2014-05-29 01:00:40,305 INFO worker.TaskRunner (TaskRunner.java:run(370)) -
> Accumulated Received Task: 2
> 2014-05-29 01:00:40,305 INFO worker.TaskRunner (TaskRunner.java:run(379)) -
> Initializing: ta_1401325188560_0001_000001_000000_01
> 2014-05-29 01:00:40,316 INFO worker.TaskAttemptContext
> (TaskAttemptContext.java:setState(110)) - Query status of
> ta_1401325188560_0001_000001_000000_01 is changed to TA_PENDING
> 2014-05-29 01:00:40,316 INFO worker.Task (Task.java:<init>(188)) -
> ==================================
> 2014-05-29 01:00:40,318 INFO worker.Task (Task.java:<init>(189)) - *
> Subquery ta_1401325188560_0001_000001_000000_01 is initialized
> 2014-05-29 01:00:40,318 INFO worker.Task (Task.java:<init>(190)) - *
> InterQuery: true, Use HASH_SHUFFLE shuffle
> 2014-05-29 01:00:40,318 INFO worker.Task (Task.java:<init>(193)) - *
> Fragments (num: 1)
> 2014-05-29 01:00:40,318 INFO worker.Task (Task.java:<init>(194)) - * Fetches
> (total:0) :
> 2014-05-29 01:00:40,318 INFO worker.Task (Task.java:<init>(198)) - * Local
> task dir:
> file:/grid/d/tmp/yarn/usercache/mzhou/appcache/application_1400096295333_0092/container_1400096295333_0092_01_000004/${LOCAL_DIRS}/q_1401325188560_0001/output/1/0_1
> 2014-05-29 01:00:40,318 INFO worker.Task (Task.java:<init>(203)) -
> ==================================
> 2014-05-29 01:00:40,319 INFO worker.TaskAttemptContext
> (TaskAttemptContext.java:setState(110)) - Query status of
> ta_1401325188560_0001_000001_000000_01 is changed to TA_RUNNING
> 2014-05-29 01:00:40,319 INFO planner.PhysicalPlannerImpl
> (PhysicalPlannerImpl.java:createInMemoryHashAggregation(901)) - The planner
> chooses [Hash Aggregation]
> 2014-05-29 01:00:40,325 WARN security.UserGroupInformation
> (UserGroupInformation.java:doAs(1551)) - PriviledgedActionException as:mzhou
> (auth:SIMPLE) cause:org.apache.hadoop.security.AccessControlException: Client
> cannot authenticate via:[TOKEN, KERBEROS]
> 2014-05-29 01:00:40,326 WARN ipc.Client (Client.java:run(669)) - Exception
> encountered while connecting to the server :
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate
> via:[TOKEN, KERBEROS]
> 2014-05-29 01:00:40,326 WARN security.UserGroupInformation
> (UserGroupInformation.java:doAs(1551)) - PriviledgedActionException as:mzhou
> (auth:SIMPLE) cause:java.io.IOException:
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate
> via:[TOKEN, KERBEROS]
> 2014-05-29 01:00:40,328 ERROR worker.Task (Task.java:run(393)) -
> 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:
> "host4.grid.domain.com/172.20.1.34"; destination host is:
> "host1.grid.domain.com":9000;
> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:764)
> at org.apache.hadoop.ipc.Client.call(Client.java:1410)
> at org.apache.hadoop.ipc.Client.call(Client.java:1359)
> at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
> at com.sun.proxy.$Proxy9.getBlockLocations(Unknown Source)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:606)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:186)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
> at com.sun.proxy.$Proxy9.getBlockLocations(Unknown Source)
> at
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(ClientNamenodeProtocolTranslatorPB.java:206)
> at
> org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1131)
> at
> org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1121)
> at
> org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1111)
> at
> org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength(DFSInputStream.java:272)
> at
> org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:239)
> at org.apache.hadoop.hdfs.DFSInputStream.<init>(DFSInputStream.java:232)
> at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:1279)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:296)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem$3.doCall(DistributedFileSystem.java:292)
> at
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> at
> org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:292)
> at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:765)
> at org.apache.tajo.storage.CSVFile$CSVScanner.init(CSVFile.java:303)
> at
> org.apache.tajo.engine.planner.physical.SeqScanExec.initScanner(SeqScanExec.java:197)
> at
> org.apache.tajo.engine.planner.physical.SeqScanExec.init(SeqScanExec.java:179)
> at
> org.apache.tajo.engine.planner.physical.UnaryPhysicalExec.init(UnaryPhysicalExec.java:52)
> at
> org.apache.tajo.engine.planner.physical.UnaryPhysicalExec.init(UnaryPhysicalExec.java:52)
> at
> org.apache.tajo.engine.planner.physical.HashShuffleFileWriteExec.init(HashShuffleFileWriteExec.java:81)
> at org.apache.tajo.worker.Task.run(Task.java:383)
> at org.apache.tajo.worker.TaskRunner$1.run(TaskRunner.java:391)
> at java.lang.Thread.run(Thread.java:744)
> 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:674)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1548)
> at
> org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:637)
> at
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:721)
> at org.apache.hadoop.ipc.Client$Connection.access$2700(Client.java:367)
> at org.apache.hadoop.ipc.Client.getConnection(Client.java:1458)
> at org.apache.hadoop.ipc.Client.call(Client.java:1377)
> ... 32 more
> Caused by: org.apache.hadoop.security.AccessControlException: Client cannot
> authenticate via:[TOKEN, KERBEROS]
> at
> org.apache.hadoop.security.SaslRpcClient.selectSaslClient(SaslRpcClient.java:170)
> at
> org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:387)
> at
> org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:547)
> at org.apache.hadoop.ipc.Client$Connection.access$1800(Client.java:367)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:713)
> at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:709)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:415)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1548)
> at
> org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:708)
> ... 35 more
> {noformat}
--
This message was sent by Atlassian JIRA
(v6.2#6252)