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
22 import static org.apache.hadoop.hbase.HBaseTestingUtility.COLUMNS;
23 import static org.apache.hadoop.hbase.HBaseTestingUtility.FIRST_CHAR;
24 import static org.apache.hadoop.hbase.HBaseTestingUtility.LAST_CHAR;
25 import static org.apache.hadoop.hbase.HBaseTestingUtility.START_KEY;
26 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam1;
27 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam2;
28 import static org.apache.hadoop.hbase.HBaseTestingUtility.fam3;
29 import static org.junit.Assert.assertArrayEquals;
30 import static org.junit.Assert.assertEquals;
31 import static org.junit.Assert.assertFalse;
32 import static org.junit.Assert.assertNotNull;
33 import static org.junit.Assert.assertNull;
34 import static org.junit.Assert.assertTrue;
35 import static org.junit.Assert.fail;
36 import static org.mockito.Matchers.any;
37 import static org.mockito.Matchers.anyBoolean;
38 import static org.mockito.Matchers.anyLong;
39 import static org.mockito.Matchers.anyString;
40 import static org.mockito.Mockito.atLeast;
41 import static org.mockito.Mockito.doThrow;
42 import static org.mockito.Mockito.mock;
43 import static org.mockito.Mockito.never;
44 import static org.mockito.Mockito.spy;
45 import static org.mockito.Mockito.times;
46 import static org.mockito.Mockito.verify;
47 import static org.mockito.Mockito.when;
48
49 import java.io.IOException;
50 import java.io.InterruptedIOException;
51 import java.security.PrivilegedExceptionAction;
52 import java.util.ArrayList;
53 import java.util.Arrays;
54 import java.util.Collection;
55 import java.util.Collections;
56 import java.util.HashMap;
57 import java.util.List;
58 import java.util.Map;
59 import java.util.NavigableMap;
60 import java.util.TreeMap;
61 import java.util.UUID;
62 import java.util.concurrent.Callable;
63 import java.util.concurrent.CountDownLatch;
64 import java.util.concurrent.ExecutorService;
65 import java.util.concurrent.Executors;
66 import java.util.concurrent.Future;
67 import java.util.concurrent.TimeUnit;
68 import java.util.concurrent.atomic.AtomicBoolean;
69 import java.util.concurrent.atomic.AtomicInteger;
70 import java.util.concurrent.atomic.AtomicReference;
71
72 import org.apache.commons.lang.RandomStringUtils;
73 import org.apache.commons.logging.Log;
74 import org.apache.commons.logging.LogFactory;
75 import org.apache.hadoop.conf.Configuration;
76 import org.apache.hadoop.fs.FSDataOutputStream;
77 import org.apache.hadoop.fs.FileStatus;
78 import org.apache.hadoop.fs.FileSystem;
79 import org.apache.hadoop.fs.Path;
80 import org.apache.hadoop.hbase.Cell;
81 import org.apache.hadoop.hbase.CellComparator;
82 import org.apache.hadoop.hbase.CellUtil;
83 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
84 import org.apache.hadoop.hbase.DroppedSnapshotException;
85 import org.apache.hadoop.hbase.HBaseConfiguration;
86 import org.apache.hadoop.hbase.HBaseTestCase;
87 import org.apache.hadoop.hbase.HBaseTestingUtility;
88 import org.apache.hadoop.hbase.HColumnDescriptor;
89 import org.apache.hadoop.hbase.HConstants;
90 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
91 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
92 import org.apache.hadoop.hbase.HRegionInfo;
93 import org.apache.hadoop.hbase.HTableDescriptor;
94 import org.apache.hadoop.hbase.KeyValue;
95 import org.apache.hadoop.hbase.KeyValueUtil;
96 import org.apache.hadoop.hbase.MiniHBaseCluster;
97 import org.apache.hadoop.hbase.MultithreadedTestUtil;
98 import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
99 import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
100 import org.apache.hadoop.hbase.NotServingRegionException;
101 import org.apache.hadoop.hbase.RegionTooBusyException;
102 import org.apache.hadoop.hbase.ServerName;
103 import org.apache.hadoop.hbase.TableName;
104 import org.apache.hadoop.hbase.Tag;
105 import org.apache.hadoop.hbase.TagType;
106 import org.apache.hadoop.hbase.Waiter;
107 import org.apache.hadoop.hbase.client.Append;
108 import org.apache.hadoop.hbase.client.Delete;
109 import org.apache.hadoop.hbase.client.Durability;
110 import org.apache.hadoop.hbase.client.Get;
111 import org.apache.hadoop.hbase.client.Increment;
112 import org.apache.hadoop.hbase.client.Mutation;
113 import org.apache.hadoop.hbase.client.Put;
114 import org.apache.hadoop.hbase.client.Result;
115 import org.apache.hadoop.hbase.client.RowMutations;
116 import org.apache.hadoop.hbase.client.Scan;
117 import org.apache.hadoop.hbase.client.Table;
118 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
119 import org.apache.hadoop.hbase.filter.BinaryComparator;
120 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
121 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
122 import org.apache.hadoop.hbase.filter.Filter;
123 import org.apache.hadoop.hbase.filter.FilterBase;
124 import org.apache.hadoop.hbase.filter.FilterList;
125 import org.apache.hadoop.hbase.filter.NullComparator;
126 import org.apache.hadoop.hbase.filter.PrefixFilter;
127 import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter;
128 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
129 import org.apache.hadoop.hbase.io.hfile.HFile;
130 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
131 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
132 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
133 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
134 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
135 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
136 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
137 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
138 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
139 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
140 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
141 import org.apache.hadoop.hbase.regionserver.Region.Operation;
142 import org.apache.hadoop.hbase.regionserver.Region.RowLock;
143 import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem;
144 import org.apache.hadoop.hbase.regionserver.handler.FinishRegionRecoveringHandler;
145 import org.apache.hadoop.hbase.regionserver.wal.FSHLog;
146 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
147 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
148 import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
149 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
150 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
151 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
152 import org.apache.hadoop.hbase.security.User;
153 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
154 import org.apache.hadoop.hbase.testclassification.MediumTests;
155 import org.apache.hadoop.hbase.util.Bytes;
156 import org.apache.hadoop.hbase.util.ClassSize;
157 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
158 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
159 import org.apache.hadoop.hbase.util.FSUtils;
160 import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
161 import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
162 import org.apache.hadoop.hbase.util.PairOfSameType;
163 import org.apache.hadoop.hbase.util.Threads;
164 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
165 import org.apache.hadoop.hbase.wal.FaultyFSLog;
166 import org.apache.hadoop.hbase.wal.WAL;
167 import org.apache.hadoop.hbase.wal.WALFactory;
168 import org.apache.hadoop.hbase.wal.WALKey;
169 import org.apache.hadoop.hbase.wal.WALProvider;
170 import org.apache.hadoop.hbase.wal.WALProvider.Writer;
171 import org.apache.hadoop.hbase.wal.WALSplitter;
172 import org.junit.After;
173 import org.junit.Assert;
174 import org.junit.Before;
175 import org.junit.Rule;
176 import org.junit.Test;
177 import org.junit.experimental.categories.Category;
178 import org.junit.rules.TestName;
179 import org.mockito.ArgumentCaptor;
180 import org.mockito.ArgumentMatcher;
181 import org.mockito.Mockito;
182 import org.mockito.invocation.InvocationOnMock;
183 import org.mockito.stubbing.Answer;
184
185 import com.google.common.collect.ImmutableList;
186 import com.google.common.collect.Lists;
187 import com.google.common.collect.Maps;
188 import com.google.protobuf.ByteString;
189
190
191
192
193
194
195
196 @Category(MediumTests.class)
197 @SuppressWarnings("deprecation")
198 public class TestHRegion {
199
200
201 private static final Log LOG = LogFactory.getLog(TestHRegion.class);
202 @Rule public TestName name = new TestName();
203
204 private static final String COLUMN_FAMILY = "MyCF";
205 private static final byte [] COLUMN_FAMILY_BYTES = Bytes.toBytes(COLUMN_FAMILY);
206
207 HRegion region = null;
208
209 private static HBaseTestingUtility TEST_UTIL;
210 public static Configuration CONF ;
211 private String dir;
212 private static FileSystem FILESYSTEM;
213 private final int MAX_VERSIONS = 2;
214
215
216 protected byte[] tableName;
217 protected String method;
218 protected final byte[] qual1 = Bytes.toBytes("qual1");
219 protected final byte[] qual2 = Bytes.toBytes("qual2");
220 protected final byte[] qual3 = Bytes.toBytes("qual3");
221 protected final byte[] value1 = Bytes.toBytes("value1");
222 protected final byte[] value2 = Bytes.toBytes("value2");
223 protected final byte[] row = Bytes.toBytes("rowA");
224 protected final byte[] row2 = Bytes.toBytes("rowB");
225
226 protected final MetricsAssertHelper metricsAssertHelper = CompatibilitySingletonFactory
227 .getInstance(MetricsAssertHelper.class);
228
229 @Before
230 public void setup() throws IOException {
231 TEST_UTIL = HBaseTestingUtility.createLocalHTU();
232 FILESYSTEM = TEST_UTIL.getTestFileSystem();
233 CONF = TEST_UTIL.getConfiguration();
234 dir = TEST_UTIL.getDataTestDir("TestHRegion").toString();
235 method = name.getMethodName();
236 tableName = Bytes.toBytes(name.getMethodName());
237 }
238
239 @After
240 public void tearDown() throws Exception {
241
242 HBaseTestingUtility.closeRegionAndWAL(region);
243 EnvironmentEdgeManagerTestHelper.reset();
244 LOG.info("Cleaning test directory: " + TEST_UTIL.getDataTestDir());
245 TEST_UTIL.cleanupTestDir();
246 }
247
248 String getName() {
249 return name.getMethodName();
250 }
251
252
253
254
255
256 @Test (timeout = 100000)
257 public void testSequenceId() throws IOException {
258 region = initHRegion(tableName, name.getMethodName(), CONF, COLUMN_FAMILY_BYTES);
259 assertEquals(HConstants.NO_SEQNUM, region.getMaxFlushedSeqId());
260
261 assertEquals(0, (long)region.getMaxStoreSeqId().get(COLUMN_FAMILY_BYTES));
262 HBaseTestingUtility.closeRegionAndWAL(this.region);
263 assertEquals(HConstants.NO_SEQNUM, region.getMaxFlushedSeqId());
264 assertEquals(0, (long)region.getMaxStoreSeqId().get(COLUMN_FAMILY_BYTES));
265
266 region = initHRegion(tableName, name.getMethodName(), CONF, COLUMN_FAMILY_BYTES);
267 byte [] value = Bytes.toBytes(name.getMethodName());
268
269 Put put = new Put(value);
270 put.addColumn(COLUMN_FAMILY_BYTES, null, value);
271 region.put(put);
272
273 assertEquals(HConstants.NO_SEQNUM, region.getMaxFlushedSeqId());
274 assertEquals(0, (long)region.getMaxStoreSeqId().get(COLUMN_FAMILY_BYTES));
275 region.flush(true);
276 long max = region.getMaxFlushedSeqId();
277 HBaseTestingUtility.closeRegionAndWAL(this.region);
278 assertEquals(max, region.getMaxFlushedSeqId());
279 this.region = null;
280 }
281
282
283
284
285
286
287
288
289
290
291
292 @Test (timeout=60000)
293 public void testCloseCarryingSnapshot() throws IOException {
294 region = initHRegion(tableName, name.getMethodName(), CONF, COLUMN_FAMILY_BYTES);
295 Store store = region.getStore(COLUMN_FAMILY_BYTES);
296
297 byte [] value = Bytes.toBytes(name.getMethodName());
298
299 Put put = new Put(value);
300 put.add(COLUMN_FAMILY_BYTES, null, value);
301
302 region.put(put);
303 StoreFlushContext storeFlushCtx = store.createFlushContext(12345);
304 storeFlushCtx.prepare();
305
306 put.add(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
307 region.put(put);
308
309 HBaseTestingUtility.closeRegionAndWAL(region);
310 assertEquals(0, region.getMemstoreSize());
311 region = null;
312 }
313
314
315
316
317
318 @Test (timeout=60000)
319 public void testMemstoreSnapshotSize() throws IOException {
320 class MyFaultyFSLog extends FaultyFSLog {
321 StoreFlushContext storeFlushCtx;
322 public MyFaultyFSLog(FileSystem fs, Path rootDir, String logName, Configuration conf)
323 throws IOException {
324 super(fs, rootDir, logName, conf);
325 }
326
327 void setStoreFlushCtx(StoreFlushContext storeFlushCtx) {
328 this.storeFlushCtx = storeFlushCtx;
329 }
330
331 @Override
332 public void sync(long txid, boolean forceSync) throws IOException {
333 storeFlushCtx.prepare();
334 super.sync(txid, forceSync);
335 }
336 }
337
338 FileSystem fs = FileSystem.get(CONF);
339 Path rootDir = new Path(dir + "testMemstoreSnapshotSize");
340 MyFaultyFSLog faultyLog = new MyFaultyFSLog(fs, rootDir, "testMemstoreSnapshotSize", CONF);
341 region = initHRegion(tableName, null, null, name.getMethodName(),
342 CONF, false, Durability.SYNC_WAL, faultyLog, COLUMN_FAMILY_BYTES);
343
344 Store store = region.getStore(COLUMN_FAMILY_BYTES);
345
346 byte [] value = Bytes.toBytes(name.getMethodName());
347 faultyLog.setStoreFlushCtx(store.createFlushContext(12345));
348
349 Put put = new Put(value);
350 put.add(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
351 faultyLog.setFailureType(FaultyFSLog.FailureType.SYNC);
352
353 boolean threwIOE = false;
354 try {
355 region.put(put);
356 } catch (IOException ioe) {
357 threwIOE = true;
358 } finally {
359 assertTrue("The regionserver should have thrown an exception", threwIOE);
360 }
361 long sz = store.getFlushableSize();
362 assertTrue("flushable size should be zero, but it is " + sz, sz == 0);
363 }
364
365
366
367
368 @Test
369 public void testMemstoreSizeWithFlushCanceling() throws IOException {
370 FileSystem fs = FileSystem.get(CONF);
371 Path rootDir = new Path(dir + "testMemstoreSizeWithFlushCanceling");
372 FSHLog hLog = new FSHLog(fs, rootDir, "testMemstoreSizeWithFlushCanceling", CONF);
373 region = initHRegion(tableName, null, null, name.getMethodName(),
374 CONF, false, Durability.SYNC_WAL, hLog, COLUMN_FAMILY_BYTES);
375 Store store = region.getStore(COLUMN_FAMILY_BYTES);
376 assertEquals(0, region.getMemstoreSize());
377
378
379 byte [] value = Bytes.toBytes(name.getMethodName());
380 Put put = new Put(value);
381 put.add(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
382 region.put(put);
383 long onePutSize = region.getMemstoreSize();
384 assertTrue(onePutSize > 0);
385 region.flush(true);
386 assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize());
387 assertEquals("flushable size should be zero", 0, store.getFlushableSize());
388
389
390 RegionCoprocessorHost normalCPHost = region.getCoprocessorHost();
391 RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class);
392 when(mockedCPHost.preFlush(Mockito.isA(HStore.class), Mockito.isA(InternalScanner.class))).
393 thenReturn(null);
394 region.setCoprocessorHost(mockedCPHost);
395 region.put(put);
396 region.flush(true);
397 assertEquals("memstoreSize should NOT be zero", onePutSize, region.getMemstoreSize());
398 assertEquals("flushable size should NOT be zero", onePutSize, store.getFlushableSize());
399
400
401 region.setCoprocessorHost(normalCPHost);
402 region.flush(true);
403 assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize());
404 assertEquals("flushable size should be zero", 0, store.getFlushableSize());
405 }
406
407 @Test
408 public void testMemstoreSizeAccountingWithFailedPostBatchMutate() throws IOException {
409 String testName = "testMemstoreSizeAccountingWithFailedPostBatchMutate";
410 FileSystem fs = FileSystem.get(CONF);
411 Path rootDir = new Path(dir + testName);
412 FSHLog hLog = new FSHLog(fs, rootDir, testName, CONF);
413 region = initHRegion(tableName, null, null, name.getMethodName(),
414 CONF, false, Durability.SYNC_WAL, hLog, COLUMN_FAMILY_BYTES);
415 Store store = region.getStore(COLUMN_FAMILY_BYTES);
416 assertEquals(0, region.getMemstoreSize());
417
418
419 byte [] value = Bytes.toBytes(name.getMethodName());
420 Put put = new Put(value);
421 put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
422 region.put(put);
423 long onePutSize = region.getMemstoreSize();
424 assertTrue(onePutSize > 0);
425
426 RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class);
427 doThrow(new IOException())
428 .when(mockedCPHost).postBatchMutate(Mockito.<MiniBatchOperationInProgress<Mutation>>any());
429 region.setCoprocessorHost(mockedCPHost);
430
431 put = new Put(value);
432 put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("dfg"), value);
433 try {
434 region.put(put);
435 fail("Should have failed with IOException");
436 } catch (IOException expected) {
437 }
438 long expectedSize = onePutSize * 2 - ClassSize.ARRAY;
439 assertEquals("memstoreSize should be incremented", expectedSize, region.getMemstoreSize());
440 assertEquals("flushable size should be incremented", expectedSize, store.getFlushableSize());
441
442 region.setCoprocessorHost(null);
443 }
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460 @Test (timeout=60000)
461 public void testFlushSizeAccounting() throws Exception {
462 final Configuration conf = HBaseConfiguration.create(CONF);
463
464 conf.setInt("hbase.hstore.flush.retries.number", 1);
465 final User user =
466 User.createUserForTesting(conf, this.name.getMethodName(), new String[]{"foo"});
467
468 conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);
469 user.runAs(new PrivilegedExceptionAction<Object>() {
470 @Override
471 public Object run() throws Exception {
472
473 FileSystem fs = FileSystem.get(conf);
474 Assert.assertEquals(FaultyFileSystem.class, fs.getClass());
475 FaultyFileSystem ffs = (FaultyFileSystem)fs;
476 HRegion region = null;
477 try {
478
479 region = initHRegion(tableName, name.getMethodName(), conf, COLUMN_FAMILY_BYTES);
480 long size = region.getMemstoreSize();
481 Assert.assertEquals(0, size);
482
483 Put p1 = new Put(row);
484 p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[])null));
485 region.put(p1);
486 final long sizeOfOnePut = region.getMemstoreSize();
487
488 try {
489 LOG.info("Flushing");
490 region.flush(true);
491 Assert.fail("Didn't bubble up IOE!");
492 } catch (DroppedSnapshotException dse) {
493
494 region.closing.set(false);
495 } catch (Exception e) {
496
497 region.closing.set(false);
498 }
499
500 ffs.fault.set(false);
501
502 try {
503 region.getWAL().rollWriter(true);
504 } catch (Exception e) {
505 int x = 0;
506 }
507
508 Assert.assertEquals(sizeOfOnePut, region.getMemstoreSize());
509
510
511 Put p2 = new Put(row);
512 p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual2, 2, (byte[])null));
513 p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual3, 3, (byte[])null));
514 region.put(p2);
515 long expectedSize = sizeOfOnePut * 3- ClassSize.ARRAY;
516 Assert.assertEquals(expectedSize, region.getMemstoreSize());
517
518
519
520 region.flush(true);
521
522 Assert.assertEquals(sizeOfOnePut * 2 - ClassSize.ARRAY, region.getMemstoreSize());
523 } catch (Exception e) {
524 int x = 0;
525 } finally {
526 HRegion.closeHRegion(region);
527 }
528 return null;
529 }
530 });
531 FileSystem.closeAllForUGI(user.getUGI());
532 }
533
534 @Test (timeout=60000)
535 public void testCloseWithFailingFlush() throws Exception {
536 final Configuration conf = HBaseConfiguration.create(CONF);
537
538 conf.setInt("hbase.hstore.flush.retries.number", 1);
539 final User user =
540 User.createUserForTesting(conf, this.name.getMethodName(), new String[]{"foo"});
541
542 conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);
543 user.runAs(new PrivilegedExceptionAction<Object>() {
544 @Override
545 public Object run() throws Exception {
546
547 FileSystem fs = FileSystem.get(conf);
548 Assert.assertEquals(FaultyFileSystem.class, fs.getClass());
549 FaultyFileSystem ffs = (FaultyFileSystem)fs;
550 HRegion region = null;
551 try {
552
553 region = initHRegion(tableName, name.getMethodName(), conf, COLUMN_FAMILY_BYTES);
554 long size = region.getMemstoreSize();
555 Assert.assertEquals(0, size);
556
557 Put p1 = new Put(row);
558 p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[])null));
559 region.put(p1);
560
561 Store store = region.getStore(COLUMN_FAMILY_BYTES);
562 StoreFlushContext storeFlushCtx = store.createFlushContext(12345);
563 storeFlushCtx.prepare();
564
565 Put p2 = new Put(row);
566 p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual2, 2, (byte[])null));
567 p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual3, 3, (byte[])null));
568 region.put(p2);
569
570 HBaseTestingUtility.closeRegionAndWAL(region);
571 region = null;
572 fail();
573 } catch (IOException dse) {
574
575 LOG.info("Expected DroppedSnapshotException");
576 } finally {
577
578 ffs.fault.set(false);
579 if (region != null) {
580 region.getWAL().rollWriter(true);
581 HRegion.closeHRegion(region);
582 }
583 }
584 return null;
585 }
586 });
587 FileSystem.closeAllForUGI(user.getUGI());
588 }
589
590 @Test
591 public void testCompactionAffectedByScanners() throws Exception {
592 byte[] family = Bytes.toBytes("family");
593 this.region = initHRegion(tableName, method, CONF, family);
594
595 Put put = new Put(Bytes.toBytes("r1"));
596 put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
597 region.put(put);
598 region.flush(true);
599
600 Scan scan = new Scan();
601 scan.setMaxVersions(3);
602
603 RegionScanner scanner1 = region.getScanner(scan);
604
605 Delete delete = new Delete(Bytes.toBytes("r1"));
606 region.delete(delete);
607 region.flush(true);
608
609
610 RegionScanner scanner2 = region.getScanner(scan);
611
612 List<Cell> results = new ArrayList<Cell>();
613
614 System.out.println("Smallest read point:" + region.getSmallestReadPoint());
615
616
617 region.compact(true);
618
619
620 RegionScanner scanner3 = region.getScanner(scan);
621
622
623 scanner1.next(results);
624 System.out.println(results);
625 assertEquals(1, results.size());
626
627 results.clear();
628 scanner2.next(results);
629 System.out.println(results);
630 assertEquals(0, results.size());
631
632 results.clear();
633 scanner3.next(results);
634 System.out.println(results);
635 assertEquals(0, results.size());
636 }
637
638 @Test
639 public void testToShowNPEOnRegionScannerReseek() throws Exception {
640 byte[] family = Bytes.toBytes("family");
641 this.region = initHRegion(tableName, method, CONF, family);
642
643 Put put = new Put(Bytes.toBytes("r1"));
644 put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
645 region.put(put);
646 put = new Put(Bytes.toBytes("r2"));
647 put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
648 region.put(put);
649 region.flush(true);
650
651 Scan scan = new Scan();
652 scan.setMaxVersions(3);
653
654 RegionScanner scanner1 = region.getScanner(scan);
655
656 System.out.println("Smallest read point:" + region.getSmallestReadPoint());
657
658 region.compact(true);
659
660 scanner1.reseek(Bytes.toBytes("r2"));
661 List<Cell> results = new ArrayList<Cell>();
662 scanner1.next(results);
663 Cell keyValue = results.get(0);
664 Assert.assertTrue(Bytes.compareTo(CellUtil.cloneRow(keyValue), Bytes.toBytes("r2")) == 0);
665 scanner1.close();
666 }
667
668 @Test
669 public void testSkipRecoveredEditsReplay() throws Exception {
670 String method = "testSkipRecoveredEditsReplay";
671 TableName tableName = TableName.valueOf(method);
672 byte[] family = Bytes.toBytes("family");
673 this.region = initHRegion(tableName, method, CONF, family);
674 final WALFactory wals = new WALFactory(CONF, null, method);
675 try {
676 Path regiondir = region.getRegionFileSystem().getRegionDir();
677 FileSystem fs = region.getRegionFileSystem().getFileSystem();
678 byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
679
680 Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
681
682 long maxSeqId = 1050;
683 long minSeqId = 1000;
684
685 for (long i = minSeqId; i <= maxSeqId; i += 10) {
686 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
687 fs.create(recoveredEdits);
688 WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits);
689
690 long time = System.nanoTime();
691 WALEdit edit = new WALEdit();
692 edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
693 .toBytes(i)));
694 writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time,
695 HConstants.DEFAULT_CLUSTER_ID), edit));
696
697 writer.close();
698 }
699 MonitoredTask status = TaskMonitor.get().createStatus(method);
700 Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
701 for (Store store : region.getStores()) {
702 maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId - 1);
703 }
704 long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, status);
705 assertEquals(maxSeqId, seqId);
706 region.getMVCC().advanceTo(seqId);
707 Get get = new Get(row);
708 Result result = region.get(get);
709 for (long i = minSeqId; i <= maxSeqId; i += 10) {
710 List<Cell> kvs = result.getColumnCells(family, Bytes.toBytes(i));
711 assertEquals(1, kvs.size());
712 assertArrayEquals(Bytes.toBytes(i), CellUtil.cloneValue(kvs.get(0)));
713 }
714 } finally {
715 HRegion.closeHRegion(this.region);
716 this.region = null;
717 wals.close();
718 }
719 }
720
721 @Test
722 public void testSkipRecoveredEditsReplaySomeIgnored() throws Exception {
723 String method = "testSkipRecoveredEditsReplaySomeIgnored";
724 TableName tableName = TableName.valueOf(method);
725 byte[] family = Bytes.toBytes("family");
726 this.region = initHRegion(tableName, method, CONF, family);
727 final WALFactory wals = new WALFactory(CONF, null, method);
728 try {
729 Path regiondir = region.getRegionFileSystem().getRegionDir();
730 FileSystem fs = region.getRegionFileSystem().getFileSystem();
731 byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
732
733 Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
734
735 long maxSeqId = 1050;
736 long minSeqId = 1000;
737
738 for (long i = minSeqId; i <= maxSeqId; i += 10) {
739 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
740 fs.create(recoveredEdits);
741 WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits);
742
743 long time = System.nanoTime();
744 WALEdit edit = new WALEdit();
745 edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
746 .toBytes(i)));
747 writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time,
748 HConstants.DEFAULT_CLUSTER_ID), edit));
749
750 writer.close();
751 }
752 long recoverSeqId = 1030;
753 MonitoredTask status = TaskMonitor.get().createStatus(method);
754 Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
755 for (Store store : region.getStores()) {
756 maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1);
757 }
758 long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, status);
759 assertEquals(maxSeqId, seqId);
760 region.getMVCC().advanceTo(seqId);
761 Get get = new Get(row);
762 Result result = region.get(get);
763 for (long i = minSeqId; i <= maxSeqId; i += 10) {
764 List<Cell> kvs = result.getColumnCells(family, Bytes.toBytes(i));
765 if (i < recoverSeqId) {
766 assertEquals(0, kvs.size());
767 } else {
768 assertEquals(1, kvs.size());
769 assertArrayEquals(Bytes.toBytes(i), CellUtil.cloneValue(kvs.get(0)));
770 }
771 }
772 } finally {
773 HRegion.closeHRegion(this.region);
774 this.region = null;
775 wals.close();
776 }
777 }
778
779 @Test
780 public void testSkipRecoveredEditsReplayAllIgnored() throws Exception {
781 byte[] family = Bytes.toBytes("family");
782 this.region = initHRegion(tableName, method, CONF, family);
783 Path regiondir = region.getRegionFileSystem().getRegionDir();
784 FileSystem fs = region.getRegionFileSystem().getFileSystem();
785
786 Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
787 for (int i = 1000; i < 1050; i += 10) {
788 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
789 FSDataOutputStream dos = fs.create(recoveredEdits);
790 dos.writeInt(i);
791 dos.close();
792 }
793 long minSeqId = 2000;
794 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", minSeqId - 1));
795 FSDataOutputStream dos = fs.create(recoveredEdits);
796 dos.close();
797
798 Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
799 for (Store store : region.getStores()) {
800 maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId);
801 }
802 long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, null);
803 assertEquals(minSeqId, seqId);
804 }
805
806 @Test
807 public void testSkipRecoveredEditsReplayTheLastFileIgnored() throws Exception {
808 String method = "testSkipRecoveredEditsReplayTheLastFileIgnored";
809 TableName tableName = TableName.valueOf(method);
810 byte[] family = Bytes.toBytes("family");
811 this.region = initHRegion(tableName, method, CONF, family);
812 final WALFactory wals = new WALFactory(CONF, null, method);
813 try {
814 Path regiondir = region.getRegionFileSystem().getRegionDir();
815 FileSystem fs = region.getRegionFileSystem().getFileSystem();
816 byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
817 byte[][] columns = region.getTableDesc().getFamiliesKeys().toArray(new byte[0][]);
818
819 assertEquals(0, region.getStoreFileList(columns).size());
820
821 Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
822
823 long maxSeqId = 1050;
824 long minSeqId = 1000;
825
826 for (long i = minSeqId; i <= maxSeqId; i += 10) {
827 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
828 fs.create(recoveredEdits);
829 WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits);
830
831 long time = System.nanoTime();
832 WALEdit edit = null;
833 if (i == maxSeqId) {
834 edit = WALEdit.createCompaction(region.getRegionInfo(),
835 CompactionDescriptor.newBuilder()
836 .setTableName(ByteString.copyFrom(tableName.getName()))
837 .setFamilyName(ByteString.copyFrom(regionName))
838 .setEncodedRegionName(ByteString.copyFrom(regionName))
839 .setStoreHomeDirBytes(ByteString.copyFrom(Bytes.toBytes(regiondir.toString())))
840 .setRegionName(ByteString.copyFrom(region.getRegionInfo().getRegionName()))
841 .build());
842 } else {
843 edit = new WALEdit();
844 edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
845 .toBytes(i)));
846 }
847 writer.append(new WAL.Entry(new HLogKey(regionName, tableName, i, time,
848 HConstants.DEFAULT_CLUSTER_ID), edit));
849 writer.close();
850 }
851
852 long recoverSeqId = 1030;
853 Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
854 MonitoredTask status = TaskMonitor.get().createStatus(method);
855 for (Store store : region.getStores()) {
856 maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1);
857 }
858 long seqId = region.replayRecoveredEditsIfAny(maxSeqIdInStores, null, status);
859 assertEquals(maxSeqId, seqId);
860
861
862 assertEquals(1, region.getStoreFileList(columns).size());
863
864 } finally {
865 HRegion.closeHRegion(this.region);
866 this.region = null;
867 wals.close();
868 }
869 }
870
871 @Test
872 public void testRecoveredEditsReplayCompaction() throws Exception {
873 testRecoveredEditsReplayCompaction(false);
874 testRecoveredEditsReplayCompaction(true);
875 }
876 public void testRecoveredEditsReplayCompaction(boolean mismatchedRegionName) throws Exception {
877 String method = name.getMethodName();
878 TableName tableName = TableName.valueOf(method);
879 byte[] family = Bytes.toBytes("family");
880 this.region = initHRegion(tableName, method, CONF, family);
881 final WALFactory wals = new WALFactory(CONF, null, method);
882 try {
883 Path regiondir = region.getRegionFileSystem().getRegionDir();
884 FileSystem fs = region.getRegionFileSystem().getFileSystem();
885 byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
886
887 long maxSeqId = 3;
888 long minSeqId = 0;
889
890 for (long i = minSeqId; i < maxSeqId; i++) {
891 Put put = new Put(Bytes.toBytes(i));
892 put.add(family, Bytes.toBytes(i), Bytes.toBytes(i));
893 region.put(put);
894 region.flush(true);
895 }
896
897
898 assertEquals(3, region.getStore(family).getStorefilesCount());
899 List<Path> storeFiles = new ArrayList<Path>(3);
900 for (StoreFile sf : region.getStore(family).getStorefiles()) {
901 storeFiles.add(sf.getPath());
902 }
903
904
905 CONF.setBoolean("hbase.hstore.compaction.complete", false);
906 region.compactStores();
907
908
909 assertEquals(3, region.getStore(family).getStorefilesCount());
910
911
912 Path tmpDir = new Path(region.getRegionFileSystem().getTempDir(), Bytes.toString(family));
913 FileStatus[] files = FSUtils.listStatus(fs, tmpDir);
914 String errorMsg = "Expected to find 1 file in the region temp directory "
915 + "from the compaction, could not find any";
916 assertNotNull(errorMsg, files);
917 assertEquals(errorMsg, 1, files.length);
918
919 Path newFile = region.getRegionFileSystem().commitStoreFile(Bytes.toString(family),
920 files[0].getPath());
921
922 byte[] encodedNameAsBytes = this.region.getRegionInfo().getEncodedNameAsBytes();
923 byte[] fakeEncodedNameAsBytes = new byte [encodedNameAsBytes.length];
924 for (int i=0; i < encodedNameAsBytes.length; i++) {
925
926 fakeEncodedNameAsBytes[i] = (byte) (encodedNameAsBytes[i] + 1);
927 }
928
929 CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(this.region
930 .getRegionInfo(), mismatchedRegionName ? fakeEncodedNameAsBytes : null, family,
931 storeFiles, Lists.newArrayList(newFile),
932 region.getRegionFileSystem().getStoreDir(Bytes.toString(family)));
933
934 WALUtil.writeCompactionMarker(region.getWAL(), this.region.getTableDesc(),
935 this.region.getRegionInfo(), compactionDescriptor, region.getMVCC());
936
937 Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
938
939 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", 1000));
940 fs.create(recoveredEdits);
941 WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits);
942
943 long time = System.nanoTime();
944
945 writer.append(new WAL.Entry(new HLogKey(regionName, tableName, 10, time,
946 HConstants.DEFAULT_CLUSTER_ID), WALEdit.createCompaction(region.getRegionInfo(),
947 compactionDescriptor)));
948 writer.close();
949
950
951 region.getTableDesc();
952 region.getRegionInfo();
953 HBaseTestingUtility.closeRegionAndWAL(this.region);
954 try {
955 region = HRegion.openHRegion(region, null);
956 } catch (WrongRegionException wre) {
957 fail("Matching encoded region name should not have produced WrongRegionException");
958 }
959
960
961 Collection<StoreFile> sfs = region.getStore(family).getStorefiles();
962 for (StoreFile sf : sfs) {
963 LOG.info(sf.getPath());
964 }
965 if (!mismatchedRegionName) {
966 assertEquals(1, region.getStore(family).getStorefilesCount());
967 }
968 files = FSUtils.listStatus(fs, tmpDir);
969 assertTrue("Expected to find 0 files inside " + tmpDir, files == null || files.length == 0);
970
971 for (long i = minSeqId; i < maxSeqId; i++) {
972 Get get = new Get(Bytes.toBytes(i));
973 Result result = region.get(get);
974 byte[] value = result.getValue(family, Bytes.toBytes(i));
975 assertArrayEquals(Bytes.toBytes(i), value);
976 }
977 } finally {
978 HRegion.closeHRegion(this.region);
979 this.region = null;
980 wals.close();
981 }
982 }
983
984 @Test
985 public void testFlushMarkers() throws Exception {
986
987 String method = name.getMethodName();
988 TableName tableName = TableName.valueOf(method);
989 byte[] family = Bytes.toBytes("family");
990 Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + ".log");
991 final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration());
992 FSUtils.setRootDir(walConf, logDir);
993 final WALFactory wals = new WALFactory(walConf, null, method);
994 final WAL wal = wals.getWAL(tableName.getName(), tableName.getNamespace());
995
996 this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
997 HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
998 try {
999 Path regiondir = region.getRegionFileSystem().getRegionDir();
1000 FileSystem fs = region.getRegionFileSystem().getFileSystem();
1001 byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
1002
1003 long maxSeqId = 3;
1004 long minSeqId = 0;
1005
1006 for (long i = minSeqId; i < maxSeqId; i++) {
1007 Put put = new Put(Bytes.toBytes(i));
1008 put.add(family, Bytes.toBytes(i), Bytes.toBytes(i));
1009 region.put(put);
1010 region.flush(true);
1011 }
1012
1013
1014 assertEquals(3, region.getStore(family).getStorefilesCount());
1015 List<String> storeFiles = new ArrayList<String>(3);
1016 for (StoreFile sf : region.getStore(family).getStorefiles()) {
1017 storeFiles.add(sf.getPath().getName());
1018 }
1019
1020
1021 wal.shutdown();
1022 WAL.Reader reader = WALFactory.createReader(fs, DefaultWALProvider.getCurrentFileName(wal),
1023 TEST_UTIL.getConfiguration());
1024 try {
1025 List<WAL.Entry> flushDescriptors = new ArrayList<WAL.Entry>();
1026 long lastFlushSeqId = -1;
1027 while (true) {
1028 WAL.Entry entry = reader.next();
1029 if (entry == null) {
1030 break;
1031 }
1032 Cell cell = entry.getEdit().getCells().get(0);
1033 if (WALEdit.isMetaEditFamily(cell)) {
1034 FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(cell);
1035 assertNotNull(flushDesc);
1036 assertArrayEquals(tableName.getName(), flushDesc.getTableName().toByteArray());
1037 if (flushDesc.getAction() == FlushAction.START_FLUSH) {
1038 assertTrue(flushDesc.getFlushSequenceNumber() > lastFlushSeqId);
1039 } else if (flushDesc.getAction() == FlushAction.COMMIT_FLUSH) {
1040 assertTrue(flushDesc.getFlushSequenceNumber() == lastFlushSeqId);
1041 }
1042 lastFlushSeqId = flushDesc.getFlushSequenceNumber();
1043 assertArrayEquals(regionName, flushDesc.getEncodedRegionName().toByteArray());
1044 assertEquals(1, flushDesc.getStoreFlushesCount());
1045 StoreFlushDescriptor storeFlushDesc = flushDesc.getStoreFlushes(0);
1046 assertArrayEquals(family, storeFlushDesc.getFamilyName().toByteArray());
1047 assertEquals("family", storeFlushDesc.getStoreHomeDir());
1048 if (flushDesc.getAction() == FlushAction.START_FLUSH) {
1049 assertEquals(0, storeFlushDesc.getFlushOutputCount());
1050 } else {
1051 assertEquals(1, storeFlushDesc.getFlushOutputCount());
1052 assertTrue(storeFiles.contains(storeFlushDesc.getFlushOutput(0)));
1053 }
1054
1055 flushDescriptors.add(entry);
1056 }
1057 }
1058
1059 assertEquals(3 * 2, flushDescriptors.size());
1060
1061
1062
1063 Path recoveredEditsDir = WALSplitter.getRegionDirRecoveredEditsDir(regiondir);
1064
1065 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", 1000));
1066 fs.create(recoveredEdits);
1067 WALProvider.Writer writer = wals.createRecoveredEditsWriter(fs, recoveredEdits);
1068
1069 for (WAL.Entry entry : flushDescriptors) {
1070 writer.append(entry);
1071 }
1072 writer.close();
1073 } finally {
1074 if (null != reader) {
1075 try {
1076 reader.close();
1077 } catch (IOException exception) {
1078 LOG.warn("Problem closing wal: " + exception.getMessage());
1079 LOG.debug("exception details", exception);
1080 }
1081 }
1082 }
1083
1084
1085 HBaseTestingUtility.closeRegionAndWAL(this.region);
1086 region = HRegion.openHRegion(region, null);
1087
1088
1089 for (long i = minSeqId; i < maxSeqId; i++) {
1090 Get get = new Get(Bytes.toBytes(i));
1091 Result result = region.get(get);
1092 byte[] value = result.getValue(family, Bytes.toBytes(i));
1093 assertArrayEquals(Bytes.toBytes(i), value);
1094 }
1095 } finally {
1096 HBaseTestingUtility.closeRegionAndWAL(this.region);
1097 this.region = null;
1098 wals.close();
1099 }
1100 }
1101
1102 class IsFlushWALMarker extends ArgumentMatcher<WALEdit> {
1103 volatile FlushAction[] actions;
1104 public IsFlushWALMarker(FlushAction... actions) {
1105 this.actions = actions;
1106 }
1107 @Override
1108 public boolean matches(Object edit) {
1109 List<Cell> cells = ((WALEdit)edit).getCells();
1110 if (cells.isEmpty()) {
1111 return false;
1112 }
1113 if (WALEdit.isMetaEditFamily(cells.get(0))) {
1114 FlushDescriptor desc = null;
1115 try {
1116 desc = WALEdit.getFlushDescriptor(cells.get(0));
1117 } catch (IOException e) {
1118 LOG.warn(e);
1119 return false;
1120 }
1121 if (desc != null) {
1122 for (FlushAction action : actions) {
1123 if (desc.getAction() == action) {
1124 return true;
1125 }
1126 }
1127 }
1128 }
1129 return false;
1130 }
1131 public IsFlushWALMarker set(FlushAction... actions) {
1132 this.actions = actions;
1133 return this;
1134 }
1135 }
1136
1137 @Test (timeout=60000)
1138 public void testFlushMarkersWALFail() throws Exception {
1139
1140 String method = name.getMethodName();
1141 TableName tableName = TableName.valueOf(method);
1142 byte[] family = Bytes.toBytes("family");
1143
1144
1145 Path logDir = TEST_UTIL.getDataTestDirOnTestFS(method + "log");
1146
1147 final Configuration walConf = new Configuration(TEST_UTIL.getConfiguration());
1148 FSUtils.setRootDir(walConf, logDir);
1149
1150
1151 class FailAppendFlushMarkerWAL extends FSHLog {
1152 volatile FlushAction [] flushActions = null;
1153
1154 public FailAppendFlushMarkerWAL(FileSystem fs, Path root, String logDir, Configuration conf)
1155 throws IOException {
1156 super(fs, root, logDir, conf);
1157 }
1158
1159 @Override
1160 protected Writer createWriterInstance(Path path) throws IOException {
1161 final Writer w = super.createWriterInstance(path);
1162 return new Writer() {
1163 @Override
1164 public void close() throws IOException {
1165 w.close();
1166 }
1167
1168 @Override
1169 public void sync(boolean forceSync) throws IOException {
1170 w.sync(forceSync);
1171 }
1172
1173 @Override
1174 public void append(Entry entry) throws IOException {
1175 List<Cell> cells = entry.getEdit().getCells();
1176 if (WALEdit.isMetaEditFamily(cells.get(0))) {
1177 FlushDescriptor desc = WALEdit.getFlushDescriptor(cells.get(0));
1178 if (desc != null) {
1179 for (FlushAction flushAction: flushActions) {
1180 if (desc.getAction().equals(flushAction)) {
1181 throw new IOException("Failed to append flush marker! " + flushAction);
1182 }
1183 }
1184 }
1185 }
1186 w.append(entry);
1187 }
1188
1189 @Override
1190 public long getLength() throws IOException {
1191 return w.getLength();
1192 }
1193 };
1194 }
1195 }
1196 FailAppendFlushMarkerWAL wal =
1197 new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf),
1198 getName(), walConf);
1199 this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
1200 HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
1201 int i = 0;
1202 Put put = new Put(Bytes.toBytes(i));
1203 put.setDurability(Durability.SKIP_WAL);
1204 put.add(family, Bytes.toBytes(i), Bytes.toBytes(i));
1205 region.put(put);
1206
1207
1208 wal.flushActions = new FlushAction [] {FlushAction.START_FLUSH};
1209
1210
1211 try {
1212 region.flush(true);
1213 fail("This should have thrown exception");
1214 } catch (DroppedSnapshotException unexpected) {
1215
1216 throw unexpected;
1217 } catch (IOException expected) {
1218
1219 }
1220
1221
1222 region.close(true);
1223 wal.close();
1224
1225
1226 wal.flushActions = new FlushAction [] {FlushAction.COMMIT_FLUSH};
1227 wal = new FailAppendFlushMarkerWAL(FileSystem.get(walConf), FSUtils.getRootDir(walConf),
1228 getName(), walConf);
1229 this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
1230 HConstants.EMPTY_END_ROW, method, CONF, false, Durability.USE_DEFAULT, wal, family);
1231 region.put(put);
1232
1233
1234
1235
1236 wal.flushActions = new FlushAction [] {FlushAction.COMMIT_FLUSH, FlushAction.ABORT_FLUSH};
1237
1238 try {
1239 region.flush(true);
1240 fail("This should have thrown exception");
1241 } catch (DroppedSnapshotException expected) {
1242
1243
1244 } catch (IOException unexpected) {
1245 throw unexpected;
1246 }
1247 }
1248
1249 @Test
1250 public void testGetWhileRegionClose() throws IOException {
1251 TableName tableName = TableName.valueOf(name.getMethodName());
1252 Configuration hc = initSplit();
1253 int numRows = 100;
1254 byte[][] families = { fam1, fam2, fam3 };
1255
1256
1257 String method = name.getMethodName();
1258 this.region = initHRegion(tableName, method, hc, families);
1259
1260 final int startRow = 100;
1261 putData(startRow, numRows, qual1, families);
1262 putData(startRow, numRows, qual2, families);
1263 putData(startRow, numRows, qual3, families);
1264 final AtomicBoolean done = new AtomicBoolean(false);
1265 final AtomicInteger gets = new AtomicInteger(0);
1266 GetTillDoneOrException[] threads = new GetTillDoneOrException[10];
1267 try {
1268
1269 for (int i = 0; i < threads.length / 2; i++) {
1270 threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow), done, gets);
1271 threads[i].setDaemon(true);
1272 threads[i].start();
1273 }
1274
1275
1276 this.region.closing.set(true);
1277 for (int i = threads.length / 2; i < threads.length; i++) {
1278 threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow), done, gets);
1279 threads[i].setDaemon(true);
1280 threads[i].start();
1281 }
1282 } finally {
1283 if (this.region != null) {
1284 HBaseTestingUtility.closeRegionAndWAL(this.region);
1285 this.region = null;
1286 }
1287 }
1288 done.set(true);
1289 for (GetTillDoneOrException t : threads) {
1290 try {
1291 t.join();
1292 } catch (InterruptedException e) {
1293 e.printStackTrace();
1294 }
1295 if (t.e != null) {
1296 LOG.info("Exception=" + t.e);
1297 assertFalse("Found a NPE in " + t.getName(), t.e instanceof NullPointerException);
1298 }
1299 }
1300 }
1301
1302
1303
1304
1305
1306 class GetTillDoneOrException extends Thread {
1307 private final Get g;
1308 private final AtomicBoolean done;
1309 private final AtomicInteger count;
1310 private Exception e;
1311
1312 GetTillDoneOrException(final int i, final byte[] r, final AtomicBoolean d, final AtomicInteger c) {
1313 super("getter." + i);
1314 this.g = new Get(r);
1315 this.done = d;
1316 this.count = c;
1317 }
1318
1319 @Override
1320 public void run() {
1321 while (!this.done.get()) {
1322 try {
1323 assertTrue(region.get(g).size() > 0);
1324 this.count.incrementAndGet();
1325 } catch (Exception e) {
1326 this.e = e;
1327 break;
1328 }
1329 }
1330 }
1331 }
1332
1333
1334
1335
1336 @Test
1337 public void testWeirdCacheBehaviour() throws Exception {
1338 byte[] TABLE = Bytes.toBytes("testWeirdCacheBehaviour");
1339 byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"), Bytes.toBytes("trans-type"),
1340 Bytes.toBytes("trans-date"), Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
1341 this.region = initHRegion(TABLE, getName(), CONF, FAMILIES);
1342 String value = "this is the value";
1343 String value2 = "this is some other value";
1344 String keyPrefix1 = "prefix1";
1345 String keyPrefix2 = "prefix2";
1346 String keyPrefix3 = "prefix3";
1347 putRows(this.region, 3, value, keyPrefix1);
1348 putRows(this.region, 3, value, keyPrefix2);
1349 putRows(this.region, 3, value, keyPrefix3);
1350 putRows(this.region, 3, value2, keyPrefix1);
1351 putRows(this.region, 3, value2, keyPrefix2);
1352 putRows(this.region, 3, value2, keyPrefix3);
1353 System.out.println("Checking values for key: " + keyPrefix1);
1354 assertEquals("Got back incorrect number of rows from scan", 3,
1355 getNumberOfRows(keyPrefix1, value2, this.region));
1356 System.out.println("Checking values for key: " + keyPrefix2);
1357 assertEquals("Got back incorrect number of rows from scan", 3,
1358 getNumberOfRows(keyPrefix2, value2, this.region));
1359 System.out.println("Checking values for key: " + keyPrefix3);
1360 assertEquals("Got back incorrect number of rows from scan", 3,
1361 getNumberOfRows(keyPrefix3, value2, this.region));
1362 deleteColumns(this.region, value2, keyPrefix1);
1363 deleteColumns(this.region, value2, keyPrefix2);
1364 deleteColumns(this.region, value2, keyPrefix3);
1365 System.out.println("Starting important checks.....");
1366 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1, 0,
1367 getNumberOfRows(keyPrefix1, value2, this.region));
1368 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2, 0,
1369 getNumberOfRows(keyPrefix2, value2, this.region));
1370 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3, 0,
1371 getNumberOfRows(keyPrefix3, value2, this.region));
1372 }
1373
1374 @Test
1375 public void testAppendWithReadOnlyTable() throws Exception {
1376 byte[] TABLE = Bytes.toBytes("readOnlyTable");
1377 this.region = initHRegion(TABLE, getName(), CONF, true, Bytes.toBytes("somefamily"));
1378 boolean exceptionCaught = false;
1379 Append append = new Append(Bytes.toBytes("somerow"));
1380 append.setDurability(Durability.SKIP_WAL);
1381 append.add(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"),
1382 Bytes.toBytes("somevalue"));
1383 try {
1384 region.append(append);
1385 } catch (IOException e) {
1386 exceptionCaught = true;
1387 }
1388 assertTrue(exceptionCaught);
1389 }
1390
1391 @Test
1392 public void testIncrWithReadOnlyTable() throws Exception {
1393 byte[] TABLE = Bytes.toBytes("readOnlyTable");
1394 this.region = initHRegion(TABLE, getName(), CONF, true, Bytes.toBytes("somefamily"));
1395 boolean exceptionCaught = false;
1396 Increment inc = new Increment(Bytes.toBytes("somerow"));
1397 inc.setDurability(Durability.SKIP_WAL);
1398 inc.addColumn(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), 1L);
1399 try {
1400 region.increment(inc);
1401 } catch (IOException e) {
1402 exceptionCaught = true;
1403 }
1404 assertTrue(exceptionCaught);
1405 }
1406
1407 private void deleteColumns(HRegion r, String value, String keyPrefix) throws IOException {
1408 InternalScanner scanner = buildScanner(keyPrefix, value, r);
1409 int count = 0;
1410 boolean more = false;
1411 List<Cell> results = new ArrayList<Cell>();
1412 do {
1413 more = scanner.next(results);
1414 if (results != null && !results.isEmpty())
1415 count++;
1416 else
1417 break;
1418 Delete delete = new Delete(CellUtil.cloneRow(results.get(0)));
1419 delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
1420 r.delete(delete);
1421 results.clear();
1422 } while (more);
1423 assertEquals("Did not perform correct number of deletes", 3, count);
1424 }
1425
1426 private int getNumberOfRows(String keyPrefix, String value, HRegion r) throws Exception {
1427 InternalScanner resultScanner = buildScanner(keyPrefix, value, r);
1428 int numberOfResults = 0;
1429 List<Cell> results = new ArrayList<Cell>();
1430 boolean more = false;
1431 do {
1432 more = resultScanner.next(results);
1433 if (results != null && !results.isEmpty())
1434 numberOfResults++;
1435 else
1436 break;
1437 for (Cell kv : results) {
1438 System.out.println("kv=" + kv.toString() + ", " + Bytes.toString(CellUtil.cloneValue(kv)));
1439 }
1440 results.clear();
1441 } while (more);
1442 return numberOfResults;
1443 }
1444
1445 private InternalScanner buildScanner(String keyPrefix, String value, HRegion r)
1446 throws IOException {
1447
1448 FilterList allFilters = new FilterList();
1449 allFilters.addFilter(new PrefixFilter(Bytes.toBytes(keyPrefix)));
1450
1451 SingleColumnValueFilter filter = new SingleColumnValueFilter(Bytes.toBytes("trans-tags"),
1452 Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes.toBytes(value));
1453 filter.setFilterIfMissing(true);
1454 allFilters.addFilter(filter);
1455 Scan scan = new Scan();
1456 scan.addFamily(Bytes.toBytes("trans-blob"));
1457 scan.addFamily(Bytes.toBytes("trans-type"));
1458 scan.addFamily(Bytes.toBytes("trans-date"));
1459 scan.addFamily(Bytes.toBytes("trans-tags"));
1460 scan.addFamily(Bytes.toBytes("trans-group"));
1461 scan.setFilter(allFilters);
1462 return r.getScanner(scan);
1463 }
1464
1465 private void putRows(HRegion r, int numRows, String value, String key) throws IOException {
1466 for (int i = 0; i < numRows; i++) {
1467 String row = key + "_" + i
1468 System.out.println(String.format("Saving row: %s, with value %s", row, value));
1469 Put put = new Put(Bytes.toBytes(row));
1470 put.setDurability(Durability.SKIP_WAL);
1471 put.add(Bytes.toBytes("trans-blob"), null, Bytes.toBytes("value for blob"));
1472 put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
1473 put.add(Bytes.toBytes("trans-date"), null, Bytes.toBytes("20090921010101999"));
1474 put.add(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"), Bytes.toBytes(value));
1475 put.add(Bytes.toBytes("trans-group"), null, Bytes.toBytes("adhocTransactionGroupId"));
1476 r.put(put);
1477 }
1478 }
1479
1480 @Test
1481 public void testFamilyWithAndWithoutColon() throws Exception {
1482 byte[] b = Bytes.toBytes(getName());
1483 byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
1484 this.region = initHRegion(b, getName(), CONF, cf);
1485 Put p = new Put(b);
1486 byte[] cfwithcolon = Bytes.toBytes(COLUMN_FAMILY + ":");
1487 p.add(cfwithcolon, cfwithcolon, cfwithcolon);
1488 boolean exception = false;
1489 try {
1490 this.region.put(p);
1491 } catch (NoSuchColumnFamilyException e) {
1492 exception = true;
1493 }
1494 assertTrue(exception);
1495 }
1496
1497 @Test
1498 public void testBatchPut_whileNoRowLocksHeld() throws IOException {
1499 byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
1500 byte[] qual = Bytes.toBytes("qual");
1501 byte[] val = Bytes.toBytes("val");
1502 this.region = initHRegion(Bytes.toBytes(getName()), getName(), CONF, cf);
1503 MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
1504 long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
1505 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
1506
1507 LOG.info("First a batch put with all valid puts");
1508 final Put[] puts = new Put[10];
1509 for (int i = 0; i < 10; i++) {
1510 puts[i] = new Put(Bytes.toBytes("row_" + i));
1511 puts[i].add(cf, qual, val);
1512 }
1513
1514 OperationStatus[] codes = this.region.batchMutate(puts);
1515 assertEquals(10, codes.length);
1516 for (int i = 0; i < 10; i++) {
1517 assertEquals(OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
1518 }
1519 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 1, source);
1520
1521 LOG.info("Next a batch put with one invalid family");
1522 puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
1523 codes = this.region.batchMutate(puts);
1524 assertEquals(10, codes.length);
1525 for (int i = 0; i < 10; i++) {
1526 assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY : OperationStatusCode.SUCCESS,
1527 codes[i].getOperationStatusCode());
1528 }
1529
1530 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 2, source);
1531 }
1532
1533 @Test
1534 public void testBatchPut_whileMultipleRowLocksHeld() throws Exception {
1535 byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
1536 byte[] qual = Bytes.toBytes("qual");
1537 byte[] val = Bytes.toBytes("val");
1538 this.region = initHRegion(Bytes.toBytes(getName()), getName(), CONF, cf);
1539 MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
1540 long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
1541 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
1542
1543 final Put[] puts = new Put[10];
1544 for (int i = 0; i < 10; i++) {
1545 puts[i] = new Put(Bytes.toBytes("row_" + i));
1546 puts[i].add(cf, qual, val);
1547 }
1548 puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
1549
1550 LOG.info("batchPut will have to break into four batches to avoid row locks");
1551 RowLock rowLock1 = region.getRowLock(Bytes.toBytes("row_2"));
1552 RowLock rowLock2 = region.getRowLock(Bytes.toBytes("row_1"));
1553 RowLock rowLock3 = region.getRowLock(Bytes.toBytes("row_3"));
1554 RowLock rowLock4 = region.getRowLock(Bytes.toBytes("row_3"), true);
1555
1556
1557 MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(CONF);
1558 final AtomicReference<OperationStatus[]> retFromThread = new AtomicReference<>();
1559 final CountDownLatch startingPuts = new CountDownLatch(1);
1560 final CountDownLatch startingClose = new CountDownLatch(1);
1561 TestThread putter = new TestThread(ctx) {
1562 @Override
1563 public void doWork() throws IOException {
1564 startingPuts.countDown();
1565 retFromThread.set(region.batchMutate(puts));
1566 }
1567 };
1568 LOG.info("...starting put thread while holding locks");
1569 ctx.addThread(putter);
1570 ctx.startThreads();
1571
1572
1573
1574
1575
1576
1577 Thread regionCloseThread = new TestThread(ctx) {
1578 @Override
1579 public void doWork() {
1580 try {
1581 startingPuts.await();
1582
1583
1584 Thread.sleep(10);
1585 startingClose.countDown();
1586 HBaseTestingUtility.closeRegionAndWAL(region);
1587 region = null;
1588 } catch (IOException e) {
1589 throw new RuntimeException(e);
1590 } catch (InterruptedException e) {
1591 throw new RuntimeException(e);
1592 }
1593 }
1594 };
1595 regionCloseThread.start();
1596
1597 startingClose.await();
1598 startingPuts.await();
1599 Thread.sleep(100);
1600 LOG.info("...releasing row lock 1, which should let put thread continue");
1601 rowLock1.release();
1602 rowLock2.release();
1603 rowLock3.release();
1604 waitForCounter(source, "syncTimeNumOps", syncs + 1);
1605
1606 LOG.info("...joining on put thread");
1607 ctx.stop();
1608 regionCloseThread.join();
1609
1610 OperationStatus[] codes = retFromThread.get();
1611 for (int i = 0; i < codes.length; i++) {
1612 assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY : OperationStatusCode.SUCCESS,
1613 codes[i].getOperationStatusCode());
1614 }
1615 rowLock4.release();
1616 }
1617
1618 private void waitForCounter(MetricsWALSource source, String metricName, long expectedCount)
1619 throws InterruptedException {
1620 long startWait = System.currentTimeMillis();
1621 long currentCount;
1622 while ((currentCount = metricsAssertHelper.getCounter(metricName, source)) < expectedCount) {
1623 Thread.sleep(100);
1624 if (System.currentTimeMillis() - startWait > 10000) {
1625 fail(String.format("Timed out waiting for '%s' >= '%s', currentCount=%s", metricName,
1626 expectedCount, currentCount));
1627 }
1628 }
1629 }
1630
1631 @Test
1632 public void testBatchPutWithTsSlop() throws Exception {
1633 byte[] b = Bytes.toBytes(getName());
1634 byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
1635 byte[] qual = Bytes.toBytes("qual");
1636 byte[] val = Bytes.toBytes("val");
1637
1638
1639 CONF.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
1640 this.region = initHRegion(b, getName(), CONF, cf);
1641
1642 MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
1643 long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
1644 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
1645
1646 final Put[] puts = new Put[10];
1647 for (int i = 0; i < 10; i++) {
1648 puts[i] = new Put(Bytes.toBytes("row_" + i), Long.MAX_VALUE - 100);
1649 puts[i].add(cf, qual, val);
1650 }
1651
1652 OperationStatus[] codes = this.region.batchMutate(puts);
1653 assertEquals(10, codes.length);
1654 for (int i = 0; i < 10; i++) {
1655 assertEquals(OperationStatusCode.SANITY_CHECK_FAILURE, codes[i].getOperationStatusCode());
1656 }
1657 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
1658 }
1659
1660
1661
1662
1663 @Test
1664 public void testCheckAndMutate_WithEmptyRowValue() throws IOException {
1665 byte[] row1 = Bytes.toBytes("row1");
1666 byte[] fam1 = Bytes.toBytes("fam1");
1667 byte[] qf1 = Bytes.toBytes("qualifier");
1668 byte[] emptyVal = new byte[] {};
1669 byte[] val1 = Bytes.toBytes("value1");
1670 byte[] val2 = Bytes.toBytes("value2");
1671
1672
1673 String method = this.getName();
1674 this.region = initHRegion(tableName, method, CONF, fam1);
1675
1676 Put put = new Put(row1);
1677 put.add(fam1, qf1, emptyVal);
1678
1679
1680 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1681 emptyVal), put, true);
1682 assertTrue(res);
1683
1684
1685 put = new Put(row1);
1686 put.add(fam1, qf1, val1);
1687
1688
1689 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1690 put, true);
1691 assertTrue(res);
1692
1693
1694 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1695 put, true);
1696 assertFalse(res);
1697
1698 Delete delete = new Delete(row1);
1699 delete.deleteColumn(fam1, qf1);
1700 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1701 delete, true);
1702 assertFalse(res);
1703
1704 put = new Put(row1);
1705 put.add(fam1, qf1, val2);
1706
1707 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
1708 put, true);
1709 assertTrue(res);
1710
1711
1712 delete = new Delete(row1);
1713 delete.deleteColumn(fam1, qf1);
1714 delete.deleteColumn(fam1, qf1);
1715 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val2),
1716 delete, true);
1717 assertTrue(res);
1718
1719 delete = new Delete(row1);
1720 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1721 delete, true);
1722 assertTrue(res);
1723
1724
1725 put = new Put(row1);
1726 put.add(fam1, qf1, val1);
1727
1728 res = region
1729 .checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new NullComparator(), put, true);
1730 assertTrue(res);
1731 }
1732
1733 @Test
1734 public void testCheckAndMutate_WithWrongValue() throws IOException {
1735 byte[] row1 = Bytes.toBytes("row1");
1736 byte[] fam1 = Bytes.toBytes("fam1");
1737 byte[] qf1 = Bytes.toBytes("qualifier");
1738 byte[] val1 = Bytes.toBytes("value1");
1739 byte[] val2 = Bytes.toBytes("value2");
1740
1741
1742 String method = this.getName();
1743 this.region = initHRegion(tableName, method, CONF, fam1);
1744
1745 Put put = new Put(row1);
1746 put.add(fam1, qf1, val1);
1747 region.put(put);
1748
1749
1750 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1751 val2), put, true);
1752 assertFalse(res);
1753
1754
1755 Delete delete = new Delete(row1);
1756 delete.deleteFamily(fam1);
1757 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val2),
1758 put, true);
1759 assertFalse(res);
1760 }
1761
1762 @Test
1763 public void testCheckAndMutate_WithCorrectValue() throws IOException {
1764 byte[] row1 = Bytes.toBytes("row1");
1765 byte[] fam1 = Bytes.toBytes("fam1");
1766 byte[] qf1 = Bytes.toBytes("qualifier");
1767 byte[] val1 = Bytes.toBytes("value1");
1768
1769
1770 String method = this.getName();
1771 this.region = initHRegion(tableName, method, CONF, fam1);
1772
1773 Put put = new Put(row1);
1774 put.add(fam1, qf1, val1);
1775 region.put(put);
1776
1777
1778 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1779 val1), put, true);
1780 assertTrue( res);
1781
1782
1783 Delete delete = new Delete(row1);
1784 delete.deleteColumn(fam1, qf1);
1785 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
1786 delete, true);
1787 assertTrue(res);
1788 }
1789
1790 @Test
1791 public void testCheckAndMutate_WithNonEqualCompareOp() throws IOException {
1792 byte[] row1 = Bytes.toBytes("row1");
1793 byte[] fam1 = Bytes.toBytes("fam1");
1794 byte[] qf1 = Bytes.toBytes("qualifier");
1795 byte[] val1 = Bytes.toBytes("value1");
1796 byte[] val2 = Bytes.toBytes("value2");
1797 byte[] val3 = Bytes.toBytes("value3");
1798 byte[] val4 = Bytes.toBytes("value4");
1799
1800
1801 String method = this.getName();
1802 this.region = initHRegion(tableName, method, CONF, fam1);
1803
1804 Put put = new Put(row1);
1805 put.add(fam1, qf1, val3);
1806 region.put(put);
1807
1808
1809 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
1810 new BinaryComparator(val3), put, true);
1811 assertEquals(false, res);
1812
1813
1814 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
1815 new BinaryComparator(val4), put, true);
1816 assertEquals(false, res);
1817
1818
1819
1820 put = new Put(row1);
1821 put.add(fam1, qf1, val2);
1822 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
1823 new BinaryComparator(val2), put, true);
1824 assertEquals(true, res);
1825
1826
1827 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
1828 new BinaryComparator(val3), put, true);
1829 assertEquals(false, res);
1830
1831
1832
1833 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
1834 new BinaryComparator(val2), put, true);
1835 assertEquals(true, res);
1836
1837
1838
1839 put = new Put(row1);
1840 put.add(fam1, qf1, val3);
1841 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
1842 new BinaryComparator(val1), put, true);
1843 assertEquals(true, res);
1844
1845
1846 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
1847 new BinaryComparator(val3), put, true);
1848 assertEquals(false, res);
1849
1850
1851 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
1852 new BinaryComparator(val2), put, true);
1853 assertEquals(false, res);
1854
1855
1856
1857 put = new Put(row1);
1858 put.add(fam1, qf1, val2);
1859 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
1860 new BinaryComparator(val4), put, true);
1861 assertEquals(true, res);
1862
1863
1864 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
1865 new BinaryComparator(val1), put, true);
1866 assertEquals(false, res);
1867
1868
1869
1870 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
1871 new BinaryComparator(val2), put, true);
1872 assertEquals(true, res);
1873
1874
1875 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
1876 new BinaryComparator(val3), put, true);
1877 assertEquals(true, res);
1878 }
1879
1880 @Test
1881 public void testCheckAndPut_ThatPutWasWritten() throws IOException {
1882 byte[] row1 = Bytes.toBytes("row1");
1883 byte[] fam1 = Bytes.toBytes("fam1");
1884 byte[] fam2 = Bytes.toBytes("fam2");
1885 byte[] qf1 = Bytes.toBytes("qualifier");
1886 byte[] val1 = Bytes.toBytes("value1");
1887 byte[] val2 = Bytes.toBytes("value2");
1888
1889 byte[][] families = { fam1, fam2 };
1890
1891
1892 String method = this.getName();
1893 this.region = initHRegion(tableName, method, CONF, families);
1894
1895 Put put = new Put(row1);
1896 put.add(fam1, qf1, val1);
1897 region.put(put);
1898
1899
1900 long ts = System.currentTimeMillis();
1901 KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2);
1902 put = new Put(row1);
1903 put.add(kv);
1904
1905
1906 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1907 val1), put, true);
1908 assertEquals(true, res);
1909
1910 Get get = new Get(row1);
1911 get.addColumn(fam2, qf1);
1912 Cell[] actual = region.get(get).rawCells();
1913
1914 Cell[] expected = { kv };
1915
1916 assertEquals(expected.length, actual.length);
1917 for (int i = 0; i < actual.length; i++) {
1918 assertEquals(expected[i], actual[i]);
1919 }
1920 }
1921
1922 @Test
1923 public void testCheckAndPut_wrongRowInPut() throws IOException {
1924 TableName tableName = TableName.valueOf(name.getMethodName());
1925 this.region = initHRegion(tableName, this.getName(), CONF, COLUMNS);
1926 Put put = new Put(row2);
1927 put.add(fam1, qual1, value1);
1928 try {
1929 region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL,
1930 new BinaryComparator(value2), put, false);
1931 fail();
1932 } catch (org.apache.hadoop.hbase.DoNotRetryIOException expected) {
1933
1934 }
1935 }
1936
1937 @Test
1938 public void testCheckAndDelete_ThatDeleteWasWritten() throws IOException {
1939 byte[] row1 = Bytes.toBytes("row1");
1940 byte[] fam1 = Bytes.toBytes("fam1");
1941 byte[] fam2 = Bytes.toBytes("fam2");
1942 byte[] qf1 = Bytes.toBytes("qualifier1");
1943 byte[] qf2 = Bytes.toBytes("qualifier2");
1944 byte[] qf3 = Bytes.toBytes("qualifier3");
1945 byte[] val1 = Bytes.toBytes("value1");
1946 byte[] val2 = Bytes.toBytes("value2");
1947 byte[] val3 = Bytes.toBytes("value3");
1948 byte[] emptyVal = new byte[] {};
1949
1950 byte[][] families = { fam1, fam2 };
1951
1952
1953 String method = this.getName();
1954 this.region = initHRegion(tableName, method, CONF, families);
1955
1956 Put put = new Put(row1);
1957 put.add(fam1, qf1, val1);
1958 region.put(put);
1959 Threads.sleep(2);
1960
1961 put = new Put(row1);
1962 put.add(fam1, qf1, val2);
1963 put.add(fam2, qf1, val3);
1964 put.add(fam2, qf2, val2);
1965 put.add(fam2, qf3, val1);
1966 put.add(fam1, qf3, val1);
1967 region.put(put);
1968
1969
1970 Delete delete = new Delete(row1);
1971 delete.deleteColumn(fam1, qf1);
1972 delete.deleteColumn(fam2, qf1);
1973 delete.deleteColumn(fam1, qf3);
1974 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1975 val2), delete, true);
1976 assertEquals(true, res);
1977
1978 Get get = new Get(row1);
1979 get.addColumn(fam1, qf1);
1980 get.addColumn(fam1, qf3);
1981 get.addColumn(fam2, qf2);
1982 Result r = region.get(get);
1983 assertEquals(2, r.size());
1984 assertArrayEquals(val1, r.getValue(fam1, qf1));
1985 assertArrayEquals(val2, r.getValue(fam2, qf2));
1986
1987
1988 delete = new Delete(row1);
1989 delete.deleteFamily(fam2);
1990 res = region.checkAndMutate(row1, fam2, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1991 delete, true);
1992 assertEquals(true, res);
1993
1994 get = new Get(row1);
1995 r = region.get(get);
1996 assertEquals(1, r.size());
1997 assertArrayEquals(val1, r.getValue(fam1, qf1));
1998
1999
2000 delete = new Delete(row1);
2001 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
2002 delete, true);
2003 assertEquals(true, res);
2004 get = new Get(row1);
2005 r = region.get(get);
2006 assertEquals(0, r.size());
2007 }
2008
2009
2010
2011
2012 @Test
2013 public void testDelete_multiDeleteColumn() throws IOException {
2014 byte[] row1 = Bytes.toBytes("row1");
2015 byte[] fam1 = Bytes.toBytes("fam1");
2016 byte[] qual = Bytes.toBytes("qualifier");
2017 byte[] value = Bytes.toBytes("value");
2018
2019 Put put = new Put(row1);
2020 put.add(fam1, qual, 1, value);
2021 put.add(fam1, qual, 2, value);
2022
2023 String method = this.getName();
2024 this.region = initHRegion(tableName, method, CONF, fam1);
2025 region.put(put);
2026
2027
2028 Delete delete = new Delete(row1);
2029 delete.deleteColumn(fam1, qual);
2030 delete.deleteColumn(fam1, qual);
2031 region.delete(delete);
2032
2033 Get get = new Get(row1);
2034 get.addFamily(fam1);
2035 Result r = region.get(get);
2036 assertEquals(0, r.size());
2037 }
2038
2039 @Test
2040 public void testDelete_CheckFamily() throws IOException {
2041 byte[] row1 = Bytes.toBytes("row1");
2042 byte[] fam1 = Bytes.toBytes("fam1");
2043 byte[] fam2 = Bytes.toBytes("fam2");
2044 byte[] fam3 = Bytes.toBytes("fam3");
2045 byte[] fam4 = Bytes.toBytes("fam4");
2046
2047
2048 String method = this.getName();
2049 this.region = initHRegion(tableName, method, CONF, fam1, fam2, fam3);
2050 List<Cell> kvs = new ArrayList<Cell>();
2051 kvs.add(new KeyValue(row1, fam4, null, null));
2052
2053
2054 byte[] family = fam2;
2055 try {
2056 NavigableMap<byte[], List<Cell>> deleteMap = new TreeMap<byte[], List<Cell>>(
2057 Bytes.BYTES_COMPARATOR);
2058 deleteMap.put(family, kvs);
2059 region.delete(deleteMap, Durability.SYNC_WAL);
2060 } catch (Exception e) {
2061 assertTrue("Family " + new String(family) + " does not exist", false);
2062 }
2063
2064
2065 boolean ok = false;
2066 family = fam4;
2067 try {
2068 NavigableMap<byte[], List<Cell>> deleteMap = new TreeMap<byte[], List<Cell>>(
2069 Bytes.BYTES_COMPARATOR);
2070 deleteMap.put(family, kvs);
2071 region.delete(deleteMap, Durability.SYNC_WAL);
2072 } catch (Exception e) {
2073 ok = true;
2074 }
2075 assertTrue("Family " + new String(family) + " does exist", ok);
2076 }
2077
2078 @Test
2079 public void testDelete_mixed() throws IOException, InterruptedException {
2080 byte[] fam = Bytes.toBytes("info");
2081 byte[][] families = { fam };
2082 String method = this.getName();
2083 this.region = initHRegion(tableName, method, CONF, families);
2084 EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
2085
2086 byte[] row = Bytes.toBytes("table_name");
2087
2088 byte[] serverinfo = Bytes.toBytes("serverinfo");
2089 byte[] splitA = Bytes.toBytes("splitA");
2090 byte[] splitB = Bytes.toBytes("splitB");
2091
2092
2093 Put put = new Put(row);
2094 put.add(fam, splitA, Bytes.toBytes("reference_A"));
2095 region.put(put);
2096
2097 put = new Put(row);
2098 put.add(fam, splitB, Bytes.toBytes("reference_B"));
2099 region.put(put);
2100
2101 put = new Put(row);
2102 put.add(fam, serverinfo, Bytes.toBytes("ip_address"));
2103 region.put(put);
2104
2105
2106 Delete delete = new Delete(row);
2107 delete.deleteColumns(fam, splitA);
2108 region.delete(delete);
2109
2110
2111 Get get = new Get(row).addColumn(fam, serverinfo);
2112 Result result = region.get(get);
2113 assertEquals(1, result.size());
2114
2115 get = new Get(row).addColumn(fam, splitA);
2116 result = region.get(get);
2117 assertEquals(0, result.size());
2118
2119 get = new Get(row).addColumn(fam, splitB);
2120 result = region.get(get);
2121 assertEquals(1, result.size());
2122
2123
2124 put = new Put(row);
2125 put.add(fam, splitA, Bytes.toBytes("reference_A"));
2126 region.put(put);
2127 get = new Get(row);
2128 result = region.get(get);
2129 assertEquals(3, result.size());
2130
2131
2132 delete = new Delete(row);
2133 region.delete(delete);
2134 assertEquals(0, region.get(get).size());
2135
2136 region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
2137 result = region.get(get);
2138 assertEquals(1, result.size());
2139 }
2140
2141 @Test
2142 public void testDeleteRowWithFutureTs() throws IOException {
2143 byte[] fam = Bytes.toBytes("info");
2144 byte[][] families = { fam };
2145 String method = this.getName();
2146 this.region = initHRegion(tableName, method, CONF, families);
2147 byte[] row = Bytes.toBytes("table_name");
2148
2149 byte[] serverinfo = Bytes.toBytes("serverinfo");
2150
2151
2152 Put put = new Put(row);
2153 put.add(fam, serverinfo, HConstants.LATEST_TIMESTAMP - 5, Bytes.toBytes("value"));
2154 region.put(put);
2155
2156
2157 Delete delete = new Delete(row);
2158 region.delete(delete);
2159
2160
2161 Get get = new Get(row).addColumn(fam, serverinfo);
2162 Result result = region.get(get);
2163 assertEquals(1, result.size());
2164
2165
2166 delete = new Delete(row, HConstants.LATEST_TIMESTAMP - 3);
2167 region.delete(delete);
2168
2169
2170 get = new Get(row).addColumn(fam, serverinfo);
2171 result = region.get(get);
2172 assertEquals(0, result.size());
2173 }
2174
2175
2176
2177
2178
2179 @Test
2180 public void testPutWithLatestTS() throws IOException {
2181 byte[] fam = Bytes.toBytes("info");
2182 byte[][] families = { fam };
2183 String method = this.getName();
2184 this.region = initHRegion(tableName, method, CONF, families);
2185 byte[] row = Bytes.toBytes("row1");
2186
2187 byte[] qual = Bytes.toBytes("qual");
2188
2189
2190 Put put = new Put(row);
2191 put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
2192 region.put(put);
2193
2194
2195 Get get = new Get(row).addColumn(fam, qual);
2196 Result result = region.get(get);
2197 assertEquals(1, result.size());
2198 Cell kv = result.rawCells()[0];
2199 LOG.info("Got: " + kv);
2200 assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
2201 kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
2202
2203
2204
2205 row = Bytes.toBytes("row2");
2206 put = new Put(row);
2207 put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
2208 region.put(put);
2209
2210
2211 get = new Get(row).addColumn(fam, qual);
2212 result = region.get(get);
2213 assertEquals(1, result.size());
2214 kv = result.rawCells()[0];
2215 LOG.info("Got: " + kv);
2216 assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
2217 kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
2218 }
2219
2220
2221
2222
2223
2224
2225 @Test
2226 public void testPutWithTsSlop() throws IOException {
2227 byte[] fam = Bytes.toBytes("info");
2228 byte[][] families = { fam };
2229 String method = this.getName();
2230
2231
2232 CONF.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
2233 this.region = initHRegion(tableName, method, CONF, families);
2234 boolean caughtExcep = false;
2235 try {
2236
2237 region.put(new Put(row).add(fam, Bytes.toBytes("qual"), Bytes.toBytes("value")));
2238
2239 region.put(new Put(row).add(fam, Bytes.toBytes("qual"), System.currentTimeMillis() + 2000,
2240 Bytes.toBytes("value")));
2241 fail("Expected IOE for TS out of configured timerange");
2242 } catch (FailedSanityCheckException ioe) {
2243 LOG.debug("Received expected exception", ioe);
2244 caughtExcep = true;
2245 }
2246 assertTrue("Should catch FailedSanityCheckException", caughtExcep);
2247 }
2248
2249 @Test
2250 public void testScanner_DeleteOneFamilyNotAnother() throws IOException {
2251 byte[] fam1 = Bytes.toBytes("columnA");
2252 byte[] fam2 = Bytes.toBytes("columnB");
2253 this.region = initHRegion(tableName, getName(), CONF, fam1, fam2);
2254 byte[] rowA = Bytes.toBytes("rowA");
2255 byte[] rowB = Bytes.toBytes("rowB");
2256
2257 byte[] value = Bytes.toBytes("value");
2258
2259 Delete delete = new Delete(rowA);
2260 delete.deleteFamily(fam1);
2261
2262 region.delete(delete);
2263
2264
2265 Put put = new Put(rowA);
2266 put.add(fam2, null, value);
2267 region.put(put);
2268
2269 put = new Put(rowB);
2270 put.add(fam1, null, value);
2271 put.add(fam2, null, value);
2272 region.put(put);
2273
2274 Scan scan = new Scan();
2275 scan.addFamily(fam1).addFamily(fam2);
2276 InternalScanner s = region.getScanner(scan);
2277 List<Cell> results = new ArrayList<Cell>();
2278 s.next(results);
2279 assertTrue(CellUtil.matchingRow(results.get(0), rowA));
2280
2281 results.clear();
2282 s.next(results);
2283 assertTrue(CellUtil.matchingRow(results.get(0), rowB));
2284 }
2285
2286 @Test
2287 public void testDataInMemoryWithoutWAL() throws IOException {
2288 FileSystem fs = FileSystem.get(CONF);
2289 Path rootDir = new Path(dir + "testDataInMemoryWithoutWAL");
2290 FSHLog hLog = new FSHLog(fs, rootDir, "testDataInMemoryWithoutWAL", CONF);
2291 region = initHRegion(tableName, null, null, name.getMethodName(),
2292 CONF, false, Durability.SYNC_WAL, hLog, COLUMN_FAMILY_BYTES);
2293
2294 Cell originalCell = CellUtil.createCell(row, COLUMN_FAMILY_BYTES, qual1,
2295 System.currentTimeMillis(), KeyValue.Type.Put.getCode(), value1);
2296 final long originalSize = KeyValueUtil.length(originalCell);
2297
2298 Cell addCell = CellUtil.createCell(row, COLUMN_FAMILY_BYTES, qual1,
2299 System.currentTimeMillis(), KeyValue.Type.Put.getCode(), Bytes.toBytes("xxxxxxxxxx"));
2300 final long addSize = KeyValueUtil.length(addCell);
2301
2302 LOG.info("originalSize:" + originalSize
2303 + ", addSize:" + addSize);
2304
2305
2306
2307
2308 testDataInMemoryWithoutWAL(region,
2309 new Put(row).add(originalCell).setDurability(Durability.SKIP_WAL),
2310 new Put(row).add(addCell).setDurability(Durability.SKIP_WAL),
2311 originalSize + addSize);
2312
2313
2314 testDataInMemoryWithoutWAL(region,
2315 new Put(row).add(originalCell).setDurability(Durability.SKIP_WAL),
2316 new Put(row).add(addCell).setDurability(Durability.SYNC_WAL),
2317 originalSize + addSize);
2318
2319
2320 testDataInMemoryWithoutWAL(region,
2321 new Put(row).add(originalCell).setDurability(Durability.SYNC_WAL),
2322 new Put(row).add(addCell).setDurability(Durability.SKIP_WAL),
2323 0);
2324
2325
2326 testDataInMemoryWithoutWAL(region,
2327 new Put(row).add(originalCell).setDurability(Durability.SYNC_WAL),
2328 new Put(row).add(addCell).setDurability(Durability.SYNC_WAL),
2329 0);
2330 }
2331
2332 private static void testDataInMemoryWithoutWAL(HRegion region, Put originalPut,
2333 final Put addPut, long delta) throws IOException {
2334 final long initSize = region.getDataInMemoryWithoutWAL();
2335
2336 RegionCoprocessorHost normalCPHost = region.getCoprocessorHost();
2337 RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class);
2338 Answer<Boolean> answer = new Answer<Boolean>() {
2339 @Override
2340 public Boolean answer(InvocationOnMock invocation) throws Throwable {
2341 MiniBatchOperationInProgress<Mutation> mb = invocation.getArgumentAt(0,
2342 MiniBatchOperationInProgress.class);
2343 mb.addOperationsFromCP(0, new Mutation[]{addPut});
2344 return false;
2345 }
2346 };
2347 when(mockedCPHost.preBatchMutate(Mockito.isA(MiniBatchOperationInProgress.class)))
2348 .then(answer);
2349 region.setCoprocessorHost(mockedCPHost);
2350 region.put(originalPut);
2351 region.setCoprocessorHost(normalCPHost);
2352 final long finalSize = region.getDataInMemoryWithoutWAL();
2353 assertEquals("finalSize:" + finalSize + ", initSize:"
2354 + initSize + ", delta:" + delta,finalSize, initSize + delta);
2355 }
2356
2357 @Test
2358 public void testDeleteColumns_PostInsert() throws IOException, InterruptedException {
2359 Delete delete = new Delete(row);
2360 delete.deleteColumns(fam1, qual1);
2361 doTestDelete_AndPostInsert(delete);
2362 }
2363
2364 @Test
2365 public void testDeleteFamily_PostInsert() throws IOException, InterruptedException {
2366 Delete delete = new Delete(row);
2367 delete.deleteFamily(fam1);
2368 doTestDelete_AndPostInsert(delete);
2369 }
2370
2371 public void doTestDelete_AndPostInsert(Delete delete) throws IOException, InterruptedException {
2372 TableName tableName = TableName.valueOf(name.getMethodName());
2373 this.region = initHRegion(tableName, getName(), CONF, fam1);
2374 EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
2375 Put put = new Put(row);
2376 put.add(fam1, qual1, value1);
2377 region.put(put);
2378
2379
2380 region.delete(delete);
2381
2382
2383 put = new Put(row);
2384 put.add(fam1, qual1, value2);
2385 region.put(put);
2386
2387
2388 Get get = new Get(row);
2389 get.addColumn(fam1, qual1);
2390
2391 Result r = region.get(get);
2392 assertEquals(1, r.size());
2393 assertArrayEquals(value2, r.getValue(fam1, qual1));
2394
2395
2396 Scan scan = new Scan(row);
2397 scan.addColumn(fam1, qual1);
2398 InternalScanner s = region.getScanner(scan);
2399
2400 List<Cell> results = new ArrayList<Cell>();
2401 assertEquals(false, s.next(results));
2402 assertEquals(1, results.size());
2403 Cell kv = results.get(0);
2404
2405 assertArrayEquals(value2, CellUtil.cloneValue(kv));
2406 assertArrayEquals(fam1, CellUtil.cloneFamily(kv));
2407 assertArrayEquals(qual1, CellUtil.cloneQualifier(kv));
2408 assertArrayEquals(row, CellUtil.cloneRow(kv));
2409 }
2410
2411 @Test
2412 public void testDelete_CheckTimestampUpdated() throws IOException {
2413 TableName tableName = TableName.valueOf(name.getMethodName());
2414 byte[] row1 = Bytes.toBytes("row1");
2415 byte[] col1 = Bytes.toBytes("col1");
2416 byte[] col2 = Bytes.toBytes("col2");
2417 byte[] col3 = Bytes.toBytes("col3");
2418
2419
2420 String method = this.getName();
2421 this.region = initHRegion(tableName, method, CONF, fam1);
2422
2423 List<Cell> kvs = new ArrayList<Cell>();
2424 kvs.add(new KeyValue(row1, fam1, col1, null));
2425 kvs.add(new KeyValue(row1, fam1, col2, null));
2426 kvs.add(new KeyValue(row1, fam1, col3, null));
2427
2428 NavigableMap<byte[], List<Cell>> deleteMap = new TreeMap<byte[], List<Cell>>(
2429 Bytes.BYTES_COMPARATOR);
2430 deleteMap.put(fam1, kvs);
2431 region.delete(deleteMap, Durability.SYNC_WAL);
2432
2433
2434
2435 long now = System.currentTimeMillis();
2436 DefaultMemStore memstore = (DefaultMemStore) ((HStore) region.getStore(fam1)).memstore;
2437 Cell firstCell = memstore.activeSection.getCellSkipListSet().first();
2438 assertTrue(firstCell.getTimestamp() <= now);
2439 now = firstCell.getTimestamp();
2440 for (Cell cell : memstore.activeSection.getCellSkipListSet()) {
2441 assertTrue(cell.getTimestamp() <= now);
2442 now = cell.getTimestamp();
2443 }
2444 }
2445
2446
2447
2448
2449 @Test
2450 public void testGet_FamilyChecker() throws IOException {
2451 byte[] row1 = Bytes.toBytes("row1");
2452 byte[] fam1 = Bytes.toBytes("fam1");
2453 byte[] fam2 = Bytes.toBytes("False");
2454 byte[] col1 = Bytes.toBytes("col1");
2455
2456
2457 String method = this.getName();
2458 this.region = initHRegion(tableName, method, CONF, fam1);
2459 Get get = new Get(row1);
2460 get.addColumn(fam2, col1);
2461
2462
2463 try {
2464 region.get(get);
2465 fail("Expecting DoNotRetryIOException in get but did not get any");
2466 } catch (org.apache.hadoop.hbase.DoNotRetryIOException e) {
2467 LOG.info("Got expected DoNotRetryIOException successfully");
2468 }
2469 }
2470
2471 @Test
2472 public void testGet_Basic() throws IOException {
2473 byte[] row1 = Bytes.toBytes("row1");
2474 byte[] fam1 = Bytes.toBytes("fam1");
2475 byte[] col1 = Bytes.toBytes("col1");
2476 byte[] col2 = Bytes.toBytes("col2");
2477 byte[] col3 = Bytes.toBytes("col3");
2478 byte[] col4 = Bytes.toBytes("col4");
2479 byte[] col5 = Bytes.toBytes("col5");
2480
2481
2482 String method = this.getName();
2483 this.region = initHRegion(tableName, method, CONF, fam1);
2484
2485 Put put = new Put(row1);
2486 put.add(fam1, col1, null);
2487 put.add(fam1, col2, null);
2488 put.add(fam1, col3, null);
2489 put.add(fam1, col4, null);
2490 put.add(fam1, col5, null);
2491 region.put(put);
2492
2493 Get get = new Get(row1);
2494 get.addColumn(fam1, col2);
2495 get.addColumn(fam1, col4);
2496
2497 KeyValue kv1 = new KeyValue(row1, fam1, col2);
2498 KeyValue kv2 = new KeyValue(row1, fam1, col4);
2499 KeyValue[] expected = { kv1, kv2 };
2500
2501
2502 Result res = region.get(get);
2503 assertEquals(expected.length, res.size());
2504 for (int i = 0; i < res.size(); i++) {
2505 assertTrue(CellUtil.matchingRow(expected[i], res.rawCells()[i]));
2506 assertTrue(CellUtil.matchingFamily(expected[i], res.rawCells()[i]));
2507 assertTrue(CellUtil.matchingQualifier(expected[i], res.rawCells()[i]));
2508 }
2509
2510
2511 Get g = new Get(row1);
2512 final int count = 2;
2513 g.setFilter(new ColumnCountGetFilter(count));
2514 res = region.get(g);
2515 assertEquals(count, res.size());
2516 }
2517
2518 @Test
2519 public void testGet_Empty() throws IOException {
2520 byte[] row = Bytes.toBytes("row");
2521 byte[] fam = Bytes.toBytes("fam");
2522
2523 String method = this.getName();
2524 this.region = initHRegion(tableName, method, CONF, fam);
2525 Get get = new Get(row);
2526 get.addFamily(fam);
2527 Result r = region.get(get);
2528
2529 assertTrue(r.isEmpty());
2530 }
2531
2532
2533
2534
2535 @Test
2536 public void testMerge() throws IOException {
2537 byte[][] families = { fam1, fam2, fam3 };
2538 Configuration hc = initSplit();
2539
2540 String method = this.getName();
2541 this.region = initHRegion(tableName, method, hc, families);
2542 LOG.info("" + HBaseTestCase.addContent(region, fam3));
2543 region.flush(true);
2544 region.compactStores();
2545 region.waitForFlushesAndCompactions();
2546 for(Store s:region.getStores()) {
2547 s.closeAndArchiveCompactedFiles();
2548 }
2549 byte[] splitRow = region.checkSplit();
2550 assertNotNull(splitRow);
2551 LOG.info("SplitRow: " + Bytes.toString(splitRow));
2552 HRegion[] subregions = splitRegion(region, splitRow);
2553 try {
2554
2555 for (int i = 0; i < subregions.length; i++) {
2556 HRegion.openHRegion(subregions[i], null);
2557 subregions[i].compactStores();
2558 subregions[i].waitForFlushesAndCompactions();
2559 for(Store s:subregions[i].getStores()) {
2560 s.closeAndArchiveCompactedFiles();
2561 }
2562 }
2563 Path oldRegionPath = region.getRegionFileSystem().getRegionDir();
2564 Path oldRegion1 = subregions[0].getRegionFileSystem().getRegionDir();
2565 Path oldRegion2 = subregions[1].getRegionFileSystem().getRegionDir();
2566 long startTime = System.currentTimeMillis();
2567 region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
2568 LOG.info("Merge regions elapsed time: "
2569 + ((System.currentTimeMillis() - startTime) / 1000.0));
2570 FILESYSTEM.delete(oldRegion1, true);
2571 FILESYSTEM.delete(oldRegion2, true);
2572 FILESYSTEM.delete(oldRegionPath, true);
2573 LOG.info("splitAndMerge completed.");
2574 } finally {
2575 for (int i = 0; i < subregions.length; i++) {
2576 try {
2577 HRegion.closeHRegion(subregions[i]);
2578 } catch (IOException e) {
2579
2580 }
2581 }
2582 }
2583 }
2584
2585
2586
2587
2588
2589
2590
2591
2592
2593 HRegion[] splitRegion(final HRegion parent, final byte[] midkey) throws IOException {
2594 PairOfSameType<Region> result = null;
2595 SplitTransactionImpl st = new SplitTransactionImpl(parent, midkey);
2596
2597
2598 if (!st.prepare()) {
2599 parent.clearSplit();
2600 return null;
2601 }
2602 try {
2603 result = st.execute(null, null);
2604 } catch (IOException ioe) {
2605 try {
2606 LOG.info("Running rollback of failed split of " +
2607 parent.getRegionInfo().getRegionNameAsString() + "; " + ioe.getMessage());
2608 st.rollback(null, null);
2609 LOG.info("Successful rollback of failed split of " +
2610 parent.getRegionInfo().getRegionNameAsString());
2611 return null;
2612 } catch (RuntimeException e) {
2613
2614 LOG.info("Failed rollback of failed split of " +
2615 parent.getRegionInfo().getRegionNameAsString() + " -- aborting server", e);
2616 }
2617 }
2618 finally {
2619 parent.clearSplit();
2620 }
2621
2622 assertNotNull(result);
2623 return new HRegion[] { (HRegion)result.getFirst(), (HRegion)result.getSecond() };
2624 }
2625
2626
2627
2628
2629 @Test
2630 public void testGetScanner_WithOkFamilies() throws IOException {
2631 byte[] fam1 = Bytes.toBytes("fam1");
2632 byte[] fam2 = Bytes.toBytes("fam2");
2633
2634 byte[][] families = { fam1, fam2 };
2635
2636
2637 String method = this.getName();
2638 this.region = initHRegion(tableName, method, CONF, families);
2639 Scan scan = new Scan();
2640 scan.addFamily(fam1);
2641 scan.addFamily(fam2);
2642 try {
2643 region.getScanner(scan);
2644 } catch (Exception e) {
2645 assertTrue("Families could not be found in Region", false);
2646 }
2647 }
2648
2649 @Test
2650 public void testGetScanner_WithNotOkFamilies() throws IOException {
2651 byte[] fam1 = Bytes.toBytes("fam1");
2652 byte[] fam2 = Bytes.toBytes("fam2");
2653
2654 byte[][] families = { fam1 };
2655
2656
2657 String method = this.getName();
2658 this.region = initHRegion(tableName, method, CONF, families);
2659 Scan scan = new Scan();
2660 scan.addFamily(fam2);
2661 boolean ok = false;
2662 try {
2663 region.getScanner(scan);
2664 } catch (Exception e) {
2665 ok = true;
2666 }
2667 assertTrue("Families could not be found in Region", ok);
2668 }
2669
2670 @Test
2671 public void testGetScanner_WithNoFamilies() throws IOException {
2672 byte[] row1 = Bytes.toBytes("row1");
2673 byte[] fam1 = Bytes.toBytes("fam1");
2674 byte[] fam2 = Bytes.toBytes("fam2");
2675 byte[] fam3 = Bytes.toBytes("fam3");
2676 byte[] fam4 = Bytes.toBytes("fam4");
2677
2678 byte[][] families = { fam1, fam2, fam3, fam4 };
2679
2680
2681 String method = this.getName();
2682 this.region = initHRegion(tableName, method, CONF, families);
2683
2684 Put put = new Put(row1);
2685 put.add(fam1, null, null);
2686 put.add(fam2, null, null);
2687 put.add(fam3, null, null);
2688 put.add(fam4, null, null);
2689 region.put(put);
2690
2691 Scan scan = null;
2692 HRegion.RegionScannerImpl is = null;
2693
2694
2695
2696
2697 scan = new Scan();
2698 scan.addFamily(fam2);
2699 scan.addFamily(fam4);
2700 is = (RegionScannerImpl) region.getScanner(scan);
2701 assertEquals(1, ((RegionScannerImpl) is).storeHeap.getHeap().size());
2702
2703 scan = new Scan();
2704 is = (RegionScannerImpl) region.getScanner(scan);
2705 assertEquals(families.length - 1, ((RegionScannerImpl) is).storeHeap.getHeap().size());
2706 }
2707
2708
2709
2710
2711
2712
2713 @Test
2714 public void testGetScanner_WithRegionClosed() throws IOException {
2715 byte[] fam1 = Bytes.toBytes("fam1");
2716 byte[] fam2 = Bytes.toBytes("fam2");
2717
2718 byte[][] families = { fam1, fam2 };
2719
2720
2721 String method = this.getName();
2722 try {
2723 this.region = initHRegion(tableName, method, CONF, families);
2724 } catch (IOException e) {
2725 e.printStackTrace();
2726 fail("Got IOException during initHRegion, " + e.getMessage());
2727 }
2728 region.closed.set(true);
2729 try {
2730 region.getScanner(new Scan());
2731 fail("Expected to get an exception during getScanner on a region that is closed");
2732 } catch (NotServingRegionException e) {
2733
2734 } catch (IOException e) {
2735 fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
2736 + e.getMessage());
2737 }
2738 }
2739
2740 @Test
2741 public void testRegionScanner_Next() throws IOException {
2742 byte[] row1 = Bytes.toBytes("row1");
2743 byte[] row2 = Bytes.toBytes("row2");
2744 byte[] fam1 = Bytes.toBytes("fam1");
2745 byte[] fam2 = Bytes.toBytes("fam2");
2746 byte[] fam3 = Bytes.toBytes("fam3");
2747 byte[] fam4 = Bytes.toBytes("fam4");
2748
2749 byte[][] families = { fam1, fam2, fam3, fam4 };
2750 long ts = System.currentTimeMillis();
2751
2752
2753 String method = this.getName();
2754 this.region = initHRegion(tableName, method, CONF, families);
2755
2756 Put put = null;
2757 put = new Put(row1);
2758 put.add(fam1, (byte[]) null, ts, null);
2759 put.add(fam2, (byte[]) null, ts, null);
2760 put.add(fam3, (byte[]) null, ts, null);
2761 put.add(fam4, (byte[]) null, ts, null);
2762 region.put(put);
2763
2764 put = new Put(row2);
2765 put.add(fam1, (byte[]) null, ts, null);
2766 put.add(fam2, (byte[]) null, ts, null);
2767 put.add(fam3, (byte[]) null, ts, null);
2768 put.add(fam4, (byte[]) null, ts, null);
2769 region.put(put);
2770
2771 Scan scan = new Scan();
2772 scan.addFamily(fam2);
2773 scan.addFamily(fam4);
2774 InternalScanner is = region.getScanner(scan);
2775
2776 List<Cell> res = null;
2777
2778
2779 List<Cell> expected1 = new ArrayList<Cell>();
2780 expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null));
2781 expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null));
2782
2783 res = new ArrayList<Cell>();
2784 is.next(res);
2785 for (int i = 0; i < res.size(); i++) {
2786 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected1.get(i), res.get(i)));
2787 }
2788
2789
2790 List<Cell> expected2 = new ArrayList<Cell>();
2791 expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null));
2792 expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null));
2793
2794 res = new ArrayList<Cell>();
2795 is.next(res);
2796 for (int i = 0; i < res.size(); i++) {
2797 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected2.get(i), res.get(i)));
2798 }
2799 }
2800
2801 @Test
2802 public void testScanner_ExplicitColumns_FromMemStore_EnforceVersions() throws IOException {
2803 byte[] row1 = Bytes.toBytes("row1");
2804 byte[] qf1 = Bytes.toBytes("qualifier1");
2805 byte[] qf2 = Bytes.toBytes("qualifier2");
2806 byte[] fam1 = Bytes.toBytes("fam1");
2807 byte[][] families = { fam1 };
2808
2809 long ts1 = System.currentTimeMillis();
2810 long ts2 = ts1 + 1;
2811 long ts3 = ts1 + 2;
2812
2813
2814 String method = this.getName();
2815 this.region = initHRegion(tableName, method, CONF, families);
2816
2817 Put put = null;
2818 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2819 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2820 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2821
2822 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2823 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2824 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2825
2826 put = new Put(row1);
2827 put.add(kv13);
2828 put.add(kv12);
2829 put.add(kv11);
2830 put.add(kv23);
2831 put.add(kv22);
2832 put.add(kv21);
2833 region.put(put);
2834
2835
2836 List<Cell> expected = new ArrayList<Cell>();
2837 expected.add(kv13);
2838 expected.add(kv12);
2839
2840 Scan scan = new Scan(row1);
2841 scan.addColumn(fam1, qf1);
2842 scan.setMaxVersions(MAX_VERSIONS);
2843 List<Cell> actual = new ArrayList<Cell>();
2844 InternalScanner scanner = region.getScanner(scan);
2845
2846 boolean hasNext = scanner.next(actual);
2847 assertEquals(false, hasNext);
2848
2849
2850 for (int i = 0; i < expected.size(); i++) {
2851 assertEquals(expected.get(i), actual.get(i));
2852 }
2853 }
2854
2855 @Test
2856 public void testScanner_ExplicitColumns_FromFilesOnly_EnforceVersions() throws IOException {
2857 byte[] row1 = Bytes.toBytes("row1");
2858 byte[] qf1 = Bytes.toBytes("qualifier1");
2859 byte[] qf2 = Bytes.toBytes("qualifier2");
2860 byte[] fam1 = Bytes.toBytes("fam1");
2861 byte[][] families = { fam1 };
2862
2863 long ts1 = 1;
2864 long ts2 = ts1 + 1;
2865 long ts3 = ts1 + 2;
2866
2867
2868 String method = this.getName();
2869 this.region = initHRegion(tableName, method, CONF, families);
2870
2871 Put put = null;
2872 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2873 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2874 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2875
2876 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2877 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2878 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2879
2880 put = new Put(row1);
2881 put.add(kv13);
2882 put.add(kv12);
2883 put.add(kv11);
2884 put.add(kv23);
2885 put.add(kv22);
2886 put.add(kv21);
2887 region.put(put);
2888 region.flush(true);
2889
2890
2891 List<Cell> expected = new ArrayList<Cell>();
2892 expected.add(kv13);
2893 expected.add(kv12);
2894 expected.add(kv23);
2895 expected.add(kv22);
2896
2897 Scan scan = new Scan(row1);
2898 scan.addColumn(fam1, qf1);
2899 scan.addColumn(fam1, qf2);
2900 scan.setMaxVersions(MAX_VERSIONS);
2901 List<Cell> actual = new ArrayList<Cell>();
2902 InternalScanner scanner = region.getScanner(scan);
2903
2904 boolean hasNext = scanner.next(actual);
2905 assertEquals(false, hasNext);
2906
2907
2908 for (int i = 0; i < expected.size(); i++) {
2909 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
2910 }
2911 }
2912
2913 @Test
2914 public void testScanner_ExplicitColumns_FromMemStoreAndFiles_EnforceVersions() throws IOException {
2915 byte[] row1 = Bytes.toBytes("row1");
2916 byte[] fam1 = Bytes.toBytes("fam1");
2917 byte[][] families = { fam1 };
2918 byte[] qf1 = Bytes.toBytes("qualifier1");
2919 byte[] qf2 = Bytes.toBytes("qualifier2");
2920
2921 long ts1 = 1;
2922 long ts2 = ts1 + 1;
2923 long ts3 = ts1 + 2;
2924 long ts4 = ts1 + 3;
2925
2926
2927 String method = this.getName();
2928 this.region = initHRegion(tableName, method, CONF, families);
2929
2930 KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
2931 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2932 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2933 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2934
2935 KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
2936 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2937 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2938 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2939
2940 Put put = null;
2941 put = new Put(row1);
2942 put.add(kv14);
2943 put.add(kv24);
2944 region.put(put);
2945 region.flush(true);
2946
2947 put = new Put(row1);
2948 put.add(kv23);
2949 put.add(kv13);
2950 region.put(put);
2951 region.flush(true);
2952
2953 put = new Put(row1);
2954 put.add(kv22);
2955 put.add(kv12);
2956 region.put(put);
2957 region.flush(true);
2958
2959 put = new Put(row1);
2960 put.add(kv21);
2961 put.add(kv11);
2962 region.put(put);
2963
2964
2965 List<Cell> expected = new ArrayList<Cell>();
2966 expected.add(kv14);
2967 expected.add(kv13);
2968 expected.add(kv12);
2969 expected.add(kv24);
2970 expected.add(kv23);
2971 expected.add(kv22);
2972
2973 Scan scan = new Scan(row1);
2974 scan.addColumn(fam1, qf1);
2975 scan.addColumn(fam1, qf2);
2976 int versions = 3;
2977 scan.setMaxVersions(versions);
2978 List<Cell> actual = new ArrayList<Cell>();
2979 InternalScanner scanner = region.getScanner(scan);
2980
2981 boolean hasNext = scanner.next(actual);
2982 assertEquals(false, hasNext);
2983
2984
2985 for (int i = 0; i < expected.size(); i++) {
2986 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
2987 }
2988 }
2989
2990 @Test
2991 public void testScanner_Wildcard_FromMemStore_EnforceVersions() throws IOException {
2992 byte[] row1 = Bytes.toBytes("row1");
2993 byte[] qf1 = Bytes.toBytes("qualifier1");
2994 byte[] qf2 = Bytes.toBytes("qualifier2");
2995 byte[] fam1 = Bytes.toBytes("fam1");
2996 byte[][] families = { fam1 };
2997
2998 long ts1 = System.currentTimeMillis();
2999 long ts2 = ts1 + 1;
3000 long ts3 = ts1 + 2;
3001
3002
3003 String method = this.getName();
3004 this.region = initHRegion(tableName, method, CONF, families);
3005
3006 Put put = null;
3007 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
3008 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
3009 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
3010
3011 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
3012 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
3013 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
3014
3015 put = new Put(row1);
3016 put.add(kv13);
3017 put.add(kv12);
3018 put.add(kv11);
3019 put.add(kv23);
3020 put.add(kv22);
3021 put.add(kv21);
3022 region.put(put);
3023
3024
3025 List<Cell> expected = new ArrayList<Cell>();
3026 expected.add(kv13);
3027 expected.add(kv12);
3028 expected.add(kv23);
3029 expected.add(kv22);
3030
3031 Scan scan = new Scan(row1);
3032 scan.addFamily(fam1);
3033 scan.setMaxVersions(MAX_VERSIONS);
3034 List<Cell> actual = new ArrayList<Cell>();
3035 InternalScanner scanner = region.getScanner(scan);
3036
3037 boolean hasNext = scanner.next(actual);
3038 assertEquals(false, hasNext);
3039
3040
3041 for (int i = 0; i < expected.size(); i++) {
3042 assertEquals(expected.get(i), actual.get(i));
3043 }
3044 }
3045
3046 @Test
3047 public void testScanner_Wildcard_FromFilesOnly_EnforceVersions() throws IOException {
3048 byte[] row1 = Bytes.toBytes("row1");
3049 byte[] qf1 = Bytes.toBytes("qualifier1");
3050 byte[] qf2 = Bytes.toBytes("qualifier2");
3051 byte[] fam1 = Bytes.toBytes("fam1");
3052
3053 long ts1 = 1;
3054 long ts2 = ts1 + 1;
3055 long ts3 = ts1 + 2;
3056
3057
3058 String method = this.getName();
3059 this.region = initHRegion(tableName, method, CONF, fam1);
3060
3061 Put put = null;
3062 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
3063 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
3064 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
3065
3066 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
3067 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
3068 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
3069
3070 put = new Put(row1);
3071 put.add(kv13);
3072 put.add(kv12);
3073 put.add(kv11);
3074 put.add(kv23);
3075 put.add(kv22);
3076 put.add(kv21);
3077 region.put(put);
3078 region.flush(true);
3079
3080
3081 List<Cell> expected = new ArrayList<Cell>();
3082 expected.add(kv13);
3083 expected.add(kv12);
3084 expected.add(kv23);
3085 expected.add(kv22);
3086
3087 Scan scan = new Scan(row1);
3088 scan.addFamily(fam1);
3089 scan.setMaxVersions(MAX_VERSIONS);
3090 List<Cell> actual = new ArrayList<Cell>();
3091 InternalScanner scanner = region.getScanner(scan);
3092
3093 boolean hasNext = scanner.next(actual);
3094 assertEquals(false, hasNext);
3095
3096
3097 for (int i = 0; i < expected.size(); i++) {
3098 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
3099 }
3100 }
3101
3102 @Test
3103 public void testScanner_StopRow1542() throws IOException {
3104 byte[] family = Bytes.toBytes("testFamily");
3105 this.region = initHRegion(tableName, getName(), CONF, family);
3106 byte[] row1 = Bytes.toBytes("row111");
3107 byte[] row2 = Bytes.toBytes("row222");
3108 byte[] row3 = Bytes.toBytes("row333");
3109 byte[] row4 = Bytes.toBytes("row444");
3110 byte[] row5 = Bytes.toBytes("row555");
3111
3112 byte[] col1 = Bytes.toBytes("Pub111");
3113 byte[] col2 = Bytes.toBytes("Pub222");
3114
3115 Put put = new Put(row1);
3116 put.add(family, col1, Bytes.toBytes(10L));
3117 region.put(put);
3118
3119 put = new Put(row2);
3120 put.add(family, col1, Bytes.toBytes(15L));
3121 region.put(put);
3122
3123 put = new Put(row3);
3124 put.add(family, col2, Bytes.toBytes(20L));
3125 region.put(put);
3126
3127 put = new Put(row4);
3128 put.add(family, col2, Bytes.toBytes(30L));
3129 region.put(put);
3130
3131 put = new Put(row5);
3132 put.add(family, col1, Bytes.toBytes(40L));
3133 region.put(put);
3134
3135 Scan scan = new Scan(row3, row4);
3136 scan.setMaxVersions();
3137 scan.addColumn(family, col1);
3138 InternalScanner s = region.getScanner(scan);
3139
3140 List<Cell> results = new ArrayList<Cell>();
3141 assertEquals(false, s.next(results));
3142 assertEquals(0, results.size());
3143 }
3144
3145 @Test
3146 public void testScanner_Wildcard_FromMemStoreAndFiles_EnforceVersions() throws IOException {
3147 byte[] row1 = Bytes.toBytes("row1");
3148 byte[] fam1 = Bytes.toBytes("fam1");
3149 byte[] qf1 = Bytes.toBytes("qualifier1");
3150 byte[] qf2 = Bytes.toBytes("quateslifier2");
3151
3152 long ts1 = 1;
3153 long ts2 = ts1 + 1;
3154 long ts3 = ts1 + 2;
3155 long ts4 = ts1 + 3;
3156
3157
3158 String method = this.getName();
3159 this.region = initHRegion(tableName, method, CONF, fam1);
3160
3161 KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
3162 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
3163 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
3164 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
3165
3166 KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
3167 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
3168 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
3169 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
3170
3171 Put put = null;
3172 put = new Put(row1);
3173 put.add(kv14);
3174 put.add(kv24);
3175 region.put(put);
3176 region.flush(true);
3177
3178 put = new Put(row1);
3179 put.add(kv23);
3180 put.add(kv13);
3181 region.put(put);
3182 region.flush(true);
3183
3184 put = new Put(row1);
3185 put.add(kv22);
3186 put.add(kv12);
3187 region.put(put);
3188 region.flush(true);
3189
3190 put = new Put(row1);
3191 put.add(kv21);
3192 put.add(kv11);
3193 region.put(put);
3194
3195
3196 List<KeyValue> expected = new ArrayList<KeyValue>();
3197 expected.add(kv14);
3198 expected.add(kv13);
3199 expected.add(kv12);
3200 expected.add(kv24);
3201 expected.add(kv23);
3202 expected.add(kv22);
3203
3204 Scan scan = new Scan(row1);
3205 int versions = 3;
3206 scan.setMaxVersions(versions);
3207 List<Cell> actual = new ArrayList<Cell>();
3208 InternalScanner scanner = region.getScanner(scan);
3209
3210 boolean hasNext = scanner.next(actual);
3211 assertEquals(false, hasNext);
3212
3213
3214 for (int i = 0; i < expected.size(); i++) {
3215 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
3216 }
3217 }
3218
3219
3220
3221
3222
3223
3224
3225 @Test
3226 public void testScanner_JoinedScanners() throws IOException {
3227 byte[] cf_essential = Bytes.toBytes("essential");
3228 byte[] cf_joined = Bytes.toBytes("joined");
3229 byte[] cf_alpha = Bytes.toBytes("alpha");
3230 this.region = initHRegion(tableName, getName(), CONF, cf_essential, cf_joined, cf_alpha);
3231 byte[] row1 = Bytes.toBytes("row1");
3232 byte[] row2 = Bytes.toBytes("row2");
3233 byte[] row3 = Bytes.toBytes("row3");
3234
3235 byte[] col_normal = Bytes.toBytes("d");
3236 byte[] col_alpha = Bytes.toBytes("a");
3237
3238 byte[] filtered_val = Bytes.toBytes(3);
3239
3240 Put put = new Put(row1);
3241 put.add(cf_essential, col_normal, Bytes.toBytes(1));
3242 put.add(cf_joined, col_alpha, Bytes.toBytes(1));
3243 region.put(put);
3244
3245 put = new Put(row2);
3246 put.add(cf_essential, col_alpha, Bytes.toBytes(2));
3247 put.add(cf_joined, col_normal, Bytes.toBytes(2));
3248 put.add(cf_alpha, col_alpha, Bytes.toBytes(2));
3249 region.put(put);
3250
3251 put = new Put(row3);
3252 put.add(cf_essential, col_normal, filtered_val);
3253 put.add(cf_joined, col_normal, filtered_val);
3254 region.put(put);
3255
3256
3257
3258
3259
3260 Scan scan = new Scan();
3261 Filter filter = new SingleColumnValueExcludeFilter(cf_essential, col_normal,
3262 CompareOp.NOT_EQUAL, filtered_val);
3263 scan.setFilter(filter);
3264 scan.setLoadColumnFamiliesOnDemand(true);
3265 InternalScanner s = region.getScanner(scan);
3266
3267 List<Cell> results = new ArrayList<Cell>();
3268 assertTrue(s.next(results));
3269 assertEquals(results.size(), 1);
3270 results.clear();
3271
3272 assertTrue(s.next(results));
3273 assertEquals(results.size(), 3);
3274 assertTrue("orderCheck", CellUtil.matchingFamily(results.get(0), cf_alpha));
3275 assertTrue("orderCheck", CellUtil.matchingFamily(results.get(1), cf_essential));
3276 assertTrue("orderCheck", CellUtil.matchingFamily(results.get(2), cf_joined));
3277 results.clear();
3278
3279 assertFalse(s.next(results));
3280 assertEquals(results.size(), 0);
3281 }
3282
3283
3284
3285
3286
3287
3288 @Test
3289 public void testScanner_JoinedScannersWithLimits() throws IOException {
3290 final byte[] cf_first = Bytes.toBytes("first");
3291 final byte[] cf_second = Bytes.toBytes("second");
3292
3293 this.region = initHRegion(tableName, getName(), CONF, cf_first, cf_second);
3294 final byte[] col_a = Bytes.toBytes("a");
3295 final byte[] col_b = Bytes.toBytes("b");
3296
3297 Put put;
3298
3299 for (int i = 0; i < 10; i++) {
3300 put = new Put(Bytes.toBytes("r" + Integer.toString(i)));
3301 put.add(cf_first, col_a, Bytes.toBytes(i));
3302 if (i < 5) {
3303 put.add(cf_first, col_b, Bytes.toBytes(i));
3304 put.add(cf_second, col_a, Bytes.toBytes(i));
3305 put.add(cf_second, col_b, Bytes.toBytes(i));
3306 }
3307 region.put(put);
3308 }
3309
3310 Scan scan = new Scan();
3311 scan.setLoadColumnFamiliesOnDemand(true);
3312 Filter bogusFilter = new FilterBase() {
3313 @Override
3314 public ReturnCode filterKeyValue(Cell ignored) throws IOException {
3315 return ReturnCode.INCLUDE;
3316 }
3317 @Override
3318 public boolean isFamilyEssential(byte[] name) {
3319 return Bytes.equals(name, cf_first);
3320 }
3321 };
3322
3323 scan.setFilter(bogusFilter);
3324 InternalScanner s = region.getScanner(scan);
3325
3326
3327
3328
3329
3330
3331
3332
3333
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347
3348
3349
3350
3351
3352
3353
3354
3355 List<Cell> results = new ArrayList<Cell>();
3356 int index = 0;
3357 ScannerContext scannerContext = ScannerContext.newBuilder().setBatchLimit(3).build();
3358 while (true) {
3359 boolean more = s.next(results, scannerContext);
3360 if ((index >> 1) < 5) {
3361 if (index % 2 == 0)
3362 assertEquals(results.size(), 3);
3363 else
3364 assertEquals(results.size(), 1);
3365 } else
3366 assertEquals(results.size(), 1);
3367 results.clear();
3368 index++;
3369 if (!more)
3370 break;
3371 }
3372 }
3373
3374
3375
3376
3377
3378
3379 @Test
3380 public void testLongQualifier() throws Exception {
3381 String method = name.getMethodName();
3382 TableName tableName = TableName.valueOf(method);
3383 byte[] family = Bytes.toBytes("family");
3384 this.region = initHRegion(tableName, method, CONF, family);
3385 byte[] q = new byte[Short.MAX_VALUE+2];
3386 Arrays.fill(q, 0, q.length-1, (byte)42);
3387 for (byte i=0; i<10; i++) {
3388 Put p = new Put(Bytes.toBytes("row"));
3389
3390 q[q.length-1]=i;
3391 p.addColumn(family, q, q);
3392 region.put(p);
3393 }
3394 region.flush(false);
3395 }
3396
3397
3398
3399
3400
3401
3402
3403
3404
3405 @Test
3406 public void testBasicSplit() throws Exception {
3407 byte[][] families = { fam1, fam2, fam3 };
3408
3409 Configuration hc = initSplit();
3410
3411 String method = this.getName();
3412 this.region = initHRegion(tableName, method, hc, families);
3413
3414 LOG.info("" + HBaseTestCase.addContent(region, fam3));
3415 region.flush(true);
3416 region.compactStores();
3417 byte[] splitRow = region.checkSplit();
3418 assertNotNull(splitRow);
3419 LOG.info("SplitRow: " + Bytes.toString(splitRow));
3420 HRegion[] regions = splitRegion(region, splitRow);
3421 try {
3422
3423
3424
3425 for (int i = 0; i < regions.length; i++) {
3426 regions[i] = HRegion.openHRegion(regions[i], null);
3427 }
3428
3429
3430 assertGet(regions[0], fam3, Bytes.toBytes(START_KEY));
3431 assertGet(regions[1], fam3, splitRow);
3432
3433 assertScan(regions[0], fam3, Bytes.toBytes(START_KEY));
3434 assertScan(regions[1], fam3, splitRow);
3435
3436 for (int i = 0; i < regions.length; i++) {
3437
3438
3439 for (int j = 0; j < 2; j++) {
3440 HBaseTestCase.addContent(regions[i], fam3);
3441 }
3442 HBaseTestCase.addContent(regions[i], fam2);
3443 HBaseTestCase.addContent(regions[i], fam1);
3444 regions[i].flush(true);
3445 }
3446
3447 byte[][] midkeys = new byte[regions.length][];
3448
3449 for (int i = 0; i < regions.length; i++) {
3450 regions[i].compactStores();
3451 midkeys[i] = regions[i].checkSplit();
3452 }
3453
3454 TreeMap<String, HRegion> sortedMap = new TreeMap<String, HRegion>();
3455
3456
3457 for (int i = 0; i < regions.length; i++) {
3458 HRegion[] rs = null;
3459 if (midkeys[i] != null) {
3460 rs = splitRegion(regions[i], midkeys[i]);
3461 for (int j = 0; j < rs.length; j++) {
3462 sortedMap.put(Bytes.toString(rs[j].getRegionInfo().getRegionName()),
3463 HRegion.openHRegion(rs[j], null));
3464 }
3465 }
3466 }
3467 LOG.info("Made 4 regions");
3468
3469
3470 int interval = (LAST_CHAR - FIRST_CHAR) / 3;
3471 byte[] b = Bytes.toBytes(START_KEY);
3472 for (HRegion r : sortedMap.values()) {
3473 assertGet(r, fam3, b);
3474 b[0] += interval;
3475 }
3476 } finally {
3477 for (int i = 0; i < regions.length; i++) {
3478 try {
3479 regions[i].close();
3480 } catch (IOException e) {
3481
3482 }
3483 }
3484 }
3485 }
3486
3487 @Test
3488 public void testSplitRegion() throws IOException {
3489 byte[] qualifier = Bytes.toBytes("qualifier");
3490 Configuration hc = initSplit();
3491 int numRows = 10;
3492 byte[][] families = { fam1, fam3 };
3493
3494
3495 String method = this.getName();
3496 this.region = initHRegion(tableName, method, hc, families);
3497
3498
3499 int startRow = 100;
3500 putData(startRow, numRows, qualifier, families);
3501 int splitRow = startRow + numRows;
3502 putData(splitRow, numRows, qualifier, families);
3503 region.flush(true);
3504
3505 HRegion[] regions = null;
3506 regions = splitRegion(region, Bytes.toBytes("" + splitRow));
3507
3508 for (int i = 0; i < regions.length; i++) {
3509 regions[i] = HRegion.openHRegion(regions[i], null);
3510 }
3511
3512 assertEquals(2, regions.length);
3513
3514
3515
3516 verifyData(regions[0], startRow, numRows, qualifier, families);
3517 verifyData(regions[1], splitRow, numRows, qualifier, families);
3518 }
3519
3520 @Test
3521 public void testClearForceSplit() throws IOException {
3522 byte[] qualifier = Bytes.toBytes("qualifier");
3523 Configuration hc = initSplit();
3524 int numRows = 10;
3525 byte[][] families = { fam1, fam3 };
3526
3527
3528 String method = this.getName();
3529 this.region = initHRegion(tableName, method, hc, families);
3530
3531
3532 int startRow = 100;
3533 putData(startRow, numRows, qualifier, families);
3534 int splitRow = startRow + numRows;
3535 byte[] splitRowBytes = Bytes.toBytes("" + splitRow);
3536 putData(splitRow, numRows, qualifier, families);
3537 region.flush(true);
3538
3539 HRegion[] regions = null;
3540
3541 region.forceSplit(splitRowBytes);
3542 assertTrue(region.shouldForceSplit());
3543
3544 assertTrue(Bytes.equals(splitRowBytes, region.checkSplit()));
3545
3546
3547 HStore storeMock = Mockito.mock(HStore.class);
3548 when(storeMock.hasReferences()).thenReturn(true);
3549 when(storeMock.getFamily()).thenReturn(new HColumnDescriptor("cf"));
3550 when(storeMock.close()).thenReturn(ImmutableList.<StoreFile>of());
3551 when(storeMock.getColumnFamilyName()).thenReturn("cf");
3552 region.stores.put(Bytes.toBytes(storeMock.getColumnFamilyName()), storeMock);
3553 assertTrue(region.hasReferences());
3554
3555
3556 regions = splitRegion(region, splitRowBytes);
3557 assertNull(regions);
3558
3559
3560 assertFalse(region.shouldForceSplit());
3561
3562
3563 region.stores.remove(Bytes.toBytes(storeMock.getColumnFamilyName()));
3564 assertFalse(region.hasReferences());
3565
3566
3567 regions = splitRegion(region, splitRowBytes);
3568
3569
3570 for (int i = 0; i < regions.length; i++) {
3571 regions[i] = HRegion.openHRegion(regions[i], null);
3572 }
3573
3574 assertEquals(2, regions.length);
3575
3576
3577
3578 verifyData(regions[0], startRow, numRows, qualifier, families);
3579 verifyData(regions[1], splitRow, numRows, qualifier, families);
3580 }
3581
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592 @Test
3593 public void testFlushCacheWhileScanning() throws IOException, InterruptedException {
3594 byte[] family = Bytes.toBytes("family");
3595 int numRows = 1000;
3596 int flushAndScanInterval = 10;
3597 int compactInterval = 10 * flushAndScanInterval;
3598
3599 String method = "testFlushCacheWhileScanning";
3600 this.region = initHRegion(tableName, method, CONF, family);
3601 FlushThread flushThread = new FlushThread();
3602 try {
3603 flushThread.start();
3604
3605 Scan scan = new Scan();
3606 scan.addFamily(family);
3607 scan.setFilter(new SingleColumnValueFilter(family, qual1, CompareOp.EQUAL,
3608 new BinaryComparator(Bytes.toBytes(5L))));
3609
3610 int expectedCount = 0;
3611 List<Cell> res = new ArrayList<Cell>();
3612
3613 boolean toggle = true;
3614 for (long i = 0; i < numRows; i++) {
3615 Put put = new Put(Bytes.toBytes(i));
3616 put.setDurability(Durability.SKIP_WAL);
3617 put.add(family, qual1, Bytes.toBytes(i % 10));
3618 region.put(put);
3619
3620 if (i != 0 && i % compactInterval == 0) {
3621 LOG.debug("iteration = " + i+ " ts="+System.currentTimeMillis());
3622 region.compact(true);
3623 }
3624
3625 if (i % 10 == 5L) {
3626 expectedCount++;
3627 }
3628
3629 if (i != 0 && i % flushAndScanInterval == 0) {
3630 res.clear();
3631 InternalScanner scanner = region.getScanner(scan);
3632 if (toggle) {
3633 flushThread.flush();
3634 }
3635 while (scanner.next(res))
3636 ;
3637 if (!toggle) {
3638 flushThread.flush();
3639 }
3640 assertEquals("toggle="+toggle+"i=" + i + " ts="+System.currentTimeMillis(),
3641 expectedCount, res.size());
3642 toggle = !toggle;
3643 }
3644 }
3645
3646 } finally {
3647 try {
3648 flushThread.done();
3649 flushThread.join();
3650 flushThread.checkNoError();
3651 } catch (InterruptedException ie) {
3652 LOG.warn("Caught exception when joining with flushThread", ie);
3653 }
3654 HRegion.closeHRegion(this.region);
3655 this.region = null;
3656 }
3657 }
3658
3659 protected class FlushThread extends Thread {
3660 private volatile boolean done;
3661 private Throwable error = null;
3662
3663 FlushThread() {
3664 super("FlushThread");
3665 }
3666
3667 public void done() {
3668 done = true;
3669 synchronized (this) {
3670 interrupt();
3671 }
3672 }
3673
3674 public void checkNoError() {
3675 if (error != null) {
3676 assertNull(error);
3677 }
3678 }
3679
3680 @Override
3681 public void run() {
3682 done = false;
3683 while (!done) {
3684 synchronized (this) {
3685 try {
3686 wait();
3687 } catch (InterruptedException ignored) {
3688 if (done) {
3689 break;
3690 }
3691 }
3692 }
3693 try {
3694 region.flush(true);
3695 } catch (IOException e) {
3696 if (!done) {
3697 LOG.error("Error while flushing cache", e);
3698 error = e;
3699 }
3700 break;
3701 } catch (Throwable t) {
3702 LOG.error("Uncaught exception", t);
3703 throw t;
3704 }
3705 }
3706 }
3707
3708 public void flush() {
3709 synchronized (this) {
3710 notify();
3711 }
3712 }
3713 }
3714
3715
3716
3717
3718
3719
3720
3721
3722
3723
3724 @Test
3725 public void testWritesWhileScanning() throws IOException, InterruptedException {
3726 int testCount = 100;
3727 int numRows = 1;
3728 int numFamilies = 10;
3729 int numQualifiers = 100;
3730 int flushInterval = 7;
3731 int compactInterval = 5 * flushInterval;
3732 byte[][] families = new byte[numFamilies][];
3733 for (int i = 0; i < numFamilies; i++) {
3734 families[i] = Bytes.toBytes("family" + i);
3735 }
3736 byte[][] qualifiers = new byte[numQualifiers][];
3737 for (int i = 0; i < numQualifiers; i++) {
3738 qualifiers[i] = Bytes.toBytes("qual" + i);
3739 }
3740
3741 String method = "testWritesWhileScanning";
3742 this.region = initHRegion(tableName, method, CONF, families);
3743 FlushThread flushThread = new FlushThread();
3744 PutThread putThread = new PutThread(numRows, families, qualifiers);
3745 try {
3746 putThread.start();
3747 putThread.waitForFirstPut();
3748
3749 flushThread.start();
3750
3751 Scan scan = new Scan(Bytes.toBytes("row0"), Bytes.toBytes("row1"));
3752
3753 int expectedCount = numFamilies * numQualifiers;
3754 List<Cell> res = new ArrayList<Cell>();
3755
3756 long prevTimestamp = 0L;
3757 for (int i = 0; i < testCount; i++) {
3758
3759 if (i != 0 && i % compactInterval == 0) {
3760 region.compact(true);
3761 for (Store store : region.getStores()) {
3762 store.closeAndArchiveCompactedFiles();
3763 }
3764 }
3765
3766 if (i != 0 && i % flushInterval == 0) {
3767 flushThread.flush();
3768 }
3769
3770 boolean previousEmpty = res.isEmpty();
3771 res.clear();
3772 InternalScanner scanner = region.getScanner(scan);
3773 while (scanner.next(res))
3774 ;
3775 if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
3776 assertEquals("i=" + i, expectedCount, res.size());
3777 long timestamp = res.get(0).getTimestamp();
3778 assertTrue("Timestamps were broke: " + timestamp + " prev: " + prevTimestamp,
3779 timestamp >= prevTimestamp);
3780 prevTimestamp = timestamp;
3781 }
3782 }
3783 putThread.done();
3784 region.flush(true);
3785
3786 } finally {
3787 try {
3788 flushThread.done();
3789 flushThread.join();
3790 flushThread.checkNoError();
3791
3792 putThread.join();
3793 putThread.checkNoError();
3794 } catch (InterruptedException ie) {
3795 LOG.warn("Caught exception when joining with flushThread", ie);
3796 }
3797
3798 try {
3799 LOG.info("Before close: " + this.region.getMVCC());
3800 HRegion.closeHRegion(this.region);
3801 } catch (DroppedSnapshotException dse) {
3802
3803
3804
3805
3806 }
3807 this.region = null;
3808 }
3809 }
3810
3811 protected class PutThread extends Thread {
3812 private boolean done;
3813 private int numPutsFinished = 0;
3814
3815 private Throwable error = null;
3816 private int numRows;
3817 private byte[][] families;
3818 private byte[][] qualifiers;
3819
3820 private PutThread(int numRows, byte[][] families, byte[][] qualifiers) {
3821 super("PutThread");
3822 this.numRows = numRows;
3823 this.families = families;
3824 this.qualifiers = qualifiers;
3825 }
3826
3827
3828
3829
3830 public void waitForFirstPut() throws InterruptedException {
3831
3832 while (numPutsFinished == 0) {
3833 checkNoError();
3834 Thread.sleep(50);
3835 }
3836 }
3837
3838 public void done() {
3839 done = true;
3840 synchronized (this) {
3841 interrupt();
3842 }
3843 }
3844
3845 public void checkNoError() {
3846 if (error != null) {
3847 assertNull(error);
3848 }
3849 }
3850
3851 @Override
3852 public void run() {
3853 done = false;
3854 while (!done) {
3855 try {
3856 for (int r = 0; r < numRows; r++) {
3857 byte[] row = Bytes.toBytes("row" + r);
3858 Put put = new Put(row);
3859 put.setDurability(Durability.SKIP_WAL);
3860 byte[] value = Bytes.toBytes(String.valueOf(numPutsFinished));
3861 for (byte[] family : families) {
3862 for (byte[] qualifier : qualifiers) {
3863 put.add(family, qualifier, (long) numPutsFinished, value);
3864 }
3865 }
3866 region.put(put);
3867 numPutsFinished++;
3868 if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
3869 System.out.println("put iteration = " + numPutsFinished);
3870 Delete delete = new Delete(row, (long) numPutsFinished - 30);
3871 region.delete(delete);
3872 }
3873 numPutsFinished++;
3874 }
3875 } catch (InterruptedIOException e) {
3876
3877 LOG.info("Interrupted", e);
3878 } catch (IOException e) {
3879 LOG.error("Error while putting records", e);
3880 error = e;
3881 break;
3882 }
3883 }
3884
3885 }
3886
3887 }
3888
3889
3890
3891
3892
3893
3894
3895
3896
3897
3898 @Test
3899 public void testWritesWhileGetting() throws Exception {
3900 int testCount = 100;
3901 int numRows = 1;
3902 int numFamilies = 10;
3903 int numQualifiers = 100;
3904 int compactInterval = 100;
3905 byte[][] families = new byte[numFamilies][];
3906 for (int i = 0; i < numFamilies; i++) {
3907 families[i] = Bytes.toBytes("family" + i);
3908 }
3909 byte[][] qualifiers = new byte[numQualifiers][];
3910 for (int i = 0; i < numQualifiers; i++) {
3911 qualifiers[i] = Bytes.toBytes("qual" + i);
3912 }
3913
3914
3915 String method = "testWritesWhileGetting";
3916
3917
3918
3919
3920
3921 Configuration conf = HBaseConfiguration.create(CONF);
3922 conf.setInt("hbase.hstore.compaction.min", 1);
3923 conf.setInt("hbase.hstore.compaction.max", 1000);
3924 this.region = initHRegion(tableName, method, conf, families);
3925 PutThread putThread = null;
3926 MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(conf);
3927 try {
3928 putThread = new PutThread(numRows, families, qualifiers);
3929 putThread.start();
3930 putThread.waitForFirstPut();
3931
3932
3933 ctx.addThread(new RepeatingTestThread(ctx) {
3934
3935 @Override
3936 public void doAnAction() throws Exception {
3937 region.flush(true);
3938
3939
3940 region.compact(false);
3941 for (Store store : region.getStores()) {
3942 store.closeAndArchiveCompactedFiles();
3943 }
3944 }
3945 });
3946 ctx.startThreads();
3947
3948 Get get = new Get(Bytes.toBytes("row0"));
3949 Result result = null;
3950
3951 int expectedCount = numFamilies * numQualifiers;
3952
3953 long prevTimestamp = 0L;
3954 for (int i = 0; i < testCount; i++) {
3955 LOG.info("testWritesWhileGetting verify turn " + i);
3956 boolean previousEmpty = result == null || result.isEmpty();
3957 result = region.get(get);
3958 if (!result.isEmpty() || !previousEmpty || i > compactInterval) {
3959 assertEquals("i=" + i, expectedCount, result.size());
3960
3961
3962 long timestamp = 0;
3963 for (Cell kv : result.rawCells()) {
3964 if (CellUtil.matchingFamily(kv, families[0])
3965 && CellUtil.matchingQualifier(kv, qualifiers[0])) {
3966 timestamp = kv.getTimestamp();
3967 }
3968 }
3969 assertTrue(timestamp >= prevTimestamp);
3970 prevTimestamp = timestamp;
3971 Cell previousKV = null;
3972
3973 for (Cell kv : result.rawCells()) {
3974 byte[] thisValue = CellUtil.cloneValue(kv);
3975 if (previousKV != null) {
3976 if (Bytes.compareTo(CellUtil.cloneValue(previousKV), thisValue) != 0) {
3977 LOG.warn("These two KV should have the same value." + " Previous KV:" + previousKV
3978 + "(memStoreTS:" + previousKV.getMvccVersion() + ")" + ", New KV: " + kv
3979 + "(memStoreTS:" + kv.getMvccVersion() + ")");
3980 assertEquals(0, Bytes.compareTo(CellUtil.cloneValue(previousKV), thisValue));
3981 }
3982 }
3983 previousKV = kv;
3984 }
3985 }
3986 }
3987 } finally {
3988 if (putThread != null)
3989 putThread.done();
3990
3991 region.flush(true);
3992
3993 if (putThread != null) {
3994 putThread.join();
3995 putThread.checkNoError();
3996 }
3997
3998 ctx.stop();
3999 HRegion.closeHRegion(this.region);
4000 this.region = null;
4001 }
4002 }
4003
4004 @Test
4005 public void testHolesInMeta() throws Exception {
4006 byte[] family = Bytes.toBytes("family");
4007 this.region = initHRegion(tableName, Bytes.toBytes("x"), Bytes.toBytes("z"), method, CONF,
4008 false, family);
4009 byte[] rowNotServed = Bytes.toBytes("a");
4010 Get g = new Get(rowNotServed);
4011 try {
4012 region.get(g);
4013 fail();
4014 } catch (WrongRegionException x) {
4015
4016 }
4017 byte[] row = Bytes.toBytes("y");
4018 g = new Get(row);
4019 region.get(g);
4020 }
4021
4022 @Test
4023 public void testIndexesScanWithOneDeletedRow() throws IOException {
4024 byte[] family = Bytes.toBytes("family");
4025
4026
4027 String method = "testIndexesScanWithOneDeletedRow";
4028 this.region = initHRegion(tableName, method, CONF, family);
4029 Put put = new Put(Bytes.toBytes(1L));
4030 put.add(family, qual1, 1L, Bytes.toBytes(1L));
4031 region.put(put);
4032
4033 region.flush(true);
4034
4035 Delete delete = new Delete(Bytes.toBytes(1L), 1L);
4036 region.delete(delete);
4037
4038 put = new Put(Bytes.toBytes(2L));
4039 put.add(family, qual1, 2L, Bytes.toBytes(2L));
4040 region.put(put);
4041
4042 Scan idxScan = new Scan();
4043 idxScan.addFamily(family);
4044 idxScan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL, Arrays.<Filter> asList(
4045 new SingleColumnValueFilter(family, qual1, CompareOp.GREATER_OR_EQUAL,
4046 new BinaryComparator(Bytes.toBytes(0L))), new SingleColumnValueFilter(family, qual1,
4047 CompareOp.LESS_OR_EQUAL, new BinaryComparator(Bytes.toBytes(3L))))));
4048 InternalScanner scanner = region.getScanner(idxScan);
4049 List<Cell> res = new ArrayList<>();
4050
4051 while (scanner.next(res)) {
4052
4053 }
4054 assertEquals(1L, res.size());
4055 }
4056
4057
4058
4059
4060 @Test
4061 public void testBloomFilterSize() throws IOException {
4062 byte[] fam1 = Bytes.toBytes("fam1");
4063 byte[] qf1 = Bytes.toBytes("col");
4064 byte[] val1 = Bytes.toBytes("value1");
4065
4066 HColumnDescriptor hcd = new HColumnDescriptor(fam1).setMaxVersions(Integer.MAX_VALUE)
4067 .setBloomFilterType(BloomType.ROWCOL);
4068
4069 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
4070 htd.addFamily(hcd);
4071 HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
4072 this.region = TEST_UTIL.createLocalHRegion(info, htd);
4073 int num_unique_rows = 10;
4074 int duplicate_multiplier = 2;
4075 int num_storefiles = 4;
4076
4077 int version = 0;
4078 for (int f = 0; f < num_storefiles; f++) {
4079 for (int i = 0; i < duplicate_multiplier; i++) {
4080 for (int j = 0; j < num_unique_rows; j++) {
4081 Put put = new Put(Bytes.toBytes("row" + j));
4082 put.setDurability(Durability.SKIP_WAL);
4083 put.add(fam1, qf1, version++, val1);
4084 region.put(put);
4085 }
4086 }
4087 region.flush(true);
4088 }
4089
4090 HStore store = (HStore) region.getStore(fam1);
4091 Collection<StoreFile> storeFiles = store.getStorefiles();
4092 for (StoreFile storefile : storeFiles) {
4093 StoreFile.Reader reader = storefile.getReader();
4094 reader.loadFileInfo();
4095 reader.loadBloomfilter();
4096 assertEquals(num_unique_rows * duplicate_multiplier, reader.getEntries());
4097 assertEquals(num_unique_rows, reader.getFilterEntries());
4098 }
4099
4100 region.compact(true);
4101
4102
4103 storeFiles = store.getStorefiles();
4104 for (StoreFile storefile : storeFiles) {
4105 StoreFile.Reader reader = storefile.getReader();
4106 reader.loadFileInfo();
4107 reader.loadBloomfilter();
4108 assertEquals(num_unique_rows * duplicate_multiplier * num_storefiles, reader.getEntries());
4109 assertEquals(num_unique_rows, reader.getFilterEntries());
4110 }
4111 }
4112
4113 @Test
4114 public void testAllColumnsWithBloomFilter() throws IOException {
4115 byte[] TABLE = Bytes.toBytes("testAllColumnsWithBloomFilter");
4116 byte[] FAMILY = Bytes.toBytes("family");
4117
4118
4119 HColumnDescriptor hcd = new HColumnDescriptor(FAMILY).setMaxVersions(Integer.MAX_VALUE)
4120 .setBloomFilterType(BloomType.ROWCOL);
4121 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
4122 htd.addFamily(hcd);
4123 HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
4124 this.region = TEST_UTIL.createLocalHRegion(info, htd);
4125
4126 byte row[] = Bytes.toBytes("row:" + 0);
4127 byte column[] = Bytes.toBytes("column:" + 0);
4128 Put put = new Put(row);
4129 put.setDurability(Durability.SKIP_WAL);
4130 for (long idx = 1; idx <= 4; idx++) {
4131 put.add(FAMILY, column, idx, Bytes.toBytes("value-version-" + idx));
4132 }
4133 region.put(put);
4134
4135
4136 region.flush(true);
4137
4138
4139 Get get = new Get(row);
4140 get.setMaxVersions();
4141 Cell[] kvs = region.get(get).rawCells();
4142
4143
4144 assertEquals(4, kvs.length);
4145 checkOneCell(kvs[0], FAMILY, 0, 0, 4);
4146 checkOneCell(kvs[1], FAMILY, 0, 0, 3);
4147 checkOneCell(kvs[2], FAMILY, 0, 0, 2);
4148 checkOneCell(kvs[3], FAMILY, 0, 0, 1);
4149 }
4150
4151
4152
4153
4154
4155
4156 @Test
4157 public void testDeleteRowWithBloomFilter() throws IOException {
4158 byte[] familyName = Bytes.toBytes("familyName");
4159
4160
4161 HColumnDescriptor hcd = new HColumnDescriptor(familyName).setMaxVersions(Integer.MAX_VALUE)
4162 .setBloomFilterType(BloomType.ROWCOL);
4163
4164 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
4165 htd.addFamily(hcd);
4166 HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
4167 this.region = TEST_UTIL.createLocalHRegion(info, htd);
4168
4169 byte row[] = Bytes.toBytes("row1");
4170 byte col[] = Bytes.toBytes("col1");
4171
4172 Put put = new Put(row);
4173 put.add(familyName, col, 1, Bytes.toBytes("SomeRandomValue"));
4174 region.put(put);
4175 region.flush(true);
4176
4177 Delete del = new Delete(row);
4178 region.delete(del);
4179 region.flush(true);
4180
4181
4182 Get get = new Get(row);
4183 get.addColumn(familyName, col);
4184
4185 Cell[] keyValues = region.get(get).rawCells();
4186 assertEquals(0,keyValues.length);
4187 }
4188
4189 @Test
4190 public void testgetHDFSBlocksDistribution() throws Exception {
4191 HBaseTestingUtility htu = new HBaseTestingUtility();
4192
4193
4194
4195
4196
4197
4198
4199 htu.getConfiguration().setInt("dfs.replication", 2);
4200
4201
4202 MiniHBaseCluster cluster = null;
4203 String dataNodeHosts[] = new String[] { "host1", "host2", "host3" };
4204 int regionServersCount = 3;
4205
4206 try {
4207 cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts);
4208 byte[][] families = { fam1, fam2 };
4209 Table ht = htu.createTable(Bytes.toBytes(this.getName()), families);
4210
4211
4212 byte row[] = Bytes.toBytes("row1");
4213 byte col[] = Bytes.toBytes("col1");
4214
4215 Put put = new Put(row);
4216 put.add(fam1, col, 1, Bytes.toBytes("test1"));
4217 put.add(fam2, col, 1, Bytes.toBytes("test2"));
4218 ht.put(put);
4219
4220 HRegion firstRegion = htu.getHBaseCluster().getRegions(TableName.valueOf(this.getName()))
4221 .get(0);
4222 firstRegion.flush(true);
4223 HDFSBlocksDistribution blocksDistribution1 = firstRegion.getHDFSBlocksDistribution();
4224
4225
4226
4227
4228
4229 long uniqueBlocksWeight1 = blocksDistribution1.getUniqueBlocksTotalWeight();
4230 StringBuilder sb = new StringBuilder();
4231 for (String host: blocksDistribution1.getTopHosts()) {
4232 if (sb.length() > 0) sb.append(", ");
4233 sb.append(host);
4234 sb.append("=");
4235 sb.append(blocksDistribution1.getWeight(host));
4236 }
4237
4238 String topHost = blocksDistribution1.getTopHosts().get(0);
4239 long topHostWeight = blocksDistribution1.getWeight(topHost);
4240 String msg = "uniqueBlocksWeight=" + uniqueBlocksWeight1 + ", topHostWeight=" +
4241 topHostWeight + ", topHost=" + topHost + "; " + sb.toString();
4242 LOG.info(msg);
4243 assertTrue(msg, uniqueBlocksWeight1 == topHostWeight);
4244
4245
4246
4247 HDFSBlocksDistribution blocksDistribution2 = HRegion.computeHDFSBlocksDistribution(
4248 htu.getConfiguration(), firstRegion.getTableDesc(), firstRegion.getRegionInfo());
4249 long uniqueBlocksWeight2 = blocksDistribution2.getUniqueBlocksTotalWeight();
4250
4251 assertTrue(uniqueBlocksWeight1 == uniqueBlocksWeight2);
4252
4253 ht.close();
4254 } finally {
4255 if (cluster != null) {
4256 htu.shutdownMiniCluster();
4257 }
4258 }
4259 }
4260
4261
4262
4263
4264
4265
4266
4267 @Test
4268 public void testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization() throws Exception {
4269 TableName tableName = TableName.valueOf(name.getMethodName());
4270 HRegionInfo info = null;
4271 try {
4272 FileSystem fs = Mockito.mock(FileSystem.class);
4273 Mockito.when(fs.exists((Path) Mockito.anyObject())).thenThrow(new IOException());
4274 HTableDescriptor htd = new HTableDescriptor(tableName);
4275 htd.addFamily(new HColumnDescriptor("cf"));
4276 info = new HRegionInfo(htd.getTableName(), HConstants.EMPTY_BYTE_ARRAY,
4277 HConstants.EMPTY_BYTE_ARRAY, false);
4278 Path path = new Path(dir + "testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization");
4279 region = HRegion.newHRegion(path, null, fs, CONF, info, htd, null);
4280
4281 region.initialize();
4282 fail("Region initialization should fail due to IOException");
4283 } catch (IOException io) {
4284 List<MonitoredTask> tasks = TaskMonitor.get().getTasks();
4285 for (MonitoredTask monitoredTask : tasks) {
4286 if (!(monitoredTask instanceof MonitoredRPCHandler)
4287 && monitoredTask.getDescription().contains(region.toString())) {
4288 assertTrue("Region state should be ABORTED.",
4289 monitoredTask.getState().equals(MonitoredTask.State.ABORTED));
4290 break;
4291 }
4292 }
4293 }
4294 }
4295
4296
4297
4298
4299
4300 @Test
4301 public void testRegionInfoFileCreation() throws IOException {
4302 Path rootDir = new Path(dir + "testRegionInfoFileCreation");
4303
4304 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testtb"));
4305 htd.addFamily(new HColumnDescriptor("cf"));
4306
4307 HRegionInfo hri = new HRegionInfo(htd.getTableName());
4308
4309
4310 region = HRegion.createHRegion(hri, rootDir, CONF, htd, null, false, true);
4311
4312 Path regionDir = region.getRegionFileSystem().getRegionDir();
4313 FileSystem fs = region.getRegionFileSystem().getFileSystem();
4314 HRegion.closeHRegion(region);
4315
4316 Path regionInfoFile = new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE);
4317
4318
4319 assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir",
4320 fs.exists(regionInfoFile));
4321
4322
4323 region = HRegion.openHRegion(rootDir, hri, htd, null, CONF);
4324 assertEquals(regionDir, region.getRegionFileSystem().getRegionDir());
4325 HRegion.closeHRegion(region);
4326
4327
4328 assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir",
4329 fs.exists(regionInfoFile));
4330
4331
4332 fs.delete(regionInfoFile, true);
4333 assertFalse(HRegionFileSystem.REGION_INFO_FILE + " should be removed from the region dir",
4334 fs.exists(regionInfoFile));
4335
4336 region = HRegion.openHRegion(rootDir, hri, htd, null, CONF);
4337
4338 assertEquals(regionDir, region.getRegionFileSystem().getRegionDir());
4339 HRegion.closeHRegion(region);
4340
4341
4342 assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir",
4343 fs.exists(new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE)));
4344
4345 region = null;
4346 }
4347
4348
4349
4350
4351 private static class Incrementer implements Runnable {
4352 private HRegion region;
4353 private final static byte[] incRow = Bytes.toBytes("incRow");
4354 private final static byte[] family = Bytes.toBytes("family");
4355 private final static byte[] qualifier = Bytes.toBytes("qualifier");
4356 private final static long ONE = 1l;
4357 private int incCounter;
4358
4359 public Incrementer(HRegion region, int incCounter) {
4360 this.region = region;
4361 this.incCounter = incCounter;
4362 }
4363
4364 @Override
4365 public void run() {
4366 int count = 0;
4367 while (count < incCounter) {
4368 Increment inc = new Increment(incRow);
4369 inc.addColumn(family, qualifier, ONE);
4370 count++;
4371 try {
4372 region.increment(inc);
4373 } catch (IOException e) {
4374 e.printStackTrace();
4375 break;
4376 }
4377 }
4378 }
4379 }
4380
4381
4382
4383
4384
4385 @Test
4386 public void testParallelIncrementWithMemStoreFlush() throws Exception {
4387 byte[] family = Incrementer.family;
4388 this.region = initHRegion(tableName, method, CONF, family);
4389 final HRegion region = this.region;
4390 final AtomicBoolean incrementDone = new AtomicBoolean(false);
4391 Runnable flusher = new Runnable() {
4392 @Override
4393 public void run() {
4394 while (!incrementDone.get()) {
4395 try {
4396 region.flush(true);
4397 } catch (Exception e) {
4398 e.printStackTrace();
4399 }
4400 }
4401 }
4402 };
4403
4404
4405 int threadNum = 20;
4406 int incCounter = 100;
4407 long expected = threadNum * incCounter;
4408 Thread[] incrementers = new Thread[threadNum];
4409 Thread flushThread = new Thread(flusher);
4410 for (int i = 0; i < threadNum; i++) {
4411 incrementers[i] = new Thread(new Incrementer(this.region, incCounter));
4412 incrementers[i].start();
4413 }
4414 flushThread.start();
4415 for (int i = 0; i < threadNum; i++) {
4416 incrementers[i].join();
4417 }
4418
4419 incrementDone.set(true);
4420 flushThread.join();
4421
4422 Get get = new Get(Incrementer.incRow);
4423 get.addColumn(Incrementer.family, Incrementer.qualifier);
4424 get.setMaxVersions(1);
4425 Result res = this.region.get(get);
4426 List<Cell> kvs = res.getColumnCells(Incrementer.family, Incrementer.qualifier);
4427
4428
4429 assertEquals(kvs.size(), 1);
4430 Cell kv = kvs.get(0);
4431 assertEquals(expected, Bytes.toLong(kv.getValueArray(), kv.getValueOffset()));
4432 }
4433
4434
4435
4436
4437 private static class Appender implements Runnable {
4438 private HRegion region;
4439 private final static byte[] appendRow = Bytes.toBytes("appendRow");
4440 private final static byte[] family = Bytes.toBytes("family");
4441 private final static byte[] qualifier = Bytes.toBytes("qualifier");
4442 private final static byte[] CHAR = Bytes.toBytes("a");
4443 private int appendCounter;
4444
4445 public Appender(HRegion region, int appendCounter) {
4446 this.region = region;
4447 this.appendCounter = appendCounter;
4448 }
4449
4450 @Override
4451 public void run() {
4452 int count = 0;
4453 while (count < appendCounter) {
4454 Append app = new Append(appendRow);
4455 app.add(family, qualifier, CHAR);
4456 count++;
4457 try {
4458 region.append(app);
4459 } catch (IOException e) {
4460 e.printStackTrace();
4461 break;
4462 }
4463 }
4464 }
4465 }
4466
4467
4468
4469
4470
4471 @Test
4472 public void testParallelAppendWithMemStoreFlush() throws Exception {
4473 byte[] family = Appender.family;
4474 this.region = initHRegion(tableName, method, CONF, family);
4475 final HRegion region = this.region;
4476 final AtomicBoolean appendDone = new AtomicBoolean(false);
4477 Runnable flusher = new Runnable() {
4478 @Override
4479 public void run() {
4480 while (!appendDone.get()) {
4481 try {
4482 region.flush(true);
4483 } catch (Exception e) {
4484 e.printStackTrace();
4485 }
4486 }
4487 }
4488 };
4489
4490
4491
4492 int threadNum = 20;
4493 int appendCounter = 100;
4494 byte[] expected = new byte[threadNum * appendCounter];
4495 for (int i = 0; i < threadNum * appendCounter; i++) {
4496 System.arraycopy(Appender.CHAR, 0, expected, i, 1);
4497 }
4498 Thread[] appenders = new Thread[threadNum];
4499 Thread flushThread = new Thread(flusher);
4500 for (int i = 0; i < threadNum; i++) {
4501 appenders[i] = new Thread(new Appender(this.region, appendCounter));
4502 appenders[i].start();
4503 }
4504 flushThread.start();
4505 for (int i = 0; i < threadNum; i++) {
4506 appenders[i].join();
4507 }
4508
4509 appendDone.set(true);
4510 flushThread.join();
4511
4512 Get get = new Get(Appender.appendRow);
4513 get.addColumn(Appender.family, Appender.qualifier);
4514 get.setMaxVersions(1);
4515 Result res = this.region.get(get);
4516 List<Cell> kvs = res.getColumnCells(Appender.family, Appender.qualifier);
4517
4518
4519 assertEquals(kvs.size(), 1);
4520 Cell kv = kvs.get(0);
4521 byte[] appendResult = new byte[kv.getValueLength()];
4522 System.arraycopy(kv.getValueArray(), kv.getValueOffset(), appendResult, 0, kv.getValueLength());
4523 assertArrayEquals(expected, appendResult);
4524 }
4525
4526
4527
4528
4529
4530 @Test
4531 public void testPutWithMemStoreFlush() throws Exception {
4532 byte[] family = Bytes.toBytes("family");
4533 ;
4534 byte[] qualifier = Bytes.toBytes("qualifier");
4535 byte[] row = Bytes.toBytes("putRow");
4536 byte[] value = null;
4537 this.region = initHRegion(tableName, method, CONF, family);
4538 Put put = null;
4539 Get get = null;
4540 List<Cell> kvs = null;
4541 Result res = null;
4542
4543 put = new Put(row);
4544 value = Bytes.toBytes("value0");
4545 put.add(family, qualifier, 1234567l, value);
4546 region.put(put);
4547 get = new Get(row);
4548 get.addColumn(family, qualifier);
4549 get.setMaxVersions();
4550 res = this.region.get(get);
4551 kvs = res.getColumnCells(family, qualifier);
4552 assertEquals(1, kvs.size());
4553 assertArrayEquals(Bytes.toBytes("value0"), CellUtil.cloneValue(kvs.get(0)));
4554
4555 region.flush(true);
4556 get = new Get(row);
4557 get.addColumn(family, qualifier);
4558 get.setMaxVersions();
4559 res = this.region.get(get);
4560 kvs = res.getColumnCells(family, qualifier);
4561 assertEquals(1, kvs.size());
4562 assertArrayEquals(Bytes.toBytes("value0"), CellUtil.cloneValue(kvs.get(0)));
4563
4564 put = new Put(row);
4565 value = Bytes.toBytes("value1");
4566 put.add(family, qualifier, 1234567l, value);
4567 region.put(put);
4568 get = new Get(row);
4569 get.addColumn(family, qualifier);
4570 get.setMaxVersions();
4571 res = this.region.get(get);
4572 kvs = res.getColumnCells(family, qualifier);
4573 assertEquals(1, kvs.size());
4574 assertArrayEquals(Bytes.toBytes("value1"), CellUtil.cloneValue(kvs.get(0)));
4575
4576 region.flush(true);
4577 get = new Get(row);
4578 get.addColumn(family, qualifier);
4579 get.setMaxVersions();
4580 res = this.region.get(get);
4581 kvs = res.getColumnCells(family, qualifier);
4582 assertEquals(1, kvs.size());
4583 assertArrayEquals(Bytes.toBytes("value1"), CellUtil.cloneValue(kvs.get(0)));
4584 }
4585
4586 @Test
4587 public void testDurability() throws Exception {
4588 String method = "testDurability";
4589
4590
4591
4592
4593
4594 durabilityTest(method, Durability.SYNC_WAL, Durability.SYNC_WAL, 0, true, true, false);
4595 durabilityTest(method, Durability.SYNC_WAL, Durability.FSYNC_WAL, 0, true, true, false);
4596 durabilityTest(method, Durability.SYNC_WAL, Durability.USE_DEFAULT, 0, true, true, false);
4597
4598 durabilityTest(method, Durability.FSYNC_WAL, Durability.SYNC_WAL, 0, true, true, false);
4599 durabilityTest(method, Durability.FSYNC_WAL, Durability.FSYNC_WAL, 0, true, true, false);
4600 durabilityTest(method, Durability.FSYNC_WAL, Durability.USE_DEFAULT, 0, true, true, false);
4601
4602 durabilityTest(method, Durability.ASYNC_WAL, Durability.SYNC_WAL, 0, true, true, false);
4603 durabilityTest(method, Durability.ASYNC_WAL, Durability.FSYNC_WAL, 0, true, true, false);
4604
4605 durabilityTest(method, Durability.SKIP_WAL, Durability.SYNC_WAL, 0, true, true, false);
4606 durabilityTest(method, Durability.SKIP_WAL, Durability.FSYNC_WAL, 0, true, true, false);
4607
4608 durabilityTest(method, Durability.USE_DEFAULT, Durability.SYNC_WAL, 0, true, true, false);
4609 durabilityTest(method, Durability.USE_DEFAULT, Durability.FSYNC_WAL, 0, true, true, false);
4610 durabilityTest(method, Durability.USE_DEFAULT, Durability.USE_DEFAULT, 0, true, true, false);
4611
4612
4613 durabilityTest(method, Durability.SYNC_WAL, Durability.ASYNC_WAL, 0, true, false, false);
4614 durabilityTest(method, Durability.FSYNC_WAL, Durability.ASYNC_WAL, 0, true, false, false);
4615 durabilityTest(method, Durability.ASYNC_WAL, Durability.ASYNC_WAL, 0, true, false, false);
4616 durabilityTest(method, Durability.SKIP_WAL, Durability.ASYNC_WAL, 0, true, false, false);
4617 durabilityTest(method, Durability.USE_DEFAULT, Durability.ASYNC_WAL, 0, true, false, false);
4618 durabilityTest(method, Durability.ASYNC_WAL, Durability.USE_DEFAULT, 0, true, false, false);
4619
4620 durabilityTest(method, Durability.SYNC_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
4621 durabilityTest(method, Durability.FSYNC_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
4622 durabilityTest(method, Durability.ASYNC_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
4623 durabilityTest(method, Durability.SKIP_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
4624 durabilityTest(method, Durability.USE_DEFAULT, Durability.ASYNC_WAL, 5000, true, false, true);
4625 durabilityTest(method, Durability.ASYNC_WAL, Durability.USE_DEFAULT, 5000, true, false, true);
4626
4627
4628 durabilityTest(method, Durability.SYNC_WAL, Durability.SKIP_WAL, 0, true, false, false);
4629 durabilityTest(method, Durability.FSYNC_WAL, Durability.SKIP_WAL, 0, true, false, false);
4630 durabilityTest(method, Durability.ASYNC_WAL, Durability.SKIP_WAL, 0, true, false, false);
4631 durabilityTest(method, Durability.SKIP_WAL, Durability.SKIP_WAL, 0, true, false, false);
4632 durabilityTest(method, Durability.USE_DEFAULT, Durability.SKIP_WAL, 0, true, false, false);
4633 durabilityTest(method, Durability.SKIP_WAL, Durability.USE_DEFAULT, 0, true, false, false);
4634
4635 }
4636
4637 private void durabilityTest(String method, Durability tableDurability,
4638 Durability mutationDurability, long timeout, boolean expectAppend, final boolean expectSync,
4639 final boolean expectSyncFromLogSyncer) throws Exception {
4640 Configuration conf = HBaseConfiguration.create(CONF);
4641 method = method + "_" + tableDurability.name() + "_" + mutationDurability.name();
4642 TableName tableName = TableName.valueOf(method);
4643 byte[] family = Bytes.toBytes("family");
4644 Path logDir = new Path(new Path(dir + method), "log");
4645 final Configuration walConf = new Configuration(conf);
4646 FSUtils.setRootDir(walConf, logDir);
4647 final WALFactory wals = new WALFactory(walConf, null, UUID.randomUUID().toString());
4648 final WAL wal = spy(wals.getWAL(tableName.getName(), tableName.getNamespace()));
4649 this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
4650 HConstants.EMPTY_END_ROW, method, conf, false, tableDurability, wal,
4651 new byte[][] { family });
4652
4653 Put put = new Put(Bytes.toBytes("r1"));
4654 put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
4655 put.setDurability(mutationDurability);
4656 region.put(put);
4657
4658
4659 verify(wal, expectAppend ? times(1) : never())
4660 .append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(),
4661 (WALEdit)any(), Mockito.anyBoolean());
4662
4663
4664 if (expectSync || expectSyncFromLogSyncer) {
4665 TEST_UTIL.waitFor(timeout, new Waiter.Predicate<Exception>() {
4666 @Override
4667 public boolean evaluate() throws Exception {
4668 try {
4669 if (expectSync) {
4670 verify(wal, times(1)).sync(anyLong());
4671 } else if (expectSyncFromLogSyncer) {
4672 verify(wal, times(1)).sync();
4673 }
4674 } catch (Throwable ignore) {
4675 }
4676 return true;
4677 }
4678 });
4679 } else {
4680
4681 verify(wal, never()).sync();
4682 }
4683
4684 HRegion.closeHRegion(this.region);
4685 this.region = null;
4686 }
4687
4688 @Test
4689 public void testRegionReplicaSecondary() throws IOException {
4690
4691
4692 Path rootDir = new Path(dir + "testRegionReplicaSecondary");
4693 FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
4694
4695 byte[][] families = new byte[][] {
4696 Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
4697 };
4698 byte[] cq = Bytes.toBytes("cq");
4699 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testRegionReplicaSecondary"));
4700 for (byte[] family : families) {
4701 htd.addFamily(new HColumnDescriptor(family));
4702 }
4703
4704 long time = System.currentTimeMillis();
4705 HRegionInfo primaryHri = new HRegionInfo(htd.getTableName(),
4706 HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4707 false, time, 0);
4708 HRegionInfo secondaryHri = new HRegionInfo(htd.getTableName(),
4709 HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4710 false, time, 1);
4711
4712 HRegion primaryRegion = null, secondaryRegion = null;
4713
4714 try {
4715 primaryRegion = HRegion.createHRegion(primaryHri,
4716 rootDir, TEST_UTIL.getConfiguration(), htd);
4717
4718
4719 putData(primaryRegion, 0, 1000, cq, families);
4720
4721
4722 primaryRegion.flush(true);
4723
4724
4725 secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF);
4726
4727 verifyData(secondaryRegion, 0, 1000, cq, families);
4728 } finally {
4729 if (primaryRegion != null) {
4730 HRegion.closeHRegion(primaryRegion);
4731 }
4732 if (secondaryRegion != null) {
4733 HRegion.closeHRegion(secondaryRegion);
4734 }
4735 }
4736 }
4737
4738 @Test
4739 public void testRegionReplicaSecondaryIsReadOnly() throws IOException {
4740
4741
4742 Path rootDir = new Path(dir + "testRegionReplicaSecondary");
4743 FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
4744
4745 byte[][] families = new byte[][] {
4746 Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
4747 };
4748 byte[] cq = Bytes.toBytes("cq");
4749 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testRegionReplicaSecondary"));
4750 for (byte[] family : families) {
4751 htd.addFamily(new HColumnDescriptor(family));
4752 }
4753
4754 long time = System.currentTimeMillis();
4755 HRegionInfo primaryHri = new HRegionInfo(htd.getTableName(),
4756 HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4757 false, time, 0);
4758 HRegionInfo secondaryHri = new HRegionInfo(htd.getTableName(),
4759 HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4760 false, time, 1);
4761
4762 HRegion primaryRegion = null, secondaryRegion = null;
4763
4764 try {
4765 primaryRegion = HRegion.createHRegion(primaryHri,
4766 rootDir, TEST_UTIL.getConfiguration(), htd);
4767
4768
4769 putData(primaryRegion, 0, 1000, cq, families);
4770
4771
4772 primaryRegion.flush(true);
4773
4774
4775 secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF);
4776
4777 try {
4778 putData(secondaryRegion, 0, 1000, cq, families);
4779 fail("Should have thrown exception");
4780 } catch (IOException ex) {
4781
4782 }
4783 } finally {
4784 if (primaryRegion != null) {
4785 HRegion.closeHRegion(primaryRegion);
4786 }
4787 if (secondaryRegion != null) {
4788 HRegion.closeHRegion(secondaryRegion);
4789 }
4790 }
4791 }
4792
4793 static WALFactory createWALFactory(Configuration conf, Path rootDir) throws IOException {
4794 Configuration confForWAL = new Configuration(conf);
4795 confForWAL.set(HConstants.HBASE_DIR, rootDir.toString());
4796 return new WALFactory(confForWAL,
4797 Collections.<WALActionsListener>singletonList(new MetricsWAL()),
4798 "hregion-" + RandomStringUtils.randomNumeric(8));
4799 }
4800
4801 @Test
4802 public void testCompactionFromPrimary() throws IOException {
4803 Path rootDir = new Path(dir + "testRegionReplicaSecondary");
4804 FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
4805
4806 byte[][] families = new byte[][] {
4807 Bytes.toBytes("cf1"), Bytes.toBytes("cf2"), Bytes.toBytes("cf3")
4808 };
4809 byte[] cq = Bytes.toBytes("cq");
4810 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testRegionReplicaSecondary"));
4811 for (byte[] family : families) {
4812 htd.addFamily(new HColumnDescriptor(family));
4813 }
4814
4815 long time = System.currentTimeMillis();
4816 HRegionInfo primaryHri = new HRegionInfo(htd.getTableName(),
4817 HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4818 false, time, 0);
4819 HRegionInfo secondaryHri = new HRegionInfo(htd.getTableName(),
4820 HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW,
4821 false, time, 1);
4822
4823 HRegion primaryRegion = null, secondaryRegion = null;
4824
4825 try {
4826 primaryRegion = HRegion.createHRegion(primaryHri,
4827 rootDir, TEST_UTIL.getConfiguration(), htd);
4828
4829
4830 putData(primaryRegion, 0, 1000, cq, families);
4831
4832
4833 primaryRegion.flush(true);
4834
4835
4836 secondaryRegion = HRegion.openHRegion(rootDir, secondaryHri, htd, null, CONF);
4837
4838
4839 Collection<StoreFile> storeFiles = primaryRegion.getStore(families[0]).getStorefiles();
4840 primaryRegion.getRegionFileSystem().removeStoreFiles(Bytes.toString(families[0]), storeFiles);
4841 Collection<StoreFileInfo> storeFileInfos = primaryRegion.getRegionFileSystem().getStoreFiles(families[0]);
4842 Assert.assertTrue(storeFileInfos == null || storeFileInfos.size() == 0);
4843
4844 verifyData(secondaryRegion, 0, 1000, cq, families);
4845 } finally {
4846 if (primaryRegion != null) {
4847 HRegion.closeHRegion(primaryRegion);
4848 }
4849 if (secondaryRegion != null) {
4850 HRegion.closeHRegion(secondaryRegion);
4851 }
4852 }
4853 }
4854
4855 private void putData(int startRow, int numRows, byte[] qf, byte[]... families) throws IOException {
4856 putData(this.region, startRow, numRows, qf, families);
4857 }
4858
4859 private void putData(HRegion region,
4860 int startRow, int numRows, byte[] qf, byte[]... families) throws IOException {
4861 putData(region, Durability.SKIP_WAL, startRow, numRows, qf, families);
4862 }
4863
4864 static void putData(HRegion region, Durability durability,
4865 int startRow, int numRows, byte[] qf, byte[]... families) throws IOException {
4866 for (int i = startRow; i < startRow + numRows; i++) {
4867 Put put = new Put(Bytes.toBytes("" + i));
4868 put.setDurability(durability);
4869 for (byte[] family : families) {
4870 put.add(family, qf, null);
4871 }
4872 region.put(put);
4873 }
4874 }
4875
4876 static void verifyData(HRegion newReg, int startRow, int numRows, byte[] qf, byte[]... families)
4877 throws IOException {
4878 for (int i = startRow; i < startRow + numRows; i++) {
4879 byte[] row = Bytes.toBytes("" + i);
4880 Get get = new Get(row);
4881 for (byte[] family : families) {
4882 get.addColumn(family, qf);
4883 }
4884 Result result = newReg.get(get);
4885 Cell[] raw = result.rawCells();
4886 assertEquals(families.length, result.size());
4887 for (int j = 0; j < families.length; j++) {
4888 assertTrue(CellUtil.matchingRow(raw[j], row));
4889 assertTrue(CellUtil.matchingFamily(raw[j], families[j]));
4890 assertTrue(CellUtil.matchingQualifier(raw[j], qf));
4891 }
4892 }
4893 }
4894
4895 static void assertGet(final HRegion r, final byte[] family, final byte[] k) throws IOException {
4896
4897 Get get = new Get(k).addFamily(family).setMaxVersions();
4898 Cell[] results = r.get(get).rawCells();
4899 for (int j = 0; j < results.length; j++) {
4900 byte[] tmp = CellUtil.cloneValue(results[j]);
4901
4902 assertTrue(Bytes.equals(k, tmp));
4903 }
4904 }
4905
4906
4907
4908
4909
4910
4911
4912
4913
4914
4915
4916
4917 private void assertScan(final HRegion r, final byte[] fs, final byte[] firstValue)
4918 throws IOException {
4919 byte[][] families = { fs };
4920 Scan scan = new Scan();
4921 for (int i = 0; i < families.length; i++)
4922 scan.addFamily(families[i]);
4923 InternalScanner s = r.getScanner(scan);
4924 try {
4925 List<Cell> curVals = new ArrayList<Cell>();
4926 boolean first = true;
4927 OUTER_LOOP: while (s.next(curVals)) {
4928 for (Cell kv : curVals) {
4929 byte[] val = CellUtil.cloneValue(kv);
4930 byte[] curval = val;
4931 if (first) {
4932 first = false;
4933 assertTrue(Bytes.compareTo(curval, firstValue) == 0);
4934 } else {
4935
4936 break OUTER_LOOP;
4937 }
4938 }
4939 }
4940 } finally {
4941 s.close();
4942 }
4943 }
4944
4945
4946
4947
4948
4949 @Test
4950 public void testFlushResult() throws IOException {
4951 String method = name.getMethodName();
4952 byte[] tableName = Bytes.toBytes(method);
4953 byte[] family = Bytes.toBytes("family");
4954
4955 this.region = initHRegion(tableName, method, family);
4956
4957
4958 HRegion.FlushResult fr = region.flush(true);
4959 assertFalse(fr.isFlushSucceeded());
4960 assertFalse(fr.isCompactionNeeded());
4961
4962
4963 for (int i = 0; i < 2; i++) {
4964 Put put = new Put(tableName).add(family, family, tableName);
4965 region.put(put);
4966 fr = region.flush(true);
4967 assertTrue(fr.isFlushSucceeded());
4968 assertFalse(fr.isCompactionNeeded());
4969 }
4970
4971
4972 for (int i = 0; i < 2; i++) {
4973 Put put = new Put(tableName).add(family, family, tableName);
4974 region.put(put);
4975 fr = region.flush(true);
4976 assertTrue(fr.isFlushSucceeded());
4977 assertTrue(fr.isCompactionNeeded());
4978 }
4979 }
4980
4981 private Configuration initSplit() {
4982
4983 CONF.setInt("hbase.hstore.compactionThreshold", 2);
4984
4985
4986 CONF.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000);
4987
4988 CONF.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, 10 * 1000);
4989
4990
4991 CONF.setLong("hbase.client.pause", 15 * 1000);
4992
4993
4994
4995 CONF.setLong(HConstants.HREGION_MAX_FILESIZE, 1024 * 128);
4996 return CONF;
4997 }
4998
4999
5000
5001
5002
5003
5004
5005
5006
5007
5008 public static HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
5009 byte[]... families) throws IOException {
5010 return initHRegion(tableName.getName(), null, null, callingMethod, conf, false, families);
5011 }
5012
5013
5014
5015
5016
5017
5018
5019
5020
5021
5022 public static HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
5023 byte[]... families) throws IOException {
5024 return initHRegion(tableName, null, null, callingMethod, conf, false, families);
5025 }
5026
5027
5028
5029
5030
5031
5032
5033
5034
5035
5036
5037 public static HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
5038 boolean isReadOnly, byte[]... families) throws IOException {
5039 return initHRegion(tableName, null, null, callingMethod, conf, isReadOnly, families);
5040 }
5041
5042 public static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
5043 String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
5044 throws IOException {
5045 return initHRegion(tableName, startKey, stopKey, callingMethod, conf, isReadOnly,
5046 Durability.SYNC_WAL, null, families);
5047 }
5048
5049
5050
5051
5052
5053
5054
5055
5056
5057
5058
5059
5060
5061 public static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
5062 String callingMethod, Configuration conf, boolean isReadOnly, Durability durability,
5063 WAL wal, byte[]... families) throws IOException {
5064 return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, callingMethod, conf,
5065 isReadOnly, durability, wal, families);
5066 }
5067
5068
5069
5070
5071
5072 private void checkOneCell(Cell kv, byte[] cf, int rowIdx, int colIdx, long ts) {
5073 String ctx = "rowIdx=" + rowIdx + "; colIdx=" + colIdx + "; ts=" + ts;
5074 assertEquals("Row mismatch which checking: " + ctx, "row:" + rowIdx,
5075 Bytes.toString(CellUtil.cloneRow(kv)));
5076 assertEquals("ColumnFamily mismatch while checking: " + ctx, Bytes.toString(cf),
5077 Bytes.toString(CellUtil.cloneFamily(kv)));
5078 assertEquals("Column qualifier mismatch while checking: " + ctx, "column:" + colIdx,
5079 Bytes.toString(CellUtil.cloneQualifier(kv)));
5080 assertEquals("Timestamp mismatch while checking: " + ctx, ts, kv.getTimestamp());
5081 assertEquals("Value mismatch while checking: " + ctx, "value-version-" + ts,
5082 Bytes.toString(CellUtil.cloneValue(kv)));
5083 }
5084
5085 @Test (timeout=60000)
5086 public void testReverseScanner_FromMemStore_SingleCF_Normal()
5087 throws IOException {
5088 byte[] rowC = Bytes.toBytes("rowC");
5089 byte[] rowA = Bytes.toBytes("rowA");
5090 byte[] rowB = Bytes.toBytes("rowB");
5091 byte[] cf = Bytes.toBytes("CF");
5092 byte[][] families = { cf };
5093 byte[] col = Bytes.toBytes("C");
5094 long ts = 1;
5095 String method = this.getName();
5096 this.region = initHRegion(tableName, method, families);
5097 KeyValue kv1 = new KeyValue(rowC, cf, col, ts, KeyValue.Type.Put, null);
5098 KeyValue kv11 = new KeyValue(rowC, cf, col, ts + 1, KeyValue.Type.Put,
5099 null);
5100 KeyValue kv2 = new KeyValue(rowA, cf, col, ts, KeyValue.Type.Put, null);
5101 KeyValue kv3 = new KeyValue(rowB, cf, col, ts, KeyValue.Type.Put, null);
5102 Put put = null;
5103 put = new Put(rowC);
5104 put.add(kv1);
5105 put.add(kv11);
5106 region.put(put);
5107 put = new Put(rowA);
5108 put.add(kv2);
5109 region.put(put);
5110 put = new Put(rowB);
5111 put.add(kv3);
5112 region.put(put);
5113
5114 Scan scan = new Scan(rowC);
5115 scan.setMaxVersions(5);
5116 scan.setReversed(true);
5117 InternalScanner scanner = region.getScanner(scan);
5118 List<Cell> currRow = new ArrayList<Cell>();
5119 boolean hasNext = scanner.next(currRow);
5120 assertEquals(2, currRow.size());
5121 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
5122 assertTrue(hasNext);
5123 currRow.clear();
5124 hasNext = scanner.next(currRow);
5125 assertEquals(1, currRow.size());
5126 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
5127 assertTrue(hasNext);
5128 currRow.clear();
5129 hasNext = scanner.next(currRow);
5130 assertEquals(1, currRow.size());
5131 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
5132 assertFalse(hasNext);
5133 scanner.close();
5134 }
5135
5136 @Test (timeout=60000)
5137 public void testReverseScanner_FromMemStore_SingleCF_LargerKey()
5138 throws IOException {
5139 byte[] rowC = Bytes.toBytes("rowC");
5140 byte[] rowA = Bytes.toBytes("rowA");
5141 byte[] rowB = Bytes.toBytes("rowB");
5142 byte[] rowD = Bytes.toBytes("rowD");
5143 byte[] cf = Bytes.toBytes("CF");
5144 byte[][] families = { cf };
5145 byte[] col = Bytes.toBytes("C");
5146 long ts = 1;
5147 String method = this.getName();
5148 this.region = initHRegion(tableName, method, families);
5149 KeyValue kv1 = new KeyValue(rowC, cf, col, ts, KeyValue.Type.Put, null);
5150 KeyValue kv11 = new KeyValue(rowC, cf, col, ts + 1, KeyValue.Type.Put,
5151 null);
5152 KeyValue kv2 = new KeyValue(rowA, cf, col, ts, KeyValue.Type.Put, null);
5153 KeyValue kv3 = new KeyValue(rowB, cf, col, ts, KeyValue.Type.Put, null);
5154 Put put = null;
5155 put = new Put(rowC);
5156 put.add(kv1);
5157 put.add(kv11);
5158 region.put(put);
5159 put = new Put(rowA);
5160 put.add(kv2);
5161 region.put(put);
5162 put = new Put(rowB);
5163 put.add(kv3);
5164 region.put(put);
5165
5166 Scan scan = new Scan(rowD);
5167 List<Cell> currRow = new ArrayList<Cell>();
5168 scan.setReversed(true);
5169 scan.setMaxVersions(5);
5170 InternalScanner scanner = region.getScanner(scan);
5171 boolean hasNext = scanner.next(currRow);
5172 assertEquals(2, currRow.size());
5173 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
5174 assertTrue(hasNext);
5175 currRow.clear();
5176 hasNext = scanner.next(currRow);
5177 assertEquals(1, currRow.size());
5178 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
5179 assertTrue(hasNext);
5180 currRow.clear();
5181 hasNext = scanner.next(currRow);
5182 assertEquals(1, currRow.size());
5183 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
5184 assertFalse(hasNext);
5185 scanner.close();
5186 }
5187
5188 @Test (timeout=60000)
5189 public void testReverseScanner_FromMemStore_SingleCF_FullScan()
5190 throws IOException {
5191 byte[] rowC = Bytes.toBytes("rowC");
5192 byte[] rowA = Bytes.toBytes("rowA");
5193 byte[] rowB = Bytes.toBytes("rowB");
5194 byte[] cf = Bytes.toBytes("CF");
5195 byte[][] families = { cf };
5196 byte[] col = Bytes.toBytes("C");
5197 long ts = 1;
5198 String method = this.getName();
5199 this.region = initHRegion(tableName, method, families);
5200 KeyValue kv1 = new KeyValue(rowC, cf, col, ts, KeyValue.Type.Put, null);
5201 KeyValue kv11 = new KeyValue(rowC, cf, col, ts + 1, KeyValue.Type.Put,
5202 null);
5203 KeyValue kv2 = new KeyValue(rowA, cf, col, ts, KeyValue.Type.Put, null);
5204 KeyValue kv3 = new KeyValue(rowB, cf, col, ts, KeyValue.Type.Put, null);
5205 Put put = null;
5206 put = new Put(rowC);
5207 put.add(kv1);
5208 put.add(kv11);
5209 region.put(put);
5210 put = new Put(rowA);
5211 put.add(kv2);
5212 region.put(put);
5213 put = new Put(rowB);
5214 put.add(kv3);
5215 region.put(put);
5216 Scan scan = new Scan();
5217 List<Cell> currRow = new ArrayList<Cell>();
5218 scan.setReversed(true);
5219 InternalScanner scanner = region.getScanner(scan);
5220 boolean hasNext = scanner.next(currRow);
5221 assertEquals(1, currRow.size());
5222 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
5223 assertTrue(hasNext);
5224 currRow.clear();
5225 hasNext = scanner.next(currRow);
5226 assertEquals(1, currRow.size());
5227 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
5228 assertTrue(hasNext);
5229 currRow.clear();
5230 hasNext = scanner.next(currRow);
5231 assertEquals(1, currRow.size());
5232 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowA));
5233 assertFalse(hasNext);
5234 scanner.close();
5235 }
5236
5237 @Test (timeout=60000)
5238 public void testReverseScanner_moreRowsMayExistAfter() throws IOException {
5239
5240 byte[] rowA = Bytes.toBytes("rowA");
5241 byte[] rowB = Bytes.toBytes("rowB");
5242 byte[] rowC = Bytes.toBytes("rowC");
5243 byte[] rowD = Bytes.toBytes("rowD");
5244 byte[] rowE = Bytes.toBytes("rowE");
5245 byte[] cf = Bytes.toBytes("CF");
5246 byte[][] families = { cf };
5247 byte[] col1 = Bytes.toBytes("col1");
5248 byte[] col2 = Bytes.toBytes("col2");
5249 long ts = 1;
5250 String method = this.getName();
5251 this.region = initHRegion(tableName, method, families);
5252 KeyValue kv1 = new KeyValue(rowA, cf, col1, ts, KeyValue.Type.Put, null);
5253 KeyValue kv2 = new KeyValue(rowB, cf, col1, ts, KeyValue.Type.Put, null);
5254 KeyValue kv3 = new KeyValue(rowC, cf, col1, ts, KeyValue.Type.Put, null);
5255 KeyValue kv4_1 = new KeyValue(rowD, cf, col1, ts, KeyValue.Type.Put, null);
5256 KeyValue kv4_2 = new KeyValue(rowD, cf, col2, ts, KeyValue.Type.Put, null);
5257 KeyValue kv5 = new KeyValue(rowE, cf, col1, ts, KeyValue.Type.Put, null);
5258 Put put = null;
5259 put = new Put(rowA);
5260 put.add(kv1);
5261 region.put(put);
5262 put = new Put(rowB);
5263 put.add(kv2);
5264 region.put(put);
5265 put = new Put(rowC);
5266 put.add(kv3);
5267 region.put(put);
5268 put = new Put(rowD);
5269 put.add(kv4_1);
5270 region.put(put);
5271 put = new Put(rowD);
5272 put.add(kv4_2);
5273 region.put(put);
5274 put = new Put(rowE);
5275 put.add(kv5);
5276 region.put(put);
5277 region.flush(true);
5278 Scan scan = new Scan(rowD, rowA);
5279 scan.addColumn(families[0], col1);
5280 scan.setReversed(true);
5281 List<Cell> currRow = new ArrayList<Cell>();
5282 InternalScanner scanner = region.getScanner(scan);
5283 boolean hasNext = scanner.next(currRow);
5284 assertEquals(1, currRow.size());
5285 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
5286 assertTrue(hasNext);
5287 currRow.clear();
5288 hasNext = scanner.next(currRow);
5289 assertEquals(1, currRow.size());
5290 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
5291 assertTrue(hasNext);
5292 currRow.clear();
5293 hasNext = scanner.next(currRow);
5294 assertEquals(1, currRow.size());
5295 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
5296 assertFalse(hasNext);
5297 scanner.close();
5298
5299 scan = new Scan(rowD, rowA);
5300 scan.addColumn(families[0], col2);
5301 scan.setReversed(true);
5302 currRow.clear();
5303 scanner = region.getScanner(scan);
5304 hasNext = scanner.next(currRow);
5305 assertEquals(1, currRow.size());
5306 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
5307 scanner.close();
5308 }
5309
5310 @Test (timeout=60000)
5311 public void testReverseScanner_smaller_blocksize() throws IOException {
5312
5313 byte[] rowA = Bytes.toBytes("rowA");
5314 byte[] rowB = Bytes.toBytes("rowB");
5315 byte[] rowC = Bytes.toBytes("rowC");
5316 byte[] rowD = Bytes.toBytes("rowD");
5317 byte[] rowE = Bytes.toBytes("rowE");
5318 byte[] cf = Bytes.toBytes("CF");
5319 byte[][] families = { cf };
5320 byte[] col1 = Bytes.toBytes("col1");
5321 byte[] col2 = Bytes.toBytes("col2");
5322 long ts = 1;
5323 String method = this.getName();
5324 Configuration conf = new Configuration(CONF);
5325 conf.setInt("test.block.size", 1);
5326 this.region = initHRegion(tableName, method, conf, families);
5327 KeyValue kv1 = new KeyValue(rowA, cf, col1, ts, KeyValue.Type.Put, null);
5328 KeyValue kv2 = new KeyValue(rowB, cf, col1, ts, KeyValue.Type.Put, null);
5329 KeyValue kv3 = new KeyValue(rowC, cf, col1, ts, KeyValue.Type.Put, null);
5330 KeyValue kv4_1 = new KeyValue(rowD, cf, col1, ts, KeyValue.Type.Put, null);
5331 KeyValue kv4_2 = new KeyValue(rowD, cf, col2, ts, KeyValue.Type.Put, null);
5332 KeyValue kv5 = new KeyValue(rowE, cf, col1, ts, KeyValue.Type.Put, null);
5333 Put put = null;
5334 put = new Put(rowA);
5335 put.add(kv1);
5336 region.put(put);
5337 put = new Put(rowB);
5338 put.add(kv2);
5339 region.put(put);
5340 put = new Put(rowC);
5341 put.add(kv3);
5342 region.put(put);
5343 put = new Put(rowD);
5344 put.add(kv4_1);
5345 region.put(put);
5346 put = new Put(rowD);
5347 put.add(kv4_2);
5348 region.put(put);
5349 put = new Put(rowE);
5350 put.add(kv5);
5351 region.put(put);
5352 region.flush(true);
5353 Scan scan = new Scan(rowD, rowA);
5354 scan.addColumn(families[0], col1);
5355 scan.setReversed(true);
5356 List<Cell> currRow = new ArrayList<Cell>();
5357 InternalScanner scanner = region.getScanner(scan);
5358 boolean hasNext = scanner.next(currRow);
5359 assertEquals(1, currRow.size());
5360 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
5361 assertTrue(hasNext);
5362 currRow.clear();
5363 hasNext = scanner.next(currRow);
5364 assertEquals(1, currRow.size());
5365 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowC));
5366 assertTrue(hasNext);
5367 currRow.clear();
5368 hasNext = scanner.next(currRow);
5369 assertEquals(1, currRow.size());
5370 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowB));
5371 assertFalse(hasNext);
5372 scanner.close();
5373
5374 scan = new Scan(rowD, rowA);
5375 scan.addColumn(families[0], col2);
5376 scan.setReversed(true);
5377 currRow.clear();
5378 scanner = region.getScanner(scan);
5379 hasNext = scanner.next(currRow);
5380 assertEquals(1, currRow.size());
5381 assertTrue(Bytes.equals(currRow.get(0).getRow(), rowD));
5382 scanner.close();
5383 }
5384
5385 @Test (timeout=60000)
5386 public void testReverseScanner_FromMemStoreAndHFiles_MultiCFs1()
5387 throws IOException {
5388 byte[] row0 = Bytes.toBytes("row0");
5389 byte[] row1 = Bytes.toBytes("row1");
5390 byte[] row2 = Bytes.toBytes("row2");
5391 byte[] row3 = Bytes.toBytes("row3");
5392 byte[] row4 = Bytes.toBytes("row4");
5393 byte[] row5 = Bytes.toBytes("row5");
5394 byte[] cf1 = Bytes.toBytes("CF1");
5395 byte[] cf2 = Bytes.toBytes("CF2");
5396 byte[] cf3 = Bytes.toBytes("CF3");
5397 byte[][] families = { cf1, cf2, cf3 };
5398 byte[] col = Bytes.toBytes("C");
5399 long ts = 1;
5400 String method = this.getName();
5401 Configuration conf = new Configuration(CONF);
5402
5403 conf.setInt("hbase.hstore.compactionThreshold", 10000);
5404 this.region = initHRegion(tableName, method, conf, families);
5405
5406 KeyValue kv0_1_1 = new KeyValue(row0, cf1, col, ts, KeyValue.Type.Put,
5407 null);
5408 KeyValue kv1_2_1 = new KeyValue(row1, cf2, col, ts, KeyValue.Type.Put,
5409 null);
5410 KeyValue kv1_2_2 = new KeyValue(row1, cf1, col, ts + 1,
5411 KeyValue.Type.Put, null);
5412 KeyValue kv2_4_1 = new KeyValue(row2, cf2, col, ts, KeyValue.Type.Put,
5413 null);
5414 KeyValue kv2_4_2 = new KeyValue(row2, cf1, col, ts, KeyValue.Type.Put,
5415 null);
5416 KeyValue kv2_4_3 = new KeyValue(row2, cf3, col, ts, KeyValue.Type.Put,
5417 null);
5418 KeyValue kv2_4_4 = new KeyValue(row2, cf1, col, ts + 4,
5419 KeyValue.Type.Put, null);
5420 KeyValue kv3_2_1 = new KeyValue(row3, cf2, col, ts, KeyValue.Type.Put,
5421 null);
5422 KeyValue kv3_2_2 = new KeyValue(row3, cf1, col, ts + 4,
5423 KeyValue.Type.Put, null);
5424 KeyValue kv4_5_1 = new KeyValue(row4, cf1, col, ts, KeyValue.Type.Put,
5425 null);
5426 KeyValue kv4_5_2 = new KeyValue(row4, cf3, col, ts, KeyValue.Type.Put,
5427 null);
5428 KeyValue kv4_5_3 = new KeyValue(row4, cf3, col, ts + 5,
5429 KeyValue.Type.Put, null);
5430 KeyValue kv4_5_4 = new KeyValue(row4, cf2, col, ts, KeyValue.Type.Put,
5431 null);
5432 KeyValue kv4_5_5 = new KeyValue(row4, cf1, col, ts + 3,
5433 KeyValue.Type.Put, null);
5434 KeyValue kv5_2_1 = new KeyValue(row5, cf2, col, ts, KeyValue.Type.Put,
5435 null);
5436 KeyValue kv5_2_2 = new KeyValue(row5, cf3, col, ts, KeyValue.Type.Put,
5437 null);
5438
5439 Put put = null;
5440 put = new Put(row1);
5441 put.add(kv1_2_1);
5442 region.put(put);
5443 put = new Put(row2);
5444 put.add(kv2_4_1);
5445 region.put(put);
5446 put = new Put(row4);
5447 put.add(kv4_5_4);
5448 put.add(kv4_5_5);
5449 region.put(put);
5450 region.flush(true);
5451
5452 put = new Put(row4);
5453 put.add(kv4_5_1);
5454 put.add(kv4_5_3);
5455 region.put(put);
5456 put = new Put(row1);
5457 put.add(kv1_2_2);
5458 region.put(put);
5459 put = new Put(row2);
5460 put.add(kv2_4_4);
5461 region.put(put);
5462 region.flush(true);
5463
5464 put = new Put(row4);
5465 put.add(kv4_5_2);
5466 region.put(put);
5467 put = new Put(row2);
5468 put.add(kv2_4_2);
5469 put.add(kv2_4_3);
5470 region.put(put);
5471 put = new Put(row3);
5472 put.add(kv3_2_2);
5473 region.put(put);
5474 region.flush(true);
5475
5476
5477 put = new Put(row0);
5478 put.add(kv0_1_1);
5479 region.put(put);
5480 put = new Put(row3);
5481 put.add(kv3_2_1);
5482 region.put(put);
5483 put = new Put(row5);
5484 put.add(kv5_2_1);
5485 put.add(kv5_2_2);
5486 region.put(put);
5487
5488 Scan scan = new Scan(row4);
5489 scan.setMaxVersions(5);
5490 scan.setBatch(3);
5491 scan.setReversed(true);
5492 InternalScanner scanner = region.getScanner(scan);
5493 List<Cell> currRow = new ArrayList<Cell>();
5494 boolean hasNext = false;
5495
5496
5497
5498 hasNext = scanner.next(currRow);
5499 assertEquals(3, currRow.size());
5500 assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
5501 assertTrue(hasNext);
5502 currRow.clear();
5503 hasNext = scanner.next(currRow);
5504 assertEquals(2, currRow.size());
5505 assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
5506 assertTrue(hasNext);
5507
5508 currRow.clear();
5509 hasNext = scanner.next(currRow);
5510 assertEquals(2, currRow.size());
5511 assertTrue(Bytes.equals(currRow.get(0).getRow(), row3));
5512 assertTrue(hasNext);
5513
5514
5515 currRow.clear();
5516 hasNext = scanner.next(currRow);
5517 assertEquals(3, currRow.size());
5518 assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
5519 assertTrue(hasNext);
5520 currRow.clear();
5521 hasNext = scanner.next(currRow);
5522 assertEquals(1, currRow.size());
5523 assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
5524 assertTrue(hasNext);
5525
5526 currRow.clear();
5527 hasNext = scanner.next(currRow);
5528 assertEquals(2, currRow.size());
5529 assertTrue(Bytes.equals(currRow.get(0).getRow(), row1));
5530 assertTrue(hasNext);
5531
5532 currRow.clear();
5533 hasNext = scanner.next(currRow);
5534 assertEquals(1, currRow.size());
5535 assertTrue(Bytes.equals(currRow.get(0).getRow(), row0));
5536 assertFalse(hasNext);
5537
5538 scanner.close();
5539 }
5540
5541 @Test (timeout=60000)
5542 public void testReverseScanner_FromMemStoreAndHFiles_MultiCFs2()
5543 throws IOException {
5544 byte[] row1 = Bytes.toBytes("row1");
5545 byte[] row2 = Bytes.toBytes("row2");
5546 byte[] row3 = Bytes.toBytes("row3");
5547 byte[] row4 = Bytes.toBytes("row4");
5548 byte[] cf1 = Bytes.toBytes("CF1");
5549 byte[] cf2 = Bytes.toBytes("CF2");
5550 byte[] cf3 = Bytes.toBytes("CF3");
5551 byte[] cf4 = Bytes.toBytes("CF4");
5552 byte[][] families = { cf1, cf2, cf3, cf4 };
5553 byte[] col = Bytes.toBytes("C");
5554 long ts = 1;
5555 String method = this.getName();
5556 Configuration conf = new Configuration(CONF);
5557
5558 conf.setInt("hbase.hstore.compactionThreshold", 10000);
5559 this.region = initHRegion(tableName, method, conf, families);
5560 KeyValue kv1 = new KeyValue(row1, cf1, col, ts, KeyValue.Type.Put, null);
5561 KeyValue kv2 = new KeyValue(row2, cf2, col, ts, KeyValue.Type.Put, null);
5562 KeyValue kv3 = new KeyValue(row3, cf3, col, ts, KeyValue.Type.Put, null);
5563 KeyValue kv4 = new KeyValue(row4, cf4, col, ts, KeyValue.Type.Put, null);
5564
5565 Put put = new Put(row1);
5566 put.add(kv1);
5567 region.put(put);
5568 region.flush(true);
5569
5570 put = new Put(row2);
5571 put.add(kv2);
5572 region.put(put);
5573 region.flush(true);
5574
5575 put = new Put(row3);
5576 put.add(kv3);
5577 region.put(put);
5578 region.flush(true);
5579
5580 put = new Put(row4);
5581 put.add(kv4);
5582 region.put(put);
5583
5584 Scan scan = new Scan(row4);
5585 scan.setReversed(true);
5586 scan.setBatch(10);
5587 InternalScanner scanner = region.getScanner(scan);
5588 List<Cell> currRow = new ArrayList<Cell>();
5589 boolean hasNext = scanner.next(currRow);
5590 assertEquals(1, currRow.size());
5591 assertTrue(Bytes.equals(currRow.get(0).getRow(), row4));
5592 assertTrue(hasNext);
5593 currRow.clear();
5594 hasNext = scanner.next(currRow);
5595 assertEquals(1, currRow.size());
5596 assertTrue(Bytes.equals(currRow.get(0).getRow(), row3));
5597 assertTrue(hasNext);
5598 currRow.clear();
5599 hasNext = scanner.next(currRow);
5600 assertEquals(1, currRow.size());
5601 assertTrue(Bytes.equals(currRow.get(0).getRow(), row2));
5602 assertTrue(hasNext);
5603 currRow.clear();
5604 hasNext = scanner.next(currRow);
5605 assertEquals(1, currRow.size());
5606 assertTrue(Bytes.equals(currRow.get(0).getRow(), row1));
5607 assertFalse(hasNext);
5608 }
5609
5610
5611
5612
5613 @Test (timeout = 60000)
5614 public void testReverseScanner_StackOverflow() throws IOException {
5615 byte[] cf1 = Bytes.toBytes("CF1");
5616 byte[][] families = {cf1};
5617 byte[] col = Bytes.toBytes("C");
5618 String method = this.getName();
5619 this.region = initHRegion(tableName, method, CONF, families);
5620
5621 Put put = new Put(Bytes.toBytes("19998"));
5622 put.add(cf1, col, Bytes.toBytes("val"));
5623 region.put(put);
5624 region.flushcache(true, true);
5625 Put put2 = new Put(Bytes.toBytes("19997"));
5626 put2.add(cf1, col, Bytes.toBytes("val"));
5627 region.put(put2);
5628
5629 Scan scan = new Scan(Bytes.toBytes("19998"));
5630 scan.setReversed(true);
5631 InternalScanner scanner = region.getScanner(scan);
5632
5633
5634
5635 for (int i = 10000; i < 20000; i++) {
5636 Put p = new Put(Bytes.toBytes("" + i));
5637 p.add(cf1, col, Bytes.toBytes("" + i));
5638 region.put(p);
5639 }
5640 region.flushcache(true, true);
5641
5642
5643
5644 for (int i = 10000; i < 20000; i++) {
5645 Put p = new Put(Bytes.toBytes("" + i));
5646 p.add(cf1, col, Bytes.toBytes("" + i));
5647 region.put(p);
5648 }
5649
5650 List<Cell> currRow = new ArrayList<>();
5651 boolean hasNext;
5652 do {
5653 hasNext = scanner.next(currRow);
5654 } while (hasNext);
5655 assertEquals(2, currRow.size());
5656 assertArrayEquals(Bytes.toBytes("19998"), currRow.get(0).getRow());
5657 assertArrayEquals(Bytes.toBytes("19997"), currRow.get(1).getRow());
5658 }
5659
5660 @Test (timeout=60000)
5661 public void testSplitRegionWithReverseScan() throws IOException {
5662 byte [] tableName = Bytes.toBytes("testSplitRegionWithReverseScan");
5663 byte [] qualifier = Bytes.toBytes("qualifier");
5664 Configuration hc = initSplit();
5665 int numRows = 3;
5666 byte [][] families = {fam1};
5667
5668
5669 String method = this.getName();
5670 this.region = initHRegion(tableName, method, hc, families);
5671
5672
5673 int startRow = 100;
5674 putData(startRow, numRows, qualifier, families);
5675 int splitRow = startRow + numRows;
5676 putData(splitRow, numRows, qualifier, families);
5677 region.flush(true);
5678
5679 HRegion [] regions = null;
5680 regions = splitRegion(region, Bytes.toBytes("" + splitRow));
5681
5682 for (int i = 0; i < regions.length; i++) {
5683 regions[i] = HRegion.openHRegion(regions[i], null);
5684 }
5685
5686 assertEquals(2, regions.length);
5687
5688
5689
5690 verifyData(regions[0], startRow, numRows, qualifier, families);
5691 verifyData(regions[1], splitRow, numRows, qualifier, families);
5692
5693
5694 Scan scan = new Scan(Bytes.toBytes(String.valueOf(startRow + 10 * numRows)));
5695 scan.setReversed(true);
5696 InternalScanner scanner = regions[1].getScanner(scan);
5697 List<Cell> currRow = new ArrayList<Cell>();
5698 boolean more = false;
5699 int verify = startRow + 2 * numRows - 1;
5700 do {
5701 more = scanner.next(currRow);
5702 assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
5703 verify--;
5704 currRow.clear();
5705 } while(more);
5706 assertEquals(verify, startRow + numRows - 1);
5707 scanner.close();
5708
5709 scan = new Scan(Bytes.toBytes(String.valueOf(startRow + 2 * numRows - 1)));
5710 scan.setReversed(true);
5711 scanner = regions[1].getScanner(scan);
5712 verify = startRow + 2 * numRows - 1;
5713 do {
5714 more = scanner.next(currRow);
5715 assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
5716 verify--;
5717 currRow.clear();
5718 } while(more);
5719 assertEquals(verify, startRow + numRows - 1);
5720 scanner.close();
5721
5722 scan = new Scan(Bytes.toBytes(String.valueOf(startRow + numRows)));
5723 scan.setReversed(true);
5724 scanner = regions[0].getScanner(scan);
5725 verify = startRow + numRows - 1;
5726 do {
5727 more = scanner.next(currRow);
5728 assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
5729 verify--;
5730 currRow.clear();
5731 } while(more);
5732 assertEquals(verify, 99);
5733 scanner.close();
5734
5735 scan = new Scan(Bytes.toBytes(String.valueOf(startRow + numRows - 1)));
5736 scan.setReversed(true);
5737 scanner = regions[0].getScanner(scan);
5738 verify = startRow + numRows - 1;
5739 do {
5740 more = scanner.next(currRow);
5741 assertEquals(Bytes.toString(currRow.get(0).getRow()), verify + "");
5742 verify--;
5743 currRow.clear();
5744 } while(more);
5745 assertEquals(verify, startRow - 1);
5746 scanner.close();
5747 }
5748
5749 @Test
5750 public void testWriteRequestsCounter() throws IOException {
5751 byte[] fam = Bytes.toBytes("info");
5752 byte[][] families = { fam };
5753 this.region = initHRegion(tableName, method, CONF, families);
5754
5755 Assert.assertEquals(0L, region.getWriteRequestsCount());
5756
5757 Put put = new Put(row);
5758 put.add(fam, fam, fam);
5759
5760 Assert.assertEquals(0L, region.getWriteRequestsCount());
5761 region.put(put);
5762 Assert.assertEquals(1L, region.getWriteRequestsCount());
5763 region.put(put);
5764 Assert.assertEquals(2L, region.getWriteRequestsCount());
5765 region.put(put);
5766 Assert.assertEquals(3L, region.getWriteRequestsCount());
5767
5768 region.delete(new Delete(row));
5769 Assert.assertEquals(4L, region.getWriteRequestsCount());
5770 }
5771
5772 @Test
5773 public void testOpenRegionWrittenToWAL() throws Exception {
5774 final ServerName serverName = ServerName.valueOf("testOpenRegionWrittenToWAL", 100, 42);
5775 final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
5776
5777 HTableDescriptor htd
5778 = new HTableDescriptor(TableName.valueOf("testOpenRegionWrittenToWAL"));
5779 htd.addFamily(new HColumnDescriptor(fam1));
5780 htd.addFamily(new HColumnDescriptor(fam2));
5781
5782 HRegionInfo hri = new HRegionInfo(htd.getTableName(),
5783 HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY);
5784
5785
5786 region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL.getConfiguration(),
5787 htd);
5788 assertNotNull(region);
5789
5790
5791 region.put(new Put(Bytes.toBytes("a")).add(fam1, fam1, fam1));
5792 region.flush(true);
5793 HBaseTestingUtility.closeRegionAndWAL(region);
5794
5795 ArgumentCaptor<WALEdit> editCaptor = ArgumentCaptor.forClass(WALEdit.class);
5796
5797
5798 WAL wal = mockWAL();
5799 when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
5800
5801 region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
5802 TEST_UTIL.getConfiguration(), rss, null);
5803
5804 verify(wal, times(1)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any()
5805 , editCaptor.capture(), anyBoolean());
5806
5807 WALEdit edit = editCaptor.getValue();
5808 assertNotNull(edit);
5809 assertNotNull(edit.getCells());
5810 assertEquals(1, edit.getCells().size());
5811 RegionEventDescriptor desc = WALEdit.getRegionEventDescriptor(edit.getCells().get(0));
5812 assertNotNull(desc);
5813
5814 LOG.info("RegionEventDescriptor from WAL: " + desc);
5815
5816 assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
5817 assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
5818 assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
5819 hri.getEncodedNameAsBytes()));
5820 assertTrue(desc.getLogSequenceNumber() > 0);
5821 assertEquals(serverName, ProtobufUtil.toServerName(desc.getServer()));
5822 assertEquals(2, desc.getStoresCount());
5823
5824 StoreDescriptor store = desc.getStores(0);
5825 assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam1));
5826 assertEquals(store.getStoreHomeDir(), Bytes.toString(fam1));
5827 assertEquals(1, store.getStoreFileCount());
5828 assertFalse(store.getStoreFile(0).contains("/"));
5829
5830 store = desc.getStores(1);
5831 assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam2));
5832 assertEquals(store.getStoreHomeDir(), Bytes.toString(fam2));
5833 assertEquals(0, store.getStoreFileCount());
5834 }
5835
5836
5837 static class HRegionWithSeqId extends HRegion {
5838 public HRegionWithSeqId(final Path tableDir, final WAL wal, final FileSystem fs,
5839 final Configuration confParam, final HRegionInfo regionInfo,
5840 final HTableDescriptor htd, final RegionServerServices rsServices) {
5841 super(tableDir, wal, fs, confParam, regionInfo, htd, rsServices);
5842 }
5843 @Override
5844 protected long getNextSequenceId(WAL wal) throws IOException {
5845 return 42;
5846 }
5847 }
5848
5849 @Test
5850 public void testFlushedFileWithNoTags() throws Exception {
5851 String method = "testFlushedFileWithNoTags";
5852 HTableDescriptor htd = new HTableDescriptor(tableName);
5853 htd.addFamily(new HColumnDescriptor(fam1));
5854 region = initHRegion(Bytes.toBytes(method), method, TEST_UTIL.getConfiguration(), fam1);
5855 Put put = new Put(Bytes.toBytes("a-b-0-0"));
5856 put.addColumn(fam1, qual1, Bytes.toBytes("c1-value"));
5857 region.put(put);
5858 region.flush(true);
5859 Store store = region.getStore(fam1);
5860 Collection<StoreFile> storefiles = store.getStorefiles();
5861 for (StoreFile sf : storefiles) {
5862 assertFalse("Tags should not be present "
5863 ,sf.getReader().getHFileReader().getFileContext().isIncludesTags());
5864 }
5865 }
5866
5867 @Test
5868 public void testOpenRegionWrittenToWALForLogReplay() throws Exception {
5869
5870 final ServerName serverName = ServerName.valueOf("testOpenRegionWrittenToWALForLogReplay",
5871 100, 42);
5872 final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
5873
5874 HTableDescriptor htd
5875 = new HTableDescriptor(TableName.valueOf("testOpenRegionWrittenToWALForLogReplay"));
5876 htd.addFamily(new HColumnDescriptor(fam1));
5877 htd.addFamily(new HColumnDescriptor(fam2));
5878
5879 HRegionInfo hri = new HRegionInfo(htd.getTableName(),
5880 HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY);
5881
5882
5883 region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), TEST_UTIL
5884 .getConfiguration(), htd);
5885 assertNotNull(region);
5886
5887
5888 region.put(new Put(Bytes.toBytes("a")).add(fam1, fam1, fam1));
5889 region.flush(true);
5890 HRegion.closeHRegion(region);
5891
5892 ArgumentCaptor<WALEdit> editCaptor = ArgumentCaptor.forClass(WALEdit.class);
5893
5894
5895 WAL wal = mockWAL();
5896 when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
5897
5898
5899 HashMap<String, Region> recoveringRegions = Maps.newHashMap();
5900 recoveringRegions.put(region.getRegionInfo().getEncodedName(), null);
5901 when(rss.getRecoveringRegions()).thenReturn(recoveringRegions);
5902
5903 Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
5904 conf.set(HConstants.REGION_IMPL, HRegionWithSeqId.class.getName());
5905 region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
5906 conf, rss, null);
5907
5908
5909
5910 verify(wal, times(0)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any()
5911 , editCaptor.capture(), anyBoolean());
5912
5913
5914 new FinishRegionRecoveringHandler(rss, region.getRegionInfo().getEncodedName(), "/foo")
5915 .prepare().process();
5916
5917
5918 verify(wal, times(1)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any()
5919 , editCaptor.capture(), anyBoolean());
5920
5921 WALEdit edit = editCaptor.getValue();
5922 assertNotNull(edit);
5923 assertNotNull(edit.getCells());
5924 assertEquals(1, edit.getCells().size());
5925 RegionEventDescriptor desc = WALEdit.getRegionEventDescriptor(edit.getCells().get(0));
5926 assertNotNull(desc);
5927
5928 LOG.info("RegionEventDescriptor from WAL: " + desc);
5929
5930 assertEquals(RegionEventDescriptor.EventType.REGION_OPEN, desc.getEventType());
5931 assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
5932 assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
5933 hri.getEncodedNameAsBytes()));
5934 assertTrue(desc.getLogSequenceNumber() > 0);
5935 assertEquals(serverName, ProtobufUtil.toServerName(desc.getServer()));
5936 assertEquals(2, desc.getStoresCount());
5937
5938 StoreDescriptor store = desc.getStores(0);
5939 assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam1));
5940 assertEquals(store.getStoreHomeDir(), Bytes.toString(fam1));
5941 assertEquals(1, store.getStoreFileCount());
5942 assertFalse(store.getStoreFile(0).contains("/"));
5943
5944 store = desc.getStores(1);
5945 assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam2));
5946 assertEquals(store.getStoreHomeDir(), Bytes.toString(fam2));
5947 assertEquals(0, store.getStoreFileCount());
5948 }
5949
5950
5951
5952
5953
5954
5955
5956 private WAL mockWAL() throws IOException {
5957 WAL wal = mock(WAL.class);
5958 Mockito.when(wal.append((HTableDescriptor)Mockito.any(), (HRegionInfo)Mockito.any(),
5959 (WALKey)Mockito.any(), (WALEdit)Mockito.any(), Mockito.anyBoolean())).
5960 thenAnswer(new Answer<Long>() {
5961 @Override
5962 public Long answer(InvocationOnMock invocation) throws Throwable {
5963 WALKey key = invocation.getArgumentAt(2, WALKey.class);
5964 MultiVersionConcurrencyControl.WriteEntry we = key.getMvcc().begin();
5965 key.setWriteEntry(we);
5966 return 1L;
5967 }
5968
5969 });
5970 return wal;
5971 }
5972
5973 @Test
5974 public void testCloseRegionWrittenToWAL() throws Exception {
5975
5976 Path rootDir = new Path(dir + name.getMethodName());
5977 FSUtils.setRootDir(TEST_UTIL.getConfiguration(), rootDir);
5978
5979 final ServerName serverName = ServerName.valueOf("testCloseRegionWrittenToWAL", 100, 42);
5980 final RegionServerServices rss = spy(TEST_UTIL.createMockRegionServerService(serverName));
5981
5982 HTableDescriptor htd
5983 = new HTableDescriptor(TableName.valueOf("testOpenRegionWrittenToWAL"));
5984 htd.addFamily(new HColumnDescriptor(fam1));
5985 htd.addFamily(new HColumnDescriptor(fam2));
5986
5987 final HRegionInfo hri = new HRegionInfo(htd.getTableName(),
5988 HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY);
5989
5990 ArgumentCaptor<WALEdit> editCaptor = ArgumentCaptor.forClass(WALEdit.class);
5991
5992
5993 WAL wal = mockWAL();
5994 when(rss.getWAL((HRegionInfo) any())).thenReturn(wal);
5995
5996
5997
5998 region = HRegion.createHRegion(hri, rootDir, TEST_UTIL.getConfiguration(), htd, rss.getWAL(hri));
5999 region = HRegion.openHRegion(hri, htd, rss.getWAL(hri),
6000 TEST_UTIL.getConfiguration(), rss, null);
6001
6002
6003 region.close(false);
6004
6005
6006 verify(wal, times(2)).append((HTableDescriptor)any(), (HRegionInfo)any(), (WALKey)any(),
6007 editCaptor.capture(), anyBoolean());
6008
6009 WALEdit edit = editCaptor.getAllValues().get(1);
6010 assertNotNull(edit);
6011 assertNotNull(edit.getCells());
6012 assertEquals(1, edit.getCells().size());
6013 RegionEventDescriptor desc = WALEdit.getRegionEventDescriptor(edit.getCells().get(0));
6014 assertNotNull(desc);
6015
6016 LOG.info("RegionEventDescriptor from WAL: " + desc);
6017
6018 assertEquals(RegionEventDescriptor.EventType.REGION_CLOSE, desc.getEventType());
6019 assertTrue(Bytes.equals(desc.getTableName().toByteArray(), htd.getName()));
6020 assertTrue(Bytes.equals(desc.getEncodedRegionName().toByteArray(),
6021 hri.getEncodedNameAsBytes()));
6022 assertTrue(desc.getLogSequenceNumber() > 0);
6023 assertEquals(serverName, ProtobufUtil.toServerName(desc.getServer()));
6024 assertEquals(2, desc.getStoresCount());
6025
6026 StoreDescriptor store = desc.getStores(0);
6027 assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam1));
6028 assertEquals(store.getStoreHomeDir(), Bytes.toString(fam1));
6029 assertEquals(0, store.getStoreFileCount());
6030
6031 store = desc.getStores(1);
6032 assertTrue(Bytes.equals(store.getFamilyName().toByteArray(), fam2));
6033 assertEquals(store.getStoreHomeDir(), Bytes.toString(fam2));
6034 assertEquals(0, store.getStoreFileCount());
6035 }
6036
6037
6038
6039
6040 @Test (timeout=24000)
6041 public void testRegionTooBusy() throws IOException {
6042 String method = "testRegionTooBusy";
6043 byte[] tableName = Bytes.toBytes(method);
6044 byte[] family = Bytes.toBytes("family");
6045 long defaultBusyWaitDuration = CONF.getLong("hbase.busy.wait.duration",
6046 HRegion.DEFAULT_BUSY_WAIT_DURATION);
6047 CONF.setLong("hbase.busy.wait.duration", 1000);
6048 region = initHRegion(tableName, method, CONF, family);
6049 final AtomicBoolean stopped = new AtomicBoolean(true);
6050 Thread t = new Thread(new Runnable() {
6051 @Override
6052 public void run() {
6053 try {
6054 region.lock.writeLock().lock();
6055 stopped.set(false);
6056 while (!stopped.get()) {
6057 Thread.sleep(100);
6058 }
6059 } catch (InterruptedException ie) {
6060 } finally {
6061 region.lock.writeLock().unlock();
6062 }
6063 }
6064 });
6065 t.start();
6066 Get get = new Get(row);
6067 try {
6068 while (stopped.get()) {
6069 Thread.sleep(100);
6070 }
6071 region.get(get);
6072 fail("Should throw RegionTooBusyException");
6073 } catch (InterruptedException ie) {
6074 fail("test interrupted");
6075 } catch (RegionTooBusyException e) {
6076
6077 } finally {
6078 stopped.set(true);
6079 try {
6080 t.join();
6081 } catch (Throwable e) {
6082 }
6083
6084 HRegion.closeHRegion(region);
6085 region = null;
6086 CONF.setLong("hbase.busy.wait.duration", defaultBusyWaitDuration);
6087 }
6088 }
6089
6090 @Test
6091 public void testCellTTLs() throws IOException {
6092 IncrementingEnvironmentEdge edge = new IncrementingEnvironmentEdge();
6093 EnvironmentEdgeManager.injectEdge(edge);
6094
6095 final byte[] row = Bytes.toBytes("testRow");
6096 final byte[] q1 = Bytes.toBytes("q1");
6097 final byte[] q2 = Bytes.toBytes("q2");
6098 final byte[] q3 = Bytes.toBytes("q3");
6099 final byte[] q4 = Bytes.toBytes("q4");
6100
6101 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testCellTTLs"));
6102 HColumnDescriptor hcd = new HColumnDescriptor(fam1);
6103 hcd.setTimeToLive(10);
6104 htd.addFamily(hcd);
6105
6106 Configuration conf = new Configuration(TEST_UTIL.getConfiguration());
6107 conf.setInt(HFile.FORMAT_VERSION_KEY, HFile.MIN_FORMAT_VERSION_WITH_TAGS);
6108
6109 region = HRegion.createHRegion(new HRegionInfo(htd.getTableName(),
6110 HConstants.EMPTY_BYTE_ARRAY, HConstants.EMPTY_BYTE_ARRAY),
6111 TEST_UTIL.getDataTestDir(), conf, htd);
6112 assertNotNull(region);
6113 long now = EnvironmentEdgeManager.currentTime();
6114
6115 region.put(new Put(row).add(new KeyValue(row, fam1, q1, now,
6116 HConstants.EMPTY_BYTE_ARRAY, new Tag[] {
6117
6118 new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
6119
6120 region.put(new Put(row).add(fam1, q2, now, HConstants.EMPTY_BYTE_ARRAY));
6121
6122 region.put(new Put(row).add(new KeyValue(row, fam1, q3, now + 10000 - 1,
6123 HConstants.EMPTY_BYTE_ARRAY, new Tag[] {
6124
6125 new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(5000L)) } )));
6126
6127 region.put(new Put(row).add(fam1, q4, now + 10000 - 1, HConstants.EMPTY_BYTE_ARRAY));
6128
6129
6130 region.flush(true);
6131
6132
6133 Result r = region.get(new Get(row));
6134 assertNotNull(r.getValue(fam1, q1));
6135 assertNotNull(r.getValue(fam1, q2));
6136 assertNotNull(r.getValue(fam1, q3));
6137 assertNotNull(r.getValue(fam1, q4));
6138
6139
6140 edge.incrementTime(5000);
6141
6142 r = region.get(new Get(row));
6143 assertNull(r.getValue(fam1, q1));
6144 assertNotNull(r.getValue(fam1, q2));
6145 assertNotNull(r.getValue(fam1, q3));
6146 assertNotNull(r.getValue(fam1, q4));
6147
6148
6149 edge.incrementTime(5000);
6150
6151 r = region.get(new Get(row));
6152 assertNull(r.getValue(fam1, q1));
6153 assertNull(r.getValue(fam1, q2));
6154 assertNotNull(r.getValue(fam1, q3));
6155 assertNotNull(r.getValue(fam1, q4));
6156
6157
6158 edge.incrementTime(5000);
6159
6160 r = region.get(new Get(row));
6161 assertNull(r.getValue(fam1, q1));
6162 assertNull(r.getValue(fam1, q2));
6163 assertNull(r.getValue(fam1, q3));
6164 assertNotNull(r.getValue(fam1, q4));
6165
6166
6167 edge.incrementTime(10000);
6168
6169 r = region.get(new Get(row));
6170 assertNull(r.getValue(fam1, q1));
6171 assertNull(r.getValue(fam1, q2));
6172 assertNull(r.getValue(fam1, q3));
6173 assertNull(r.getValue(fam1, q4));
6174
6175
6176
6177
6178 region.put(new Put(row).add(fam1, q1, Bytes.toBytes(1L)));
6179 r = region.get(new Get(row));
6180 byte[] val = r.getValue(fam1, q1);
6181 assertNotNull(val);
6182 assertEquals(Bytes.toLong(val), 1L);
6183
6184
6185 Increment incr = new Increment(row).addColumn(fam1, q1, 1L);
6186 incr.setTTL(5000);
6187 region.increment(incr);
6188
6189
6190 r = region.get(new Get(row));
6191 val = r.getValue(fam1, q1);
6192 assertNotNull(val);
6193 assertEquals(Bytes.toLong(val), 2L);
6194
6195
6196 edge.incrementTime(5000);
6197
6198
6199 r = region.get(new Get(row));
6200 val = r.getValue(fam1, q1);
6201 assertNotNull(val);
6202 assertEquals(Bytes.toLong(val), 1L);
6203
6204
6205 edge.incrementTime(5000);
6206
6207
6208 r = region.get(new Get(row));
6209 assertNull(r.getValue(fam1, q1));
6210 }
6211
6212 @Test
6213 public void testIncrementTimestampsAreMonotonic() throws IOException {
6214 region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
6215 ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
6216 EnvironmentEdgeManager.injectEdge(edge);
6217
6218 edge.setValue(10);
6219 Increment inc = new Increment(row);
6220 inc.setDurability(Durability.SKIP_WAL);
6221 inc.addColumn(fam1, qual1, 1L);
6222 region.increment(inc);
6223
6224 Result result = region.get(new Get(row));
6225 Cell c = result.getColumnLatestCell(fam1, qual1);
6226 assertNotNull(c);
6227 assertEquals(c.getTimestamp(), 10L);
6228
6229 edge.setValue(1);
6230 region.increment(inc);
6231 result = region.get(new Get(row));
6232 c = result.getColumnLatestCell(fam1, qual1);
6233 assertEquals(c.getTimestamp(), 11L);
6234 assertEquals(Bytes.toLong(c.getValueArray(), c.getValueOffset(), c.getValueLength()), 2L);
6235 }
6236
6237 @Test
6238 public void testAppendTimestampsAreMonotonic() throws IOException {
6239 region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
6240 ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
6241 EnvironmentEdgeManager.injectEdge(edge);
6242
6243 edge.setValue(10);
6244 Append a = new Append(row);
6245 a.setDurability(Durability.SKIP_WAL);
6246 a.add(fam1, qual1, qual1);
6247 region.append(a);
6248
6249 Result result = region.get(new Get(row));
6250 Cell c = result.getColumnLatestCell(fam1, qual1);
6251 assertNotNull(c);
6252 assertEquals(c.getTimestamp(), 10L);
6253
6254 edge.setValue(1);
6255 region.append(a);
6256 result = region.get(new Get(row));
6257 c = result.getColumnLatestCell(fam1, qual1);
6258 assertEquals(c.getTimestamp(), 11L);
6259
6260 byte[] expected = new byte[qual1.length*2];
6261 System.arraycopy(qual1, 0, expected, 0, qual1.length);
6262 System.arraycopy(qual1, 0, expected, qual1.length, qual1.length);
6263
6264 assertTrue(Bytes.equals(c.getValueArray(), c.getValueOffset(), c.getValueLength(),
6265 expected, 0, expected.length));
6266 }
6267
6268 @Test
6269 public void testCheckAndMutateTimestampsAreMonotonic() throws IOException {
6270 region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
6271 ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
6272 EnvironmentEdgeManager.injectEdge(edge);
6273
6274 edge.setValue(10);
6275 Put p = new Put(row);
6276 p.setDurability(Durability.SKIP_WAL);
6277 p.addColumn(fam1, qual1, qual1);
6278 region.put(p);
6279
6280 Result result = region.get(new Get(row));
6281 Cell c = result.getColumnLatestCell(fam1, qual1);
6282 assertNotNull(c);
6283 assertEquals(c.getTimestamp(), 10L);
6284
6285 edge.setValue(1);
6286 p = new Put(row);
6287 p.setDurability(Durability.SKIP_WAL);
6288 p.addColumn(fam1, qual1, qual2);
6289 region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL, new BinaryComparator(qual1), p, false);
6290 result = region.get(new Get(row));
6291 c = result.getColumnLatestCell(fam1, qual1);
6292 assertEquals(c.getTimestamp(), 10L);
6293
6294 assertTrue(Bytes.equals(c.getValueArray(), c.getValueOffset(), c.getValueLength(),
6295 qual2, 0, qual2.length));
6296 }
6297
6298 @Test(timeout = 60000)
6299 public void testBatchMutateWithWrongRegionException() throws Exception {
6300 final byte[] a = Bytes.toBytes("a");
6301 final byte[] b = Bytes.toBytes("b");
6302 final byte[] c = Bytes.toBytes("c");
6303
6304 int prevLockTimeout = CONF.getInt("hbase.rowlock.wait.duration", 30000);
6305 CONF.setInt("hbase.rowlock.wait.duration", 1000);
6306 region = initHRegion(tableName, a, c, name.getMethodName(), CONF, false, fam1);
6307
6308 Mutation[] mutations = new Mutation[] {
6309 new Put(a).addImmutable(fam1, null, null),
6310 new Put(c).addImmutable(fam1, null, null),
6311 new Put(b).addImmutable(fam1, null, null),
6312 };
6313
6314 OperationStatus[] status = region.batchMutate(mutations);
6315 assertEquals(status[0].getOperationStatusCode(), OperationStatusCode.SUCCESS);
6316 assertEquals(status[1].getOperationStatusCode(), OperationStatusCode.SANITY_CHECK_FAILURE);
6317 assertEquals(status[2].getOperationStatusCode(), OperationStatusCode.SUCCESS);
6318
6319
6320
6321 final CountDownLatch obtainedRowLock = new CountDownLatch(1);
6322 ExecutorService exec = Executors.newFixedThreadPool(2);
6323 Future<Void> f1 = exec.submit(new Callable<Void>() {
6324 @Override
6325 public Void call() throws Exception {
6326 LOG.info("Acquiring row lock");
6327 RowLock rl = region.getRowLock(b);
6328 obtainedRowLock.countDown();
6329 LOG.info("Waiting for 5 seconds before releasing lock");
6330 Threads.sleep(5000);
6331 LOG.info("Releasing row lock");
6332 rl.release();
6333 return null;
6334 }
6335 });
6336 obtainedRowLock.await(30, TimeUnit.SECONDS);
6337
6338 Future<Void> f2 = exec.submit(new Callable<Void>() {
6339 @Override
6340 public Void call() throws Exception {
6341 Mutation[] mutations = new Mutation[] {
6342 new Put(a).addImmutable(fam1, null, null),
6343 new Put(b).addImmutable(fam1, null, null),
6344 };
6345
6346
6347 OperationStatus[] status = region.batchMutate(mutations);
6348 assertEquals(status[0].getOperationStatusCode(), OperationStatusCode.SUCCESS);
6349 assertEquals(status[1].getOperationStatusCode(), OperationStatusCode.SUCCESS);
6350 return null;
6351 }
6352 });
6353
6354 f1.get();
6355 f2.get();
6356
6357 CONF.setInt("hbase.rowlock.wait.duration", prevLockTimeout);
6358 }
6359
6360 @Test
6361 public void testBatchMutateWithZeroRowLockWait() throws Exception {
6362 final byte[] a = Bytes.toBytes("a");
6363 final byte[] b = Bytes.toBytes("b");
6364 final byte[] c = Bytes.toBytes("c");
6365
6366 Configuration conf = new Configuration(CONF);
6367 conf.setInt("hbase.rowlock.wait.duration", 0);
6368 final HRegionInfo hri = new HRegionInfo(TableName.valueOf(tableName), a, c);
6369 final HTableDescriptor htd = new HTableDescriptor(tableName);
6370 htd.addFamily(new HColumnDescriptor(fam1));
6371 region = HRegion.createHRegion(hri, TEST_UTIL.getDataTestDir(), conf, htd, TEST_UTIL.createWal(conf, TEST_UTIL.getDataTestDir(), TEST_UTIL.getDataTestDirOnTestFS(method + ".log"), hri));
6372
6373 Mutation[] mutations = new Mutation[] {
6374 new Put(a).addImmutable(fam1, null, null),
6375 new Put(b).addImmutable(fam1, null, null),
6376 };
6377
6378 OperationStatus[] status = region.batchMutate(mutations);
6379 assertEquals(OperationStatusCode.SUCCESS, status[0].getOperationStatusCode());
6380 assertEquals(OperationStatusCode.SUCCESS, status[1].getOperationStatusCode());
6381
6382
6383
6384 final CountDownLatch obtainedRowLock = new CountDownLatch(1);
6385 ExecutorService exec = Executors.newFixedThreadPool(2);
6386 Future<Void> f1 = exec.submit(new Callable<Void>() {
6387 @Override
6388 public Void call() throws Exception {
6389 LOG.info("Acquiring row lock");
6390 RowLock rl = region.getRowLock(b);
6391 obtainedRowLock.countDown();
6392 LOG.info("Waiting for 5 seconds before releasing lock");
6393 Threads.sleep(5000);
6394 LOG.info("Releasing row lock");
6395 rl.release();
6396 return null;
6397 }
6398 });
6399 obtainedRowLock.await(30, TimeUnit.SECONDS);
6400
6401 Future<Void> f2 = exec.submit(new Callable<Void>() {
6402 @Override
6403 public Void call() throws Exception {
6404 Mutation[] mutations = new Mutation[] {
6405 new Put(a).addImmutable(fam1, null, null),
6406 new Put(b).addImmutable(fam1, null, null),
6407 };
6408
6409
6410
6411 OperationStatus[] status = region.batchMutate(mutations);
6412 assertEquals(OperationStatusCode.SUCCESS, status[0].getOperationStatusCode());
6413 assertEquals(OperationStatusCode.SUCCESS, status[1].getOperationStatusCode());
6414 return null;
6415 }
6416 });
6417
6418 f1.get();
6419 f2.get();
6420 }
6421
6422 @Test
6423 public void testCheckAndRowMutateTimestampsAreMonotonic() throws IOException {
6424 region = initHRegion(tableName, name.getMethodName(), CONF, fam1);
6425 ManualEnvironmentEdge edge = new ManualEnvironmentEdge();
6426 EnvironmentEdgeManager.injectEdge(edge);
6427
6428 edge.setValue(10);
6429 Put p = new Put(row);
6430 p.setDurability(Durability.SKIP_WAL);
6431 p.addColumn(fam1, qual1, qual1);
6432 region.put(p);
6433
6434 Result result = region.get(new Get(row));
6435 Cell c = result.getColumnLatestCell(fam1, qual1);
6436 assertNotNull(c);
6437 assertEquals(c.getTimestamp(), 10L);
6438
6439 edge.setValue(1);
6440 p = new Put(row);
6441 p.setDurability(Durability.SKIP_WAL);
6442 p.addColumn(fam1, qual1, qual2);
6443 RowMutations rm = new RowMutations(row);
6444 rm.add(p);
6445 region.checkAndRowMutate(row, fam1, qual1, CompareOp.EQUAL, new BinaryComparator(qual1),
6446 rm, false);
6447 result = region.get(new Get(row));
6448 c = result.getColumnLatestCell(fam1, qual1);
6449 assertEquals(c.getTimestamp(), 10L);
6450
6451 assertTrue(Bytes.equals(c.getValueArray(), c.getValueOffset(), c.getValueLength(),
6452 qual2, 0, qual2.length));
6453 }
6454
6455 static HRegion initHRegion(byte[] tableName, String callingMethod,
6456 byte[]... families) throws IOException {
6457 return initHRegion(tableName, callingMethod, HBaseConfiguration.create(),
6458 families);
6459 }
6460
6461
6462
6463
6464
6465 @Test
6466 public void testWritesWhileRollWriter() throws IOException {
6467 int testCount = 10;
6468 int numRows = 1024;
6469 int numFamilies = 2;
6470 int numQualifiers = 2;
6471 final byte[][] families = new byte[numFamilies][];
6472 for (int i = 0; i < numFamilies; i++) {
6473 families[i] = Bytes.toBytes("family" + i);
6474 }
6475 final byte[][] qualifiers = new byte[numQualifiers][];
6476 for (int i = 0; i < numQualifiers; i++) {
6477 qualifiers[i] = Bytes.toBytes("qual" + i);
6478 }
6479
6480 String method = "testWritesWhileRollWriter";
6481 CONF.setInt("hbase.regionserver.wal.disruptor.event.count", 2);
6482 this.region = initHRegion(tableName, method, CONF, families);
6483 try {
6484 List<Thread> threads = new ArrayList<Thread>();
6485 for (int i = 0; i < numRows; i++) {
6486 final int count = i;
6487 Thread t = new Thread(new Runnable() {
6488
6489 @Override
6490 public void run() {
6491 byte[] row = Bytes.toBytes("row" + count);
6492 Put put = new Put(row);
6493 put.setDurability(Durability.SYNC_WAL);
6494 byte[] value = Bytes.toBytes(String.valueOf(count));
6495 for (byte[] family : families) {
6496 for (byte[] qualifier : qualifiers) {
6497 put.addColumn(family, qualifier, (long) count, value);
6498 }
6499 }
6500 try {
6501 region.put(put);
6502 } catch (IOException e) {
6503 throw new RuntimeException(e);
6504 }
6505 }
6506 });
6507 threads.add(t);
6508 }
6509 for (Thread t : threads) {
6510 t.start();
6511 }
6512
6513 for (int i = 0; i < testCount; i++) {
6514 region.getWAL().rollWriter();
6515 Thread.yield();
6516 }
6517 } finally {
6518 try {
6519 HBaseTestingUtility.closeRegionAndWAL(this.region);
6520 CONF.setInt("hbase.regionserver.wal.disruptor.event.count", 16 * 1024);
6521 } catch (DroppedSnapshotException dse) {
6522
6523
6524
6525
6526 }
6527 this.region = null;
6528 }
6529 }
6530
6531 @Test
6532 public void testMutateRow_WriteRequestCount() throws Exception {
6533 byte[] row1 = Bytes.toBytes("row1");
6534 byte[] fam1 = Bytes.toBytes("fam1");
6535 byte[] qf1 = Bytes.toBytes("qualifier");
6536 byte[] val1 = Bytes.toBytes("value1");
6537
6538 RowMutations rm = new RowMutations(row1);
6539 Put put = new Put(row1);
6540 put.addColumn(fam1, qf1, val1);
6541 rm.add(put);
6542
6543 this.region = initHRegion(tableName, method, CONF, fam1);
6544 long wrcBeforeMutate = this.region.writeRequestsCount.get();
6545 this.region.mutateRow(rm);
6546 long wrcAfterMutate = this.region.writeRequestsCount.get();
6547 Assert.assertEquals(wrcBeforeMutate + rm.getMutations().size(), wrcAfterMutate);
6548 }
6549
6550 @Test
6551 public void testReverseScanShouldNotScanMemstoreIfReadPtLesser() throws Exception {
6552 byte[] cf1 = Bytes.toBytes("CF1");
6553 byte[][] families = { cf1 };
6554 byte[] col = Bytes.toBytes("C");
6555 String method = this.getName();
6556 HBaseConfiguration conf = new HBaseConfiguration();
6557 this.region = initHRegion(tableName, method, conf, families);
6558
6559 Put put = new Put(Bytes.toBytes("19996"));
6560 put.addColumn(cf1, col, Bytes.toBytes("val"));
6561 region.put(put);
6562 Put put2 = new Put(Bytes.toBytes("19995"));
6563 put2.addColumn(cf1, col, Bytes.toBytes("val"));
6564 region.put(put2);
6565
6566
6567 Scan scan = new Scan(Bytes.toBytes("19996"));
6568 scan.setReversed(true);
6569 RegionScanner scanner = region.getScanner(scan);
6570
6571
6572 region.flushcache(true, true);
6573
6574
6575
6576 for (int i = 10000; i < 20000; i++) {
6577 Put p = new Put(Bytes.toBytes("" + i));
6578 p.addColumn(cf1, col, Bytes.toBytes("" + i));
6579 region.put(p);
6580 }
6581
6582 List<Cell> currRow = new ArrayList<>();
6583 boolean hasNext;
6584 boolean assertDone = false;
6585 do {
6586 hasNext = scanner.next(currRow);
6587
6588
6589 if (!assertDone) {
6590 StoreScanner current =
6591 (StoreScanner) (((RegionScannerImpl) scanner).storeHeap).getCurrentForTesting();
6592 List<KeyValueScanner> scanners = current.getAllScannersForTesting();
6593 assertEquals("There should be only one scanner the store file scanner", 1,
6594 scanners.size());
6595 assertDone = true;
6596 }
6597 } while (hasNext);
6598
6599 assertEquals(2, currRow.size());
6600 assertEquals("19996", Bytes.toString(currRow.get(0).getRowArray(),
6601 currRow.get(0).getRowOffset(), currRow.get(0).getRowLength()));
6602 assertEquals("19995", Bytes.toString(currRow.get(1).getRowArray(),
6603 currRow.get(1).getRowOffset(), currRow.get(1).getRowLength()));
6604 }
6605
6606 @Test
6607 public void testReverseScanWhenPutCellsAfterOpenReverseScan() throws Exception {
6608 byte[] cf1 = Bytes.toBytes("CF1");
6609 byte[][] families = { cf1 };
6610 byte[] col = Bytes.toBytes("C");
6611
6612 HBaseConfiguration conf = new HBaseConfiguration();
6613 this.region = initHRegion(tableName, method, conf, families);
6614
6615 Put put = new Put(Bytes.toBytes("199996"));
6616 put.addColumn(cf1, col, Bytes.toBytes("val"));
6617 region.put(put);
6618 Put put2 = new Put(Bytes.toBytes("199995"));
6619 put2.addColumn(cf1, col, Bytes.toBytes("val"));
6620 region.put(put2);
6621
6622
6623 Scan scan = new Scan(Bytes.toBytes("199996"));
6624 scan.setReversed(true);
6625 RegionScanner scanner = region.getScanner(scan);
6626
6627
6628 for (int i = 100000; i < 200000; i++) {
6629 Put p = new Put(Bytes.toBytes("" + i));
6630 p.addColumn(cf1, col, Bytes.toBytes("" + i));
6631 region.put(p);
6632 }
6633 List<Cell> currRow = new ArrayList<>();
6634 boolean hasNext;
6635 do {
6636 hasNext = scanner.next(currRow);
6637 } while (hasNext);
6638
6639 assertEquals(2, currRow.size());
6640 assertEquals("199996", Bytes.toString(currRow.get(0).getRowArray(),
6641 currRow.get(0).getRowOffset(), currRow.get(0).getRowLength()));
6642 assertEquals("199995", Bytes.toString(currRow.get(1).getRowArray(),
6643 currRow.get(1).getRowOffset(), currRow.get(1).getRowLength()));
6644 }
6645
6646 @Test
6647 public void testCloseNoInterrupt() throws Exception {
6648 byte[] cf1 = Bytes.toBytes("CF1");
6649 byte[][] families = { cf1 };
6650 final int SLEEP_TIME = 10 * 1000;
6651
6652 Configuration conf = new Configuration(CONF);
6653
6654 conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, false);
6655 conf.setInt(HRegion.CLOSE_WAIT_INTERVAL, 1000);
6656 region = initHRegion(tableName, method, conf, families);
6657
6658 final CountDownLatch latch = new CountDownLatch(1);
6659 final AtomicBoolean holderInterrupted = new AtomicBoolean();
6660 Thread holder = new Thread(new Runnable() {
6661 @Override
6662 public void run() {
6663 try {
6664 LOG.info("Starting region operation holder");
6665 region.startRegionOperation(Operation.SCAN);
6666 latch.countDown();
6667 try {
6668 Thread.sleep(SLEEP_TIME);
6669 } catch (InterruptedException e) {
6670 LOG.info("Interrupted");
6671 holderInterrupted.set(true);
6672 }
6673 } catch (Exception e) {
6674 throw new RuntimeException(e);
6675 } finally {
6676 try {
6677 region.closeRegionOperation();
6678 } catch (IOException e) {
6679 }
6680 LOG.info("Stopped region operation holder");
6681 }
6682 }
6683 });
6684
6685 holder.start();
6686 latch.await();
6687 region.close();
6688 region = null;
6689 holder.join();
6690
6691 assertFalse("Region lock holder should not have been interrupted", holderInterrupted.get());
6692 }
6693
6694 @Test
6695 public void testCloseInterrupt() throws Exception {
6696 byte[] cf1 = Bytes.toBytes("CF1");
6697 byte[][] families = { cf1 };
6698 final int SLEEP_TIME = 10 * 1000;
6699
6700 Configuration conf = new Configuration(CONF);
6701
6702 conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true);
6703
6704 conf.setInt(HRegion.CLOSE_WAIT_INTERVAL, 1000);
6705 region = initHRegion(tableName, method, conf, families);
6706
6707 final CountDownLatch latch = new CountDownLatch(1);
6708 final AtomicBoolean holderInterrupted = new AtomicBoolean();
6709 Thread holder = new Thread(new Runnable() {
6710 @Override
6711 public void run() {
6712 try {
6713 LOG.info("Starting region operation holder");
6714 region.startRegionOperation(Operation.SCAN);
6715 latch.countDown();
6716 try {
6717 Thread.sleep(SLEEP_TIME);
6718 } catch (InterruptedException e) {
6719 LOG.info("Interrupted");
6720 holderInterrupted.set(true);
6721 }
6722 } catch (Exception e) {
6723 throw new RuntimeException(e);
6724 } finally {
6725 try {
6726 region.closeRegionOperation();
6727 } catch (IOException e) {
6728 }
6729 LOG.info("Stopped region operation holder");
6730 }
6731 }
6732 });
6733
6734 holder.start();
6735 latch.await();
6736 region.close();
6737 region = null;
6738 holder.join();
6739
6740 assertTrue("Region lock holder was not interrupted", holderInterrupted.get());
6741 }
6742
6743 @Test
6744 public void testCloseAbort() throws Exception {
6745 byte[] cf1 = Bytes.toBytes("CF1");
6746 byte[][] families = { cf1 };
6747 final int SLEEP_TIME = 10 * 1000;
6748
6749 Configuration conf = new Configuration(CONF);
6750
6751 conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true);
6752
6753 conf.setInt(HRegion.CLOSE_WAIT_TIME, SLEEP_TIME / 2);
6754
6755 conf.setInt(HRegion.CLOSE_WAIT_INTERVAL, SLEEP_TIME / 4);
6756 region = initHRegion(tableName, method, conf, families);
6757 RegionServerServices rsServices = mock(RegionServerServices.class);
6758 when(rsServices.getServerName()).thenReturn(ServerName.valueOf("localhost", 1000, 1000));
6759 region.rsServices = rsServices;
6760
6761 final CountDownLatch latch = new CountDownLatch(1);
6762 Thread holder = new Thread(new Runnable() {
6763 @Override
6764 public void run() {
6765 try {
6766 LOG.info("Starting region operation holder");
6767 region.startRegionOperation(Operation.SCAN);
6768 latch.countDown();
6769
6770 int timeRemaining = SLEEP_TIME;
6771 while (timeRemaining > 0) {
6772 long start = EnvironmentEdgeManager.currentTime();
6773 try {
6774 Thread.sleep(timeRemaining);
6775 } catch (InterruptedException e) {
6776 LOG.info("Interrupted");
6777 }
6778 long end = EnvironmentEdgeManager.currentTime();
6779 timeRemaining -= end - start;
6780 if (timeRemaining < 0) {
6781 timeRemaining = 0;
6782 }
6783 if (timeRemaining > 0) {
6784 LOG.info("Sleeping again, remaining time " + timeRemaining + " ms");
6785 }
6786 }
6787 } catch (Exception e) {
6788 throw new RuntimeException(e);
6789 } finally {
6790 try {
6791 region.closeRegionOperation();
6792 } catch (IOException e) {
6793 }
6794 LOG.info("Stopped region operation holder");
6795 }
6796 }
6797 });
6798
6799 holder.start();
6800 latch.await();
6801 try {
6802 region.close();
6803 } catch (IOException e) {
6804 LOG.info("Caught expected exception", e);
6805 }
6806 region = null;
6807 holder.join();
6808
6809
6810 verify(rsServices, atLeast(1)).abort(anyString(),any(Throwable.class));
6811 }
6812
6813 @Test
6814 public void testInterruptProtection() throws Exception {
6815 byte[] cf1 = Bytes.toBytes("CF1");
6816 byte[][] families = { cf1 };
6817 final int SLEEP_TIME = 10 * 1000;
6818
6819 Configuration conf = new Configuration(CONF);
6820
6821 conf.setBoolean(HRegion.CLOSE_WAIT_ABORT, true);
6822 conf.setInt(HRegion.CLOSE_WAIT_INTERVAL, 1000);
6823 region = initHRegion(tableName, method, conf, families);
6824
6825 final CountDownLatch latch = new CountDownLatch(1);
6826 final AtomicBoolean holderInterrupted = new AtomicBoolean();
6827 Thread holder = new Thread(new Runnable() {
6828 @Override
6829 public void run() {
6830 try {
6831 LOG.info("Starting region operation holder");
6832 region.startRegionOperation(Operation.SCAN);
6833 LOG.info("Protecting against interrupts");
6834 region.disableInterrupts();
6835 try {
6836 latch.countDown();
6837 try {
6838 Thread.sleep(SLEEP_TIME);
6839 } catch (InterruptedException e) {
6840 LOG.info("Interrupted");
6841 holderInterrupted.set(true);
6842 }
6843 } finally {
6844 region.enableInterrupts();
6845 }
6846 } catch (Exception e) {
6847 throw new RuntimeException(e);
6848 } finally {
6849 try {
6850 region.closeRegionOperation();
6851 } catch (IOException e) {
6852 }
6853 LOG.info("Stopped region operation holder");
6854 }
6855 }
6856 });
6857
6858 holder.start();
6859 latch.await();
6860 region.close();
6861 region = null;
6862 holder.join();
6863
6864 assertFalse("Region lock holder should not have been interrupted", holderInterrupted.get());
6865 }
6866
6867 }