YARN-5911. DrainDispatcher does not drain all events on stop even if setDrainEventsOnStop is true. Contributed by Varun Saxena.
(cherry picked from commit 4667564162
)
This commit is contained in:
parent
c11e0ef621
commit
b96ed43897
|
@ -151,7 +151,7 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
|
||||||
while (!isDrained() && eventHandlingThread != null
|
while (!isDrained() && eventHandlingThread != null
|
||||||
&& eventHandlingThread.isAlive()
|
&& eventHandlingThread.isAlive()
|
||||||
&& System.currentTimeMillis() < endTime) {
|
&& System.currentTimeMillis() < endTime) {
|
||||||
waitForDrained.wait(1000);
|
waitForDrained.wait(100);
|
||||||
LOG.info("Waiting for AsyncDispatcher to drain. Thread state is :" +
|
LOG.info("Waiting for AsyncDispatcher to drain. Thread state is :" +
|
||||||
eventHandlingThread.getState());
|
eventHandlingThread.getState());
|
||||||
}
|
}
|
||||||
|
@ -308,4 +308,8 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
|
||||||
protected boolean isDrained() {
|
protected boolean isDrained() {
|
||||||
return drained;
|
return drained;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected boolean isStopped() {
|
||||||
|
return stopped;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -25,7 +25,6 @@ import java.util.concurrent.LinkedBlockingQueue;
|
||||||
@SuppressWarnings("rawtypes")
|
@SuppressWarnings("rawtypes")
|
||||||
public class DrainDispatcher extends AsyncDispatcher {
|
public class DrainDispatcher extends AsyncDispatcher {
|
||||||
private volatile boolean drained = false;
|
private volatile boolean drained = false;
|
||||||
private volatile boolean stopped = false;
|
|
||||||
private final BlockingQueue<Event> queue;
|
private final BlockingQueue<Event> queue;
|
||||||
private final Object mutex;
|
private final Object mutex;
|
||||||
|
|
||||||
|
@ -69,7 +68,7 @@ public class DrainDispatcher extends AsyncDispatcher {
|
||||||
return new Runnable() {
|
return new Runnable() {
|
||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
while (!stopped && !Thread.currentThread().isInterrupted()) {
|
while (!isStopped() && !Thread.currentThread().isInterrupted()) {
|
||||||
synchronized (mutex) {
|
synchronized (mutex) {
|
||||||
// !drained if dispatch queued new events on this dispatcher
|
// !drained if dispatch queued new events on this dispatcher
|
||||||
drained = queue.isEmpty();
|
drained = queue.isEmpty();
|
||||||
|
@ -109,10 +108,4 @@ public class DrainDispatcher extends AsyncDispatcher {
|
||||||
return drained;
|
return drained;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void serviceStop() throws Exception {
|
|
||||||
stopped = true;
|
|
||||||
super.serviceStop();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.mockito.Mockito.*;
|
import static org.mockito.Mockito.*;
|
||||||
|
|
||||||
public class TestAsyncDispatcher {
|
public class TestAsyncDispatcher {
|
||||||
|
@ -77,5 +78,46 @@ public class TestAsyncDispatcher {
|
||||||
disp.waitForEventThreadToWait();
|
disp.waitForEventThreadToWait();
|
||||||
disp.close();
|
disp.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("rawtypes")
|
||||||
|
private static class DummyHandler implements EventHandler<Event> {
|
||||||
|
@Override
|
||||||
|
public void handle(Event event) {
|
||||||
|
try {
|
||||||
|
Thread.sleep(500);
|
||||||
|
} catch (InterruptedException e) {}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private enum DummyType {
|
||||||
|
DUMMY
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings({ "rawtypes", "unchecked" })
|
||||||
|
private void dispatchDummyEvents(Dispatcher disp, int count) {
|
||||||
|
for (int i = 0; i < count; i++) {
|
||||||
|
Event event = mock(Event.class);
|
||||||
|
when(event.getType()).thenReturn(DummyType.DUMMY);
|
||||||
|
disp.getEventHandler().handle(event);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// Test if drain dispatcher drains events on stop.
|
||||||
|
@SuppressWarnings({ "rawtypes" })
|
||||||
|
@Test(timeout=10000)
|
||||||
|
public void testDrainDispatcherDrainEventsOnStop() throws Exception {
|
||||||
|
YarnConfiguration conf = new YarnConfiguration();
|
||||||
|
conf.setInt(YarnConfiguration.DISPATCHER_DRAIN_EVENTS_TIMEOUT, 2000);
|
||||||
|
BlockingQueue<Event> queue = new LinkedBlockingQueue<Event>();
|
||||||
|
DrainDispatcher disp = new DrainDispatcher(queue);
|
||||||
|
disp.init(conf);
|
||||||
|
disp.register(DummyType.class, new DummyHandler());
|
||||||
|
disp.setDrainEventsOnStop();
|
||||||
|
disp.start();
|
||||||
|
disp.waitForEventThreadToWait();
|
||||||
|
dispatchDummyEvents(disp, 2);
|
||||||
|
disp.close();
|
||||||
|
assertEquals(0, queue.size());
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue