Details
-
Sub-task
-
Status: Closed
-
Major
-
Resolution: Fixed
-
None
-
None
-
Reviewed
Description
When node labels are in use, and yarn.node-labels.fs-store.root-dir is set to a hdfs:// path, and the cluster is using kerberos, the RM fails to start while trying to unmarshal the label store. The following error/stack trace is observed:
2014-10-31 11:55:53,807 INFO service.AbstractService (AbstractService.java:noteFailure(272)) - Service o rg.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager failed in state INITED; cause: java.io.IOExcepti on: Failed on local exception: java.io.IOException: javax.security.sasl.SaslException: GSS initiate faile d [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to find any Kerberos tg t)]; Host Details : local host is: "host.running.rm/10.0.0.34"; destination hos t is: "host.running.nn":8020; java.io.IOException: Failed on local exception: java.io.IOException: javax.security.sasl.SaslException: G SS initiate failed [Caused by GSSException: No valid credentials provided (Mechanism level: Failed to fin d any Kerberos tgt)]; Host Details : local host is: "host.running.rm/10.0.0.34" ; destination host is: "host.running.nn":8020; at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:764) at org.apache.hadoop.ipc.Client.call(Client.java:1472) at org.apache.hadoop.ipc.Client.call(Client.java:1399) at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232) at com.sun.proxy.$Proxy14.mkdirs(Unknown Source) at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.mkdirs(ClientNamenodeProt ocolTranslatorPB.java:539) at sun.reflect.GeneratedMethodAccessor7.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.$Proxy15.mkdirs(Unknown Source) at org.apache.hadoop.hdfs.DFSClient.primitiveMkdir(DFSClient.java:2731) at org.apache.hadoop.hdfs.DFSClient.mkdirs(DFSClient.java:2702) at org.apache.hadoop.hdfs.DistributedFileSystem$17.doCall(DistributedFileSystem.java:870) at org.apache.hadoop.hdfs.DistributedFileSystem$17.doCall(DistributedFileSystem.java:866) at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) at org.apache.hadoop.hdfs.DistributedFileSystem.mkdirsInternal(DistributedFileSystem.java:866) at org.apache.hadoop.hdfs.DistributedFileSystem.mkdirs(DistributedFileSystem.java:859) at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:1817) at org.apache.hadoop.yarn.nodelabels.FileSystemNodeLabelsStore.init(FileSystemNodeLabelsStore.java:87) at org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.initNodeLabelStore(CommonNodeLabelsManager.java:206) at org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager.serviceInit(CommonNodeLabelsManager.java:199) at org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager.serviceInit(RMNodeLabelsManager.java:62) at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163) at org.apache.hadoop.service.CompositeService.serviceInit(CompositeService.java:107) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$RMActiveServices.serviceInit(ResourceManager.java:547) at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.createAndInitActiveServices(ResourceManager.java:986) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.serviceInit(ResourceManager.java:245) at org.apache.hadoop.service.AbstractService.init(AbstractService.java:163) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.main(ResourceManager.java:1216) 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:680) 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:1628) at org.apache.hadoop.ipc.Client$Connection.handleSaslConnectionFailure(Client.java:643) at org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:730) at org.apache.hadoop.ipc.Client$Connection.access$2800(Client.java:368) at org.apache.hadoop.ipc.Client.getConnection(Client.java:1521) at org.apache.hadoop.ipc.Client.call(Client.java:1438) ... 30 more
I think this is a startup ordering issue, in that the scheduler is initialized before the RM would prime the cred cache. My reasoning is based on what happens when I don't set the yarn.node-labels.fs-store.root-dir property, so no HDFS interaction happens when the scheduler initializes. Here is the relevant snippet from the log:
2014-10-31 12:04:09,739 INFO capacity.CapacityScheduler (CapacityScheduler.java:parseQueue(602)) - Initialized queu e: default: capacity=1.0, absoluteCapacity=1.0, usedResources=<memory:0, vCores:0>, usedCapacity=0.0, absoluteUsedCa pacity=0.0, numApps=0, numContainers=0 2014-10-31 12:04:09,739 INFO capacity.CapacityScheduler (CapacityScheduler.java:parseQueue(602)) - Initialized queu e: root: numChildQueue= 1, capacity=1.0, absoluteCapacity=1.0, usedResources=<memory:0, vCores:0>usedCapacity=0.0, n umApps=0, numContainers=0 2014-10-31 12:04:09,742 INFO capacity.CapacityScheduler (CapacityScheduler.java:initializeQueues(466)) - Initialize d root queue root: numChildQueue= 1, capacity=1.0, absoluteCapacity=1.0, usedResources=<memory:0, vCores:0>usedCapac ity=0.0, numApps=0, numContainers=0 2014-10-31 12:04:09,742 INFO capacity.CapacityScheduler (CapacityScheduler.java:initializeQueueMappings(435)) - Ini tialized queue mappings, override: false 2014-10-31 12:04:09,742 INFO capacity.CapacityScheduler (CapacityScheduler.java:initScheduler(304)) - Initialized C apacityScheduler with calculator=class org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator, minimumAlloca tion=<<memory:256, vCores:1>>, maximumAllocation=<<memory:2048, vCores:32>>, asynchronousScheduling=false, asyncSche duleInterval=5ms 2014-10-31 12:04:09,866 INFO security.UserGroupInformation (UserGroupInformation.java:loginUserFromKeytab(938)) - L ogin successful for user rm/host.running.rm@SLIDER1.EXAMPLE.COM using keytab file /etc/sec urity/keytabs/rm.service.keytab
You can see the scheduler initializes, and only then does the cred cache get primed. This results in a successful RM start, but of course my HDFS-backed labels are now not loaded.
I think that if the cred cached were initialized before the scheduler, this error would not happen.
Attachments
Attachments
Issue Links
- breaks
-
YARN-2805 RM2 in HA setup tries to login using the RM1's kerberos principal
- Closed