1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.executor;
20
21 import static org.junit.Assert.*;
22 import java.io.IOException;
23 import java.io.StringWriter;
24 import java.util.concurrent.CountDownLatch;
25 import java.util.concurrent.ThreadPoolExecutor;
26 import java.util.concurrent.atomic.AtomicBoolean;
27 import java.util.concurrent.atomic.AtomicInteger;
28
29 import org.apache.commons.logging.Log;
30 import org.apache.commons.logging.LogFactory;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.*;
33 import org.apache.hadoop.hbase.Waiter.Predicate;
34 import org.apache.hadoop.hbase.executor.ExecutorService.Executor;
35 import org.apache.hadoop.hbase.executor.ExecutorService.ExecutorStatus;
36 import org.apache.hadoop.hbase.testclassification.SmallTests;
37 import org.junit.Assert;
38 import org.junit.Test;
39 import org.junit.experimental.categories.Category;
40
41 import static org.mockito.Mockito.*;
42
43 @Category(SmallTests.class)
44 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="JLM_JSR166_UTILCONCURRENT_MONITORENTER",
45 justification="Use of an atomic type both as monitor and condition variable is intended")
46 public class TestExecutorService {
47 private static final Log LOG = LogFactory.getLog(TestExecutorService.class);
48
49 @Test
50 public void testExecutorService() throws Exception {
51 int maxThreads = 5;
52 int maxTries = 10;
53 int sleepInterval = 10;
54
55 Server mockedServer = mock(Server.class);
56 when(mockedServer.getConfiguration()).thenReturn(HBaseConfiguration.create());
57
58
59 ExecutorService executorService = new ExecutorService("unit_test");
60 executorService.startExecutorService(
61 ExecutorType.MASTER_SERVER_OPERATIONS, maxThreads);
62
63 Executor executor =
64 executorService.getExecutor(ExecutorType.MASTER_SERVER_OPERATIONS);
65 ThreadPoolExecutor pool = executor.threadPoolExecutor;
66
67
68 assertEquals(0, pool.getPoolSize());
69
70 AtomicBoolean lock = new AtomicBoolean(true);
71 AtomicInteger counter = new AtomicInteger(0);
72
73
74 for (int i = 0; i < maxThreads; i++) {
75 executorService.submit(
76 new TestEventHandler(mockedServer, EventType.M_SERVER_SHUTDOWN,
77 lock, counter));
78 }
79
80
81 int tries = 0;
82 while (counter.get() < maxThreads && tries < maxTries) {
83 LOG.info("Waiting for all event handlers to start...");
84 Thread.sleep(sleepInterval);
85 tries++;
86 }
87
88
89 assertEquals(maxThreads, counter.get());
90 assertEquals(maxThreads, pool.getPoolSize());
91
92 ExecutorStatus status = executor.getStatus();
93 assertTrue(status.queuedEvents.isEmpty());
94 assertEquals(5, status.running.size());
95 checkStatusDump(status);
96
97
98
99 synchronized (lock) {
100 lock.set(false);
101 lock.notifyAll();
102 }
103
104
105 while (counter.get() < (maxThreads * 2) && tries < maxTries) {
106 System.out.println("Waiting for all event handlers to finish...");
107 Thread.sleep(sleepInterval);
108 tries++;
109 }
110
111 assertEquals(maxThreads * 2, counter.get());
112 assertEquals(maxThreads, pool.getPoolSize());
113
114
115
116 for (int i = 0; i < (2 * maxThreads); i++) {
117 executorService.submit(
118 new TestEventHandler(mockedServer, EventType.M_SERVER_SHUTDOWN,
119 lock, counter));
120 }
121
122 synchronized (lock) {
123 lock.set(false);
124 lock.notifyAll();
125 }
126
127
128 Thread.sleep(ExecutorService.Executor.keepAliveTimeInMillis * 2);
129 assertEquals(maxThreads, pool.getPoolSize());
130
131 executorService.shutdown();
132
133 assertEquals(0, executorService.getAllExecutorStatuses().size());
134
135
136 executorService.submit(
137 new TestEventHandler(mockedServer, EventType.M_SERVER_SHUTDOWN,
138 lock, counter));
139 }
140
141 private void checkStatusDump(ExecutorStatus status) throws IOException {
142 StringWriter sw = new StringWriter();
143 status.dumpTo(sw, "");
144 String dump = sw.toString();
145 LOG.info("Got status dump:\n" + dump);
146
147 assertTrue(dump.contains("Waiting on java.util.concurrent.atomic.AtomicBoolean"));
148 }
149
150 public static class TestEventHandler extends EventHandler {
151 private AtomicBoolean lock;
152 private AtomicInteger counter;
153
154 public TestEventHandler(Server server, EventType eventType,
155 AtomicBoolean lock, AtomicInteger counter) {
156 super(server, eventType);
157 this.lock = lock;
158 this.counter = counter;
159 }
160
161 @Override
162 public void process() throws IOException {
163 int num = counter.incrementAndGet();
164 LOG.info("Running process #" + num + ", threadName=" +
165 Thread.currentThread().getName());
166 synchronized (lock) {
167 while (lock.get()) {
168 try {
169 lock.wait();
170 } catch (InterruptedException e) {
171
172 }
173 }
174 }
175 counter.incrementAndGet();
176 }
177 }
178
179 @Test
180 public void testAborting() throws Exception {
181 final Configuration conf = HBaseConfiguration.create();
182 final Server server = mock(Server.class);
183 when(server.getConfiguration()).thenReturn(conf);
184
185 ExecutorService executorService = new ExecutorService("unit_test");
186 executorService.startExecutorService(
187 ExecutorType.MASTER_SERVER_OPERATIONS, 1);
188
189
190 executorService.submit(new EventHandler(server, EventType.M_SERVER_SHUTDOWN) {
191 @Override
192 public void process() throws IOException {
193 throw new RuntimeException("Should cause abort");
194 }
195 });
196
197 Waiter.waitFor(conf, 30000, new Predicate<Exception>() {
198 @Override
199 public boolean evaluate() throws Exception {
200 try {
201 verify(server, times(1)).abort(anyString(), (Throwable) anyObject());
202 return true;
203 } catch (Throwable t) {
204 return false;
205 }
206 }
207 });
208
209 executorService.shutdown();
210 }
211
212 @Test
213 public void testSnapshotHandlers() throws Exception {
214 final Configuration conf = HBaseConfiguration.create();
215 final Server server = mock(Server.class);
216 when(server.getConfiguration()).thenReturn(conf);
217
218 final ExecutorService executorService = new ExecutorService("testSnapshotHandlers");
219 executorService.startExecutorService(ExecutorType.MASTER_SNAPSHOT_OPERATIONS, 1);
220
221 final CountDownLatch latch = new CountDownLatch(1);
222 executorService.submit(new EventHandler(server, EventType.C_M_SNAPSHOT_TABLE) {
223 @Override
224 public void process() throws IOException {
225 try {
226 latch.await();
227 } catch (InterruptedException e) {
228 Thread.currentThread().interrupt();
229 }
230 }
231 });
232
233 int activeCount = executorService.getExecutor(ExecutorType.MASTER_SNAPSHOT_OPERATIONS)
234 .getThreadPoolExecutor().getActiveCount();
235 Assert.assertEquals(activeCount, 1);
236 latch.countDown();
237 Waiter.waitFor(conf, 3000, new Predicate<Exception>() {
238 @Override
239 public boolean evaluate() throws Exception {
240 int count = executorService.getExecutor(ExecutorType.MASTER_SNAPSHOT_OPERATIONS)
241 .getThreadPoolExecutor().getActiveCount();
242 return count == 0;
243 }
244 });
245 }
246 }
247