[jira] [Commented] (YARN-3574) RM hangs on stopping MetricsSinkAdapter when transitioning to standby

2015-05-06 Thread Jian He (JIRA)

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

Jian He commented on YARN-3574:
---

Thanks [~rohithsharma] for the investigation  !

Maybe in MetricsSinkAdapter#stop, we can do  the {{IOUtils.cleanup}}  first and 
then interrupt. 
{code}
  void stop() {
stopping = true;
sinkThread.interrupt();
try {
  sinkThread.join();
} catch (InterruptedException e) {
  LOG.warn("Stop interrupted", e);
}
if (sink instanceof Closeable) {
  IOUtils.cleanup(LOG, (Closeable)sink);
}
  }
{code}

converting this to hadoop-common ticket

> RM hangs on stopping MetricsSinkAdapter when transitioning to standby
> -
>
> Key: YARN-3574
> URL: https://issues.apache.org/jira/browse/YARN-3574
> Project: Hadoop YARN
>  Issue Type: Bug
>Reporter: Jian He
>Assignee: Brahma Reddy Battula
>
> We've seen a situation that one RM hangs on stopping the MetricsSinkAdapter
> {code}
> "main-EventThread" daemon prio=10 tid=0x7f9b24031000 nid=0x2d18 in 
> Object.wait() [0x7f9afe7eb000]
>java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> - waiting on <0xc058dcf8> (a 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter$1)
> at java.lang.Thread.join(Thread.java:1281)
> - locked <0xc058dcf8> (a 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter$1)
> at java.lang.Thread.join(Thread.java:1355)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter.stop(MetricsSinkAdapter.java:202)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.stopSinks(MetricsSystemImpl.java:472)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.stop(MetricsSystemImpl.java:213)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.shutdown(MetricsSystemImpl.java:592)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.lib.DefaultMetricsSystem.shutdownInstance(DefaultMetricsSystem.java:72)
> at 
> org.apache.hadoop.metrics2.lib.DefaultMetricsSystem.shutdown(DefaultMetricsSystem.java:68)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$RMActiveServices.serviceStop(ResourceManager.java:605)
> at 
> org.apache.hadoop.service.AbstractService.stop(AbstractService.java:221)
> - locked <0xc0503568> (a java.lang.Object)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.stopActiveServices(ResourceManager.java:1024)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.transitionToStandby(ResourceManager.java:1076)
> - locked <0xc03fe3b8> (a 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.AdminService.transitionToStandby(AdminService.java:322)
> - locked <0xc0502b10> (a 
> org.apache.hadoop.yarn.server.resourcemanager.AdminService)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.EmbeddedElectorService.becomeStandby(EmbeddedElectorService.java:135)
> at 
> org.apache.hadoop.ha.ActiveStandbyElector.becomeStandby(ActiveStandbyElector.java:911)
> at 
> org.apache.hadoop.ha.ActiveStandbyElector.processResult(ActiveStandbyElector.java:428)
> - locked <0xc0718940> (a 
> org.apache.hadoop.ha.ActiveStandbyElector)
> at 
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:605)
> at 
> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
> {code}
> {code}
> "timeline" daemon prio=10 tid=0x7f9b34d55000 nid=0x1d93 runnable 
> [0x7f9b0cbbf000]
>java.lang.Thread.State: RUNNABLE
> at java.net.SocketInputStream.socketRead0(Native Method)
> at java.net.SocketInputStream.read(SocketInputStream.java:152)
> at java.net.SocketInputStream.read(SocketInputStream.java:122)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:254)
> - locked <0xc0f522c8> (a java.io.BufferedInputStream)
> at 
> org.apache.commons.httpclient.HttpParser.readRawLine(HttpParser.java:78)
> at 
> org.apache.commons.httpclient.HttpParser.readLine(HttpParser.java:106)
> at 
> org.apache.commons.httpclient.HttpConnection.readLine(HttpConnection.java:1116)
> at 
> org.apache

[jira] [Commented] (YARN-3574) RM hangs on stopping MetricsSinkAdapter when transitioning to standby

2015-05-06 Thread Rohith (JIRA)

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

Rohith commented on YARN-3574:
--

Done some more investigation on this modifying the above program that start 
ServerSocket. It is found that SocketInputStream.read() is NOT Interruptible!!
{code}
package com.rohith;

import java.io.DataInputStream;
import java.io.IOException;
import java.io.InputStream;
import java.net.ServerSocket;
import java.net.Socket;
import java.net.UnknownHostException;

public class Test {
  Thread sinkThread;
  Socket socket = null;
  InputStream in = null;
  private volatile boolean stopping = false;

