|
@@ -18,6 +18,7 @@
|
|
|
|
|
|
package org.apache.hadoop.tools.util;
|
|
|
|
|
|
+import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.tools.util.ProducerConsumer;
|
|
|
import org.apache.hadoop.tools.util.WorkReport;
|
|
|
import org.apache.hadoop.tools.util.WorkRequest;
|
|
@@ -27,6 +28,7 @@ import org.junit.Test;
|
|
|
|
|
|
import java.lang.Exception;
|
|
|
import java.lang.Integer;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
public class TestProducerConsumer {
|
|
|
public class CopyProcessor implements WorkRequestProcessor<Integer, Integer> {
|
|
@@ -64,6 +66,7 @@ public class TestProducerConsumer {
|
|
|
} catch (InterruptedException ie) {
|
|
|
Assert.assertTrue(false);
|
|
|
}
|
|
|
+ worker.shutdown();
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -89,6 +92,7 @@ public class TestProducerConsumer {
|
|
|
}
|
|
|
Assert.assertEquals(0, sum);
|
|
|
Assert.assertEquals(numRequests, numReports);
|
|
|
+ workers.shutdown();
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -105,5 +109,85 @@ public class TestProducerConsumer {
|
|
|
} catch (InterruptedException ie) {
|
|
|
Assert.assertTrue(false);
|
|
|
}
|
|
|
+ worker.shutdown();
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test
|
|
|
+ public void testSimpleProducerConsumerShutdown() throws InterruptedException,
|
|
|
+ TimeoutException {
|
|
|
+ // create a producer-consumer thread pool with one thread.
|
|
|
+ ProducerConsumer<Integer, Integer> worker =
|
|
|
+ new ProducerConsumer<Integer, Integer>(1);
|
|
|
+ worker.addWorker(new CopyProcessor());
|
|
|
+ // interrupt worker threads
|
|
|
+ worker.shutdown();
|
|
|
+ // Regression test for HDFS-9612
|
|
|
+ // Periodically check, and make sure that worker threads are ultimately
|
|
|
+ // terminated after interrupts
|
|
|
+ GenericTestUtils.waitForThreadTermination("pool-.*-thread.*",100,10000);
|
|
|
+ }
|
|
|
+
|
|
|
+ @Test(timeout=10000)
|
|
|
+ public void testMultipleProducerConsumerShutdown()
|
|
|
+ throws InterruptedException, TimeoutException {
|
|
|
+ int numWorkers = 10;
|
|
|
+ // create a producer consumer thread pool with 10 threads.
|
|
|
+ final ProducerConsumer<Integer, Integer> worker =
|
|
|
+ new ProducerConsumer<Integer, Integer>(numWorkers);
|
|
|
+ for (int i=0; i< numWorkers; i++) {
|
|
|
+ worker.addWorker(new CopyProcessor());
|
|
|
+ }
|
|
|
+
|
|
|
+ // starts two thread: a source thread which put in work, and a sink thread
|
|
|
+ // which takes a piece of work from ProducerConsumer
|
|
|
+ class SourceThread extends Thread {
|
|
|
+ public void run() {
|
|
|
+ while (true) {
|
|
|
+ try {
|
|
|
+ worker.put(new WorkRequest<Integer>(42));
|
|
|
+ Thread.sleep(1);
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ // The source thread put requests into producer-consumer.
|
|
|
+ SourceThread source = new SourceThread();
|
|
|
+ source.start();
|
|
|
+ class SinkThread extends Thread {
|
|
|
+ public void run() {
|
|
|
+ try {
|
|
|
+ while (true) {
|
|
|
+ WorkReport<Integer> report = worker.take();
|
|
|
+ Assert.assertEquals(42, report.getItem().intValue());
|
|
|
+ }
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ };
|
|
|
+ // The sink thread gets proceessed items from producer-consumer
|
|
|
+ SinkThread sink = new SinkThread();
|
|
|
+ sink.start();
|
|
|
+ // sleep 1 second and then shut down source.
|
|
|
+ // This makes sure producer consumer gets some work to do
|
|
|
+ Thread.sleep(1000);
|
|
|
+ // after 1 second, stop source thread to stop pushing items.
|
|
|
+ source.interrupt();
|
|
|
+ // wait until all work is consumed by sink
|
|
|
+ while (worker.hasWork()) {
|
|
|
+ Thread.sleep(1);
|
|
|
+ }
|
|
|
+ worker.shutdown();
|
|
|
+ // Regression test for HDFS-9612
|
|
|
+ // make sure worker threads are terminated after workers are asked to
|
|
|
+ // shutdown.
|
|
|
+ GenericTestUtils.waitForThreadTermination("pool-.*-thread.*",100,10000);
|
|
|
+
|
|
|
+ sink.interrupt();
|
|
|
+
|
|
|
+ source.join();
|
|
|
+ sink.join();
|
|
|
}
|
|
|
}
|