1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.executor;
21
22 import static org.junit.Assert.*;
23 import static org.junit.Assert.assertEquals;
24
25 import java.io.IOException;
26 import java.io.PrintWriter;
27 import java.io.StringWriter;
28 import java.util.concurrent.ThreadPoolExecutor;
29 import java.util.concurrent.atomic.AtomicBoolean;
30 import java.util.concurrent.atomic.AtomicInteger;
31
32 import org.apache.commons.logging.Log;
33 import org.apache.commons.logging.LogFactory;
34 import org.apache.hadoop.hbase.HBaseConfiguration;
35 import org.apache.hadoop.hbase.Server;
36 import org.apache.hadoop.hbase.executor.EventHandler.EventType;
37 import org.apache.hadoop.hbase.executor.ExecutorService.Executor;
38 import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorStatus;
39 import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorType;
40 import org.junit.Test;
41 import static org.mockito.Mockito.*;
42
43 public class TestExecutorService {
44 private static final Log LOG = LogFactory.getLog(TestExecutorService.class);
45
46 @Test
47 public void testExecutorService() throws Exception {
48 int maxThreads = 5;
49 int maxTries = 10;
50 int sleepInterval = 10;
51
52 Server mockedServer = mock(Server.class);
53 when(mockedServer.getConfiguration()).thenReturn(HBaseConfiguration.create());
54
55
56 ExecutorService executorService = new ExecutorService("unit_test");
57 executorService.startExecutorService(
58 ExecutorType.MASTER_SERVER_OPERATIONS, maxThreads);
59
60 Executor executor =
61 executorService.getExecutor(ExecutorType.MASTER_SERVER_OPERATIONS);
62 ThreadPoolExecutor pool = executor.threadPoolExecutor;
63
64
65 assertEquals(0, pool.getPoolSize());
66
67 AtomicBoolean lock = new AtomicBoolean(true);
68 AtomicInteger counter = new AtomicInteger(0);
69
70
71 for (int i = 0; i < maxThreads; i++) {
72 executorService.submit(
73 new TestEventHandler(mockedServer, EventType.M_SERVER_SHUTDOWN,
74 lock, counter));
75 }
76
77
78 int tries = 0;
79 while (counter.get() < maxThreads && tries < maxTries) {
80 LOG.info("Waiting for all event handlers to start...");
81 Thread.sleep(sleepInterval);
82 tries++;
83 }
84
85
86 assertEquals(maxThreads, counter.get());
87 assertEquals(maxThreads, pool.getPoolSize());
88
89 ExecutorStatus status = executor.getStatus();
90 assertTrue(status.queuedEvents.isEmpty());
91 assertEquals(5, status.running.size());
92 checkStatusDump(status);
93
94
95
96 synchronized (lock) {
97 lock.set(false);
98 lock.notifyAll();
99 }
100
101
102 while (counter.get() < (maxThreads * 2) && tries < maxTries) {
103 System.out.println("Waiting for all event handlers to finish...");
104 Thread.sleep(sleepInterval);
105 tries++;
106 }
107
108 assertEquals(maxThreads * 2, counter.get());
109 assertEquals(maxThreads, pool.getPoolSize());
110
111
112
113 for (int i = 0; i < (2 * maxThreads); i++) {
114 executorService.submit(
115 new TestEventHandler(mockedServer, EventType.M_SERVER_SHUTDOWN,
116 lock, counter));
117 }
118
119 synchronized (lock) {
120 lock.set(false);
121 lock.notifyAll();
122 }
123
124
125 Thread.sleep(executor.keepAliveTimeInMillis * 2);
126 assertEquals(maxThreads, pool.getPoolSize());
127
128 executorService.shutdown();
129
130 assertEquals(0, executorService.getAllExecutorStatuses().size());
131
132
133 executorService.submit(
134 new TestEventHandler(mockedServer, EventType.M_SERVER_SHUTDOWN,
135 lock, counter));
136 }
137
138 private void checkStatusDump(ExecutorStatus status) throws IOException {
139 StringWriter sw = new StringWriter();
140 status.dumpTo(sw, "");
141 String dump = sw.toString();
142 LOG.info("Got status dump:\n" + dump);
143
144 assertTrue(dump.contains("Waiting on java.util.concurrent.atomic.AtomicBoolean"));
145 }
146
147 public static class TestEventHandler extends EventHandler {
148 private AtomicBoolean lock;
149 private AtomicInteger counter;
150
151 public TestEventHandler(Server server, EventType eventType,
152 AtomicBoolean lock, AtomicInteger counter) {
153 super(server, eventType);
154 this.lock = lock;
155 this.counter = counter;
156 }
157
158 @Override
159 public void process() throws IOException {
160 int num = counter.incrementAndGet();
161 LOG.info("Running process #" + num + ", threadName=" +
162 Thread.currentThread().getName());
163 synchronized (lock) {
164 while (lock.get()) {
165 try {
166 lock.wait();
167 } catch (InterruptedException e) {
168
169 }
170 }
171 }
172 counter.incrementAndGet();
173 }
174 }
175 }