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