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