hadoop-yarn-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Varun Saxena (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (YARN-3779) Aggregated Logs Deletion doesnt work after refreshing Log Retention Settings in secure cluster
Date Tue, 09 Jun 2015 16:44:01 GMT

    [ https://issues.apache.org/jira/browse/YARN-3779?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14579194#comment-14579194
] 

Varun Saxena commented on YARN-3779:
------------------------------------

Sorry the correct sequence of error logs is as under. After first GSSException, client i.e.
historyserver keeps on retrying before giving up.

{noformat}
2015-06-05 22:49:24,541 INFO Timer-3  org.apache.hadoop.yarn.logaggregation.AggregatedLogDeletionService:
aggregated log deletion started.
2015-06-05 22:49:24,541 INFO IPC Server handler 0 on 10033  org.apache.hadoop.mapreduce.v2.hs.HSAuditLogger:
USER=hdfs	IP=10.19.92.82	OPERATION=refreshLogRetentionSettings	TARGET=HSAdminServer	RESULT=SUCCESS
2015-06-05 22:49:24,550 DEBUG Timer-3  org.apache.hadoop.hdfs.client.impl.DfsClientConf$ShortCircuitConf:
dfs.client.use.legacy.blockreader.local = false
2015-06-05 22:49:24,550 DEBUG Timer-3  org.apache.hadoop.hdfs.client.impl.DfsClientConf$ShortCircuitConf:
dfs.client.read.shortcircuit = false
2015-06-05 22:49:24,550 DEBUG Timer-3  org.apache.hadoop.hdfs.client.impl.DfsClientConf$ShortCircuitConf:
dfs.client.domain.socket.data.traffic = false
2015-06-05 22:49:24,550 DEBUG Timer-3  org.apache.hadoop.hdfs.client.impl.DfsClientConf$ShortCircuitConf:
dfs.domain.socket.path = 
2015-06-05 22:49:24,550 DEBUG Timer-3  org.apache.hadoop.hdfs.DFSClient: Sets dfs.client.block.write.replace-datanode-on-failure.replication
to 0
2015-06-05 22:49:24,552 DEBUG Timer-3  org.apache.hadoop.hdfs.HAUtil: No HA service delegation
token found for logical URI hdfs://hacluster
2015-06-05 22:49:24,552 DEBUG Timer-3  org.apache.hadoop.hdfs.client.impl.DfsClientConf$ShortCircuitConf:
dfs.client.use.legacy.blockreader.local = false
2015-06-05 22:49:24,552 DEBUG Timer-3  org.apache.hadoop.hdfs.client.impl.DfsClientConf$ShortCircuitConf:
dfs.client.read.shortcircuit = false
2015-06-05 22:49:24,552 DEBUG Timer-3  org.apache.hadoop.hdfs.client.impl.DfsClientConf$ShortCircuitConf:
dfs.client.domain.socket.data.traffic = false
2015-06-05 22:49:24,552 DEBUG Timer-3  org.apache.hadoop.hdfs.client.impl.DfsClientConf$ShortCircuitConf:
dfs.domain.socket.path = 
2015-06-05 22:49:24,552 DEBUG Timer-3  org.apache.hadoop.io.retry.RetryUtils: multipleLinearRandomRetry
= null
2015-06-05 22:49:24,553 DEBUG Timer-3  org.apache.hadoop.ipc.Client: getting client out of
cache: org.apache.hadoop.ipc.Client@28194a50
2015-06-05 22:49:24,554 DEBUG Timer-3  org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil:
DataTransferProtocol using SaslPropertiesResolver, configured QOP dfs.data.transfer.protection
= authentication, configured class dfs.data.transfer.saslproperties.resolver.class = class
org.apache.hadoop.security.SaslPropertiesResolver
2015-06-05 22:49:24,554 DEBUG Timer-3  org.apache.hadoop.ipc.Client: The ping interval is
60000 ms.
2015-06-05 22:49:24,554 DEBUG Timer-3  org.apache.hadoop.ipc.Client: Connecting to /10.19.92.88:65110
2015-06-05 22:49:24,555 DEBUG Timer-3  org.apache.hadoop.security.UserGroupInformation: PrivilegedAction
as:hdfs/huawei@HADOOP.COM (auth:KERBEROS) from:org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:749)
2015-06-05 22:49:24,557 DEBUG Timer-3  org.apache.hadoop.security.SaslRpcClient: Get kerberos
info proto:interface org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB info:@org.apache.hadoop.security.KerberosInfo(clientPrincipal=,
serverPrincipal=dfs.namenode.kerberos.principal)
2015-06-05 22:49:24,557 DEBUG Timer-3  org.apache.hadoop.security.SaslRpcClient: getting serverKey:
dfs.namenode.kerberos.principal conf value: hdfs/huawei@HADOOP.COM principal: hdfs/huawei@HADOOP.COM
2015-06-05 22:49:24,557 DEBUG Timer-3  org.apache.hadoop.security.SaslRpcClient: RPC Server's
Kerberos principal name for protocol=org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB
is hdfs/huawei@HADOOP.COM
2015-06-05 22:49:24,557 DEBUG Timer-3  org.apache.hadoop.security.SaslRpcClient: Creating
SASL GSSAPI(KERBEROS)  client to authenticate to service at huawei
2015-06-05 22:49:24,558 DEBUG Timer-3  org.apache.hadoop.security.SaslRpcClient: Use KERBEROS
authentication for protocol ClientNamenodeProtocolPB
2015-06-05 22:49:24,559 DEBUG Timer-3  org.apache.hadoop.security.UserGroupInformation: PrivilegedActionException
as:hdfs/huawei@HADOOP.COM (auth:KERBEROS) cause:javax.security.sasl.SaslException: GSS initiate
failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to
find any Kerberos tgt)]
2015-06-05 22:49:24,560 DEBUG Timer-3  org.apache.hadoop.security.UserGroupInformation: PrivilegedAction
as:hdfs/huawei@HADOOP.COM (auth:KERBEROS) from:org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:668)
2015-06-05 22:49:24,561 WARN Timer-3  org.apache.hadoop.ipc.Client: Exception encountered
while connecting to the server : javax.security.sasl.SaslException: GSS initiate failed [Caused
by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos
tgt)]
2015-06-05 22:49:24,562 DEBUG Timer-3  org.apache.hadoop.ipc.Client: closing ipc connection
to /10.19.92.88:65110: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException:
No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException:
No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
	at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:709)
	at java.security.AccessController.doPrivileged(Native Method)
