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