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