1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.procedure;
19
20 import static org.junit.Assert.assertEquals;
21 import static org.mockito.Matchers.any;
22 import static org.mockito.Matchers.anyListOf;
23 import static org.mockito.Matchers.eq;
24 import static org.mockito.Mockito.never;
25 import static org.mockito.Mockito.spy;
26 import static org.mockito.Mockito.when;
27
28 import java.io.IOException;
29 import java.util.ArrayList;
30 import java.util.Arrays;
31 import java.util.List;
32 import java.util.concurrent.CountDownLatch;
33 import java.util.concurrent.ThreadPoolExecutor;
34 import java.util.concurrent.atomic.AtomicInteger;
35
36 import org.apache.commons.logging.Log;
37 import org.apache.commons.logging.LogFactory;
38 import org.apache.hadoop.hbase.Abortable;
39 import org.apache.hadoop.hbase.HBaseTestingUtility;
40 import org.apache.hadoop.hbase.MediumTests;
41 import org.apache.hadoop.hbase.errorhandling.ForeignException;
42 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
43 import org.apache.hadoop.hbase.errorhandling.TimeoutException;
44 import org.apache.hadoop.hbase.procedure.Subprocedure.SubprocedureImpl;
45 import org.apache.hadoop.hbase.util.Pair;
46 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
47 import org.junit.AfterClass;
48 import org.junit.BeforeClass;
49 import org.junit.Test;
50 import org.junit.experimental.categories.Category;
51 import org.mockito.Mockito;
52 import org.mockito.internal.matchers.ArrayEquals;
53 import org.mockito.invocation.InvocationOnMock;
54 import org.mockito.stubbing.Answer;
55 import org.mockito.verification.VerificationMode;
56
57 import com.google.common.collect.Lists;
58
59
60
61
62 @Category(MediumTests.class)
63 public class TestZKProcedure {
64
65 private static final Log LOG = LogFactory.getLog(TestZKProcedure.class);
66 private static HBaseTestingUtility UTIL = new HBaseTestingUtility();
67 private static final String COORDINATOR_NODE_NAME = "coordinator";
68 private static final long KEEP_ALIVE = 100;
69 private static final int POOL_SIZE = 1;
70 private static final long TIMEOUT = 10000;
71 private static final long WAKE_FREQUENCY = 500;
72 private static final String opName = "op";
73 private static final byte[] data = new byte[] { 1, 2 };
74 private static final VerificationMode once = Mockito.times(1);
75
76 @BeforeClass
77 public static void setupTest() throws Exception {
78 UTIL.startMiniZKCluster();
79 }
80
81 @AfterClass
82 public static void cleanupTest() throws Exception {
83 UTIL.shutdownMiniZKCluster();
84 }
85
86 private static ZooKeeperWatcher newZooKeeperWatcher() throws IOException {
87 return new ZooKeeperWatcher(UTIL.getConfiguration(), "testing utility", new Abortable() {
88 @Override
89 public void abort(String why, Throwable e) {
90 throw new RuntimeException(
91 "Unexpected abort in distributed three phase commit test:" + why, e);
92 }
93
94 @Override
95 public boolean isAborted() {
96 return false;
97 }
98 });
99 }
100
101 @Test
102 public void testEmptyMemberSet() throws Exception {
103 runCommit();
104 }
105
106 @Test
107 public void testSingleMember() throws Exception {
108 runCommit("one");
109 }
110
111 @Test
112 public void testMultipleMembers() throws Exception {
113 runCommit("one", "two", "three", "four" );
114 }
115
116 private void runCommit(String... members) throws Exception {
117
118 if (members == null) {
119 members = new String[0];
120 }
121 List<String> expected = Arrays.asList(members);
122
123
124 ZooKeeperWatcher coordZkw = newZooKeeperWatcher();
125 String opDescription = "coordination test - " + members.length + " cohort members";
126
127
128 ZKProcedureCoordinatorRpcs coordinatorComms = new ZKProcedureCoordinatorRpcs(
129 coordZkw, opDescription, COORDINATOR_NODE_NAME);
130 ThreadPoolExecutor pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, KEEP_ALIVE, POOL_SIZE, WAKE_FREQUENCY);
131 ProcedureCoordinator coordinator = new ProcedureCoordinator(coordinatorComms, pool) {
132 @Override
133 public Procedure createProcedure(ForeignExceptionDispatcher fed, String procName, byte[] procArgs,
134 List<String> expectedMembers) {
135 return Mockito.spy(super.createProcedure(fed, procName, procArgs, expectedMembers));
136 }
137 };
138
139
140
141 SubprocedureFactory subprocFactory = Mockito.mock(SubprocedureFactory.class);
142 List<Pair<ProcedureMember, ZKProcedureMemberRpcs>> procMembers = new ArrayList<Pair<ProcedureMember, ZKProcedureMemberRpcs>>(
143 members.length);
144
145 for (String member : members) {
146 ZooKeeperWatcher watcher = newZooKeeperWatcher();
147 ZKProcedureMemberRpcs comms = new ZKProcedureMemberRpcs(watcher, opDescription, member);
148 ThreadPoolExecutor pool2 = ProcedureMember.defaultPool(WAKE_FREQUENCY, KEEP_ALIVE, 1, member);
149 ProcedureMember procMember = new ProcedureMember(comms, pool2, subprocFactory);
150 procMembers.add(new Pair<ProcedureMember, ZKProcedureMemberRpcs>(procMember, comms));
151 comms.start(procMember);
152 }
153
154
155 final List<Subprocedure> subprocs = new ArrayList<Subprocedure>();
156 for (int i = 0; i < procMembers.size(); i++) {
157 ForeignExceptionDispatcher cohortMonitor = new ForeignExceptionDispatcher();
158 Subprocedure commit = Mockito
159 .spy(new SubprocedureImpl(procMembers.get(i).getFirst(), opName, cohortMonitor,
160 WAKE_FREQUENCY, TIMEOUT));
161 subprocs.add(commit);
162 }
163
164
165 final AtomicInteger i = new AtomicInteger(0);
166 Mockito.when(subprocFactory.buildSubprocedure(Mockito.eq(opName),
167 (byte[]) Mockito.argThat(new ArrayEquals(data)))).thenAnswer(
168 new Answer<Subprocedure>() {
169 @Override
170 public Subprocedure answer(InvocationOnMock invocation) throws Throwable {
171 int index = i.getAndIncrement();
172 LOG.debug("Task size:" + subprocs.size() + ", getting:" + index);
173 Subprocedure commit = subprocs.get(index);
174 return commit;
175 }
176 });
177
178
179
180
181
182
183 Procedure task = coordinator.startProcedure(new ForeignExceptionDispatcher(), opName, data, expected);
184
185
186
187
188 waitAndVerifyProc(task, once, once, never(), once, false);
189 verifyCohortSuccessful(expected, subprocFactory, subprocs, once, once, never(), once, false);
190
191
192 closeAll(coordinator, coordinatorComms, procMembers);
193 }
194
195
196
197
198
199 @Test
200 public void testMultiCohortWithMemberTimeoutDuringPrepare() throws Exception {
201 String opDescription = "error injection coordination";
202 String[] cohortMembers = new String[] { "one", "two", "three" };
203 List<String> expected = Lists.newArrayList(cohortMembers);
204
205 final int memberErrorIndex = 2;
206 final CountDownLatch coordinatorReceivedErrorLatch = new CountDownLatch(1);
207
208
209 ZooKeeperWatcher coordinatorWatcher = newZooKeeperWatcher();
210 ZKProcedureCoordinatorRpcs coordinatorController = new ZKProcedureCoordinatorRpcs(
211 coordinatorWatcher, opDescription, COORDINATOR_NODE_NAME);
212 ThreadPoolExecutor pool = ProcedureCoordinator.defaultPool(COORDINATOR_NODE_NAME, KEEP_ALIVE, POOL_SIZE, WAKE_FREQUENCY);
213 ProcedureCoordinator coordinator = spy(new ProcedureCoordinator(coordinatorController, pool));
214
215
216 SubprocedureFactory subprocFactory = Mockito.mock(SubprocedureFactory.class);
217 List<Pair<ProcedureMember, ZKProcedureMemberRpcs>> members = new ArrayList<Pair<ProcedureMember, ZKProcedureMemberRpcs>>(
218 expected.size());
219 for (String member : expected) {
220 ZooKeeperWatcher watcher = newZooKeeperWatcher();
221 ZKProcedureMemberRpcs controller = new ZKProcedureMemberRpcs(watcher, opDescription, member);
222 ThreadPoolExecutor pool2 = ProcedureMember.defaultPool(WAKE_FREQUENCY, KEEP_ALIVE, 1, member);
223 ProcedureMember mem = new ProcedureMember(controller, pool2, subprocFactory);
224 members.add(new Pair<ProcedureMember, ZKProcedureMemberRpcs>(mem, controller));
225 controller.start(mem);
226 }
227
228
229 final List<Subprocedure> cohortTasks = new ArrayList<Subprocedure>();
230 final int[] elem = new int[1];
231 for (int i = 0; i < members.size(); i++) {
232 ForeignExceptionDispatcher cohortMonitor = new ForeignExceptionDispatcher();
233 ProcedureMember comms = members.get(i).getFirst();
234 Subprocedure commit = Mockito
235 .spy(new SubprocedureImpl(comms, opName, cohortMonitor, WAKE_FREQUENCY, TIMEOUT));
236
237 Mockito.doAnswer(new Answer<Void>() {
238 @Override
239 public Void answer(InvocationOnMock invocation) throws Throwable {
240 int index = elem[0];
241 if (index == memberErrorIndex) {
242 LOG.debug("Sending error to coordinator");
243 ForeignException remoteCause = new ForeignException("TIMER",
244 new TimeoutException("subprocTimeout" , 1, 2, 0));
245 Subprocedure r = ((Subprocedure) invocation.getMock());
246 LOG.error("Remote commit failure, not propagating error:" + remoteCause);
247 r.monitor.receive(remoteCause);
248
249
250 try {
251 Procedure.waitForLatch(coordinatorReceivedErrorLatch, new ForeignExceptionDispatcher(),
252 WAKE_FREQUENCY, "coordinator received error");
253 } catch (InterruptedException e) {
254 LOG.debug("Wait for latch interrupted, done:" + (coordinatorReceivedErrorLatch.getCount() == 0));
255
256 Thread.currentThread().interrupt();
257 }
258 }
259 elem[0] = ++index;
260 return null;
261 }
262 }).when(commit).acquireBarrier();
263 cohortTasks.add(commit);
264 }
265
266
267 final int[] i = new int[] { 0 };
268 Mockito.when(
269 subprocFactory.buildSubprocedure(Mockito.eq(opName),
270 (byte[]) Mockito.argThat(new ArrayEquals(data)))).thenAnswer(
271 new Answer<Subprocedure>() {
272 @Override
273 public Subprocedure answer(InvocationOnMock invocation) throws Throwable {
274 int index = i[0];
275 Subprocedure commit = cohortTasks.get(index);
276 index++;
277 i[0] = index;
278 return commit;
279 }
280 });
281
282
283 ForeignExceptionDispatcher coordinatorTaskErrorMonitor = Mockito
284 .spy(new ForeignExceptionDispatcher());
285 Procedure coordinatorTask = Mockito.spy(new Procedure(coordinator,
286 coordinatorTaskErrorMonitor, WAKE_FREQUENCY, TIMEOUT,
287 opName, data, expected));
288 when(coordinator.createProcedure(any(ForeignExceptionDispatcher.class), eq(opName), eq(data), anyListOf(String.class)))
289 .thenReturn(coordinatorTask);
290
291 Mockito.doAnswer(new Answer<Void>() {
292 @Override
293 public Void answer(InvocationOnMock invocation) throws Throwable {
294
295 invocation.callRealMethod();
296
297 coordinatorReceivedErrorLatch.countDown();
298 return null;
299 }
300 }).when(coordinatorTask).receive(Mockito.any(ForeignException.class));
301
302
303
304
305
306 Procedure task = coordinator.startProcedure(coordinatorTaskErrorMonitor, opName, data, expected);
307 assertEquals("Didn't mock coordinator task", coordinatorTask, task);
308
309
310 try {
311 task.waitForCompleted();
312 } catch (ForeignException fe) {
313
314 }
315
316
317
318
319 waitAndVerifyProc(coordinatorTask, once, never(), once, once, true);
320 verifyCohortSuccessful(expected, subprocFactory, cohortTasks, once, never(), once,
321 once, true);
322
323
324 closeAll(coordinator, coordinatorController, members);
325 }
326
327
328
329
330
331
332 private void waitAndVerifyProc(Procedure proc, VerificationMode prepare,
333 VerificationMode commit, VerificationMode cleanup, VerificationMode finish, boolean opHasError)
334 throws Exception {
335 boolean caughtError = false;
336 try {
337 proc.waitForCompleted();
338 } catch (ForeignException fe) {
339 caughtError = true;
340 }
341
342 Mockito.verify(proc, prepare).sendGlobalBarrierStart();
343 Mockito.verify(proc, commit).sendGlobalBarrierReached();
344 Mockito.verify(proc, finish).sendGlobalBarrierComplete();
345 assertEquals("Operation error state was unexpected", opHasError, proc.getErrorMonitor()
346 .hasException());
347 assertEquals("Operation error state was unexpected", opHasError, caughtError);
348
349 }
350
351
352
353
354
355
356 private void waitAndVerifySubproc(Subprocedure op, VerificationMode prepare,
357 VerificationMode commit, VerificationMode cleanup, VerificationMode finish, boolean opHasError)
358 throws Exception {
359 boolean caughtError = false;
360 try {
361 op.waitForLocallyCompleted();
362 } catch (ForeignException fe) {
363 caughtError = true;
364 }
365
366 Mockito.verify(op, prepare).acquireBarrier();
367 Mockito.verify(op, commit).insideBarrier();
368
369
370 assertEquals("Operation error state was unexpected", opHasError, op.getErrorCheckable()
371 .hasException());
372 assertEquals("Operation error state was unexpected", opHasError, caughtError);
373
374 }
375
376 private void verifyCohortSuccessful(List<String> cohortNames,
377 SubprocedureFactory subprocFactory, Iterable<Subprocedure> cohortTasks,
378 VerificationMode prepare, VerificationMode commit, VerificationMode cleanup,
379 VerificationMode finish, boolean opHasError) throws Exception {
380
381
382 Mockito.verify(subprocFactory, Mockito.times(cohortNames.size())).buildSubprocedure(
383 Mockito.eq(opName), (byte[]) Mockito.argThat(new ArrayEquals(data)));
384
385 int j = 0;
386 for (Subprocedure op : cohortTasks) {
387 LOG.debug("Checking mock:" + (j++));
388 waitAndVerifySubproc(op, prepare, commit, cleanup, finish, opHasError);
389 }
390 }
391
392 private void closeAll(
393 ProcedureCoordinator coordinator,
394 ZKProcedureCoordinatorRpcs coordinatorController,
395 List<Pair<ProcedureMember, ZKProcedureMemberRpcs>> cohort)
396 throws IOException {
397
398 for (Pair<ProcedureMember, ZKProcedureMemberRpcs> member : cohort) {
399 member.getFirst().close();
400 member.getSecond().close();
401 }
402 coordinator.close();
403 coordinatorController.close();
404 }
405 }