  public void start() {
sinkThread = new Thread() {
  public void run() {

try {
  socket = new Socket("localhost", );
} catch (UnknownHostException e2) {
  e2.printStackTrace();
} catch (IOException e2) {
  e2.printStackTrace();
}


try {
  in = socket.getInputStream();
} catch (IOException e1) {
  e1.printStackTrace();
}
while (!stopping) {
  try {
System.out.println("in.read");
in.read();
  } catch (IOException e) {
e.printStackTrace();
  }
}
  };
};
sinkThread.setDaemon(true);
sinkThread.start();
  }

  public void stop() {
stopping = true;

try {
  socket.close();
} catch (IOException e1) {
  e1.printStackTrace();
}

System.out.println("Interrupting.. ");
System.out.println(sinkThread.getState());
sinkThread.interrupt();
try {
  System.out.println("Joining.. ");
  sinkThread.join();
} catch (InterruptedException e) {
  System.out.println("Stop interrupted " + e);
}
System.out.println("Stopped successfully");
  }

  public static void main(String[] args) throws InterruptedException {
try {
  ServerSocket ss = new ServerSocket();
} catch (IOException e) {
  e.printStackTrace();
}

System.out.println("Made socket, now reading from socket");

Test t1 = new Test();
t1.start();
Thread.sleep(2000);
t1.stop();
  }

}
{code}

