Uploaded image for project: 'Hadoop YARN'
  1. Hadoop YARN
  2. YARN-9802 YARN Timeline Service v2 (post GA features)
  3. YARN-9821

NM hangs at serviceStop when ATSV2 Backend Hbase is Down

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.2.0, 3.3.0
    • 3.3.0, 3.2.1
    • ATSv2
    • None
    • Reviewed

    Description

      NM hangs at serviceStop when ATSV2 Backend Hbase is Down.

      
      "Thread-197" #302 prio=5 os_prio=0 tid=0x00007f5f389ba000 nid=0x631d waiting for monitor entry [0x00007f5f1f29b000]
         java.lang.Thread.State: BLOCKED (on object monitor)
      	at org.apache.hadoop.hbase.client.BufferedMutatorImpl.close(BufferedMutatorImpl.java:249)
      	- waiting to lock <0x00000006c834d148> (a org.apache.hadoop.hbase.client.BufferedMutatorImpl)
      	at org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator.close(TypedBufferedMutator.java:62)
      	at org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl.serviceStop(HBaseTimelineWriterImpl.java:636)
      	at org.apache.hadoop.service.AbstractService.stop(AbstractService.java:220)
      	- locked <0x00000006c7c05808> (a java.lang.Object)
      	at org.apache.hadoop.service.AbstractService.close(AbstractService.java:247)
      	at org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorManager.serviceStop(TimelineCollectorManager.java:244)
      	at org.apache.hadoop.yarn.server.timelineservice.collector.NodeTimelineCollectorManager.serviceStop(NodeTimelineCollectorManager.java:164)
      	at org.apache.hadoop.service.AbstractService.stop(AbstractService.java:220)
      	- locked <0x00000006c7c05890> (a java.lang.Object)
      	at org.apache.hadoop.yarn.server.timelineservice.collector.PerNodeTimelineCollectorsAuxService.serviceStop(PerNodeTimelineCollectorsAuxService.java:113)
      	at org.apache.hadoop.service.AbstractService.stop(AbstractService.java:220)
      	- locked <0x00000006c7c058f8> (a java.lang.Object)
      	at org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServices.serviceStop(AuxServices.java:330)
      	- locked <0x00000006c7c23400> (a java.util.Collections$SynchronizedMap)
      	at org.apache.hadoop.service.AbstractService.stop(AbstractService.java:220)
      	- locked <0x00000006c7c059a8> (a java.lang.Object)
      	at org.apache.hadoop.service.ServiceOperations.stop(ServiceOperations.java:54)
      	at org.apache.hadoop.service.ServiceOperations.stopQuietly(ServiceOperations.java:102)
      	at org.apache.hadoop.service.CompositeService.stop(CompositeService.java:158)
      	at org.apache.hadoop.service.CompositeService.serviceStop(CompositeService.java:132)
      	at org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl.serviceStop(ContainerManagerImpl.java:720)
      	at org.apache.hadoop.service.AbstractService.stop(AbstractService.java:220)
      	- locked <0x00000006c7c05a98> (a java.lang.Object)
      	at org.apache.hadoop.service.ServiceOperations.stop(ServiceOperations.java:54)
      	at org.apache.hadoop.service.ServiceOperations.stopQuietly(ServiceOperations.java:102)
      	at org.apache.hadoop.service.CompositeService.stop(CompositeService.java:158)
      	at org.apache.hadoop.service.CompositeService.serviceStop(CompositeService.java:132)
      	at org.apache.hadoop.yarn.server.nodemanager.NodeManager.serviceStop(NodeManager.java:526)
      	at org.apache.hadoop.service.AbstractService.stop(AbstractService.java:220)
      	- locked <0x00000006c7c05c88> (a java.lang.Object)
      	at org.apache.hadoop.yarn.server.nodemanager.NodeManager$1.run(NodeManager.java:552)
      	
      	
      "qtp183259297-76" #76 daemon prio=5 os_prio=0 tid=0x00007f5f567ed000 nid=0x5fb7 in Object.wait() [0x00007f5f23ad7000]
         java.lang.Thread.State: TIMED_WAITING (on object monitor)
      	at java.lang.Object.wait(Native Method)
      	at java.lang.Object.wait(Object.java:460)
      	at java.util.concurrent.TimeUnit.timedWait(TimeUnit.java:348)
      	at org.apache.hadoop.hbase.client.ResultBoundedCompletionService.pollForSpecificCompletedTask(ResultBoundedCompletionService.java:258)
      	- locked <0x0000000784ee8220> (a [Lorg.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture;)
      	at org.apache.hadoop.hbase.client.ResultBoundedCompletionService.pollForFirstSuccessfullyCompletedTask(ResultBoundedCompletionService.java:214)
      	at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:228)
      	at org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:58)
      	at org.apache.hadoop.hbase.client.RpcRetryingCallerImpl.callWithoutRetries(RpcRetryingCallerImpl.java:192)
      	at org.apache.hadoop.hbase.client.ClientScanner.call(ClientScanner.java:269)
      	at org.apache.hadoop.hbase.client.ClientScanner.loadCache(ClientScanner.java:437)
      	at org.apache.hadoop.hbase.client.ClientScanner.nextWithSyncCache(ClientScanner.java:312)
      	at org.apache.hadoop.hbase.client.ClientScanner.next(ClientScanner.java:597)
      	at org.apache.hadoop.hbase.client.ConnectionImplementation.locateRegionInMeta(ConnectionImplementation.java:834)
      	at org.apache.hadoop.hbase.client.ConnectionImplementation.locateRegion(ConnectionImplementation.java:732)
      	at org.apache.hadoop.hbase.client.AsyncProcess.submit(AsyncProcess.java:281)
      	at org.apache.hadoop.hbase.client.AsyncProcess.submit(AsyncProcess.java:236)
      	at org.apache.hadoop.hbase.client.BufferedMutatorImpl.backgroundFlushCommits(BufferedMutatorImpl.java:321)
      	at org.apache.hadoop.hbase.client.BufferedMutatorImpl.flush(BufferedMutatorImpl.java:285)
      	- locked <0x00000006c834d148> (a org.apache.hadoop.hbase.client.BufferedMutatorImpl)
      	at org.apache.hadoop.yarn.server.timelineservice.storage.common.TypedBufferedMutator.flush(TypedBufferedMutator.java:66)
      	at org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl.flush(HBaseTimelineWriterImpl.java:618)
      	at org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector.flushBufferedTimelineEntities(TimelineCollector.java:215)
      	at org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollector.putEntities(TimelineCollector.java:165)
      	- locked <0x00000006c7c05770> (a org.apache.hadoop.yarn.server.timelineservice.storage.HBaseTimelineWriterImpl)
      	at org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorWebService.putEntities(TimelineCollectorWebService.java:186)
      	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:498)
      	at com.sun.jersey.spi.container.JavaMethodInvokerFactory$1.invoke(JavaMethodInvokerFactory.java:60)
      	at com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$ResponseOutInvoker._dispatch(AbstractResourceMethodDispatchProvider.java:205)
      	at com.sun.jersey.server.impl.model.method.dispatch.ResourceJavaMethodDispatcher.dispatch(ResourceJavaMethodDispatcher.java:75)
      	at com.sun.jersey.server.impl.uri.rules.HttpMethodRule.accept(HttpMethodRule.java:302)
      	at com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
      	at com.sun.jersey.server.impl.uri.rules.ResourceClassRule.accept(ResourceClassRule.java:108)
      	at com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
      	at com.sun.jersey.server.impl.uri.rules.RootResourceClassesRule.accept(RootResourceClassesRule.java:84)
      	at com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1542)
      	at com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1473)
      	at com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1419)
      	at com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1409)
      	at com.sun.jersey.spi.container.servlet.WebComponent.service(WebComponent.java:409)
      	at com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:558)
      	at com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:733)
      	at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
      	at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:848)
      	at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1772)
      	at org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:644)
      	at org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter.doFilter(DelegationTokenAuthenticationFilter.java:304)
      	at org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:592)
      	at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
      	at org.apache.hadoop.http.HttpServer2$QuotingInputFilter.doFilter(HttpServer2.java:1624)
      	at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
      	at org.apache.hadoop.http.NoCacheFilter.doFilter(NoCacheFilter.java:45)
      	at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
      	at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:582)
      	at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:143)
      	at org.eclipse.jetty.security.SecurityHandler.handle(SecurityHandler.java:548)
      	at org.eclipse.jetty.server.session.SessionHandler.doHandle(SessionHandler.java:226)
      	at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)
      	at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:512)
      	at org.eclipse.jetty.server.session.SessionHandler.doScope(SessionHandler.java:185)
      	at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112)
      	at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
      	at org.eclipse.jetty.server.handler.HandlerCollection.handle(HandlerCollection.java:119)
      	at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
      	at org.eclipse.jetty.server.Server.handle(Server.java:539)
      	at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:333)
      	at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)
      	at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)
      	at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:108)
      	at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
      	at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)
      	at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)
      	at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.execute(ExecuteProduceConsume.java:100)
      	at org.eclipse.jetty.io.ManagedSelector.run(ManagedSelector.java:147)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)
      	at org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)
      	at java.lang.Thread.run(Thread.java:748)
      

      Attachments

        1. YARN-9821-001.patch
          4 kB
          Prabhu Joseph
        2. YARN-9821-002.patch
          4 kB
          Prabhu Joseph

        Activity

          People

            prabhujoseph Prabhu Joseph
            prabhujoseph Prabhu Joseph
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: