Author: jianhe
Date: Wed Jul 16 18:27:09 2014
New Revision: 1611127
URL: http://svn.apache.org/r1611127
Log:
Merge r1611126 from trunk. YARN-2264. Fixed a race condition in DrainDispatcher
which may cause random test failures. Contributed by Li Lu
Modified:
hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt
hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
Modified: hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt
URL:
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt?rev=1611127&r1=1611126&r2=1611127&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-yarn-project/CHANGES.txt Wed Jul 16
18:27:09 2014
@@ -41,6 +41,9 @@ Release 2.6.0 - UNRELEASED
YARN-2260. Fixed ResourceManager's RMNode to correctly remember containers
when nodes resync during work-preserving RM restart. (Jian He via vinodkv)
+ YARN-2264. Fixed a race condition in DrainDispatcher which may cause random
+ test failures. (Li Lu via jianhe)
+
Release 2.5.0 - UNRELEASED
INCOMPATIBLE CHANGES
Modified:
hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
URL:
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java?rev=1611127&r1=1611126&r2=1611127&view=diff
==============================================================================
---
hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
(original)
+++
hadoop/common/branches/branch-2/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/event/DrainDispatcher.java
Wed Jul 16 18:27:09 2014
@@ -28,6 +28,7 @@ public class DrainDispatcher extends Asy
// and similar grotesqueries
private volatile boolean drained = false;
private final BlockingQueue<Event> queue;
+ final Object mutex;
public DrainDispatcher() {
this(new LinkedBlockingQueue<Event>());
@@ -36,6 +37,7 @@ public class DrainDispatcher extends Asy
private DrainDispatcher(BlockingQueue<Event> eventQueue) {
super(eventQueue);
this.queue = eventQueue;
+ this.mutex = this;
}
/**
@@ -53,8 +55,10 @@ public class DrainDispatcher extends Asy
@Override
public void run() {
while (!Thread.currentThread().isInterrupted()) {
- // !drained if dispatch queued new events on this dispatcher
- drained = queue.isEmpty();
+ synchronized (mutex) {
+ // !drained if dispatch queued new events on this dispatcher
+ drained = queue.isEmpty();
+ }
Event event;
try {
event = queue.take();
@@ -75,8 +79,10 @@ public class DrainDispatcher extends Asy
return new EventHandler() {
@Override
public void handle(Event event) {
- drained = false;
- actual.handle(event);
+ synchronized (mutex) {
+ actual.handle(event);
+ drained = false;
+ }
}
};
}