[
https://issues.apache.org/jira/browse/TEZ-4032?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16779907#comment-16779907
]
Jonathan Eagles commented on TEZ-4032:
--------------------------------------
[~zhangbutao], change looks mostly good to me. A couple of things I want to
check with you. 1) Is Scope.Vertex needed on new configuration items. This
exposes this information to the VertexImpl running inside the AM. Should this
be Scope.Client or Scope.AM. Don't have enough background know what code this
config is intended for. 2) What email address should I attribute to this
contribution. I can see a chinamobile email address associated with your github
account. Is there a better email address?. 3) the public redundant modifier is
better left for now since the whole file needs to be changed. It is better to
do that all under a new jira. 4) the white space changes in tez-api could be
fixed if a new version of a patch is needed (ported from hadoop). Otherwise,
those can be cleaned as a pre-checkin step. 5) I filed TEZ-4049 to address the
findbugs issues already present in tez code.
Thanks again and I will be watching for your reply.
> TEZ will throw "Client cannot authenticate via:[TOKEN, KERBEROS]" when used
> with HDFS federation(non viewfs, only hdfs schema used).
> --------------------------------------------------------------------------------------------------------------------------------------
>
> Key: TEZ-4032
> URL: https://issues.apache.org/jira/browse/TEZ-4032
> Project: Apache Tez
> Issue Type: Bug
> Affects Versions: 0.9.1
> Reporter: zhangbutao
> Assignee: zhangbutao
> Priority: Major
> Attachments: TEZ-4032.001.patch, TEZ-4032.002.patch,
> TEZ-4032.003.patch
>
> Time Spent: 20m
> Remaining Estimate: 0h
>
> I execute hive tez job in HDFS federation and kerberos. The hadoop cluster
> has multiple namespace (hdfs://ns1,hdfs://ns2,hdfs://ns3 ...)and we don't
> use viewfs schema. Hive tez job will throw error as follows when the table
> is created in hdfs://ns2 (default configuration fs.defaluFS=hdfs://ns1):
> {code:java}
> 2019-01-21 15:43:46,507 [WARN] [TezChild] |ipc.Client|: Exception encountered
> while connecting to the server :
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate
> via:[TOKEN, KERBEROS]
> 2019-01-21 15:43:46,507 [INFO] [TezChild] |retry.RetryInvocationHandler|:
> java.io.IOException: DestHost:destPort docker5.cmss.com:8020 ,
> LocalHost:localPort docker1.cmss.com/10.254.10.116:0. Failed on local
> exception: java.io.IOException:
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate
> via:[TOKEN, KERBEROS], while invoking
> ClientNamenodeProtocolTranslatorPB.getFileInfo over
> docker5.cmss.com/10.254.2.106:8020 after 14 failover attempts. Trying to
> failover after sleeping for 10827ms.
> 2019-01-21 15:43:57,338 [WARN] [TezChild] |ipc.Client|: Exception encountered
> while connecting to the server :
> org.apache.hadoop.security.AccessControlException: Client cannot authenticate
> via:[TOKEN, KERBEROS]
> 2019-01-21 15:43:57,363 [ERROR] [TezChild] |tez.MapRecordSource|:
> org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while
> processing writable (null)
> at
> org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:568)
> at
> org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.processRow(MapRecordSource.java:92)
> at
> org.apache.hadoop.hive.ql.exec.tez.MapRecordSource.pushRecord(MapRecordSource.java:76)
> at
> org.apache.hadoop.hive.ql.exec.tez.MapRecordProcessor.run(MapRecordProcessor.java:419)
> at
> org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:267)
> at
> org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:250)
> at
> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
> at
> org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
> at
> org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
> 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:1682)
> at
> org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
> at
> org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
> at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
> at
> com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:108)
> at
> com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:41)
> at
> com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:77)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.hadoop.hive.ql.metadata.HiveException:
> org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException:
> DestHost:destPort docker4.cmss.com:8020 , LocalHost:localPort
> docker1.cmss.com/10.254.10.116:0. Failed on local exception:
> java.io.IOException: org.apache.hadoop.security.AccessControlException:
> Client cannot authenticate via:[TOKEN, KERBEROS]
> at
> org.apache.hadoop.hive.ql.exec.FileSinkOperator.createBucketFiles(FileSinkOperator.java:742)
> at
> org.apache.hadoop.hive.ql.exec.FileSinkOperator.process(FileSinkOperator.java:897)
> at
> org.apache.hadoop.hive.ql.exec.Operator.baseForward(Operator.java:995)
> at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:941)
> at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:928)
> at
> org.apache.hadoop.hive.ql.exec.SelectOperator.process(SelectOperator.java:95)
> at
> org.apache.hadoop.hive.ql.exec.Operator.baseForward(Operator.java:995)
> at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:941)
> at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:928)
> at
> org.apache.hadoop.hive.ql.exec.UDTFOperator.forwardUDTFOutput(UDTFOperator.java:133)
> at
> org.apache.hadoop.hive.ql.udf.generic.UDTFCollector.collect(UDTFCollector.java:45)
> at
> org.apache.hadoop.hive.ql.udf.generic.GenericUDTF.forward(GenericUDTF.java:110)
> at
> org.apache.hadoop.hive.ql.udf.generic.GenericUDTFInline.process(GenericUDTFInline.java:64)
> at
> org.apache.hadoop.hive.ql.exec.UDTFOperator.process(UDTFOperator.java:116)
> at
> org.apache.hadoop.hive.ql.exec.Operator.baseForward(Operator.java:995)
> at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:941)
> at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:928)
> at
> org.apache.hadoop.hive.ql.exec.SelectOperator.process(SelectOperator.java:95)
> at
> org.apache.hadoop.hive.ql.exec.Operator.baseForward(Operator.java:995)
> at org.apache.hadoop.hive.ql.exec.Operator.forward(Operator.java:941)
> at
> org.apache.hadoop.hive.ql.exec.TableScanOperator.process(TableScanOperator.java:125)
> at
> org.apache.hadoop.hive.ql.exec.MapOperator$MapOpCtx.forward(MapOperator.java:153)
> at
> org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:555)
> ... 20 more
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)