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.regionserver;
20  
21  import static org.junit.Assert.assertEquals;
22  
23  import com.google.protobuf.Message;
24  import com.google.protobuf.ServiceException;
25  
26  import java.io.IOException;
27  import java.util.ArrayList;
28  import java.util.Collection;
29  import java.util.Collections;
30  import java.util.List;
31  import java.util.concurrent.atomic.AtomicBoolean;
32  
33  import org.apache.commons.logging.Log;
34  import org.apache.commons.logging.LogFactory;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.Cell;
37  import org.apache.hadoop.hbase.CellUtil;
38  import org.apache.hadoop.hbase.HBaseTestingUtility;
39  import org.apache.hadoop.hbase.HConstants;
40  import org.apache.hadoop.hbase.HTableDescriptor;
41  import org.apache.hadoop.hbase.KeyValue;
42  import org.apache.hadoop.hbase.TableName;
43  import org.apache.hadoop.hbase.client.Get;
44  import org.apache.hadoop.hbase.client.IsolationLevel;
45  import org.apache.hadoop.hbase.client.Mutation;
46  import org.apache.hadoop.hbase.client.Put;
47  import org.apache.hadoop.hbase.client.Result;
48  import org.apache.hadoop.hbase.client.Scan;
49  import org.apache.hadoop.hbase.client.Table;
50  import org.apache.hadoop.hbase.client.coprocessor.RowProcessorClient;
51  import org.apache.hadoop.hbase.coprocessor.BaseRowProcessorEndpoint;
52  import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
53  import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
54  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorRequest;
55  import org.apache.hadoop.hbase.coprocessor.protobuf.generated.IncrementCounterProcessorTestProtos.IncCounterProcessorResponse;
56  import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel;
57  import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessRequest;
58  import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.ProcessResponse;
59  import org.apache.hadoop.hbase.protobuf.generated.RowProcessorProtos.RowProcessorService;
60  import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
61  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
62  import org.apache.hadoop.hbase.testclassification.MediumTests;
63  import org.apache.hadoop.hbase.util.ByteStringer;
64  import org.apache.hadoop.hbase.util.Bytes;
65  import org.apache.hadoop.hbase.wal.WAL;
66  import org.apache.hadoop.hbase.wal.WALKey;
67  import org.junit.AfterClass;
68  import org.junit.Assert;
69  import org.junit.BeforeClass;
70  import org.junit.Test;
71  import org.junit.experimental.categories.Category;
72  
73  /**
74   * Test HRegion#processRowsWithLocks
75   */
76  @Category(MediumTests.class)
77  public class TestRegionProcessRowsWithLocks {
78  
79    private static final Log LOG = LogFactory.getLog(TestRegionProcessRowsWithLocks.class);
80  
81    private static final TableName TABLE = TableName.valueOf("testtable");
82    private final static byte[] ROW = Bytes.toBytes("testrow");
83    private final static byte[] FAM = Bytes.toBytes("friendlist");
84  
85    // Column names
86    private final static byte[] A = Bytes.toBytes("a");
87    private final static byte[] B = Bytes.toBytes("b");
88    private final static byte[] C = Bytes.toBytes("c");
89    private final static byte[] D = Bytes.toBytes("d");
90    private final static byte[] E = Bytes.toBytes("e");
91    private final static byte[] F = Bytes.toBytes("f");
92    private final static byte[] G = Bytes.toBytes("g");
93    private final static byte[] COUNTER = Bytes.toBytes("counter");
94  
95    private static HBaseTestingUtility util = new HBaseTestingUtility();
96    private static volatile int expectedCounter = 0;
97  
98    private volatile static Table table = null;
99    private static final AtomicBoolean throwsException = new AtomicBoolean(false);
100 
101   @BeforeClass
102   public static void setupBeforeClass() throws Exception {
103     Configuration conf = util.getConfiguration();
104     conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
105         RowProcessorEndpoint.class.getName());
106     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 2);
107     conf.setLong("hbase.hregion.row.processor.timeout", 1000L);
108     util.startMiniCluster();
109   }
110 
111   @AfterClass
112   public static void tearDownAfterClass() throws Exception {
113     util.shutdownMiniCluster();
114   }
115 
116   public void prepareTestData() throws IOException {
117     try {
118       util.getHBaseAdmin().disableTable(TABLE);
119       util.getHBaseAdmin().deleteTable(TABLE);
120     } catch (Exception e) {
121       // ignore table not found
122     }
123     table = util.createTable(TABLE, FAM);
124     {
125       Put put = new Put(ROW);
126       put.add(FAM, A, Bytes.add(B, C));    // B, C are friends of A
127       put.add(FAM, B, Bytes.add(D, E, F)); // D, E, F are friends of B
128       put.add(FAM, C, G);                  // G is a friend of C
129       table.put(put);
130     }
131     expectedCounter = 0;
132   }
133 
134   @Test
135   public void testProcessNormal() throws ServiceException, IOException {
136     prepareTestData();
137     List<HRegion> regions = util.getHBaseCluster().getRegions(TABLE);
138     HRegion region = regions.get(0);
139     long startMemstoreSize = region.getMemstoreSize();
140     long startFlushableSize = region.getStore(FAM).getFlushableSize();
141     int finalCounter = incrementCounter(table);
142     assertEquals(expectedCounter, finalCounter);
143     Get get = new Get(ROW);
144     Result result = table.get(get);
145     LOG.debug("row keyvalues:" + stringifyKvs(result.listCells()));
146     int getR = Bytes.toInt(CellUtil.cloneValue(result.getColumnLatestCell(FAM, COUNTER)));
147     assertEquals(expectedCounter, getR);
148 
149     long endMemstoreSize = region.getMemstoreSize();
150     long endFlushableSize = region.getStore(FAM).getFlushableSize();
151     Assert.assertEquals("Should equal.", (endMemstoreSize - startMemstoreSize),
152         (endFlushableSize - startFlushableSize));
153   }
154 
155   @Test
156   public void testProcessExceptionAndRollBack() throws IOException {
157     prepareTestData();
158     List<HRegion> regions = util.getHBaseCluster().getRegions(TABLE);
159     HRegion region = regions.get(0);
160     long startMemstoreSize = region.getMemstoreSize();
161     long startFlushableSize = region.getStore(FAM).getFlushableSize();
162     WAL wal = region.getWAL();
163     wal.registerWALActionsListener(new WALActionsListener.Base() {
164       @Override
165       public void visitLogEntryBeforeWrite(HTableDescriptor htd, WALKey logKey, WALEdit logEdit)
166           throws IOException {
167         if (throwsException.get()) {
168           throwsException.set(false);
169           throw new IOException("throw test IOException");
170         }
171       }
172     });
173     try {
174       incrementCounter(table);
175       Assert.fail("Should throw IOException.");
176     } catch (ServiceException | IOException e) {
177     }
178 
179     long endMemstoreSize = region.getMemstoreSize();
180     long endFlushableSize = region.getStore(FAM).getFlushableSize();
181     LOG.info(
182         "MemstoreSize deta=" + (endMemstoreSize - startMemstoreSize) + ",FlushableSize deta=" + (
183             endFlushableSize - startFlushableSize));
184     Assert.assertEquals("Should equal.", (endMemstoreSize - startMemstoreSize),
185         (endFlushableSize - startFlushableSize));
186   }
187 
188   private int incrementCounter(Table table) throws ServiceException, IOException {
189     CoprocessorRpcChannel channel = table.coprocessorService(ROW);
190     RowProcessorEndpoint.IncrementCounterProcessor processor =
191         new RowProcessorEndpoint.IncrementCounterProcessor(ROW);
192     RowProcessorService.BlockingInterface service = RowProcessorService.newBlockingStub(channel);
193     ProcessRequest request = RowProcessorClient.getRowProcessorPB(processor);
194     ProcessResponse protoResult = service.process(null, request);
195     IncCounterProcessorResponse response =
196         IncCounterProcessorResponse.parseFrom(protoResult.getRowProcessorResult());
197     Integer result = response.getResponse();
198     return result;
199   }
200 
201   /**
202    * This class defines two RowProcessors:
203    * IncrementCounterProcessor and FriendsOfFriendsProcessor.
204    * We define the RowProcessors as the inner class of the endpoint.
205    * So they can be loaded with the endpoint on the coprocessor.
206    */
207   public static class RowProcessorEndpoint<S extends Message, T extends Message>
208       extends BaseRowProcessorEndpoint<S, T> implements CoprocessorService {
209     public static class IncrementCounterProcessor
210         extends BaseRowProcessor<IncCounterProcessorRequest, IncCounterProcessorResponse> {
211       int counter = 0;
212       byte[] row = new byte[0];
213 
214       /**
215        * Empty constructor for Writable
216        */
217       public IncrementCounterProcessor() {
218       }
219 
220       public IncrementCounterProcessor(byte[] row) {
221         this.row = row;
222       }
223 
224       @Override
225       public Collection<byte[]> getRowsToLock() {
226         return Collections.singleton(row);
227       }
228 
229       @Override
230       public IncCounterProcessorResponse getResult() {
231         IncCounterProcessorResponse.Builder i = IncCounterProcessorResponse.newBuilder();
232         i.setResponse(counter);
233         return i.build();
234       }
235 
236       @Override
237       public boolean readOnly() {
238         return false;
239       }
240 
241       @Override
242       public void process(long now, HRegion region, List<Mutation> mutations, WALEdit walEdit)
243           throws IOException {
244         // Scan current counter
245         List<Cell> kvs = new ArrayList<Cell>();
246         Scan scan = new Scan(row, row);
247         scan.addColumn(FAM, COUNTER);
248         doScan(region, scan, kvs);
249         LOG.info("kvs.size()="+kvs.size());
250         counter = kvs.size() == 0 ? 0 : Bytes.toInt(CellUtil.cloneValue(kvs.iterator().next()));
251         LOG.info("counter=" + counter);
252 
253         // Assert counter value
254         assertEquals(expectedCounter, counter);
255 
256         // Increment counter and send it to both memstore and wal edit
257         counter += 1;
258         expectedCounter += 1;
259 
260         Put p = new Put(row);
261         KeyValue kv = new KeyValue(row, FAM, COUNTER, now, Bytes.toBytes(counter));
262         p.add(kv);
263         mutations.add(p);
264         walEdit.add(kv);
265 
266         // We can also inject some meta data to the walEdit
267         KeyValue metaKv =
268             new KeyValue(row, WALEdit.METAFAMILY, Bytes.toBytes("I just increment counter"),
269                 Bytes.toBytes(counter));
270         walEdit.add(metaKv);
271         throwsException.set(true);
272       }
273 
274       @Override
275       public IncCounterProcessorRequest getRequestData() throws IOException {
276         IncCounterProcessorRequest.Builder builder = IncCounterProcessorRequest.newBuilder();
277         builder.setCounter(counter);
278         builder.setRow(ByteStringer.wrap(row));
279         return builder.build();
280       }
281 
282       @Override
283       public void initialize(IncCounterProcessorRequest msg) {
284         this.row = msg.getRow().toByteArray();
285         this.counter = msg.getCounter();
286       }
287     }
288 
289     public static void doScan(HRegion region, Scan scan, List<Cell> result) throws IOException {
290       InternalScanner scanner = null;
291       try {
292         scan.setIsolationLevel(IsolationLevel.READ_UNCOMMITTED);
293         scanner = region.getScanner(scan);
294         result.clear();
295         scanner.next(result);
296       } finally {
297         if (scanner != null) {
298           scanner.close();
299         }
300       }
301     }
302   }
303 
304   static String stringifyKvs(Collection<Cell> kvs) {
305     StringBuilder out = new StringBuilder();
306     out.append("[");
307     if (kvs != null) {
308       for (Cell kv : kvs) {
309         byte[] col = CellUtil.cloneQualifier(kv);
310         byte[] val = CellUtil.cloneValue(kv);
311         if (Bytes.equals(col, COUNTER)) {
312           out.append(Bytes.toStringBinary(col) + ":" + Bytes.toInt(val) + " ");
313         } else {
314           out.append(Bytes.toStringBinary(col) + ":" + Bytes.toStringBinary(val) + " ");
315         }
316       }
317     }
318     out.append("]");
319     return out.toString();
320   }
321 
322 }