1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.procedure2;
20
21 import java.io.IOException;
22 import java.io.InputStream;
23 import java.io.OutputStream;
24 import java.util.ArrayList;
25 import java.util.concurrent.atomic.AtomicBoolean;
26 import java.util.concurrent.atomic.AtomicLong;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.fs.FileSystem;
31 import org.apache.hadoop.fs.Path;
32 import org.apache.hadoop.hbase.HBaseCommonTestingUtility;
33 import org.apache.hadoop.hbase.procedure2.store.ProcedureStore;
34 import org.apache.hadoop.hbase.testclassification.SmallTests;
35
36 import org.junit.After;
37 import org.junit.Before;
38 import org.junit.Test;
39 import org.junit.experimental.categories.Category;
40
41 import static org.junit.Assert.assertEquals;
42 import static org.junit.Assert.assertTrue;
43
44 @Category(SmallTests.class)
45 public class TestYieldProcedures {
46 private static final Log LOG = LogFactory.getLog(TestYieldProcedures.class);
47
48 private static final int PROCEDURE_EXECUTOR_SLOTS = 1;
49 private static final Procedure NULL_PROC = null;
50
51 private ProcedureExecutor<TestProcEnv> procExecutor;
52 private TestRunQueue procRunnables;
53 private ProcedureStore procStore;
54
55 private HBaseCommonTestingUtility htu;
56 private FileSystem fs;
57 private Path testDir;
58 private Path logDir;
59
60 @Before
61 public void setUp() throws IOException {
62 htu = new HBaseCommonTestingUtility();
63 testDir = htu.getDataTestDir();
64 fs = testDir.getFileSystem(htu.getConfiguration());
65 assertTrue(testDir.depth() > 1);
66
67 logDir = new Path(testDir, "proc-logs");
68 procStore = ProcedureTestingUtility.createWalStore(htu.getConfiguration(), logDir);
69 procRunnables = new TestRunQueue();
70 procExecutor = new ProcedureExecutor(htu.getConfiguration(), new TestProcEnv(),
71 procStore, procRunnables);
72 procStore.start(PROCEDURE_EXECUTOR_SLOTS);
73 procExecutor.start(PROCEDURE_EXECUTOR_SLOTS, true);
74 }
75
76 @After
77 public void tearDown() throws IOException {
78 procExecutor.stop();
79 procStore.stop(false);
80 fs.delete(logDir, true);
81 }
82
83 @Test
84 public void testYieldEachExecutionStep() throws Exception {
85 final int NUM_STATES = 3;
86
87 TestStateMachineProcedure[] procs = new TestStateMachineProcedure[3];
88 for (int i = 0; i < procs.length; ++i) {
89 procs[i] = new TestStateMachineProcedure(true, false);
90 procExecutor.submitProcedure(procs[i]);
91 }
92 ProcedureTestingUtility.waitNoProcedureRunning(procExecutor);
93
94 for (int i = 0; i < procs.length; ++i) {
95 assertEquals(NUM_STATES * 2, procs[i].getExecutionInfo().size());
96
97
98 int index = 0;
99 for (int execStep = 0; execStep < NUM_STATES; ++execStep) {
100 TestStateMachineProcedure.ExecutionInfo info = procs[i].getExecutionInfo().get(index++);
101 assertEquals(false, info.isRollback());
102 assertEquals(execStep, info.getStep().ordinal());
103 }
104
105
106 for (int execStep = NUM_STATES - 1; execStep >= 0; --execStep) {
107 TestStateMachineProcedure.ExecutionInfo info = procs[i].getExecutionInfo().get(index++);
108 assertEquals(true, info.isRollback());
109 assertEquals(execStep, info.getStep().ordinal());
110 }
111 }
112
113
114 assertEquals(0, procRunnables.size());
115 assertEquals(0, procRunnables.addFrontCalls);
116 assertEquals(18, procRunnables.addBackCalls);
117 assertEquals(15, procRunnables.yieldCalls);
118 assertEquals(19, procRunnables.pollCalls);
119 assertEquals(3, procRunnables.completionCalls);
120 }
121
122 @Test
123 public void testYieldOnInterrupt() throws Exception {
124 final int NUM_STATES = 3;
125 int count = 0;
126
127 TestStateMachineProcedure proc = new TestStateMachineProcedure(true, true);
128 ProcedureTestingUtility.submitAndWait(procExecutor, proc);
129
130
131 assertEquals(NUM_STATES * 4, proc.getExecutionInfo().size());
132 for (int i = 0; i < NUM_STATES; ++i) {
133 TestStateMachineProcedure.ExecutionInfo info = proc.getExecutionInfo().get(count++);
134 assertEquals(false, info.isRollback());
135 assertEquals(i, info.getStep().ordinal());
136
137 info = proc.getExecutionInfo().get(count++);
138 assertEquals(false, info.isRollback());
139 assertEquals(i, info.getStep().ordinal());
140 }
141
142
143 for (int i = NUM_STATES - 1; i >= 0; --i) {
144 TestStateMachineProcedure.ExecutionInfo info = proc.getExecutionInfo().get(count++);
145 assertEquals(true, info.isRollback());
146 assertEquals(i, info.getStep().ordinal());
147
148 info = proc.getExecutionInfo().get(count++);
149 assertEquals(true, info.isRollback());
150 assertEquals(i, info.getStep().ordinal());
151 }
152
153
154 assertEquals(0, procRunnables.size());
155 assertEquals(0, procRunnables.addFrontCalls);
156 assertEquals(12, procRunnables.addBackCalls);
157 assertEquals(11, procRunnables.yieldCalls);
158 assertEquals(13, procRunnables.pollCalls);
159 assertEquals(1, procRunnables.completionCalls);
160 }
161
162 @Test
163 public void testYieldException() {
164 TestYieldProcedure proc = new TestYieldProcedure();
165 ProcedureTestingUtility.submitAndWait(procExecutor, proc);
166 assertEquals(6, proc.step);
167
168
169 assertEquals(0, procRunnables.size());
170 assertEquals(0, procRunnables.addFrontCalls);
171 assertEquals(6, procRunnables.addBackCalls);
172 assertEquals(5, procRunnables.yieldCalls);
173 assertEquals(7, procRunnables.pollCalls);
174 assertEquals(1, procRunnables.completionCalls);
175 }
176
177 private static class TestProcEnv {
178 public final AtomicLong timestamp = new AtomicLong(0);
179
180 public long nextTimestamp() {
181 return timestamp.incrementAndGet();
182 }
183 }
184
185 public static class TestStateMachineProcedure
186 extends StateMachineProcedure<TestProcEnv, TestStateMachineProcedure.State> {
187 enum State { STATE_1, STATE_2, STATE_3 }
188
189 public static class ExecutionInfo {
190 private final boolean rollback;
191 private final long timestamp;
192 private final State step;
193
194 public ExecutionInfo(long timestamp, State step, boolean isRollback) {
195 this.timestamp = timestamp;
196 this.step = step;
197 this.rollback = isRollback;
198 }
199
200 public State getStep() { return step; }
201 public long getTimestamp() { return timestamp; }
202 public boolean isRollback() { return rollback; }
203 }
204
205 private final ArrayList<ExecutionInfo> executionInfo = new ArrayList<ExecutionInfo>();
206 private final AtomicBoolean aborted = new AtomicBoolean(false);
207 private final boolean throwInterruptOnceOnEachStep;
208 private final boolean abortOnFinalStep;
209
210 public TestStateMachineProcedure() {
211 this(false, false);
212 }
213
214 public TestStateMachineProcedure(boolean abortOnFinalStep,
215 boolean throwInterruptOnceOnEachStep) {
216 this.abortOnFinalStep = abortOnFinalStep;
217 this.throwInterruptOnceOnEachStep = throwInterruptOnceOnEachStep;
218 }
219
220 public ArrayList<ExecutionInfo> getExecutionInfo() {
221 return executionInfo;
222 }
223
224 @Override
225 protected StateMachineProcedure.Flow executeFromState(TestProcEnv env, State state)
226 throws InterruptedException {
227 final long ts = env.nextTimestamp();
228 LOG.info(getProcId() + " execute step " + state + " ts=" + ts);
229 executionInfo.add(new ExecutionInfo(ts, state, false));
230 Thread.sleep(150);
231
232 if (throwInterruptOnceOnEachStep && ((executionInfo.size() - 1) % 2) == 0) {
233 LOG.debug("THROW INTERRUPT");
234 throw new InterruptedException("test interrupt");
235 }
236
237 switch (state) {
238 case STATE_1:
239 setNextState(State.STATE_2);
240 break;
241 case STATE_2:
242 setNextState(State.STATE_3);
243 break;
244 case STATE_3:
245 if (abortOnFinalStep) {
246 setFailure("test", new IOException("Requested abort on final step"));
247 }
248 return Flow.NO_MORE_STATE;
249 default:
250 throw new UnsupportedOperationException();
251 }
252 return Flow.HAS_MORE_STATE;
253 }
254
255 @Override
256 protected void rollbackState(TestProcEnv env, final State state)
257 throws InterruptedException {
258 final long ts = env.nextTimestamp();
259 LOG.debug(getProcId() + " rollback state " + state + " ts=" + ts);
260 executionInfo.add(new ExecutionInfo(ts, state, true));
261 Thread.sleep(150);
262
263 if (throwInterruptOnceOnEachStep && ((executionInfo.size() - 1) % 2) == 0) {
264 LOG.debug("THROW INTERRUPT");
265 throw new InterruptedException("test interrupt");
266 }
267
268 switch (state) {
269 case STATE_1:
270 break;
271 case STATE_2:
272 break;
273 case STATE_3:
274 break;
275 default:
276 throw new UnsupportedOperationException();
277 }
278 }
279
280 @Override
281 protected State getState(final int stateId) {
282 return State.values()[stateId];
283 }
284
285 @Override
286 protected int getStateId(final State state) {
287 return state.ordinal();
288 }
289
290 @Override
291 protected State getInitialState() {
292 return State.STATE_1;
293 }
294
295 @Override
296 protected boolean isYieldBeforeExecuteFromState(TestProcEnv env, State state) {
297 return true;
298 }
299
300 @Override
301 protected boolean abort(TestProcEnv env) {
302 aborted.set(true);
303 return true;
304 }
305 }
306
307 public static class TestYieldProcedure extends Procedure<TestProcEnv> {
308 private int step = 0;
309
310 public TestYieldProcedure() {
311 }
312
313 @Override
314 protected Procedure[] execute(final TestProcEnv env) throws ProcedureYieldException {
315 LOG.info("execute step " + step);
316 if (step++ < 5) {
317 throw new ProcedureYieldException();
318 }
319 return null;
320 }
321
322 @Override
323 protected void rollback(TestProcEnv env) {
324 }
325
326 @Override
327 protected boolean abort(TestProcEnv env) {
328 return false;
329 }
330
331 @Override
332 protected boolean isYieldAfterExecutionStep(final TestProcEnv env) {
333 return true;
334 }
335
336 @Override
337 protected void serializeStateData(final OutputStream stream) throws IOException {
338 }
339
340 @Override
341 protected void deserializeStateData(final InputStream stream) throws IOException {
342 }
343 }
344
345 private static class TestRunQueue extends ProcedureSimpleRunQueue {
346 private int completionCalls;
347 private int addFrontCalls;
348 private int addBackCalls;
349 private int yieldCalls;
350 private int pollCalls;
351
352 public TestRunQueue() {}
353
354 @Override
355 public void addFront(final Procedure proc) {
356 addFrontCalls++;
357 super.addFront(proc);
358 }
359
360 @Override
361 public void addBack(final Procedure proc) {
362 addBackCalls++;
363 super.addBack(proc);
364 }
365
366 @Override
367 public void yield(final Procedure proc) {
368 yieldCalls++;
369 super.yield(proc);
370 }
371
372 @Override
373 public Procedure poll() {
374 pollCalls++;
375 return super.poll();
376 }
377
378 @Override
379 public void completionCleanup(Procedure proc) {
380 completionCalls++;
381 }
382 }
383 }