1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.coprocessor;
20
21 import static org.junit.Assert.assertEquals;
22 import static org.junit.Assert.assertTrue;
23
24 import java.io.IOException;
25 import java.util.ArrayList;
26 import java.util.Arrays;
27 import java.util.Collection;
28 import java.util.Collections;
29 import java.util.HashSet;
30 import java.util.List;
31 import java.util.Set;
32 import java.util.concurrent.CountDownLatch;
33 import java.util.concurrent.atomic.AtomicInteger;
34 import java.util.concurrent.atomic.AtomicLong;
35
36 import org.apache.hadoop.hbase.TableName;
37 import org.apache.hadoop.hbase.client.Table;
38 import org.apache.hadoop.hbase.util.ByteStringer;
39 import org.apache.hadoop.conf.Configuration;
40 import org.apache.hadoop.hbase.Cell;
41 import org.apache.hadoop.hbase.CellUtil;
42 import org.apache.hadoop.hbase.HBaseTestingUtility;
43 import org.apache.hadoop.hbase.HConstants;
44 import org.apache.hadoop.hbase.KeyValue;
45 import org.apache.hadoop.hbase.testclassification.MediumTests;
46 import org.apache.hadoop.hbase.client.Delete;
47 import org.apache.hadoop.hbase.client.Get;
48 import org.apache.hadoop.hbase.client.IsolationLevel;
49 import org.apache.hadoop.hbase.client.Mutation;
50 import org.apache.hadoop.hbase.client.Put;
51 import org.apache.hadoop.hbase.client.Scan;
52 import org.apache.hadoop.hbase.client.coprocessor.RowProcessorClient;
53 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos;
54 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorRequest;
55 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.FriendsOfFriendsProcessorResponse;
56 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorRequest;
57 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorResponse;
58 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.RowSwapProcessorRequest;
59 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.RowSwapProcessorResponse;
60 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.TimeoutProcessorRequest;
61 import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.TimeoutProcessorResponse;
62 import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
63 import org.apache.hadoop.hbase.ipc.RpcScheduler;
64 import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest;
65 import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse;
66 import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.RowProcessorService;
67 import org.apache.hadoop.hbase.regionserver.BaseRowProcessor;
68 import org.apache.hadoop.hbase.regionserver.HRegion;
69 import org.apache.hadoop.hbase.regionserver.InternalScanner;
70 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
71 import org.apache.hadoop.hbase.util.Bytes;
72 import org.junit.AfterClass;
73 import org.junit.BeforeClass;
74 import org.junit.Test;
75 import org.junit.experimental.categories.Category;
76
77 import com.google.protobuf.Message;
78 import org.apache.commons.logging.Log;
79 import org.apache.commons.logging.LogFactory;
80
81
82
83
84
85 @Category(MediumTests.class)
86 public class TestRowProcessorEndpoint {
87
88 private static final Log LOG = LogFactory.getLog(TestRowProcessorEndpoint.class);
89
90 private static final TableName TABLE = TableName.valueOf("testtable");
91 private final static byte[] ROW = Bytes.toBytes("testrow");
92 private final static byte[] ROW2 = Bytes.toBytes("testrow2");
93 private final static byte[] FAM = Bytes.toBytes("friendlist");
94
95
96 private final static byte[] A = Bytes.toBytes("a");
97 private final static byte[] B = Bytes.toBytes("b");
98 private final static byte[] C = Bytes.toBytes("c");
99 private final static byte[] D = Bytes.toBytes("d");
100 private final static byte[] E = Bytes.toBytes("e");
101 private final static byte[] F = Bytes.toBytes("f");
102 private final static byte[] G = Bytes.toBytes("g");
103 private final static byte[] COUNTER = Bytes.toBytes("counter");
104 private final static AtomicLong myTimer = new AtomicLong(0);
105 private final AtomicInteger failures = new AtomicInteger(0);
106
107 private static HBaseTestingUtility util = new HBaseTestingUtility();
108 private static volatile int expectedCounter = 0;
109 private static int rowSize, row2Size;
110
111 private volatile static Table table = null;
112 private volatile static boolean swapped = false;
113 private volatile CountDownLatch startSignal;
114 private volatile CountDownLatch doneSignal;
115
116 @BeforeClass
117 public static void setupBeforeClass() throws Exception {
118 Configuration conf = util.getConfiguration();
119 conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
120 RowProcessorEndpoint.class.getName());
121 conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
122 conf.setLong("hbase.hregion.row.processor.timeout", 1000L);
123 conf.setLong(RpcScheduler.IPC_SERVER_MAX_CALLQUEUE_LENGTH, 2048);
124 util.startMiniCluster();
125 }
126
127 @AfterClass
128 public static void tearDownAfterClass() throws Exception {
129 util.shutdownMiniCluster();
130 }
131
132 public void prepareTestData() throws Exception {
133 try {
134 util.getHBaseAdmin().disableTable(TABLE);
135 util.getHBaseAdmin().deleteTable(TABLE);
136 } catch (Exception e) {
137
138 }
139 table = util.createTable(TABLE, FAM);
140 {
141 Put put = new Put(ROW);
142 put.add(FAM, A, Bytes.add(B, C));
143 put.add(FAM, B, Bytes.add(D, E, F));
144 put.add(FAM, C, G);
145 table.put(put);
146 rowSize = put.size();
147 }
148 Put put = new Put(ROW2);
149 put.add(FAM, D, E);
150 put.add(FAM, F, G);
151 table.put(put);
152 row2Size = put.size();
153 }
154
155 @Test
156 public void testDoubleScan() throws Throwable {
157 prepareTestData();
158
159 CoprocessorRpcChannel channel = table.coprocessorService(ROW);
160 RowProcessorEndpoint.FriendsOfFriendsProcessor processor =
161 new RowProcessorEndpoint.FriendsOfFriendsProcessor(ROW, A);
162 RowProcessorService.BlockingInterface service =
163 RowProcessorService.newBlockingStub(channel);
164 ProcessRequest request = RowProcessorClient.getRowProcessorPB(processor);
165 ProcessResponse protoResult = service.process(null, request);
166 FriendsOfFriendsProcessorResponse response =
167 FriendsOfFriendsProcessorResponse.parseFrom(protoResult.getRowProcessorResult());
168 Set<String> result = new HashSet<String>();
169 result.addAll(response.getResultList());
170 Set<String> expected =
171 new HashSet<String>(Arrays.asList(new String[]{"d", "e", "f", "g"}));
172 Get get = new Get(ROW);
173 LOG.debug("row keyvalues:" + stringifyKvs(table.get(get).listCells()));
174 assertEquals(expected, result);
175 }
176
177 @Test
178 public void testReadModifyWrite() throws Throwable {
179 prepareTestData();
180 failures.set(0);
181 int numThreads = 100;
182 concurrentExec(new IncrementRunner(), numThreads);
183 Get get = new Get(ROW);
184 LOG.debug("row keyvalues:" + stringifyKvs(table.get(get).listCells()));
185 int finalCounter = incrementCounter(table);
186 int failureNumber = failures.get();
187 if (failureNumber > 0) {
188 LOG.debug("We failed " + failureNumber + " times during test");
189 }
190 assertEquals(numThreads + 1 - failureNumber, finalCounter);
191 }
192
193 class IncrementRunner implements Runnable {
194 @Override
195 public void run() {
196 try {
197 incrementCounter(table);
198 } catch (Throwable e) {
199 failures.incrementAndGet();
200 e.printStackTrace();
201 }
202 }
203 }
204
205 private int incrementCounter(Table table) throws Throwable {
206 CoprocessorRpcChannel channel = table.coprocessorService(ROW);
207 RowProcessorEndpoint.IncrementCounterProcessor processor =
208 new RowProcessorEndpoint.IncrementCounterProcessor(ROW);
209 RowProcessorService.BlockingInterface service =
210 RowProcessorService.newBlockingStub(channel);
211 ProcessRequest request = RowProcessorClient.getRowProcessorPB(processor);
212 ProcessResponse protoResult = service.process(null, request);
213 IncCounterProcessorResponse response = IncCounterProcessorResponse
214 .parseFrom(protoResult.getRowProcessorResult());
215 Integer result = response.getResponse();
216 return result;
217 }
218
219 private void concurrentExec(
220 final Runnable task, final int numThreads) throws Throwable {
221 startSignal = new CountDownLatch(numThreads);
222 doneSignal = new CountDownLatch(numThreads);
223 for (int i = 0; i < numThreads; ++i) {
224 new Thread(new Runnable() {
225 @Override
226 public void run() {
227 try {
228 startSignal.countDown();
229 startSignal.await();
230 task.run();
231 } catch (Throwable e) {
232 failures.incrementAndGet();
233 e.printStackTrace();
234 }
235 doneSignal.countDown();
236 }
237 }).start();
238 }
239 doneSignal.await();
240 }
241
242 @Test
243 public void testMultipleRows() throws Throwable {
244 prepareTestData();
245 failures.set(0);
246 int numThreads = 100;
247 concurrentExec(new SwapRowsRunner(), numThreads);
248 LOG.debug("row keyvalues:" +
249 stringifyKvs(table.get(new Get(ROW)).listCells()));
250 LOG.debug("row2 keyvalues:" +
251 stringifyKvs(table.get(new Get(ROW2)).listCells()));
252 int failureNumber = failures.get();
253 if (failureNumber > 0) {
254 LOG.debug("We failed " + failureNumber + " times during test");
255 }
256 if (!swapped) {
257 assertEquals(rowSize, table.get(new Get(ROW)).listCells().size());
258 assertEquals(row2Size, table.get(new Get(ROW2)).listCells().size());
259 } else {
260 assertEquals(rowSize, table.get(new Get(ROW2)).listCells().size());
261 assertEquals(row2Size, table.get(new Get(ROW)).listCells().size());
262 }
263 }
264
265 class SwapRowsRunner implements Runnable {
266 @Override
267 public void run() {
268 try {
269 swapRows(table);
270 } catch (Throwable e) {
271 failures.incrementAndGet();
272 e.printStackTrace();
273 }
274 }
275 }
276
277 private void swapRows(Table table) throws Throwable {
278 CoprocessorRpcChannel channel = table.coprocessorService(ROW);
279 RowProcessorEndpoint.RowSwapProcessor processor =
280 new RowProcessorEndpoint.RowSwapProcessor(ROW, ROW2);
281 RowProcessorService.BlockingInterface service =
282 RowProcessorService.newBlockingStub(channel);
283 ProcessRequest request = RowProcessorClient.getRowProcessorPB(processor);
284 service.process(null, request);
285 }
286
287 @Test
288 public void testTimeout() throws Throwable {
289 prepareTestData();
290 CoprocessorRpcChannel channel = table.coprocessorService(ROW);
291 RowProcessorEndpoint.TimeoutProcessor processor =
292 new RowProcessorEndpoint.TimeoutProcessor(ROW);
293 RowProcessorService.BlockingInterface service =
294 RowProcessorService.newBlockingStub(channel);
295 ProcessRequest request = RowProcessorClient.getRowProcessorPB(processor);
296 boolean exceptionCaught = false;
297 try {
298 service.process(null, request);
299 } catch (Exception e) {
300 exceptionCaught = true;
301 }
302 assertTrue(exceptionCaught);
303 }
304
305
306
307
308
309
310
311
312 public static class RowProcessorEndpoint<S extends Message,T extends Message>
313 extends BaseRowProcessorEndpoint<S,T> implements CoprocessorService {
314 public static class IncrementCounterProcessor extends
315 BaseRowProcessor<IncrementCounterProcessorTestProtos.IncCounterProcessorRequest,
316 IncrementCounterProcessorTestProtos.IncCounterProcessorResponse> {
317 int counter = 0;
318 byte[] row = new byte[0];
319
320
321
322
323 IncrementCounterProcessor() {
324 }
325
326 IncrementCounterProcessor(byte[] row) {
327 this.row = row;
328 }
329
330 @Override
331 public Collection<byte[]> getRowsToLock() {
332 return Collections.singleton(row);
333 }
334
335 @Override
336 public IncCounterProcessorResponse getResult() {
337 IncCounterProcessorResponse.Builder i = IncCounterProcessorResponse.newBuilder();
338 i.setResponse(counter);
339 return i.build();
340 }
341
342 @Override
343 public boolean readOnly() {
344 return false;
345 }
346
347 @Override
348 public void process(long now, HRegion region,
349 List<Mutation> mutations, WALEdit walEdit) throws IOException {
350
351 List<Cell> kvs = new ArrayList<Cell>();
352 Scan scan = new Scan(row, row);
353 scan.addColumn(FAM, COUNTER);
354 doScan(region, scan, kvs);
355 counter = kvs.size() == 0 ? 0 :
356 Bytes.toInt(CellUtil.cloneValue(kvs.iterator().next()));
357
358
359 assertEquals(expectedCounter, counter);
360
361
362 counter += 1;
363 expectedCounter += 1;
364
365
366 Put p = new Put(row);
367 KeyValue kv =
368 new KeyValue(row, FAM, COUNTER, now, Bytes.toBytes(counter));
369 p.add(kv);
370 mutations.add(p);
371 walEdit.add(kv);
372
373
374 KeyValue metaKv = new KeyValue(
375 row, WALEdit.METAFAMILY,
376 Bytes.toBytes("I just increment counter"),
377 Bytes.toBytes(counter));
378 walEdit.add(metaKv);
379 }
380
381 @Override
382 public IncCounterProcessorRequest getRequestData() throws IOException {
383 IncCounterProcessorRequest.Builder builder = IncCounterProcessorRequest.newBuilder();
384 builder.setCounter(counter);
385 builder.setRow(ByteStringer.wrap(row));
386 return builder.build();
387 }
388
389 @Override
390 public void initialize(IncCounterProcessorRequest msg) {
391 this.row = msg.getRow().toByteArray();
392 this.counter = msg.getCounter();
393 }
394 }
395
396 public static class FriendsOfFriendsProcessor extends
397 BaseRowProcessor<FriendsOfFriendsProcessorRequest, FriendsOfFriendsProcessorResponse> {
398 byte[] row = null;
399 byte[] person = null;
400 final Set<String> result = new HashSet<String>();
401
402
403
404
405 FriendsOfFriendsProcessor() {
406 }
407
408 FriendsOfFriendsProcessor(byte[] row, byte[] person) {
409 this.row = row;
410 this.person = person;
411 }
412
413 @Override
414 public Collection<byte[]> getRowsToLock() {
415 return Collections.singleton(row);
416 }
417
418 @Override
419 public FriendsOfFriendsProcessorResponse getResult() {
420 FriendsOfFriendsProcessorResponse.Builder builder =
421 FriendsOfFriendsProcessorResponse.newBuilder();
422 builder.addAllResult(result);
423 return builder.build();
424 }
425
426 @Override
427 public boolean readOnly() {
428 return true;
429 }
430
431 @Override
432 public void process(long now, HRegion region,
433 List<Mutation> mutations, WALEdit walEdit) throws IOException {
434 List<Cell> kvs = new ArrayList<Cell>();
435 {
436 Scan scan = new Scan(row, row);
437 scan.addColumn(FAM, person);
438 doScan(region, scan, kvs);
439 }
440
441
442 Scan scan = new Scan(row, row);
443 for (Cell kv : kvs) {
444 byte[] friends = CellUtil.cloneValue(kv);
445 for (byte f : friends) {
446 scan.addColumn(FAM, new byte[]{f});
447 }
448 }
449 doScan(region, scan, kvs);
450
451
452 result.clear();
453 for (Cell kv : kvs) {
454 for (byte b : CellUtil.cloneValue(kv)) {
455 result.add((char)b + "");
456 }
457 }
458 }
459
460 @Override
461 public FriendsOfFriendsProcessorRequest getRequestData() throws IOException {
462 FriendsOfFriendsProcessorRequest.Builder builder =
463 FriendsOfFriendsProcessorRequest.newBuilder();
464 builder.setPerson(ByteStringer.wrap(person));
465 builder.setRow(ByteStringer.wrap(row));
466 builder.addAllResult(result);
467 FriendsOfFriendsProcessorRequest f = builder.build();
468 return f;
469 }
470
471 @Override
472 public void initialize(FriendsOfFriendsProcessorRequest request)
473 throws IOException {
474 this.person = request.getPerson().toByteArray();
475 this.row = request.getRow().toByteArray();
476 result.clear();
477 result.addAll(request.getResultList());
478 }
479 }
480
481 public static class RowSwapProcessor extends
482 BaseRowProcessor<RowSwapProcessorRequest, RowSwapProcessorResponse> {
483 byte[] row1 = new byte[0];
484 byte[] row2 = new byte[0];
485
486
487
488
489 RowSwapProcessor() {
490 }
491
492 RowSwapProcessor(byte[] row1, byte[] row2) {
493 this.row1 = row1;
494 this.row2 = row2;
495 }
496
497 @Override
498 public Collection<byte[]> getRowsToLock() {
499 List<byte[]> rows = new ArrayList<byte[]>();
500 rows.add(row1);
501 rows.add(row2);
502 return rows;
503 }
504
505 @Override
506 public boolean readOnly() {
507 return false;
508 }
509
510 @Override
511 public RowSwapProcessorResponse getResult() {
512 return RowSwapProcessorResponse.getDefaultInstance();
513 }
514
515 @Override
516 public void process(long now, HRegion region,
517 List<Mutation> mutations, WALEdit walEdit) throws IOException {
518
519
520
521 now = myTimer.getAndIncrement();
522
523
524 List<Cell> kvs1 = new ArrayList<Cell>();
525 List<Cell> kvs2 = new ArrayList<Cell>();
526 doScan(region, new Scan(row1, row1), kvs1);
527 doScan(region, new Scan(row2, row2), kvs2);
528
529
530 if (swapped) {
531 assertEquals(rowSize, kvs2.size());
532 assertEquals(row2Size, kvs1.size());
533 } else {
534 assertEquals(rowSize, kvs1.size());
535 assertEquals(row2Size, kvs2.size());
536 }
537 swapped = !swapped;
538
539
540 List<List<Cell>> kvs = new ArrayList<List<Cell>>();
541 kvs.add(kvs1);
542 kvs.add(kvs2);
543 byte[][] rows = new byte[][]{row1, row2};
544 for (int i = 0; i < kvs.size(); ++i) {
545 for (Cell kv : kvs.get(i)) {
546
547 Delete d = new Delete(rows[i]);
548 KeyValue kvDelete =
549 new KeyValue(rows[i], CellUtil.cloneFamily(kv), CellUtil.cloneQualifier(kv),
550 kv.getTimestamp(), KeyValue.Type.Delete);
551 d.addDeleteMarker(kvDelete);
552 Put p = new Put(rows[1 - i]);
553 KeyValue kvAdd =
554 new KeyValue(rows[1 - i], CellUtil.cloneFamily(kv), CellUtil.cloneQualifier(kv),
555 now, CellUtil.cloneValue(kv));
556 p.add(kvAdd);
557 mutations.add(d);
558 walEdit.add(kvDelete);
559 mutations.add(p);
560 walEdit.add(kvAdd);
561 }
562 }
563 }
564
565 @Override
566 public String getName() {
567 return "swap";
568 }
569
570 @Override
571 public RowSwapProcessorRequest getRequestData() throws IOException {
572 RowSwapProcessorRequest.Builder builder = RowSwapProcessorRequest.newBuilder();
573 builder.setRow1(ByteStringer.wrap(row1));
574 builder.setRow2(ByteStringer.wrap(row2));
575 return builder.build();
576 }
577
578 @Override
579 public void initialize(RowSwapProcessorRequest msg) {
580 this.row1 = msg.getRow1().toByteArray();
581 this.row2 = msg.getRow2().toByteArray();
582 }
583 }
584
585 public static class TimeoutProcessor extends
586 BaseRowProcessor<TimeoutProcessorRequest, TimeoutProcessorResponse> {
587
588 byte[] row = new byte[0];
589
590
591
592
593 public TimeoutProcessor() {
594 }
595
596 public TimeoutProcessor(byte[] row) {
597 this.row = row;
598 }
599
600 public Collection<byte[]> getRowsToLock() {
601 return Collections.singleton(row);
602 }
603
604 @Override
605 public TimeoutProcessorResponse getResult() {
606 return TimeoutProcessorResponse.getDefaultInstance();
607 }
608
609 @Override
610 public void process(long now, HRegion region,
611 List<Mutation> mutations, WALEdit walEdit) throws IOException {
612 try {
613
614 Thread.sleep(100 * 1000L);
615 } catch (Exception e) {
616 throw new IOException(e);
617 }
618 }
619
620 @Override
621 public boolean readOnly() {
622 return true;
623 }
624
625 @Override
626 public String getName() {
627 return "timeout";
628 }
629
630 @Override
631 public TimeoutProcessorRequest getRequestData() throws IOException {
632 TimeoutProcessorRequest.Builder builder = TimeoutProcessorRequest.newBuilder();
633 builder.setRow(ByteStringer.wrap(row));
634 return builder.build();
635 }
636
637 @Override
638 public void initialize(TimeoutProcessorRequest msg) throws IOException {
639 this.row = msg.getRow().toByteArray();
640 }
641 }
642
643 public static void doScan(
644 HRegion region, Scan scan, List<Cell> result) throws IOException {
645 InternalScanner scanner = null;
646 try {
647 scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
648 scanner = region.getScanner(scan);
649 result.clear();
650 scanner.next(result);
651 } finally {
652 if (scanner != null) scanner.close();
653 }
654 }
655 }
656
657 static String stringifyKvs(Collection<Cell> kvs) {
658 StringBuilder out = new StringBuilder();
659 out.append("[");
660 if (kvs != null) {
661 for (Cell kv : kvs) {
662 byte[] col = CellUtil.cloneQualifier(kv);
663 byte[] val = CellUtil.cloneValue(kv);
664 if (Bytes.equals(col, COUNTER)) {
665 out.append(Bytes.toStringBinary(col) + ":" +
666 Bytes.toInt(val) + " ");
667 } else {
668 out.append(Bytes.toStringBinary(col) + ":" +
669 Bytes.toStringBinary(val) + " ");
670 }
671 }
672 }
673 out.append("]");
674 return out.toString();
675 }
676
677 }