View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
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   * Verifies ProcessRowEndpoint works.
78   * The tested RowProcessor performs two scans and a read-modify-write.
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    // Column names
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       // ignore table not found
132     }
133     table = util.createTable(TABLE, FAM);
134     {
135       Put put = new Put(ROW);
136       put.add(FAM, A, Bytes.add(B, C));    // B, C are friends of A
137       put.add(FAM, B, Bytes.add(D, E, F)); // D, E, F are friends of B
138       put.add(FAM, C, G);                  // G is a friend of C
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    * This class defines two RowProcessors:
288    * IncrementCounterProcessor and FriendsOfFriendsProcessor.
289    *
290    * We define the RowProcessors as the inner class of the endpoint.
291    * So they can be loaded with the endpoint on the coprocessor.
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        * Empty constructor for Writable
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         // Scan current counter
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         // Assert counter value
340         assertEquals(expectedCounter, counter);
341 
342         // Increment counter and send it to both memstore and wal edit
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         // We can also inject some meta data to the walEdit
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        * Empty constructor for Writable
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         { // First scan to get friends of the person
415           Scan scan = new Scan(row, row);
416           scan.addColumn(FAM, person);
417           doScan(region, scan, kvs);
418         }
419 
420         // Second scan to get friends of friends
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         // Collect result
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        * Empty constructor for Writable
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         // Override the time to avoid race-condition in the unit test caused by
499         // inacurate timer on some machines
500         now = myTimer.getAndIncrement();
501 
502         // Scan both rows
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         // Assert swapped
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         // Add and delete keyvalues
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             // Delete from the current row and add to the other row
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        * Empty constructor for Writable
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           // Sleep for a long time so it timeout
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 }