You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ji...@apache.org on 2015/07/22 01:14:24 UTC
[40/44] hadoop git commit: YARN-3878. AsyncDispatcher can hang while
stopping if it is configured for draining events on stop. Contributed by
Varun Saxena
YARN-3878. AsyncDispatcher can hang while stopping if it is configured for draining events on stop. Contributed by Varun Saxena
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/393fe717
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/393fe717
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/393fe717
Branch: refs/heads/YARN-1197
Commit: 393fe71771e3ac6bc0efe59d9aaf19d3576411b3
Parents: a26cc66
Author: Jian He <ji...@apache.org>
Authored: Tue Jul 21 15:05:41 2015 -0700
Committer: Jian He <ji...@apache.org>
Committed: Tue Jul 21 15:05:41 2015 -0700
----------------------------------------------------------------------
hadoop-yarn-project/CHANGES.txt | 3 +
.../hadoop/yarn/event/AsyncDispatcher.java | 8 +++
.../hadoop/yarn/event/DrainDispatcher.java | 11 +++-
.../hadoop/yarn/event/TestAsyncDispatcher.java | 62 ++++++++++++++++++++
4 files changed, 83 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hadoop/blob/393fe717/hadoop-yarn-project/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 79e9ae2..5100cdf 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -682,6 +682,9 @@ Release 2.7.2 - UNRELEASED
YARN-3535. Scheduler must re-request container resources when RMContainer transitions
from ALLOCATED to KILLED (rohithsharma and peng.zhang via asuresh)
+ YARN-3878. AsyncDispatcher can hang while stopping if it is configured for
+ draining events on stop. (Varun Saxena via jianhe)
+
Release 2.7.1 - 2015-07-06
INCOMPATIBLE CHANGES
http://git-wip-us.apache.org/repos/asf/hadoop/blob/393fe717/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
index c54b9c7..48312a3 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
@@ -246,6 +246,9 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
if (!stopped) {
LOG.warn("AsyncDispatcher thread interrupted", e);
}
+ // Need to reset drained flag to true if event queue is empty,
+ // otherwise dispatcher will hang on stop.
+ drained = eventQueue.isEmpty();
throw new YarnRuntimeException(e);
}
};
@@ -287,6 +290,11 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
}
@VisibleForTesting
+ protected boolean isEventThreadWaiting() {
+ return eventHandlingThread.getState() == Thread.State.WAITING;
+ }
+
+ @VisibleForTesting
protected boolean isDrained() {
return this.drained;
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/393fe717/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
index da5ae44..e4a5a82 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
@@ -27,11 +27,20 @@ public class DrainDispatcher extends AsyncDispatcher {
this(new LinkedBlockingQueue<Event>());
}
- private DrainDispatcher(BlockingQueue<Event> eventQueue) {
+ public DrainDispatcher(BlockingQueue<Event> eventQueue) {
super(eventQueue);
}
/**
+ * Wait till event thread enters WAITING state (i.e. waiting for new events).
+ */
+ public void waitForEventThreadToWait() {
+ while (!isEventThreadWaiting()) {
+ Thread.yield();
+ }
+ }
+
+ /**
* Busy loop waiting for all queued events to drain.
*/
public void await() {
http://git-wip-us.apache.org/repos/asf/hadoop/blob/393fe717/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
new file mode 100644
index 0000000..b5fd923
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/TestAsyncDispatcher.java
@@ -0,0 +1,62 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.event;
+
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestAsyncDispatcher {
+
+ /* This test checks whether dispatcher hangs on close if following two things
+ * happen :
+ * 1. A thread which was putting event to event queue is interrupted.
+ * 2. Event queue is empty on close.
+ */
+ @SuppressWarnings({ "unchecked", "rawtypes" })
+ @Test(timeout=10000)
+ public void testDispatcherOnCloseIfQueueEmpty() throws Exception {
+ BlockingQueue<Event> eventQueue = spy(new LinkedBlockingQueue<Event>());
+ Event event = mock(Event.class);
+ doThrow(new InterruptedException()).when(eventQueue).put(event);
+ DrainDispatcher disp = new DrainDispatcher(eventQueue);
+ disp.init(new Configuration());
+ disp.setDrainEventsOnStop();
+ disp.start();
+ // Wait for event handler thread to start and begin waiting for events.
+ disp.waitForEventThreadToWait();
+ try {
+ disp.getEventHandler().handle(event);
+ } catch (YarnRuntimeException e) {
+ }
+ // Queue should be empty and dispatcher should not hang on close
+ Assert.assertTrue("Event Queue should have been empty",
+ eventQueue.isEmpty());
+ disp.close();
+ }
+}
+