> RM hangs on stopping MetricsSinkAdapter when transitioning to standby
> -
>
> Key: YARN-3574
> URL: https://issues.apache.org/jira/browse/YARN-3574
> Project: Hadoop YARN
>  Issue Type: Bug
>Reporter: Jian He
>Assignee: Brahma Reddy Battula
>
> We've seen a situation that one RM hangs on stopping the MetricsSinkAdapter
> {code}
> "main-EventThread" daemon prio=10 tid=0x7f9b24031000 nid=0x2d18 in 
> Object.wait() [0x7f9afe7eb000]
>java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> - waiting on <0xc058dcf8> (a 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter$1)
> at java.lang.Thread.join(Thread.java:1281)
> - locked <0xc058dcf8> (a 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter$1)
> at java.lang.Thread.join(Thread.java:1355)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter.stop(MetricsSinkAdapter.java:202)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.stopSinks(MetricsSystemImpl.java:472)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.stop(MetricsSystemImpl.java:213)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.shutdown(MetricsSystemImpl.java:592)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.lib.DefaultMetricsSystem.shutdownInstance(DefaultMetricsSystem.java:72)
> at 
> org.apache.hadoop.metrics2.lib.DefaultMetricsSystem.shutdown(DefaultMetricsSystem.java:68)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$RMActiveServices.serviceStop(ResourceManager.java:605)
> at 
> org.apache.hadoop.service.AbstractService.stop(AbstractService.java:221)
> - locked <0xc0503568> (a java.lang.Object)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.stopActiveServices(ResourceManager.java:1024)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.transitionToStandby(ResourceManager.java:1076)
> - locked <0xc03fe3b8> (a 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.AdminService.transitionToStandby(AdminService.java:322)
> - locked <0x00

[jira] [Commented] (YARN-3574) RM hangs on stopping MetricsSinkAdapter when transitioning to standby

2015-05-05 Thread Rohith (JIRA)

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

Rohith commented on YARN-3574:
--

Very interesting bug!!  Going back to Java basics, Thread.interrupt() does not 
guarentee the interrupt for running thread unless thread is waintig/sleeping 
for something. 
In this issue I think {{queue.consumeAll(this);}} processing something which 
never given chance to interrupt it.

Just to reproduce this, small program wrote below code. If we run below code 
withoug commenting Thread.sleep, thread never get interrupted. Adding small 
sleep , result in thread get interrupted.
{code}
package com.test.basic;

public class Test1 {
  Thread sinkThread;
  private volatile boolean stopping = false;

  public void start() {
sinkThread = new Thread() {
  public void run() {
while (!stopping) {
  try {
while (true) {
  // Thread.sleep(1);
}
  } catch (Exception e) {
System.out.println("Interuppted..");
  }
}
  };
};
sinkThread.setDaemon(true);
sinkThread.start();
  }

  public void stop() {
stopping = true;
System.out.println("Interrupting.. ");
sinkThread.interrupt();
try {
  System.out.println("Joining.. ");
  sinkThread.join();
} catch (InterruptedException e) {
  System.out.println("Stop interrupted " + e);
}
System.out.println("Stopped successfully");
  }

  public static void main(String[] args) throws InterruptedException {
Test1 t1 = new Test1();
t1.start();
Thread.sleep(2000);
t1.stop();
  }
}
{code}

> RM hangs on stopping MetricsSinkAdapter when transitioning to standby
> -
>
> Key: YARN-3574
> URL: https://issues.apache.org/jira/browse/YARN-3574
> Project: Hadoop YARN
>  Issue Type: Bug
>Reporter: Jian He
>Assignee: Brahma Reddy Battula
>
> We've seen a situation that one RM hangs on stopping the MetricsSinkAdapter
> {code}
> "main-EventThread" daemon prio=10 tid=0x7f9b24031000 nid=0x2d18 in 
> Object.wait() [0x7f9afe7eb000]
>java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> - waiting on <0xc058dcf8> (a 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter$1)
> at java.lang.Thread.join(Thread.java:1281)
> - locked <0xc058dcf8> (a 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter$1)
> at java.lang.Thread.join(Thread.java:1355)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter.stop(MetricsSinkAdapter.java:202)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.stopSinks(MetricsSystemImpl.java:472)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.stop(MetricsSystemImpl.java:213)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.shutdown(MetricsSystemImpl.java:592)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.lib.DefaultMetricsSystem.shutdownInstance(DefaultMetricsSystem.java:72)
> at 
> org.apache.hadoop.metrics2.lib.DefaultMetricsSystem.shutdown(DefaultMetricsSystem.java:68)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$RMActiveServices.serviceStop(ResourceManager.java:605)
> at 
> org.apache.hadoop.service.AbstractService.stop(AbstractService.java:221)
> - locked <0xc0503568> (a java.lang.Object)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.stopActiveServices(ResourceManager.java:1024)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.transitionToStandby(ResourceManager.java:1076)
> - locked <0xc03fe3b8> (a 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.AdminService.transitionToStandby(AdminService.java:322)
> - locked <0xc0502b10> (a 
> org.apache.hadoop.yarn.server.resourcemanager.AdminService)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.EmbeddedElectorService.becomeStandby(EmbeddedElectorService.java:135)
> at 
> org.apache.hadoop.ha.ActiveStandbyElector.becomeStandby(ActiveStandbyElector.java:911)
> at 
> org.apache.hadoop.ha.ActiveStandbyElector.processResult(ActiveStandbyElector.java:428)
> - locked <0xc0718940> (a 
> org.apache.hadoop.ha.ActiveStandbyElector)
> at 
> org.apache.zookeeper.ClientCnxn$Ev

[jira] [Commented] (YARN-3574) RM hangs on stopping MetricsSinkAdapter when transitioning to standby

2015-05-04 Thread Jian He (JIRA)

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

Jian He commented on YARN-3574:
---

[~brahmareddy],  I'm also not able to repro.. I wondered if any other folks 
have seen this issue before.
we found this while doing ambari integration testing. I added one more stack 
trace for the blocking thread in the description. 



> RM hangs on stopping MetricsSinkAdapter when transitioning to standby
> -
>
> Key: YARN-3574
> URL: https://issues.apache.org/jira/browse/YARN-3574
> Project: Hadoop YARN
>  Issue Type: Bug
>Reporter: Jian He
>Assignee: Brahma Reddy Battula
>
> We've seen a situation that one RM hangs on stopping the MetricsSinkAdapter
> {code}
> "main-EventThread" daemon prio=10 tid=0x7f9b24031000 nid=0x2d18 in 
> Object.wait() [0x7f9afe7eb000]
>java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> - waiting on <0xc058dcf8> (a 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter$1)
> at java.lang.Thread.join(Thread.java:1281)
> - locked <0xc058dcf8> (a 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter$1)
> at java.lang.Thread.join(Thread.java:1355)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter.stop(MetricsSinkAdapter.java:202)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.stopSinks(MetricsSystemImpl.java:472)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.stop(MetricsSystemImpl.java:213)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.shutdown(MetricsSystemImpl.java:592)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.lib.DefaultMetricsSystem.shutdownInstance(DefaultMetricsSystem.java:72)
> at 
> org.apache.hadoop.metrics2.lib.DefaultMetricsSystem.shutdown(DefaultMetricsSystem.java:68)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$RMActiveServices.serviceStop(ResourceManager.java:605)
> at 
> org.apache.hadoop.service.AbstractService.stop(AbstractService.java:221)
> - locked <0xc0503568> (a java.lang.Object)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.stopActiveServices(ResourceManager.java:1024)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.transitionToStandby(ResourceManager.java:1076)
> - locked <0xc03fe3b8> (a 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.AdminService.transitionToStandby(AdminService.java:322)
> - locked <0xc0502b10> (a 
> org.apache.hadoop.yarn.server.resourcemanager.AdminService)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.EmbeddedElectorService.becomeStandby(EmbeddedElectorService.java:135)
> at 
> org.apache.hadoop.ha.ActiveStandbyElector.becomeStandby(ActiveStandbyElector.java:911)
> at 
> org.apache.hadoop.ha.ActiveStandbyElector.processResult(ActiveStandbyElector.java:428)
> - locked <0xc0718940> (a 
> org.apache.hadoop.ha.ActiveStandbyElector)
> at 
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:605)
> at 
> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
> {code}
> {code}
> "timeline" daemon prio=10 tid=0x7f9b34d55000 nid=0x1d93 runnable 
> [0x7f9b0cbbf000]
>java.lang.Thread.State: RUNNABLE
> at java.net.SocketInputStream.socketRead0(Native Method)
> at java.net.SocketInputStream.read(SocketInputStream.java:152)
> at java.net.SocketInputStream.read(SocketInputStream.java:122)
> at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
> at java.io.BufferedInputStream.read(BufferedInputStream.java:254)
> - locked <0xc0f522c8> (a java.io.BufferedInputStream)
> at 
> org.apache.commons.httpclient.HttpParser.readRawLine(HttpParser.java:78)
> at 
> org.apache.commons.httpclient.HttpParser.readLine(HttpParser.java:106)
> at 
> org.apache.commons.httpclient.HttpConnection.readLine(HttpConnection.java:1116)
> at 
> org.apache.commons.httpclient.HttpMethodBase.readStatusLine(HttpMethodBase.java:1973)
> at 
> org.apache.commons.httpclient.HttpMethodBase.readResponse(HttpMethodBase.java:1735)
> at 
> org.apache.commons.httpclient.HttpMethodBase.exec

[jira] [Commented] (YARN-3574) RM hangs on stopping MetricsSinkAdapter when transitioning to standby

2015-05-04 Thread Brahma Reddy Battula (JIRA)

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

Brahma Reddy Battula commented on YARN-3574:


[~jianhe] I would like to work on this.. I am not able to reproduce this .. can 
you please give scenario ..?

> RM hangs on stopping MetricsSinkAdapter when transitioning to standby
> -
>
> Key: YARN-3574
> URL: https://issues.apache.org/jira/browse/YARN-3574
> Project: Hadoop YARN
>  Issue Type: Bug
>Reporter: Jian He
>
> We've seen a situation that one RM hangs on stopping the MetricsSinkAdapter
> {code}
> "main-EventThread" daemon prio=10 tid=0x7f9b24031000 nid=0x2d18 in 
> Object.wait() [0x7f9afe7eb000]
>java.lang.Thread.State: WAITING (on object monitor)
> at java.lang.Object.wait(Native Method)
> - waiting on <0xc058dcf8> (a 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter$1)
> at java.lang.Thread.join(Thread.java:1281)
> - locked <0xc058dcf8> (a 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter$1)
> at java.lang.Thread.join(Thread.java:1355)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSinkAdapter.stop(MetricsSinkAdapter.java:202)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.stopSinks(MetricsSystemImpl.java:472)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.stop(MetricsSystemImpl.java:213)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl.shutdown(MetricsSystemImpl.java:592)
> - locked <0xc04cc1a0> (a 
> org.apache.hadoop.metrics2.impl.MetricsSystemImpl)
> at 
> org.apache.hadoop.metrics2.lib.DefaultMetricsSystem.shutdownInstance(DefaultMetricsSystem.java:72)
> at 
> org.apache.hadoop.metrics2.lib.DefaultMetricsSystem.shutdown(DefaultMetricsSystem.java:68)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$RMActiveServices.serviceStop(ResourceManager.java:605)
> at 
> org.apache.hadoop.service.AbstractService.stop(AbstractService.java:221)
> - locked <0xc0503568> (a java.lang.Object)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.stopActiveServices(ResourceManager.java:1024)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager.transitionToStandby(ResourceManager.java:1076)
> - locked <0xc03fe3b8> (a 
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.AdminService.transitionToStandby(AdminService.java:322)
> - locked <0xc0502b10> (a 
> org.apache.hadoop.yarn.server.resourcemanager.AdminService)
> at 
> org.apache.hadoop.yarn.server.resourcemanager.EmbeddedElectorService.becomeStandby(EmbeddedElectorService.java:135)
> at 
> org.apache.hadoop.ha.ActiveStandbyElector.becomeStandby(ActiveStandbyElector.java:911)
> at 
> org.apache.hadoop.ha.ActiveStandbyElector.processResult(ActiveStandbyElector.java:428)
> - locked <0xc0718940> (a 
> org.apache.hadoop.ha.ActiveStandbyElector)
> at 
> org.apache.zookeeper.ClientCnxn$EventThread.processEvent(ClientCnxn.java:605)
> at 
> org.apache.zookeeper.ClientCnxn$EventThread.run(ClientCnxn.java:498)
> {code}
>  looks like the {{sinkThread.interrupt();}} in MetricsSinkAdapter#stop 
> doesn't really interrupt the thread, which cause it to hang at join.
> This appears only once.



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