.......

2015-06-05 22:49:24,562 DEBUG Timer-3  org.apache.hadoop.ipc.Client: IPC Client (1125964210)
connection to /10.19.92.88:65110 from hdfs/huawei@HADOOP.COM: closed
2015-06-05 22:49:24,567 INFO Timer-3  org.apache.hadoop.io.retry.RetryInvocationHandler: Exception
while invoking getListing of class ClientNamenodeProtocolTranslatorPB over /10.19.92.88:65110.
Trying to fail over immediately.
java.io.IOException: Failed on local exception: java.io.IOException: javax.security.sasl.SaslException:
GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level:
Failed to find any Kerberos tgt)]; Host Details : local host is: "HOST-10-19-92-82/10.19.92.82";
destination host is: "host-10-19-92-88":65110; 
	at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:776)
	at org.apache.hadoop.ipc.Client.call(Client.java:1516)
	at org.apache.hadoop.ipc.Client.call(Client.java:1443)
...........
2015-06-05 22:49:24,568 DEBUG Timer-3  org.apache.hadoop.io.retry.RetryUtils: multipleLinearRandomRetry
= null
2015-06-05 22:49:24,569 DEBUG Timer-3  org.apache.hadoop.ipc.Client: getting client out of
cache: org.apache.hadoop.ipc.Client@28194a50
2015-06-05 22:49:24,569 DEBUG Timer-3  org.apache.hadoop.ipc.Client: The ping interval is
60000 ms.
2015-06-05 22:49:24,569 DEBUG Timer-3  org.apache.hadoop.ipc.Client: Connecting to /10.19.92.95:65110
2015-06-05 22:49:24,574 DEBUG Timer-3  org.apache.hadoop.security.UserGroupInformation: PrivilegedAction
as:hdfs/huawei@HADOOP.COM (auth:KERBEROS) from:org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:749)
2015-06-05 22:49:24,577 DEBUG Timer-3  org.apache.hadoop.security.SaslRpcClient: Get kerberos
info proto:interface org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB info:@org.apache.hadoop.security.KerberosInfo(clientPrincipal=,
serverPrincipal=dfs.namenode.kerberos.principal)
2015-06-05 22:49:24,577 DEBUG Timer-3  org.apache.hadoop.security.SaslRpcClient: getting serverKey:
dfs.namenode.kerberos.principal conf value: hdfs/huawei@HADOOP.COM principal: hdfs/huawei@HADOOP.COM
2015-06-05 22:49:24,578 DEBUG Timer-3  org.apache.hadoop.security.SaslRpcClient: RPC Server's
Kerberos principal name for protocol=org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolPB
is hdfs/huawei@HADOOP.COM
2015-06-05 22:49:24,578 DEBUG Timer-3  org.apache.hadoop.security.SaslRpcClient: Creating
SASL GSSAPI(KERBEROS)  client to authenticate to service at huawei
2015-06-05 22:49:24,579 DEBUG Timer-3  org.apache.hadoop.security.SaslRpcClient: Use KERBEROS
authentication for protocol ClientNamenodeProtocolPB
2015-06-05 22:49:24,580 DEBUG Timer-3  org.apache.hadoop.security.UserGroupInformation: PrivilegedActionException
as:hdfs/huawei@HADOOP.COM (auth:KERBEROS) cause:javax.security.sasl.SaslException: GSS initiate
failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to
find any Kerberos tgt)]
2015-06-05 22:49:24,585 DEBUG Timer-3  org.apache.hadoop.security.UserGroupInformation: PrivilegedAction
as:hdfs/huawei@HADOOP.COM (auth:KERBEROS) from:org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:668)
2015-06-05 22:49:24,585 WARN Timer-3  org.apache.hadoop.ipc.Client: Exception encountered
while connecting to the server : javax.security.sasl.SaslException: GSS initiate failed [Caused
by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos
tgt)]
2015-06-05 22:49:24,585 DEBUG Timer-3  org.apache.hadoop.security.UserGroupInformation: PrivilegedActionException
as:hdfs/huawei@HADOOP.COM (auth:KERBEROS) cause:java.io.IOException: javax.security.sasl.SaslException:
GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level:
Failed to find any Kerberos tgt)]
.....
....(several similar logs)

2015-06-05 22:49:24,699 ERROR Timer-3  org.apache.hadoop.yarn.logaggregation.AggregatedLogDeletionService:
Error reading root log dir this deletion attempt is being aborted
java.io.IOException: Failed on local exception: java.io.IOException: javax.security.sasl.SaslException:
GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level:
Failed to find any Kerberos tgt)]; Host Details : local host is: "HOST-10-19-92-82/10.19.92.82";
destination host is: "host-10-19-92-95":65110; 
	at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:776)
	at org.apache.hadoop.ipc.Client.call(Client.java:1516)
	at org.apache.hadoop.ipc.Client.call(Client.java:1443)
	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
....

2015-06-05 22:49:24,699 INFO Timer-3  org.apache.hadoop.yarn.logaggregation.AggregatedLogDeletionService:
aggregated log deletion finished.
{noformat}

> Aggregated Logs Deletion doesnt work after refreshing Log Retention Settings in secure
cluster
> ----------------------------------------------------------------------------------------------
>
>                 Key: YARN-3779
>                 URL: https://issues.apache.org/jira/browse/YARN-3779
>             Project: Hadoop YARN
>          Issue Type: Bug
>    Affects Versions: 2.7.0
>         Environment: mrV2, secure mode
>            Reporter: Zhang Wei
>            Assignee: Varun Saxena
>            Priority: Critical
>         Attachments: YARN-3779.01.patch, YARN-3779.02.patch
>
>
> {{GSSException}} is thrown everytime log aggregation deletion is attempted after executing
bin/mapred hsadmin -refreshLogRetentionSettings in a secure cluster.
> The problem can be reproduced by following steps:
> 1. startup historyserver in secure cluster.
> 2. Log deletion happens as per expectation. 
> 3. execute {{mapred hsadmin -refreshLogRetentionSettings}} command to refresh the configuration
value.
> 4. All the subsequent attempts of log deletion fail with {{GSSException}}
> Following exception can be found in historyserver's log if log deletion is enabled. 
> {noformat}
> 2015-06-04 14:14:40,070 | ERROR | Timer-3 | Error reading root log dir this deletion
attempt is being aborted | AggregatedLogDeletionService.java:127
> java.io.IOException: Failed on local exception: java.io.IOException: javax.security.sasl.SaslException:
GSS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level:
Failed to find any Kerberos tgt)]; Host Details : local host is: "vm-31/9.91.12.31"; destination
host is: "vm-33":25000; 
>         at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:764)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1414)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1363)
>         at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
>         at com.sun.proxy.$Proxy9.getListing(Unknown Source)
>         at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getListing(ClientNamenodeProtocolTranslatorPB.java:519)
>         at sun.reflect.GeneratedMethodAccessor16.invoke(Unknown Source)
>         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:187)
>         at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
>         at com.sun.proxy.$Proxy10.getListing(Unknown Source)
>         at org.apache.hadoop.hdfs.DFSClient.listPaths(DFSClient.java:1767)
>         at org.apache.hadoop.hdfs.DFSClient.listPaths(DFSClient.java:1750)
>         at org.apache.hadoop.hdfs.DistributedFileSystem.listStatusInternal(DistributedFileSystem.java:691)
>         at org.apache.hadoop.hdfs.DistributedFileSystem.access$600(DistributedFileSystem.java:102)
>         at org.apache.hadoop.hdfs.DistributedFileSystem$15.doCall(DistributedFileSystem.java:753)
>         at org.apache.hadoop.hdfs.DistributedFileSystem$15.doCall(DistributedFileSystem.java:749)
>         at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>         at org.apache.hadoop.hdfs.DistributedFileSystem.listStatus(DistributedFileSystem.java:749)
>         at org.apache.hadoop.yarn.logaggregation.AggregatedLogDeletionService$LogDeletionTask.run(AggregatedLogDeletionService.java:68)
>         at java.util.TimerThread.mainLoop(Timer.java:555)
>         at java.util.TimerThread.run(Timer.java:505)
> Caused by: java.io.IOException: javax.security.sasl.SaslException: GSS initiate failed
[Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any
Kerberos tgt)]
>         at org.apache.hadoop.ipc.Client$Connection$1.run(Client.java:677)
>         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:1641)
>         at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:640)
>         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:724)
>         at org.apache.hadoop.ipc.Client$Connection.access$2800(Client.java:367)
>         at org.apache.hadoop.ipc.Client.getConnection(Client.java:1462)
>         at org.apache.hadoop.ipc.Client.call(Client.java:1381)
>         ... 21 more
> Caused by: javax.security.sasl.SaslException: GSS initiate failed [Caused by GSSException:
No valid credentials provided (Mechanism level: Failed to find any Kerberos tgt)]
>         at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:212)
>         at org.apache.hadoop.security.SaslRpcClient.saslConnect(SaslRpcClient.java:411)
>         at org.apache.hadoop.ipc.Client$Connection.setupSaslConnection(Client.java:550)
>         at org.apache.hadoop.ipc.Client$Connection.access$1800(Client.java:367)
>         at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:716)
>         at org.apache.hadoop.ipc.Client$Connection$2.run(Client.java:712)
>         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:1641)
>         at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:711)
>         ... 24 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:121)
>         at sun.security.jgss.krb5.Krb5MechFactory.getMechanismContext(Krb5MechFactory.java:187)
>         at sun.security.jgss.GSSManagerImpl.getMechanismContext(GSSManagerImpl.java:223)
>         at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:212)
>         at sun.security.jgss.GSSContextImpl.initSecContext(GSSContextImpl.java:179)
>         at com.sun.security.sasl.gsskerb.GssKrb5Client.evaluateChallenge(GssKrb5Client.java:193)
>         ... 33 more
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message